diff --git a/README.md b/README.md index ddf9b15a613..da9c5483097 100644 --- a/README.md +++ b/README.md @@ -77,7 +77,7 @@ import com.ververica.cdc.connectors.mysql.source.MySqlSource; public class MySqlSourceExample { public static void main(String[] args) throws Exception { - MySqlSource mySqlSource = MySqlSource.builder() + MySqlSource mySqlChangeEventSource = MySqlSource.builder() .hostname("yourHostname") .port(yourPort) .databaseList("yourDatabaseName") // set captured database @@ -93,7 +93,7 @@ public class MySqlSourceExample { env.enableCheckpointing(3000); env - .fromSource(mySqlSource, WatermarkStrategy.noWatermarks(), "MySQL Source") + .fromSource(mySqlChangeEventSource, WatermarkStrategy.noWatermarks(), "MySQL Source") // set 4 parallel source tasks .setParallelism(4) .print().setParallelism(1); // use parallelism 1 for sink to keep message ordering diff --git a/docs/content/about.md b/docs/content/about.md index 3d0b6232e27..b1d6d5580e4 100644 --- a/docs/content/about.md +++ b/docs/content/about.md @@ -91,7 +91,7 @@ import com.ververica.cdc.connectors.mysql.source.MySqlSource; public class MySqlBinlogSourceExample { public static void main(String[] args) throws Exception { - MySqlSource mySqlSource = MySqlSource.builder() + MySqlSource mySqlChangeEventSource = MySqlSource.builder() .hostname("yourHostname") .port(yourPort) .databaseList("yourDatabaseName") // set captured database @@ -107,7 +107,7 @@ public class MySqlBinlogSourceExample { env.enableCheckpointing(3000); env - .fromSource(mySqlSource, WatermarkStrategy.noWatermarks(), "MySQL Source") + .fromSource(mySqlChangeEventSource, WatermarkStrategy.noWatermarks(), "MySQL Source") // set 4 parallel source tasks .setParallelism(4) .print().setParallelism(1); // use parallelism 1 for sink to keep message ordering diff --git a/docs/content/connectors/mysql-cdc.md b/docs/content/connectors/mysql-cdc.md index 21b084a24a8..86174cf2329 100644 --- a/docs/content/connectors/mysql-cdc.md +++ b/docs/content/connectors/mysql-cdc.md @@ -481,7 +481,7 @@ import com.ververica.cdc.connectors.mysql.source.MySqlSource; public class MySqlSourceExample { public static void main(String[] args) throws Exception { - MySqlSource mySqlSource = MySqlSource.builder() + MySqlSource mySqlChangeEventSource = MySqlSource.builder() .hostname("yourHostname") .port(yourPort) .databaseList("yourDatabaseName") // set captured database @@ -497,7 +497,7 @@ public class MySqlSourceExample { env.enableCheckpointing(3000); env - .fromSource(mySqlSource, WatermarkStrategy.noWatermarks(), "MySQL Source") + .fromSource(mySqlChangeEventSource, WatermarkStrategy.noWatermarks(), "MySQL Source") // set 4 parallel source tasks .setParallelism(4) .print().setParallelism(1); // use parallelism 1 for sink to keep message ordering diff --git a/flink-connector-base/pom.xml b/flink-cdc-base/pom.xml similarity index 87% rename from flink-connector-base/pom.xml rename to flink-cdc-base/pom.xml index a127ee73178..d1f5313a391 100644 --- a/flink-connector-base/pom.xml +++ b/flink-cdc-base/pom.xml @@ -25,10 +25,9 @@ under the License. 4.0.0 - flink-connector-base + flink-cdc-base - com.ververica flink-connector-debezium @@ -114,5 +113,25 @@ under the License. test-jar test + + + + + + io.debezium + debezium-connector-mysql + ${debezium.version} + test + + + + + + + org.testcontainers + mysql + ${testcontainers.version} + test + diff --git a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/config/BaseSourceConfig.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/config/BaseSourceConfig.java new file mode 100644 index 00000000000..0cc4578ef96 --- /dev/null +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/config/BaseSourceConfig.java @@ -0,0 +1,95 @@ +/* + * 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 com.ververica.cdc.connectors.base.config; + +import com.ververica.cdc.connectors.base.options.StartupOptions; +import com.ververica.cdc.connectors.base.source.JdbcIncrementalSource; +import io.debezium.config.Configuration; + +import java.util.Properties; + +/** A basic Source configuration which is used by {@link JdbcIncrementalSource}. */ +public abstract class BaseSourceConfig implements SourceConfig { + + private static final long serialVersionUID = 1L; + + protected final StartupOptions startupOptions; + protected final int splitSize; + protected final int splitMetaGroupSize; + protected final double distributionFactorUpper; + protected final double distributionFactorLower; + protected final boolean includeSchemaChanges; + + // -------------------------------------------------------------------------------------------- + // Debezium Configurations + // -------------------------------------------------------------------------------------------- + protected final Properties dbzProperties; + protected transient Configuration dbzConfiguration; + + public BaseSourceConfig( + StartupOptions startupOptions, + int splitSize, + int splitMetaGroupSize, + double distributionFactorUpper, + double distributionFactorLower, + boolean includeSchemaChanges, + Properties dbzProperties, + Configuration dbzConfiguration) { + this.startupOptions = startupOptions; + this.splitSize = splitSize; + this.splitMetaGroupSize = splitMetaGroupSize; + this.distributionFactorUpper = distributionFactorUpper; + this.distributionFactorLower = distributionFactorLower; + this.includeSchemaChanges = includeSchemaChanges; + this.dbzProperties = dbzProperties; + this.dbzConfiguration = dbzConfiguration; + } + + public StartupOptions getStartupOptions() { + return startupOptions; + } + + public int getSplitSize() { + return splitSize; + } + + public int getSplitMetaGroupSize() { + return splitMetaGroupSize; + } + + public double getDistributionFactorUpper() { + return distributionFactorUpper; + } + + public double getDistributionFactorLower() { + return distributionFactorLower; + } + + public boolean isIncludeSchemaChanges() { + return includeSchemaChanges; + } + + public Properties getDbzProperties() { + return dbzProperties; + } + + public Configuration getDbzConfiguration() { + return Configuration.from(dbzProperties); + } +} diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/config/SourceConfig.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/config/JdbcSourceConfig.java similarity index 61% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/config/SourceConfig.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/config/JdbcSourceConfig.java index 1ee915632cb..3c04c474036 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/config/SourceConfig.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/config/JdbcSourceConfig.java @@ -16,19 +16,22 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.base.source.config; +package com.ververica.cdc.connectors.base.config; -import com.ververica.cdc.connectors.base.source.ChangeEventHybridSource; +import com.ververica.cdc.connectors.base.options.StartupOptions; +import com.ververica.cdc.connectors.base.source.JdbcIncrementalSource; import io.debezium.config.Configuration; +import io.debezium.relational.RelationalDatabaseConnectorConfig; -import java.io.Serializable; import java.time.Duration; import java.util.List; import java.util.Properties; -/** A basic Source configuration which is used by {@link ChangeEventHybridSource}. */ -public class SourceConfig implements Serializable { - private static final long serialVersionUID = 1L; +/** + * A Source configuration which is used by {@link JdbcIncrementalSource} which used JDBC data + * source. + */ +public abstract class JdbcSourceConfig extends BaseSourceConfig { protected final String driverClassName; protected final String hostname; @@ -37,45 +40,42 @@ public class SourceConfig implements Serializable { protected final String password; protected final List databaseList; protected final List tableList; - - protected final StartupOptions startupOptions; - protected final int splitSize; - protected final int splitMetaGroupSize; protected final int fetchSize; protected final String serverTimeZone; protected final Duration connectTimeout; protected final int connectMaxRetries; protected final int connectionPoolSize; - protected final double distributionFactorUpper; - protected final double distributionFactorLower; - protected final boolean includeSchemaChanges; - // -------------------------------------------------------------------------------------------- - // Debezium Configurations - // -------------------------------------------------------------------------------------------- - protected final Properties dbzProperties; - protected final Configuration dbzConfiguration; - - public SourceConfig( + + public JdbcSourceConfig( + StartupOptions startupOptions, + List databaseList, + List tableList, + int splitSize, + int splitMetaGroupSize, + double distributionFactorUpper, + double distributionFactorLower, + boolean includeSchemaChanges, + Properties dbzProperties, + Configuration dbzConfiguration, String driverClassName, String hostname, int port, String username, String password, - List databaseList, - List tableList, - StartupOptions startupOptions, - int splitSize, - int splitMetaGroupSize, int fetchSize, String serverTimeZone, Duration connectTimeout, int connectMaxRetries, - int connectionPoolSize, - double distributionFactorUpper, - double distributionFactorLower, - boolean includeSchemaChanges, - Properties dbzProperties, - Configuration dbzConfiguration) { + int connectionPoolSize) { + super( + startupOptions, + splitSize, + splitMetaGroupSize, + distributionFactorUpper, + distributionFactorLower, + includeSchemaChanges, + dbzProperties, + dbzConfiguration); this.driverClassName = driverClassName; this.hostname = hostname; this.port = port; @@ -83,21 +83,15 @@ public SourceConfig( this.password = password; this.databaseList = databaseList; this.tableList = tableList; - this.startupOptions = startupOptions; - this.splitSize = splitSize; - this.splitMetaGroupSize = splitMetaGroupSize; this.fetchSize = fetchSize; this.serverTimeZone = serverTimeZone; this.connectTimeout = connectTimeout; this.connectMaxRetries = connectMaxRetries; this.connectionPoolSize = connectionPoolSize; - this.distributionFactorUpper = distributionFactorUpper; - this.distributionFactorLower = distributionFactorLower; - this.includeSchemaChanges = includeSchemaChanges; - this.dbzProperties = dbzProperties; - this.dbzConfiguration = dbzConfiguration; } + public abstract RelationalDatabaseConnectorConfig getDbzConnectorConfig(); + public String getDriverClassName() { return driverClassName; } @@ -126,18 +120,6 @@ public List getTableList() { return tableList; } - public StartupOptions getStartupOptions() { - return startupOptions; - } - - public int getSplitSize() { - return splitSize; - } - - public int getSplitMetaGroupSize() { - return splitMetaGroupSize; - } - public int getFetchSize() { return fetchSize; } @@ -157,24 +139,4 @@ public int getConnectMaxRetries() { public int getConnectionPoolSize() { return connectionPoolSize; } - - public double getDistributionFactorUpper() { - return distributionFactorUpper; - } - - public double getDistributionFactorLower() { - return distributionFactorLower; - } - - public boolean isIncludeSchemaChanges() { - return includeSchemaChanges; - } - - public Properties getDbzProperties() { - return dbzProperties; - } - - public Configuration getDbzConfiguration() { - return dbzConfiguration; - } } diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/config/SourceConfigFactory.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/config/JdbcSourceConfigFactory.java similarity index 61% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/config/SourceConfigFactory.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/config/JdbcSourceConfigFactory.java index 54d65327e6a..6e6117c06be 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/config/SourceConfigFactory.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/config/JdbcSourceConfigFactory.java @@ -16,29 +16,23 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.base.source.config; +package com.ververica.cdc.connectors.base.config; import org.apache.flink.annotation.Internal; -import java.io.Serializable; +import com.ververica.cdc.connectors.base.config.SourceConfig.Factory; +import com.ververica.cdc.connectors.base.options.JdbcSourceOptions; +import com.ververica.cdc.connectors.base.options.SourceOptions; +import com.ververica.cdc.connectors.base.options.StartupOptions; + import java.time.Duration; import java.util.Arrays; import java.util.List; import java.util.Properties; -import static com.ververica.cdc.connectors.base.source.config.SourceOptions.CHUNK_META_GROUP_SIZE; -import static com.ververica.cdc.connectors.base.source.config.SourceOptions.CONNECTION_POOL_SIZE; -import static com.ververica.cdc.connectors.base.source.config.SourceOptions.CONNECT_MAX_RETRIES; -import static com.ververica.cdc.connectors.base.source.config.SourceOptions.CONNECT_TIMEOUT; -import static com.ververica.cdc.connectors.base.source.config.SourceOptions.SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE; -import static com.ververica.cdc.connectors.base.source.config.SourceOptions.SCAN_SNAPSHOT_FETCH_SIZE; -import static com.ververica.cdc.connectors.base.source.config.SourceOptions.SERVER_TIME_ZONE; -import static com.ververica.cdc.connectors.base.source.config.SourceOptions.SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND; -import static com.ververica.cdc.connectors.base.source.config.SourceOptions.SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND; - -/** A factory to initialize {@link SourceConfig}. */ +/** A {@link Factory} to provide {@link SourceConfig} of JDBC data source. */ @Internal -public abstract class SourceConfigFactory implements Serializable { +public abstract class JdbcSourceConfigFactory implements Factory { private static final long serialVersionUID = 1L; @@ -49,28 +43,28 @@ public abstract class SourceConfigFactory implements Serializable { protected List databaseList; protected List tableList; protected StartupOptions startupOptions = StartupOptions.initial(); - protected int splitSize = SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE.defaultValue(); - protected int splitMetaGroupSize = CHUNK_META_GROUP_SIZE.defaultValue(); - protected int fetchSize = SCAN_SNAPSHOT_FETCH_SIZE.defaultValue(); - protected String serverTimeZone = SERVER_TIME_ZONE.defaultValue(); - protected Duration connectTimeout = CONNECT_TIMEOUT.defaultValue(); - protected int connectMaxRetries = CONNECT_MAX_RETRIES.defaultValue(); - protected int connectionPoolSize = CONNECTION_POOL_SIZE.defaultValue(); + protected boolean includeSchemaChanges = false; protected double distributionFactorUpper = - SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND.defaultValue(); + SourceOptions.SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND.defaultValue(); protected double distributionFactorLower = - SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND.defaultValue(); - protected boolean includeSchemaChanges = false; + SourceOptions.SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND.defaultValue(); + protected int splitSize = SourceOptions.SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE.defaultValue(); + protected int splitMetaGroupSize = SourceOptions.CHUNK_META_GROUP_SIZE.defaultValue(); + protected int fetchSize = SourceOptions.SCAN_SNAPSHOT_FETCH_SIZE.defaultValue(); + protected String serverTimeZone = JdbcSourceOptions.SERVER_TIME_ZONE.defaultValue(); + protected Duration connectTimeout = JdbcSourceOptions.CONNECT_TIMEOUT.defaultValue(); + protected int connectMaxRetries = JdbcSourceOptions.CONNECT_MAX_RETRIES.defaultValue(); + protected int connectionPoolSize = JdbcSourceOptions.CONNECTION_POOL_SIZE.defaultValue(); protected Properties dbzProperties; /** Integer port number of the database server. */ - public SourceConfigFactory hostname(String hostname) { + public JdbcSourceConfigFactory hostname(String hostname) { this.hostname = hostname; return this; } /** Integer port number of the database server. */ - public SourceConfigFactory port(int port) { + public JdbcSourceConfigFactory port(int port) { this.port = port; return this; } @@ -80,7 +74,7 @@ public SourceConfigFactory port(int port) { * database name not included in the whitelist will be excluded from monitoring. By default all * databases will be monitored. */ - public SourceConfigFactory databaseList(String... databaseList) { + public JdbcSourceConfigFactory databaseList(String... databaseList) { this.databaseList = Arrays.asList(databaseList); return this; } @@ -88,22 +82,22 @@ public SourceConfigFactory databaseList(String... databaseList) { /** * An optional 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 databaseName.tableName. By default the connector will monitor + * Each identifier is of t、he form databaseName.tableName. By default the connector will monitor * every non-system table in each monitored database. */ - public SourceConfigFactory tableList(String... tableList) { + public JdbcSourceConfigFactory tableList(String... tableList) { this.tableList = Arrays.asList(tableList); return this; } /** Name of the user to use when connecting to the database server. */ - public SourceConfigFactory username(String username) { + public JdbcSourceConfigFactory username(String username) { this.username = username; return this; } /** Password to use when connecting to the database server. */ - public SourceConfigFactory password(String password) { + public JdbcSourceConfigFactory password(String password) { this.password = password; return this; } @@ -113,7 +107,7 @@ public SourceConfigFactory password(String password) { * TIMESTAMP type in MYSQL converted to STRING. See more * https://debezium.io/documentation/reference/1.5/connectors/mysql.html#mysql-temporal-types */ - public SourceConfigFactory serverTimeZone(String timeZone) { + public JdbcSourceConfigFactory serverTimeZone(String timeZone) { this.serverTimeZone = timeZone; return this; } @@ -122,7 +116,7 @@ public SourceConfigFactory serverTimeZone(String timeZone) { * The split size (number of rows) of table snapshot, captured tables are split into multiple * splits when read the snapshot of table. */ - public SourceConfigFactory splitSize(int splitSize) { + public JdbcSourceConfigFactory splitSize(int splitSize) { this.splitSize = splitSize; return this; } @@ -131,7 +125,7 @@ public SourceConfigFactory splitSize(int splitSize) { * The group size of split meta, if the meta size exceeds the group size, the meta will be will * be divided into multiple groups. */ - public SourceConfigFactory splitMetaGroupSize(int splitMetaGroupSize) { + public JdbcSourceConfigFactory splitMetaGroupSize(int splitMetaGroupSize) { this.splitMetaGroupSize = splitMetaGroupSize; return this; } @@ -140,7 +134,7 @@ public SourceConfigFactory splitMetaGroupSize(int splitMetaGroupSize) { * The upper bound of split key evenly distribution factor, the factor is used to determine * whether the table is evenly distribution or not. */ - public SourceConfigFactory distributionFactorUpper(double distributionFactorUpper) { + public JdbcSourceConfigFactory distributionFactorUpper(double distributionFactorUpper) { this.distributionFactorUpper = distributionFactorUpper; return this; } @@ -149,13 +143,13 @@ public SourceConfigFactory distributionFactorUpper(double distributionFactorUppe * The lower bound of split key evenly distribution factor, the factor is used to determine * whether the table is evenly distribution or not. */ - public SourceConfigFactory distributionFactorLower(double distributionFactorLower) { + public JdbcSourceConfigFactory distributionFactorLower(double distributionFactorLower) { this.distributionFactorLower = distributionFactorLower; return this; } /** The maximum fetch size for per poll when read table snapshot. */ - public SourceConfigFactory fetchSize(int fetchSize) { + public JdbcSourceConfigFactory fetchSize(int fetchSize) { this.fetchSize = fetchSize; return this; } @@ -164,37 +158,37 @@ public SourceConfigFactory fetchSize(int fetchSize) { * The maximum time that the connector should wait after trying to connect to the MySQL database * server before timing out. */ - public SourceConfigFactory connectTimeout(Duration connectTimeout) { + public JdbcSourceConfigFactory connectTimeout(Duration connectTimeout) { this.connectTimeout = connectTimeout; return this; } /** The connection pool size. */ - public SourceConfigFactory connectionPoolSize(int connectionPoolSize) { + public JdbcSourceConfigFactory connectionPoolSize(int connectionPoolSize) { this.connectionPoolSize = connectionPoolSize; return this; } /** The max retry times to get connection. */ - public SourceConfigFactory connectMaxRetries(int connectMaxRetries) { + public JdbcSourceConfigFactory connectMaxRetries(int connectMaxRetries) { this.connectMaxRetries = connectMaxRetries; return this; } /** Whether the {@link SourceConfig} should output the schema changes or not. */ - public SourceConfigFactory includeSchemaChanges(boolean includeSchemaChanges) { + public JdbcSourceConfigFactory includeSchemaChanges(boolean includeSchemaChanges) { this.includeSchemaChanges = includeSchemaChanges; return this; } /** The Debezium MySQL connector properties. For example, "snapshot.mode". */ - public SourceConfigFactory debeziumProperties(Properties properties) { + public JdbcSourceConfigFactory debeziumProperties(Properties properties) { this.dbzProperties = properties; return this; } /** Specifies the startup options. */ - public SourceConfigFactory startupOptions(StartupOptions startupOptions) { + public JdbcSourceConfigFactory startupOptions(StartupOptions startupOptions) { switch (startupOptions.startupMode) { case INITIAL: case LATEST_OFFSET: @@ -207,6 +201,6 @@ public SourceConfigFactory startupOptions(StartupOptions startupOptions) { return this; } - /** Creates a new {@link SourceConfig} for the given subtask {@code subtaskId}. */ - public abstract SourceConfig createConfig(int subtaskId); + @Override + public abstract JdbcSourceConfig create(int subtask); } diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/schema/BaseSchema.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/config/SourceConfig.java similarity index 53% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/schema/BaseSchema.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/config/SourceConfig.java index 6c98e16e5e6..e3f6d486e3c 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/schema/BaseSchema.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/config/SourceConfig.java @@ -16,22 +16,34 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.base.schema; - -import io.debezium.jdbc.JdbcConnection; -import io.debezium.relational.TableId; -import io.debezium.relational.history.TableChanges.TableChange; - -/** Provides as a tool class to obtain table schema information. */ -public interface BaseSchema { - - /** - * Gets table schema for the given table path. e.g. request to MySQL server by running `SHOW - * CREATE TABLE` if cache missed. - * - * @param jdbc jdbc connection. - * @param tableId Unique identifier for a database table. - * @return An abstract representation of the structure to the tables of a relational database. - */ - TableChange getTableSchema(JdbcConnection jdbc, TableId tableId); +package com.ververica.cdc.connectors.base.config; + +import org.apache.flink.annotation.Experimental; + +import com.ververica.cdc.connectors.base.options.StartupOptions; + +import java.io.Serializable; + +/** The Source configuration which offers configuration. */ +@Experimental +public interface SourceConfig extends Serializable { + + StartupOptions getStartupOptions(); + + int getSplitSize(); + + int getSplitMetaGroupSize(); + + double getDistributionFactorUpper(); + + double getDistributionFactorLower(); + + boolean isIncludeSchemaChanges(); + + /** Factory for the {@code SourceConfig}. */ + @FunctionalInterface + interface Factory extends Serializable { + + SourceConfig create(int subtask); + } } diff --git a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/dialect/DataSourceDialect.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/dialect/DataSourceDialect.java new file mode 100644 index 00000000000..1897288cb4f --- /dev/null +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/dialect/DataSourceDialect.java @@ -0,0 +1,76 @@ +/* + * 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 com.ververica.cdc.connectors.base.dialect; + +import org.apache.flink.annotation.Experimental; +import org.apache.flink.annotation.Internal; + +import com.ververica.cdc.connectors.base.config.SourceConfig; +import com.ververica.cdc.connectors.base.source.assigner.splitter.ChunkSplitter; +import com.ververica.cdc.connectors.base.source.meta.offset.Offset; +import com.ververica.cdc.connectors.base.source.meta.split.SourceSplitBase; +import com.ververica.cdc.connectors.base.source.reader.external.FetchTask; +import io.debezium.schema.DataCollectionId; + +import java.io.Serializable; +import java.util.List; +import java.util.Map; + +/** + * The dialect of data source. + * + * @param The identifier type of data collection. + * @param The schema type of data collection. + * @param The source config of data source. + */ +@Experimental +public interface DataSourceDialect + extends Serializable { + + /** Get the name of dialect. */ + String getName(); + + /** Discovers the list of data collection to capture. */ + List discoverDataCollections(C sourceConfig); + + /** + * Discovers the captured data collections' schema by {@link SourceConfig}. + * + * @param sourceConfig a basic source configuration. + */ + Map discoverDataCollectionSchemas(C sourceConfig); + + /** + * Displays current offset from the database e.g. query Mysql binary logs by query + * SHOW MASTER STATUS. + */ + Offset displayCurrentOffset(C sourceConfig); + + /** Check if the CollectionId is case sensitive or not. */ + boolean isDataCollectionIdCaseSensitive(C sourceConfig); + + /** Returns the {@link ChunkSplitter} which used to split collection to splits. */ + ChunkSplitter createChunkSplitter(C sourceConfig); + + /** The fetch task used to fetch data of a snapshot split or stream split. */ + FetchTask createFetchTask(SourceSplitBase sourceSplitBase); + + /** The task context used fot fetch task to fetch data from external systems. */ + FetchTask.Context createFetchTaskContext(SourceSplitBase sourceSplitBase); +} diff --git a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/dialect/JdbcDataSourceDialect.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/dialect/JdbcDataSourceDialect.java new file mode 100644 index 00000000000..8298ab643de --- /dev/null +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/dialect/JdbcDataSourceDialect.java @@ -0,0 +1,80 @@ +/* + * 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 com.ververica.cdc.connectors.base.dialect; + +import org.apache.flink.annotation.Experimental; +import org.apache.flink.annotation.Internal; +import org.apache.flink.util.FlinkRuntimeException; + +import com.ververica.cdc.connectors.base.config.JdbcSourceConfig; +import com.ververica.cdc.connectors.base.config.SourceConfig; +import com.ververica.cdc.connectors.base.relational.connection.JdbcConnectionFactory; +import com.ververica.cdc.connectors.base.relational.connection.JdbcConnectionPoolFactory; +import com.ververica.cdc.connectors.base.source.meta.split.SourceSplitBase; +import com.ververica.cdc.connectors.base.source.reader.external.FetchTask; +import com.ververica.cdc.connectors.base.source.reader.external.JdbcSourceFetchTaskContext; +import io.debezium.jdbc.JdbcConnection; +import io.debezium.relational.TableId; +import io.debezium.relational.history.TableChanges.TableChange; + +import java.util.List; +import java.util.Map; + +/** The dialect of JDBC data source. */ +@Experimental +public interface JdbcDataSourceDialect + extends DataSourceDialect { + + /** Discovers the list of table to capture. */ + List discoverDataCollections(JdbcSourceConfig sourceConfig); + + /** Discovers the captured tables' schema by {@link SourceConfig}. */ + Map discoverDataCollectionSchemas(JdbcSourceConfig sourceConfig); + + /** + * Creates and opens a new {@link JdbcConnection} backing connection pool. + * + * @param sourceConfig a basic source configuration. + * @return a utility that simplifies using a JDBC connection. + */ + default JdbcConnection openJdbcConnection(JdbcSourceConfig sourceConfig) { + JdbcConnection jdbc = + new JdbcConnection( + sourceConfig.getDbzConfiguration(), + new JdbcConnectionFactory(sourceConfig, getPooledDataSourceFactory())); + try { + jdbc.connect(); + } catch (Exception e) { + throw new FlinkRuntimeException(e); + } + return jdbc; + } + + /** Get a connection pool factory to create connection pool. */ + JdbcConnectionPoolFactory getPooledDataSourceFactory(); + + /** Query and build the schema of table. */ + TableChange queryTableSchema(JdbcConnection jdbc, TableId tableId); + + @Override + FetchTask createFetchTask(SourceSplitBase sourceSplitBase); + + @Override + JdbcSourceFetchTaskContext createFetchTaskContext(SourceSplitBase sourceSplitBase); +} diff --git a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/options/JdbcSourceOptions.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/options/JdbcSourceOptions.java new file mode 100644 index 00000000000..50a1b8b5dcb --- /dev/null +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/options/JdbcSourceOptions.java @@ -0,0 +1,108 @@ +/* + * 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 com.ververica.cdc.connectors.base.options; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; + +import com.ververica.cdc.connectors.base.source.JdbcIncrementalSource; + +import java.time.Duration; + +/** Configurations for {@link JdbcIncrementalSource} of JDBC data source. */ +public class JdbcSourceOptions extends SourceOptions { + + public static final ConfigOption HOSTNAME = + ConfigOptions.key("hostname") + .stringType() + .noDefaultValue() + .withDescription("IP address or hostname of the MySQL database server."); + + public static final ConfigOption PORT = + ConfigOptions.key("port") + .intType() + .defaultValue(3306) + .withDescription("Integer port number of the MySQL database server."); + + public static final ConfigOption USERNAME = + ConfigOptions.key("username") + .stringType() + .noDefaultValue() + .withDescription( + "Name of the MySQL database to use when connecting to the MySQL database server."); + + public static final ConfigOption PASSWORD = + ConfigOptions.key("password") + .stringType() + .noDefaultValue() + .withDescription( + "Password to use when connecting to the MySQL database server."); + + public static final ConfigOption DATABASE_NAME = + ConfigOptions.key("database-name") + .stringType() + .noDefaultValue() + .withDescription("Database name of the MySQL server to monitor."); + + public static final ConfigOption TABLE_NAME = + ConfigOptions.key("table-name") + .stringType() + .noDefaultValue() + .withDescription("Table name of the MySQL database to monitor."); + + public static final ConfigOption SERVER_TIME_ZONE = + ConfigOptions.key("server-time-zone") + .stringType() + .defaultValue("UTC") + .withDescription("The session time zone in database server."); + + public static final ConfigOption SERVER_ID = + ConfigOptions.key("server-id") + .stringType() + .noDefaultValue() + .withDescription( + "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 recommended 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 static final ConfigOption CONNECT_TIMEOUT = + ConfigOptions.key("connect.timeout") + .durationType() + .defaultValue(Duration.ofSeconds(30)) + .withDescription( + "The maximum time that the connector should wait after trying to connect to the MySQL database server before timing out."); + + public static final ConfigOption CONNECTION_POOL_SIZE = + ConfigOptions.key("connection.pool.size") + .intType() + .defaultValue(20) + .withDescription("The connection pool size."); + + public static final ConfigOption CONNECT_MAX_RETRIES = + ConfigOptions.key("connect.max-retries") + .intType() + .defaultValue(3) + .withDescription( + "The max retry times that the connector should retry to build MySQL database server connection."); +} diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/config/SourceOptions.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/options/SourceOptions.java similarity index 58% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/config/SourceOptions.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/options/SourceOptions.java index 3e3ab52cb2f..18f3eec3c64 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/config/SourceOptions.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/options/SourceOptions.java @@ -16,77 +16,16 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.base.source.config; +package com.ververica.cdc.connectors.base.options; -import org.apache.flink.annotation.Experimental; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; -import com.ververica.cdc.connectors.base.source.ChangeEventHybridSource; +import com.ververica.cdc.connectors.base.source.JdbcIncrementalSource; -import java.time.Duration; - -/** Configurations for {@link ChangeEventHybridSource}. */ +/** Configurations for {@link JdbcIncrementalSource}. */ public class SourceOptions { - public static final ConfigOption HOSTNAME = - ConfigOptions.key("hostname") - .stringType() - .noDefaultValue() - .withDescription("IP address or hostname of the MySQL database server."); - - public static final ConfigOption PORT = - ConfigOptions.key("port") - .intType() - .defaultValue(3306) - .withDescription("Integer port number of the MySQL database server."); - - public static final ConfigOption USERNAME = - ConfigOptions.key("username") - .stringType() - .noDefaultValue() - .withDescription( - "Name of the MySQL database to use when connecting to the MySQL database server."); - - public static final ConfigOption PASSWORD = - ConfigOptions.key("password") - .stringType() - .noDefaultValue() - .withDescription( - "Password to use when connecting to the MySQL database server."); - - public static final ConfigOption DATABASE_NAME = - ConfigOptions.key("database-name") - .stringType() - .noDefaultValue() - .withDescription("Database name of the MySQL server to monitor."); - - public static final ConfigOption TABLE_NAME = - ConfigOptions.key("table-name") - .stringType() - .noDefaultValue() - .withDescription("Table name of the MySQL database to monitor."); - - public static final ConfigOption SERVER_TIME_ZONE = - ConfigOptions.key("server-time-zone") - .stringType() - .defaultValue("UTC") - .withDescription("The session time zone in database server."); - - public static final ConfigOption SERVER_ID = - ConfigOptions.key("server-id") - .stringType() - .noDefaultValue() - .withDescription( - "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 recommended 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 static final ConfigOption SCAN_INCREMENTAL_SNAPSHOT_ENABLED = ConfigOptions.key("scan.incremental.snapshot.enabled") .booleanType() @@ -114,26 +53,6 @@ public class SourceOptions { .withDescription( "The maximum fetch size for per poll when read table snapshot."); - public static final ConfigOption CONNECT_TIMEOUT = - ConfigOptions.key("connect.timeout") - .durationType() - .defaultValue(Duration.ofSeconds(30)) - .withDescription( - "The maximum time that the connector should wait after trying to connect to the MySQL database server before timing out."); - - public static final ConfigOption CONNECTION_POOL_SIZE = - ConfigOptions.key("connection.pool.size") - .intType() - .defaultValue(20) - .withDescription("The connection pool size."); - - public static final ConfigOption CONNECT_MAX_RETRIES = - ConfigOptions.key("connect.max-retries") - .intType() - .defaultValue(3) - .withDescription( - "The max retry times that the connector should retry to build MySQL database server connection."); - public static final ConfigOption SCAN_STARTUP_MODE = ConfigOptions.key("scan.startup.mode") .stringType() @@ -164,10 +83,6 @@ public class SourceOptions { .withDescription( "Optional timestamp used in case of \"timestamp\" startup mode"); - // ---------------------------------------------------------------------------- - // experimental options, won't add them to documentation - // ---------------------------------------------------------------------------- - @Experimental public static final ConfigOption CHUNK_META_GROUP_SIZE = ConfigOptions.key("chunk-meta.group.size") .intType() @@ -175,7 +90,6 @@ public class SourceOptions { .withDescription( "The group size of chunk meta, if the meta size exceeds the group size, the meta will be will be divided into multiple groups."); - @Experimental public static final ConfigOption SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND = ConfigOptions.key("split-key.even-distribution.factor.upper-bound") .doubleType() @@ -187,7 +101,6 @@ public class SourceOptions { + " and the query MySQL for splitting would happen when it is uneven." + " The distribution factor could be calculated by (MAX(id) - MIN(id) + 1) / rowCount."); - @Experimental public static final ConfigOption SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND = ConfigOptions.key("split-key.even-distribution.factor.lower-bound") .doubleType() diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/config/StartupMode.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/options/StartupMode.java similarity index 90% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/config/StartupMode.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/options/StartupMode.java index 39dfceed060..2c9b6a5ffe2 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/config/StartupMode.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/options/StartupMode.java @@ -16,10 +16,10 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.base.source.config; +package com.ververica.cdc.connectors.base.options; /** - * Startup modes for the MySQL CDC Consumer. + * Startup modes for the Flink CDC Connectors. * * @see StartupOptions */ diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/config/StartupOptions.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/options/StartupOptions.java similarity index 98% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/config/StartupOptions.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/options/StartupOptions.java index c25c475c693..cc844949e5e 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/config/StartupOptions.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/options/StartupOptions.java @@ -16,14 +16,14 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.base.source.config; +package com.ververica.cdc.connectors.base.options; import java.io.Serializable; import java.util.Objects; import static org.apache.flink.util.Preconditions.checkNotNull; -/** Debezium startup options. */ +/** Flink CDC Connector startup options. */ public final class StartupOptions implements Serializable { private static final long serialVersionUID = 1L; diff --git a/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/debezium/dispatcher/EventDispatcherImpl.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/relational/JdbcSourceEventDispatcher.java similarity index 68% rename from flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/debezium/dispatcher/EventDispatcherImpl.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/relational/JdbcSourceEventDispatcher.java index eec466ca1e3..d132fa074d0 100644 --- a/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/debezium/dispatcher/EventDispatcherImpl.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/relational/JdbcSourceEventDispatcher.java @@ -16,8 +16,10 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.refactor.refactor.debezium.dispatcher; +package com.ververica.cdc.connectors.base.relational; +import com.ververica.cdc.connectors.base.source.meta.offset.Offset; +import com.ververica.cdc.connectors.base.source.meta.split.SourceSplitBase; import io.debezium.config.CommonConnectorConfig; import io.debezium.connector.base.ChangeEventQueue; import io.debezium.document.DocumentWriter; @@ -26,9 +28,9 @@ import io.debezium.pipeline.source.spi.EventMetadataProvider; import io.debezium.pipeline.spi.ChangeEventCreator; import io.debezium.pipeline.spi.SchemaChangeEventEmitter; +import io.debezium.relational.TableId; import io.debezium.relational.history.HistoryRecord; import io.debezium.schema.DataCollectionFilters; -import io.debezium.schema.DataCollectionId; import io.debezium.schema.DatabaseSchema; import io.debezium.schema.HistorizedDatabaseSchema; import io.debezium.schema.SchemaChangeEvent; @@ -46,8 +48,6 @@ import java.util.HashMap; import java.util.Map; -import static com.ververica.cdc.connectors.refactor.refactor.source.dialect.task.context.MySqlSnapshotReaderContext.MySqlEventMetadataProvider.SERVER_ID_KEY; - /** * A subclass implementation of {@link EventDispatcher}. * @@ -57,30 +57,41 @@ * this is useful for downstream to deserialize the {@link HistoryRecord} back. * */ -public class EventDispatcherImpl extends EventDispatcher { - - private static final Logger LOG = LoggerFactory.getLogger(EventDispatcherImpl.class); +public class JdbcSourceEventDispatcher extends EventDispatcher { + private static final Logger LOG = LoggerFactory.getLogger(JdbcSourceEventDispatcher.class); + public static final String HISTORY_RECORD_FIELD = "historyRecord"; + public static final String SERVER_ID_KEY = "server_id"; public static final String BINLOG_FILENAME_OFFSET_KEY = "file"; public static final String BINLOG_POSITION_OFFSET_KEY = "pos"; - public static final String HISTORY_RECORD_FIELD = "historyRecord"; private static final DocumentWriter DOCUMENT_WRITER = DocumentWriter.defaultWriter(); + private static final SchemaNameAdjuster SCHEMA_NAME_ADJUSTER = SchemaNameAdjuster.create(); + public static final String WATERMARK_SIGNAL = "_split_watermark_signal_"; + public static final String SPLIT_ID_KEY = "split_id"; + public static final String WATERMARK_KIND = "watermark_kind"; + public static final String SIGNAL_EVENT_KEY_SCHEMA_NAME = + "io.debezium.connector.flink.cdc.embedded.watermark.key"; + public static final String SIGNAL_EVENT_VALUE_SCHEMA_NAME = + "io.debezium.connector.flink.cdc.embedded.watermark.value"; private final ChangeEventQueue queue; private final HistorizedDatabaseSchema historizedSchema; - private final DataCollectionFilters.DataCollectionFilter filter; + private final DataCollectionFilters.DataCollectionFilter filter; private final CommonConnectorConfig connectorConfig; - private final TopicSelector topicSelector; + private final TopicSelector topicSelector; private final Schema schemaChangeKeySchema; private final Schema schemaChangeValueSchema; + private final Schema signalEventKeySchema; + private final Schema signalEventValueSchema; + private final String topic; - public EventDispatcherImpl( + public JdbcSourceEventDispatcher( CommonConnectorConfig connectorConfig, - TopicSelector topicSelector, - DatabaseSchema schema, + TopicSelector topicSelector, + DatabaseSchema schema, ChangeEventQueue queue, - DataCollectionFilters.DataCollectionFilter filter, + DataCollectionFilters.DataCollectionFilter filter, ChangeEventCreator changeEventCreator, EventMetadataProvider metadataProvider, SchemaNameAdjuster schemaNameAdjuster) { @@ -95,12 +106,13 @@ public EventDispatcherImpl( schemaNameAdjuster); this.historizedSchema = schema instanceof HistorizedDatabaseSchema - ? (HistorizedDatabaseSchema) schema + ? (HistorizedDatabaseSchema) schema : null; this.filter = filter; this.queue = queue; this.connectorConfig = connectorConfig; this.topicSelector = topicSelector; + this.topic = topicSelector.getPrimaryTopic(); this.schemaChangeKeySchema = SchemaBuilder.struct() .name( @@ -122,6 +134,18 @@ public EventDispatcherImpl( connectorConfig.getSourceInfoStructMaker().schema()) .field(HISTORY_RECORD_FIELD, Schema.OPTIONAL_STRING_SCHEMA) .build(); + this.signalEventKeySchema = + SchemaBuilder.struct() + .name(SCHEMA_NAME_ADJUSTER.adjust(SIGNAL_EVENT_KEY_SCHEMA_NAME)) + .field(SPLIT_ID_KEY, Schema.STRING_SCHEMA) + .field(WATERMARK_SIGNAL, Schema.BOOLEAN_SCHEMA) + .build(); + this.signalEventValueSchema = + SchemaBuilder.struct() + .name(SCHEMA_NAME_ADJUSTER.adjust(SIGNAL_EVENT_VALUE_SCHEMA_NAME)) + .field(SPLIT_ID_KEY, Schema.STRING_SCHEMA) + .field(WATERMARK_KIND, Schema.STRING_SCHEMA) + .build(); } public ChangeEventQueue getQueue() { @@ -130,7 +154,7 @@ public ChangeEventQueue getQueue() { @Override public void dispatchSchemaChangeEvent( - T dataCollectionId, SchemaChangeEventEmitter schemaChangeEventEmitter) + TableId dataCollectionId, SchemaChangeEventEmitter schemaChangeEventEmitter) throws InterruptedException { if (dataCollectionId != null && !filter.isIncluded(dataCollectionId)) { if (historizedSchema == null || historizedSchema.storeOnlyMonitoredTables()) { @@ -143,13 +167,14 @@ public void dispatchSchemaChangeEvent( @Override public void dispatchSchemaChangeEvent( - Collection dataCollectionIds, SchemaChangeEventEmitter schemaChangeEventEmitter) + Collection dataCollectionIds, + SchemaChangeEventEmitter schemaChangeEventEmitter) throws InterruptedException { boolean anyNonfilteredEvent = false; if (dataCollectionIds == null || dataCollectionIds.isEmpty()) { anyNonfilteredEvent = true; } else { - for (T dataCollectionId : dataCollectionIds) { + for (TableId dataCollectionId : dataCollectionIds) { if (filter.isIncluded(dataCollectionId)) { anyNonfilteredEvent = true; break; @@ -227,4 +252,54 @@ public void schemaChangeEvent(SchemaChangeEvent event) throws InterruptedExcepti } } } + + public void dispatchWatermarkEvent( + Map sourcePartition, + SourceSplitBase sourceSplit, + Offset watermark, + WatermarkKind watermarkKind) + throws InterruptedException { + + SourceRecord sourceRecord = + new SourceRecord( + sourcePartition, + watermark.getOffset(), + topic, + signalEventKeySchema, + signalRecordKey(sourceSplit.splitId()), + signalEventValueSchema, + signalRecordValue(sourceSplit.splitId(), watermarkKind)); + queue.enqueue(new DataChangeEvent(sourceRecord)); + } + + private Struct signalRecordKey(String splitId) { + Struct result = new Struct(signalEventKeySchema); + result.put(SPLIT_ID_KEY, splitId); + result.put(WATERMARK_SIGNAL, true); + return result; + } + + private Struct signalRecordValue(String splitId, WatermarkKind watermarkKind) { + Struct result = new Struct(signalEventValueSchema); + result.put(SPLIT_ID_KEY, splitId); + result.put(WATERMARK_KIND, watermarkKind.toString()); + return result; + } + + /** The watermark kind. */ + public enum WatermarkKind { + LOW, + HIGH, + BINLOG_END; + + public WatermarkKind fromString(String kindString) { + if (LOW.name().equalsIgnoreCase(kindString)) { + return LOW; + } else if (HIGH.name().equalsIgnoreCase(kindString)) { + return HIGH; + } else { + return BINLOG_END; + } + } + } } diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/BaseRecordEmitter.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/relational/JdbcSourceRecordEmitter.java similarity index 82% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/BaseRecordEmitter.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/relational/JdbcSourceRecordEmitter.java index 28b9771456e..a6299611d54 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/BaseRecordEmitter.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/relational/JdbcSourceRecordEmitter.java @@ -16,16 +16,17 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.base.source.reader; +package com.ververica.cdc.connectors.base.relational; import org.apache.flink.api.connector.source.SourceOutput; import org.apache.flink.connector.base.source.reader.RecordEmitter; import org.apache.flink.util.Collector; +import com.ververica.cdc.connectors.base.source.meta.offset.Offset; +import com.ververica.cdc.connectors.base.source.meta.offset.OffsetFactory; +import com.ververica.cdc.connectors.base.source.meta.split.SourceSplitState; import com.ververica.cdc.connectors.base.source.metrics.SourceReaderMetrics; -import com.ververica.cdc.connectors.base.source.offset.Offset; -import com.ververica.cdc.connectors.base.source.offset.OffsetFactory; -import com.ververica.cdc.connectors.base.source.split.SourceSplitState; +import com.ververica.cdc.connectors.base.source.reader.JdbcIncrementalSourceReader; import com.ververica.cdc.debezium.DebeziumDeserializationSchema; import com.ververica.cdc.debezium.history.FlinkJsonTableChangeSerializer; import io.debezium.document.Array; @@ -38,24 +39,24 @@ import java.util.HashMap; import java.util.Map; -import static com.ververica.cdc.connectors.base.source.utils.RecordUtils.getFetchTimestamp; -import static com.ververica.cdc.connectors.base.source.utils.RecordUtils.getHistoryRecord; -import static com.ververica.cdc.connectors.base.source.utils.RecordUtils.getMessageTimestamp; -import static com.ververica.cdc.connectors.base.source.utils.RecordUtils.isDataChangeRecord; -import static com.ververica.cdc.connectors.base.source.utils.RecordUtils.isHighWatermarkEvent; -import static com.ververica.cdc.connectors.base.source.utils.RecordUtils.isSchemaChangeEvent; -import static com.ververica.cdc.connectors.base.source.utils.RecordUtils.isWatermarkEvent; +import static com.ververica.cdc.connectors.base.utils.SourceRecordUtils.getFetchTimestamp; +import static com.ververica.cdc.connectors.base.utils.SourceRecordUtils.getHistoryRecord; +import static com.ververica.cdc.connectors.base.utils.SourceRecordUtils.getMessageTimestamp; +import static com.ververica.cdc.connectors.base.utils.SourceRecordUtils.isDataChangeRecord; +import static com.ververica.cdc.connectors.base.utils.SourceRecordUtils.isHighWatermarkEvent; +import static com.ververica.cdc.connectors.base.utils.SourceRecordUtils.isSchemaChangeEvent; +import static com.ververica.cdc.connectors.base.utils.SourceRecordUtils.isWatermarkEvent; /** - * The {@link RecordEmitter} implementation for {@link ParallelSourceReader}. + * The {@link RecordEmitter} implementation for {@link JdbcIncrementalSourceReader}. * *

The {@link RecordEmitter} buffers the snapshot records of split and call the binlog reader to * emit records rather than emit the records directly. */ -public final class BaseRecordEmitter +public class JdbcSourceRecordEmitter implements RecordEmitter { - private static final Logger LOG = LoggerFactory.getLogger(BaseRecordEmitter.class); + private static final Logger LOG = LoggerFactory.getLogger(JdbcSourceRecordEmitter.class); private static final FlinkJsonTableChangeSerializer TABLE_CHANGE_SERIALIZER = new FlinkJsonTableChangeSerializer(); @@ -63,10 +64,9 @@ public final class BaseRecordEmitter private final SourceReaderMetrics sourceReaderMetrics; private final boolean includeSchemaChanges; private final OutputCollector outputCollector; - private OffsetFactory offsetFactory; - public BaseRecordEmitter( + public JdbcSourceRecordEmitter( DebeziumDeserializationSchema debeziumDeserializationSchema, SourceReaderMetrics sourceReaderMetrics, boolean includeSchemaChanges, diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/internal/connection/ConnectionPoolId.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/relational/connection/ConnectionPoolId.java similarity index 96% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/internal/connection/ConnectionPoolId.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/relational/connection/ConnectionPoolId.java index 9e2ebff8c7a..b6ea1db73b8 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/internal/connection/ConnectionPoolId.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/relational/connection/ConnectionPoolId.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.base.source.internal.connection; +package com.ververica.cdc.connectors.base.relational.connection; import java.io.Serializable; import java.util.Objects; diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/internal/connection/ConnectionPools.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/relational/connection/ConnectionPools.java similarity index 71% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/internal/connection/ConnectionPools.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/relational/connection/ConnectionPools.java index 91019badb20..eb60f25cb53 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/internal/connection/ConnectionPools.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/relational/connection/ConnectionPools.java @@ -16,20 +16,20 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.base.source.internal.connection; +package com.ververica.cdc.connectors.base.relational.connection; +import org.apache.flink.annotation.Experimental; import org.apache.flink.annotation.Internal; -import com.ververica.cdc.connectors.base.source.config.SourceConfig; -import com.zaxxer.hikari.HikariDataSource; +import com.ververica.cdc.connectors.base.config.SourceConfig; -/** A JDBC connection pools that consists of {@link HikariDataSource}. */ -@Internal -public interface ConnectionPools { +/** A pool collection that consists of multiple connection pools. */ +@Experimental +public interface ConnectionPools { /** * Gets a connection pool from pools, create a new pool if the pool does not exists in the * connection pools . */ - HikariDataSource getOrCreateConnectionPool(ConnectionPoolId poolId, SourceConfig sourceConfig); + P getOrCreateConnectionPool(ConnectionPoolId poolId, C sourceConfig); } diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/internal/connection/JdbcConnectionFactory.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/relational/connection/JdbcConnectionFactory.java similarity index 84% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/internal/connection/JdbcConnectionFactory.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/relational/connection/JdbcConnectionFactory.java index 877a80f91ac..8519099accd 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/internal/connection/JdbcConnectionFactory.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/relational/connection/JdbcConnectionFactory.java @@ -16,11 +16,11 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.base.source.internal.connection; +package com.ververica.cdc.connectors.base.relational.connection; import org.apache.flink.util.FlinkRuntimeException; -import com.ververica.cdc.connectors.base.source.config.SourceConfig; +import com.ververica.cdc.connectors.base.config.JdbcSourceConfig; import com.zaxxer.hikari.HikariDataSource; import io.debezium.jdbc.JdbcConfiguration; import io.debezium.jdbc.JdbcConnection; @@ -35,13 +35,13 @@ public class JdbcConnectionFactory implements JdbcConnection.ConnectionFactory { private static final Logger LOG = LoggerFactory.getLogger(JdbcConnectionFactory.class); - private final SourceConfig sourceConfig; - private final PooledDataSourceFactory pooledDataSourceFactory; + private final JdbcSourceConfig sourceConfig; + private final JdbcConnectionPoolFactory jdbcConnectionPoolFactory; public JdbcConnectionFactory( - SourceConfig sourceConfig, PooledDataSourceFactory pooledDataSourceFactory) { + JdbcSourceConfig sourceConfig, JdbcConnectionPoolFactory jdbcConnectionPoolFactory) { this.sourceConfig = sourceConfig; - this.pooledDataSourceFactory = pooledDataSourceFactory; + this.jdbcConnectionPoolFactory = jdbcConnectionPoolFactory; } @Override @@ -52,7 +52,7 @@ public Connection connect(JdbcConfiguration config) throws SQLException { new ConnectionPoolId(sourceConfig.getHostname(), sourceConfig.getPort()); HikariDataSource dataSource = - JdbcConnectionPools.getInstance(pooledDataSourceFactory) + JdbcConnectionPools.getInstance(jdbcConnectionPoolFactory) .getOrCreateConnectionPool(connectionPoolId, sourceConfig); int i = 0; diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/internal/connection/PooledDataSourceFactory.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/relational/connection/JdbcConnectionPoolFactory.java similarity index 89% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/internal/connection/PooledDataSourceFactory.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/relational/connection/JdbcConnectionPoolFactory.java index a168d55d4eb..33d381b19c1 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/internal/connection/PooledDataSourceFactory.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/relational/connection/JdbcConnectionPoolFactory.java @@ -16,20 +16,20 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.base.source.internal.connection; +package com.ververica.cdc.connectors.base.relational.connection; -import com.ververica.cdc.connectors.base.source.config.SourceConfig; +import com.ververica.cdc.connectors.base.config.JdbcSourceConfig; import com.zaxxer.hikari.HikariConfig; import com.zaxxer.hikari.HikariDataSource; /** A connection pool factory to create pooled DataSource {@link HikariDataSource}. */ -public abstract class PooledDataSourceFactory { +public abstract class JdbcConnectionPoolFactory { public static final String CONNECTION_POOL_PREFIX = "connection-pool-"; public static final String SERVER_TIMEZONE_KEY = "serverTimezone"; public static final int MINIMUM_POOL_SIZE = 1; - public HikariDataSource createPooledDataSource(SourceConfig sourceConfig) { + public HikariDataSource createPooledDataSource(JdbcSourceConfig sourceConfig) { final HikariConfig config = new HikariConfig(); String hostName = sourceConfig.getHostname(); @@ -64,5 +64,5 @@ public HikariDataSource createPooledDataSource(SourceConfig sourceConfig) { * @param sourceConfig a basic Source configuration. * @return a database url. */ - public abstract String getJdbcUrl(SourceConfig sourceConfig); + public abstract String getJdbcUrl(JdbcSourceConfig sourceConfig); } diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/internal/connection/JdbcConnectionPools.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/relational/connection/JdbcConnectionPools.java similarity index 74% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/internal/connection/JdbcConnectionPools.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/relational/connection/JdbcConnectionPools.java index 6f437e2c0dd..512cfe08996 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/internal/connection/JdbcConnectionPools.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/relational/connection/JdbcConnectionPools.java @@ -16,9 +16,9 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.base.source.internal.connection; +package com.ververica.cdc.connectors.base.relational.connection; -import com.ververica.cdc.connectors.base.source.config.SourceConfig; +import com.ververica.cdc.connectors.base.config.JdbcSourceConfig; import com.zaxxer.hikari.HikariDataSource; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -27,22 +27,22 @@ import java.util.Map; /** A Jdbc Connection pools implementation. */ -public class JdbcConnectionPools implements ConnectionPools { +public class JdbcConnectionPools implements ConnectionPools { private static final Logger LOG = LoggerFactory.getLogger(JdbcConnectionPools.class); private static JdbcConnectionPools instance; private final Map pools = new HashMap<>(); - private static PooledDataSourceFactory pooledDataSourceFactory; + private static JdbcConnectionPoolFactory jdbcConnectionPoolFactory; private JdbcConnectionPools() {} public static synchronized JdbcConnectionPools getInstance( - PooledDataSourceFactory pooledDataSourceFactory) { + JdbcConnectionPoolFactory jdbcConnectionPoolFactory) { if (instance != null) { return instance; } else { - JdbcConnectionPools.pooledDataSourceFactory = pooledDataSourceFactory; + JdbcConnectionPools.jdbcConnectionPoolFactory = jdbcConnectionPoolFactory; instance = new JdbcConnectionPools(); return instance; } @@ -50,11 +50,11 @@ public static synchronized JdbcConnectionPools getInstance( @Override public HikariDataSource getOrCreateConnectionPool( - ConnectionPoolId poolId, SourceConfig sourceConfig) { + ConnectionPoolId poolId, JdbcSourceConfig sourceConfig) { synchronized (pools) { if (!pools.containsKey(poolId)) { LOG.info("Create and register connection pool {}", poolId); - pools.put(poolId, pooledDataSourceFactory.createPooledDataSource(sourceConfig)); + pools.put(poolId, jdbcConnectionPoolFactory.createPooledDataSource(sourceConfig)); } return pools.get(poolId); } diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/ChangeEventHybridSource.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/JdbcIncrementalSource.java similarity index 59% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/ChangeEventHybridSource.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/JdbcIncrementalSource.java index cd92fc78b05..35da17ed5a1 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/ChangeEventHybridSource.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/JdbcIncrementalSource.java @@ -18,6 +18,7 @@ package com.ververica.cdc.connectors.base.source; +import org.apache.flink.annotation.Experimental; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.connector.source.Boundedness; import org.apache.flink.api.connector.source.Source; @@ -31,29 +32,27 @@ import org.apache.flink.core.io.SimpleVersionedSerializer; import org.apache.flink.util.FlinkRuntimeException; -import com.ververica.cdc.connectors.base.schema.BaseSchema; -import com.ververica.cdc.connectors.base.source.assigners.HybridSplitAssigner; -import com.ververica.cdc.connectors.base.source.assigners.SplitAssigner; -import com.ververica.cdc.connectors.base.source.assigners.StreamSplitAssigner; -import com.ververica.cdc.connectors.base.source.assigners.state.HybridPendingSplitsState; -import com.ververica.cdc.connectors.base.source.assigners.state.PendingSplitsState; -import com.ververica.cdc.connectors.base.source.assigners.state.PendingSplitsStateSerializer; -import com.ververica.cdc.connectors.base.source.assigners.state.StreamPendingSplitsState; -import com.ververica.cdc.connectors.base.source.config.SourceConfig; -import com.ververica.cdc.connectors.base.source.config.SourceConfigFactory; -import com.ververica.cdc.connectors.base.source.config.StartupMode; -import com.ververica.cdc.connectors.base.source.dialect.SnapshotEventDialect; -import com.ververica.cdc.connectors.base.source.dialect.StreamingEventDialect; -import com.ververica.cdc.connectors.base.source.enumerator.SourceEnumerator; +import com.ververica.cdc.connectors.base.config.JdbcSourceConfig; +import com.ververica.cdc.connectors.base.config.JdbcSourceConfigFactory; +import com.ververica.cdc.connectors.base.config.SourceConfig; +import com.ververica.cdc.connectors.base.dialect.JdbcDataSourceDialect; +import com.ververica.cdc.connectors.base.options.StartupMode; +import com.ververica.cdc.connectors.base.relational.JdbcSourceRecordEmitter; +import com.ververica.cdc.connectors.base.source.assigner.HybridSplitAssigner; +import com.ververica.cdc.connectors.base.source.assigner.SplitAssigner; +import com.ververica.cdc.connectors.base.source.assigner.StreamSplitAssigner; +import com.ververica.cdc.connectors.base.source.assigner.state.HybridPendingSplitsState; +import com.ververica.cdc.connectors.base.source.assigner.state.PendingSplitsState; +import com.ververica.cdc.connectors.base.source.assigner.state.PendingSplitsStateSerializer; +import com.ververica.cdc.connectors.base.source.assigner.state.StreamPendingSplitsState; +import com.ververica.cdc.connectors.base.source.enumerator.IncrementalSourceEnumerator; +import com.ververica.cdc.connectors.base.source.meta.offset.OffsetFactory; +import com.ververica.cdc.connectors.base.source.meta.split.SourceSplitBase; +import com.ververica.cdc.connectors.base.source.meta.split.SourceSplitSerializer; import com.ververica.cdc.connectors.base.source.metrics.SourceReaderMetrics; -import com.ververica.cdc.connectors.base.source.offset.OffsetFactory; -import com.ververica.cdc.connectors.base.source.reader.BaseRecordEmitter; -import com.ververica.cdc.connectors.base.source.reader.BaseSplitReader; -import com.ververica.cdc.connectors.base.source.reader.ParallelSourceReader; -import com.ververica.cdc.connectors.base.source.split.SourceSplitBase; -import com.ververica.cdc.connectors.base.source.split.SourceSplitSerializer; +import com.ververica.cdc.connectors.base.source.reader.JdbcIncrementalSourceReader; +import com.ververica.cdc.connectors.base.source.reader.JdbcSourceSplitReader; import com.ververica.cdc.debezium.DebeziumDeserializationSchema; -import com.ververica.cdc.debezium.Validator; import io.debezium.relational.TableId; import org.apache.kafka.connect.source.SourceRecord; @@ -64,50 +63,27 @@ * The common CDC Source based on FLIP-27 and Watermark Signal Algorithm which supports parallel * reading snapshot of table and then continue to capture data change by streaming reading. */ -public class ChangeEventHybridSource +@Experimental +public class JdbcIncrementalSource implements Source, ResultTypeQueryable { private static final long serialVersionUID = 1L; - private final SourceConfigFactory configFactory; - private final DebeziumDeserializationSchema deserializationSchema; + private final JdbcSourceConfigFactory configFactory; + private final JdbcDataSourceDialect dataSourceDialect; private final OffsetFactory offsetFactory; + private final DebeziumDeserializationSchema deserializationSchema; private final SourceSplitSerializer sourceSplitSerializer; - private final Validator validator; - private final SnapshotEventDialect snapshotEventDialect; - private final StreamingEventDialect streamingEventDialect; - private final BaseSchema baseSchema; - - public ChangeEventHybridSource( - SourceConfigFactory configFactory, - DebeziumDeserializationSchema deserializationSchema, - OffsetFactory offsetFactory, - SnapshotEventDialect snapshotEventDialect, - StreamingEventDialect streamingEventDialect, - BaseSchema baseSchema) { - this( - configFactory, - deserializationSchema, - offsetFactory, - snapshotEventDialect, - streamingEventDialect, - Validator.getDefaultValidator(), - baseSchema); - } - public ChangeEventHybridSource( - SourceConfigFactory configFactory, + public JdbcIncrementalSource( + JdbcSourceConfigFactory configFactory, DebeziumDeserializationSchema deserializationSchema, OffsetFactory offsetFactory, - SnapshotEventDialect snapshotEventDialect, - StreamingEventDialect streamingEventDialect, - Validator validator, - BaseSchema baseSchema) { + JdbcDataSourceDialect dataSourceDialect) { this.configFactory = configFactory; this.deserializationSchema = deserializationSchema; this.offsetFactory = offsetFactory; - this.snapshotEventDialect = snapshotEventDialect; - this.streamingEventDialect = streamingEventDialect; + this.dataSourceDialect = dataSourceDialect; this.sourceSplitSerializer = new SourceSplitSerializer() { @Override @@ -115,8 +91,6 @@ public OffsetFactory getOffsetFactory() { return offsetFactory; } }; - this.validator = validator; - this.baseSchema = baseSchema; } @Override @@ -125,25 +99,22 @@ public Boundedness getBoundedness() { } @Override - public SourceReader createReader(SourceReaderContext readerContext) { + public SourceReader createReader(SourceReaderContext readerContext) { // create source config for the given subtask (e.g. unique server id) - SourceConfig sourceConfig = configFactory.createConfig(readerContext.getIndexOfSubtask()); + JdbcSourceConfig sourceConfig = configFactory.create(readerContext.getIndexOfSubtask()); FutureCompletingBlockingQueue> elementsQueue = new FutureCompletingBlockingQueue<>(); final SourceReaderMetrics sourceReaderMetrics = new SourceReaderMetrics(readerContext.metricGroup()); sourceReaderMetrics.registerMetrics(); - Supplier splitReaderSupplier = + Supplier splitReaderSupplier = () -> - new BaseSplitReader( - sourceConfig, - readerContext.getIndexOfSubtask(), - snapshotEventDialect, - streamingEventDialect); - return new ParallelSourceReader<>( + new JdbcSourceSplitReader( + readerContext.getIndexOfSubtask(), dataSourceDialect); + return new JdbcIncrementalSourceReader<>( elementsQueue, splitReaderSupplier, - new BaseRecordEmitter<>( + new JdbcSourceRecordEmitter<>( deserializationSchema, sourceReaderMetrics, sourceConfig.isIncludeSchemaChanges(), @@ -152,48 +123,43 @@ public SourceReader createReader(SourceReaderContext readerC readerContext, sourceConfig, sourceSplitSerializer, - snapshotEventDialect); + dataSourceDialect); } @Override public SplitEnumerator createEnumerator( SplitEnumeratorContext enumContext) { - SourceConfig sourceConfig = configFactory.createConfig(0); - - // validator.validate(); - + JdbcSourceConfig sourceConfig = configFactory.create(0); final SplitAssigner splitAssigner; if (sourceConfig.getStartupOptions().startupMode == StartupMode.INITIAL) { try { final List remainingTables = - snapshotEventDialect.discoverCapturedTables(sourceConfig); + dataSourceDialect.discoverDataCollections(sourceConfig); boolean isTableIdCaseSensitive = - snapshotEventDialect.isTableIdCaseSensitive(sourceConfig); + dataSourceDialect.isDataCollectionIdCaseSensitive(sourceConfig); splitAssigner = new HybridSplitAssigner( sourceConfig, enumContext.currentParallelism(), remainingTables, isTableIdCaseSensitive, - snapshotEventDialect, - offsetFactory, - baseSchema); + dataSourceDialect, + offsetFactory); } catch (Exception e) { throw new FlinkRuntimeException( "Failed to discover captured tables for enumerator", e); } } else { - splitAssigner = - new StreamSplitAssigner(sourceConfig, streamingEventDialect, offsetFactory); + splitAssigner = new StreamSplitAssigner(sourceConfig, dataSourceDialect, offsetFactory); } - return new SourceEnumerator(enumContext, sourceConfig, splitAssigner); + return new IncrementalSourceEnumerator(enumContext, sourceConfig, splitAssigner); } @Override public SplitEnumerator restoreEnumerator( SplitEnumeratorContext enumContext, PendingSplitsState checkpoint) { - SourceConfig sourceConfig = configFactory.createConfig(0); + SourceConfig sourceConfig = configFactory.create(0); final SplitAssigner splitAssigner; if (checkpoint instanceof HybridPendingSplitsState) { @@ -202,21 +168,20 @@ public SplitEnumerator restoreEnumerator( sourceConfig, enumContext.currentParallelism(), (HybridPendingSplitsState) checkpoint, - snapshotEventDialect, - offsetFactory, - baseSchema); + dataSourceDialect, + offsetFactory); } else if (checkpoint instanceof StreamPendingSplitsState) { splitAssigner = new StreamSplitAssigner( sourceConfig, (StreamPendingSplitsState) checkpoint, - streamingEventDialect, + dataSourceDialect, offsetFactory); } else { throw new UnsupportedOperationException( "Unsupported restored PendingSplitsState: " + checkpoint); } - return new SourceEnumerator(enumContext, sourceConfig, splitAssigner); + return new IncrementalSourceEnumerator(enumContext, sourceConfig, splitAssigner); } @Override diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/assigners/HybridSplitAssigner.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/HybridSplitAssigner.java similarity index 85% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/assigners/HybridSplitAssigner.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/HybridSplitAssigner.java index 9b432d539f0..a56457b88df 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/assigners/HybridSplitAssigner.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/HybridSplitAssigner.java @@ -16,19 +16,18 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.base.source.assigners; - -import com.ververica.cdc.connectors.base.schema.BaseSchema; -import com.ververica.cdc.connectors.base.source.assigners.state.HybridPendingSplitsState; -import com.ververica.cdc.connectors.base.source.assigners.state.PendingSplitsState; -import com.ververica.cdc.connectors.base.source.config.SourceConfig; -import com.ververica.cdc.connectors.base.source.dialect.SnapshotEventDialect; -import com.ververica.cdc.connectors.base.source.offset.Offset; -import com.ververica.cdc.connectors.base.source.offset.OffsetFactory; -import com.ververica.cdc.connectors.base.source.split.FinishedSnapshotSplitInfo; -import com.ververica.cdc.connectors.base.source.split.SnapshotSplit; -import com.ververica.cdc.connectors.base.source.split.SourceSplitBase; -import com.ververica.cdc.connectors.base.source.split.StreamSplit; +package com.ververica.cdc.connectors.base.source.assigner; + +import com.ververica.cdc.connectors.base.config.SourceConfig; +import com.ververica.cdc.connectors.base.dialect.DataSourceDialect; +import com.ververica.cdc.connectors.base.source.assigner.state.HybridPendingSplitsState; +import com.ververica.cdc.connectors.base.source.assigner.state.PendingSplitsState; +import com.ververica.cdc.connectors.base.source.meta.offset.Offset; +import com.ververica.cdc.connectors.base.source.meta.offset.OffsetFactory; +import com.ververica.cdc.connectors.base.source.meta.split.FinishedSnapshotSplitInfo; +import com.ververica.cdc.connectors.base.source.meta.split.SnapshotSplit; +import com.ververica.cdc.connectors.base.source.meta.split.SourceSplitBase; +import com.ververica.cdc.connectors.base.source.meta.split.StreamSplit; import io.debezium.relational.TableId; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -61,9 +60,8 @@ public HybridSplitAssigner( int currentParallelism, List remainingTables, boolean isTableIdCaseSensitive, - SnapshotEventDialect dialect, - OffsetFactory offsetFactory, - BaseSchema baseSchema) { + DataSourceDialect dialect, + OffsetFactory offsetFactory) { this( new SnapshotSplitAssigner( sourceConfig, @@ -71,8 +69,7 @@ public HybridSplitAssigner( remainingTables, isTableIdCaseSensitive, dialect, - offsetFactory, - baseSchema), + offsetFactory), false, sourceConfig.getSplitMetaGroupSize()); this.offsetFactory = offsetFactory; @@ -82,18 +79,16 @@ public HybridSplitAssigner( SourceConfig sourceConfig, int currentParallelism, HybridPendingSplitsState checkpoint, - SnapshotEventDialect dialect, - OffsetFactory offsetFactory, - BaseSchema baseSchema) { + DataSourceDialect dialect, + OffsetFactory offsetFactory) { this( new SnapshotSplitAssigner( sourceConfig, currentParallelism, checkpoint.getSnapshotPendingSplits(), dialect, - offsetFactory, - baseSchema), - checkpoint.isBinlogSplitAssigned(), + offsetFactory), + checkpoint.isStreamSplitAssigned(), sourceConfig.getSplitMetaGroupSize()); } diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/assigners/SnapshotSplitAssigner.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/SnapshotSplitAssigner.java similarity index 87% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/assigners/SnapshotSplitAssigner.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/SnapshotSplitAssigner.java index b87e398637d..8be7bb97cf5 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/assigners/SnapshotSplitAssigner.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/SnapshotSplitAssigner.java @@ -16,19 +16,19 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.base.source.assigners; +package com.ververica.cdc.connectors.base.source.assigner; import org.apache.flink.util.FlinkRuntimeException; -import com.ververica.cdc.connectors.base.schema.BaseSchema; -import com.ververica.cdc.connectors.base.source.assigners.state.SnapshotPendingSplitsState; -import com.ververica.cdc.connectors.base.source.config.SourceConfig; -import com.ververica.cdc.connectors.base.source.dialect.SnapshotEventDialect; -import com.ververica.cdc.connectors.base.source.offset.Offset; -import com.ververica.cdc.connectors.base.source.offset.OffsetFactory; -import com.ververica.cdc.connectors.base.source.split.FinishedSnapshotSplitInfo; -import com.ververica.cdc.connectors.base.source.split.SnapshotSplit; -import com.ververica.cdc.connectors.base.source.split.SourceSplitBase; +import com.ververica.cdc.connectors.base.config.SourceConfig; +import com.ververica.cdc.connectors.base.dialect.DataSourceDialect; +import com.ververica.cdc.connectors.base.source.assigner.splitter.ChunkSplitter; +import com.ververica.cdc.connectors.base.source.assigner.state.SnapshotPendingSplitsState; +import com.ververica.cdc.connectors.base.source.meta.offset.Offset; +import com.ververica.cdc.connectors.base.source.meta.offset.OffsetFactory; +import com.ververica.cdc.connectors.base.source.meta.split.FinishedSnapshotSplitInfo; +import com.ververica.cdc.connectors.base.source.meta.split.SnapshotSplit; +import com.ververica.cdc.connectors.base.source.meta.split.SourceSplitBase; import io.debezium.relational.TableId; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -65,18 +65,16 @@ public class SnapshotSplitAssigner implements SplitAssigner { private boolean isTableIdCaseSensitive; @Nullable private Long checkpointIdToFinish; - private final SnapshotEventDialect dialect; + private final DataSourceDialect dialect; private OffsetFactory offsetFactory; - private final BaseSchema baseSchema; public SnapshotSplitAssigner( SourceConfig sourceConfig, int currentParallelism, List remainingTables, boolean isTableIdCaseSensitive, - SnapshotEventDialect dialect, - OffsetFactory offsetFactory, - BaseSchema baseSchema) { + DataSourceDialect dialect, + OffsetFactory offsetFactory) { this( sourceConfig, currentParallelism, @@ -89,17 +87,15 @@ public SnapshotSplitAssigner( isTableIdCaseSensitive, true, dialect, - offsetFactory, - baseSchema); + offsetFactory); } public SnapshotSplitAssigner( SourceConfig sourceConfig, int currentParallelism, SnapshotPendingSplitsState checkpoint, - SnapshotEventDialect dialect, - OffsetFactory offsetFactory, - BaseSchema baseSchema) { + DataSourceDialect dialect, + OffsetFactory offsetFactory) { this( sourceConfig, currentParallelism, @@ -112,8 +108,7 @@ public SnapshotSplitAssigner( checkpoint.isTableIdCaseSensitive(), checkpoint.isRemainingTablesCheckpointed(), dialect, - offsetFactory, - baseSchema); + offsetFactory); } private SnapshotSplitAssigner( @@ -127,9 +122,8 @@ private SnapshotSplitAssigner( List remainingTables, boolean isTableIdCaseSensitive, boolean isRemainingTablesCheckpointed, - SnapshotEventDialect dialect, - OffsetFactory offsetFactory, - BaseSchema baseSchema) { + DataSourceDialect dialect, + OffsetFactory offsetFactory) { this.sourceConfig = sourceConfig; this.currentParallelism = currentParallelism; this.alreadyProcessedTables = alreadyProcessedTables; @@ -142,20 +136,19 @@ private SnapshotSplitAssigner( this.isTableIdCaseSensitive = isTableIdCaseSensitive; this.dialect = dialect; this.offsetFactory = offsetFactory; - this.baseSchema = baseSchema; } @Override public void open() { - chunkSplitter = createChunkSplitter(sourceConfig); + chunkSplitter = dialect.createChunkSplitter(sourceConfig); // the legacy state didn't snapshot remaining tables, discovery remaining table here if (!isRemainingTablesCheckpointed && !assignerFinished) { try { - final List discoverTables = dialect.discoverCapturedTables(sourceConfig); + final List discoverTables = dialect.discoverDataCollections(sourceConfig); discoverTables.removeAll(alreadyProcessedTables); this.remainingTables.addAll(discoverTables); - this.isTableIdCaseSensitive = dialect.isTableIdCaseSensitive(sourceConfig); + this.isTableIdCaseSensitive = dialect.isDataCollectionIdCaseSensitive(sourceConfig); } catch (Exception e) { throw new FlinkRuntimeException( "Failed to discover remaining tables to capture", e); @@ -311,8 +304,4 @@ public Map getSplitFinishedOffsets() { private boolean allSplitsFinished() { return noMoreSplits() && assignedSplits.size() == splitFinishedOffsets.size(); } - - private ChunkSplitter createChunkSplitter(SourceConfig sourceConfig) { - return new ChunkSplitter(baseSchema, sourceConfig, dialect); - } } diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/assigners/SplitAssigner.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/SplitAssigner.java similarity index 88% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/assigners/SplitAssigner.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/SplitAssigner.java index 03065aa5d4b..67827314bfe 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/assigners/SplitAssigner.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/SplitAssigner.java @@ -16,14 +16,15 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.base.source.assigners; +package com.ververica.cdc.connectors.base.source.assigner; +import org.apache.flink.annotation.Experimental; import org.apache.flink.api.common.state.CheckpointListener; -import com.ververica.cdc.connectors.base.source.assigners.state.PendingSplitsState; -import com.ververica.cdc.connectors.base.source.offset.Offset; -import com.ververica.cdc.connectors.base.source.split.FinishedSnapshotSplitInfo; -import com.ververica.cdc.connectors.base.source.split.SourceSplitBase; +import com.ververica.cdc.connectors.base.source.assigner.state.PendingSplitsState; +import com.ververica.cdc.connectors.base.source.meta.offset.Offset; +import com.ververica.cdc.connectors.base.source.meta.split.FinishedSnapshotSplitInfo; +import com.ververica.cdc.connectors.base.source.meta.split.SourceSplitBase; import java.util.Collection; import java.util.List; @@ -31,9 +32,10 @@ import java.util.Optional; /** - * The {@code MySqlSplitAssigner} is responsible for deciding what split should be processed. It + * The {@code SplitAssigner} is responsible for deciding what split should be processed. It * determines split processing order. */ +@Experimental public interface SplitAssigner { /** diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/assigners/StreamSplitAssigner.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/StreamSplitAssigner.java similarity index 77% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/assigners/StreamSplitAssigner.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/StreamSplitAssigner.java index d22ff6f467f..ba61692637a 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/assigners/StreamSplitAssigner.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/StreamSplitAssigner.java @@ -16,18 +16,17 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.base.source.assigners; - -import com.ververica.cdc.connectors.base.source.assigners.state.PendingSplitsState; -import com.ververica.cdc.connectors.base.source.assigners.state.StreamPendingSplitsState; -import com.ververica.cdc.connectors.base.source.config.SourceConfig; -import com.ververica.cdc.connectors.base.source.dialect.Dialect; -import com.ververica.cdc.connectors.base.source.internal.connection.JdbcConnectionFactory; -import com.ververica.cdc.connectors.base.source.offset.Offset; -import com.ververica.cdc.connectors.base.source.offset.OffsetFactory; -import com.ververica.cdc.connectors.base.source.split.FinishedSnapshotSplitInfo; -import com.ververica.cdc.connectors.base.source.split.SourceSplitBase; -import com.ververica.cdc.connectors.base.source.split.StreamSplit; +package com.ververica.cdc.connectors.base.source.assigner; + +import com.ververica.cdc.connectors.base.config.SourceConfig; +import com.ververica.cdc.connectors.base.dialect.DataSourceDialect; +import com.ververica.cdc.connectors.base.source.assigner.state.PendingSplitsState; +import com.ververica.cdc.connectors.base.source.assigner.state.StreamPendingSplitsState; +import com.ververica.cdc.connectors.base.source.meta.offset.Offset; +import com.ververica.cdc.connectors.base.source.meta.offset.OffsetFactory; +import com.ververica.cdc.connectors.base.source.meta.split.FinishedSnapshotSplitInfo; +import com.ververica.cdc.connectors.base.source.meta.split.SourceSplitBase; +import com.ververica.cdc.connectors.base.source.meta.split.StreamSplit; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -49,19 +48,18 @@ public class StreamSplitAssigner implements SplitAssigner { private boolean isStreamSplitAssigned; - private JdbcConnectionFactory jdbcConnectionFactory; - private final Dialect dialect; + private final DataSourceDialect dialect; private final OffsetFactory offsetFactory; public StreamSplitAssigner( - SourceConfig sourceConfig, Dialect dialect, OffsetFactory offsetFactory) { + SourceConfig sourceConfig, DataSourceDialect dialect, OffsetFactory offsetFactory) { this(sourceConfig, false, dialect, offsetFactory); } public StreamSplitAssigner( SourceConfig sourceConfig, StreamPendingSplitsState checkpoint, - Dialect dialect, + DataSourceDialect dialect, OffsetFactory offsetFactory) { this(sourceConfig, checkpoint.isStreamSplitAssigned(), dialect, offsetFactory); } @@ -69,7 +67,7 @@ public StreamSplitAssigner( private StreamSplitAssigner( SourceConfig sourceConfig, boolean isStreamSplitAssigned, - Dialect dialect, + DataSourceDialect dialect, OffsetFactory offsetFactory) { this.sourceConfig = sourceConfig; this.isStreamSplitAssigned = isStreamSplitAssigned; @@ -127,6 +125,7 @@ public void close() {} // ------------------------------------------------------------------------------------------ public StreamSplit createStreamSplit() { + return new StreamSplit( BINLOG_SPLIT_ID, dialect.displayCurrentOffset(sourceConfig), diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/assigners/ChunkRange.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/splitter/ChunkRange.java similarity index 96% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/assigners/ChunkRange.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/splitter/ChunkRange.java index 5834b7ba890..fad9e4ecb80 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/assigners/ChunkRange.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/splitter/ChunkRange.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.base.source.assigners; +package com.ververica.cdc.connectors.base.source.assigner.splitter; import javax.annotation.Nullable; @@ -28,7 +28,7 @@ * An internal structure describes a chunk range with a chunk start (inclusive) and chunk end * (exclusive). Note that {@code null} represents unbounded chunk start/end. */ -class ChunkRange { +public class ChunkRange { private final @Nullable Object chunkStart; private final @Nullable Object chunkEnd; diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/split/StreamReader.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/splitter/ChunkSplitter.java similarity index 59% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/split/StreamReader.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/splitter/ChunkSplitter.java index 7f69b8c1818..e6a838aacbc 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/split/StreamReader.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/splitter/ChunkSplitter.java @@ -16,10 +16,19 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.base.source.reader.split; +package com.ververica.cdc.connectors.base.source.assigner.splitter; -/** - * A streaming change event reader that emits events from a DB log, such as MySQL's binlog or - * similar and filter overlapping snapshot data that {@link SnapshotReader} read. - */ -public interface StreamReader extends Reader {} +import org.apache.flink.annotation.Experimental; + +import com.ververica.cdc.connectors.base.source.meta.split.SnapshotSplit; +import io.debezium.schema.DataCollectionId; + +import java.util.Collection; + +/** The splitter used to split collection into a set of chunks. */ +@Experimental +public interface ChunkSplitter { + + /** Generates all snapshot splits (chunks) for the give data collection. */ + Collection generateSplits(ID dataCollectionId); +} diff --git a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/splitter/JdbcSourceChunkSplitter.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/splitter/JdbcSourceChunkSplitter.java new file mode 100644 index 00000000000..ac5dde64f57 --- /dev/null +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/splitter/JdbcSourceChunkSplitter.java @@ -0,0 +1,148 @@ +/* + * 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 com.ververica.cdc.connectors.base.source.assigner.splitter; + +import org.apache.flink.annotation.Experimental; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalTypeRoot; +import org.apache.flink.table.types.logical.RowType; + +import com.ververica.cdc.connectors.base.source.meta.split.SnapshotSplit; +import io.debezium.jdbc.JdbcConnection; +import io.debezium.relational.Column; +import io.debezium.relational.TableId; + +import java.sql.SQLException; +import java.util.Collection; + +import static org.apache.flink.table.api.DataTypes.FIELD; +import static org.apache.flink.table.api.DataTypes.ROW; + +/** The {@code ChunkSplitter} used to split table into a set of chunks for JDBC data source. */ +@Experimental +public interface JdbcSourceChunkSplitter extends ChunkSplitter { + + /** Generates all snapshot splits (chunks) for the give table path. */ + @Override + Collection generateSplits(TableId tableId); + + /** + * Query the maximum and minimum value of the column in the table. e.g. query string + * SELECT MIN(%s) FROM %s WHERE %s > ? + * + * @param jdbc JDBC connection. + * @param tableId table identity. + * @param columnName column name. + * @return maximum and minimum value. + */ + Object[] queryMinMax(JdbcConnection jdbc, TableId tableId, String columnName) + throws SQLException; + + /** + * Query the minimum value of the column in the table, and the minimum value must greater than + * the excludedLowerBound value. e.g. prepare query string + * SELECT MIN(%s) FROM %s WHERE %s > ? + * + * @param jdbc JDBC connection. + * @param tableId table identity. + * @param columnName column name. + * @param excludedLowerBound the minimum value should be greater than this value. + * @return minimum value. + */ + Object queryMin( + JdbcConnection jdbc, TableId tableId, String columnName, Object excludedLowerBound) + throws SQLException; + + /** + * Query the maximum value of the next chunk, and the next chunk must be greater than or equal + * to includedLowerBound value [min_1, max_1), [min_2, max_2),... [min_n, null). + * Each time this method is called it will return max1, max2... + * + * @param jdbc JDBC connection. + * @param tableId table identity. + * @param columnName column name. + * @param chunkSize chunk size. + * @param includedLowerBound the previous chunk end value. + * @return next chunk end value. + */ + Object queryNextChunkMax( + JdbcConnection jdbc, + TableId tableId, + String columnName, + int chunkSize, + Object includedLowerBound) + throws SQLException; + + /** + * Approximate total number of entries in the lookup table. + * + * @param jdbc JDBC connection. + * @param tableId table identity. + * @return approximate row count. + */ + Long queryApproximateRowCnt(JdbcConnection jdbc, TableId tableId) throws SQLException; + + /** + * Build the scan query sql of the {@link SnapshotSplit}. + * + * @param tableId table identity. + * @param splitKeyType primary key type. + * @param isFirstSplit whether the first split. + * @param isLastSplit whether the last split. + * @return query sql. + */ + String buildSplitScanQuery( + TableId tableId, RowType splitKeyType, boolean isFirstSplit, boolean isLastSplit); + + /** + * Checks whether split column is evenly distributed across its range. + * + * @param splitColumn split column. + * @return true that means split column with type BIGINT, INT, DECIMAL. + */ + default boolean isEvenlySplitColumn(Column splitColumn) { + DataType flinkType = fromDbzColumn(splitColumn); + LogicalTypeRoot typeRoot = flinkType.getLogicalType().getTypeRoot(); + + // currently, we only support the optimization that split column with type BIGINT, INT, + // DECIMAL + return typeRoot == LogicalTypeRoot.BIGINT + || typeRoot == LogicalTypeRoot.INTEGER + || typeRoot == LogicalTypeRoot.DECIMAL; + } + + /** + * Get a corresponding Flink data type from a debezium {@link Column}. + * + * @param splitColumn dbz split column. + * @return flink data type + */ + DataType fromDbzColumn(Column splitColumn); + + /** + * convert dbz column to Flink row type. + * + * @param splitColumn split column. + * @return flink row type. + */ + default RowType getSplitType(Column splitColumn) { + return (RowType) + ROW(FIELD(splitColumn.name(), fromDbzColumn(splitColumn))).getLogicalType(); + } +} diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/assigners/state/HybridPendingSplitsState.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/state/HybridPendingSplitsState.java similarity index 81% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/assigners/state/HybridPendingSplitsState.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/state/HybridPendingSplitsState.java index f7a5b5561e0..a9120d2555a 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/assigners/state/HybridPendingSplitsState.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/state/HybridPendingSplitsState.java @@ -16,27 +16,27 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.base.source.assigners.state; +package com.ververica.cdc.connectors.base.source.assigner.state; import java.util.Objects; /** A {@link PendingSplitsState} for pending hybrid (snapshot & binlog) splits. */ public class HybridPendingSplitsState extends PendingSplitsState { private final SnapshotPendingSplitsState snapshotPendingSplits; - private final boolean isBinlogSplitAssigned; + private final boolean isStreamSplitAssigned; public HybridPendingSplitsState( - SnapshotPendingSplitsState snapshotPendingSplits, boolean isBinlogSplitAssigned) { + SnapshotPendingSplitsState snapshotPendingSplits, boolean isStreamSplitAssigned) { this.snapshotPendingSplits = snapshotPendingSplits; - this.isBinlogSplitAssigned = isBinlogSplitAssigned; + this.isStreamSplitAssigned = isStreamSplitAssigned; } public SnapshotPendingSplitsState getSnapshotPendingSplits() { return snapshotPendingSplits; } - public boolean isBinlogSplitAssigned() { - return isBinlogSplitAssigned; + public boolean isStreamSplitAssigned() { + return isStreamSplitAssigned; } @Override @@ -48,13 +48,13 @@ public boolean equals(Object o) { return false; } HybridPendingSplitsState that = (HybridPendingSplitsState) o; - return isBinlogSplitAssigned == that.isBinlogSplitAssigned + return isStreamSplitAssigned == that.isStreamSplitAssigned && Objects.equals(snapshotPendingSplits, that.snapshotPendingSplits); } @Override public int hashCode() { - return Objects.hash(snapshotPendingSplits, isBinlogSplitAssigned); + return Objects.hash(snapshotPendingSplits, isStreamSplitAssigned); } @Override @@ -63,7 +63,7 @@ public String toString() { + "snapshotPendingSplits=" + snapshotPendingSplits + ", isBinlogSplitAssigned=" - + isBinlogSplitAssigned + + isStreamSplitAssigned + '}'; } } diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/assigners/state/PendingSplitsState.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/state/PendingSplitsState.java similarity index 95% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/assigners/state/PendingSplitsState.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/state/PendingSplitsState.java index 6eb3cdc54ad..722cfcecfa9 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/assigners/state/PendingSplitsState.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/state/PendingSplitsState.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.base.source.assigners.state; +package com.ververica.cdc.connectors.base.source.assigner.state; import javax.annotation.Nullable; diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/assigners/state/PendingSplitsStateSerializer.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/state/PendingSplitsStateSerializer.java similarity index 97% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/assigners/state/PendingSplitsStateSerializer.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/state/PendingSplitsStateSerializer.java index 3a33c83d9e5..63adc01998e 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/assigners/state/PendingSplitsStateSerializer.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/state/PendingSplitsStateSerializer.java @@ -16,16 +16,16 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.base.source.assigners.state; +package com.ververica.cdc.connectors.base.source.assigner.state; import org.apache.flink.core.io.SimpleVersionedSerializer; import org.apache.flink.core.memory.DataInputDeserializer; import org.apache.flink.core.memory.DataOutputSerializer; -import com.ververica.cdc.connectors.base.source.offset.Offset; -import com.ververica.cdc.connectors.base.source.split.SnapshotSplit; -import com.ververica.cdc.connectors.base.source.split.SourceSplitBase; -import com.ververica.cdc.connectors.base.source.split.SourceSplitSerializer; +import com.ververica.cdc.connectors.base.source.meta.offset.Offset; +import com.ververica.cdc.connectors.base.source.meta.split.SnapshotSplit; +import com.ververica.cdc.connectors.base.source.meta.split.SourceSplitBase; +import com.ververica.cdc.connectors.base.source.meta.split.SourceSplitSerializer; import io.debezium.relational.TableId; import java.io.IOException; @@ -156,7 +156,7 @@ private void serializeSnapshotPendingSplitsState( private void serializeHybridPendingSplitsState( HybridPendingSplitsState state, DataOutputSerializer out) throws IOException { serializeSnapshotPendingSplitsState(state.getSnapshotPendingSplits(), out); - out.writeBoolean(state.isBinlogSplitAssigned()); + out.writeBoolean(state.isStreamSplitAssigned()); } private void serializeBinlogPendingSplitsState( diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/assigners/state/SnapshotPendingSplitsState.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/state/SnapshotPendingSplitsState.java similarity index 90% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/assigners/state/SnapshotPendingSplitsState.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/state/SnapshotPendingSplitsState.java index 53bab3b3345..b3f09ff9242 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/assigners/state/SnapshotPendingSplitsState.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/state/SnapshotPendingSplitsState.java @@ -16,12 +16,12 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.base.source.assigners.state; +package com.ververica.cdc.connectors.base.source.assigner.state; -import com.ververica.cdc.connectors.base.source.enumerator.SourceEnumerator; -import com.ververica.cdc.connectors.base.source.offset.Offset; -import com.ververica.cdc.connectors.base.source.reader.BaseSplitReader; -import com.ververica.cdc.connectors.base.source.split.SnapshotSplit; +import com.ververica.cdc.connectors.base.source.enumerator.IncrementalSourceEnumerator; +import com.ververica.cdc.connectors.base.source.meta.offset.Offset; +import com.ververica.cdc.connectors.base.source.meta.split.SnapshotSplit; +import com.ververica.cdc.connectors.base.source.reader.JdbcSourceSplitReader; import io.debezium.relational.TableId; import java.util.List; @@ -44,14 +44,14 @@ public class SnapshotPendingSplitsState extends PendingSplitsState { private final List remainingSplits; /** - * The snapshot splits that the {@link SourceEnumerator} has assigned to {@link - * BaseSplitReader}s. + * The snapshot splits that the {@link IncrementalSourceEnumerator} has assigned to {@link + * JdbcSourceSplitReader}s. */ private final Map assignedSplits; /** - * The offsets of finished (snapshot) splits that the {@link SourceEnumerator} has received from - * {@link BaseSplitReader}s. + * The offsets of finished (snapshot) splits that the {@link IncrementalSourceEnumerator} has + * received from {@link JdbcSourceSplitReader}s. */ private final Map splitFinishedOffsets; diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/assigners/state/StreamPendingSplitsState.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/state/StreamPendingSplitsState.java similarity index 96% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/assigners/state/StreamPendingSplitsState.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/state/StreamPendingSplitsState.java index 33f80ae9ba3..3e297733714 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/assigners/state/StreamPendingSplitsState.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/state/StreamPendingSplitsState.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.base.source.assigners.state; +package com.ververica.cdc.connectors.base.source.assigner.state; import java.util.Objects; diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/enumerator/SourceEnumerator.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/enumerator/IncrementalSourceEnumerator.java similarity index 84% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/enumerator/SourceEnumerator.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/enumerator/IncrementalSourceEnumerator.java index 0a8cfe96d77..4afa5502c40 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/enumerator/SourceEnumerator.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/enumerator/IncrementalSourceEnumerator.java @@ -18,6 +18,7 @@ package com.ververica.cdc.connectors.base.source.enumerator; +import org.apache.flink.annotation.Experimental; import org.apache.flink.annotation.Internal; import org.apache.flink.api.connector.source.SourceEvent; import org.apache.flink.api.connector.source.SplitEnumerator; @@ -26,17 +27,17 @@ import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; -import com.ververica.cdc.connectors.base.source.assigners.SplitAssigner; -import com.ververica.cdc.connectors.base.source.assigners.state.PendingSplitsState; -import com.ververica.cdc.connectors.base.source.config.SourceConfig; -import com.ververica.cdc.connectors.base.source.events.FinishedSnapshotSplitsAckEvent; -import com.ververica.cdc.connectors.base.source.events.FinishedSnapshotSplitsReportEvent; -import com.ververica.cdc.connectors.base.source.events.FinishedSnapshotSplitsRequestEvent; -import com.ververica.cdc.connectors.base.source.events.StreamSplitMetaEvent; -import com.ververica.cdc.connectors.base.source.events.StreamSplitMetaRequestEvent; -import com.ververica.cdc.connectors.base.source.offset.Offset; -import com.ververica.cdc.connectors.base.source.split.FinishedSnapshotSplitInfo; -import com.ververica.cdc.connectors.base.source.split.SourceSplitBase; +import com.ververica.cdc.connectors.base.config.SourceConfig; +import com.ververica.cdc.connectors.base.source.assigner.SplitAssigner; +import com.ververica.cdc.connectors.base.source.assigner.state.PendingSplitsState; +import com.ververica.cdc.connectors.base.source.meta.events.FinishedSnapshotSplitsAckEvent; +import com.ververica.cdc.connectors.base.source.meta.events.FinishedSnapshotSplitsReportEvent; +import com.ververica.cdc.connectors.base.source.meta.events.FinishedSnapshotSplitsRequestEvent; +import com.ververica.cdc.connectors.base.source.meta.events.StreamSplitMetaEvent; +import com.ververica.cdc.connectors.base.source.meta.events.StreamSplitMetaRequestEvent; +import com.ververica.cdc.connectors.base.source.meta.offset.Offset; +import com.ververica.cdc.connectors.base.source.meta.split.FinishedSnapshotSplitInfo; +import com.ververica.cdc.connectors.base.source.meta.split.SourceSplitBase; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -54,9 +55,10 @@ * A CDC source enumerator that enumerates receive the split request and assign the split to source * readers. */ -@Internal -public class SourceEnumerator implements SplitEnumerator { - private static final Logger LOG = LoggerFactory.getLogger(SourceEnumerator.class); +@Experimental +public class IncrementalSourceEnumerator + implements SplitEnumerator { + private static final Logger LOG = LoggerFactory.getLogger(IncrementalSourceEnumerator.class); private static final long CHECK_EVENT_INTERVAL = 30_000L; private final SplitEnumeratorContext context; @@ -67,7 +69,7 @@ public class SourceEnumerator implements SplitEnumerator readersAwaitingSplit; private List> binlogSplitMeta; - public SourceEnumerator( + public IncrementalSourceEnumerator( SplitEnumeratorContext context, SourceConfig sourceConfig, SplitAssigner splitAssigner) { @@ -187,9 +189,11 @@ private void syncWithReaders(int[] subtaskIds, Throwable t) { if (t != null) { throw new FlinkRuntimeException("Failed to list obtain registered readers due to:", t); } - // when the SourceEnumerator restores or the communication failed between - // SourceEnumerator and SourceReader, it may missed some notification event. - // tell all SourceReader(s) to report there finished but unacked splits. + // when the IncrementalSourceEnumerator restores or the communication failed between + // IncrementalSourceEnumerator and JdbcIncrementalSourceReader, it may missed some + // notification + // event. + // tell all JdbcIncrementalSourceReader(s) to report there finished but unacked splits. if (splitAssigner.waitingForFinishedSplits()) { for (int subtaskId : subtaskIds) { context.sendEventToSourceReader( diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/events/FinishedSnapshotSplitsAckEvent.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/events/FinishedSnapshotSplitsAckEvent.java similarity index 75% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/events/FinishedSnapshotSplitsAckEvent.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/events/FinishedSnapshotSplitsAckEvent.java index 41d39c80045..f8fd7c95532 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/events/FinishedSnapshotSplitsAckEvent.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/events/FinishedSnapshotSplitsAckEvent.java @@ -16,19 +16,19 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.base.source.events; +package com.ververica.cdc.connectors.base.source.meta.events; import org.apache.flink.api.connector.source.SourceEvent; -import com.ververica.cdc.connectors.base.source.enumerator.SourceEnumerator; -import com.ververica.cdc.connectors.base.source.reader.ParallelSourceReader; +import com.ververica.cdc.connectors.base.source.enumerator.IncrementalSourceEnumerator; +import com.ververica.cdc.connectors.base.source.reader.JdbcIncrementalSourceReader; import java.util.List; /** - * The {@link SourceEvent} that {@link SourceEnumerator} sends to {@link ParallelSourceReader} to - * notify the finished snapshot splits has been received, i.e. acknowledge for {@link - * FinishedSnapshotSplitsReportEvent}. + * The {@link SourceEvent} that {@link IncrementalSourceEnumerator} sends to {@link + * JdbcIncrementalSourceReader} to notify the finished snapshot splits has been received, i.e. + * acknowledge for {@link FinishedSnapshotSplitsReportEvent}. */ public class FinishedSnapshotSplitsAckEvent implements SourceEvent { diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/events/FinishedSnapshotSplitsReportEvent.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/events/FinishedSnapshotSplitsReportEvent.java similarity index 74% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/events/FinishedSnapshotSplitsReportEvent.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/events/FinishedSnapshotSplitsReportEvent.java index 6c5a3ff212f..d0b30781d85 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/events/FinishedSnapshotSplitsReportEvent.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/events/FinishedSnapshotSplitsReportEvent.java @@ -16,19 +16,20 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.base.source.events; +package com.ververica.cdc.connectors.base.source.meta.events; import org.apache.flink.api.connector.source.SourceEvent; -import com.ververica.cdc.connectors.base.source.enumerator.SourceEnumerator; -import com.ververica.cdc.connectors.base.source.offset.Offset; -import com.ververica.cdc.connectors.base.source.reader.ParallelSourceReader; +import com.ververica.cdc.connectors.base.source.enumerator.IncrementalSourceEnumerator; +import com.ververica.cdc.connectors.base.source.meta.offset.Offset; +import com.ververica.cdc.connectors.base.source.reader.JdbcIncrementalSourceReader; import java.util.Map; /** - * The {@link SourceEvent} that {@link SourceEnumerator} sends to {@link ParallelSourceReader} to - * notify the snapshot split has read finished with the consistent binlog position. + * The {@link SourceEvent} that {@link IncrementalSourceEnumerator} sends to {@link + * JdbcIncrementalSourceReader} to notify the snapshot split has read finished with the consistent + * binlog position. */ public class FinishedSnapshotSplitsReportEvent implements SourceEvent { diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/events/FinishedSnapshotSplitsRequestEvent.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/events/FinishedSnapshotSplitsRequestEvent.java similarity index 69% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/events/FinishedSnapshotSplitsRequestEvent.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/events/FinishedSnapshotSplitsRequestEvent.java index 6da10629523..fa313435e92 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/events/FinishedSnapshotSplitsRequestEvent.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/events/FinishedSnapshotSplitsRequestEvent.java @@ -16,17 +16,17 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.base.source.events; +package com.ververica.cdc.connectors.base.source.meta.events; import org.apache.flink.api.connector.source.SourceEvent; -import com.ververica.cdc.connectors.base.source.enumerator.SourceEnumerator; -import com.ververica.cdc.connectors.base.source.reader.ParallelSourceReader; +import com.ververica.cdc.connectors.base.source.enumerator.IncrementalSourceEnumerator; +import com.ververica.cdc.connectors.base.source.reader.JdbcIncrementalSourceReader; /** - * The {@link SourceEvent} that {@link SourceEnumerator} sends to {@link ParallelSourceReader} to - * notify reader should report its finished snapshot splits, i.e. sending {@link - * FinishedSnapshotSplitsReportEvent}. + * The {@link SourceEvent} that {@link IncrementalSourceEnumerator} sends to {@link + * JdbcIncrementalSourceReader} to notify reader should report its finished snapshot splits, i.e. + * sending {@link FinishedSnapshotSplitsReportEvent}. */ public class FinishedSnapshotSplitsRequestEvent implements SourceEvent { diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/events/StreamSplitMetaEvent.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/events/StreamSplitMetaEvent.java similarity index 78% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/events/StreamSplitMetaEvent.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/events/StreamSplitMetaEvent.java index 52deb29c208..bbc89d0789c 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/events/StreamSplitMetaEvent.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/events/StreamSplitMetaEvent.java @@ -16,19 +16,19 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.base.source.events; +package com.ververica.cdc.connectors.base.source.meta.events; import org.apache.flink.api.connector.source.SourceEvent; -import com.ververica.cdc.connectors.base.source.enumerator.SourceEnumerator; -import com.ververica.cdc.connectors.base.source.reader.ParallelSourceReader; -import com.ververica.cdc.connectors.base.source.split.FinishedSnapshotSplitInfo; +import com.ververica.cdc.connectors.base.source.enumerator.IncrementalSourceEnumerator; +import com.ververica.cdc.connectors.base.source.meta.split.FinishedSnapshotSplitInfo; +import com.ververica.cdc.connectors.base.source.reader.JdbcIncrementalSourceReader; import java.util.List; /** - * The {@link SourceEvent} that {@link SourceEnumerator} sends to {@link ParallelSourceReader} to - * pass binlog meta data, i.e. {@link FinishedSnapshotSplitInfo}. + * The {@link SourceEvent} that {@link IncrementalSourceEnumerator} sends to {@link + * JdbcIncrementalSourceReader} to pass binlog meta data, i.e. {@link FinishedSnapshotSplitInfo}. */ public class StreamSplitMetaEvent implements SourceEvent { diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/events/StreamSplitMetaRequestEvent.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/events/StreamSplitMetaRequestEvent.java similarity index 76% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/events/StreamSplitMetaRequestEvent.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/events/StreamSplitMetaRequestEvent.java index d6762adaf65..486bf38f856 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/events/StreamSplitMetaRequestEvent.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/events/StreamSplitMetaRequestEvent.java @@ -16,16 +16,16 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.base.source.events; +package com.ververica.cdc.connectors.base.source.meta.events; import org.apache.flink.api.connector.source.SourceEvent; -import com.ververica.cdc.connectors.base.source.enumerator.SourceEnumerator; -import com.ververica.cdc.connectors.base.source.reader.ParallelSourceReader; +import com.ververica.cdc.connectors.base.source.enumerator.IncrementalSourceEnumerator; +import com.ververica.cdc.connectors.base.source.reader.JdbcIncrementalSourceReader; /** - * The {@link SourceEvent} that {@link SourceEnumerator} sends to {@link ParallelSourceReader} to - * pull binlog meta data, i.e. sending {@link StreamSplitMetaEvent}. + * The {@link SourceEvent} that {@link IncrementalSourceEnumerator} sends to {@link + * JdbcIncrementalSourceReader} to pull binlog meta data, i.e. sending {@link StreamSplitMetaEvent}. */ public class StreamSplitMetaRequestEvent implements SourceEvent { diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/offset/Offset.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/offset/Offset.java similarity index 96% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/offset/Offset.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/offset/Offset.java index caa208086b9..92e4dcef7be 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/offset/Offset.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/offset/Offset.java @@ -16,7 +16,9 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.base.source.offset; +package com.ververica.cdc.connectors.base.source.meta.offset; + +import org.apache.flink.annotation.Experimental; import org.apache.kafka.connect.errors.ConnectException; @@ -33,6 +35,7 @@ * from a specific {@link Offset}, we need to skip the processed change events and the processed * rows. */ +@Experimental public abstract class Offset implements Comparable, Serializable { private static final long serialVersionUID = 1L; diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/offset/OffsetDeserializerSerializer.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/offset/OffsetDeserializerSerializer.java similarity index 93% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/offset/OffsetDeserializerSerializer.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/offset/OffsetDeserializerSerializer.java index 7b4d4ffaddc..f2c4e073775 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/offset/OffsetDeserializerSerializer.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/offset/OffsetDeserializerSerializer.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.base.source.offset; +package com.ververica.cdc.connectors.base.source.meta.offset; import org.apache.flink.core.memory.DataInputDeserializer; import org.apache.flink.core.memory.DataOutputSerializer; @@ -24,19 +24,21 @@ import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; -import com.ververica.cdc.connectors.base.source.split.FinishedSnapshotSplitInfo; +import com.ververica.cdc.connectors.base.source.meta.split.FinishedSnapshotSplitInfo; import io.debezium.relational.TableId; import java.io.IOException; import java.io.Serializable; import java.util.Map; -import static com.ververica.cdc.connectors.base.source.utils.SerializerUtils.serializedStringToObject; -import static com.ververica.cdc.connectors.base.source.utils.SerializerUtils.serializedStringToRow; +import static com.ververica.cdc.connectors.base.utils.SerializerUtils.serializedStringToObject; +import static com.ververica.cdc.connectors.base.utils.SerializerUtils.serializedStringToRow; /** read {@link Offset} from input stream and write {@link Offset} to output stream. */ public interface OffsetDeserializerSerializer extends Serializable { + OffsetFactory getOffsetFactory(); + default Offset readOffsetPosition(int offsetVersion, DataInputDeserializer in) throws IOException { switch (offsetVersion) { @@ -75,8 +77,6 @@ default void writeOffsetPosition(Offset offset, DataOutputSerializer out) throws } } - OffsetFactory getOffsetFactory(); - default OffsetDeserializer createOffsetDeserializer() { return new OffsetDeserializer(getOffsetFactory()); } diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/offset/OffsetFactory.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/offset/OffsetFactory.java similarity index 95% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/offset/OffsetFactory.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/offset/OffsetFactory.java index ec3a62c6cd2..fd55a636da1 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/offset/OffsetFactory.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/offset/OffsetFactory.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.base.source.offset; +package com.ververica.cdc.connectors.base.source.meta.offset; import java.io.Serializable; import java.util.Map; diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/split/ChangeEventRecords.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/split/ChangeEventRecords.java similarity index 98% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/split/ChangeEventRecords.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/split/ChangeEventRecords.java index 6658a02508d..25984bc26e0 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/split/ChangeEventRecords.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/split/ChangeEventRecords.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.base.source.split; +package com.ververica.cdc.connectors.base.source.meta.split; import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/split/FinishedSnapshotSplitInfo.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/split/FinishedSnapshotSplitInfo.java similarity index 87% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/split/FinishedSnapshotSplitInfo.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/split/FinishedSnapshotSplitInfo.java index a2056988d8d..c1c4f91841f 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/split/FinishedSnapshotSplitInfo.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/split/FinishedSnapshotSplitInfo.java @@ -16,21 +16,21 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.base.source.split; +package com.ververica.cdc.connectors.base.source.meta.split; import org.apache.flink.core.memory.DataOutputSerializer; import org.apache.flink.util.FlinkRuntimeException; -import com.ververica.cdc.connectors.base.source.offset.Offset; -import com.ververica.cdc.connectors.base.source.offset.OffsetDeserializerSerializer; -import com.ververica.cdc.connectors.base.source.offset.OffsetFactory; +import com.ververica.cdc.connectors.base.source.meta.offset.Offset; +import com.ververica.cdc.connectors.base.source.meta.offset.OffsetDeserializerSerializer; +import com.ververica.cdc.connectors.base.source.meta.offset.OffsetFactory; +import com.ververica.cdc.connectors.base.utils.SerializerUtils; import io.debezium.relational.TableId; import java.io.IOException; import java.util.Arrays; import java.util.Objects; -import static com.ververica.cdc.connectors.base.source.utils.SerializerUtils.rowToSerializedString; import static org.apache.flink.util.Preconditions.checkNotNull; /** The information used to describe a finished snapshot split. */ @@ -124,7 +124,8 @@ public String toString() { } // ------------------------------------------------------------------------------------ - // Utils to serialize/deserialize for transmission between Enumerator and SourceReader + // Utils to serialize/deserialize for transmission between Enumerator and + // JdbcIncrementalSourceReader // ------------------------------------------------------------------------------------ public byte[] serialize() { try { @@ -140,9 +141,9 @@ public byte[] serialize() { public byte[] serialize(final DataOutputSerializer out) throws IOException { out.writeUTF(this.getTableId().toString()); out.writeUTF(this.getSplitId()); - out.writeUTF(rowToSerializedString(this.getSplitStart())); - out.writeUTF(rowToSerializedString(this.getSplitEnd())); - out.writeUTF(rowToSerializedString(this.offsetFactory)); + out.writeUTF(SerializerUtils.rowToSerializedString(this.getSplitStart())); + out.writeUTF(SerializerUtils.rowToSerializedString(this.getSplitEnd())); + out.writeUTF(SerializerUtils.rowToSerializedString(this.offsetFactory)); writeOffsetPosition(this.getHighWatermark(), out); return out.getCopyOfBuffer(); } diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/split/SnapshotSplit.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/split/SnapshotSplit.java similarity index 97% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/split/SnapshotSplit.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/split/SnapshotSplit.java index 518108c3849..7e3df3a83a7 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/split/SnapshotSplit.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/split/SnapshotSplit.java @@ -16,11 +16,11 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.base.source.split; +package com.ververica.cdc.connectors.base.source.meta.split; import org.apache.flink.table.types.logical.RowType; -import com.ververica.cdc.connectors.base.source.offset.Offset; +import com.ververica.cdc.connectors.base.source.meta.offset.Offset; import io.debezium.relational.TableId; import io.debezium.relational.history.TableChanges.TableChange; diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/split/SnapshotSplitState.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/split/SnapshotSplitState.java similarity index 94% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/split/SnapshotSplitState.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/split/SnapshotSplitState.java index c308c441c19..47c99992bab 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/split/SnapshotSplitState.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/split/SnapshotSplitState.java @@ -16,9 +16,9 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.base.source.split; +package com.ververica.cdc.connectors.base.source.meta.split; -import com.ververica.cdc.connectors.base.source.offset.Offset; +import com.ververica.cdc.connectors.base.source.meta.offset.Offset; import javax.annotation.Nullable; diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/split/SourceSplitBase.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/split/SourceSplitBase.java similarity index 87% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/split/SourceSplitBase.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/split/SourceSplitBase.java index c15b209ef9c..d05b8d97fc9 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/split/SourceSplitBase.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/split/SourceSplitBase.java @@ -16,17 +16,19 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.base.source.split; +package com.ververica.cdc.connectors.base.source.meta.split; +import org.apache.flink.annotation.Experimental; import org.apache.flink.api.connector.source.SourceSplit; import io.debezium.relational.TableId; -import io.debezium.relational.history.TableChanges; +import io.debezium.relational.history.TableChanges.TableChange; import java.util.Map; import java.util.Objects; /** The split of table comes from a Table that splits by primary key. */ +@Experimental public abstract class SourceSplitBase implements SourceSplit { protected final String splitId; @@ -40,7 +42,7 @@ public final boolean isSnapshotSplit() { return getClass() == SnapshotSplit.class; } - /** Checks whether this split is a binlog split. */ + /** Checks whether this split is a stream split. */ public final boolean isStreamSplit() { return getClass() == StreamSplit.class; } @@ -60,7 +62,7 @@ public String splitId() { return splitId; } - public abstract Map getTableSchemas(); + public abstract Map getTableSchemas(); @Override public boolean equals(Object o) { diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/split/SourceSplitSerializer.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/split/SourceSplitSerializer.java similarity index 90% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/split/SourceSplitSerializer.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/split/SourceSplitSerializer.java index db34cd95d77..c615b2ab435 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/split/SourceSplitSerializer.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/split/SourceSplitSerializer.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.base.source.split; +package com.ververica.cdc.connectors.base.source.meta.split; import org.apache.flink.core.io.SimpleVersionedSerializer; import org.apache.flink.core.memory.DataInputDeserializer; @@ -24,9 +24,10 @@ import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.utils.LogicalTypeParser; -import com.ververica.cdc.connectors.base.source.offset.Offset; -import com.ververica.cdc.connectors.base.source.offset.OffsetDeserializerSerializer; -import com.ververica.cdc.connectors.base.source.offset.OffsetFactory; +import com.ververica.cdc.connectors.base.source.meta.offset.Offset; +import com.ververica.cdc.connectors.base.source.meta.offset.OffsetDeserializerSerializer; +import com.ververica.cdc.connectors.base.source.meta.offset.OffsetFactory; +import com.ververica.cdc.connectors.base.utils.SerializerUtils; import com.ververica.cdc.debezium.history.FlinkJsonTableChangeSerializer; import io.debezium.document.Document; import io.debezium.document.DocumentReader; @@ -41,10 +42,6 @@ import java.util.List; import java.util.Map; -import static com.ververica.cdc.connectors.base.source.utils.SerializerUtils.rowToSerializedString; -import static com.ververica.cdc.connectors.base.source.utils.SerializerUtils.serializedStringToObject; -import static com.ververica.cdc.connectors.base.source.utils.SerializerUtils.serializedStringToRow; - /** A serializer for the {@link SourceSplitBase}. */ public abstract class SourceSplitSerializer implements SimpleVersionedSerializer, OffsetDeserializerSerializer { @@ -79,8 +76,8 @@ public byte[] serialize(SourceSplitBase split) throws IOException { final Object[] splitStart = snapshotSplit.getSplitStart(); final Object[] splitEnd = snapshotSplit.getSplitEnd(); // rowToSerializedString deals null case - out.writeUTF(rowToSerializedString(splitStart)); - out.writeUTF(rowToSerializedString(splitEnd)); + out.writeUTF(SerializerUtils.rowToSerializedString(splitStart)); + out.writeUTF(SerializerUtils.rowToSerializedString(splitEnd)); writeOffsetPosition(snapshotSplit.getHighWatermark(), out); writeTableSchemas(snapshotSplit.getTableSchemas(), out); final byte[] result = out.getCopyOfBuffer(); @@ -133,8 +130,8 @@ public SourceSplitBase deserializeSplit(int version, byte[] serialized) throws I TableId tableId = TableId.parse(in.readUTF()); String splitId = in.readUTF(); RowType splitKeyType = (RowType) LogicalTypeParser.parse(in.readUTF()); - Object[] splitBoundaryStart = serializedStringToRow(in.readUTF()); - Object[] splitBoundaryEnd = serializedStringToRow(in.readUTF()); + Object[] splitBoundaryStart = SerializerUtils.serializedStringToRow(in.readUTF()); + Object[] splitBoundaryEnd = SerializerUtils.serializedStringToRow(in.readUTF()); Offset highWatermark = readOffsetPosition(version, in); Map tableSchemas = readTableSchemas(version, in); @@ -234,9 +231,10 @@ private List readFinishedSplitsInfo( for (int i = 0; i < size; i++) { TableId tableId = TableId.parse(in.readUTF()); String splitId = in.readUTF(); - Object[] splitStart = serializedStringToRow(in.readUTF()); - Object[] splitEnd = serializedStringToRow(in.readUTF()); - OffsetFactory offsetFactory = (OffsetFactory) serializedStringToObject(in.readUTF()); + Object[] splitStart = SerializerUtils.serializedStringToRow(in.readUTF()); + Object[] splitEnd = SerializerUtils.serializedStringToRow(in.readUTF()); + OffsetFactory offsetFactory = + (OffsetFactory) SerializerUtils.serializedStringToObject(in.readUTF()); Offset highWatermark = readOffsetPosition(version, in); finishedSplitsInfo.add( diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/split/SourceSplitState.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/split/SourceSplitState.java similarity index 96% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/split/SourceSplitState.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/split/SourceSplitState.java index 890efb8060a..61b733150fa 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/split/SourceSplitState.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/split/SourceSplitState.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.base.source.split; +package com.ververica.cdc.connectors.base.source.meta.split; /** State of the reader, essentially a mutable version of the {@link SourceSplitBase}. */ public abstract class SourceSplitState { diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/split/StreamSplit.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/split/StreamSplit.java similarity index 97% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/split/StreamSplit.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/split/StreamSplit.java index a90e1f7ca23..4aed04c930a 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/split/StreamSplit.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/split/StreamSplit.java @@ -16,9 +16,9 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.base.source.split; +package com.ververica.cdc.connectors.base.source.meta.split; -import com.ververica.cdc.connectors.base.source.offset.Offset; +import com.ververica.cdc.connectors.base.source.meta.offset.Offset; import io.debezium.relational.TableId; import io.debezium.relational.history.TableChanges.TableChange; diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/split/StreamSplitState.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/split/StreamSplitState.java similarity index 95% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/split/StreamSplitState.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/split/StreamSplitState.java index 121246c9c51..25c51d5612f 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/split/StreamSplitState.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/split/StreamSplitState.java @@ -16,9 +16,9 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.base.source.split; +package com.ververica.cdc.connectors.base.source.meta.split; -import com.ververica.cdc.connectors.base.source.offset.Offset; +import com.ververica.cdc.connectors.base.source.meta.offset.Offset; import io.debezium.relational.TableId; import io.debezium.relational.history.TableChanges.TableChange; diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/metrics/SourceReaderMetrics.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/metrics/SourceReaderMetrics.java similarity index 87% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/metrics/SourceReaderMetrics.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/metrics/SourceReaderMetrics.java index 3c067f1fcdb..2afa8f47a60 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/metrics/SourceReaderMetrics.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/metrics/SourceReaderMetrics.java @@ -21,17 +21,17 @@ import org.apache.flink.metrics.Gauge; import org.apache.flink.metrics.MetricGroup; -import com.ververica.cdc.connectors.base.source.reader.ParallelSourceReader; +import com.ververica.cdc.connectors.base.source.reader.JdbcIncrementalSourceReader; -/** A collection class for handling metrics in {@link ParallelSourceReader}. */ +/** A collection class for handling metrics in {@link JdbcIncrementalSourceReader}. */ public class SourceReaderMetrics { private final MetricGroup metricGroup; /** - * The last record processing time, which is updated after {@link ParallelSourceReader} fetches - * a batch of data. It's mainly used to report metrics sourceIdleTime for sourceIdleTime = - * System.currentTimeMillis() - processTime. + * The last record processing time, which is updated after {@link JdbcIncrementalSourceReader} + * fetches a batch of data. It's mainly used to report metrics sourceIdleTime for sourceIdleTime + * = System.currentTimeMillis() - processTime. */ private volatile long processTime = 0L; diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/ParallelSourceReader.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/JdbcIncrementalSourceReader.java similarity index 83% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/ParallelSourceReader.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/JdbcIncrementalSourceReader.java index 855d2d513fe..dafc8c6c58f 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/ParallelSourceReader.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/JdbcIncrementalSourceReader.java @@ -18,6 +18,7 @@ package com.ververica.cdc.connectors.base.source.reader; +import org.apache.flink.annotation.Experimental; import org.apache.flink.api.connector.source.SourceEvent; import org.apache.flink.api.connector.source.SourceReaderContext; import org.apache.flink.configuration.Configuration; @@ -29,39 +30,36 @@ import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.util.Preconditions; -import com.ververica.cdc.connectors.base.source.config.SourceConfig; -import com.ververica.cdc.connectors.base.source.dialect.Dialect; -import com.ververica.cdc.connectors.base.source.events.FinishedSnapshotSplitsAckEvent; -import com.ververica.cdc.connectors.base.source.events.FinishedSnapshotSplitsReportEvent; -import com.ververica.cdc.connectors.base.source.events.FinishedSnapshotSplitsRequestEvent; -import com.ververica.cdc.connectors.base.source.events.StreamSplitMetaEvent; -import com.ververica.cdc.connectors.base.source.events.StreamSplitMetaRequestEvent; -import com.ververica.cdc.connectors.base.source.offset.Offset; -import com.ververica.cdc.connectors.base.source.split.FinishedSnapshotSplitInfo; -import com.ververica.cdc.connectors.base.source.split.SnapshotSplit; -import com.ververica.cdc.connectors.base.source.split.SnapshotSplitState; -import com.ververica.cdc.connectors.base.source.split.SourceSplitBase; -import com.ververica.cdc.connectors.base.source.split.SourceSplitSerializer; -import com.ververica.cdc.connectors.base.source.split.SourceSplitState; -import com.ververica.cdc.connectors.base.source.split.StreamSplit; -import com.ververica.cdc.connectors.base.source.split.StreamSplitState; +import com.ververica.cdc.connectors.base.config.JdbcSourceConfig; +import com.ververica.cdc.connectors.base.dialect.JdbcDataSourceDialect; +import com.ververica.cdc.connectors.base.source.meta.events.FinishedSnapshotSplitsAckEvent; +import com.ververica.cdc.connectors.base.source.meta.events.FinishedSnapshotSplitsReportEvent; +import com.ververica.cdc.connectors.base.source.meta.events.FinishedSnapshotSplitsRequestEvent; +import com.ververica.cdc.connectors.base.source.meta.events.StreamSplitMetaEvent; +import com.ververica.cdc.connectors.base.source.meta.events.StreamSplitMetaRequestEvent; +import com.ververica.cdc.connectors.base.source.meta.offset.Offset; +import com.ververica.cdc.connectors.base.source.meta.split.FinishedSnapshotSplitInfo; +import com.ververica.cdc.connectors.base.source.meta.split.SnapshotSplit; +import com.ververica.cdc.connectors.base.source.meta.split.SnapshotSplitState; +import com.ververica.cdc.connectors.base.source.meta.split.SourceSplitBase; +import com.ververica.cdc.connectors.base.source.meta.split.SourceSplitSerializer; +import com.ververica.cdc.connectors.base.source.meta.split.SourceSplitState; +import com.ververica.cdc.connectors.base.source.meta.split.StreamSplit; +import com.ververica.cdc.connectors.base.source.meta.split.StreamSplitState; import io.debezium.relational.TableId; import io.debezium.relational.history.TableChanges; import org.apache.kafka.connect.source.SourceRecord; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.sql.SQLException; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.function.Function; import java.util.function.Supplier; import java.util.stream.Collectors; -import static com.ververica.cdc.connectors.base.source.split.StreamSplit.fillTableSchemas; import static org.apache.flink.util.Preconditions.checkNotNull; import static org.apache.flink.util.Preconditions.checkState; @@ -69,28 +67,29 @@ * The multi-parallel source reader for table snapshot phase from {@link SnapshotSplit} and then * single-parallel source reader for table stream phase from {@link StreamSplit}. */ -public class ParallelSourceReader +@Experimental +public class JdbcIncrementalSourceReader extends SingleThreadMultiplexSourceReaderBase< SourceRecord, T, SourceSplitBase, SourceSplitState> { - private static final Logger LOG = LoggerFactory.getLogger(ParallelSourceReader.class); + private static final Logger LOG = LoggerFactory.getLogger(JdbcIncrementalSourceReader.class); - private final SourceConfig sourceConfig; private final Map finishedUnackedSplits; private final Map uncompletedBinlogSplits; private final int subtaskId; private final SourceSplitSerializer sourceSplitSerializer; - private final Dialect dialect; + private final JdbcSourceConfig sourceConfig; + private final JdbcDataSourceDialect dialect; - public ParallelSourceReader( + public JdbcIncrementalSourceReader( FutureCompletingBlockingQueue> elementQueue, - Supplier splitReaderSupplier, + Supplier splitReaderSupplier, RecordEmitter recordEmitter, Configuration config, SourceReaderContext context, - SourceConfig sourceConfig, + JdbcSourceConfig sourceConfig, SourceSplitSerializer sourceSplitSerializer, - Dialect dialect) { + JdbcDataSourceDialect dialect) { super( elementQueue, new SingleThreadFetcherManager<>(elementQueue, splitReaderSupplier::get), @@ -186,10 +185,10 @@ private StreamSplit discoverTableSchemasForBinlogSplit(StreamSplit split) { if (split.getTableSchemas().isEmpty()) { try { Map tableSchemas = - dialect.discoverCapturedTableSchemas(sourceConfig); + dialect.discoverDataCollectionSchemas(sourceConfig); LOG.info("The table schema discovery for binlog split {} success", splitId); - return fillTableSchemas(split, tableSchemas); - } catch (SQLException e) { + return StreamSplit.fillTableSchemas(split, tableSchemas); + } catch (Exception e) { LOG.error("Failed to obtains table schemas due to {}", e.getMessage()); throw new FlinkRuntimeException(e); } @@ -240,15 +239,7 @@ private void fillMetaDataForBinlogSplit(StreamSplitMetaEvent metadataEvent) { if (receivedMetaGroupId == expectedMetaGroupId) { List metaDataGroup = metadataEvent.getMetaGroup().stream() - // - // .map(FinishedSnapshotSplitInfo::deserialize) - .map( - new Function() { - @Override - public FinishedSnapshotSplitInfo apply(byte[] bytes) { - return sourceSplitSerializer.deserialize(bytes); - } - }) + .map(bytes -> sourceSplitSerializer.deserialize(bytes)) .collect(Collectors.toList()); uncompletedBinlogSplits.put( binlogSplit.splitId(), diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/BaseSplitReader.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/JdbcSourceSplitReader.java similarity index 59% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/BaseSplitReader.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/JdbcSourceSplitReader.java index d84a415862a..34f2a0d9a0b 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/BaseSplitReader.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/JdbcSourceSplitReader.java @@ -18,19 +18,19 @@ package com.ververica.cdc.connectors.base.source.reader; +import org.apache.flink.annotation.Experimental; import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; import org.apache.flink.connector.base.source.reader.splitreader.SplitsAddition; import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange; -import com.ververica.cdc.connectors.base.source.config.SourceConfig; -import com.ververica.cdc.connectors.base.source.dialect.SnapshotEventDialect; -import com.ververica.cdc.connectors.base.source.dialect.StreamingEventDialect; -import com.ververica.cdc.connectors.base.source.reader.split.Reader; -import com.ververica.cdc.connectors.base.source.reader.split.SnapshotReader; -import com.ververica.cdc.connectors.base.source.reader.split.StreamReader; -import com.ververica.cdc.connectors.base.source.split.ChangeEventRecords; -import com.ververica.cdc.connectors.base.source.split.SourceSplitBase; +import com.ververica.cdc.connectors.base.dialect.JdbcDataSourceDialect; +import com.ververica.cdc.connectors.base.source.meta.split.ChangeEventRecords; +import com.ververica.cdc.connectors.base.source.meta.split.SourceSplitBase; +import com.ververica.cdc.connectors.base.source.reader.external.Fetcher; +import com.ververica.cdc.connectors.base.source.reader.external.JdbcSourceFetchTaskContext; +import com.ververica.cdc.connectors.base.source.reader.external.JdbcSourceScanFetcher; +import com.ververica.cdc.connectors.base.source.reader.external.JdbcSourceStreamFetcher; import org.apache.kafka.connect.source.SourceRecord; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -42,29 +42,22 @@ import java.util.Iterator; import java.util.Queue; -/** basic class read {@link SourceSplitBase} and return {@link SourceRecord}. */ -public class BaseSplitReader implements SplitReader { +/** Basic class read {@link SourceSplitBase} and return {@link SourceRecord}. */ +@Experimental +public class JdbcSourceSplitReader implements SplitReader { - private static final Logger LOG = LoggerFactory.getLogger(BaseSplitReader.class); + private static final Logger LOG = LoggerFactory.getLogger(JdbcSourceSplitReader.class); private final Queue splits; - private final SourceConfig sourceConfig; private final int subtaskId; - @Nullable private Reader currentReader; + @Nullable private Fetcher currenFetcher; @Nullable private String currentSplitId; - private SnapshotEventDialect snapshotEventDialect; - private StreamingEventDialect streamingEventDialect; - - public BaseSplitReader( - SourceConfig sourceConfig, - int subtaskId, - SnapshotEventDialect snapshotEventDialect, - StreamingEventDialect streamingEventDialect) { - this.sourceConfig = sourceConfig; + private JdbcDataSourceDialect dataSourceDialect; + + public JdbcSourceSplitReader(int subtaskId, JdbcDataSourceDialect dataSourceDialect) { this.subtaskId = subtaskId; this.splits = new ArrayDeque<>(); - this.snapshotEventDialect = snapshotEventDialect; - this.streamingEventDialect = streamingEventDialect; + this.dataSourceDialect = dataSourceDialect; } @Override @@ -72,7 +65,7 @@ public RecordsWithSplitIds fetch() throws IOException { checkSplitOrStartNext(); Iterator dataIt = null; try { - dataIt = currentReader.pollSplitRecords(); + dataIt = currenFetcher.pollSplitRecords(); } catch (InterruptedException e) { LOG.warn("fetch data failed.", e); throw new IOException(e); @@ -100,18 +93,16 @@ public void wakeUp() {} @Override public void close() throws Exception { - if (currentReader != null) { - LOG.info( - "Close current debezium reader {}", - currentReader.getClass().getCanonicalName()); - currentReader.close(); + if (currenFetcher != null) { + LOG.info("Close current fetcher {}", currenFetcher.getClass().getCanonicalName()); + currenFetcher.close(); currentSplitId = null; } } protected void checkSplitOrStartNext() throws IOException { // the binlog reader should keep alive - if (currentReader instanceof StreamReader) { + if (currenFetcher instanceof JdbcSourceStreamFetcher) { return; } @@ -123,32 +114,28 @@ protected void checkSplitOrStartNext() throws IOException { currentSplitId = nextSplit.splitId(); if (nextSplit.isSnapshotSplit()) { - if (currentReader == null) { - SnapshotEventDialect.SnapshotContext snapshotContext = - snapshotEventDialect.createSnapshotContext(sourceConfig); - currentReader = - new SnapshotReader( - snapshotContext, - subtaskId, - snapshotEventDialect, - streamingEventDialect); + if (currenFetcher == null) { + final JdbcSourceFetchTaskContext taskContext = + dataSourceDialect.createFetchTaskContext(nextSplit); + currenFetcher = new JdbcSourceScanFetcher(taskContext, subtaskId); } } else { // point from snapshot split to binlog split - if (currentReader != null) { - LOG.info("It's turn to read binlog split, close current snapshot reader."); - currentReader.close(); + if (currenFetcher != null) { + LOG.info("It's turn to read binlog split, close current snapshot fetcher."); + currenFetcher.close(); } - // todo instance a StreamReader. - - LOG.info("StreamReader is created."); + final JdbcSourceFetchTaskContext taskContext = + dataSourceDialect.createFetchTaskContext(nextSplit); + currenFetcher = new JdbcSourceStreamFetcher(taskContext, subtaskId); + LOG.info("Stream fetcher is created."); } - currentReader.submitSplit(nextSplit); + currenFetcher.submitTask(dataSourceDialect.createFetchTask(nextSplit)); } } private boolean canAssignNextSplit() { - return currentReader == null || currentReader.isFinished(); + return currenFetcher == null || currenFetcher.isFinished(); } private ChangeEventRecords finishedSnapshotSplit() { diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/internal/converter/JdbcSourceRecordConverter.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/external/FetchTask.java similarity index 60% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/internal/converter/JdbcSourceRecordConverter.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/external/FetchTask.java index 1a4eb8d53e4..e928c580355 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/internal/converter/JdbcSourceRecordConverter.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/external/FetchTask.java @@ -16,15 +16,23 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.base.source.internal.converter; +package com.ververica.cdc.connectors.base.source.reader.external; -import org.apache.kafka.connect.source.SourceRecord; +import org.apache.flink.annotation.Experimental; -import java.io.Serializable; -import java.sql.ResultSet; +/** The task to fetching data of a Split. */ +@Experimental +public interface FetchTask { -/** The converter that is used to convert JDBC object to {@link SourceRecord}. */ -public interface JdbcSourceRecordConverter extends Serializable { + /** Execute current task. */ + void execute(Context context) throws Exception; - SourceRecord toInternal(ResultSet resultSet); + /** Returns current task is running or not. */ + boolean isRunning(); + + /** Returns the split that the task used. */ + Split getSplit(); + + /** Base context used in the execution of fetch task. */ + interface Context {} } diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/split/Reader.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/external/Fetcher.java similarity index 58% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/split/Reader.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/external/Fetcher.java index 3b47944c798..176745064c3 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/split/Reader.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/external/Fetcher.java @@ -16,40 +16,37 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.base.source.reader.split; +package com.ververica.cdc.connectors.base.source.reader.external; -import com.ververica.cdc.connectors.base.source.split.SnapshotSplit; -import com.ververica.cdc.connectors.base.source.split.StreamSplit; +import org.apache.flink.annotation.Experimental; + +import com.ververica.cdc.connectors.base.source.meta.split.SnapshotSplit; +import com.ververica.cdc.connectors.base.source.meta.split.StreamSplit; import javax.annotation.Nullable; import java.util.Iterator; /** - * Reader to read split of table, the split is either snapshot split {@link SnapshotSplit} or stream - * split {@link StreamSplit}. + * Fetcher to fetch data of a table split, the split is either snapshot split {@link SnapshotSplit} + * or stream split {@link StreamSplit}. */ -public interface Reader { - - /** Return the current split of the reader is finished or not. */ - boolean isFinished(); - - /** - * Add to split to read, this should call only the when reader is idle. - * - * @param splitToRead - */ - void submitSplit(Split splitToRead); +@Experimental +public interface Fetcher { - // void submitSplit(); - - /** Close the reader and releases all resources. */ - void close(); + /** Add to task to fetch, this should call only when the reader is idle. */ + void submitTask(FetchTask fetchTask); /** - * Reads records from Database. The method should return null when reaching the end of the + * Fetched records from data source. The method should return null when reaching the end of the * split, the empty {@link Iterator} will be returned if the data of split is on pulling. */ @Nullable Iterator pollSplitRecords() throws InterruptedException; + + /** Return the current fetch task is finished or not. */ + boolean isFinished(); + + /** Close the client and releases all resources. */ + void close(); } diff --git a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/external/JdbcSourceFetchTaskContext.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/external/JdbcSourceFetchTaskContext.java new file mode 100644 index 00000000000..deb45b65400 --- /dev/null +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/external/JdbcSourceFetchTaskContext.java @@ -0,0 +1,86 @@ +/* + * 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 com.ververica.cdc.connectors.base.source.reader.external; + +import org.apache.flink.table.types.logical.RowType; + +import com.ververica.cdc.connectors.base.config.JdbcSourceConfig; +import com.ververica.cdc.connectors.base.config.SourceConfig; +import com.ververica.cdc.connectors.base.dialect.JdbcDataSourceDialect; +import com.ververica.cdc.connectors.base.relational.JdbcSourceEventDispatcher; +import com.ververica.cdc.connectors.base.source.meta.offset.Offset; +import com.ververica.cdc.connectors.base.source.meta.split.SourceSplitBase; +import io.debezium.config.CommonConnectorConfig; +import io.debezium.connector.base.ChangeEventQueue; +import io.debezium.pipeline.DataChangeEvent; +import io.debezium.pipeline.ErrorHandler; +import io.debezium.pipeline.spi.OffsetContext; +import io.debezium.relational.RelationalDatabaseSchema; +import io.debezium.relational.Table; +import io.debezium.util.SchemaNameAdjuster; +import org.apache.kafka.connect.source.SourceRecord; + +/** The context for fetch task that fetching data of snapshot split from JDBC data source. */ +public abstract class JdbcSourceFetchTaskContext implements FetchTask.Context { + + protected final JdbcSourceConfig sourceConfig; + protected final JdbcDataSourceDialect dataSourceDialect; + protected final CommonConnectorConfig dbzConnectorConfig; + protected final SchemaNameAdjuster schemaNameAdjuster; + + public JdbcSourceFetchTaskContext( + JdbcSourceConfig sourceConfig, JdbcDataSourceDialect dataSourceDialect) { + this.sourceConfig = sourceConfig; + this.dataSourceDialect = dataSourceDialect; + this.dbzConnectorConfig = sourceConfig.getDbzConnectorConfig(); + this.schemaNameAdjuster = SchemaNameAdjuster.create(); + } + + public abstract void configure(SourceSplitBase sourceSplitBase); + + public SourceConfig getSourceConfig() { + return sourceConfig; + } + + public JdbcDataSourceDialect getDataSourceDialect() { + return dataSourceDialect; + } + + public CommonConnectorConfig getDbzConnectorConfig() { + return dbzConnectorConfig; + } + + public SchemaNameAdjuster getSchemaNameAdjuster() { + return null; + } + + public abstract RelationalDatabaseSchema getDatabaseSchema(); + + public abstract RowType getSplitType(Table table); + + public abstract ErrorHandler getErrorHandler(); + + public abstract JdbcSourceEventDispatcher getDispatcher(); + + public abstract OffsetContext getOffsetContext(); + + public abstract ChangeEventQueue getQueue(); + + public abstract Offset getStreamOffset(SourceRecord sourceRecord); +} diff --git a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/external/JdbcSourceScanFetcher.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/external/JdbcSourceScanFetcher.java new file mode 100644 index 00000000000..6ac8cf6e51f --- /dev/null +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/external/JdbcSourceScanFetcher.java @@ -0,0 +1,155 @@ +/* + * 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 com.ververica.cdc.connectors.base.source.reader.external; + +import org.apache.flink.util.FlinkRuntimeException; + +import org.apache.flink.shaded.guava18.com.google.common.util.concurrent.ThreadFactoryBuilder; + +import com.ververica.cdc.connectors.base.source.meta.split.SnapshotSplit; +import com.ververica.cdc.connectors.base.source.meta.split.SourceSplitBase; +import com.ververica.cdc.connectors.base.utils.SourceRecordUtils; +import io.debezium.connector.base.ChangeEventQueue; +import io.debezium.pipeline.DataChangeEvent; +import io.debezium.util.SchemaNameAdjuster; +import org.apache.kafka.connect.source.SourceRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * Fetcher to fetch data from table split, the split is the snapshot split {@link SnapshotSplit}. + */ +public class JdbcSourceScanFetcher implements Fetcher { + + private static final Logger LOG = LoggerFactory.getLogger(JdbcSourceScanFetcher.class); + + public AtomicBoolean hasNextElement; + public AtomicBoolean reachEnd; + + private final JdbcSourceFetchTaskContext taskContext; + private final ExecutorService executor; + private volatile ChangeEventQueue queue; + private volatile Throwable readException; + + // task to read snapshot for current split + private FetchTask snapshotSplitReadTask; + private SnapshotSplit currentSnapshotSplit; + + public JdbcSourceScanFetcher(JdbcSourceFetchTaskContext taskContext, int subtaskId) { + this.taskContext = taskContext; + ThreadFactory threadFactory = + new ThreadFactoryBuilder() + .setNameFormat("debezium-snapshot-reader-" + subtaskId) + .build(); + this.executor = Executors.newSingleThreadExecutor(threadFactory); + this.hasNextElement = new AtomicBoolean(false); + this.reachEnd = new AtomicBoolean(false); + } + + @Override + public void submitTask(FetchTask fetchTask) { + this.snapshotSplitReadTask = fetchTask; + this.currentSnapshotSplit = fetchTask.getSplit().asSnapshotSplit(); + taskContext.configure(currentSnapshotSplit); + this.queue = taskContext.getQueue(); + this.hasNextElement.set(true); + this.reachEnd.set(false); + executor.submit( + () -> { + try { + snapshotSplitReadTask.execute(taskContext); + } catch (Exception e) { + LOG.error( + String.format( + "Execute snapshot read task for mysql split %s fail", + currentSnapshotSplit), + e); + readException = e; + } + }); + } + + @Override + public boolean isFinished() { + return currentSnapshotSplit == null + || (!snapshotSplitReadTask.isRunning() && !hasNextElement.get() && reachEnd.get()); + } + + @Nullable + @Override + public Iterator pollSplitRecords() throws InterruptedException { + checkReadException(); + + if (hasNextElement.get()) { + // data input: [low watermark event][snapshot events][high watermark event][binlog + // events][binlog-end event] + // data output: [low watermark event][normalized events][high watermark event] + boolean reachBinlogEnd = false; + final List sourceRecords = new ArrayList<>(); + while (!reachBinlogEnd) { + List batch = queue.poll(); + for (DataChangeEvent event : batch) { + sourceRecords.add(event.getRecord()); + if (SourceRecordUtils.isEndWatermarkEvent(event.getRecord())) { + reachBinlogEnd = true; + break; + } + } + } + // snapshot split return its data once + hasNextElement.set(false); + return SourceRecordUtils.normalizedSplitRecords( + currentSnapshotSplit, + sourceRecords, + new SchemaNameAdjuster() { + @Override + public String adjust(String proposedName) { + return null; + } + }) + .iterator(); + } + // the data has been polled, no more data + reachEnd.compareAndSet(false, true); + return null; + } + + private void checkReadException() { + if (readException != null) { + throw new FlinkRuntimeException( + String.format( + "Read split %s error due to %s.", + currentSnapshotSplit, readException.getMessage()), + readException); + } + } + + @Override + public void close() {} +} diff --git a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/external/JdbcSourceStreamFetcher.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/external/JdbcSourceStreamFetcher.java new file mode 100644 index 00000000000..7eb61b58176 --- /dev/null +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/external/JdbcSourceStreamFetcher.java @@ -0,0 +1,220 @@ +/* + * 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 com.ververica.cdc.connectors.base.source.reader.external; + +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.util.FlinkRuntimeException; + +import org.apache.flink.shaded.guava18.com.google.common.util.concurrent.ThreadFactoryBuilder; + +import com.ververica.cdc.connectors.base.source.meta.offset.Offset; +import com.ververica.cdc.connectors.base.source.meta.split.FinishedSnapshotSplitInfo; +import com.ververica.cdc.connectors.base.source.meta.split.SourceSplitBase; +import com.ververica.cdc.connectors.base.source.meta.split.StreamSplit; +import com.ververica.cdc.connectors.base.utils.SourceRecordUtils; +import io.debezium.connector.base.ChangeEventQueue; +import io.debezium.pipeline.DataChangeEvent; +import io.debezium.relational.TableId; +import io.debezium.relational.Tables; +import org.apache.kafka.connect.source.SourceRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ThreadFactory; + +import static com.ververica.cdc.connectors.base.utils.SourceRecordUtils.getTableId; +import static com.ververica.cdc.connectors.base.utils.SourceRecordUtils.isDataChangeRecord; +import static com.ververica.cdc.connectors.base.utils.SourceRecordUtils.splitKeyRangeContains; + +/** Fetcher to fetch data from table split, the split is the stream split {@link StreamSplit}. */ +public class JdbcSourceStreamFetcher implements Fetcher { + private static final Logger LOG = LoggerFactory.getLogger(JdbcSourceStreamFetcher.class); + + private final JdbcSourceFetchTaskContext taskContext; + private final ExecutorService executor; + + private volatile ChangeEventQueue queue; + private volatile Throwable readException; + + private FetchTask streamFetchTask; + private StreamSplit currentStreamSplit; + private Map> finishedSplitsInfo; + // tableId -> the max splitHighWatermark + private Map maxSplitHighWatermarkMap; + private Tables.TableFilter capturedTableFilter; + + public JdbcSourceStreamFetcher(JdbcSourceFetchTaskContext taskContext, int subTaskId) { + this.taskContext = taskContext; + ThreadFactory threadFactory = + new ThreadFactoryBuilder().setNameFormat("debezium-reader-" + subTaskId).build(); + this.executor = Executors.newSingleThreadExecutor(threadFactory); + } + + public void submitTask(FetchTask fetchTask) { + this.streamFetchTask = fetchTask; + this.currentStreamSplit = fetchTask.getSplit().asStreamSplit(); + taskContext.configure(currentStreamSplit); + this.queue = taskContext.getQueue(); + executor.submit( + () -> { + try { + streamFetchTask.execute(taskContext); + } catch (Exception e) { + LOG.error( + String.format( + "Execute binlog read task for mysql split %s fail", + currentStreamSplit), + e); + readException = e; + } + }); + } + + @Override + public boolean isFinished() { + return currentStreamSplit == null || !streamFetchTask.isRunning(); + } + + @Nullable + @Override + public Iterator pollSplitRecords() throws InterruptedException { + checkReadException(); + final List sourceRecords = new ArrayList<>(); + if (streamFetchTask.isRunning()) { + List batch = queue.poll(); + for (DataChangeEvent event : batch) { + if (shouldEmit(event.getRecord())) { + sourceRecords.add(event.getRecord()); + } + } + return sourceRecords.iterator(); + } else { + return null; + } + } + + private void checkReadException() { + if (readException != null) { + throw new FlinkRuntimeException( + String.format( + "Read split %s error due to %s.", + currentStreamSplit, readException.getMessage()), + readException); + } + } + + @Override + public void close() {} + + /** + * Returns the record should emit or not. + * + *

The watermark signal algorithm is the binlog split reader only sends the binlog event that + * belongs to its finished snapshot splits. For each snapshot split, the binlog event is valid + * since the offset is after its high watermark. + * + *

 E.g: the data input is :
+     *    snapshot-split-0 info : [0,    1024) highWatermark0
+     *    snapshot-split-1 info : [1024, 2048) highWatermark1
+     *  the data output is:
+     *  only the binlog event belong to [0,    1024) and offset is after highWatermark0 should send,
+     *  only the binlog event belong to [1024, 2048) and offset is after highWatermark1 should send.
+     * 
+ */ + private boolean shouldEmit(SourceRecord sourceRecord) { + if (isDataChangeRecord(sourceRecord)) { + TableId tableId = getTableId(sourceRecord); + Offset position = taskContext.getStreamOffset(sourceRecord); + if (hasEnterPureBinlogPhase(tableId, position)) { + return true; + } + // only the table who captured snapshot splits need to filter + if (finishedSplitsInfo.containsKey(tableId)) { + RowType splitKeyType = + taskContext.getSplitType(taskContext.getDatabaseSchema().tableFor(tableId)); + Object[] key = + SourceRecordUtils.getSplitKey( + splitKeyType, sourceRecord, taskContext.getSchemaNameAdjuster()); + for (FinishedSnapshotSplitInfo splitInfo : finishedSplitsInfo.get(tableId)) { + if (splitKeyRangeContains( + key, splitInfo.getSplitStart(), splitInfo.getSplitEnd()) + && position.isAfter(splitInfo.getHighWatermark())) { + return true; + } + } + } + // not in the monitored splits scope, do not emit + return false; + } + // always send the schema change event and signal event + // we need record them to state of Flink + return true; + } + + private boolean hasEnterPureBinlogPhase(TableId tableId, Offset position) { + // the existed tables those have finished snapshot reading + if (maxSplitHighWatermarkMap.containsKey(tableId) + && position.isAtOrAfter(maxSplitHighWatermarkMap.get(tableId))) { + return true; + } + + return !maxSplitHighWatermarkMap.containsKey(tableId) + && capturedTableFilter.isIncluded(tableId); + } + + private void configureFilter() { + List finishedSplitInfos = + currentStreamSplit.getFinishedSnapshotSplitInfos(); + Map> splitsInfoMap = new HashMap<>(); + Map tableIdBinlogPositionMap = new HashMap<>(); + // latest-offset mode + if (finishedSplitInfos.isEmpty()) { + for (TableId tableId : currentStreamSplit.getTableSchemas().keySet()) { + tableIdBinlogPositionMap.put(tableId, currentStreamSplit.getStartingOffset()); + } + } + // initial mode + else { + for (FinishedSnapshotSplitInfo finishedSplitInfo : finishedSplitInfos) { + TableId tableId = finishedSplitInfo.getTableId(); + List list = + splitsInfoMap.getOrDefault(tableId, new ArrayList<>()); + list.add(finishedSplitInfo); + splitsInfoMap.put(tableId, list); + + Offset highWatermark = finishedSplitInfo.getHighWatermark(); + Offset maxHighWatermark = tableIdBinlogPositionMap.get(tableId); + if (maxHighWatermark == null || highWatermark.isAfter(maxHighWatermark)) { + tableIdBinlogPositionMap.put(tableId, highWatermark); + } + } + } + this.finishedSplitsInfo = splitsInfoMap; + this.maxSplitHighWatermarkMap = tableIdBinlogPositionMap; + } +} diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/utils/ObjectUtils.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/utils/ObjectUtils.java similarity index 98% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/utils/ObjectUtils.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/utils/ObjectUtils.java index 68bf23412ac..adc2face0eb 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/utils/ObjectUtils.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/utils/ObjectUtils.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.base.source.utils; +package com.ververica.cdc.connectors.base.utils; import java.math.BigDecimal; import java.math.BigInteger; diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/utils/SerializerUtils.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/utils/SerializerUtils.java similarity index 98% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/utils/SerializerUtils.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/utils/SerializerUtils.java index a11350c6f92..b1a9989fde7 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/utils/SerializerUtils.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/utils/SerializerUtils.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.base.source.utils; +package com.ververica.cdc.connectors.base.utils; import io.debezium.DebeziumException; import io.debezium.util.HexConverter; diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/utils/RecordUtils.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/utils/SourceRecordUtils.java similarity index 50% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/utils/RecordUtils.java rename to flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/utils/SourceRecordUtils.java index 932f0e42eac..f670bd807d9 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/utils/RecordUtils.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/utils/SourceRecordUtils.java @@ -16,11 +16,12 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.base.source.utils; +package com.ververica.cdc.connectors.base.utils; import org.apache.flink.table.types.logical.RowType; -import com.ververica.cdc.connectors.base.source.reader.split.dispatcher.SignalEventDispatcher.WatermarkKind; +import com.ververica.cdc.connectors.base.relational.JdbcSourceEventDispatcher.WatermarkKind; +import com.ververica.cdc.connectors.base.source.meta.split.SnapshotSplit; import io.debezium.data.Envelope; import io.debezium.document.DocumentReader; import io.debezium.relational.TableId; @@ -33,19 +34,27 @@ import java.io.IOException; import java.sql.ResultSet; import java.sql.SQLException; +import java.time.Instant; +import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import java.util.Optional; +import java.util.stream.Collectors; -import static com.ververica.cdc.connectors.base.source.reader.split.dispatcher.SignalEventDispatcher.HISTORY_RECORD_FIELD; -import static com.ververica.cdc.connectors.base.source.reader.split.dispatcher.SignalEventDispatcher.SIGNAL_EVENT_VALUE_SCHEMA_NAME; -import static com.ververica.cdc.connectors.base.source.reader.split.dispatcher.SignalEventDispatcher.WATERMARK_KIND; +import static com.ververica.cdc.connectors.base.relational.JdbcSourceEventDispatcher.HISTORY_RECORD_FIELD; +import static com.ververica.cdc.connectors.base.relational.JdbcSourceEventDispatcher.SIGNAL_EVENT_VALUE_SCHEMA_NAME; +import static com.ververica.cdc.connectors.base.relational.JdbcSourceEventDispatcher.WATERMARK_KIND; import static io.debezium.connector.AbstractSourceInfo.DATABASE_NAME_KEY; import static io.debezium.connector.AbstractSourceInfo.TABLE_NAME_KEY; +import static org.apache.flink.util.Preconditions.checkState; /** Utility class to deal record. */ -public class RecordUtils { +public class SourceRecordUtils { - private RecordUtils() {} + private SourceRecordUtils() {} public static final String SCHEMA_CHANGE_EVENT_KEY_NAME = "io.debezium.connector.mysql.SchemaChangeKey"; @@ -214,4 +223,152 @@ private static Optional getWatermarkKind(SourceRecord record) { } return Optional.empty(); } + + /** + * Normalize the records of snapshot split which represents the split records state on high + * watermark. data input: [low watermark event] [snapshot events ] [high watermark event] + * [binlog events] [binlog-end event] data output: [low watermark event] [normalized events] + * [high watermark event] + */ + public static List normalizedSplitRecords( + SnapshotSplit snapshotSplit, + List sourceRecords, + SchemaNameAdjuster nameAdjuster) { + List normalizedRecords = new ArrayList<>(); + Map snapshotRecords = new HashMap<>(); + List binlogRecords = new ArrayList<>(); + if (!sourceRecords.isEmpty()) { + + SourceRecord lowWatermark = sourceRecords.get(0); + checkState( + isLowWatermarkEvent(lowWatermark), + String.format( + "The first record should be low watermark signal event, but is %s", + lowWatermark)); + SourceRecord highWatermark = null; + int i = 1; + for (; i < sourceRecords.size(); i++) { + SourceRecord sourceRecord = sourceRecords.get(i); + if (!isHighWatermarkEvent(sourceRecord)) { + snapshotRecords.put((Struct) sourceRecord.key(), sourceRecord); + } else { + highWatermark = sourceRecord; + i++; + break; + } + } + + if (i < sourceRecords.size() - 1) { + List allBinlogRecords = + sourceRecords.subList(i, sourceRecords.size() - 1); + for (SourceRecord binlog : allBinlogRecords) { + if (isDataChangeRecord(binlog)) { + Object[] key = + getSplitKey(snapshotSplit.getSplitKeyType(), binlog, nameAdjuster); + if (splitKeyRangeContains( + key, snapshotSplit.getSplitStart(), snapshotSplit.getSplitEnd())) { + binlogRecords.add(binlog); + } + } + } + } + checkState( + isHighWatermarkEvent(highWatermark), + String.format( + "The last record should be high watermark signal event, but is %s", + highWatermark)); + normalizedRecords = + upsertBinlog(lowWatermark, highWatermark, snapshotRecords, binlogRecords); + } + return normalizedRecords; + } + + private static List upsertBinlog( + SourceRecord lowWatermarkEvent, + SourceRecord highWatermarkEvent, + Map snapshotRecords, + List binlogRecords) { + // upsert binlog events to snapshot events of split + if (!binlogRecords.isEmpty()) { + for (SourceRecord binlog : binlogRecords) { + Struct key = (Struct) binlog.key(); + Struct value = (Struct) binlog.value(); + if (value != null) { + Envelope.Operation operation = + Envelope.Operation.forCode( + value.getString(Envelope.FieldName.OPERATION)); + switch (operation) { + case CREATE: + case UPDATE: + Envelope envelope = Envelope.fromSchema(binlog.valueSchema()); + Struct source = value.getStruct(Envelope.FieldName.SOURCE); + Struct after = value.getStruct(Envelope.FieldName.AFTER); + Instant fetchTs = + Instant.ofEpochMilli( + (Long) source.get(Envelope.FieldName.TIMESTAMP)); + SourceRecord record = + new SourceRecord( + binlog.sourcePartition(), + binlog.sourceOffset(), + binlog.topic(), + binlog.kafkaPartition(), + binlog.keySchema(), + binlog.key(), + binlog.valueSchema(), + envelope.read(after, source, fetchTs)); + snapshotRecords.put(key, record); + break; + case DELETE: + snapshotRecords.remove(key); + break; + case READ: + throw new IllegalStateException( + String.format( + "Binlog record shouldn't use READ operation, the the record is %s.", + binlog)); + } + } + } + } + + final List normalizedRecords = new ArrayList<>(); + normalizedRecords.add(lowWatermarkEvent); + normalizedRecords.addAll(formatMessageTimestamp(snapshotRecords.values())); + normalizedRecords.add(highWatermarkEvent); + + return normalizedRecords; + } + + /** + * Format message timestamp(source.ts_ms) value to 0L for all records read in snapshot phase. + */ + private static List formatMessageTimestamp( + Collection snapshotRecords) { + return snapshotRecords.stream() + .map( + record -> { + Envelope envelope = Envelope.fromSchema(record.valueSchema()); + Struct value = (Struct) record.value(); + Struct updateAfter = value.getStruct(Envelope.FieldName.AFTER); + // set message timestamp (source.ts_ms) to 0L + Struct source = value.getStruct(Envelope.FieldName.SOURCE); + source.put(Envelope.FieldName.TIMESTAMP, 0L); + // extend the fetch timestamp(ts_ms) + Instant fetchTs = + Instant.ofEpochMilli( + value.getInt64(Envelope.FieldName.TIMESTAMP)); + SourceRecord sourceRecord = + new SourceRecord( + record.sourcePartition(), + record.sourceOffset(), + record.topic(), + record.kafkaPartition(), + record.keySchema(), + record.key(), + record.valueSchema(), + envelope.read(updateAfter, source, fetchTs)); + return sourceRecord; + }) + .collect(Collectors.toList()); + } } diff --git a/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/debezium/EmbeddedFlinkDatabaseHistory.java b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/EmbeddedFlinkDatabaseHistory.java similarity index 97% rename from flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/debezium/EmbeddedFlinkDatabaseHistory.java rename to flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/EmbeddedFlinkDatabaseHistory.java index 9e61fecb215..0ea3e7faf19 100644 --- a/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/debezium/EmbeddedFlinkDatabaseHistory.java +++ b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/EmbeddedFlinkDatabaseHistory.java @@ -16,9 +16,9 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.refactor.refactor.debezium; +package com.ververica.cdc.connectors.base.experimental; -import com.ververica.cdc.connectors.base.source.split.SourceSplitState; +import com.ververica.cdc.connectors.base.source.meta.split.SourceSplitState; import io.debezium.config.Configuration; import io.debezium.relational.TableId; import io.debezium.relational.Tables; diff --git a/flink-connector-mysql-cdc-new/src/test/java/com/ververica/cdc/connectors/refactor/MySqlSourceExampleTest.java b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/MySqlChangeEventSourceExampleTest.java similarity index 77% rename from flink-connector-mysql-cdc-new/src/test/java/com/ververica/cdc/connectors/refactor/MySqlSourceExampleTest.java rename to flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/MySqlChangeEventSourceExampleTest.java index e1edfd6381f..7564a4ff095 100644 --- a/flink-connector-mysql-cdc-new/src/test/java/com/ververica/cdc/connectors/refactor/MySqlSourceExampleTest.java +++ b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/MySqlChangeEventSourceExampleTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.refactor; +package com.ververica.cdc.connectors.base.experimental; import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.runtime.minicluster.RpcServiceSharing; @@ -24,13 +24,12 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.test.util.MiniClusterWithClientResource; -import com.ververica.cdc.connectors.refactor.refactor.source.MySqlSource; -import com.ververica.cdc.connectors.refactor.refactor.source.MySqlSourceBuilder; -import com.ververica.cdc.connectors.refactor.testutils.MySqlContainer; -import com.ververica.cdc.connectors.refactor.testutils.MySqlVersion; -import com.ververica.cdc.connectors.refactor.testutils.UniqueDatabase; +import com.ververica.cdc.connectors.base.experimental.testutils.MySqlContainer; +import com.ververica.cdc.connectors.base.experimental.testutils.MySqlVersion; +import com.ververica.cdc.connectors.base.experimental.testutils.UniqueDatabase; import com.ververica.cdc.debezium.JsonDebeziumDeserializationSchema; import org.junit.BeforeClass; +import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; import org.slf4j.Logger; @@ -40,13 +39,16 @@ import java.util.stream.Stream; -/** Example Tests for {@link MySqlSource}. */ -public class MySqlSourceExampleTest { +/** + * Example Tests for {@link com.ververica.cdc.connectors.base.experimental.MySqlIncrementalSource}. + */ +public class MySqlChangeEventSourceExampleTest { - private static final Logger LOG = LoggerFactory.getLogger(MySqlSourceExampleTest.class); + private static final Logger LOG = + LoggerFactory.getLogger(MySqlChangeEventSourceExampleTest.class); - protected static final int DEFAULT_PARALLELISM = 4; - protected static final MySqlContainer MYSQL_CONTAINER = createMySqlContainer(MySqlVersion.V5_7); + private static final int DEFAULT_PARALLELISM = 4; + private static final MySqlContainer MYSQL_CONTAINER = createMySqlContainer(MySqlVersion.V5_7); @Rule public final MiniClusterWithClientResource miniClusterResource = @@ -65,24 +67,14 @@ public static void startContainers() { LOG.info("Containers are started."); } - protected static MySqlContainer createMySqlContainer(MySqlVersion version) { - return (MySqlContainer) - new MySqlContainer(version) - .withConfigurationOverride("docker/server-gtids/my.cnf") - .withSetupSQL("docker/setup.sql") - .withDatabaseName("flink-test") - .withUsername("flinkuser") - .withPassword("flinkpw") - .withLogConsumer(new Slf4jLogConsumer(LOG)); - } - 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 testConsumingAllEvents() throws Exception { inventoryDatabase.createAndInitialize(); - MySqlSource mySqlSource = + MySqlIncrementalSource mySqlChangeEventSource = new MySqlSourceBuilder() .hostname(MYSQL_CONTAINER.getHost()) .port(MYSQL_CONTAINER.getDatabasePort()) @@ -99,11 +91,25 @@ public void testConsumingAllEvents() throws Exception { // enable checkpoint env.enableCheckpointing(3000); // set the source parallelism to 4 - env.fromSource(mySqlSource, WatermarkStrategy.noWatermarks(), "MySqlParallelSource") + env.fromSource( + mySqlChangeEventSource, + WatermarkStrategy.noWatermarks(), + "MySqlParallelSource") .setParallelism(4) .print() .setParallelism(1); env.execute("Print MySQL Snapshot + Binlog"); } + + private static MySqlContainer createMySqlContainer(MySqlVersion version) { + return (MySqlContainer) + 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-base/src/test/java/com/ververica/cdc/connectors/base/experimental/MySqlDialect.java b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/MySqlDialect.java new file mode 100644 index 00000000000..2524175b335 --- /dev/null +++ b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/MySqlDialect.java @@ -0,0 +1,142 @@ +/* + * 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 com.ververica.cdc.connectors.base.experimental; + +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; + +import com.ververica.cdc.connectors.base.config.JdbcSourceConfig; +import com.ververica.cdc.connectors.base.dialect.JdbcDataSourceDialect; +import com.ververica.cdc.connectors.base.experimental.config.MySqlSourceConfigFactory; +import com.ververica.cdc.connectors.base.relational.JdbcSourceEventDispatcher; +import com.ververica.cdc.connectors.base.relational.connection.JdbcConnectionPoolFactory; +import com.ververica.cdc.connectors.base.source.assigner.splitter.ChunkSplitter; +import com.ververica.cdc.connectors.base.source.meta.offset.Offset; +import com.ververica.cdc.connectors.base.source.meta.split.SourceSplitBase; +import io.debezium.jdbc.JdbcConnection; +import io.debezium.pipeline.spi.OffsetContext; +import io.debezium.relational.Column; +import io.debezium.relational.TableId; +import io.debezium.relational.history.TableChanges.TableChange; + +import java.sql.SQLException; +import java.util.List; +import java.util.Map; + +/** The {@link JdbcDataSourceDialect} implementation for MySQL datasource. */ +public class MySqlDialect implements JdbcDataSourceDialect { + + private final MySqlSourceConfigFactory configFactory; + + public MySqlDialect(MySqlSourceConfigFactory configFactory) { + this.configFactory = configFactory; + } + + @Override + public String getName() { + return null; + } + + @Override + public Offset displayCurrentOffset(JdbcSourceConfig sourceConfig) { + return null; + } + + @Override + public boolean isDataCollectionIdCaseSensitive(JdbcSourceConfig sourceConfig) { + return false; + } + + @Override + public ChunkSplitter createChunkSplitter(JdbcSourceConfig sourceConfig) { + return null; + } + + @Override + public JdbcConnectionPoolFactory getPooledDataSourceFactory() { + return null; + } + + @Override + public List discoverDataCollections(JdbcSourceConfig sourceConfig) { + return null; + } + + @Override + public Map discoverDataCollectionSchemas(JdbcSourceConfig sourceConfig) { + return null; + } + + @Override + public OffsetContext getOffsetContext( + JdbcSourceConfig sourceConfig, SourceSplitBase splitBase) { + return null; + } + + @Override + public JdbcSourceEventDispatcher getEventDispatcher( + JdbcSourceConfig sourceConfig, SourceSplitBase splitBase) { + return null; + } + + @Override + public String buildSplitScanQuery( + TableId tableId, RowType splitKeyType, boolean isFirstSplit, boolean isLastSplit) { + return null; + } + + @Override + public Object[] queryMinMax(JdbcConnection jdbc, TableId tableId, String columnName) + throws SQLException { + return new Object[0]; + } + + @Override + public Object queryMin( + JdbcConnection jdbc, TableId tableId, String columnName, Object excludedLowerBound) + throws SQLException { + return null; + } + + @Override + public Object queryNextChunkMax( + JdbcConnection jdbc, + TableId tableId, + String columnName, + int chunkSize, + Object includedLowerBound) + throws SQLException { + return null; + } + + @Override + public Long queryApproximateRowCnt(JdbcConnection jdbc, TableId tableId) throws SQLException { + return null; + } + + @Override + public TableChange queryTableSchema(JdbcConnection jdbc, TableId tableId) { + return null; + } + + @Override + public DataType fromDbzColumn(Column splitColumn) { + return null; + } +} diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/internal/converter/AbstractJdbcSourceRecordConverter.java b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/MySqlIncrementalSource.java similarity index 50% rename from flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/internal/converter/AbstractJdbcSourceRecordConverter.java rename to flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/MySqlIncrementalSource.java index e45ff60009f..6170f941039 100644 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/internal/converter/AbstractJdbcSourceRecordConverter.java +++ b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/MySqlIncrementalSource.java @@ -16,26 +16,21 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.base.source.internal.converter; +package com.ververica.cdc.connectors.base.experimental; -import org.apache.kafka.connect.source.SourceRecord; +import com.ververica.cdc.connectors.base.experimental.config.MySqlSourceConfigFactory; +import com.ververica.cdc.connectors.base.experimental.offset.BinlogOffsetFactory; +import com.ververica.cdc.connectors.base.source.JdbcIncrementalSource; +import com.ververica.cdc.debezium.DebeziumDeserializationSchema; -import java.sql.ResultSet; +/** A MySql CDC Connector Source. */ +public class MySqlIncrementalSource extends JdbcIncrementalSource { -/** basic class for all converters that convert JDBC object to {@link SourceRecord}. */ -public abstract class AbstractJdbcSourceRecordConverter implements JdbcSourceRecordConverter { - - @Override - public SourceRecord toInternal(ResultSet resultSet) { - - // todo get sourceRecord - // SourceRecord record = new SourceRecord( - // offsetContext.getPartition(), - // offsetContext.getOffset(), - // topicName, null, - // keySchema, key, - // dataCollectionSchema.getEnvelopeSchema().schema(), value, - // null, headers); - return null; + public MySqlIncrementalSource( + MySqlSourceConfigFactory configFactory, + DebeziumDeserializationSchema deserializationSchema, + BinlogOffsetFactory offsetFactory, + MySqlDialect dataSourceDialect) { + super(configFactory, deserializationSchema, offsetFactory, dataSourceDialect); } } diff --git a/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/source/MySqlSourceBuilder.java b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/MySqlSourceBuilder.java similarity index 78% rename from flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/source/MySqlSourceBuilder.java rename to flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/MySqlSourceBuilder.java index d3a18c7a7c3..dabc74b3c84 100644 --- a/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/source/MySqlSourceBuilder.java +++ b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/MySqlSourceBuilder.java @@ -16,18 +16,14 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.refactor.refactor.source; +package com.ververica.cdc.connectors.base.experimental; import org.apache.flink.annotation.PublicEvolving; -import com.ververica.cdc.connectors.base.schema.BaseSchema; -import com.ververica.cdc.connectors.base.source.config.StartupOptions; -import com.ververica.cdc.connectors.base.source.dialect.SnapshotEventDialect; -import com.ververica.cdc.connectors.base.source.dialect.StreamingEventDialect; -import com.ververica.cdc.connectors.base.source.offset.OffsetFactory; -import com.ververica.cdc.connectors.refactor.refactor.source.config.MySqlSourceConfigFactory; +import com.ververica.cdc.connectors.base.experimental.config.MySqlSourceConfigFactory; +import com.ververica.cdc.connectors.base.experimental.offset.BinlogOffsetFactory; +import com.ververica.cdc.connectors.base.options.StartupOptions; import com.ververica.cdc.debezium.DebeziumDeserializationSchema; -import com.ververica.cdc.debezium.Validator; import java.time.Duration; import java.util.Properties; @@ -35,11 +31,11 @@ import static org.apache.flink.util.Preconditions.checkNotNull; /** - * The builder class for {@link MySqlSource} to make it easier for the users to construct a {@link - * MySqlSource}. + * The builder class for {@link MySqlIncrementalSource} to make it easier for the users to construct + * a {@link MySqlIncrementalSource}. * *
{@code
- * MySqlSource
+ * MySqlIncrementalSource
  *     .builder()
  *     .hostname("localhost")
  *     .port(3306)
@@ -53,17 +49,14 @@
  * }
* *

Check the Java docs of each individual method to learn more about the settings to build a - * {@link MySqlSource}. + * {@link MySqlIncrementalSource}. */ @PublicEvolving public class MySqlSourceBuilder { private final MySqlSourceConfigFactory configFactory = new MySqlSourceConfigFactory(); + private BinlogOffsetFactory offsetFactory; + private MySqlDialect dialect; private DebeziumDeserializationSchema deserializer; - private Validator validator; - private OffsetFactory offsetFactory; - private SnapshotEventDialect snapshotEventDialect; - private StreamingEventDialect streamingEventDialect; - private BaseSchema baseSchema; public MySqlSourceBuilder hostname(String hostname) { this.configFactory.hostname(hostname); @@ -194,7 +187,7 @@ public MySqlSourceBuilder connectionPoolSize(int connectionPoolSize) { return this; } - /** Whether the {@link MySqlSource} should output the schema changes or not. */ + /** Whether the {@link MySqlIncrementalSource} should output the schema changes or not. */ public MySqlSourceBuilder includeSchemaChanges(boolean includeSchemaChanges) { this.configFactory.includeSchemaChanges(includeSchemaChanges); return this; @@ -221,45 +214,15 @@ public MySqlSourceBuilder deserializer(DebeziumDeserializationSchema deser return this; } - public MySqlSourceBuilder validator(Validator validator) { - this.validator = validator; - return this; - } - - public MySqlSourceBuilder offsetFactory(OffsetFactory offsetFactory) { - this.offsetFactory = offsetFactory; - return this; - } - - public MySqlSourceBuilder snapshotEventDialect(SnapshotEventDialect snapshotEventDialect) { - this.snapshotEventDialect = snapshotEventDialect; - return this; - } - - public MySqlSourceBuilder streamingEventDialect( - StreamingEventDialect streamingEventDialect) { - this.streamingEventDialect = streamingEventDialect; - return this; - } - - public MySqlSourceBuilder baseSchema(BaseSchema baseSchema) { - this.baseSchema = baseSchema; - return this; - } - /** - * Build the {@link MySqlSource}. + * Build the {@link MySqlIncrementalSource}. * * @return a MySqlParallelSource with the settings made for this builder. */ - public MySqlSource build() { - return new MySqlSource<>( - configFactory, - checkNotNull(deserializer), - offsetFactory, - snapshotEventDialect, - streamingEventDialect, - validator, - baseSchema); + public MySqlIncrementalSource build() { + this.offsetFactory = new BinlogOffsetFactory(); + this.dialect = new MySqlDialect(configFactory); + return new MySqlIncrementalSource<>( + configFactory, checkNotNull(deserializer), offsetFactory, dialect); } } diff --git a/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/source/connection/MysqlPooledDataSourceFactory.java b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/MysqlPooledDataSourceFactory.java similarity index 77% rename from flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/source/connection/MysqlPooledDataSourceFactory.java rename to flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/MysqlPooledDataSourceFactory.java index 0cbca0afa77..2d8391f5183 100644 --- a/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/source/connection/MysqlPooledDataSourceFactory.java +++ b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/MysqlPooledDataSourceFactory.java @@ -16,19 +16,19 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.refactor.refactor.source.connection; +package com.ververica.cdc.connectors.base.experimental; -import com.ververica.cdc.connectors.base.source.config.SourceConfig; -import com.ververica.cdc.connectors.base.source.internal.connection.PooledDataSourceFactory; +import com.ververica.cdc.connectors.base.config.JdbcSourceConfig; +import com.ververica.cdc.connectors.base.relational.connection.JdbcConnectionPoolFactory; /** A Mysql datasource factory. */ -public class MysqlPooledDataSourceFactory extends PooledDataSourceFactory { +public class MysqlPooledDataSourceFactory 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"; @Override - public String getJdbcUrl(SourceConfig sourceConfig) { + public String getJdbcUrl(JdbcSourceConfig sourceConfig) { String hostName = sourceConfig.getHostname(); int port = sourceConfig.getPort(); diff --git a/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/source/config/MySqlSourceConfig.java b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/config/MySqlSourceConfig.java similarity index 83% rename from flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/source/config/MySqlSourceConfig.java rename to flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/config/MySqlSourceConfig.java index d9580d72512..f8409f9eaf4 100644 --- a/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/source/config/MySqlSourceConfig.java +++ b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/config/MySqlSourceConfig.java @@ -16,10 +16,10 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.refactor.refactor.source.config; +package com.ververica.cdc.connectors.base.experimental.config; -import com.ververica.cdc.connectors.base.source.config.SourceConfig; -import com.ververica.cdc.connectors.base.source.config.StartupOptions; +import com.ververica.cdc.connectors.base.config.JdbcSourceConfig; +import com.ververica.cdc.connectors.base.options.StartupOptions; import io.debezium.config.Configuration; import io.debezium.connector.mysql.MySqlConnectorConfig; import io.debezium.relational.RelationalTableFilters; @@ -32,56 +32,59 @@ * 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 SourceConfig { +public class MySqlSourceConfig extends JdbcSourceConfig { + + private static final long serialVersionUID = 1L; private final MySqlConnectorConfig dbzMySqlConfig; public MySqlSourceConfig( + StartupOptions startupOptions, + List databaseList, + List tableList, + int splitSize, + int splitMetaGroupSize, + double distributionFactorUpper, + double distributionFactorLower, + boolean includeSchemaChanges, + Properties dbzProperties, + Configuration dbzConfiguration, String driverClassName, String hostname, int port, String username, String password, - List databaseList, - List tableList, - StartupOptions startupOptions, - int splitSize, - int splitMetaGroupSize, int fetchSize, String serverTimeZone, Duration connectTimeout, int connectMaxRetries, - int connectionPoolSize, - double distributionFactorUpper, - double distributionFactorLower, - boolean includeSchemaChanges, - Properties dbzProperties, - Configuration dbzConfiguration) { + int connectionPoolSize) { super( + startupOptions, + databaseList, + tableList, + splitSize, + splitMetaGroupSize, + distributionFactorUpper, + distributionFactorLower, + includeSchemaChanges, + dbzProperties, + dbzConfiguration, driverClassName, hostname, port, username, password, - databaseList, - tableList, - startupOptions, - splitSize, - splitMetaGroupSize, fetchSize, serverTimeZone, connectTimeout, connectMaxRetries, - connectionPoolSize, - distributionFactorUpper, - distributionFactorLower, - includeSchemaChanges, - dbzProperties, - dbzConfiguration); + connectionPoolSize); this.dbzMySqlConfig = new MySqlConnectorConfig(dbzConfiguration); } - public MySqlConnectorConfig getMySqlConnectorConfig() { + @Override + public MySqlConnectorConfig getDbzConnectorConfig() { return dbzMySqlConfig; } diff --git a/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/source/config/MySqlSourceConfigFactory.java b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/config/MySqlSourceConfigFactory.java similarity index 92% rename from flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/source/config/MySqlSourceConfigFactory.java rename to flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/config/MySqlSourceConfigFactory.java index deec3d3e411..144e4ddaa2b 100644 --- a/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/source/config/MySqlSourceConfigFactory.java +++ b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/config/MySqlSourceConfigFactory.java @@ -16,10 +16,10 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.refactor.refactor.source.config; +package com.ververica.cdc.connectors.base.experimental.config; -import com.ververica.cdc.connectors.base.source.config.SourceConfigFactory; -import com.ververica.cdc.connectors.refactor.refactor.debezium.EmbeddedFlinkDatabaseHistory; +import com.ververica.cdc.connectors.base.config.JdbcSourceConfigFactory; +import com.ververica.cdc.connectors.base.experimental.EmbeddedFlinkDatabaseHistory; import io.debezium.config.Configuration; import io.debezium.connector.mysql.MySqlConnectorConfig; @@ -29,7 +29,7 @@ import static org.apache.flink.util.Preconditions.checkNotNull; /** A factory to initialize {@link MySqlSourceConfig}. */ -public class MySqlSourceConfigFactory extends SourceConfigFactory { +public class MySqlSourceConfigFactory extends JdbcSourceConfigFactory { private ServerIdRange serverIdRange; @@ -48,7 +48,7 @@ public MySqlSourceConfigFactory serverId(String serverId) { } /** Creates a new {@link MySqlSourceConfig} for the given subtask {@code subtaskId}. */ - public MySqlSourceConfig createConfig(int subtaskId) { + public MySqlSourceConfig create(int subtaskId) { 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 @@ -86,6 +86,7 @@ public MySqlSourceConfig createConfig(int subtaskId) { 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)); } @@ -106,28 +107,26 @@ public MySqlSourceConfig createConfig(int subtaskId) { Configuration dbzConfiguration = Configuration.from(props); String driverClassName = dbzConfiguration.getString(MySqlConnectorConfig.JDBC_DRIVER); - return new MySqlSourceConfig( + startupOptions, + databaseList, + tableList, + splitSize, + splitMetaGroupSize, + distributionFactorUpper, + distributionFactorLower, + includeSchemaChanges, + props, + dbzConfiguration, driverClassName, hostname, port, username, password, - databaseList, - tableList, - // serverIdRange, - startupOptions, - splitSize, - splitMetaGroupSize, fetchSize, serverTimeZone, connectTimeout, connectMaxRetries, - connectionPoolSize, - distributionFactorUpper, - distributionFactorLower, - includeSchemaChanges, - props, - dbzConfiguration); + connectionPoolSize); } } diff --git a/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/source/config/ServerIdRange.java b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/config/ServerIdRange.java similarity index 95% rename from flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/source/config/ServerIdRange.java rename to flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/config/ServerIdRange.java index 94410bd838b..dc915113ba3 100644 --- a/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/source/config/ServerIdRange.java +++ b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/config/ServerIdRange.java @@ -16,9 +16,9 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.refactor.refactor.source.config; +package com.ververica.cdc.connectors.base.experimental.config; -import com.ververica.cdc.connectors.base.source.config.SourceOptions; +import com.ververica.cdc.connectors.base.options.JdbcSourceOptions; import javax.annotation.Nullable; @@ -29,7 +29,7 @@ /** * This class defines a range of server id. The boundaries of the range are inclusive. * - * @see SourceOptions#SERVER_ID + * @see JdbcSourceOptions#SERVER_ID */ public class ServerIdRange implements Serializable { private static final long serialVersionUID = 1L; diff --git a/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/source/offset/BinlogOffset.java b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/offset/BinlogOffset.java similarity index 98% rename from flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/source/offset/BinlogOffset.java rename to flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/offset/BinlogOffset.java index a17659b4b35..196d54e417a 100644 --- a/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/source/offset/BinlogOffset.java +++ b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/offset/BinlogOffset.java @@ -16,9 +16,9 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.refactor.refactor.source.offset; +package com.ververica.cdc.connectors.base.experimental.offset; -import com.ververica.cdc.connectors.base.source.offset.Offset; +import com.ververica.cdc.connectors.base.source.meta.offset.Offset; import io.debezium.connector.mysql.GtidSet; import org.apache.commons.lang3.StringUtils; diff --git a/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/source/offset/BinlogOffsetFactory.java b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/offset/BinlogOffsetFactory.java similarity index 87% rename from flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/source/offset/BinlogOffsetFactory.java rename to flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/offset/BinlogOffsetFactory.java index 70d3b812b49..664faa930d7 100644 --- a/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/source/offset/BinlogOffsetFactory.java +++ b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/offset/BinlogOffsetFactory.java @@ -16,18 +16,20 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.refactor.refactor.source.offset; +package com.ververica.cdc.connectors.base.experimental.offset; import org.apache.flink.util.FlinkRuntimeException; -import com.ververica.cdc.connectors.base.source.offset.Offset; -import com.ververica.cdc.connectors.base.source.offset.OffsetFactory; +import com.ververica.cdc.connectors.base.source.meta.offset.Offset; +import com.ververica.cdc.connectors.base.source.meta.offset.OffsetFactory; 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); diff --git a/flink-connector-mysql-cdc-new/src/test/java/com/ververica/cdc/connectors/refactor/testutils/MySqlContainer.java b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/testutils/MySqlContainer.java similarity index 98% rename from flink-connector-mysql-cdc-new/src/test/java/com/ververica/cdc/connectors/refactor/testutils/MySqlContainer.java rename to flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/testutils/MySqlContainer.java index f7c94914fe3..fb6996413a2 100644 --- a/flink-connector-mysql-cdc-new/src/test/java/com/ververica/cdc/connectors/refactor/testutils/MySqlContainer.java +++ b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/testutils/MySqlContainer.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.refactor.testutils; +package com.ververica.cdc.connectors.base.experimental.testutils; import org.testcontainers.containers.ContainerLaunchException; import org.testcontainers.containers.JdbcDatabaseContainer; diff --git a/flink-connector-mysql-cdc-new/src/test/java/com/ververica/cdc/connectors/refactor/testutils/MySqlVersion.java b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/testutils/MySqlVersion.java similarity index 94% rename from flink-connector-mysql-cdc-new/src/test/java/com/ververica/cdc/connectors/refactor/testutils/MySqlVersion.java rename to flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/testutils/MySqlVersion.java index 2c0d9db3d41..a9a5b9bc99e 100644 --- a/flink-connector-mysql-cdc-new/src/test/java/com/ververica/cdc/connectors/refactor/testutils/MySqlVersion.java +++ b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/testutils/MySqlVersion.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.refactor.testutils; +package com.ververica.cdc.connectors.base.experimental.testutils; /** MySql version enum. */ public enum MySqlVersion { diff --git a/flink-connector-mysql-cdc-new/src/test/java/com/ververica/cdc/connectors/refactor/testutils/UniqueDatabase.java b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/testutils/UniqueDatabase.java similarity index 98% rename from flink-connector-mysql-cdc-new/src/test/java/com/ververica/cdc/connectors/refactor/testutils/UniqueDatabase.java rename to flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/testutils/UniqueDatabase.java index 2160e36486c..bc89a59fd2a 100644 --- a/flink-connector-mysql-cdc-new/src/test/java/com/ververica/cdc/connectors/refactor/testutils/UniqueDatabase.java +++ b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/testutils/UniqueDatabase.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.refactor.testutils; +package com.ververica.cdc.connectors.base.experimental.testutils; import java.net.URL; import java.nio.file.Files; diff --git a/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/source/debezium/DebeziumUtils.java b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/utils/MySqlConnectionUtils.java similarity index 96% rename from flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/source/debezium/DebeziumUtils.java rename to flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/utils/MySqlConnectionUtils.java index e624216af3e..1413a425de0 100644 --- a/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/source/debezium/DebeziumUtils.java +++ b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/utils/MySqlConnectionUtils.java @@ -16,12 +16,12 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.refactor.refactor.source.debezium; +package com.ververica.cdc.connectors.base.experimental.utils; import org.apache.flink.util.FlinkRuntimeException; import com.github.shyiko.mysql.binlog.BinaryLogClient; -import com.ververica.cdc.connectors.refactor.refactor.source.offset.BinlogOffset; +import com.ververica.cdc.connectors.base.experimental.offset.BinlogOffset; import io.debezium.config.Configuration; import io.debezium.connector.mysql.MySqlConnection; import io.debezium.connector.mysql.MySqlConnectorConfig; @@ -43,9 +43,9 @@ import java.util.Map; /** Utilities related to Debezium. */ -public class DebeziumUtils { +public class MySqlConnectionUtils { - private static final Logger LOG = LoggerFactory.getLogger(DebeziumUtils.class); + private static final Logger LOG = LoggerFactory.getLogger(MySqlConnectionUtils.class); /** Creates a new {@link MySqlConnection}, but not open the connection. */ public static MySqlConnection createMySqlConnection(Configuration dbzConfiguration) { diff --git a/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/schema/MySqlTypeUtils.java b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/utils/MySqlTypeUtils.java similarity index 98% rename from flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/schema/MySqlTypeUtils.java rename to flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/utils/MySqlTypeUtils.java index 76d62fd9964..5798a6af3b6 100644 --- a/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/schema/MySqlTypeUtils.java +++ b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/utils/MySqlTypeUtils.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.refactor.refactor.schema; +package com.ververica.cdc.connectors.base.experimental.utils; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.types.DataType; diff --git a/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/source/utils/StatementUtils.java b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/utils/StatementUtils.java similarity index 98% rename from flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/source/utils/StatementUtils.java rename to flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/utils/StatementUtils.java index 3bb2661b0bd..e5b82570904 100644 --- a/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/source/utils/StatementUtils.java +++ b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/utils/StatementUtils.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.refactor.refactor.source.utils; +package com.ververica.cdc.connectors.base.experimental.utils; import org.apache.flink.table.types.logical.RowType; @@ -30,7 +30,7 @@ import java.util.Optional; import java.util.stream.Collectors; -import static com.ververica.cdc.connectors.base.source.utils.RecordUtils.rowToArray; +import static com.ververica.cdc.connectors.base.utils.SourceRecordUtils.rowToArray; /** Utils to prepare SQL statement. */ public class StatementUtils { diff --git a/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/source/utils/TableDiscoveryUtils.java b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/utils/TableDiscoveryUtils.java similarity index 95% rename from flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/source/utils/TableDiscoveryUtils.java rename to flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/utils/TableDiscoveryUtils.java index f89b18e7665..931286d4bbe 100644 --- a/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/source/utils/TableDiscoveryUtils.java +++ b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/utils/TableDiscoveryUtils.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.refactor.refactor.source.utils; +package com.ververica.cdc.connectors.base.experimental.utils; import io.debezium.jdbc.JdbcConnection; import io.debezium.relational.RelationalTableFilters; @@ -28,7 +28,7 @@ import java.util.ArrayList; import java.util.List; -import static com.ververica.cdc.connectors.refactor.refactor.source.utils.StatementUtils.quote; +import static com.ververica.cdc.connectors.base.experimental.utils.StatementUtils.quote; /** Utilities to discovery matched tables. */ public class TableDiscoveryUtils { diff --git a/flink-connector-mysql-cdc-new/src/test/resources/ddl/inventory.sql b/flink-cdc-base/src/test/resources/ddl/inventory.sql similarity index 100% rename from flink-connector-mysql-cdc-new/src/test/resources/ddl/inventory.sql rename to flink-cdc-base/src/test/resources/ddl/inventory.sql diff --git a/flink-connector-mysql-cdc-new/src/test/resources/docker/server-gtids/my.cnf b/flink-cdc-base/src/test/resources/docker/server-gtids/my.cnf similarity index 100% rename from flink-connector-mysql-cdc-new/src/test/resources/docker/server-gtids/my.cnf rename to flink-cdc-base/src/test/resources/docker/server-gtids/my.cnf diff --git a/flink-connector-mysql-cdc-new/src/test/resources/docker/setup.sql b/flink-cdc-base/src/test/resources/docker/setup.sql similarity index 100% rename from flink-connector-mysql-cdc-new/src/test/resources/docker/setup.sql rename to flink-cdc-base/src/test/resources/docker/setup.sql diff --git a/flink-connector-mysql-cdc-new/src/test/resources/log4j2-test.properties b/flink-cdc-base/src/test/resources/log4j2-test.properties similarity index 100% rename from flink-connector-mysql-cdc-new/src/test/resources/log4j2-test.properties rename to flink-cdc-base/src/test/resources/log4j2-test.properties diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/assigners/ChunkSplitter.java b/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/assigners/ChunkSplitter.java deleted file mode 100644 index 637d903fda0..00000000000 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/assigners/ChunkSplitter.java +++ /dev/null @@ -1,320 +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 com.ververica.cdc.connectors.base.source.assigners; - -import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.util.FlinkRuntimeException; - -import com.ververica.cdc.connectors.base.schema.BaseSchema; -import com.ververica.cdc.connectors.base.source.config.SourceConfig; -import com.ververica.cdc.connectors.base.source.dialect.SnapshotEventDialect; -import com.ververica.cdc.connectors.base.source.split.SnapshotSplit; -import com.ververica.cdc.connectors.base.source.utils.ObjectUtils; -import io.debezium.jdbc.JdbcConnection; -import io.debezium.relational.Column; -import io.debezium.relational.Table; -import io.debezium.relational.TableId; -import io.debezium.relational.history.TableChanges.TableChange; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.math.BigDecimal; -import java.sql.SQLException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Objects; - -import static com.ververica.cdc.connectors.base.source.utils.ObjectUtils.doubleCompare; -import static java.math.BigDecimal.ROUND_CEILING; - -/** The {@code ChunkSplitter}'s task is to split table into a set of chunks or called splits. */ -public class ChunkSplitter { - - private static final Logger LOG = LoggerFactory.getLogger(ChunkSplitter.class); - - private final SourceConfig sourceConfig; - private final BaseSchema baseSchema; - private final SnapshotEventDialect dialect; - - public ChunkSplitter( - BaseSchema baseSchema, SourceConfig sourceConfig, SnapshotEventDialect dialect) { - this.baseSchema = baseSchema; - this.sourceConfig = sourceConfig; - this.dialect = dialect; - } - - /** Generates all snapshot splits (chunks) for the give table path. */ - public Collection generateSplits(TableId tableId) { - try (JdbcConnection jdbc = dialect.openJdbcConnection(sourceConfig)) { - - LOG.info("Start splitting table {} into chunks...", tableId); - long start = System.currentTimeMillis(); - - Table table = baseSchema.getTableSchema(jdbc, tableId).getTable(); - Column splitColumn = getSplitColumn(table); - final List chunks; - try { - chunks = splitTableIntoChunks(jdbc, tableId, splitColumn); - } catch (SQLException e) { - throw new FlinkRuntimeException("Failed to split chunks for table " + tableId, e); - } - - // convert chunks into splits - List splits = new ArrayList<>(); - RowType splitType = dialect.getSplitType(splitColumn); - for (int i = 0; i < chunks.size(); i++) { - ChunkRange chunk = chunks.get(i); - SnapshotSplit split = - createSnapshotSplit( - jdbc, - tableId, - i, - splitType, - chunk.getChunkStart(), - chunk.getChunkEnd()); - splits.add(split); - } - - long end = System.currentTimeMillis(); - LOG.info( - "Split table {} into {} chunks, time cost: {}ms.", - tableId, - splits.size(), - end - start); - return splits; - } catch (Exception e) { - throw new FlinkRuntimeException( - String.format("Generate Splits for table %s error", tableId), e); - } - } - - // -------------------------------------------------------------------------------------------- - // Utilities - // -------------------------------------------------------------------------------------------- - - /** - * We can use evenly-sized chunks or unevenly-sized chunks when split table into chunks, using - * evenly-sized chunks which is much efficient, using unevenly-sized chunks which will request - * many queries and is not efficient. - */ - private List splitTableIntoChunks( - JdbcConnection jdbc, TableId tableId, Column splitColumn) throws SQLException { - final String splitColumnName = splitColumn.name(); - final Object[] minMax = dialect.queryMinMax(jdbc, tableId, splitColumnName); - final Object min = minMax[0]; - final Object max = minMax[1]; - if (min == null || max == null || min.equals(max)) { - // empty table, or only one row, return full table scan as a chunk - return Collections.singletonList(ChunkRange.all()); - } - - final int chunkSize = sourceConfig.getSplitSize(); - final double distributionFactorUpper = sourceConfig.getDistributionFactorUpper(); - final double distributionFactorLower = sourceConfig.getDistributionFactorLower(); - - if (dialect.isEvenlySplitColumn(splitColumn)) { - long approximateRowCnt = dialect.queryApproximateRowCnt(jdbc, tableId); - double distributionFactor = - calculateDistributionFactor(tableId, min, max, approximateRowCnt); - - boolean dataIsEvenlyDistributed = - doubleCompare(distributionFactor, distributionFactorLower) >= 0 - && doubleCompare(distributionFactor, distributionFactorUpper) <= 0; - - if (dataIsEvenlyDistributed) { - // the minimum dynamic chunk size is at least 1 - final int dynamicChunkSize = Math.max((int) (distributionFactor * chunkSize), 1); - return splitEvenlySizedChunks( - tableId, min, max, approximateRowCnt, dynamicChunkSize); - } else { - return splitUnevenlySizedChunks( - jdbc, tableId, splitColumnName, min, max, chunkSize); - } - } else { - return splitUnevenlySizedChunks(jdbc, tableId, splitColumnName, min, max, chunkSize); - } - } - - /** - * Split table into evenly sized chunks based on the numeric min and max value of split column, - * and tumble chunks in step size. - */ - private List splitEvenlySizedChunks( - TableId tableId, Object min, Object max, long approximateRowCnt, int chunkSize) { - LOG.info( - "Use evenly-sized chunk optimization for table {}, the approximate row count is {}, the chunk size is {}", - tableId, - approximateRowCnt, - chunkSize); - if (approximateRowCnt <= chunkSize) { - // there is no more than one chunk, return full table as a chunk - return Collections.singletonList(ChunkRange.all()); - } - - final List splits = new ArrayList<>(); - Object chunkStart = null; - Object chunkEnd = ObjectUtils.plus(min, chunkSize); - while (ObjectUtils.compare(chunkEnd, max) <= 0) { - splits.add(ChunkRange.of(chunkStart, chunkEnd)); - chunkStart = chunkEnd; - chunkEnd = ObjectUtils.plus(chunkEnd, chunkSize); - } - // add the ending split - splits.add(ChunkRange.of(chunkStart, null)); - return splits; - } - - /** Split table into unevenly sized chunks by continuously calculating next chunk max value. */ - private List splitUnevenlySizedChunks( - JdbcConnection jdbc, - TableId tableId, - String splitColumnName, - Object min, - Object max, - int chunkSize) - throws SQLException { - LOG.info( - "Use unevenly-sized chunks for table {}, the chunk size is {}", tableId, chunkSize); - final List splits = new ArrayList<>(); - Object chunkStart = null; - Object chunkEnd = nextChunkEnd(jdbc, min, tableId, splitColumnName, max, chunkSize); - int count = 0; - while (chunkEnd != null && ObjectUtils.compare(chunkEnd, max) <= 0) { - // we start from [null, min + chunk_size) and avoid [null, min) - splits.add(ChunkRange.of(chunkStart, chunkEnd)); - // may sleep a while to avoid DDOS on MySQL server - maySleep(count++, tableId); - chunkStart = chunkEnd; - chunkEnd = nextChunkEnd(jdbc, chunkEnd, tableId, splitColumnName, max, chunkSize); - } - // add the ending split - splits.add(ChunkRange.of(chunkStart, null)); - return splits; - } - - private Object nextChunkEnd( - JdbcConnection jdbc, - Object previousChunkEnd, - TableId tableId, - String splitColumnName, - Object max, - int chunkSize) - throws SQLException { - // chunk end might be null when max values are removed - Object chunkEnd = - dialect.queryNextChunkMax( - jdbc, tableId, splitColumnName, chunkSize, previousChunkEnd); - if (Objects.equals(previousChunkEnd, chunkEnd)) { - // we don't allow equal chunk start and end, - // should query the next one larger than chunkEnd - chunkEnd = dialect.queryMin(jdbc, tableId, splitColumnName, chunkEnd); - } - if (ObjectUtils.compare(chunkEnd, max) >= 0) { - return null; - } else { - return chunkEnd; - } - } - - private SnapshotSplit createSnapshotSplit( - JdbcConnection jdbc, - TableId tableId, - int chunkId, - RowType splitKeyType, - Object chunkStart, - Object chunkEnd) { - // currently, we only support single split column - Object[] splitStart = chunkStart == null ? null : new Object[] {chunkStart}; - Object[] splitEnd = chunkEnd == null ? null : new Object[] {chunkEnd}; - Map schema = new HashMap<>(); - schema.put(tableId, baseSchema.getTableSchema(jdbc, tableId)); - return new SnapshotSplit( - tableId, - splitId(tableId, chunkId), - splitKeyType, - splitStart, - splitEnd, - null, - schema); - } - - // ------------------------------------------------------------------------------------------ - /** Returns the distribution factor of the given table. */ - private double calculateDistributionFactor( - TableId tableId, Object min, Object max, long approximateRowCnt) { - - if (!min.getClass().equals(max.getClass())) { - throw new IllegalStateException( - String.format( - "Unsupported operation type, the MIN value type %s is different with MAX value type %s.", - min.getClass().getSimpleName(), max.getClass().getSimpleName())); - } - if (approximateRowCnt == 0) { - return Double.MAX_VALUE; - } - BigDecimal difference = ObjectUtils.minus(max, min); - // factor = (max - min + 1) / rowCount - final BigDecimal subRowCnt = difference.add(BigDecimal.valueOf(1)); - double distributionFactor = - subRowCnt.divide(new BigDecimal(approximateRowCnt), 4, ROUND_CEILING).doubleValue(); - LOG.info( - "The distribution factor of table {} is {} according to the min split key {}, max split key {} and approximate row count {}", - tableId, - distributionFactor, - min, - max, - approximateRowCnt); - return distributionFactor; - } - - private static String splitId(TableId tableId, int chunkId) { - return tableId.toString() + ":" + chunkId; - } - - private static void maySleep(int count, TableId tableId) { - // every 100 queries to sleep 1s - if (count % 10 == 0) { - try { - Thread.sleep(100); - } catch (InterruptedException e) { - // nothing to do - } - LOG.info("ChunkSplitter has split {} chunks for table {}", count, tableId); - } - } - - public static Column getSplitColumn(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 primaryKeys.get(0); - } -} diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/dialect/Dialect.java b/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/dialect/Dialect.java deleted file mode 100644 index b0fa9d78284..00000000000 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/dialect/Dialect.java +++ /dev/null @@ -1,201 +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 com.ververica.cdc.connectors.base.source.dialect; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.LogicalTypeRoot; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.util.FlinkRuntimeException; - -import com.ververica.cdc.connectors.base.source.config.SourceConfig; -import com.ververica.cdc.connectors.base.source.internal.connection.JdbcConnectionFactory; -import com.ververica.cdc.connectors.base.source.internal.connection.PooledDataSourceFactory; -import com.ververica.cdc.connectors.base.source.internal.converter.JdbcSourceRecordConverter; -import com.ververica.cdc.connectors.base.source.offset.Offset; -import io.debezium.connector.base.ChangeEventQueue; -import io.debezium.jdbc.JdbcConnection; -import io.debezium.pipeline.DataChangeEvent; -import io.debezium.relational.Column; -import io.debezium.relational.TableId; -import io.debezium.relational.history.TableChanges; -import org.apache.kafka.connect.source.SourceRecord; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.sql.SQLException; -import java.util.List; -import java.util.Map; - -import static org.apache.flink.table.api.DataTypes.FIELD; -import static org.apache.flink.table.api.DataTypes.ROW; - -/** Handle the SQL dialect of jdbc driver. */ -@Internal -public interface Dialect { - Logger LOG = LoggerFactory.getLogger(Dialect.class); - - /** - * Get the name of dialect. - * - * @return the dialect name. - */ - String getName(); - - /** - * Get converter that is used to convert JDBC object to {@link SourceRecord}. - * - * @param rowType the given row type. - * @return a SourceRecord converter for the database. - */ - JdbcSourceRecordConverter getSourceRecordConverter(RowType rowType); - - /** - * Get a connection pool factory to create pooled DataSource. - * - * @return a connection pool factory. - */ - PooledDataSourceFactory getPooledDataSourceFactory(); - - /** - * Creates and opens a new {@link JdbcConnection} backing connection pool. - * - * @param sourceConfig a basic source configuration. - * @return a utility that simplifies using a JDBC connection. - */ - default JdbcConnection openJdbcConnection(SourceConfig sourceConfig) { - JdbcConnection jdbc = - new JdbcConnection( - sourceConfig.getDbzConfiguration(), - new JdbcConnectionFactory(sourceConfig, getPooledDataSourceFactory())); - try { - jdbc.connect(); - } catch (Exception e) { - LOG.error("Failed to open database connection", e); - throw new FlinkRuntimeException(e); - } - return jdbc; - } - - /** - * Checks whether split column is evenly distributed across its range. - * - * @param splitColumn split column. - * @return true that means split column with type BIGINT, INT, DECIMAL. - */ - default boolean isEvenlySplitColumn(Column splitColumn) { - DataType flinkType = fromDbzColumn(splitColumn); - LogicalTypeRoot typeRoot = flinkType.getLogicalType().getTypeRoot(); - - // currently, we only support the optimization that split column with type BIGINT, INT, - // DECIMAL - return typeRoot == LogicalTypeRoot.BIGINT - || typeRoot == LogicalTypeRoot.INTEGER - || typeRoot == LogicalTypeRoot.DECIMAL; - } - - /** - * convert dbz column to Flink row type. - * - * @param splitColumn split column. - * @return flink row type. - */ - default RowType getSplitType(Column splitColumn) { - return (RowType) - ROW(FIELD(splitColumn.name(), fromDbzColumn(splitColumn))).getLogicalType(); - } - - /** - * Get a corresponding Flink data type from a debezium {@link Column}. - * - * @param splitColumn dbz split column. - * @return flink data type - */ - DataType fromDbzColumn(Column splitColumn); - - /** - * display current offset from the database e.g. query Mysql binary logs by query - * SHOW MASTER STATUS. - * - * @param sourceConfig a basic source configuration. - * @return current offset of the database. - */ - Offset displayCurrentOffset(SourceConfig sourceConfig); - - /** - * discover need captured table schema by {@link SourceConfig}. - * - * @param sourceConfig a basic source configuration. - * @return a map of the {@link TableChanges.TableChange} which are need snapshot or streaming - * reading. - * @throws SQLException when connect to database occur error. - */ - Map discoverCapturedTableSchemas(SourceConfig sourceConfig) - throws SQLException; - - /** - * discover a list of need captured table. - * - * @param sourceConfig a basic source configuration. - * @return a list of {@link TableId} that is need captured. - */ - default List discoverCapturedTables(SourceConfig sourceConfig) { - final List capturedTableIds; - try { - capturedTableIds = listTables(sourceConfig); - } catch (SQLException e) { - throw new FlinkRuntimeException("Failed to discover captured tables", e); - } - if (capturedTableIds.isEmpty()) { - throw new IllegalArgumentException( - String.format( - "Can't find any matched tables, please check your configured database-name: %s and table-name: %s", - sourceConfig.getDatabaseList(), sourceConfig.getTableList())); - } - return capturedTableIds; - } - - /** - * connect to database, fetch all tables from databases, but only return {@link TableId} - * filtered tables and databases by {@link SourceConfig#getDatabaseList()} {@link - * SourceConfig#getTableList()}. - * - * @param sourceConfig a basic source configuration. - * @return a list of the {@link TableId} of tables which are need snapshot or streaming reading. - * @throws SQLException when connect to database occur error - */ - List listTables(SourceConfig sourceConfig) throws SQLException; - - /** - * Check if the table case sensitive. - * - * @param sourceConfig a basic source configuration. - * @return {@code true} if table case sensitive, {@code false} otherwise. - */ - boolean isTableIdCaseSensitive(SourceConfig sourceConfig); - - /** - * Context of the table snapshot or stream reading. Contains result data in {@link - * ChangeEventQueue} - */ - interface Context { - - ChangeEventQueue getQueue(); - } -} diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/dialect/SnapshotEventDialect.java b/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/dialect/SnapshotEventDialect.java deleted file mode 100644 index d8ac6e34ad5..00000000000 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/dialect/SnapshotEventDialect.java +++ /dev/null @@ -1,338 +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 com.ververica.cdc.connectors.base.source.dialect; - -import org.apache.flink.table.types.logical.RowType; - -import com.ververica.cdc.connectors.base.source.config.SourceConfig; -import com.ververica.cdc.connectors.base.source.internal.converter.JdbcSourceRecordConverter; -import com.ververica.cdc.connectors.base.source.offset.Offset; -import com.ververica.cdc.connectors.base.source.reader.split.SnapshotReader; -import com.ververica.cdc.connectors.base.source.reader.split.dispatcher.SignalEventDispatcher; -import com.ververica.cdc.connectors.base.source.split.SnapshotSplit; -import com.ververica.cdc.connectors.base.source.split.SourceSplitBase; -import io.debezium.DebeziumException; -import io.debezium.jdbc.JdbcConnection; -import io.debezium.pipeline.metrics.SnapshotChangeEventSourceMetrics; -import io.debezium.pipeline.source.spi.ChangeEventSource.ChangeEventSourceContext; -import io.debezium.pipeline.source.spi.SnapshotProgressListener; -import io.debezium.pipeline.spi.SnapshotResult; -import io.debezium.relational.RelationalSnapshotChangeEventSource; -import io.debezium.relational.TableId; -import io.debezium.util.Clock; -import io.debezium.util.Strings; -import io.debezium.util.Threads; -import org.apache.kafka.connect.errors.ConnectException; -import org.apache.kafka.connect.source.SourceRecord; - -import java.io.Closeable; -import java.io.Serializable; -import java.sql.PreparedStatement; -import java.sql.ResultSet; -import java.sql.SQLException; -import java.time.Duration; -import java.util.List; - -/** A dialect to handle database event during snapshotting phase. */ -public abstract class SnapshotEventDialect implements Dialect, Closeable, Serializable { - - /** - * Build the scan query sql of the {@link SnapshotSplit} based on the given {@link - * SnapshotContext} instance. - * - * @param tableId table identity. - * @param splitKeyType primary key type. - * @param isFirstSplit whether the first split. - * @param isLastSplit whether the last split. - * @return query sql. - */ - public abstract String buildSplitScanQuery( - TableId tableId, RowType splitKeyType, boolean isFirstSplit, boolean isLastSplit); - - /** - * Query the maximum and minimum value of the column in the table. e.g. query string - * SELECT MIN(%s) FROM %s WHERE %s > ? - * - * @param jdbc JDBC connection. - * @param tableId table identity. - * @param columnName column name. - * @return maximum and minimum value. - */ - public abstract Object[] queryMinMax(JdbcConnection jdbc, TableId tableId, String columnName) - throws SQLException; - - /** - * Query the minimum value of the column in the table, and the minimum value must greater than - * the excludedLowerBound value. e.g. prepare query string - * SELECT MIN(%s) FROM %s WHERE %s > ? - * - * @param jdbc JDBC connection. - * @param tableId table identity. - * @param columnName column name. - * @param excludedLowerBound the minimum value should be greater than this value. - * @return minimum value. - */ - public abstract Object queryMin( - JdbcConnection jdbc, TableId tableId, String columnName, Object excludedLowerBound) - throws SQLException; - - /** - * Query the maximum value of the next chunk, and the next chunk must be greater than or equal - * to includedLowerBound value [min_1, max_1), [min_2, max_2),... [min_n, null). - * Each time this method is called it will return max1, max2... - * - * @param jdbc JDBC connection. - * @param tableId table identity. - * @param columnName column name. - * @param chunkSize chunk size. - * @param includedLowerBound the previous chunk end value. - * @return next chunk end value. - */ - public abstract Object queryNextChunkMax( - JdbcConnection jdbc, - TableId tableId, - String columnName, - int chunkSize, - Object includedLowerBound) - throws SQLException; - - /** - * Approximate total number of entries in the lookup table. - * - * @param jdbc JDBC connection. - * @param tableId table identity. - * @return approximate row count. - */ - public abstract Long queryApproximateRowCnt(JdbcConnection jdbc, TableId tableId) - throws SQLException; - - public abstract PreparedStatement readTableSplitDataStatement( - String selectSql, SnapshotContext context); - - /** - * Create a task to read snapshot split of table. - * - * @param statefulTaskContext the context of snapshot split and table info. - * @return a snapshot split reading task. - */ - public abstract Task createTask(SnapshotContext statefulTaskContext); - - /** - * Normalize the records of snapshot split which represents the split records state on high - * watermark. data input: [low watermark event] [snapshot events ] [high watermark event] - * [binlog events] [binlog-end event] data output: [low watermark event] [normalized events] - * [high watermark event] - */ - public abstract List normalizedSplitRecords( - SnapshotSplit currentSnapshotSplit, List sourceRecords); - - /** - * Create a {@link SnapshotContext} instance contain snapshot read information. Pass the - * snapshotContext to {@link SnapshotReader} for table snapshot scan query. - * - * @param sourceConfig a basic source config - * @return - */ - public abstract SnapshotContext createSnapshotContext(SourceConfig sourceConfig); - - @Override - public abstract void close(); - - /** Context of the table snapshot reading. Contains table and database information. */ - public interface SnapshotContext extends Context { - - /** - * The {@link SnapshotContext#configure(SourceSplitBase)} method needs to be called after - * the {@link SnapshotContext} instance is created. - */ - void configure(SourceSplitBase sourceSplitBase); - - /** return {@link SourceConfig} instance in the {@link SnapshotContext}. */ - SourceConfig getSourceConfig(); - - /** return {@link SnapshotSplit} instance in the {@link SnapshotContext}. */ - SnapshotSplit getSnapshotSplit(); - - SnapshotChangeEventSourceMetrics getSnapshotChangeEventSourceMetrics(); - } - - /** Task to read snapshot split of table. */ - public interface Task { - - SnapshotResult execute(ChangeEventSourceContext sourceContext) throws InterruptedException; - } - - /** Task to read snapshot split of table. */ - public class SnapshotSplitReadTask implements Task { - private final Clock clock = Clock.SYSTEM; - /** Interval for showing a log statement with the progress while scanning a single table. */ - private final Duration logInterval = Duration.ofMillis(10_000); - - private final SnapshotContext snapshotContext; - private final SnapshotSplit snapshotSplit; - private final JdbcSourceRecordConverter jdbcSourceRecordConverter; - private final SnapshotProgressListener snapshotProgressListener; - - public SnapshotSplitReadTask(SnapshotContext snapshotContext) { - this.snapshotContext = snapshotContext; - this.snapshotSplit = snapshotContext.getSnapshotSplit(); - this.jdbcSourceRecordConverter = - getSourceRecordConverter(snapshotSplit.getSplitKeyType()); - this.snapshotProgressListener = snapshotContext.getSnapshotChangeEventSourceMetrics(); - } - - @Override - public SnapshotResult execute(ChangeEventSourceContext sourceContext) - throws InterruptedException { - try { - return doExecute(sourceContext); - } catch (InterruptedException e) { - LOG.warn("Snapshot was interrupted before completion"); - throw e; - } catch (Exception t) { - throw new DebeziumException(t); - } - } - - public SnapshotResult doExecute(ChangeEventSourceContext context) throws Exception { - final RelationalSnapshotChangeEventSource.RelationalSnapshotContext ctx = - new BaseSnapshotContext(); - final SignalEventDispatcher signalEventDispatcher = - new SignalEventDispatcher(snapshotContext); - - final Offset lowWatermark = displayCurrentOffset(snapshotContext.getSourceConfig()); - LOG.info( - "Snapshot step 1 - Determining low watermark {} for split {}", - lowWatermark, - snapshotSplit); - ((SnapshotReader.SnapshotSplitChangeEventSourceContextImpl) (context)) - .setLowWatermark(lowWatermark); - signalEventDispatcher.dispatchWatermarkEvent( - snapshotSplit, lowWatermark, SignalEventDispatcher.WatermarkKind.LOW); - - LOG.info("Snapshot step 2 - Snapshotting data"); - createDataEvents(ctx, snapshotSplit.getTableId()); - - final Offset highWatermark = displayCurrentOffset(snapshotContext.getSourceConfig()); - LOG.info( - "Snapshot step 3 - Determining high watermark {} for split {}", - highWatermark, - snapshotSplit); - signalEventDispatcher.dispatchWatermarkEvent( - snapshotSplit, highWatermark, SignalEventDispatcher.WatermarkKind.HIGH); - ((SnapshotReader.SnapshotSplitChangeEventSourceContextImpl) (context)) - .setHighWatermark(highWatermark); - - return SnapshotResult.completed(ctx.offset); - } - - private void createDataEvents( - RelationalSnapshotChangeEventSource.RelationalSnapshotContext ctx, TableId tableId) - throws Exception { - // EventDispatcher.SnapshotReceiver snapshotReceiver = - // dispatcher.getSnapshotChangeEventReceiver(); - LOG.debug("Snapshotting table {}", tableId); - createDataEventsForTable(ctx, tableId); - // snapshotReceiver.completeSnapshot(); - } - - /** Dispatches the data change events for the records of a single table. */ - private void createDataEventsForTable( - RelationalSnapshotChangeEventSource.RelationalSnapshotContext ctx, - // EventDispatcher.SnapshotReceiver snapshotReceiver, - TableId tableId) - throws InterruptedException { - - long exportStart = clock.currentTimeInMillis(); - LOG.info( - "Exporting data from split '{}' of table {}", snapshotSplit.splitId(), tableId); - - final String selectSql = - buildSplitScanQuery( - snapshotSplit.getTableId(), - snapshotSplit.getSplitKeyType(), - snapshotSplit.getSplitStart() == null, - snapshotSplit.getSplitEnd() == null); - LOG.info( - "For split '{}' of table {} using select statement: '{}'", - snapshotSplit.splitId(), - tableId, - selectSql); - - try (PreparedStatement selectStatement = - readTableSplitDataStatement(selectSql, this.snapshotContext); - 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 = new - // Object[columnArray.getGreatestColumnPosition()]; - // for (int i = 0; i < columnArray.getColumns().length; i++) - // { - // Column actualColumn = table.columns().get(i); - // row[columnArray.getColumns()[i].position() - 1] = - // readField(rs, i + 1, actualColumn, tableId); - // } - if (logTimer.expired()) { - long stop = clock.currentTimeInMillis(); - LOG.info( - "Exported {} records for split '{}' after {}", - rows, - snapshotSplit.splitId(), - Strings.duration(stop - exportStart)); - snapshotProgressListener.rowsScanned(tableId, rows); - logTimer = getTableScanLogTimer(); - } - SourceRecord sourceRecord = jdbcSourceRecordConverter.toInternal(rs); - ctx.offset.event(tableId, clock.currentTime()); - // dispatcher.dispatchSnapshotEvent( - // tableId, - // getChangeRecordEmitter(snapshotContext, tableId, - // 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 " + tableId + " failed", e); - } - } - - private Threads.Timer getTableScanLogTimer() { - return Threads.timer(clock, logInterval); - } - } - - private static class BaseSnapshotContext - extends RelationalSnapshotChangeEventSource.RelationalSnapshotContext { - - public BaseSnapshotContext() throws SQLException { - super(""); - } - } -} diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/dialect/StreamingEventDialect.java b/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/dialect/StreamingEventDialect.java deleted file mode 100644 index 42126122c72..00000000000 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/dialect/StreamingEventDialect.java +++ /dev/null @@ -1,49 +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 com.ververica.cdc.connectors.base.source.dialect; - -import com.ververica.cdc.connectors.base.source.split.StreamSplit; -import io.debezium.pipeline.source.spi.ChangeEventSource.ChangeEventSourceContext; -import io.debezium.pipeline.spi.SnapshotResult; - -import java.io.Closeable; -import java.io.Serializable; - -/** A dialect to handle database event during streaming process. */ -public abstract class StreamingEventDialect implements Dialect, Closeable, Serializable { - - public abstract Task createTask(StreamSplit backfillStreamSplit); - - @Override - public abstract void close(); - - /** Task to read split of table. */ - public interface Task { - - SnapshotResult execute(ChangeEventSourceContext sourceContext); - } - - /** Task to read stream split of table. */ - public class StreamSplitReadTask implements Task { - @Override - public SnapshotResult execute(ChangeEventSourceContext sourceContext) { - return null; - } - } -} diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/split/SnapshotReader.java b/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/split/SnapshotReader.java deleted file mode 100644 index 3fbb38dfe9d..00000000000 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/split/SnapshotReader.java +++ /dev/null @@ -1,287 +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 com.ververica.cdc.connectors.base.source.reader.split; - -import org.apache.flink.util.FlinkRuntimeException; - -import org.apache.flink.shaded.guava18.com.google.common.util.concurrent.ThreadFactoryBuilder; - -import com.ververica.cdc.connectors.base.source.dialect.SnapshotEventDialect; -import com.ververica.cdc.connectors.base.source.dialect.StreamingEventDialect; -import com.ververica.cdc.connectors.base.source.offset.Offset; -import com.ververica.cdc.connectors.base.source.reader.split.dispatcher.SignalEventDispatcher; -import com.ververica.cdc.connectors.base.source.split.SnapshotSplit; -import com.ververica.cdc.connectors.base.source.split.SourceSplitBase; -import com.ververica.cdc.connectors.base.source.split.StreamSplit; -import com.ververica.cdc.connectors.base.source.utils.RecordUtils; -import io.debezium.connector.base.ChangeEventQueue; -import io.debezium.pipeline.DataChangeEvent; -import io.debezium.pipeline.source.spi.ChangeEventSource; -import io.debezium.pipeline.spi.SnapshotResult; -import io.debezium.util.SchemaNameAdjuster; -import org.apache.kafka.connect.source.SourceRecord; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import javax.annotation.Nullable; - -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.ThreadFactory; -import java.util.concurrent.atomic.AtomicBoolean; - -/** Reader to read split of table, the split is the snapshot split {@link SnapshotSplit}. */ -public class SnapshotReader implements Reader { - - private static final Logger LOG = LoggerFactory.getLogger(SnapshotReader.class); - private final SnapshotEventDialect.SnapshotContext statefulTaskContext; - private final ExecutorService executor; - - private volatile ChangeEventQueue queue; - private volatile boolean currentTaskRunning; - private volatile Throwable readException; - - // task to read snapshot for current split - private SnapshotEventDialect.Task splitSnapshotReadTask; - private SnapshotSplit currentSnapshotSplit; - private SchemaNameAdjuster nameAdjuster; - public AtomicBoolean hasNextElement; - public AtomicBoolean reachEnd; - - private final StreamingEventDialect streamingEventDialect; - private final SnapshotEventDialect snapshotEventDialect; - - public SnapshotReader( - SnapshotEventDialect.SnapshotContext statefulTaskContext, - int subtaskId, - SnapshotEventDialect snapshotEventDialect, - StreamingEventDialect streamingEventDialect) { - this.statefulTaskContext = statefulTaskContext; - ThreadFactory threadFactory = - new ThreadFactoryBuilder() - .setNameFormat("debezium-snapshot-reader-" + subtaskId) - .build(); - this.executor = Executors.newSingleThreadExecutor(threadFactory); - this.currentTaskRunning = false; - this.hasNextElement = new AtomicBoolean(false); - this.reachEnd = new AtomicBoolean(false); - this.snapshotEventDialect = snapshotEventDialect; - this.streamingEventDialect = streamingEventDialect; - } - - @Override - public void submitSplit(SourceSplitBase splitToRead) { - this.currentSnapshotSplit = splitToRead.asSnapshotSplit(); - statefulTaskContext.configure(currentSnapshotSplit); - this.queue = statefulTaskContext.getQueue(); - // this.nameAdjuster = statefulTaskContext.getSchemaNameAdjuster(); - this.hasNextElement.set(true); - this.reachEnd.set(false); - this.splitSnapshotReadTask = snapshotEventDialect.createTask(statefulTaskContext); - executor.submit( - () -> { - try { - currentTaskRunning = true; - // execute snapshot read task - final SnapshotSplitChangeEventSourceContextImpl sourceContext = - new SnapshotSplitChangeEventSourceContextImpl(); - SnapshotResult snapshotResult = - splitSnapshotReadTask.execute(sourceContext); - - final StreamSplit backfillStreamSplit = - createBackfillStreamSplit(sourceContext); - // optimization that skip the binlog read when the low watermark equals high - // watermark - final boolean binlogBackfillRequired = - backfillStreamSplit - .getEndingOffset() - .isAfter(backfillStreamSplit.getStartingOffset()); - if (!binlogBackfillRequired) { - dispatchHighWatermark(backfillStreamSplit); - currentTaskRunning = false; - return; - } - - // execute binlog read task - if (snapshotResult.isCompletedOrSkipped()) { - final StreamingEventDialect.Task backfillBinlogReadTask = - createBackfillStreamReadTask(backfillStreamSplit); - backfillBinlogReadTask.execute( - new SnapshotStreamSplitChangeEventSourceContextImpl()); - } else { - readException = - new IllegalStateException( - String.format( - "Read snapshot for mysql split %s fail", - currentSnapshotSplit)); - } - } catch (Exception e) { - currentTaskRunning = false; - LOG.error( - String.format( - "Execute snapshot read task for mysql split %s fail", - currentSnapshotSplit), - e); - readException = e; - } - }); - } - - private StreamSplit createBackfillStreamSplit( - SnapshotSplitChangeEventSourceContextImpl sourceContext) { - return new StreamSplit( - currentSnapshotSplit.splitId(), - sourceContext.getLowWatermark(), - sourceContext.getHighWatermark(), - new ArrayList<>(), - currentSnapshotSplit.getTableSchemas(), - 0); - } - - private StreamingEventDialect.Task createBackfillStreamReadTask( - StreamSplit backfillStreamSplit) { - return streamingEventDialect.createTask(backfillStreamSplit); - } - - private void dispatchHighWatermark(StreamSplit backFillBinlogSplit) - throws InterruptedException { - final SignalEventDispatcher signalEventDispatcher = - new SignalEventDispatcher(statefulTaskContext); - signalEventDispatcher.dispatchWatermarkEvent( - backFillBinlogSplit, - backFillBinlogSplit.getEndingOffset(), - SignalEventDispatcher.WatermarkKind.BINLOG_END); - } - - @Override - public boolean isFinished() { - return currentSnapshotSplit == null - || (!currentTaskRunning && !hasNextElement.get() && reachEnd.get()); - } - - @Nullable - @Override - public Iterator pollSplitRecords() throws InterruptedException { - checkReadException(); - - if (hasNextElement.get()) { - // data input: [low watermark event][snapshot events][high watermark event][binlog - // events][binlog-end event] - // data output: [low watermark event][normalized events][high watermark event] - boolean reachBinlogEnd = false; - final List sourceRecords = new ArrayList<>(); - while (!reachBinlogEnd) { - List batch = queue.poll(); - for (DataChangeEvent event : batch) { - sourceRecords.add(event.getRecord()); - if (RecordUtils.isEndWatermarkEvent(event.getRecord())) { - reachBinlogEnd = true; - break; - } - } - } - // snapshot split return its data once - hasNextElement.set(false); - return snapshotEventDialect - .normalizedSplitRecords(currentSnapshotSplit, sourceRecords) - .iterator(); - } - // the data has been polled, no more data - reachEnd.compareAndSet(false, true); - return null; - } - - private void checkReadException() { - if (readException != null) { - throw new FlinkRuntimeException( - String.format( - "Read split %s error due to %s.", - currentSnapshotSplit, readException.getMessage()), - readException); - } - } - - @Override - public void close() { - try { - snapshotEventDialect.close(); - streamingEventDialect.close(); - // if (statefulTaskContext.getConnection() != null) { - // statefulTaskContext.getConnection().close(); - // } - // if (statefulTaskContext.getBinaryLogClient() != null) { - // statefulTaskContext.getBinaryLogClient().disconnect(); - // } - } catch (Exception e) { - LOG.error("Close snapshot reader error", e); - } - } - - /** - * {@link ChangeEventSource.ChangeEventSourceContext} implementation that keeps low/high - * watermark for each {@link SnapshotSplit}. - */ - public class SnapshotSplitChangeEventSourceContextImpl - implements ChangeEventSource.ChangeEventSourceContext { - - private Offset lowWatermark; - private Offset highWatermark; - - public Offset getLowWatermark() { - return lowWatermark; - } - - public void setLowWatermark(Offset lowWatermark) { - this.lowWatermark = lowWatermark; - } - - public Offset getHighWatermark() { - return highWatermark; - } - - public void setHighWatermark(Offset highWatermark) { - this.highWatermark = highWatermark; - } - - @Override - public boolean isRunning() { - return lowWatermark != null && highWatermark != null; - } - } - - /** - * The {@link ChangeEventSource.ChangeEventSourceContext} implementation for bounded binlog task - * of a snapshot split task. - */ - public class SnapshotStreamSplitChangeEventSourceContextImpl - implements ChangeEventSource.ChangeEventSourceContext { - - public void finished() { - currentTaskRunning = false; - } - - @Override - public boolean isRunning() { - return currentTaskRunning; - } - } -} diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/split/dispatcher/SignalEventDispatcher.java b/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/split/dispatcher/SignalEventDispatcher.java deleted file mode 100644 index e4060bd6f18..00000000000 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/split/dispatcher/SignalEventDispatcher.java +++ /dev/null @@ -1,147 +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 com.ververica.cdc.connectors.base.source.reader.split.dispatcher; - -import com.ververica.cdc.connectors.base.source.dialect.Dialect; -import com.ververica.cdc.connectors.base.source.offset.Offset; -import com.ververica.cdc.connectors.base.source.split.SourceSplitBase; -import io.debezium.connector.base.ChangeEventQueue; -import io.debezium.pipeline.DataChangeEvent; -import io.debezium.util.SchemaNameAdjuster; -import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.data.SchemaBuilder; -import org.apache.kafka.connect.data.Struct; -import org.apache.kafka.connect.source.SourceRecord; - -import java.util.Map; - -/** - * A dispatcher to dispatch watermark signal events. - * - *

The watermark signal event is used to describe the start point and end point of a split scan. - * The Watermark Signal Algorithm is inspired by https://arxiv.org/pdf/2010.12597v1.pdf. - */ -public class SignalEventDispatcher { - - private static final SchemaNameAdjuster SCHEMA_NAME_ADJUSTER = SchemaNameAdjuster.create(); - - public static final String DATABASE_NAME = "db"; - public static final String TABLE_NAME = "table"; - public static final String WATERMARK_SIGNAL = "_split_watermark_signal_"; - public static final String SPLIT_ID_KEY = "split_id"; - // public static final String BINLOG_FILENAME_OFFSET_KEY = "file"; - // public static final String BINLOG_POSITION_OFFSET_KEY = "pos"; - public static final String HISTORY_RECORD_FIELD = "historyRecord"; - public static final String WATERMARK_KIND = "watermark_kind"; - public static final String SIGNAL_EVENT_KEY_SCHEMA_NAME = - "io.debezium.connector.flink.cdc.embedded.watermark.key"; - public static final String SIGNAL_EVENT_VALUE_SCHEMA_NAME = - "io.debezium.connector.flink.cdc.embedded.watermark.value"; - - private final Schema signalEventKeySchema; - private final Schema signalEventValueSchema; - private final Map sourcePartition; - private final String topic; - private final ChangeEventQueue queue; - - public SignalEventDispatcher( - Map sourcePartition, String topic, ChangeEventQueue queue) { - this.sourcePartition = sourcePartition; - this.topic = topic; - this.queue = queue; - this.signalEventKeySchema = - SchemaBuilder.struct() - .name(SCHEMA_NAME_ADJUSTER.adjust(SIGNAL_EVENT_KEY_SCHEMA_NAME)) - .field(SPLIT_ID_KEY, Schema.STRING_SCHEMA) - .field(WATERMARK_SIGNAL, Schema.BOOLEAN_SCHEMA) - .build(); - this.signalEventValueSchema = - SchemaBuilder.struct() - .name(SCHEMA_NAME_ADJUSTER.adjust(SIGNAL_EVENT_VALUE_SCHEMA_NAME)) - .field(SPLIT_ID_KEY, Schema.STRING_SCHEMA) - .field(WATERMARK_KIND, Schema.STRING_SCHEMA) - .build(); - } - - public SignalEventDispatcher(Dialect.Context statefulTaskContext) { - // todo - this.sourcePartition = null; - this.topic = null; - this.queue = statefulTaskContext.getQueue(); - this.signalEventKeySchema = - SchemaBuilder.struct() - .name(SCHEMA_NAME_ADJUSTER.adjust(SIGNAL_EVENT_KEY_SCHEMA_NAME)) - .field(SPLIT_ID_KEY, Schema.STRING_SCHEMA) - .field(WATERMARK_SIGNAL, Schema.BOOLEAN_SCHEMA) - .build(); - this.signalEventValueSchema = - SchemaBuilder.struct() - .name(SCHEMA_NAME_ADJUSTER.adjust(SIGNAL_EVENT_VALUE_SCHEMA_NAME)) - .field(SPLIT_ID_KEY, Schema.STRING_SCHEMA) - .field(WATERMARK_KIND, Schema.STRING_SCHEMA) - .build(); - } - - public void dispatchWatermarkEvent( - SourceSplitBase sourceSplit, Offset watermark, WatermarkKind watermarkKind) - throws InterruptedException { - - SourceRecord sourceRecord = - new SourceRecord( - sourcePartition, - watermark.getOffset(), - topic, - signalEventKeySchema, - signalRecordKey(sourceSplit.splitId()), - signalEventValueSchema, - signalRecordValue(sourceSplit.splitId(), watermarkKind)); - queue.enqueue(new DataChangeEvent(sourceRecord)); - } - - private Struct signalRecordKey(String splitId) { - Struct result = new Struct(signalEventKeySchema); - result.put(SPLIT_ID_KEY, splitId); - result.put(WATERMARK_SIGNAL, true); - return result; - } - - private Struct signalRecordValue(String splitId, WatermarkKind watermarkKind) { - Struct result = new Struct(signalEventValueSchema); - result.put(SPLIT_ID_KEY, splitId); - result.put(WATERMARK_KIND, watermarkKind.toString()); - return result; - } - - /** The watermark kind. */ - public enum WatermarkKind { - LOW, - HIGH, - BINLOG_END; - - public WatermarkKind fromString(String kindString) { - if (LOW.name().equalsIgnoreCase(kindString)) { - return LOW; - } else if (HIGH.name().equalsIgnoreCase(kindString)) { - return HIGH; - } else { - return BINLOG_END; - } - } - } -} diff --git a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/split/task/context/StatefulTaskContext.java b/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/split/task/context/StatefulTaskContext.java deleted file mode 100644 index 1664a7bafa1..00000000000 --- a/flink-connector-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/split/task/context/StatefulTaskContext.java +++ /dev/null @@ -1,327 +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 com.ververica.cdc.connectors.base.source.reader.split.task.context; -// -// import com.github.shyiko.mysql.binlog.BinaryLogClient; -// import com.ververica.cdc.connectors.base.source.config.SourceConfig; -// import com.ververica.cdc.connectors.mysql.debezium.DebeziumUtils; -// import com.ververica.cdc.connectors.mysql.debezium.EmbeddedFlinkDatabaseHistory; -// import com.ververica.cdc.connectors.mysql.debezium.dispatcher.EventDispatcherImpl; -// import com.ververica.cdc.connectors.mysql.source.config.MySqlSourceConfig; -// import com.ververica.cdc.connectors.mysql.source.offset.BinlogOffset; -// import com.ververica.cdc.connectors.mysql.source.split.MySqlSplit; -// 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.MySqlStreamingChangeEventSourceMetrics; -// 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.metrics.StreamingChangeEventSourceMetrics; -// import io.debezium.pipeline.source.spi.EventMetadataProvider; -// import io.debezium.pipeline.spi.OffsetContext; -// import io.debezium.relational.TableId; -// import io.debezium.schema.DataCollectionId; -// import io.debezium.schema.TopicSelector; -// import io.debezium.util.Clock; -// import io.debezium.util.Collect; -// import io.debezium.util.SchemaNameAdjuster; -// import org.apache.kafka.connect.data.Struct; -// import org.slf4j.Logger; -// import org.slf4j.LoggerFactory; -// -// import java.time.Instant; -// import java.util.List; -// import java.util.Map; -// -// import static -// com.ververica.cdc.connectors.mysql.source.offset.BinlogOffset.BINLOG_FILENAME_OFFSET_KEY; -// -/// ** -// * A stateful task context that contains entries the debezium mysql connector task required. -// * -// *

The offset change and schema change should record to MySqlSplitState when emit the record, -// * thus the Flink's state mechanism can help to store/restore when failover happens. -// */ -// public class StatefulTaskContext { -// -// private static final Logger LOG = LoggerFactory.getLogger(StatefulTaskContext.class); -// private static final Clock clock = Clock.SYSTEM; -// -// private final SourceConfig sourceConfig; -// private final MySqlConnectorConfig connectorConfig; -// private final MySqlEventMetadataProvider metadataProvider; -// private final SchemaNameAdjuster schemaNameAdjuster; -// private final MySqlConnection connection; -// private final BinaryLogClient binaryLogClient; -// -// private MySqlDatabaseSchema databaseSchema; -// private MySqlTaskContextImpl taskContext; -// private MySqlOffsetContext offsetContext; -// private TopicSelector topicSelector; -// private SnapshotChangeEventSourceMetrics snapshotChangeEventSourceMetrics; -// private StreamingChangeEventSourceMetrics streamingChangeEventSourceMetrics; -// private EventDispatcherImpl dispatcher; -// private ChangeEventQueue queue; -// private ErrorHandler errorHandler; -// -// public StatefulTaskContext( -// MySqlSourceConfig sourceConfig, -// BinaryLogClient binaryLogClient, -// MySqlConnection connection) { -// this.sourceConfig = sourceConfig; -// this.connectorConfig = sourceConfig.getMySqlConnectorConfig(); -// this.schemaNameAdjuster = SchemaNameAdjuster.create(); -// this.metadataProvider = new MySqlEventMetadataProvider(); -// this.binaryLogClient = binaryLogClient; -// this.connection = connection; -// } -// -// public void configure(MySqlSplit mySqlSplit) { -// // initial stateful objects -// final boolean tableIdCaseInsensitive = connection.isTableIdCaseSensitive(); -// this.topicSelector = MySqlTopicSelector.defaultSelector(connectorConfig); -// EmbeddedFlinkDatabaseHistory.registerHistory( -// sourceConfig -// .getDbzConfiguration() -// .getString(EmbeddedFlinkDatabaseHistory.DATABASE_HISTORY_INSTANCE_NAME), -// mySqlSplit.getTableSchemas().values()); -// this.databaseSchema = -// DebeziumUtils.createMySqlDatabaseSchema(connectorConfig, tableIdCaseInsensitive); -// this.offsetContext = -// loadStartingOffsetState(new MySqlOffsetContext.Loader(connectorConfig), -// mySqlSplit); -// validateAndLoadDatabaseHistory(offsetContext, databaseSchema); -// -// this.taskContext = -// new MySqlTaskContextImpl(connectorConfig, databaseSchema, binaryLogClient); -// final int queueSize = -// mySqlSplit.isSnapshotSplit() -// ? Integer.MAX_VALUE -// : connectorConfig.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 EventDispatcherImpl<>( -// connectorConfig, -// topicSelector, -// databaseSchema, -// queue, -// connectorConfig.getTableFilters().dataCollectionFilter(), -// DataChangeEvent::new, -// metadataProvider, -// schemaNameAdjuster); -// -// final MySqlChangeEventSourceMetricsFactory changeEventSourceMetricsFactory = -// new MySqlChangeEventSourceMetricsFactory( -// new MySqlStreamingChangeEventSourceMetrics( -// taskContext, queue, metadataProvider)); -// this.snapshotChangeEventSourceMetrics = -// changeEventSourceMetricsFactory.getSnapshotMetrics( -// taskContext, queue, metadataProvider); -// this.streamingChangeEventSourceMetrics = -// changeEventSourceMetricsFactory.getStreamingMetrics( -// taskContext, queue, metadataProvider); -// this.errorHandler = new MySqlErrorHandler(connectorConfig.getLogicalName(), queue); -// } -// -// private void validateAndLoadDatabaseHistory( -// MySqlOffsetContext offset, MySqlDatabaseSchema schema) { -// schema.initializeStorage(); -// schema.recover(offset); -// } -// -// /** Loads the connector's persistent offset (if present) via the given loader. */ -// private MySqlOffsetContext loadStartingOffsetState( -// OffsetContext.Loader loader, MySqlSplit mySqlSplit) { -// BinlogOffset offset = -// mySqlSplit.isSnapshotSplit() -// ? BinlogOffset.INITIAL_OFFSET -// : mySqlSplit.asBinlogSplit().getStartingOffset(); -// -// MySqlOffsetContext 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(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; -// } -// -// /** 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(); -// } -// } -// -// public static Clock getClock() { -// return clock; -// } -// -// public MySqlSourceConfig getSourceConfig() { -// return sourceConfig; -// } -// -// public MySqlConnectorConfig getConnectorConfig() { -// return connectorConfig; -// } -// -// public MySqlConnection getConnection() { -// return connection; -// } -// -// public BinaryLogClient getBinaryLogClient() { -// return binaryLogClient; -// } -// -// public MySqlDatabaseSchema getDatabaseSchema() { -// return databaseSchema; -// } -// -// public MySqlTaskContextImpl getTaskContext() { -// return taskContext; -// } -// -// public EventDispatcherImpl getDispatcher() { -// return dispatcher; -// } -// -// public ChangeEventQueue getQueue() { -// return queue; -// } -// -// public ErrorHandler getErrorHandler() { -// return errorHandler; -// } -// -// public MySqlOffsetContext getOffsetContext() { -// return offsetContext; -// } -// -// public TopicSelector getTopicSelector() { -// return topicSelector; -// } -// -// public SnapshotChangeEventSourceMetrics getSnapshotChangeEventSourceMetrics() { -// snapshotChangeEventSourceMetrics.reset(); -// return snapshotChangeEventSourceMetrics; -// } -// -// public StreamingChangeEventSourceMetrics getStreamingChangeEventSourceMetrics() { -// streamingChangeEventSourceMetrics.reset(); -// return streamingChangeEventSourceMetrics; -// } -// -// public SchemaNameAdjuster getSchemaNameAdjuster() { -// return schemaNameAdjuster; -// } -// } diff --git a/flink-connector-mysql-cdc-new/pom.xml b/flink-connector-mysql-cdc-new/pom.xml deleted file mode 100644 index 7a4cf279186..00000000000 --- a/flink-connector-mysql-cdc-new/pom.xml +++ /dev/null @@ -1,185 +0,0 @@ - - - - - flink-cdc-connectors - com.ververica - 2.2-SNAPSHOT - - 4.0.0 - - flink-connector-mysql-cdc-new - flink-connector-mysql-cdc-new - jar - - - - com.ververica - flink-connector-debezium - ${project.version} - - - kafka-log4j-appender - org.apache.kafka - - - - - - com.ververica - flink-connector-base - ${project.version} - - - io.debezium - debezium-connector-mysql - ${debezium.version} - - - - - com.esri.geometry - esri-geometry-api - ${geometry.version} - - - com.fasterxml.jackson.core - jackson-core - - - - - - com.zaxxer - HikariCP - 4.0.3 - - - - - - com.ververica - flink-connector-test-util - ${project.version} - test - - - - io.debezium - debezium-core - ${debezium.version} - test-jar - test - - - - com.alibaba - fastjson - 1.2.78 - test - - - - - - org.apache.flink - flink-table-planner-blink_${scala.binary.version} - ${flink.version} - test - - - - org.apache.flink - flink-table-runtime-blink_${scala.binary.version} - ${flink.version} - test - - - - org.apache.flink - flink-test-utils_${scala.binary.version} - ${flink.version} - test - - - org.apache.flink - flink-connector-test-utils - ${flink.version} - test - - - - org.apache.flink - flink-core - ${flink.version} - test-jar - test - - - - org.apache.flink - flink-streaming-java_${scala.binary.version} - ${flink.version} - test-jar - test - - - - org.apache.flink - flink-table-common - ${flink.version} - test-jar - test - - - - org.apache.flink - flink-tests - ${flink.version} - test-jar - test - - - - org.apache.flink - flink-table-planner-blink_${scala.binary.version} - ${flink.version} - test-jar - test - - - - - - org.testcontainers - mysql - ${testcontainers.version} - test - - - - com.jayway.jsonpath - json-path - 2.4.0 - test - - - - diff --git a/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/debezium/task/context/MySqlTaskContextImpl.java b/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/debezium/task/context/MySqlTaskContextImpl.java deleted file mode 100644 index 11fcfd61026..00000000000 --- a/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/debezium/task/context/MySqlTaskContextImpl.java +++ /dev/null @@ -1,43 +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 com.ververica.cdc.connectors.refactor.refactor.debezium.task.context; - -import com.github.shyiko.mysql.binlog.BinaryLogClient; -import io.debezium.connector.mysql.MySqlConnectorConfig; -import io.debezium.connector.mysql.MySqlDatabaseSchema; -import io.debezium.connector.mysql.MySqlTaskContext; - -/** A subclass implementation of {@link MySqlTaskContext} which reuses one BinaryLogClient. */ -public 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; - } -} diff --git a/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/schema/MySqlSchema.java b/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/schema/MySqlSchema.java deleted file mode 100644 index 9365891ed49..00000000000 --- a/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/schema/MySqlSchema.java +++ /dev/null @@ -1,105 +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 com.ververica.cdc.connectors.refactor.refactor.schema; - -import org.apache.flink.util.FlinkRuntimeException; - -import com.ververica.cdc.connectors.base.schema.BaseSchema; -import com.ververica.cdc.connectors.refactor.refactor.source.config.MySqlSourceConfig; -import com.ververica.cdc.connectors.refactor.refactor.source.debezium.DebeziumUtils; -import com.ververica.cdc.connectors.refactor.refactor.source.utils.StatementUtils; -import io.debezium.connector.mysql.MySqlConnectorConfig; -import io.debezium.connector.mysql.MySqlDatabaseSchema; -import io.debezium.connector.mysql.MySqlOffsetContext; -import io.debezium.jdbc.JdbcConnection; -import io.debezium.relational.TableId; -import io.debezium.relational.history.TableChanges; -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; - -/** Provides as a tool class to obtain mysql table schema information. */ -public class MySqlSchema implements BaseSchema { - - private final MySqlConnectorConfig connectorConfig; - private final MySqlDatabaseSchema databaseSchema; - private final Map schemasByTableId; - - public MySqlSchema(MySqlSourceConfig sourceConfig, boolean isTableIdCaseSensitive) { - this.connectorConfig = sourceConfig.getMySqlConnectorConfig(); - this.databaseSchema = - DebeziumUtils.createMySqlDatabaseSchema(connectorConfig, isTableIdCaseSensitive); - this.schemasByTableId = new HashMap<>(); - } - - @Override - public TableChanges.TableChange getTableSchema(JdbcConnection jdbc, TableId tableId) { - - // read schema from cache first - TableChanges.TableChange schema = schemasByTableId.get(tableId); - if (schema == null) { - schema = readTableSchema(jdbc, tableId); - schemasByTableId.put(tableId, schema); - } - return schema; - } - - // ------------------------------------------------------------------------------------------ - // Helpers - // ------------------------------------------------------------------------------------------ - - private TableChanges.TableChange readTableSchema(JdbcConnection jdbc, TableId tableId) { - final Map tableChangeMap = new HashMap<>(); - final String sql = "SHOW CREATE TABLE " + StatementUtils.quote(tableId); - try { - jdbc.query( - sql, - rs -> { - if (rs.next()) { - final String ddl = rs.getString(2); - final MySqlOffsetContext offsetContext = - MySqlOffsetContext.initial(connectorConfig); - List schemaChangeEvents = - databaseSchema.parseSnapshotDdl( - 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-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/source/MySqlSource.java b/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/source/MySqlSource.java deleted file mode 100644 index 45f39023344..00000000000 --- a/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/source/MySqlSource.java +++ /dev/null @@ -1,66 +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 com.ververica.cdc.connectors.refactor.refactor.source; - -import com.ververica.cdc.connectors.base.schema.BaseSchema; -import com.ververica.cdc.connectors.base.source.ChangeEventHybridSource; -import com.ververica.cdc.connectors.base.source.config.SourceConfigFactory; -import com.ververica.cdc.connectors.base.source.dialect.SnapshotEventDialect; -import com.ververica.cdc.connectors.base.source.dialect.StreamingEventDialect; -import com.ververica.cdc.connectors.base.source.offset.OffsetFactory; -import com.ververica.cdc.debezium.DebeziumDeserializationSchema; -import com.ververica.cdc.debezium.Validator; - -/** A MySql CDC Connector Source. */ -public class MySqlSource extends ChangeEventHybridSource { - - public MySqlSource( - SourceConfigFactory configFactory, - DebeziumDeserializationSchema deserializationSchema, - OffsetFactory offsetFactory, - SnapshotEventDialect snapshotEventDialect, - StreamingEventDialect streamingEventDialect, - Validator validator, - BaseSchema baseSchema) { - super( - configFactory, - deserializationSchema, - offsetFactory, - snapshotEventDialect, - streamingEventDialect, - validator, - baseSchema); - } - - public MySqlSource( - SourceConfigFactory configFactory, - DebeziumDeserializationSchema deserializationSchema, - OffsetFactory offsetFactory, - SnapshotEventDialect snapshotEventDialect, - StreamingEventDialect streamingEventDialect, - BaseSchema baseSchema) { - super( - configFactory, - deserializationSchema, - offsetFactory, - snapshotEventDialect, - streamingEventDialect, - baseSchema); - } -} diff --git a/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/source/dialect/MysqlDialect.java b/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/source/dialect/MysqlDialect.java deleted file mode 100644 index 5ec90e21897..00000000000 --- a/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/source/dialect/MysqlDialect.java +++ /dev/null @@ -1,97 +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 com.ververica.cdc.connectors.refactor.refactor.source.dialect; - -import org.apache.flink.table.types.DataType; -import org.apache.flink.util.FlinkRuntimeException; - -import com.ververica.cdc.connectors.base.source.config.SourceConfig; -import com.ververica.cdc.connectors.base.source.dialect.Dialect; -import com.ververica.cdc.connectors.base.source.offset.Offset; -import com.ververica.cdc.connectors.refactor.refactor.schema.MySqlSchema; -import com.ververica.cdc.connectors.refactor.refactor.schema.MySqlTypeUtils; -import com.ververica.cdc.connectors.refactor.refactor.source.config.MySqlSourceConfig; -import com.ververica.cdc.connectors.refactor.refactor.source.debezium.DebeziumUtils; -import com.ververica.cdc.connectors.refactor.refactor.source.utils.TableDiscoveryUtils; -import io.debezium.connector.mysql.MySqlConnection; -import io.debezium.jdbc.JdbcConnection; -import io.debezium.relational.Column; -import io.debezium.relational.TableId; -import io.debezium.relational.history.TableChanges; - -import java.sql.SQLException; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -/** This is the base class for {@link MysqlSnapshotDialect} and {@link MysqlStreamingDialect} . */ -public interface MysqlDialect extends Dialect { - - @Override - default DataType fromDbzColumn(Column splitColumn) { - return MySqlTypeUtils.fromDbzColumn(splitColumn); - } - - @Override - default Offset displayCurrentOffset(SourceConfig sourceConfig) { - try (JdbcConnection jdbcConnection = openJdbcConnection(sourceConfig)) { - return DebeziumUtils.currentBinlogOffset(jdbcConnection); - } catch (Exception e) { - throw new FlinkRuntimeException("Read the binlog offset error", e); - } - } - - @Override - default Map discoverCapturedTableSchemas( - SourceConfig sourceConfig) throws SQLException { - final List capturedTableIds = discoverCapturedTables(sourceConfig); - - try (MySqlConnection jdbc = - DebeziumUtils.createMySqlConnection(sourceConfig.getDbzConfiguration())) { - // fetch table schemas - MySqlSchema mySqlSchema = - new MySqlSchema( - (MySqlSourceConfig) sourceConfig, jdbc.isTableIdCaseSensitive()); - Map tableSchemas = new HashMap<>(); - for (TableId tableId : capturedTableIds) { - TableChanges.TableChange tableSchema = mySqlSchema.getTableSchema(jdbc, tableId); - tableSchemas.put(tableId, tableSchema); - } - return tableSchemas; - } - } - - @Override - default List listTables(SourceConfig sourceConfig) throws SQLException { - MySqlSourceConfig mySqlSourceConfig = (MySqlSourceConfig) sourceConfig; - try (JdbcConnection jdbcConnection = openJdbcConnection(sourceConfig)) { - return TableDiscoveryUtils.listTables( - jdbcConnection, mySqlSourceConfig.getTableFilters()); - } - } - - @Override - default boolean isTableIdCaseSensitive(SourceConfig sourceConfig) { - try (JdbcConnection jdbcConnection = openJdbcConnection(sourceConfig)) { - return DebeziumUtils.isTableIdCaseSensitive(jdbcConnection); - } catch (SQLException e) { - throw new FlinkRuntimeException("Error reading MySQL variables: " + e.getMessage(), e); - } - } -} diff --git a/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/source/dialect/MysqlSnapshotDialect.java b/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/source/dialect/MysqlSnapshotDialect.java deleted file mode 100644 index f6ac4489f42..00000000000 --- a/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/source/dialect/MysqlSnapshotDialect.java +++ /dev/null @@ -1,134 +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 com.ververica.cdc.connectors.refactor.refactor.source.dialect; - -import org.apache.flink.table.types.logical.RowType; - -import com.ververica.cdc.connectors.base.source.config.SourceConfig; -import com.ververica.cdc.connectors.base.source.dialect.SnapshotEventDialect; -import com.ververica.cdc.connectors.base.source.internal.connection.PooledDataSourceFactory; -import com.ververica.cdc.connectors.base.source.internal.converter.JdbcSourceRecordConverter; -import com.ververica.cdc.connectors.base.source.split.SnapshotSplit; -import com.ververica.cdc.connectors.refactor.refactor.source.config.MySqlSourceConfig; -import com.ververica.cdc.connectors.refactor.refactor.source.connection.MysqlPooledDataSourceFactory; -import com.ververica.cdc.connectors.refactor.refactor.source.dialect.task.context.MySqlSnapshotReaderContext; -import com.ververica.cdc.connectors.refactor.refactor.source.utils.StatementUtils; -import io.debezium.jdbc.JdbcConnection; -import io.debezium.relational.TableId; -import org.apache.kafka.connect.source.SourceRecord; - -import java.sql.PreparedStatement; -import java.sql.SQLException; -import java.util.List; - -/** A MySql Dialect to handle database event during snapshotting phase. */ -public class MysqlSnapshotDialect extends SnapshotEventDialect implements MysqlDialect { - - private final MysqlPooledDataSourceFactory mysqlPooledDataSourceFactory; - - public MysqlSnapshotDialect(MysqlPooledDataSourceFactory mysqlPooledDataSourceFactory) { - this.mysqlPooledDataSourceFactory = mysqlPooledDataSourceFactory; - } - - @Override - public String buildSplitScanQuery( - TableId tableId, RowType splitKeyType, boolean isFirstSplit, boolean isLastSplit) { - return StatementUtils.buildSplitScanQuery(tableId, splitKeyType, isFirstSplit, isLastSplit); - } - - @Override - public Object[] queryMinMax(JdbcConnection jdbc, TableId tableId, String columnName) - throws SQLException { - return StatementUtils.queryMinMax(jdbc, tableId, columnName); - } - - @Override - public Object queryMin( - JdbcConnection jdbc, TableId tableId, String columnName, Object excludedLowerBound) - throws SQLException { - return StatementUtils.queryMin(jdbc, tableId, columnName, excludedLowerBound); - } - - @Override - public Object queryNextChunkMax( - JdbcConnection jdbc, - TableId tableId, - String columnName, - int chunkSize, - Object includedLowerBound) - throws SQLException { - return StatementUtils.queryNextChunkMax( - jdbc, tableId, columnName, chunkSize, includedLowerBound); - } - - @Override - public Long queryApproximateRowCnt(JdbcConnection jdbc, TableId tableId) throws SQLException { - return StatementUtils.queryApproximateRowCnt(jdbc, tableId); - } - - @Override - public PreparedStatement readTableSplitDataStatement( - String selectSql, SnapshotContext context) { - SnapshotSplit snapshotSplit = context.getSnapshotSplit(); - MySqlSourceConfig mySqlSourceConfig = (MySqlSourceConfig) context.getSourceConfig(); - return StatementUtils.readTableSplitDataStatement( - openJdbcConnection(context.getSourceConfig()), - selectSql, - snapshotSplit.getSplitStart() == null, - snapshotSplit.getSplitEnd() == null, - snapshotSplit.getSplitStart(), - snapshotSplit.getSplitEnd(), - snapshotSplit.getSplitKeyType().getFieldCount(), - mySqlSourceConfig.getMySqlConnectorConfig().getQueryFetchSize()); - } - - @Override - public Task createTask(SnapshotContext statefulTaskContext) { - return new SnapshotEventDialect.SnapshotSplitReadTask(statefulTaskContext); - } - - @Override - public List normalizedSplitRecords( - SnapshotSplit currentSnapshotSplit, List sourceRecords) { - return null; - } - - @Override - public SnapshotContext createSnapshotContext(SourceConfig sourceConfig) { - return new MySqlSnapshotReaderContext(sourceConfig); - } - - @Override - public void close() {} - - @Override - public String getName() { - return null; - } - - @Override - public JdbcSourceRecordConverter getSourceRecordConverter(RowType rowType) { - return null; - } - - @Override - public PooledDataSourceFactory getPooledDataSourceFactory() { - return mysqlPooledDataSourceFactory; - } -} diff --git a/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/source/dialect/MysqlStreamingDialect.java b/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/source/dialect/MysqlStreamingDialect.java deleted file mode 100644 index 6e07fe65950..00000000000 --- a/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/source/dialect/MysqlStreamingDialect.java +++ /dev/null @@ -1,52 +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 com.ververica.cdc.connectors.refactor.refactor.source.dialect; - -import org.apache.flink.table.types.logical.RowType; - -import com.ververica.cdc.connectors.base.source.dialect.StreamingEventDialect; -import com.ververica.cdc.connectors.base.source.internal.connection.PooledDataSourceFactory; -import com.ververica.cdc.connectors.base.source.internal.converter.JdbcSourceRecordConverter; -import com.ververica.cdc.connectors.base.source.split.StreamSplit; - -/** A Mysql Streaming Dialect to handle database event during streaming process. */ -public class MysqlStreamingDialect extends StreamingEventDialect implements MysqlDialect { - @Override - public Task createTask(StreamSplit backfillStreamSplit) { - return null; - } - - @Override - public void close() {} - - @Override - public String getName() { - return null; - } - - @Override - public JdbcSourceRecordConverter getSourceRecordConverter(RowType rowType) { - return null; - } - - @Override - public PooledDataSourceFactory getPooledDataSourceFactory() { - return null; - } -} diff --git a/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/source/dialect/task/context/MySqlSnapshotReaderContext.java b/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/source/dialect/task/context/MySqlSnapshotReaderContext.java deleted file mode 100644 index 398ac2c3ee4..00000000000 --- a/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/source/dialect/task/context/MySqlSnapshotReaderContext.java +++ /dev/null @@ -1,281 +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 com.ververica.cdc.connectors.refactor.refactor.source.dialect.task.context; - -import com.github.shyiko.mysql.binlog.BinaryLogClient; -import com.ververica.cdc.connectors.base.source.config.SourceConfig; -import com.ververica.cdc.connectors.base.source.dialect.SnapshotEventDialect; -import com.ververica.cdc.connectors.base.source.offset.Offset; -import com.ververica.cdc.connectors.base.source.split.SnapshotSplit; -import com.ververica.cdc.connectors.base.source.split.SourceSplitBase; -import com.ververica.cdc.connectors.refactor.refactor.debezium.EmbeddedFlinkDatabaseHistory; -import com.ververica.cdc.connectors.refactor.refactor.debezium.task.context.MySqlTaskContextImpl; -import com.ververica.cdc.connectors.refactor.refactor.source.config.MySqlSourceConfig; -import com.ververica.cdc.connectors.refactor.refactor.source.debezium.DebeziumUtils; -import com.ververica.cdc.connectors.refactor.refactor.source.offset.BinlogOffset; -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.MySqlStreamingChangeEventSourceMetrics; -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.metrics.StreamingChangeEventSourceMetrics; -import io.debezium.pipeline.source.spi.EventMetadataProvider; -import io.debezium.pipeline.spi.OffsetContext; -import io.debezium.relational.TableId; -import io.debezium.schema.DataCollectionId; -import io.debezium.schema.TopicSelector; -import io.debezium.util.Clock; -import io.debezium.util.Collect; -import io.debezium.util.SchemaNameAdjuster; -import org.apache.kafka.connect.data.Struct; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.time.Instant; -import java.util.List; -import java.util.Map; - -/** - * Provides table and database information and {@link SnapshotSplit} during MySql database - * snapshotting. - */ -public class MySqlSnapshotReaderContext implements SnapshotEventDialect.SnapshotContext { - - private static final Logger LOG = LoggerFactory.getLogger(MySqlSnapshotReaderContext.class); - private final Clock clock = Clock.SYSTEM; - - private final MySqlSourceConfig sourceConfig; - private final MySqlConnectorConfig connectorConfig; - private final MySqlEventMetadataProvider metadataProvider; - private final SchemaNameAdjuster schemaNameAdjuster; - private final MySqlConnection connection; - private final BinaryLogClient binaryLogClient; - - private MySqlDatabaseSchema databaseSchema; - private MySqlTaskContextImpl taskContext; - private MySqlOffsetContext offsetContext; - private TopicSelector topicSelector; - private SnapshotChangeEventSourceMetrics snapshotChangeEventSourceMetrics; - private StreamingChangeEventSourceMetrics streamingChangeEventSourceMetrics; - private ChangeEventQueue queue; - private ErrorHandler errorHandler; - private SnapshotSplit currentSnapshotSplit; - - public MySqlSnapshotReaderContext(SourceConfig sourceConfig) { - this.sourceConfig = (MySqlSourceConfig) sourceConfig; - - this.connectorConfig = this.sourceConfig.getMySqlConnectorConfig(); - this.schemaNameAdjuster = SchemaNameAdjuster.create(); - this.metadataProvider = new MySqlEventMetadataProvider(); - this.binaryLogClient = DebeziumUtils.createBinaryClient(sourceConfig.getDbzConfiguration()); - this.connection = DebeziumUtils.createMySqlConnection(sourceConfig.getDbzConfiguration()); - } - - @Override - public SnapshotSplit getSnapshotSplit() { - if (currentSnapshotSplit == null) { - throw new IllegalArgumentException( - "The snapshotSplit is empty, Please confirm whether the configure method has been called."); - } - return currentSnapshotSplit; - } - - @Override - public SourceConfig getSourceConfig() { - return sourceConfig; - } - - @Override - public void configure(SourceSplitBase mySqlSplit) { - // initial stateful objects - final boolean tableIdCaseInsensitive = connection.isTableIdCaseSensitive(); - this.topicSelector = MySqlTopicSelector.defaultSelector(connectorConfig); - this.currentSnapshotSplit = (SnapshotSplit) mySqlSplit; - EmbeddedFlinkDatabaseHistory.registerHistory( - sourceConfig - .getDbzConfiguration() - .getString(EmbeddedFlinkDatabaseHistory.DATABASE_HISTORY_INSTANCE_NAME), - mySqlSplit.getTableSchemas().values()); - this.databaseSchema = - DebeziumUtils.createMySqlDatabaseSchema(connectorConfig, tableIdCaseInsensitive); - this.offsetContext = - loadStartingOffsetState(new MySqlOffsetContext.Loader(connectorConfig), mySqlSplit); - validateAndLoadDatabaseHistory(offsetContext, databaseSchema); - - this.taskContext = - new MySqlTaskContextImpl(connectorConfig, databaseSchema, binaryLogClient); - final int queueSize = - mySqlSplit.isSnapshotSplit() - ? Integer.MAX_VALUE - : connectorConfig.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(); - - final MySqlChangeEventSourceMetricsFactory changeEventSourceMetricsFactory = - new MySqlChangeEventSourceMetricsFactory( - new MySqlStreamingChangeEventSourceMetrics( - taskContext, queue, metadataProvider)); - this.snapshotChangeEventSourceMetrics = - changeEventSourceMetricsFactory.getSnapshotMetrics( - taskContext, queue, metadataProvider); - this.streamingChangeEventSourceMetrics = - changeEventSourceMetricsFactory.getStreamingMetrics( - taskContext, queue, metadataProvider); - this.errorHandler = new MySqlErrorHandler(connectorConfig.getLogicalName(), queue); - } - - private void validateAndLoadDatabaseHistory( - MySqlOffsetContext offset, MySqlDatabaseSchema schema) { - schema.initializeStorage(); - schema.recover(offset); - } - - /** 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 = - (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; - } - - @Override - public ChangeEventQueue getQueue() { - return queue; - } - - /** 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(); - } - } - - public SnapshotChangeEventSourceMetrics getSnapshotChangeEventSourceMetrics() { - snapshotChangeEventSourceMetrics.reset(); - return snapshotChangeEventSourceMetrics; - } - - public StreamingChangeEventSourceMetrics getStreamingChangeEventSourceMetrics() { - streamingChangeEventSourceMetrics.reset(); - return streamingChangeEventSourceMetrics; - } -} diff --git a/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/source/validate/MySqlValidator.java b/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/source/validate/MySqlValidator.java deleted file mode 100644 index ef75380a81f..00000000000 --- a/flink-connector-mysql-cdc-new/src/main/java/com/ververica/cdc/connectors/refactor/refactor/source/validate/MySqlValidator.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 com.ververica.cdc.connectors.refactor.refactor.source.validate; - -import org.apache.flink.table.api.TableException; -import org.apache.flink.table.api.ValidationException; -import org.apache.flink.util.FlinkRuntimeException; - -import com.ververica.cdc.connectors.base.source.dialect.Dialect; -import com.ververica.cdc.connectors.refactor.refactor.source.config.MySqlSourceConfig; -import com.ververica.cdc.connectors.refactor.refactor.source.debezium.DebeziumUtils; -import com.ververica.cdc.debezium.Validator; -import io.debezium.jdbc.JdbcConnection; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.sql.SQLException; -import java.util.Arrays; -import java.util.Properties; - -import static io.debezium.config.Configuration.from; - -/** - * A Mysql Validator to validate the connected mysql database satisfies the cdc connector's - * requirements. - */ -public class MySqlValidator implements Validator { - private static final Logger LOG = LoggerFactory.getLogger(MySqlValidator.class); - private static final long serialVersionUID = 1L; - - private static final String BINLOG_FORMAT_ROW = "ROW"; - private static final String BINLOG_FORMAT_IMAGE_FULL = "FULL"; - - private final Properties dbzProperties; - private final MySqlSourceConfig sourceConfig; - private final Dialect dialect; - - public MySqlValidator(Properties dbzProperties, Dialect dialect) { - this.dbzProperties = dbzProperties; - this.sourceConfig = null; - this.dialect = dialect; - } - - public MySqlValidator(MySqlSourceConfig sourceConfig, Dialect dialect) { - this.dbzProperties = sourceConfig.getDbzProperties(); - this.sourceConfig = sourceConfig; - this.dialect = dialect; - } - - @Override - public void validate() { - JdbcConnection connection = null; - try { - if (sourceConfig != null) { - connection = dialect.openJdbcConnection(sourceConfig); - } else { - // for the legacy source - connection = DebeziumUtils.createMySqlConnection(from(dbzProperties)); - } - checkVersion(connection); - checkBinlogFormat(connection); - checkBinlogRowImage(connection); - } catch (SQLException ex) { - throw new TableException( - "Unexpected error while connecting to MySQL and validating", ex); - } finally { - if (connection != null) { - try { - connection.close(); - } catch (SQLException e) { - throw new FlinkRuntimeException("Closing connection error", e); - } - } - } - LOG.info("MySQL validation passed."); - } - - private void checkVersion(JdbcConnection connection) throws SQLException { - String version = - connection.queryAndMap("SELECT VERSION()", rs -> rs.next() ? rs.getString(1) : ""); - - // Only care about the major version and minor version - Integer[] versionNumbers = - Arrays.stream(version.split("\\.")) - .limit(2) - .map(Integer::new) - .toArray(Integer[]::new); - boolean isSatisfied; - if (versionNumbers[0] > 5) { - isSatisfied = true; - } else if (versionNumbers[0] < 5) { - isSatisfied = false; - } else { - isSatisfied = versionNumbers[1] >= 7; - } - if (!isSatisfied) { - throw new ValidationException( - String.format( - "Currently Flink MySql CDC connector only supports MySql " - + "whose version is larger or equal to 5.7, but actual is %s.%s.", - versionNumbers[0], versionNumbers[1])); - } - } - - /** Check whether the binlog format is ROW. */ - private void checkBinlogFormat(JdbcConnection connection) throws SQLException { - String mode = - connection - .queryAndMap( - "SHOW GLOBAL VARIABLES LIKE 'binlog_format'", - rs -> rs.next() ? rs.getString(2) : "") - .toUpperCase(); - if (!BINLOG_FORMAT_ROW.equals(mode)) { - throw new ValidationException( - String.format( - "The MySQL server is configured with binlog_format %s rather than %s, which is " - + "required for this connector to work properly. Change the MySQL configuration to use a " - + "binlog_format=ROW and restart the connector.", - mode, BINLOG_FORMAT_ROW)); - } - } - - /** Check whether the binlog row image is FULL. */ - private void checkBinlogRowImage(JdbcConnection connection) throws SQLException { - String rowImage = - connection - .queryAndMap( - "SHOW GLOBAL VARIABLES LIKE 'binlog_row_image'", - rs -> { - if (rs.next()) { - return rs.getString(2); - } - // This setting was introduced in MySQL 5.6+ with default of - // 'FULL'. - // For older versions, assume 'FULL'. - return BINLOG_FORMAT_IMAGE_FULL; - }) - .toUpperCase(); - if (!rowImage.equals(BINLOG_FORMAT_IMAGE_FULL)) { - throw new ValidationException( - String.format( - "The MySQL server is configured with binlog_row_image %s rather than %s, which is " - + "required for this connector to work properly. Change the MySQL configuration to use a " - + "binlog_row_image=FULL and restart the connector.", - rowImage, BINLOG_FORMAT_IMAGE_FULL)); - } - } -} diff --git a/flink-connector-mysql-cdc-new/src/test/resources/ddl/column_type_test.sql b/flink-connector-mysql-cdc-new/src/test/resources/ddl/column_type_test.sql deleted file mode 100644 index 61ac5e4907a..00000000000 --- a/flink-connector-mysql-cdc-new/src/test/resources/ddl/column_type_test.sql +++ /dev/null @@ -1,87 +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. - --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: column_type_test --- ---------------------------------------------------------------------------------------------------------------- - -CREATE TABLE full_types ( - id INT AUTO_INCREMENT NOT NULL, - tiny_c TINYINT, - tiny_un_c TINYINT UNSIGNED, - small_c SMALLINT, - small_un_c SMALLINT UNSIGNED, - medium_c MEDIUMINT, - medium_un_c MEDIUMINT UNSIGNED, - int_c INTEGER , - int_un_c INTEGER UNSIGNED, - int11_c INT(11) , - big_c BIGINT, - big_un_c BIGINT UNSIGNED, - varchar_c VARCHAR(255), - char_c CHAR(3), - real_c REAL, - float_c FLOAT, - double_c DOUBLE, - decimal_c DECIMAL(8, 4), - numeric_c NUMERIC(6, 0), - big_decimal_c DECIMAL(65, 1), - bit1_c BIT, - tiny1_c TINYINT(1), - boolean_c BOOLEAN, - date_c DATE, - time_c TIME(0), - datetime3_c DATETIME(3), - datetime6_c DATETIME(6), - timestamp_c TIMESTAMP, - file_uuid BINARY(16), - bit_c BIT(64), - text_c TEXT, - tiny_blob_c TINYBLOB, - blob_c BLOB, - medium_blob_c MEDIUMBLOB, - long_blob_c LONGBLOB, - year_c YEAR, - enum_c enum('red', 'white') default 'red', - set_c SET('a', 'b'), - json_c JSON, - point_c POINT, - geometry_c GEOMETRY, - linestring_c LINESTRING, - polygon_c POLYGON, - multipoint_c MULTIPOINT, - multiline_c MULTILINESTRING, - multipolygon_c MULTIPOLYGON, - geometrycollection_c GEOMETRYCOLLECTION, - PRIMARY KEY (id) -) DEFAULT CHARSET=utf8; - -INSERT INTO full_types VALUES ( - DEFAULT, 127, 255, 32767, 65535, 8388607, 16777215, 2147483647, 4294967295, 2147483647, 9223372036854775807, - 18446744073709551615, - 'Hello World', 'abc', 123.102, 123.102, 404.4443, 123.4567, 345.6, 34567892.1, 0, 1, true, - '2020-07-17', '18:00:22', '2020-07-17 18:00:22.123', '2020-07-17 18:00:22.123456', '2020-07-17 18:00:22', - unhex(replace('651aed08-390f-4893-b2f1-36923e7b7400','-','')), b'0000010000000100000001000000010000000100000001000000010000000100', - 'text',UNHEX(HEX(16)),UNHEX(HEX(16)),UNHEX(HEX(16)),UNHEX(HEX(16)), 2021, - 'red', 'a,b,a', '{"key1": "value1"}', - ST_GeomFromText('POINT(1 1)'), - ST_GeomFromText('POLYGON((1 1, 2 1, 2 2, 1 2, 1 1))'), - ST_GeomFromText('LINESTRING(3 0, 3 3, 3 5)'), - ST_GeomFromText('POLYGON((1 1, 2 1, 2 2, 1 2, 1 1))'), - ST_GeomFromText('MULTIPOINT((1 1),(2 2))'), - ST_GeomFromText('MultiLineString((1 1,2 2,3 3),(4 4,5 5))'), - ST_GeomFromText('MULTIPOLYGON(((0 0, 10 0, 10 10, 0 10, 0 0)), ((5 5, 7 5, 7 7, 5 7, 5 5)))'), - ST_GeomFromText('GEOMETRYCOLLECTION(POINT(10 10), POINT(30 30), LINESTRING(15 15, 20 20))') -); \ No newline at end of file diff --git a/flink-connector-mysql-cdc-new/src/test/resources/ddl/column_type_test_mysql8.sql b/flink-connector-mysql-cdc-new/src/test/resources/ddl/column_type_test_mysql8.sql deleted file mode 100644 index 4a8219528dd..00000000000 --- a/flink-connector-mysql-cdc-new/src/test/resources/ddl/column_type_test_mysql8.sql +++ /dev/null @@ -1,87 +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. - --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: column_type_test --- ---------------------------------------------------------------------------------------------------------------- - -CREATE TABLE full_types ( - id INT AUTO_INCREMENT NOT NULL, - tiny_c TINYINT, - tiny_un_c TINYINT UNSIGNED, - small_c SMALLINT, - small_un_c SMALLINT UNSIGNED, - medium_c MEDIUMINT, - medium_un_c MEDIUMINT UNSIGNED, - int_c INTEGER , - int_un_c INTEGER UNSIGNED, - int11_c INT(11) , - big_c BIGINT, - big_un_c BIGINT UNSIGNED, - varchar_c VARCHAR(255), - char_c CHAR(3), - real_c REAL, - float_c FLOAT, - double_c DOUBLE, - decimal_c DECIMAL(8, 4), - numeric_c NUMERIC(6, 0), - big_decimal_c DECIMAL(65, 1), - bit1_c BIT, - tiny1_c TINYINT(1), - boolean_c BOOLEAN, - date_c DATE, - time_c TIME(0), - datetime3_c DATETIME(3), - datetime6_c DATETIME(6), - timestamp_c TIMESTAMP, - file_uuid BINARY(16), - bit_c BIT(64), - text_c TEXT, - tiny_blob_c TINYBLOB, - blob_c BLOB, - medium_blob_c MEDIUMBLOB, - long_blob_c LONGBLOB, - year_c YEAR, - enum_c enum('red', 'white') default 'red', - set_c SET('a', 'b'), - json_c JSON, - point_c POINT, - geometry_c GEOMETRY, - linestring_c LINESTRING, - polygon_c POLYGON, - multipoint_c MULTIPOINT, - multiline_c MULTILINESTRING, - multipolygon_c MULTIPOLYGON, - geometrycollection_c GEOMCOLLECTION, - PRIMARY KEY (id) -) DEFAULT CHARSET=utf8; - -INSERT INTO full_types VALUES ( - DEFAULT, 127, 255, 32767, 65535, 8388607, 16777215, 2147483647, 4294967295, 2147483647, 9223372036854775807, - 18446744073709551615, - 'Hello World', 'abc', 123.102, 123.102, 404.4443, 123.4567, 345.6, 34567892.1, 0, 1, true, - '2020-07-17', '18:00:22', '2020-07-17 18:00:22.123', '2020-07-17 18:00:22.123456', '2020-07-17 18:00:22', - unhex(replace('651aed08-390f-4893-b2f1-36923e7b7400','-','')), b'0000010000000100000001000000010000000100000001000000010000000100', - 'text',UNHEX(HEX(16)),UNHEX(HEX(16)),UNHEX(HEX(16)),UNHEX(HEX(16)), 2021, - 'red', 'a,b,a', '{"key1": "value1"}', - ST_GeomFromText('POINT(1 1)'), - ST_GeomFromText('POLYGON((1 1, 2 1, 2 2, 1 2, 1 1))'), - ST_GeomFromText('LINESTRING(3 0, 3 3, 3 5)'), - ST_GeomFromText('POLYGON((1 1, 2 1, 2 2, 1 2, 1 1))'), - ST_GeomFromText('MULTIPOINT((1 1),(2 2))'), - ST_GeomFromText('MultiLineString((1 1,2 2,3 3),(4 4,5 5))'), - ST_GeomFromText('MULTIPOLYGON(((0 0, 10 0, 10 10, 0 10, 0 0)), ((5 5, 7 5, 7 7, 5 7, 5 5)))'), - ST_GeomFromText('GEOMETRYCOLLECTION(POINT(10 10), POINT(30 30), LINESTRING(15 15, 20 20))') -); \ No newline at end of file diff --git a/flink-connector-mysql-cdc-new/src/test/resources/ddl/customer.sql b/flink-connector-mysql-cdc-new/src/test/resources/ddl/customer.sql deleted file mode 100644 index 7d3fe892279..00000000000 --- a/flink-connector-mysql-cdc-new/src/test/resources/ddl/customer.sql +++ /dev/null @@ -1,251 +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. - --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: customer --- ---------------------------------------------------------------------------------------------------------------- - --- Create and populate our users using a single insert with many rows -CREATE TABLE customers ( - id INTEGER NOT NULL PRIMARY KEY, - name VARCHAR(255) NOT NULL DEFAULT 'flink', - address VARCHAR(1024), - phone_number VARCHAR(512) -); - -INSERT INTO customers -VALUES (101,"user_1","Shanghai","123567891234"), - (102,"user_2","Shanghai","123567891234"), - (103,"user_3","Shanghai","123567891234"), - (109,"user_4","Shanghai","123567891234"), - (110,"user_5","Shanghai","123567891234"), - (111,"user_6","Shanghai","123567891234"), - (118,"user_7","Shanghai","123567891234"), - (121,"user_8","Shanghai","123567891234"), - (123,"user_9","Shanghai","123567891234"), - (1009,"user_10","Shanghai","123567891234"), - (1010,"user_11","Shanghai","123567891234"), - (1011,"user_12","Shanghai","123567891234"), - (1012,"user_13","Shanghai","123567891234"), - (1013,"user_14","Shanghai","123567891234"), - (1014,"user_15","Shanghai","123567891234"), - (1015,"user_16","Shanghai","123567891234"), - (1016,"user_17","Shanghai","123567891234"), - (1017,"user_18","Shanghai","123567891234"), - (1018,"user_19","Shanghai","123567891234"), - (1019,"user_20","Shanghai","123567891234"), - (2000,"user_21","Shanghai","123567891234"); - --- table has same name prefix with 'customers.*' -CREATE TABLE customers_1 ( - id INTEGER NOT NULL PRIMARY KEY, - name VARCHAR(255) NOT NULL DEFAULT 'flink', - address VARCHAR(1024), - phone_number VARCHAR(512) -); - -INSERT INTO customers_1 -VALUES (101,"user_1","Shanghai","123567891234"), - (102,"user_2","Shanghai","123567891234"), - (103,"user_3","Shanghai","123567891234"), - (109,"user_4","Shanghai","123567891234"), - (110,"user_5","Shanghai","123567891234"), - (111,"user_6","Shanghai","123567891234"), - (118,"user_7","Shanghai","123567891234"), - (121,"user_8","Shanghai","123567891234"), - (123,"user_9","Shanghai","123567891234"), - (1009,"user_10","Shanghai","123567891234"), - (1010,"user_11","Shanghai","123567891234"), - (1011,"user_12","Shanghai","123567891234"), - (1012,"user_13","Shanghai","123567891234"), - (1013,"user_14","Shanghai","123567891234"), - (1014,"user_15","Shanghai","123567891234"), - (1015,"user_16","Shanghai","123567891234"), - (1016,"user_17","Shanghai","123567891234"), - (1017,"user_18","Shanghai","123567891234"), - (1018,"user_19","Shanghai","123567891234"), - (1019,"user_20","Shanghai","123567891234"), - (2000,"user_21","Shanghai","123567891234"); - --- create table whose split key is evenly distributed -CREATE TABLE customers_even_dist ( - id INTEGER NOT NULL PRIMARY KEY, - name VARCHAR(255) NOT NULL , - address VARCHAR(1024), - phone_number VARCHAR(512) -); -INSERT INTO customers_even_dist -VALUES (101,'user_1','Shanghai','123567891234'), - (102,'user_2','Shanghai','123567891234'), - (103,'user_3','Shanghai','123567891234'), - (104,'user_4','Shanghai','123567891234'), - (105,'user_5','Shanghai','123567891234'), - (106,'user_6','Shanghai','123567891234'), - (107,'user_7','Shanghai','123567891234'), - (108,'user_8','Shanghai','123567891234'), - (109,'user_9','Shanghai','123567891234'), - (110,'user_10','Shanghai','123567891234'); - --- create table whose split key is evenly distributed and sparse -CREATE TABLE customers_sparse_dist ( - id INTEGER NOT NULL PRIMARY KEY, - name VARCHAR(255) NOT NULL , - address VARCHAR(1024), - phone_number VARCHAR(512) -); -INSERT INTO customers_sparse_dist -VALUES (2,'user_1','Shanghai','123567891234'), - (4,'user_2','Shanghai','123567891234'), - (6,'user_3','Shanghai','123567891234'), - (8,'user_4','Shanghai','123567891234'), - (10,'user_5','Shanghai','123567891234'), - (16,'user_6','Shanghai','123567891234'), - (17,'user_7','Shanghai','123567891234'), - (18,'user_8','Shanghai','123567891234'), - (20,'user_9','Shanghai','123567891234'), - (22,'user_10','Shanghai','123567891234'); - --- create table whose split key is evenly distributed and dense -CREATE TABLE customers_dense_dist ( - id1 INTEGER NOT NULL, - id2 VARCHAR(255) NOT NULL , - address VARCHAR(1024), - phone_number VARCHAR(512), - PRIMARY KEY(id1, id2) -); -INSERT INTO customers_dense_dist -VALUES (1,'user_1','Shanghai','123567891234'), - (1,'user_2','Shanghai','123567891234'), - (1,'user_3','Shanghai','123567891234'), - (1,'user_4','Shanghai','123567891234'), - (2,'user_5','Shanghai','123567891234'), - (2,'user_6','Shanghai','123567891234'), - (2,'user_7','Shanghai','123567891234'), - (3,'user_8','Shanghai','123567891234'), - (3,'user_9','Shanghai','123567891234'), - (3,'user_10','Shanghai','123567891234'); - --- table has combined primary key -CREATE TABLE customer_card ( - card_no BIGINT NOT NULL, - level VARCHAR(10) NOT NULL, - name VARCHAR(255) NOT NULL DEFAULT 'flink', - note VARCHAR(1024), - PRIMARY KEY(card_no, level) -); - -insert into customer_card -VALUES (20001, 'LEVEL_4', 'user_1', 'user with level 4'), - (20002, 'LEVEL_4', 'user_2', 'user with level 4'), - (20003, 'LEVEL_4', 'user_3', 'user with level 4'), - (20004, 'LEVEL_4', 'user_4', 'user with level 4'), - (20004, 'LEVEL_1', 'user_4', 'user with level 4'), - (20004, 'LEVEL_2', 'user_4', 'user with level 4'), - (20004, 'LEVEL_3', 'user_4', 'user with level 4'), - (30006, 'LEVEL_3', 'user_5', 'user with level 3'), - (30007, 'LEVEL_3', 'user_6', 'user with level 3'), - (30008, 'LEVEL_3', 'user_7', 'user with level 3'), - (30009, 'LEVEL_3', 'user_8', 'user with level 3'), - (30009, 'LEVEL_2', 'user_8', 'user with level 3'), - (30009, 'LEVEL_1', 'user_8', 'user with level 3'), - (40001, 'LEVEL_2', 'user_9', 'user with level 2'), - (40002, 'LEVEL_2', 'user_10', 'user with level 2'), - (40003, 'LEVEL_2', 'user_11', 'user with level 2'), - (50001, 'LEVEL_1', 'user_12', 'user with level 1'), - (50002, 'LEVEL_1', 'user_13', 'user with level 1'), - (50003, 'LEVEL_1', 'user_14', 'user with level 1'); - --- table has single line -CREATE TABLE customer_card_single_line ( - card_no BIGINT NOT NULL, - level VARCHAR(10) NOT NULL, - name VARCHAR(255) NOT NULL DEFAULT 'flink', - note VARCHAR(1024), - PRIMARY KEY(card_no, level) -); - -insert into customer_card_single_line -VALUES (20001, 'LEVEL_1', 'user_1', 'user with level 1'); - - --- table has combined primary key -CREATE TABLE shopping_cart ( - product_no INT NOT NULL, - product_kind VARCHAR(255), - user_id VARCHAR(255) NOT NULL, - description VARCHAR(255) NOT NULL, - PRIMARY KEY(user_id, product_no, product_kind) -); - -insert into shopping_cart -VALUES (101, 'KIND_001', 'user_1', 'my shopping cart'), - (101, 'KIND_002', 'user_1', 'my shopping cart'), - (102, 'KIND_007', 'user_1', 'my shopping cart'), - (102, 'KIND_008', 'user_1', 'my shopping cart'), - (501, 'KIND_100', 'user_2', 'my shopping list'), - (701, 'KIND_999', 'user_3', 'my shopping list'), - (801, 'KIND_010', 'user_4', 'my shopping list'), - (600, 'KIND_009', 'user_4', 'my shopping list'), - (401, 'KIND_002', 'user_5', 'leo list'), - (401, 'KIND_007', 'user_5', 'leo list'), - (404, 'KIND_008', 'user_5', 'leo list'), - (600, 'KIND_009', 'user_6', 'my shopping cart'); - --- table has bigint unsigned auto increment primary key -CREATE TABLE shopping_cart_big ( - product_no BIGINT UNSIGNED AUTO_INCREMENT NOT NULL, - product_kind VARCHAR(255), - user_id VARCHAR(255) NOT NULL, - description VARCHAR(255) NOT NULL, - PRIMARY KEY(product_no) -); - -insert into shopping_cart_big -VALUES (default, 'KIND_001', 'user_1', 'my shopping cart'), - (default, 'KIND_002', 'user_1', 'my shopping cart'), - (default, 'KIND_003', 'user_1', 'my shopping cart'); - --- table has decimal primary key -CREATE TABLE shopping_cart_dec ( - product_no DECIMAL(10, 4) NOT NULL, - product_kind VARCHAR(255), - user_id VARCHAR(255) NOT NULL, - description VARCHAR(255) DEFAULT 'flink', - PRIMARY KEY(product_no) -); - -insert into shopping_cart_dec -VALUES (123456.123, 'KIND_001', 'user_1', 'my shopping cart'), - (123457.456, 'KIND_002', 'user_2', 'my shopping cart'), - (123458.6789, 'KIND_003', 'user_3', 'my shopping cart'), - (123459.1234, 'KIND_004', 'user_4', null); - --- create table whose primary key are produced by snowflake algorithm -CREATE TABLE address ( - id BIGINT UNSIGNED NOT NULL PRIMARY KEY, - country VARCHAR(255) NOT NULL, - city VARCHAR(255) NOT NULL, - detail_address VARCHAR(1024) -); - -INSERT INTO address -VALUES (416874195632735147, 'China', 'Beijing', 'West Town address 1'), - (416927583791428523, 'China', 'Beijing', 'West Town address 2'), - (417022095255614379, 'China', 'Beijing', 'West Town address 3'), - (417111867899200427, 'America', 'New York', 'East Town address 1'), - (417271541558096811, 'America', 'New York', 'East Town address 2'), - (417272886855938987, 'America', 'New York', 'East Town address 3'), - (417420106184475563, 'Germany', 'Berlin', 'West Town address 1'), - (418161258277847979, 'Germany', 'Berlin', 'West Town address 2'); diff --git a/flink-connector-mysql-cdc-new/src/test/resources/ddl/user_1.sql b/flink-connector-mysql-cdc-new/src/test/resources/ddl/user_1.sql deleted file mode 100644 index 15be9479c42..00000000000 --- a/flink-connector-mysql-cdc-new/src/test/resources/ddl/user_1.sql +++ /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. - --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: user_1 --- ---------------------------------------------------------------------------------------------------------------- - --- Create user_table_1_1 table -CREATE TABLE user_table_1_1 ( - id INTEGER NOT NULL PRIMARY KEY, - name VARCHAR(255) NOT NULL DEFAULT 'flink', - address VARCHAR(1024), - phone_number VARCHAR(512), - email VARCHAR(255) -); - --- Create user_table_1_2 table -CREATE TABLE user_table_1_2 ( - id INTEGER NOT NULL PRIMARY KEY, - name VARCHAR(255) NOT NULL DEFAULT 'flink', - address VARCHAR(1024), - phone_number VARCHAR(512) -); - -INSERT INTO user_table_1_1 -VALUES (111,"user_111","Shanghai","123567891234","user_111@foo.com"); - -INSERT INTO user_table_1_2 -VALUES (121,"user_121","Shanghai","123567891234"); \ No newline at end of file diff --git a/flink-connector-mysql-cdc-new/src/test/resources/ddl/user_2.sql b/flink-connector-mysql-cdc-new/src/test/resources/ddl/user_2.sql deleted file mode 100644 index 1c41fc5e417..00000000000 --- a/flink-connector-mysql-cdc-new/src/test/resources/ddl/user_2.sql +++ /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. - --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: user_2 --- ---------------------------------------------------------------------------------------------------------------- - --- Create user_table_2_1 table -CREATE TABLE user_table_2_1 ( - id INTEGER NOT NULL PRIMARY KEY, - name VARCHAR(255) NOT NULL DEFAULT 'flink', - address VARCHAR(1024), - phone_number VARCHAR(512) -); - --- Create user_table_2_2 table -CREATE TABLE user_table_2_2 ( - id INTEGER NOT NULL PRIMARY KEY, - name VARCHAR(255) NOT NULL DEFAULT 'flink', - address VARCHAR(1024), - phone_number VARCHAR(512), - age INTEGER -); - -INSERT INTO user_table_2_1 -VALUES (211,"user_211","Shanghai","123567891234"); - -INSERT INTO user_table_2_2 -VALUES (221,"user_221","Shanghai","123567891234", 18); \ No newline at end of file diff --git a/flink-connector-mysql-cdc-new/src/test/resources/docker/server/my.cnf b/flink-connector-mysql-cdc-new/src/test/resources/docker/server/my.cnf deleted file mode 100644 index 2078f4604eb..00000000000 --- a/flink-connector-mysql-cdc-new/src/test/resources/docker/server/my.cnf +++ /dev/null @@ -1,59 +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. - -# For advice on how to change settings please see -# http://dev.mysql.com/doc/refman/5.7/en/server-configuration-defaults.html - -[mysqld] -# -# Remove leading # and set to the amount of RAM for the most important data -# cache in MySQL. Start at 70% of total RAM for dedicated server, else 10%. -# innodb_buffer_pool_size = 128M -# -# Remove leading # to turn on a very important data integrity option: logging -# changes to the binary log between backups. -# log_bin -# -# Remove leading # to set options mainly useful for reporting servers. -# The server defaults are faster for transactions and fast SELECTs. -# Adjust sizes as needed, experiment to find the optimal values. -# join_buffer_size = 128M -# sort_buffer_size = 2M -# read_rnd_buffer_size = 2M -skip-host-cache -skip-name-resolve -#datadir=/var/lib/mysql -#socket=/var/lib/mysql/mysql.sock -secure-file-priv=/var/lib/mysql -user=mysql - -# Disabling symbolic-links is recommended to prevent assorted security risks -symbolic-links=0 - -#log-error=/var/log/mysqld.log -#pid-file=/var/run/mysqld/mysqld.pid - -# ---------------------------------------------- -# Enable the binlog for replication & CDC -# ---------------------------------------------- - -# Enable binary replication log and set the prefix, expiration, and log format. -# The prefix is arbitrary, expiration can be short for integration tests but would -# be longer on a production system. Row-level info is required for ingest to work. -# Server ID is required, but this will vary on production systems -server-id = 223344 -log_bin = mysql-bin -expire_logs_days = 1 -binlog_format = row diff --git a/flink-connector-mysql-cdc-new/src/test/resources/file/debezium-data-schema-exclude.json b/flink-connector-mysql-cdc-new/src/test/resources/file/debezium-data-schema-exclude.json deleted file mode 100644 index 7ae4643dfd0..00000000000 --- a/flink-connector-mysql-cdc-new/src/test/resources/file/debezium-data-schema-exclude.json +++ /dev/null @@ -1,158 +0,0 @@ -{ - "expected_snapshot": { - "before": null, - "after": { - "id": 1, - "tiny_c": 127, - "tiny_un_c": 255, - "small_c": 32767, - "small_un_c": 65535, - "medium_c": 8388607, - "medium_un_c": 16777215, - "int_c": 2147483647, - "int_un_c": 4294967295, - "int11_c": 2147483647, - "big_c": 9223372036854775807, - "big_un_c": "AP//////////", - "varchar_c": "Hello World", - "char_c": "abc", - "real_c": 123.102, - "float_c": 123.10199737548828, - "double_c": 404.4443, - "decimal_c": "EtaH", - "numeric_c": "AVo=", - "big_decimal_c": "FJqkSQ==", - "bit1_c": false, - "tiny1_c": 1, - "boolean_c": 1, - "date_c": 18460, - "time_c": 64822000000, - "datetime3_c": 1595008822123, - "datetime6_c": 1595008822123456, - "timestamp_c": "2020-07-17T18:00:22Z", - "file_uuid": "ZRrtCDkPSJOy8TaSPnt0AA==", - "bit_c": "BAQEBAQEBAQ=", - "text_c": "text", - "tiny_blob_c": "EA==", - "blob_c": "EA==", - "medium_blob_c": "EA==", - "long_blob_c": "EA==", - "year_c": 2021, - "enum_c": "red", - "set_c": "a,b", - "json_c": "{\"key1\": \"value1\"}", - "point_c": {"x":1.0,"y":1.0,"wkb":"AQEAAAAAAAAAAADwPwAAAAAAAPA/"}, - "geometry_c":{"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/"}, - "linestring_c": {"wkb":"AQIAAAADAAAAAAAAAAAACEAAAAAAAAAAAAAAAAAAAAhAAAAAAAAACEAAAAAAAAAIQAAAAAAAABRA"}, - "polygon_c": {"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/"}, - "multipoint_c": {"wkb":"AQQAAAACAAAAAQEAAAAAAAAAAADwPwAAAAAAAPA/AQEAAAAAAAAAAAAAQAAAAAAAAABA"}, - "multiline_c": {"wkb":"AQUAAAACAAAAAQIAAAADAAAAAAAAAAAA8D8AAAAAAADwPwAAAAAAAABAAAAAAAAAAEAAAAAAAAAIQAAAAAAAAAhAAQIAAAACAAAAAAAAAAAAEEAAAAAAAAAQQAAAAAAAABRAAAAAAAAAFEA="}, - "multipolygon_c": {"wkb":"AQYAAAACAAAAAQMAAAABAAAABQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAAAAAAAAAAAAAAAQMAAAABAAAABQAAAAAAAAAAABRAAAAAAAAAFEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAUQAAAAAAAABRA"}, - "geometrycollection_c": {"wkb":"AQcAAAADAAAAAQEAAAAAAAAAAAAkQAAAAAAAACRAAQEAAAAAAAAAAAA+QAAAAAAAAD5AAQIAAAACAAAAAAAAAAAALkAAAAAAAAAuQAAAAAAAADRAAAAAAAAANEA="} - }, - "op": "r", - "transaction": null - }, - "expected_binlog": { - "before": { - "id": 1, - "tiny_c": 127, - "tiny_un_c": 255, - "small_c": 32767, - "small_un_c": 65535, - "medium_c": 8388607, - "medium_un_c": 16777215, - "int_c": 2147483647, - "int_un_c": 4294967295, - "int11_c": 2147483647, - "big_c": 9223372036854775807, - "big_un_c": "AP//////////", - "varchar_c": "Hello World", - "char_c": "abc", - "real_c": 123.102, - "float_c": 123.10199737548828, - "double_c": 404.4443, - "decimal_c": "EtaH", - "numeric_c": "AVo=", - "big_decimal_c": "FJqkSQ==", - "bit1_c": false, - "tiny1_c": 1, - "boolean_c": 1, - "date_c": 18460, - "time_c": 64822000000, - "datetime3_c": 1595008822123, - "datetime6_c": 1595008822123456, - "timestamp_c": "2020-07-17T18:00:22Z", - "file_uuid": "ZRrtCDkPSJOy8TaSPnt0AA==", - "bit_c": "BAQEBAQEBAQ=", - "text_c": "text", - "tiny_blob_c": "EA==", - "blob_c": "EA==", - "medium_blob_c": "EA==", - "long_blob_c": "EA==", - "year_c": 2021, - "enum_c": "red", - "set_c": "a,b", - "json_c": "{\"key1\":\"value1\"}", - "point_c": {"x":1.0,"y":1.0,"wkb":"AQEAAAAAAAAAAADwPwAAAAAAAPA/"}, - "geometry_c":{"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/"}, - "linestring_c": {"wkb":"AQIAAAADAAAAAAAAAAAACEAAAAAAAAAAAAAAAAAAAAhAAAAAAAAACEAAAAAAAAAIQAAAAAAAABRA"}, - "polygon_c": {"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/"}, - "multipoint_c": {"wkb":"AQQAAAACAAAAAQEAAAAAAAAAAADwPwAAAAAAAPA/AQEAAAAAAAAAAAAAQAAAAAAAAABA"}, - "multiline_c": {"wkb":"AQUAAAACAAAAAQIAAAADAAAAAAAAAAAA8D8AAAAAAADwPwAAAAAAAABAAAAAAAAAAEAAAAAAAAAIQAAAAAAAAAhAAQIAAAACAAAAAAAAAAAAEEAAAAAAAAAQQAAAAAAAABRAAAAAAAAAFEA="}, - "multipolygon_c": {"wkb":"AQYAAAACAAAAAQMAAAABAAAABQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAAAAAAAAAAAAAAAQMAAAABAAAABQAAAAAAAAAAABRAAAAAAAAAFEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAUQAAAAAAAABRA"}, - "geometrycollection_c": {"wkb":"AQcAAAADAAAAAQEAAAAAAAAAAAAkQAAAAAAAACRAAQEAAAAAAAAAAAA+QAAAAAAAAD5AAQIAAAACAAAAAAAAAAAALkAAAAAAAAAuQAAAAAAAADRAAAAAAAAANEA="} - }, - "after": { - "id": 1, - "tiny_c": 127, - "tiny_un_c": 255, - "small_c": 32767, - "small_un_c": 65535, - "medium_c": 8388607, - "medium_un_c": 16777215, - "int_c": 2147483647, - "int_un_c": 4294967295, - "int11_c": 2147483647, - "big_c": 9223372036854775807, - "big_un_c": "AP//////////", - "varchar_c": "Hello World", - "char_c": "abc", - "real_c": 123.102, - "float_c": 123.10199737548828, - "double_c": 404.4443, - "decimal_c": "EtaH", - "numeric_c": "AVo=", - "big_decimal_c": "FJqkSQ==", - "bit1_c": false, - "tiny1_c": 1, - "boolean_c": 1, - "date_c": 18460, - "time_c": 64822000000, - "datetime3_c": 1595008822123, - "datetime6_c": 1595008822123456, - "timestamp_c": "2020-07-17T18:33:22Z", - "file_uuid": "ZRrtCDkPSJOy8TaSPnt0AA==", - "bit_c": "BAQEBAQEBAQ=", - "text_c": "text", - "tiny_blob_c": "EA==", - "blob_c": "EA==", - "medium_blob_c": "EA==", - "long_blob_c": "EA==", - "year_c": 2021, - "enum_c": "red", - "set_c": "a,b", - "json_c": "{\"key1\":\"value1\"}", - "point_c": {"x":1.0,"y":1.0,"wkb":"AQEAAAAAAAAAAADwPwAAAAAAAPA/"}, - "geometry_c":{"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/"}, - "linestring_c": {"wkb":"AQIAAAADAAAAAAAAAAAACEAAAAAAAAAAAAAAAAAAAAhAAAAAAAAACEAAAAAAAAAIQAAAAAAAABRA"}, - "polygon_c": {"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/"}, - "multipoint_c": {"wkb":"AQQAAAACAAAAAQEAAAAAAAAAAADwPwAAAAAAAPA/AQEAAAAAAAAAAAAAQAAAAAAAAABA"}, - "multiline_c": {"wkb":"AQUAAAACAAAAAQIAAAADAAAAAAAAAAAA8D8AAAAAAADwPwAAAAAAAABAAAAAAAAAAEAAAAAAAAAIQAAAAAAAAAhAAQIAAAACAAAAAAAAAAAAEEAAAAAAAAAQQAAAAAAAABRAAAAAAAAAFEA="}, - "multipolygon_c": {"wkb":"AQYAAAACAAAAAQMAAAABAAAABQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAAAAAAAAAAAAAAAQMAAAABAAAABQAAAAAAAAAAABRAAAAAAAAAFEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAUQAAAAAAAABRA"}, - "geometrycollection_c": {"wkb":"AQcAAAADAAAAAQEAAAAAAAAAAAAkQAAAAAAAACRAAQEAAAAAAAAAAAA+QAAAAAAAAD5AAQIAAAACAAAAAAAAAAAALkAAAAAAAAAuQAAAAAAAADRAAAAAAAAANEA="} - }, - "op": "u", - "transaction": null - } -} \ No newline at end of file diff --git a/flink-connector-mysql-cdc-new/src/test/resources/file/debezium-data-schema-include.json b/flink-connector-mysql-cdc-new/src/test/resources/file/debezium-data-schema-include.json deleted file mode 100644 index c192b381a56..00000000000 --- a/flink-connector-mysql-cdc-new/src/test/resources/file/debezium-data-schema-include.json +++ /dev/null @@ -1,1141 +0,0 @@ -{ - "expected_snapshot": { - "schema": { - "type": "struct", - "fields": [ - { - "type": "struct", - "fields": [ - { - "type": "int32", - "optional": false, - "field": "id" - }, - { - "type": "int16", - "optional": true, - "field": "tiny_c" - }, - { - "type": "int16", - "optional": true, - "field": "tiny_un_c" - }, - { - "type": "int16", - "optional": true, - "field": "small_c" - }, - { - "type": "int32", - "optional": true, - "field": "small_un_c" - }, - { - "type": "int32", - "optional": true, - "field": "medium_c" - }, - { - "type": "int32", - "optional": true, - "field": "medium_un_c" - }, - { - "type": "int32", - "optional": true, - "field": "int_c" - }, - { - "type": "int64", - "optional": true, - "field": "int_un_c" - }, - { - "type": "int32", - "optional": true, - "field": "int11_c" - }, - { - "type": "int64", - "optional": true, - "field": "big_c" - }, - { - "type": "int64", - "optional": true, - "field": "big_un_c" - }, - { - "type": "string", - "optional": true, - "field": "varchar_c" - }, - { - "type": "string", - "optional": true, - "field": "char_c" - }, - { - "type": "double", - "optional": true, - "field": "real_c" - }, - { - "type": "double", - "optional": true, - "field": "float_c" - }, - { - "type": "double", - "optional": true, - "field": "double_c" - }, - { - "type": "bytes", - "optional": true, - "name": "org.apache.kafka.connect.data.Decimal", - "version": 1, - "parameters": { - "scale": "4", - "connect.decimal.precision": "8" - }, - "field": "decimal_c" - }, - { - "type": "bytes", - "optional": true, - "name": "org.apache.kafka.connect.data.Decimal", - "version": 1, - "parameters": { - "scale": "0", - "connect.decimal.precision": "6" - }, - "field": "numeric_c" - }, - { - "type": "bytes", - "optional": true, - "name": "org.apache.kafka.connect.data.Decimal", - "version": 1, - "parameters": { - "scale": "65", - "connect.decimal.precision": "1" - }, - "field": "big_decimal_c" - }, - { - "type": "int16", - "optional": true, - "field": "bit1_c" - }, - { - "type": "int16", - "optional": true, - "field": "tiny1_c" - }, - { - "type": "int16", - "optional": true, - "field": "boolean_c" - }, - { - "type": "int32", - "optional": true, - "name": "io.debezium.time.Date", - "version": 1, - "field": "date_c" - }, - { - "type": "int64", - "optional": true, - "name": "io.debezium.time.MicroTime", - "version": 1, - "field": "time_c" - }, - { - "type": "int64", - "optional": true, - "name": "io.debezium.time.Timestamp", - "version": 1, - "field": "datetime3_c" - }, - { - "type": "int64", - "optional": true, - "name": "io.debezium.time.MicroTimestamp", - "version": 1, - "field": "datetime6_c" - }, - { - "type": "string", - "optional": false, - "name": "io.debezium.time.ZonedTimestamp", - "version": 1, - "default": "1970-01-01T00:00:00Z", - "field": "timestamp_c" - }, - { - "type": "bytes", - "optional": true, - "field": "file_uuid" - }, - { - "type": "bytes", - "optional": true, - "field": "bit_c" - }, - { - "type": "string", - "optional": true, - "field": "text_c" - }, - { - "type": "bytes", - "optional": true, - "field": "tiny_blob_c" - }, - { - "type": "bytes", - "optional": true, - "field": "blob_c" - }, - { - "type": "bytes", - "optional": true, - "field": "medium_blob_c" - }, - { - "type": "bytes", - "optional": true, - "field": "long_blob_c" - }, - { - "type": "int32", - "optional": true, - "field": "year_c" - }, - { - "type": "string", - "optional": true, - "field": "enum_c" - }, - { - "type": "string", - "optional": true, - "field": "set_c" - }, - { - "type": "string", - "optional": true, - "field": "json_c" - }, - { - "type": "string", - "optional": true, - "field": "point_c" - }, - { - "type": "string", - "optional": true, - "field": "geometry_c" - }, - { - "type": "string", - "optional": true, - "field": "linestring_c" - }, - { - "type": "string", - "optional": true, - "field": "polygon_c" - }, - { - "type": "string", - "optional": true, - "field": "multipoint_c" - }, - { - "type": "string", - "optional": true, - "field": "multiline_c" - }, - { - "type": "string", - "optional": true, - "field": "multipolygon_c" - }, - { - "type": "string", - "optional": true, - "field": "geometrycollection_c" - } - ], - "optional": true, - "name": "mysql_binlog_source.column_type_test_2jid2b.full_types.Value", - "field": "before" - }, - { - "type": "struct", - "fields": [ - { - "type": "int32", - "optional": false, - "field": "id" - }, - { - "type": "int16", - "optional": true, - "field": "tiny_c" - }, - { - "type": "int16", - "optional": true, - "field": "tiny_un_c" - }, - { - "type": "int16", - "optional": true, - "field": "small_c" - }, - { - "type": "int32", - "optional": true, - "field": "small_un_c" - }, - { - "type": "int32", - "optional": true, - "field": "medium_c" - }, - { - "type": "int32", - "optional": true, - "field": "medium_un_c" - }, - { - "type": "int32", - "optional": true, - "field": "int_c" - }, - { - "type": "int64", - "optional": true, - "field": "int_un_c" - }, - { - "type": "int32", - "optional": true, - "field": "int11_c" - }, - { - "type": "int64", - "optional": true, - "field": "big_c" - }, - { - "type": "int64", - "optional": true, - "field": "big_un_c" - }, - { - "type": "string", - "optional": true, - "field": "varchar_c" - }, - { - "type": "string", - "optional": true, - "field": "char_c" - }, - { - "type": "double", - "optional": true, - "field": "real_c" - }, - { - "type": "double", - "optional": true, - "field": "float_c" - }, - { - "type": "double", - "optional": true, - "field": "double_c" - }, - { - "type": "bytes", - "optional": true, - "name": "org.apache.kafka.connect.data.Decimal", - "version": 1, - "parameters": { - "scale": "4", - "connect.decimal.precision": "8" - }, - "field": "decimal_c" - }, - { - "type": "bytes", - "optional": true, - "name": "org.apache.kafka.connect.data.Decimal", - "version": 1, - "parameters": { - "scale": "0", - "connect.decimal.precision": "6" - }, - "field": "numeric_c" - }, - { - "type": "bytes", - "optional": true, - "name": "org.apache.kafka.connect.data.Decimal", - "version": 1, - "parameters": { - "scale": "65", - "connect.decimal.precision": "1" - }, - "field": "big_decimal_c" - }, - { - "type": "int16", - "optional": true, - "field": "bit1_c" - }, - { - "type": "int16", - "optional": true, - "field": "tiny1_c" - }, - { - "type": "int16", - "optional": true, - "field": "boolean_c" - }, - { - "type": "int32", - "optional": true, - "name": "io.debezium.time.Date", - "version": 1, - "field": "date_c" - }, - { - "type": "int64", - "optional": true, - "name": "io.debezium.time.MicroTime", - "version": 1, - "field": "time_c" - }, - { - "type": "int64", - "optional": true, - "name": "io.debezium.time.Timestamp", - "version": 1, - "field": "datetime3_c" - }, - { - "type": "int64", - "optional": true, - "name": "io.debezium.time.MicroTimestamp", - "version": 1, - "field": "datetime6_c" - }, - { - "type": "string", - "optional": false, - "name": "io.debezium.time.ZonedTimestamp", - "version": 1, - "default": "1970-01-01T00:00:00Z", - "field": "timestamp_c" - }, - { - "type": "bytes", - "optional": true, - "field": "file_uuid" - }, - { - "type": "bytes", - "optional": true, - "field": "bit_c" - }, - { - "type": "string", - "optional": true, - "field": "text_c" - }, - { - "type": "bytes", - "optional": true, - "field": "tiny_blob_c" - }, - { - "type": "bytes", - "optional": true, - "field": "blob_c" - }, - { - "type": "bytes", - "optional": true, - "field": "medium_blob_c" - }, - { - "type": "bytes", - "optional": true, - "field": "long_blob_c" - }, - { - "type": "int32", - "optional": true, - "field": "year_c" - }, - { - "type": "string", - "optional": true, - "field": "enum_c" - }, - { - "type": "string", - "optional": true, - "field": "set_c" - }, - { - "type": "string", - "optional": true, - "field": "json_c" - }, - { - "type": "string", - "optional": true, - "field": "point_c" - }, - { - "type": "string", - "optional": true, - "field": "geometry_c" - }, - { - "type": "string", - "optional": true, - "field": "linestring_c" - }, - { - "type": "string", - "optional": true, - "field": "polygon_c" - }, - { - "type": "string", - "optional": true, - "field": "multipoint_c" - }, - { - "type": "string", - "optional": true, - "field": "multiline_c" - }, - { - "type": "string", - "optional": true, - "field": "multipolygon_c" - }, - { - "type": "string", - "optional": true, - "field": "geometrycollection_c" - } - ], - "optional": true, - "name": "mysql_binlog_source.column_type_test_2jid2b.full_types.Value", - "field": "after" - }, - { - "type": "struct", - "fields": [ - { - "type": "string", - "optional": false, - "field": "version" - }, - { - "type": "string", - "optional": false, - "field": "connector" - }, - { - "type": "string", - "optional": false, - "field": "name" - }, - { - "type": "int64", - "optional": false, - "field": "ts_ms" - }, - { - "type": "string", - "optional": true, - "name": "io.debezium.data.Enum", - "version": 1, - "parameters": { - "allowed": "true,last,false" - }, - "default": "false", - "field": "snapshot" - }, - { - "type": "string", - "optional": false, - "field": "db" - }, - { - "type": "string", - "optional": true, - "field": "sequence" - }, - { - "type": "string", - "optional": true, - "field": "table" - }, - { - "type": "int64", - "optional": false, - "field": "server_id" - }, - { - "type": "string", - "optional": true, - "field": "gtid" - }, - { - "type": "string", - "optional": false, - "field": "file" - }, - { - "type": "int64", - "optional": false, - "field": "pos" - }, - { - "type": "int32", - "optional": false, - "field": "row" - }, - { - "type": "int64", - "optional": true, - "field": "thread" - }, - { - "type": "string", - "optional": true, - "field": "query" - } - ], - "optional": false, - "name": "io.debezium.connector.mysql.Source", - "field": "source" - }, - { - "type": "string", - "optional": false, - "field": "op" - }, - { - "type": "int64", - "optional": true, - "field": "ts_ms" - }, - { - "type": "struct", - "fields": [ - { - "type": "string", - "optional": false, - "field": "id" - }, - { - "type": "int64", - "optional": false, - "field": "total_order" - }, - { - "type": "int64", - "optional": false, - "field": "data_collection_order" - } - ], - "optional": true, - "field": "transaction" - } - ], - "optional": false, - "name": "mysql_binlog_source.column_type_test_2jid2b.full_types.Envelope" - }, - "payload": { - "before": null, - "after": { - "id": 1, - "tiny_c": 127, - "tiny_un_c": 255, - "small_c": 32767, - "small_un_c": 65535, - "medium_c": 8388607, - "medium_un_c": 16777215, - "int_c": 2147483647, - "int_un_c": 4294967295, - "int11_c": 2147483647, - "big_c": 9223372036854775807, - "big_un_c": "AP//////////", - "varchar_c": "Hello World", - "char_c": "abc", - "real_c": 123.102, - "float_c": 123.10199737548828, - "double_c": 404.4443, - "decimal_c": "EtaH", - "numeric_c": "AVo=", - "big_decimal_c": "FJqkSQ==", - "bit1_c": false, - "tiny1_c": 1, - "boolean_c": 1, - "date_c": 18460, - "time_c": 64822000000, - "datetime3_c": 1595008822123, - "datetime6_c": 1595008822123456, - "timestamp_c": "2020-07-17T18:00:22Z", - "file_uuid": "ZRrtCDkPSJOy8TaSPnt0AA==", - "bit_c": "BAQEBAQEBAQ=", - "text_c": "text", - "tiny_blob_c": "EA==", - "blob_c": "EA==", - "medium_blob_c": "EA==", - "long_blob_c": "EA==", - "year_c": 2021, - "enum_c": "red", - "set_c": "a,b", - "json_c": "{\"key1\": \"value1\"}", - "point_c": {"x":1.0,"y":1.0,"wkb":"AQEAAAAAAAAAAADwPwAAAAAAAPA/"}, - "geometry_c":{"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/"}, - "linestring_c": {"wkb":"AQIAAAADAAAAAAAAAAAACEAAAAAAAAAAAAAAAAAAAAhAAAAAAAAACEAAAAAAAAAIQAAAAAAAABRA"}, - "polygon_c": {"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/"}, - "multipoint_c": {"wkb":"AQQAAAACAAAAAQEAAAAAAAAAAADwPwAAAAAAAPA/AQEAAAAAAAAAAAAAQAAAAAAAAABA"}, - "multiline_c": {"wkb":"AQUAAAACAAAAAQIAAAADAAAAAAAAAAAA8D8AAAAAAADwPwAAAAAAAABAAAAAAAAAAEAAAAAAAAAIQAAAAAAAAAhAAQIAAAACAAAAAAAAAAAAEEAAAAAAAAAQQAAAAAAAABRAAAAAAAAAFEA="}, - "multipolygon_c": {"wkb":"AQYAAAACAAAAAQMAAAABAAAABQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAAAAAAAAAAAAAAAQMAAAABAAAABQAAAAAAAAAAABRAAAAAAAAAFEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAUQAAAAAAAABRA"}, - "geometrycollection_c": {"wkb":"AQcAAAADAAAAAQEAAAAAAAAAAAAkQAAAAAAAACRAAQEAAAAAAAAAAAA+QAAAAAAAAD5AAQIAAAACAAAAAAAAAAAALkAAAAAAAAAuQAAAAAAAADRAAAAAAAAANEA="} - }, - "source": { - "version": "1.5.2.Final", - "connector": "mysql", - "name": "mysql_binlog_source", - "ts_ms": 1630315344066, - "snapshot": "last", - "db": "column_type_test_2jid2b", - "sequence": null, - "table": "full_types", - "server_id": 0, - "gtid": null, - "file": "mysql-bin.000003", - "pos": 1532, - "row": 0, - "thread": null, - "query": null - }, - "op": "r", - "ts_ms": 1630315344085, - "transaction": null - } - }, - "expected_binlog": { - "schema": { - "type": "struct", - "fields": [ - { - "type": "int32", - "optional": false, - "field": "id" - }, - { - "type": "int16", - "optional": true, - "field": "tiny_c" - }, - { - "type": "int16", - "optional": true, - "field": "tiny_un_c" - }, - { - "type": "int16", - "optional": true, - "field": "small_c" - }, - { - "type": "int32", - "optional": true, - "field": "small_un_c" - }, - { - "type": "int32", - "optional": true, - "field": "medium_c" - }, - { - "type": "int32", - "optional": true, - "field": "medium_un_c" - }, - { - "type": "int32", - "optional": true, - "field": "int_c" - }, - { - "type": "int64", - "optional": true, - "field": "int_un_c" - }, - { - "type": "int32", - "optional": true, - "field": "int11_c" - }, - { - "type": "int64", - "optional": true, - "field": "big_c" - }, - { - "type": "int64", - "optional": true, - "field": "big_un_c" - }, - { - "type": "string", - "optional": true, - "field": "varchar_c" - }, - { - "type": "string", - "optional": true, - "field": "char_c" - }, - { - "type": "double", - "optional": true, - "field": "real_c" - }, - { - "type": "double", - "optional": true, - "field": "float_c" - }, - { - "type": "double", - "optional": true, - "field": "double_c" - }, - { - "type": "bytes", - "optional": true, - "name": "org.apache.kafka.connect.data.Decimal", - "version": 1, - "parameters": { - "scale": "4", - "connect.decimal.precision": "8" - }, - "field": "decimal_c" - }, - { - "type": "bytes", - "optional": true, - "name": "org.apache.kafka.connect.data.Decimal", - "version": 1, - "parameters": { - "scale": "0", - "connect.decimal.precision": "6" - }, - "field": "numeric_c" - }, - { - "type": "bytes", - "optional": true, - "name": "org.apache.kafka.connect.data.Decimal", - "version": 1, - "parameters": { - "scale": "65", - "connect.decimal.precision": "1" - }, - "field": "big_decimal_c" - }, - { - "type": "int16", - "optional": true, - "field": "bit1_c" - }, - { - "type": "int16", - "optional": true, - "field": "tiny1_c" - }, - { - "type": "int16", - "optional": true, - "field": "boolean_c" - }, - { - "type": "int32", - "optional": true, - "name": "io.debezium.time.Date", - "version": 1, - "field": "date_c" - }, - { - "type": "int64", - "optional": true, - "name": "io.debezium.time.MicroTime", - "version": 1, - "field": "time_c" - }, - { - "type": "int64", - "optional": true, - "name": "io.debezium.time.Timestamp", - "version": 1, - "field": "datetime3_c" - }, - { - "type": "int64", - "optional": true, - "name": "io.debezium.time.MicroTimestamp", - "version": 1, - "field": "datetime6_c" - }, - { - "type": "string", - "optional": false, - "name": "io.debezium.time.ZonedTimestamp", - "version": 1, - "default": "1970-01-01T00:00:00Z", - "field": "timestamp_c" - }, - { - "type": "bytes", - "optional": true, - "field": "file_uuid" - }, - { - "type": "bytes", - "optional": true, - "field": "bit_c" - }, - { - "type": "string", - "optional": true, - "field": "text_c" - }, - { - "type": "bytes", - "optional": true, - "field": "tiny_blob_c" - }, - { - "type": "bytes", - "optional": true, - "field": "blob_c" - }, - { - "type": "bytes", - "optional": true, - "field": "medium_blob_c" - }, - { - "type": "bytes", - "optional": true, - "field": "long_blob_c" - }, - { - "type": "int32", - "optional": true, - "field": "year_c" - }, - { - "type": "string", - "optional": true, - "field": "enum_c" - }, - { - "type": "string", - "optional": true, - "field": "set_c" - }, - { - "type": "string", - "optional": true, - "field": "json_c" - }, - { - "type": "string", - "optional": true, - "field": "point_c" - }, - { - "type": "string", - "optional": true, - "field": "geometry_c" - }, - { - "type": "string", - "optional": true, - "field": "linestring_c" - }, - { - "type": "string", - "optional": true, - "field": "polygon_c" - }, - { - "type": "string", - "optional": true, - "field": "multipoint_c" - }, - { - "type": "string", - "optional": true, - "field": "multiline_c" - }, - { - "type": "string", - "optional": true, - "field": "multipolygon_c" - }, - { - "type": "string", - "optional": true, - "field": "geometrycollection_c" - } - ], - "optional": false, - "name": "mysql_binlog_source.column_type_test_13p6nkc.full_types.Envelope" - }, - "payload": { - "before": { - "id": 1, - "tiny_c": 127, - "tiny_un_c": 255, - "small_c": 32767, - "small_un_c": 65535, - "medium_c": 8388607, - "medium_un_c": 16777215, - "int_c": 2147483647, - "int_un_c": 4294967295, - "int11_c": 2147483647, - "big_c": 9223372036854775807, - "big_un_c": "AP//////////", - "varchar_c": "Hello World", - "char_c": "abc", - "real_c": 123.102, - "float_c": 123.10199737548828, - "double_c": 404.4443, - "decimal_c": "EtaH", - "numeric_c": "AVo=", - "big_decimal_c": "FJqkSQ==", - "bit1_c": false, - "tiny1_c": 1, - "boolean_c": 1, - "date_c": 18460, - "time_c": 64822000000, - "datetime3_c": 1595008822123, - "datetime6_c": 1595008822123456, - "timestamp_c": "2020-07-17T18:00:22Z", - "file_uuid": "ZRrtCDkPSJOy8TaSPnt0AA==", - "bit_c": "BAQEBAQEBAQ=", - "text_c": "text", - "tiny_blob_c": "EA==", - "blob_c": "EA==", - "medium_blob_c": "EA==", - "long_blob_c": "EA==", - "year_c": 2021, - "enum_c": "red", - "set_c": "a,b", - "json_c": "{\"key1\":\"value1\"}", - "point_c": {"x":1.0,"y":1.0,"wkb":"AQEAAAAAAAAAAADwPwAAAAAAAPA/"}, - "geometry_c":{"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/"}, - "linestring_c": {"wkb":"AQIAAAADAAAAAAAAAAAACEAAAAAAAAAAAAAAAAAAAAhAAAAAAAAACEAAAAAAAAAIQAAAAAAAABRA"}, - "polygon_c": {"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/"}, - "multipoint_c": {"wkb":"AQQAAAACAAAAAQEAAAAAAAAAAADwPwAAAAAAAPA/AQEAAAAAAAAAAAAAQAAAAAAAAABA"}, - "multiline_c": {"wkb":"AQUAAAACAAAAAQIAAAADAAAAAAAAAAAA8D8AAAAAAADwPwAAAAAAAABAAAAAAAAAAEAAAAAAAAAIQAAAAAAAAAhAAQIAAAACAAAAAAAAAAAAEEAAAAAAAAAQQAAAAAAAABRAAAAAAAAAFEA="}, - "multipolygon_c": {"wkb":"AQYAAAACAAAAAQMAAAABAAAABQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAAAAAAAAAAAAAAAQMAAAABAAAABQAAAAAAAAAAABRAAAAAAAAAFEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAUQAAAAAAAABRA"}, - "geometrycollection_c": {"wkb":"AQcAAAADAAAAAQEAAAAAAAAAAAAkQAAAAAAAACRAAQEAAAAAAAAAAAA+QAAAAAAAAD5AAQIAAAACAAAAAAAAAAAALkAAAAAAAAAuQAAAAAAAADRAAAAAAAAANEA="} - }, - "after": { - "id": 1, - "tiny_c": 127, - "tiny_un_c": 255, - "small_c": 32767, - "small_un_c": 65535, - "medium_c": 8388607, - "medium_un_c": 16777215, - "int_c": 2147483647, - "int_un_c": 4294967295, - "int11_c": 2147483647, - "big_c": 9223372036854775807, - "big_un_c": "AP//////////", - "varchar_c": "Hello World", - "char_c": "abc", - "real_c": 123.102, - "float_c": 123.10199737548828, - "double_c": 404.4443, - "decimal_c": "EtaH", - "numeric_c": "AVo=", - "big_decimal_c": "FJqkSQ==", - "bit1_c": false, - "tiny1_c": 1, - "boolean_c": 1, - "date_c": 18460, - "time_c": 64822000000, - "datetime3_c": 1595008822123, - "datetime6_c": 1595008822123456, - "timestamp_c": "2020-07-17T18:33:22Z", - "file_uuid": "ZRrtCDkPSJOy8TaSPnt0AA==", - "bit_c": "BAQEBAQEBAQ=", - "text_c": "text", - "tiny_blob_c": "EA==", - "blob_c": "EA==", - "medium_blob_c": "EA==", - "long_blob_c": "EA==", - "year_c": 2021, - "enum_c": "red", - "set_c": "a,b", - "json_c": "{\"key1\":\"value1\"}", - "point_c": {"x":1.0,"y":1.0,"wkb":"AQEAAAAAAAAAAADwPwAAAAAAAPA/"}, - "geometry_c":{"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/"}, - "linestring_c": {"wkb":"AQIAAAADAAAAAAAAAAAACEAAAAAAAAAAAAAAAAAAAAhAAAAAAAAACEAAAAAAAAAIQAAAAAAAABRA"}, - "polygon_c": {"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/"}, - "multipoint_c": {"wkb":"AQQAAAACAAAAAQEAAAAAAAAAAADwPwAAAAAAAPA/AQEAAAAAAAAAAAAAQAAAAAAAAABA"}, - "multiline_c": {"wkb":"AQUAAAACAAAAAQIAAAADAAAAAAAAAAAA8D8AAAAAAADwPwAAAAAAAABAAAAAAAAAAEAAAAAAAAAIQAAAAAAAAAhAAQIAAAACAAAAAAAAAAAAEEAAAAAAAAAQQAAAAAAAABRAAAAAAAAAFEA="}, - "multipolygon_c": {"wkb":"AQYAAAACAAAAAQMAAAABAAAABQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAAAAAAAAAAAAAAAQMAAAABAAAABQAAAAAAAAAAABRAAAAAAAAAFEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAUQAAAAAAAABRA"}, - "geometrycollection_c": {"wkb":"AQcAAAADAAAAAQEAAAAAAAAAAAAkQAAAAAAAACRAAQEAAAAAAAAAAAA+QAAAAAAAAD5AAQIAAAACAAAAAAAAAAAALkAAAAAAAAAuQAAAAAAAADRAAAAAAAAANEA="} - }, - "source": { - "version": "1.5.2.Final", - "connector": "mysql", - "name": "mysql_binlog_source", - "ts_ms": 1630324170000, - "snapshot": "false", - "db": "column_type_test_13p6nkc", - "sequence": null, - "table": "full_types", - "server_id": 223344, - "gtid": null, - "file": "mysql-bin.000003", - "pos": 1816, - "row": 0, - "thread": null, - "query": null - }, - "op": "u", - "ts_ms": 1630324170753, - "transaction": null - } - } -} \ No newline at end of file diff --git a/pom.xml b/pom.xml index 60d7dd58974..493dc634ed6 100644 --- a/pom.xml +++ b/pom.xml @@ -33,6 +33,8 @@ under the License. pom + flink-cdc-base + flink-cdc-e2e-tests flink-connector-debezium flink-connector-test-util flink-connector-mysql-cdc @@ -49,9 +51,6 @@ under the License. flink-sql-connector-oceanbase-cdc flink-sql-connector-sqlserver-cdc flink-sql-connector-tidb-cdc - flink-cdc-e2e-tests - flink-connector-base - flink-connector-mysql-cdc-new