[cdc-base] Improve the Incremental Snapshot Interfaces

pull/1011/head
Leonard Xu 3 years ago committed by Leonard Xu
parent 31c6659d2a
commit 3ef4f2fa31

@ -77,7 +77,7 @@ import com.ververica.cdc.connectors.mysql.source.MySqlSource;
public class MySqlSourceExample {
public static void main(String[] args) throws Exception {
MySqlSource<String> mySqlSource = MySqlSource.<String>builder()
MySqlSource<String> mySqlChangeEventSource = MySqlSource.<String>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

@ -91,7 +91,7 @@ import com.ververica.cdc.connectors.mysql.source.MySqlSource;
public class MySqlBinlogSourceExample {
public static void main(String[] args) throws Exception {
MySqlSource<String> mySqlSource = MySqlSource.<String>builder()
MySqlSource<String> mySqlChangeEventSource = MySqlSource.<String>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

@ -481,7 +481,7 @@ import com.ververica.cdc.connectors.mysql.source.MySqlSource;
public class MySqlSourceExample {
public static void main(String[] args) throws Exception {
MySqlSource<String> mySqlSource = MySqlSource.<String>builder()
MySqlSource<String> mySqlChangeEventSource = MySqlSource.<String>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

@ -25,10 +25,9 @@ under the License.
</parent>
<modelVersion>4.0.0</modelVersion>
<artifactId>flink-connector-base</artifactId>
<artifactId>flink-cdc-base</artifactId>
<dependencies>
<!-- Debezium dependencies -->
<dependency>
<groupId>com.ververica</groupId>
<artifactId>flink-connector-debezium</artifactId>
@ -114,5 +113,25 @@ under the License.
<type>test-jar</type>
<scope>test</scope>
</dependency>
<!-- Debezium test dependencies -->
<dependency>
<groupId>io.debezium</groupId>
<artifactId>debezium-connector-mysql</artifactId>
<version>${debezium.version}</version>
<scope>test</scope>
</dependency>
<!-- test dependencies on TestContainers -->
<dependency>
<groupId>org.testcontainers</groupId>
<artifactId>mysql</artifactId>
<version>${testcontainers.version}</version>
<scope>test</scope>
</dependency>
</dependencies>
</project>

@ -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);
}
}

@ -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<String> databaseList;
protected final List<String> 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<String> databaseList,
List<String> 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<String> databaseList,
List<String> 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 class SourceConfig implements Serializable {
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 class SourceConfig implements Serializable {
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 class SourceConfig implements Serializable {
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;
}
}

@ -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<String> databaseList;
protected List<String> 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 abstract class SourceConfigFactory implements Serializable {
* 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 abstract class SourceConfigFactory implements Serializable {
/**
* 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 the 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 abstract class SourceConfigFactory implements Serializable {
* 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 abstract class SourceConfigFactory implements Serializable {
* 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 abstract class SourceConfigFactory implements Serializable {
* 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 abstract class SourceConfigFactory implements Serializable {
* 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 abstract class SourceConfigFactory implements Serializable {
* 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 abstract class SourceConfigFactory implements Serializable {
* 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 abstract class SourceConfigFactory implements Serializable {
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);
}

@ -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);
}
}

@ -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 <ID> The identifier type of data collection.
* @param <S> The schema type of data collection.
* @param <C> The source config of data source.
*/
@Experimental
public interface DataSourceDialect<ID extends DataCollectionId, S, C extends SourceConfig>
extends Serializable {
/** Get the name of dialect. */
String getName();
/** Discovers the list of data collection to capture. */
List<ID> discoverDataCollections(C sourceConfig);
/**
* Discovers the captured data collections' schema by {@link SourceConfig}.
*
* @param sourceConfig a basic source configuration.
*/
Map<ID, S> discoverDataCollectionSchemas(C sourceConfig);
/**
* Displays current offset from the database e.g. query Mysql binary logs by query <code>
* SHOW MASTER STATUS</code>.
*/
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<ID> createChunkSplitter(C sourceConfig);
/** The fetch task used to fetch data of a snapshot split or stream split. */
FetchTask<SourceSplitBase> createFetchTask(SourceSplitBase sourceSplitBase);
/** The task context used fot fetch task to fetch data from external systems. */
FetchTask.Context createFetchTaskContext(SourceSplitBase sourceSplitBase);
}

@ -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<TableId, TableChange, JdbcSourceConfig> {
/** Discovers the list of table to capture. */
List<TableId> discoverDataCollections(JdbcSourceConfig sourceConfig);
/** Discovers the captured tables' schema by {@link SourceConfig}. */
Map<TableId, TableChange> 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<SourceSplitBase> createFetchTask(SourceSplitBase sourceSplitBase);
@Override
JdbcSourceFetchTaskContext createFetchTaskContext(SourceSplitBase sourceSplitBase);
}

@ -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<String> HOSTNAME =
ConfigOptions.key("hostname")
.stringType()
.noDefaultValue()
.withDescription("IP address or hostname of the MySQL database server.");
public static final ConfigOption<Integer> PORT =
ConfigOptions.key("port")
.intType()
.defaultValue(3306)
.withDescription("Integer port number of the MySQL database server.");
public static final ConfigOption<String> 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<String> PASSWORD =
ConfigOptions.key("password")
.stringType()
.noDefaultValue()
.withDescription(
"Password to use when connecting to the MySQL database server.");
public static final ConfigOption<String> DATABASE_NAME =
ConfigOptions.key("database-name")
.stringType()
.noDefaultValue()
.withDescription("Database name of the MySQL server to monitor.");
public static final ConfigOption<String> TABLE_NAME =
ConfigOptions.key("table-name")
.stringType()
.noDefaultValue()
.withDescription("Table name of the MySQL database to monitor.");
public static final ConfigOption<String> SERVER_TIME_ZONE =
ConfigOptions.key("server-time-zone")
.stringType()
.defaultValue("UTC")
.withDescription("The session time zone in database server.");
public static final ConfigOption<String> 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<Duration> 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<Integer> CONNECTION_POOL_SIZE =
ConfigOptions.key("connection.pool.size")
.intType()
.defaultValue(20)
.withDescription("The connection pool size.");
public static final ConfigOption<Integer> 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.");
}

@ -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<String> HOSTNAME =
ConfigOptions.key("hostname")
.stringType()
.noDefaultValue()
.withDescription("IP address or hostname of the MySQL database server.");
public static final ConfigOption<Integer> PORT =
ConfigOptions.key("port")
.intType()
.defaultValue(3306)
.withDescription("Integer port number of the MySQL database server.");
public static final ConfigOption<String> 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<String> PASSWORD =
ConfigOptions.key("password")
.stringType()
.noDefaultValue()
.withDescription(
"Password to use when connecting to the MySQL database server.");
public static final ConfigOption<String> DATABASE_NAME =
ConfigOptions.key("database-name")
.stringType()
.noDefaultValue()
.withDescription("Database name of the MySQL server to monitor.");
public static final ConfigOption<String> TABLE_NAME =
ConfigOptions.key("table-name")
.stringType()
.noDefaultValue()
.withDescription("Table name of the MySQL database to monitor.");
public static final ConfigOption<String> SERVER_TIME_ZONE =
ConfigOptions.key("server-time-zone")
.stringType()
.defaultValue("UTC")
.withDescription("The session time zone in database server.");
public static final ConfigOption<String> 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<Boolean> 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<Duration> 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<Integer> CONNECTION_POOL_SIZE =
ConfigOptions.key("connection.pool.size")
.intType()
.defaultValue(20)
.withDescription("The connection pool size.");
public static final ConfigOption<Integer> 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<String> 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<Integer> 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<Double> 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<Double> SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND =
ConfigOptions.key("split-key.even-distribution.factor.lower-bound")
.doubleType()

@ -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
*/

@ -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;

@ -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.EventDispatcher;
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.Collection;
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 @@ import static com.ververica.cdc.connectors.refactor.refactor.source.dialect.task
* this is useful for downstream to deserialize the {@link HistoryRecord} back.
* </pre>
*/
public class EventDispatcherImpl<T extends DataCollectionId> extends EventDispatcher<T> {
private static final Logger LOG = LoggerFactory.getLogger(EventDispatcherImpl.class);
public class JdbcSourceEventDispatcher extends EventDispatcher<TableId> {
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<DataChangeEvent> queue;
private final HistorizedDatabaseSchema historizedSchema;
private final DataCollectionFilters.DataCollectionFilter<T> filter;
private final DataCollectionFilters.DataCollectionFilter<TableId> filter;
private final CommonConnectorConfig connectorConfig;
private final TopicSelector<T> topicSelector;
private final TopicSelector<TableId> 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<T> topicSelector,
DatabaseSchema<T> schema,
TopicSelector<TableId> topicSelector,
DatabaseSchema<TableId> schema,
ChangeEventQueue<DataChangeEvent> queue,
DataCollectionFilters.DataCollectionFilter<T> filter,
DataCollectionFilters.DataCollectionFilter<TableId> filter,
ChangeEventCreator changeEventCreator,
EventMetadataProvider metadataProvider,
SchemaNameAdjuster schemaNameAdjuster) {
@ -95,12 +106,13 @@ public class EventDispatcherImpl<T extends DataCollectionId> extends EventDispat
schemaNameAdjuster);
this.historizedSchema =
schema instanceof HistorizedDatabaseSchema
? (HistorizedDatabaseSchema<T>) schema
? (HistorizedDatabaseSchema<TableId>) 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 class EventDispatcherImpl<T extends DataCollectionId> extends EventDispat
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<DataChangeEvent> getQueue() {
@ -130,7 +154,7 @@ public class EventDispatcherImpl<T extends DataCollectionId> extends EventDispat
@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 class EventDispatcherImpl<T extends DataCollectionId> extends EventDispat
@Override
public void dispatchSchemaChangeEvent(
Collection<T> dataCollectionIds, SchemaChangeEventEmitter schemaChangeEventEmitter)
Collection<TableId> 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 class EventDispatcherImpl<T extends DataCollectionId> extends EventDispat
}
}
}
public void dispatchWatermarkEvent(
Map<String, ?> 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;
}
}
}
}

@ -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 org.slf4j.LoggerFactory;
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}.
*
* <p>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<T>
public class JdbcSourceRecordEmitter<T>
implements RecordEmitter<SourceRecord, T, SourceSplitState> {
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<T>
private final SourceReaderMetrics sourceReaderMetrics;
private final boolean includeSchemaChanges;
private final OutputCollector<T> outputCollector;
private OffsetFactory offsetFactory;
public BaseRecordEmitter(
public JdbcSourceRecordEmitter(
DebeziumDeserializationSchema<T> debeziumDeserializationSchema,
SourceReaderMetrics sourceReaderMetrics,
boolean includeSchemaChanges,

@ -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;

@ -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<P, C extends SourceConfig> {
/**
* 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);
}

@ -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 class JdbcConnectionFactory implements JdbcConnection.ConnectionFactory {
new ConnectionPoolId(sourceConfig.getHostname(), sourceConfig.getPort());
HikariDataSource dataSource =
JdbcConnectionPools.getInstance(pooledDataSourceFactory)
JdbcConnectionPools.getInstance(jdbcConnectionPoolFactory)
.getOrCreateConnectionPool(connectionPoolId, sourceConfig);
int i = 0;

@ -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 abstract class PooledDataSourceFactory {
* @param sourceConfig a basic Source configuration.
* @return a database url.
*/
public abstract String getJdbcUrl(SourceConfig sourceConfig);
public abstract String getJdbcUrl(JdbcSourceConfig sourceConfig);
}

@ -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.HashMap;
import java.util.Map;
/** A Jdbc Connection pools implementation. */
public class JdbcConnectionPools implements ConnectionPools {
public class JdbcConnectionPools implements ConnectionPools<HikariDataSource, JdbcSourceConfig> {
private static final Logger LOG = LoggerFactory.getLogger(JdbcConnectionPools.class);
private static JdbcConnectionPools instance;
private final Map<ConnectionPoolId, HikariDataSource> 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 class JdbcConnectionPools implements ConnectionPools {
@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);
}

@ -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.connector.base.source.reader.synchronization.FutureCompl
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 @@ import java.util.function.Supplier;
* 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<T>
@Experimental
public class JdbcIncrementalSource<T>
implements Source<T, SourceSplitBase, PendingSplitsState>, ResultTypeQueryable<T> {
private static final long serialVersionUID = 1L;
private final SourceConfigFactory configFactory;
private final DebeziumDeserializationSchema<T> deserializationSchema;
private final JdbcSourceConfigFactory configFactory;
private final JdbcDataSourceDialect dataSourceDialect;
private final OffsetFactory offsetFactory;
private final DebeziumDeserializationSchema<T> 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<T> 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<T> 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 class ChangeEventHybridSource<T>
return offsetFactory;
}
};
this.validator = validator;
this.baseSchema = baseSchema;
}
@Override
@ -125,25 +99,22 @@ public class ChangeEventHybridSource<T>
}
@Override
public SourceReader<T, SourceSplitBase> 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<RecordsWithSplitIds<SourceRecord>> elementsQueue =
new FutureCompletingBlockingQueue<>();
final SourceReaderMetrics sourceReaderMetrics =
new SourceReaderMetrics(readerContext.metricGroup());
sourceReaderMetrics.registerMetrics();
Supplier<BaseSplitReader> splitReaderSupplier =
Supplier<JdbcSourceSplitReader> 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 class ChangeEventHybridSource<T>
readerContext,
sourceConfig,
sourceSplitSerializer,
snapshotEventDialect);
dataSourceDialect);
}
@Override
public SplitEnumerator<SourceSplitBase, PendingSplitsState> createEnumerator(
SplitEnumeratorContext<SourceSplitBase> 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<TableId> 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<SourceSplitBase, PendingSplitsState> restoreEnumerator(
SplitEnumeratorContext<SourceSplitBase> 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 class ChangeEventHybridSource<T>
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

@ -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 class HybridSplitAssigner implements SplitAssigner {
int currentParallelism,
List<TableId> remainingTables,
boolean isTableIdCaseSensitive,
SnapshotEventDialect dialect,
OffsetFactory offsetFactory,
BaseSchema baseSchema) {
DataSourceDialect dialect,
OffsetFactory offsetFactory) {
this(
new SnapshotSplitAssigner(
sourceConfig,
@ -71,8 +69,7 @@ public class HybridSplitAssigner implements SplitAssigner {
remainingTables,
isTableIdCaseSensitive,
dialect,
offsetFactory,
baseSchema),
offsetFactory),
false,
sourceConfig.getSplitMetaGroupSize());
this.offsetFactory = offsetFactory;
@ -82,18 +79,16 @@ public class HybridSplitAssigner implements SplitAssigner {
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());
}

@ -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<TableId> remainingTables,
boolean isTableIdCaseSensitive,
SnapshotEventDialect dialect,
OffsetFactory offsetFactory,
BaseSchema baseSchema) {
DataSourceDialect dialect,
OffsetFactory offsetFactory) {
this(
sourceConfig,
currentParallelism,
@ -89,17 +87,15 @@ public class SnapshotSplitAssigner implements SplitAssigner {
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 class SnapshotSplitAssigner implements SplitAssigner {
checkpoint.isTableIdCaseSensitive(),
checkpoint.isRemainingTablesCheckpointed(),
dialect,
offsetFactory,
baseSchema);
offsetFactory);
}
private SnapshotSplitAssigner(
@ -127,9 +122,8 @@ public class SnapshotSplitAssigner implements SplitAssigner {
List<TableId> 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 @@ public class SnapshotSplitAssigner implements SplitAssigner {
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<TableId> discoverTables = dialect.discoverCapturedTables(sourceConfig);
final List<TableId> 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 class SnapshotSplitAssigner implements SplitAssigner {
private boolean allSplitsFinished() {
return noMoreSplits() && assignedSplits.size() == splitFinishedOffsets.size();
}
private ChunkSplitter createChunkSplitter(SourceConfig sourceConfig) {
return new ChunkSplitter(baseSchema, sourceConfig, dialect);
}
}

@ -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.Map;
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 {
/**

@ -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 class StreamSplitAssigner implements SplitAssigner {
private StreamSplitAssigner(
SourceConfig sourceConfig,
boolean isStreamSplitAssigned,
Dialect dialect,
DataSourceDialect dialect,
OffsetFactory offsetFactory) {
this.sourceConfig = sourceConfig;
this.isStreamSplitAssigned = isStreamSplitAssigned;
@ -127,6 +125,7 @@ public class StreamSplitAssigner implements SplitAssigner {
// ------------------------------------------------------------------------------------------
public StreamSplit createStreamSplit() {
return new StreamSplit(
BINLOG_SPLIT_ID,
dialect.displayCurrentOffset(sourceConfig),

@ -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 @@ import static org.apache.flink.util.Preconditions.checkArgument;
* 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;

@ -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<T, Split> extends Reader<T, Split> {}
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<ID extends DataCollectionId> {
/** Generates all snapshot splits (chunks) for the give data collection. */
Collection<SnapshotSplit> generateSplits(ID dataCollectionId);
}

@ -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<TableId> {
/** Generates all snapshot splits (chunks) for the give table path. */
@Override
Collection<SnapshotSplit> generateSplits(TableId tableId);
/**
* Query the maximum and minimum value of the column in the table. e.g. query string <code>
* SELECT MIN(%s) FROM %s WHERE %s > ?</code>
*
* @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 <code>
* SELECT MIN(%s) FROM %s WHERE %s > ?</code>
*
* @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 <code>includedLowerBound</code> 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();
}
}

@ -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 class HybridPendingSplitsState extends PendingSplitsState {
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 class HybridPendingSplitsState extends PendingSplitsState {
+ "snapshotPendingSplits="
+ snapshotPendingSplits
+ ", isBinlogSplitAssigned="
+ isBinlogSplitAssigned
+ isStreamSplitAssigned
+ '}';
}
}

@ -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;

@ -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 @@ public class PendingSplitsStateSerializer implements SimpleVersionedSerializer<P
private void serializeHybridPendingSplitsState(
HybridPendingSplitsState state, DataOutputSerializer out) throws IOException {
serializeSnapshotPendingSplitsState(state.getSnapshotPendingSplits(), out);
out.writeBoolean(state.isBinlogSplitAssigned());
out.writeBoolean(state.isStreamSplitAssigned());
}
private void serializeBinlogPendingSplitsState(

@ -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<SnapshotSplit> 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<String, SnapshotSplit> 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<String, Offset> splitFinishedOffsets;

@ -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;

@ -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.util.FlinkRuntimeException;
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 @@ import java.util.stream.Collectors;
* A CDC source enumerator that enumerates receive the split request and assign the split to source
* readers.
*/
@Internal
public class SourceEnumerator implements SplitEnumerator<SourceSplitBase, PendingSplitsState> {
private static final Logger LOG = LoggerFactory.getLogger(SourceEnumerator.class);
@Experimental
public class IncrementalSourceEnumerator
implements SplitEnumerator<SourceSplitBase, PendingSplitsState> {
private static final Logger LOG = LoggerFactory.getLogger(IncrementalSourceEnumerator.class);
private static final long CHECK_EVENT_INTERVAL = 30_000L;
private final SplitEnumeratorContext<SourceSplitBase> context;
@ -67,7 +69,7 @@ public class SourceEnumerator implements SplitEnumerator<SourceSplitBase, Pendin
private final TreeSet<Integer> readersAwaitingSplit;
private List<List<FinishedSnapshotSplitInfo>> binlogSplitMeta;
public SourceEnumerator(
public IncrementalSourceEnumerator(
SplitEnumeratorContext<SourceSplitBase> context,
SourceConfig sourceConfig,
SplitAssigner splitAssigner) {
@ -187,9 +189,11 @@ public class SourceEnumerator implements SplitEnumerator<SourceSplitBase, Pendin
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(

@ -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 {

@ -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 {

@ -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 {

@ -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 {

@ -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 {

@ -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 @@ import java.util.Objects;
* from a specific {@link Offset}, we need to skip the processed change events and the processed
* rows.
*/
@Experimental
public abstract class Offset implements Comparable<Offset>, Serializable {
private static final long serialVersionUID = 1L;

@ -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.util.FlinkRuntimeException;
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 @@ public interface OffsetDeserializerSerializer extends Serializable {
}
}
OffsetFactory getOffsetFactory();
default OffsetDeserializer createOffsetDeserializer() {
return new OffsetDeserializer(getOffsetFactory());
}

@ -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;

@ -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;

@ -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 class FinishedSnapshotSplitInfo implements OffsetDeserializerSerializer {
}
// ------------------------------------------------------------------------------------
// 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 class FinishedSnapshotSplitInfo implements OffsetDeserializerSerializer {
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();
}

@ -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;

@ -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;

@ -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 abstract class SourceSplitBase implements SourceSplit {
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 abstract class SourceSplitBase implements SourceSplit {
return splitId;
}
public abstract Map<TableId, TableChanges.TableChange> getTableSchemas();
public abstract Map<TableId, TableChange> getTableSchemas();
@Override
public boolean equals(Object o) {

@ -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.core.memory.DataOutputSerializer;
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.HashMap;
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<SourceSplitBase>, OffsetDeserializerSerializer {
@ -79,8 +76,8 @@ public abstract class SourceSplitSerializer
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 abstract class SourceSplitSerializer
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<TableId, TableChange> tableSchemas = readTableSchemas(version, in);
@ -234,9 +231,10 @@ public abstract class SourceSplitSerializer
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(

@ -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 {

@ -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;

@ -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;

@ -21,17 +21,17 @@ package com.ververica.cdc.connectors.base.source.metrics;
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;

@ -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.connector.base.source.reader.synchronization.FutureCompl
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 @@ import static org.apache.flink.util.Preconditions.checkState;
* 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<T>
@Experimental
public class JdbcIncrementalSourceReader<T>
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<String, SnapshotSplit> finishedUnackedSplits;
private final Map<String, StreamSplit> 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<RecordsWithSplitIds<SourceRecord>> elementQueue,
Supplier<BaseSplitReader> splitReaderSupplier,
Supplier<JdbcSourceSplitReader> splitReaderSupplier,
RecordEmitter<SourceRecord, T, SourceSplitState> 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 @@ public class ParallelSourceReader<T>
if (split.getTableSchemas().isEmpty()) {
try {
Map<TableId, TableChanges.TableChange> 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 @@ public class ParallelSourceReader<T>
if (receivedMetaGroupId == expectedMetaGroupId) {
List<FinishedSnapshotSplitInfo> metaDataGroup =
metadataEvent.getMetaGroup().stream()
//
// .map(FinishedSnapshotSplitInfo::deserialize)
.map(
new Function<byte[], FinishedSnapshotSplitInfo>() {
@Override
public FinishedSnapshotSplitInfo apply(byte[] bytes) {
return sourceSplitSerializer.deserialize(bytes);
}
})
.map(bytes -> sourceSplitSerializer.deserialize(bytes))
.collect(Collectors.toList());
uncompletedBinlogSplits.put(
binlogSplit.splitId(),

@ -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.ArrayDeque;
import java.util.Iterator;
import java.util.Queue;
/** basic class read {@link SourceSplitBase} and return {@link SourceRecord}. */
public class BaseSplitReader implements SplitReader<SourceRecord, SourceSplitBase> {
/** Basic class read {@link SourceSplitBase} and return {@link SourceRecord}. */
@Experimental
public class JdbcSourceSplitReader implements SplitReader<SourceRecord, SourceSplitBase> {
private static final Logger LOG = LoggerFactory.getLogger(BaseSplitReader.class);
private static final Logger LOG = LoggerFactory.getLogger(JdbcSourceSplitReader.class);
private final Queue<SourceSplitBase> splits;
private final SourceConfig sourceConfig;
private final int subtaskId;
@Nullable private Reader<SourceRecord, SourceSplitBase> currentReader;
@Nullable private Fetcher<SourceRecord, SourceSplitBase> 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 class BaseSplitReader implements SplitReader<SourceRecord, SourceSplitBas
checkSplitOrStartNext();
Iterator<SourceRecord> 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 class BaseSplitReader implements SplitReader<SourceRecord, SourceSplitBas
@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 @@ public class BaseSplitReader implements SplitReader<SourceRecord, SourceSplitBas
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() {

@ -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<Split> {
/** 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 {}
}

@ -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<T, Split> {
/** 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<T, Split> {
// 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<Split> 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<T> pollSplitRecords() throws InterruptedException;
/** Return the current fetch task is finished or not. */
boolean isFinished();
/** Close the client and releases all resources. */
void close();
}

@ -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<DataChangeEvent> getQueue();
public abstract Offset getStreamOffset(SourceRecord sourceRecord);
}

@ -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<SourceRecord, SourceSplitBase> {
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<DataChangeEvent> queue;
private volatile Throwable readException;
// task to read snapshot for current split
private FetchTask<SourceSplitBase> 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<SourceSplitBase> 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<SourceRecord> 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<SourceRecord> sourceRecords = new ArrayList<>();
while (!reachBinlogEnd) {
List<DataChangeEvent> 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() {}
}

@ -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<SourceRecord, SourceSplitBase> {
private static final Logger LOG = LoggerFactory.getLogger(JdbcSourceStreamFetcher.class);
private final JdbcSourceFetchTaskContext taskContext;
private final ExecutorService executor;
private volatile ChangeEventQueue<DataChangeEvent> queue;
private volatile Throwable readException;
private FetchTask<SourceSplitBase> streamFetchTask;
private StreamSplit currentStreamSplit;
private Map<TableId, List<FinishedSnapshotSplitInfo>> finishedSplitsInfo;
// tableId -> the max splitHighWatermark
private Map<TableId, Offset> 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<SourceSplitBase> 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<SourceRecord> pollSplitRecords() throws InterruptedException {
checkReadException();
final List<SourceRecord> sourceRecords = new ArrayList<>();
if (streamFetchTask.isRunning()) {
List<DataChangeEvent> 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.
*
* <p>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.
*
* <pre> 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.
* </pre>
*/
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<FinishedSnapshotSplitInfo> finishedSplitInfos =
currentStreamSplit.getFinishedSnapshotSplitInfos();
Map<TableId, List<FinishedSnapshotSplitInfo>> splitsInfoMap = new HashMap<>();
Map<TableId, Offset> 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<FinishedSnapshotSplitInfo> 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;
}
}

@ -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;

@ -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;

@ -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 org.apache.kafka.connect.source.SourceRecord;
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 @@ public class RecordUtils {
}
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<SourceRecord> normalizedSplitRecords(
SnapshotSplit snapshotSplit,
List<SourceRecord> sourceRecords,
SchemaNameAdjuster nameAdjuster) {
List<SourceRecord> normalizedRecords = new ArrayList<>();
Map<Struct, SourceRecord> snapshotRecords = new HashMap<>();
List<SourceRecord> 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<SourceRecord> 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<SourceRecord> upsertBinlog(
SourceRecord lowWatermarkEvent,
SourceRecord highWatermarkEvent,
Map<Struct, SourceRecord> snapshotRecords,
List<SourceRecord> 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<SourceRecord> 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<SourceRecord> formatMessageTimestamp(
Collection<SourceRecord> 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());
}
}

@ -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;

@ -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.runtime.testutils.MiniClusterResourceConfiguration;
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 org.testcontainers.lifecycle.Startables;
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 class MySqlSourceExampleTest {
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<String> mySqlSource =
MySqlIncrementalSource<String> mySqlChangeEventSource =
new MySqlSourceBuilder()
.hostname(MYSQL_CONTAINER.getHost())
.port(MYSQL_CONTAINER.getDatabasePort())
@ -99,11 +91,25 @@ public class MySqlSourceExampleTest {
// 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));
}
}

@ -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<TableId> discoverDataCollections(JdbcSourceConfig sourceConfig) {
return null;
}
@Override
public Map<TableId, TableChange> 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;
}
}

@ -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<T> extends JdbcIncrementalSource<T> {
/** 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<T> deserializationSchema,
BinlogOffsetFactory offsetFactory,
MySqlDialect dataSourceDialect) {
super(configFactory, deserializationSchema, offsetFactory, dataSourceDialect);
}
}

@ -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 java.util.Properties;
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}.
*
* <pre>{@code
* MySqlSource
* MySqlIncrementalSource
* .<String>builder()
* .hostname("localhost")
* .port(3306)
@ -53,17 +49,14 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
* }</pre>
*
* <p>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<T> {
private final MySqlSourceConfigFactory configFactory = new MySqlSourceConfigFactory();
private BinlogOffsetFactory offsetFactory;
private MySqlDialect dialect;
private DebeziumDeserializationSchema<T> deserializer;
private Validator validator;
private OffsetFactory offsetFactory;
private SnapshotEventDialect snapshotEventDialect;
private StreamingEventDialect streamingEventDialect;
private BaseSchema baseSchema;
public MySqlSourceBuilder<T> hostname(String hostname) {
this.configFactory.hostname(hostname);
@ -194,7 +187,7 @@ public class MySqlSourceBuilder<T> {
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<T> includeSchemaChanges(boolean includeSchemaChanges) {
this.configFactory.includeSchemaChanges(includeSchemaChanges);
return this;
@ -221,45 +214,15 @@ public class MySqlSourceBuilder<T> {
return this;
}
public MySqlSourceBuilder<T> validator(Validator validator) {
this.validator = validator;
return this;
}
public MySqlSourceBuilder<T> offsetFactory(OffsetFactory offsetFactory) {
this.offsetFactory = offsetFactory;
return this;
}
public MySqlSourceBuilder<T> snapshotEventDialect(SnapshotEventDialect snapshotEventDialect) {
this.snapshotEventDialect = snapshotEventDialect;
return this;
}
public MySqlSourceBuilder<T> streamingEventDialect(
StreamingEventDialect streamingEventDialect) {
this.streamingEventDialect = streamingEventDialect;
return this;
}
public MySqlSourceBuilder<T> 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<T> build() {
return new MySqlSource<>(
configFactory,
checkNotNull(deserializer),
offsetFactory,
snapshotEventDialect,
streamingEventDialect,
validator,
baseSchema);
public MySqlIncrementalSource<T> build() {
this.offsetFactory = new BinlogOffsetFactory();
this.dialect = new MySqlDialect(configFactory);
return new MySqlIncrementalSource<>(
configFactory, checkNotNull(deserializer), offsetFactory, dialect);
}
}

@ -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();

@ -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 @@ import java.util.Properties;
* 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<String> databaseList,
List<String> 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<String> databaseList,
List<String> 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;
}

@ -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 java.util.UUID;
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 class MySqlSourceConfigFactory extends SourceConfigFactory {
}
/** 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 class MySqlSourceConfigFactory extends SourceConfigFactory {
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 class MySqlSourceConfigFactory extends SourceConfigFactory {
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);
}
}

@ -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 @@ import static org.apache.flink.util.Preconditions.checkArgument;
/**
* 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;

@ -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;

@ -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<String, String> offset) {
return new BinlogOffset(offset);

@ -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;

@ -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 {

@ -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;

@ -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.HashMap;
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) {

@ -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;

@ -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.Iterator;
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 {

@ -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.sql.SQLException;
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 {

@ -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<SnapshotSplit> 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<ChunkRange> 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<SnapshotSplit> 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<ChunkRange> 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<ChunkRange> 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<ChunkRange> 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<ChunkRange> 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<ChunkRange> 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<TableId, TableChange> 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<Column> 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);
}
}

@ -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 <code>
* SHOW MASTER STATUS</code>.
*
* @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<TableId, TableChanges.TableChange> 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<TableId> discoverCapturedTables(SourceConfig sourceConfig) {
final List<TableId> 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<TableId> 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<DataChangeEvent> getQueue();
}
}

@ -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 <code>
* SELECT MIN(%s) FROM %s WHERE %s > ?</code>
*
* @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 <code>
* SELECT MIN(%s) FROM %s WHERE %s > ?</code>
*
* @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 <code>includedLowerBound</code> 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<SourceRecord> normalizedSplitRecords(
SnapshotSplit currentSnapshotSplit, List<SourceRecord> 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("");
}
}
}

@ -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;
}
}
}

@ -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<SourceRecord, SourceSplitBase> {
private static final Logger LOG = LoggerFactory.getLogger(SnapshotReader.class);
private final SnapshotEventDialect.SnapshotContext statefulTaskContext;
private final ExecutorService executor;
private volatile ChangeEventQueue<DataChangeEvent> 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<SourceRecord> 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<SourceRecord> sourceRecords = new ArrayList<>();
while (!reachBinlogEnd) {
List<DataChangeEvent> 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;
}
}
}

@ -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.
*
* <p>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<String, ?> sourcePartition;
private final String topic;
private final ChangeEventQueue<DataChangeEvent> queue;
public SignalEventDispatcher(
Map<String, ?> sourcePartition, String topic, ChangeEventQueue<DataChangeEvent> 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;
}
}
}
}

@ -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.
// *
// * <p>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<TableId> topicSelector;
// private SnapshotChangeEventSourceMetrics snapshotChangeEventSourceMetrics;
// private StreamingChangeEventSourceMetrics streamingChangeEventSourceMetrics;
// private EventDispatcherImpl<TableId> dispatcher;
// private ChangeEventQueue<DataChangeEvent> 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<DataChangeEvent>()
// .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<String> 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<String, String> 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<TableId> getDispatcher() {
// return dispatcher;
// }
//
// public ChangeEventQueue<DataChangeEvent> getQueue() {
// return queue;
// }
//
// public ErrorHandler getErrorHandler() {
// return errorHandler;
// }
//
// public MySqlOffsetContext getOffsetContext() {
// return offsetContext;
// }
//
// public TopicSelector<TableId> getTopicSelector() {
// return topicSelector;
// }
//
// public SnapshotChangeEventSourceMetrics getSnapshotChangeEventSourceMetrics() {
// snapshotChangeEventSourceMetrics.reset();
// return snapshotChangeEventSourceMetrics;
// }
//
// public StreamingChangeEventSourceMetrics getStreamingChangeEventSourceMetrics() {
// streamingChangeEventSourceMetrics.reset();
// return streamingChangeEventSourceMetrics;
// }
//
// public SchemaNameAdjuster getSchemaNameAdjuster() {
// return schemaNameAdjuster;
// }
// }

@ -1,185 +0,0 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
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.
-->
<project xmlns="http://maven.apache.org/POM/4.0.0"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<parent>
<artifactId>flink-cdc-connectors</artifactId>
<groupId>com.ververica</groupId>
<version>2.2-SNAPSHOT</version>
</parent>
<modelVersion>4.0.0</modelVersion>
<artifactId>flink-connector-mysql-cdc-new</artifactId>
<name>flink-connector-mysql-cdc-new</name>
<packaging>jar</packaging>
<dependencies>
<dependency>
<groupId>com.ververica</groupId>
<artifactId>flink-connector-debezium</artifactId>
<version>${project.version}</version>
<exclusions>
<exclusion>
<artifactId>kafka-log4j-appender</artifactId>
<groupId>org.apache.kafka</groupId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>com.ververica</groupId>
<artifactId>flink-connector-base</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>io.debezium</groupId>
<artifactId>debezium-connector-mysql</artifactId>
<version>${debezium.version}</version>
</dependency>
<!-- geometry dependencies -->
<dependency>
<groupId>com.esri.geometry</groupId>
<artifactId>esri-geometry-api</artifactId>
<version>${geometry.version}</version>
<exclusions>
<exclusion>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-core</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>com.zaxxer</groupId>
<artifactId>HikariCP</artifactId>
<version>4.0.3</version>
</dependency>
<!-- test dependencies on Debezium -->
<dependency>
<groupId>com.ververica</groupId>
<artifactId>flink-connector-test-util</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>io.debezium</groupId>
<artifactId>debezium-core</artifactId>
<version>${debezium.version}</version>
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>com.alibaba</groupId>
<artifactId>fastjson</artifactId>
<version>1.2.78</version>
<scope>test</scope>
</dependency>
<!-- test dependencies on Flink -->
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table-planner-blink_${scala.binary.version}</artifactId>
<version>${flink.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table-runtime-blink_${scala.binary.version}</artifactId>
<version>${flink.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils_${scala.binary.version}</artifactId>
<version>${flink.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-connector-test-utils</artifactId>
<version>${flink.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-core</artifactId>
<version>${flink.version}</version>
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
<version>${flink.version}</version>
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table-common</artifactId>
<version>${flink.version}</version>
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-tests</artifactId>
<version>${flink.version}</version>
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table-planner-blink_${scala.binary.version}</artifactId>
<version>${flink.version}</version>
<type>test-jar</type>
<scope>test</scope>
</dependency>
<!-- test dependencies on TestContainers -->
<dependency>
<groupId>org.testcontainers</groupId>
<artifactId>mysql</artifactId>
<version>${testcontainers.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>com.jayway.jsonpath</groupId>
<artifactId>json-path</artifactId>
<version>2.4.0</version>
<scope>test</scope>
</dependency>
</dependencies>
</project>

@ -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;
}
}

@ -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<TableId, TableChanges.TableChange> 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<TableId, TableChanges.TableChange> 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<SchemaChangeEvent> 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);
}
}

@ -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<T> extends ChangeEventHybridSource<T> {
public MySqlSource(
SourceConfigFactory configFactory,
DebeziumDeserializationSchema<T> deserializationSchema,
OffsetFactory offsetFactory,
SnapshotEventDialect snapshotEventDialect,
StreamingEventDialect streamingEventDialect,
Validator validator,
BaseSchema baseSchema) {
super(
configFactory,
deserializationSchema,
offsetFactory,
snapshotEventDialect,
streamingEventDialect,
validator,
baseSchema);
}
public MySqlSource(
SourceConfigFactory configFactory,
DebeziumDeserializationSchema<T> deserializationSchema,
OffsetFactory offsetFactory,
SnapshotEventDialect snapshotEventDialect,
StreamingEventDialect streamingEventDialect,
BaseSchema baseSchema) {
super(
configFactory,
deserializationSchema,
offsetFactory,
snapshotEventDialect,
streamingEventDialect,
baseSchema);
}
}

@ -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<TableId, TableChanges.TableChange> discoverCapturedTableSchemas(
SourceConfig sourceConfig) throws SQLException {
final List<TableId> capturedTableIds = discoverCapturedTables(sourceConfig);
try (MySqlConnection jdbc =
DebeziumUtils.createMySqlConnection(sourceConfig.getDbzConfiguration())) {
// fetch table schemas
MySqlSchema mySqlSchema =
new MySqlSchema(
(MySqlSourceConfig) sourceConfig, jdbc.isTableIdCaseSensitive());
Map<TableId, TableChanges.TableChange> tableSchemas = new HashMap<>();
for (TableId tableId : capturedTableIds) {
TableChanges.TableChange tableSchema = mySqlSchema.getTableSchema(jdbc, tableId);
tableSchemas.put(tableId, tableSchema);
}
return tableSchemas;
}
}
@Override
default List<TableId> 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);
}
}
}

@ -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<SourceRecord> normalizedSplitRecords(
SnapshotSplit currentSnapshotSplit, List<SourceRecord> 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;
}
}

@ -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;
}
}

@ -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<TableId> topicSelector;
private SnapshotChangeEventSourceMetrics snapshotChangeEventSourceMetrics;
private StreamingChangeEventSourceMetrics streamingChangeEventSourceMetrics;
private ChangeEventQueue<DataChangeEvent> 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<DataChangeEvent>()
.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<String> 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<DataChangeEvent> 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<String, String> 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;
}
}

Some files were not shown because too many files have changed in this diff Show More

Loading…
Cancel
Save