[cdc-base] Improve the Incremental Snapshot Interfaces
parent
31c6659d2a
commit
3ef4f2fa31
@ -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);
|
||||
}
|
||||
}
|
@ -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.");
|
||||
}
|
@ -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();
|
||||
}
|
||||
}
|
@ -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;
|
||||
}
|
||||
}
|
@ -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;
|
||||
}
|
||||
}
|
@ -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…
Reference in New Issue