From 2b017346c3132fe8d1cd384ede7969e6e2f41892 Mon Sep 17 00:00:00 2001 From: Tigran Manasyan Date: Mon, 19 Jun 2023 09:45:14 +0700 Subject: [PATCH] [debezium] Bump debezium version to 1.9.7.Final (#2156) Co-authored-by: gongzhongqiang <764629910@qq.com> --- .../base/dialect/JdbcDataSourceDialect.java | 15 +- .../relational/JdbcSourceEventDispatcher.java | 13 +- .../source/EmbeddedFlinkDatabaseHistory.java | 6 + .../source/reader/external/FetchTask.java | 6 +- .../IncrementalSourceScanFetcher.java | 8 + .../IncrementalSourceStreamFetcher.java | 9 +- .../external/JdbcSourceFetchTaskContext.java | 3 + .../EmbeddedFlinkDatabaseHistory.java | 6 + .../base/experimental/MySqlDialect.java | 19 + .../fetch/MySqlScanFetchTask.java | 222 +-- .../fetch/MySqlSourceFetchTaskContext.java | 36 +- .../fetch/MySqlStreamFetchTask.java | 32 +- .../base/experimental/utils/MySqlSchema.java | 9 +- .../cdc/connectors/tests/OracleE2eITCase.java | 1 + .../connectors/tests/PostgresE2eITCase.java | 1 + .../connectors/tests/SqlServerE2eITCase.java | 1 + .../FlinkJsonTableChangeSerializer.java | 46 +- .../internal/DebeziumChangeFetcher.java | 2 +- .../internal/FlinkDatabaseSchemaHistory.java | 13 + .../internal/FlinkOffsetBackingStore.java | 5 +- ...izedRelationalDatabaseConnectorConfig.java | 175 +++ flink-connector-mongodb-cdc/pom.xml | 6 + .../reader/fetch/MongoDBFetchTaskContext.java | 10 +- .../reader/fetch/MongoDBScanFetchTask.java | 3 + .../reader/fetch/MongoDBStreamFetchTask.java | 3 + flink-connector-mysql-cdc/pom.xml | 6 + .../mysql/debezium/DebeziumUtils.java | 8 +- .../EmbeddedFlinkDatabaseHistory.java | 12 +- .../dispatcher/EventDispatcherImpl.java | 14 +- .../debezium/reader/BinlogSplitReader.java | 2 + .../debezium/reader/SnapshotSplitReader.java | 15 +- .../task/MySqlBinlogSplitReadTask.java | 13 +- .../task/MySqlSnapshotSplitReadTask.java | 65 +- .../task/context/MySqlErrorHandler.java | 10 +- .../task/context/StatefulTaskContext.java | 35 +- .../connectors/mysql/schema/MySqlSchema.java | 36 +- .../connectors/mysql/source/MySqlSource.java | 7 +- .../mysql/source/assigners/ChunkSplitter.java | 4 +- .../source/assigners/MySqlChunkSplitter.java | 55 +- .../assigners/MySqlSnapshotSplitAssigner.java | 7 +- .../config/MySqlSourceConfigFactory.java | 11 +- .../source/reader/MySqlSourceReader.java | 10 +- .../mysql/source/utils/StatementUtils.java | 2 +- .../source/utils/TableDiscoveryUtils.java | 32 +- .../connector/mysql/MySqlConnection.java | 96 +- .../mysql/MySqlDefaultValueConverter.java | 451 ------ .../MySqlStreamingChangeEventSource.java | 311 ++-- .../listener/DefaultValueParserListener.java | 71 +- .../mysql/LegacyMySqlSourceTest.java | 9 + .../reader/BinlogSplitReaderTest.java | 29 +- .../reader/SnapshotSplitReaderTest.java | 17 +- .../PendingSplitsStateSerializerTest.java | 5 + .../source/reader/MySqlRecordEmitterTest.java | 26 +- .../source/reader/MySqlSourceReaderTest.java | 7 +- .../mysql/table/MySqlConnectorITCase.java | 2 + .../oceanbase/source/OceanBaseConnection.java | 11 +- flink-connector-oracle-cdc/pom.xml | 5 + .../cdc/connectors/oracle/OracleSource.java | 2 + .../reader/fetch/OracleScanFetchTask.java | 140 +- .../fetch/OracleSourceFetchTaskContext.java | 37 +- .../reader/fetch/OracleStreamFetchTask.java | 28 +- .../source/utils/OracleConnectionUtils.java | 10 +- .../oracle/source/utils/OracleUtils.java | 11 +- .../connector/oracle/OracleErrorHandler.java | 105 -- .../oracle/logminer/LogMinerAdapter.java | 448 ++++++ .../logminer/LogMinerChangeRecordEmitter.java | 259 +--- .../LogMinerStreamingChangeEventSource.java | 1067 +++++++++---- .../oracle/logminer/TransactionalBuffer.java | 1322 ----------------- .../AbstractLogMinerEventProcessor.java | 1235 +++++++++++++++ .../RelationalChangeRecordEmitter.java | 289 ++++ .../connectors/oracle/OracleSourceTest.java | 2 + .../oracle/source/OracleSourceITCase.java | 3 +- .../oracle/table/OracleConnectorITCase.java | 150 ++ .../source/PostgresConnectionPoolFactory.java | 4 - .../postgres/source/PostgresDialect.java | 3 + .../source/fetch/PostgresScanFetchTask.java | 83 +- .../fetch/PostgresSourceFetchTaskContext.java | 70 +- .../source/fetch/PostgresStreamFetchTask.java | 27 +- .../source/utils/CustomPostgresSchema.java | 11 +- .../postgresql/PostgresObjectUtils.java | 13 +- .../connector/postgresql/connection/Lsn.java | 149 ++ .../connection/PostgresConnection.java | 144 +- .../connectors/postgres/PostgresTestBase.java | 9 + .../source/PostgresSourceExampleTest.java | 3 +- .../postgres/source/PostgresSourceITCase.java | 4 +- .../PostgresSourceFetchTaskContextTest.java | 7 +- .../source/dialect/SqlServerDialect.java | 8 +- .../reader/fetch/SqlServerScanFetchTask.java | 133 +- .../SqlServerSourceFetchTaskContext.java | 61 +- .../fetch/SqlServerStreamFetchTask.java | 26 +- .../utils/SqlServerConnectionUtils.java | 18 +- .../source/utils/SqlServerUtils.java | 10 +- .../SqlServerStreamingChangeEventSource.java | 235 ++- .../fetch/SqlServerScanFetchTaskTest.java | 39 +- .../table/SqlServerConnectorITCase.java | 2 +- pom.xml | 2 +- tools/maven/suppressions.xml | 3 + 97 files changed, 4861 insertions(+), 3345 deletions(-) create mode 100644 flink-connector-debezium/src/main/java/io/debezium/relational/HistorizedRelationalDatabaseConnectorConfig.java delete mode 100644 flink-connector-mysql-cdc/src/main/java/io/debezium/connector/mysql/MySqlDefaultValueConverter.java delete mode 100644 flink-connector-oracle-cdc/src/main/java/io/debezium/connector/oracle/OracleErrorHandler.java create mode 100644 flink-connector-oracle-cdc/src/main/java/io/debezium/connector/oracle/logminer/LogMinerAdapter.java delete mode 100644 flink-connector-oracle-cdc/src/main/java/io/debezium/connector/oracle/logminer/TransactionalBuffer.java create mode 100644 flink-connector-oracle-cdc/src/main/java/io/debezium/connector/oracle/logminer/processor/AbstractLogMinerEventProcessor.java create mode 100644 flink-connector-oracle-cdc/src/main/java/io/debezium/relational/RelationalChangeRecordEmitter.java create mode 100644 flink-connector-postgres-cdc/src/main/java/io/debezium/connector/postgresql/connection/Lsn.java diff --git a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/dialect/JdbcDataSourceDialect.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/dialect/JdbcDataSourceDialect.java index c48a8045a5..eff5a9fc25 100644 --- a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/dialect/JdbcDataSourceDialect.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/dialect/JdbcDataSourceDialect.java @@ -17,11 +17,9 @@ package com.ververica.cdc.connectors.base.dialect; import org.apache.flink.annotation.Experimental; -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; @@ -51,18 +49,7 @@ public interface JdbcDataSourceDialect extends DataSourceDialect */ -public class JdbcSourceEventDispatcher extends EventDispatcher { +public class JdbcSourceEventDispatcher

extends EventDispatcher { private static final Logger LOG = LoggerFactory.getLogger(JdbcSourceEventDispatcher.class); public static final String HISTORY_RECORD_FIELD = "historyRecord"; @@ -132,7 +134,9 @@ public ChangeEventQueue getQueue() { @Override public void dispatchSchemaChangeEvent( - TableId dataCollectionId, SchemaChangeEventEmitter schemaChangeEventEmitter) + P partition, + TableId dataCollectionId, + SchemaChangeEventEmitter schemaChangeEventEmitter) throws InterruptedException { if (dataCollectionId != null && !filter.isIncluded(dataCollectionId)) { if (historizedSchema == null || historizedSchema.storeOnlyCapturedTables()) { @@ -141,6 +145,11 @@ public void dispatchSchemaChangeEvent( } } schemaChangeEventEmitter.emitSchemaChangeEvent(new SchemaChangeEventReceiver()); + IncrementalSnapshotChangeEventSource incrementalEventSource = + getIncrementalSnapshotChangeEventSource(); + if (incrementalEventSource != null) { + incrementalEventSource.processSchemaChange(partition, dataCollectionId); + } } @Override diff --git a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/EmbeddedFlinkDatabaseHistory.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/EmbeddedFlinkDatabaseHistory.java index df7a4321a2..c08cd5fbe5 100644 --- a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/EmbeddedFlinkDatabaseHistory.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/EmbeddedFlinkDatabaseHistory.java @@ -107,6 +107,12 @@ public void recover( listener.recoveryStopped(); } + @Override + public void recover( + Map, Map> offsets, Tables schema, DdlParser ddlParser) { + offsets.forEach((source, position) -> recover(source, position, schema, ddlParser)); + } + @Override public void stop() { listener.stopped(); diff --git a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/external/FetchTask.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/external/FetchTask.java index 3e53eccc3f..f2c5ebe82e 100644 --- a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/external/FetchTask.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/external/FetchTask.java @@ -44,8 +44,8 @@ public interface FetchTask { /** Returns the split that the task used. */ Split getSplit(); - /** Stops current task, most of the implementations don't need this. */ - default void stop() {} + /** Close current task. * */ + void close(); /** Base context used in the execution of fetch task. */ interface Context { @@ -66,5 +66,7 @@ interface Context { void rewriteOutputBuffer(Map outputBuffer, SourceRecord changeRecord); List formatMessageTimestamp(Collection snapshotRecords); + + void close() throws Exception; } } diff --git a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/external/IncrementalSourceScanFetcher.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/external/IncrementalSourceScanFetcher.java index 3849df66e1..7466363ec8 100644 --- a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/external/IncrementalSourceScanFetcher.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/external/IncrementalSourceScanFetcher.java @@ -189,6 +189,14 @@ private void checkReadException() { @Override public void close() { try { + if (taskContext != null) { + taskContext.close(); + } + + if (snapshotSplitReadTask != null) { + snapshotSplitReadTask.close(); + } + if (executorService != null) { executorService.shutdown(); if (!executorService.awaitTermination( diff --git a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/external/IncrementalSourceStreamFetcher.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/external/IncrementalSourceStreamFetcher.java index df06e78d04..36c77494d1 100644 --- a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/external/IncrementalSourceStreamFetcher.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/external/IncrementalSourceStreamFetcher.java @@ -244,10 +244,15 @@ private void configureFilter() { this.pureStreamPhaseTables.clear(); } - public void stopReadTask() { + public void stopReadTask() throws Exception { this.currentTaskRunning = false; + + if (taskContext != null) { + taskContext.close(); + } + if (streamFetchTask != null) { - streamFetchTask.stop(); + streamFetchTask.close(); } } } diff --git a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/external/JdbcSourceFetchTaskContext.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/external/JdbcSourceFetchTaskContext.java index c98d259a65..e3b275c030 100644 --- a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/external/JdbcSourceFetchTaskContext.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/external/JdbcSourceFetchTaskContext.java @@ -27,6 +27,7 @@ import io.debezium.data.Envelope; import io.debezium.pipeline.ErrorHandler; import io.debezium.pipeline.spi.OffsetContext; +import io.debezium.pipeline.spi.Partition; import io.debezium.relational.RelationalDatabaseSchema; import io.debezium.relational.Table; import io.debezium.relational.TableId; @@ -168,4 +169,6 @@ public SchemaNameAdjuster getSchemaNameAdjuster() { public abstract JdbcSourceEventDispatcher getDispatcher(); public abstract OffsetContext getOffsetContext(); + + public abstract Partition getPartition(); } diff --git a/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/EmbeddedFlinkDatabaseHistory.java b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/EmbeddedFlinkDatabaseHistory.java index 84295fb4f6..39d85edfe2 100644 --- a/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/EmbeddedFlinkDatabaseHistory.java +++ b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/EmbeddedFlinkDatabaseHistory.java @@ -107,6 +107,12 @@ public void recover( listener.recoveryStopped(); } + @Override + public void recover( + Map, Map> offsets, Tables schema, DdlParser ddlParser) { + offsets.forEach((source, position) -> recover(source, position, schema, ddlParser)); + } + @Override public void stop() { listener.stopped(); diff --git a/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/MySqlDialect.java b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/MySqlDialect.java index ff4d4bd516..afac6096f6 100644 --- a/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/MySqlDialect.java +++ b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/MySqlDialect.java @@ -29,12 +29,14 @@ import com.ververica.cdc.connectors.base.experimental.fetch.MySqlStreamFetchTask; import com.ververica.cdc.connectors.base.experimental.utils.MySqlSchema; import com.ververica.cdc.connectors.base.experimental.utils.TableDiscoveryUtils; +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.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.connector.mysql.MySqlConnection; +import io.debezium.jdbc.JdbcConfiguration; import io.debezium.jdbc.JdbcConnection; import io.debezium.relational.TableId; import io.debezium.relational.history.TableChanges; @@ -54,6 +56,8 @@ @Experimental public class MySqlDialect implements JdbcDataSourceDialect { + private static final String QUOTED_CHARACTER = "`"; + private static final long serialVersionUID = 1L; private final MySqlSourceConfigFactory configFactory; private final MySqlSourceConfig sourceConfig; @@ -69,6 +73,21 @@ public String getName() { return "MySQL"; } + public JdbcConnection openJdbcConnection(JdbcSourceConfig sourceConfig) { + JdbcConnection jdbc = + new JdbcConnection( + JdbcConfiguration.adapt(sourceConfig.getDbzConfiguration()), + new JdbcConnectionFactory(sourceConfig, getPooledDataSourceFactory()), + QUOTED_CHARACTER, + QUOTED_CHARACTER); + try { + jdbc.connect(); + } catch (Exception e) { + throw new FlinkRuntimeException(e); + } + return jdbc; + } + @Override public Offset displayCurrentOffset(JdbcSourceConfig sourceConfig) { try (JdbcConnection jdbcConnection = openJdbcConnection(sourceConfig)) { diff --git a/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/fetch/MySqlScanFetchTask.java b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/fetch/MySqlScanFetchTask.java index 04993f1f2f..a6a103dc97 100644 --- a/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/fetch/MySqlScanFetchTask.java +++ b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/fetch/MySqlScanFetchTask.java @@ -30,16 +30,14 @@ import io.debezium.connector.mysql.MySqlConnectorConfig; import io.debezium.connector.mysql.MySqlDatabaseSchema; import io.debezium.connector.mysql.MySqlOffsetContext; -import io.debezium.connector.mysql.MySqlValueConverters; +import io.debezium.connector.mysql.MySqlPartition; import io.debezium.heartbeat.Heartbeat; import io.debezium.pipeline.EventDispatcher; import io.debezium.pipeline.source.AbstractSnapshotChangeEventSource; import io.debezium.pipeline.source.spi.ChangeEventSource; import io.debezium.pipeline.source.spi.SnapshotProgressListener; import io.debezium.pipeline.spi.ChangeRecordEmitter; -import io.debezium.pipeline.spi.OffsetContext; import io.debezium.pipeline.spi.SnapshotResult; -import io.debezium.relational.Column; import io.debezium.relational.RelationalSnapshotChangeEventSource; import io.debezium.relational.SnapshotChangeRecordEmitter; import io.debezium.relational.Table; @@ -52,15 +50,11 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.UnsupportedEncodingException; -import java.sql.Blob; import java.sql.PreparedStatement; import java.sql.ResultSet; import java.sql.SQLException; -import java.sql.Types; import java.time.Duration; import java.util.ArrayList; -import java.util.Calendar; import java.util.Map; import static com.ververica.cdc.connectors.base.experimental.utils.MySqlConnectionUtils.createMySqlConnection; @@ -85,6 +79,11 @@ public SnapshotSplit getSplit() { return split; } + @Override + public void close() { + taskRunning = false; + } + @Override public boolean isRunning() { return taskRunning; @@ -105,9 +104,11 @@ public void execute(Context context) throws Exception { split); SnapshotSplitChangeEventSourceContext changeEventSourceContext = new SnapshotSplitChangeEventSourceContext(); - SnapshotResult snapshotResult = + SnapshotResult snapshotResult = snapshotSplitReadTask.execute( - changeEventSourceContext, sourceFetchContext.getOffsetContext()); + changeEventSourceContext, + sourceFetchContext.getPartition(), + sourceFetchContext.getOffsetContext()); final StreamSplit backfillBinlogSplit = createBackfillBinlogSplit(changeEventSourceContext); // optimization that skip the binlog read when the low watermark equals high @@ -119,7 +120,7 @@ public void execute(Context context) throws Exception { if (!binlogBackfillRequired) { dispatchBinlogEndEvent( backfillBinlogSplit, - ((MySqlSourceFetchTaskContext) context).getOffsetContext().getPartition(), + sourceFetchContext.getPartition().getSourcePartition(), ((MySqlSourceFetchTaskContext) context).getDispatcher()); taskRunning = false; return; @@ -130,6 +131,7 @@ public void execute(Context context) throws Exception { createBackfillBinlogReadTask(backfillBinlogSplit, sourceFetchContext); backfillBinlogReadTask.execute( new SnapshotBinlogSplitChangeEventSourceContext(), + sourceFetchContext.getPartition(), sourceFetchContext.getOffsetContext()); } else { taskRunning = false; @@ -151,11 +153,6 @@ private StreamSplit createBackfillBinlogSplit( private MySqlBinlogSplitReadTask createBackfillBinlogReadTask( StreamSplit backfillBinlogSplit, MySqlSourceFetchTaskContext context) { - final MySqlOffsetContext.Loader loader = - new MySqlOffsetContext.Loader(context.getSourceConfig().getDbzConnectorConfig()); - final MySqlOffsetContext mySqlOffsetContext = - (MySqlOffsetContext) - loader.load(backfillBinlogSplit.getStartingOffset().getOffset()); // we should only capture events for the current table, // otherwise, we may can't find corresponding schema Configuration dezConf = @@ -169,7 +166,6 @@ private MySqlBinlogSplitReadTask createBackfillBinlogReadTask( // task to read binlog and backfill for current split return new MySqlBinlogSplitReadTask( new MySqlConnectorConfig(dezConf), - mySqlOffsetContext, createMySqlConnection(context.getSourceConfig().getDbzConfiguration()), context.getDispatcher(), context.getErrorHandler(), @@ -181,7 +177,7 @@ private MySqlBinlogSplitReadTask createBackfillBinlogReadTask( private void dispatchBinlogEndEvent( StreamSplit backFillBinlogSplit, Map sourcePartition, - JdbcSourceEventDispatcher eventDispatcher) + JdbcSourceEventDispatcher eventDispatcher) throws InterruptedException { eventDispatcher.dispatchWatermarkEvent( sourcePartition, @@ -191,7 +187,8 @@ private void dispatchBinlogEndEvent( } /** A wrapped task to fetch snapshot split of table. */ - public static class MySqlSnapshotSplitReadTask extends AbstractSnapshotChangeEventSource { + public static class MySqlSnapshotSplitReadTask + extends AbstractSnapshotChangeEventSource { private static final Logger LOG = LoggerFactory.getLogger(MySqlSnapshotSplitReadTask.class); @@ -201,19 +198,19 @@ public static class MySqlSnapshotSplitReadTask extends AbstractSnapshotChangeEve private final MySqlConnectorConfig connectorConfig; private final MySqlDatabaseSchema databaseSchema; private final MySqlConnection jdbcConnection; - private final JdbcSourceEventDispatcher dispatcher; + private final JdbcSourceEventDispatcher dispatcher; private final Clock clock; private final SnapshotSplit snapshotSplit; private final MySqlOffsetContext offsetContext; - private final SnapshotProgressListener snapshotProgressListener; + private final SnapshotProgressListener snapshotProgressListener; public MySqlSnapshotSplitReadTask( MySqlConnectorConfig connectorConfig, MySqlOffsetContext previousOffset, - SnapshotProgressListener snapshotProgressListener, + SnapshotProgressListener snapshotProgressListener, MySqlDatabaseSchema databaseSchema, MySqlConnection jdbcConnection, - JdbcSourceEventDispatcher dispatcher, + JdbcSourceEventDispatcher dispatcher, SnapshotSplit snapshotSplit) { super(connectorConfig, snapshotProgressListener); this.offsetContext = previousOffset; @@ -227,13 +224,15 @@ public MySqlSnapshotSplitReadTask( } @Override - public SnapshotResult execute( - ChangeEventSourceContext context, OffsetContext previousOffset) + public SnapshotResult execute( + ChangeEventSourceContext context, + MySqlPartition partition, + MySqlOffsetContext previousOffset) throws InterruptedException { - SnapshottingTask snapshottingTask = getSnapshottingTask(previousOffset); - final SnapshotContext ctx; + SnapshottingTask snapshottingTask = getSnapshottingTask(partition, previousOffset); + final MySqlSnapshotContext ctx; try { - ctx = prepare(context); + ctx = prepare(partition); } catch (Exception e) { LOG.error("Failed to initialize snapshot context.", e); throw new RuntimeException(e); @@ -249,14 +248,13 @@ public SnapshotResult execute( } @Override - protected SnapshotResult doExecute( + protected SnapshotResult doExecute( ChangeEventSourceContext context, - OffsetContext previousOffset, + MySqlOffsetContext previousOffset, SnapshotContext snapshotContext, SnapshottingTask snapshottingTask) throws Exception { - final RelationalSnapshotChangeEventSource.RelationalSnapshotContext ctx = - (RelationalSnapshotChangeEventSource.RelationalSnapshotContext) snapshotContext; + final MySqlSnapshotContext ctx = (MySqlSnapshotContext) snapshotContext; ctx.offset = offsetContext; final BinlogOffset lowWatermark = currentBinlogOffset(jdbcConnection); @@ -266,7 +264,10 @@ protected SnapshotResult doExecute( snapshotSplit); ((SnapshotSplitChangeEventSourceContext) (context)).setLowWatermark(lowWatermark); dispatcher.dispatchWatermarkEvent( - offsetContext.getPartition(), snapshotSplit, lowWatermark, WatermarkKind.LOW); + snapshotContext.partition.getSourcePartition(), + snapshotSplit, + lowWatermark, + WatermarkKind.LOW); LOG.info("Snapshot step 2 - Snapshotting data"); createDataEvents(ctx, snapshotSplit.getTableId()); @@ -278,34 +279,36 @@ protected SnapshotResult doExecute( snapshotSplit); ((SnapshotSplitChangeEventSourceContext) (context)).setHighWatermark(highWatermark); dispatcher.dispatchWatermarkEvent( - offsetContext.getPartition(), snapshotSplit, highWatermark, WatermarkKind.HIGH); + snapshotContext.partition.getSourcePartition(), + snapshotSplit, + highWatermark, + WatermarkKind.HIGH); return SnapshotResult.completed(ctx.offset); } @Override - protected SnapshottingTask getSnapshottingTask(OffsetContext previousOffset) { + protected SnapshottingTask getSnapshottingTask( + MySqlPartition partition, MySqlOffsetContext previousOffset) { return new SnapshottingTask(false, true); } @Override - protected SnapshotContext prepare(ChangeEventSourceContext changeEventSourceContext) - throws Exception { - return new MySqlSnapshotContext(); + protected MySqlSnapshotContext prepare(MySqlPartition partition) throws Exception { + return new MySqlSnapshotContext(partition); } private static class MySqlSnapshotContext - extends RelationalSnapshotChangeEventSource.RelationalSnapshotContext { + extends RelationalSnapshotChangeEventSource.RelationalSnapshotContext< + MySqlPartition, MySqlOffsetContext> { - public MySqlSnapshotContext() throws SQLException { - super(""); + public MySqlSnapshotContext(MySqlPartition partition) throws SQLException { + super(partition, ""); } } - private void createDataEvents( - RelationalSnapshotChangeEventSource.RelationalSnapshotContext snapshotContext, - TableId tableId) + private void createDataEvents(MySqlSnapshotContext snapshotContext, TableId tableId) throws Exception { - EventDispatcher.SnapshotReceiver snapshotReceiver = + EventDispatcher.SnapshotReceiver snapshotReceiver = dispatcher.getSnapshotChangeEventReceiver(); LOG.debug("Snapshotting table {}", tableId); createDataEventsForTable( @@ -315,8 +318,8 @@ private void createDataEvents( /** Dispatches the data change events for the records of a single table. */ private void createDataEventsForTable( - RelationalSnapshotChangeEventSource.RelationalSnapshotContext snapshotContext, - EventDispatcher.SnapshotReceiver snapshotReceiver, + MySqlSnapshotContext snapshotContext, + EventDispatcher.SnapshotReceiver snapshotReceiver, Table table) throws InterruptedException { @@ -356,12 +359,8 @@ private void createDataEventsForTable( 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, table); - } + final Object[] row = + jdbcConnection.rowToArray(table, databaseSchema, rs, columnArray); if (logTimer.expired()) { long stop = clock.currentTimeInMillis(); LOG.info( @@ -369,10 +368,12 @@ private void createDataEventsForTable( rows, snapshotSplit.splitId(), Strings.duration(stop - exportStart)); - snapshotProgressListener.rowsScanned(table.id(), rows); + snapshotProgressListener.rowsScanned( + snapshotContext.partition, table.id(), rows); logTimer = getTableScanLogTimer(); } dispatcher.dispatchSnapshotEvent( + snapshotContext.partition, table.id(), getChangeRecordEmitter(snapshotContext, table.id(), row), snapshotReceiver); @@ -387,128 +388,23 @@ private void createDataEventsForTable( } } - protected ChangeRecordEmitter getChangeRecordEmitter( - SnapshotContext snapshotContext, TableId tableId, Object[] row) { + protected ChangeRecordEmitter getChangeRecordEmitter( + MySqlSnapshotContext snapshotContext, TableId tableId, Object[] row) { snapshotContext.offset.event(tableId, clock.currentTime()); - return new SnapshotChangeRecordEmitter(snapshotContext.offset, row, clock); + return new SnapshotChangeRecordEmitter<>( + snapshotContext.partition, snapshotContext.offset, row, clock); } private Threads.Timer getTableScanLogTimer() { return Threads.timer(clock, LOG_INTERVAL); } - - /** - * Read JDBC return value and deal special type like time, timestamp. - * - *

Note https://issues.redhat.com/browse/DBZ-3238 has fixed this issue, please remove - * this method once we bump Debezium version to 1.6 - */ - private Object readField(ResultSet rs, int fieldNo, Column actualColumn, Table actualTable) - throws SQLException { - if (actualColumn.jdbcType() == Types.TIME) { - return readTimeField(rs, fieldNo); - } else if (actualColumn.jdbcType() == Types.DATE) { - return readDateField(rs, fieldNo, actualColumn, actualTable); - } - // This is for DATETIME columns (a logical date + time without time zone) - // by reading them with a calendar based on the default time zone, we make sure that the - // value - // is constructed correctly using the database's (or connection's) time zone - else if (actualColumn.jdbcType() == Types.TIMESTAMP) { - return readTimestampField(rs, fieldNo, actualColumn, actualTable); - } - // JDBC's rs.GetObject() will return a Boolean for all TINYINT(1) columns. - // TINYINT columns are reprtoed as SMALLINT by JDBC driver - else if (actualColumn.jdbcType() == Types.TINYINT - || actualColumn.jdbcType() == Types.SMALLINT) { - // It seems that rs.wasNull() returns false when default value is set and NULL is - // inserted - // We thus need to use getObject() to identify if the value was provided and if yes - // then - // read it again to get correct scale - return rs.getObject(fieldNo) == null ? null : rs.getInt(fieldNo); - } - // DBZ-2673 - // It is necessary to check the type names as types like ENUM and SET are - // also reported as JDBC type char - else if ("CHAR".equals(actualColumn.typeName()) - || "VARCHAR".equals(actualColumn.typeName()) - || "TEXT".equals(actualColumn.typeName())) { - return rs.getBytes(fieldNo); - } else { - return rs.getObject(fieldNo); - } - } - - /** - * As MySQL connector/J implementation is broken for MySQL type "TIME" we have to use a - * binary-ish workaround. https://issues.jboss.org/browse/DBZ-342 - */ - private Object readTimeField(ResultSet rs, int fieldNo) throws SQLException { - Blob b = rs.getBlob(fieldNo); - if (b == null) { - return null; // Don't continue parsing time field if it is null - } - - try { - return MySqlValueConverters.stringToDuration( - new String(b.getBytes(1, (int) (b.length())), "UTF-8")); - } catch (UnsupportedEncodingException e) { - LOG.error("Could not read MySQL TIME value as UTF-8"); - throw new RuntimeException(e); - } - } - - /** - * In non-string mode the date field can contain zero in any of the date part which we need - * to handle as all-zero. - */ - private Object readDateField(ResultSet rs, int fieldNo, Column column, Table table) - throws SQLException { - Blob b = rs.getBlob(fieldNo); - if (b == null) { - return null; // Don't continue parsing date field if it is null - } - - try { - return MySqlValueConverters.stringToLocalDate( - new String(b.getBytes(1, (int) (b.length())), "UTF-8"), column, table); - } catch (UnsupportedEncodingException e) { - LOG.error("Could not read MySQL TIME value as UTF-8"); - throw new RuntimeException(e); - } - } - - /** - * In non-string mode the time field can contain zero in any of the date part which we need - * to handle as all-zero. - */ - private Object readTimestampField(ResultSet rs, int fieldNo, Column column, Table table) - throws SQLException { - Blob b = rs.getBlob(fieldNo); - if (b == null) { - return null; // Don't continue parsing timestamp field if it is null - } - - try { - return MySqlValueConverters.containsZeroValuesInDatePart( - (new String(b.getBytes(1, (int) (b.length())), "UTF-8")), - column, - table) - ? null - : rs.getTimestamp(fieldNo, Calendar.getInstance()); - } catch (UnsupportedEncodingException e) { - LOG.error("Could not read MySQL TIME value as UTF-8"); - throw new RuntimeException(e); - } - } } /** * {@link ChangeEventSource.ChangeEventSourceContext} implementation that keeps low/high * watermark for each {@link SnapshotSplit}. */ - public class SnapshotSplitChangeEventSourceContext + public static class SnapshotSplitChangeEventSourceContext implements ChangeEventSource.ChangeEventSourceContext { private BinlogOffset lowWatermark; diff --git a/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/fetch/MySqlSourceFetchTaskContext.java b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/fetch/MySqlSourceFetchTaskContext.java index 49b4a33c4a..cf1eecc322 100644 --- a/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/fetch/MySqlSourceFetchTaskContext.java +++ b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/fetch/MySqlSourceFetchTaskContext.java @@ -37,6 +37,7 @@ import io.debezium.connector.mysql.MySqlDatabaseSchema; import io.debezium.connector.mysql.MySqlErrorHandler; import io.debezium.connector.mysql.MySqlOffsetContext; +import io.debezium.connector.mysql.MySqlPartition; import io.debezium.connector.mysql.MySqlStreamingChangeEventSourceMetrics; import io.debezium.connector.mysql.MySqlTaskContext; import io.debezium.connector.mysql.MySqlTopicSelector; @@ -46,6 +47,7 @@ import io.debezium.pipeline.metrics.SnapshotChangeEventSourceMetrics; import io.debezium.pipeline.source.spi.EventMetadataProvider; import io.debezium.pipeline.spi.OffsetContext; +import io.debezium.pipeline.spi.Offsets; import io.debezium.relational.Table; import io.debezium.relational.TableId; import io.debezium.relational.Tables; @@ -75,10 +77,11 @@ public class MySqlSourceFetchTaskContext extends JdbcSourceFetchTaskContext { private MySqlDatabaseSchema databaseSchema; private MySqlTaskContextImpl taskContext; private MySqlOffsetContext offsetContext; - private SnapshotChangeEventSourceMetrics snapshotChangeEventSourceMetrics; + private SnapshotChangeEventSourceMetrics snapshotChangeEventSourceMetrics; private MySqlStreamingChangeEventSourceMetrics streamingChangeEventSourceMetrics; private TopicSelector topicSelector; - private JdbcSourceEventDispatcher dispatcher; + private JdbcSourceEventDispatcher dispatcher; + private MySqlPartition mySqlPartition; private ChangeEventQueue queue; private MySqlErrorHandler errorHandler; @@ -109,6 +112,8 @@ public void configure(SourceSplitBase sourceSplitBase) { this.offsetContext = loadStartingOffsetState( new MySqlOffsetContext.Loader(connectorConfig), sourceSplitBase); + this.mySqlPartition = new MySqlPartition(connectorConfig.getLogicalName()); + validateAndLoadDatabaseHistory(offsetContext, databaseSchema); this.taskContext = @@ -131,7 +136,7 @@ public void configure(SourceSplitBase sourceSplitBase) { // .buffering() .build(); this.dispatcher = - new JdbcSourceEventDispatcher( + new JdbcSourceEventDispatcher<>( connectorConfig, topicSelector, databaseSchema, @@ -152,7 +157,7 @@ public void configure(SourceSplitBase sourceSplitBase) { (MySqlStreamingChangeEventSourceMetrics) changeEventSourceMetricsFactory.getStreamingMetrics( taskContext, queue, metadataProvider); - this.errorHandler = new MySqlErrorHandler(connectorConfig.getLogicalName(), queue); + this.errorHandler = new MySqlErrorHandler(connectorConfig, queue); } @Override @@ -168,6 +173,11 @@ public BinaryLogClient getBinaryLogClient() { return binaryLogClient; } + @Override + public MySqlPartition getPartition() { + return mySqlPartition; + } + public MySqlTaskContextImpl getTaskContext() { return taskContext; } @@ -182,7 +192,7 @@ public MySqlOffsetContext getOffsetContext() { return offsetContext; } - public SnapshotChangeEventSourceMetrics getSnapshotChangeEventSourceMetrics() { + public SnapshotChangeEventSourceMetrics getSnapshotChangeEventSourceMetrics() { return snapshotChangeEventSourceMetrics; } @@ -206,7 +216,7 @@ public RowType getSplitType(Table table) { } @Override - public JdbcSourceEventDispatcher getDispatcher() { + public JdbcSourceEventDispatcher getDispatcher() { return dispatcher; } @@ -225,16 +235,20 @@ public Offset getStreamOffset(SourceRecord sourceRecord) { return MySqlUtils.getBinlogPosition(sourceRecord); } + @Override + public void close() throws Exception { + connection.close(); + } + /** Loads the connector's persistent offset (if present) via the given loader. */ private MySqlOffsetContext loadStartingOffsetState( - OffsetContext.Loader loader, SourceSplitBase mySqlSplit) { + OffsetContext.Loader loader, SourceSplitBase mySqlSplit) { Offset offset = mySqlSplit.isSnapshotSplit() ? BinlogOffset.INITIAL_OFFSET : mySqlSplit.asStreamSplit().getStartingOffset(); - MySqlOffsetContext mySqlOffsetContext = - (MySqlOffsetContext) loader.load(offset.getOffset()); + MySqlOffsetContext mySqlOffsetContext = loader.load(offset.getOffset()); if (!isBinlogAvailable(mySqlOffsetContext)) { throw new IllegalStateException( @@ -274,11 +288,11 @@ private boolean isBinlogAvailable(MySqlOffsetContext offset) { private void validateAndLoadDatabaseHistory( MySqlOffsetContext offset, MySqlDatabaseSchema schema) { schema.initializeStorage(); - schema.recover(offset); + schema.recover(Offsets.of(mySqlPartition, offset)); } /** A subclass implementation of {@link MySqlTaskContext} which reuses one BinaryLogClient. */ - public class MySqlTaskContextImpl extends MySqlTaskContext { + public static class MySqlTaskContextImpl extends MySqlTaskContext { private final BinaryLogClient reusedBinaryLogClient; diff --git a/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/fetch/MySqlStreamFetchTask.java b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/fetch/MySqlStreamFetchTask.java index e3bb780a74..00b85846bc 100644 --- a/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/fetch/MySqlStreamFetchTask.java +++ b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/fetch/MySqlStreamFetchTask.java @@ -27,6 +27,7 @@ import io.debezium.connector.mysql.MySqlConnection; import io.debezium.connector.mysql.MySqlConnectorConfig; import io.debezium.connector.mysql.MySqlOffsetContext; +import io.debezium.connector.mysql.MySqlPartition; import io.debezium.connector.mysql.MySqlStreamingChangeEventSource; import io.debezium.connector.mysql.MySqlStreamingChangeEventSourceMetrics; import io.debezium.connector.mysql.MySqlTaskContext; @@ -59,7 +60,6 @@ public void execute(Context context) throws Exception { binlogSplitReadTask = new MySqlBinlogSplitReadTask( sourceFetchContext.getDbzConnectorConfig(), - sourceFetchContext.getOffsetContext(), sourceFetchContext.getConnection(), sourceFetchContext.getDispatcher(), sourceFetchContext.getErrorHandler(), @@ -69,7 +69,9 @@ public void execute(Context context) throws Exception { BinlogSplitChangeEventSourceContext changeEventSourceContext = new BinlogSplitChangeEventSourceContext(); binlogSplitReadTask.execute( - changeEventSourceContext, sourceFetchContext.getOffsetContext()); + changeEventSourceContext, + sourceFetchContext.getPartition(), + sourceFetchContext.getOffsetContext()); } @Override @@ -82,6 +84,11 @@ public StreamSplit getSplit() { return split; } + @Override + public void close() { + taskRunning = false; + } + /** * A wrapped task to read all binlog for table and also supports read bounded (from lowWatermark * to highWatermark) binlog. @@ -90,16 +97,14 @@ public static class MySqlBinlogSplitReadTask extends MySqlStreamingChangeEventSo private static final Logger LOG = LoggerFactory.getLogger(MySqlBinlogSplitReadTask.class); private final StreamSplit binlogSplit; - private final MySqlOffsetContext offsetContext; - private final JdbcSourceEventDispatcher dispatcher; + private final JdbcSourceEventDispatcher dispatcher; private final ErrorHandler errorHandler; private ChangeEventSourceContext context; public MySqlBinlogSplitReadTask( MySqlConnectorConfig connectorConfig, - MySqlOffsetContext offsetContext, MySqlConnection connection, - JdbcSourceEventDispatcher dispatcher, + JdbcSourceEventDispatcher dispatcher, ErrorHandler errorHandler, MySqlTaskContext taskContext, MySqlStreamingChangeEventSourceMetrics metrics, @@ -114,20 +119,23 @@ public MySqlBinlogSplitReadTask( metrics); this.binlogSplit = binlogSplit; this.dispatcher = dispatcher; - this.offsetContext = offsetContext; this.errorHandler = errorHandler; } @Override - public void execute(ChangeEventSourceContext context, MySqlOffsetContext offsetContext) + public void execute( + ChangeEventSourceContext context, + MySqlPartition partition, + MySqlOffsetContext offsetContext) throws InterruptedException { this.context = context; - super.execute(context, offsetContext); + super.execute(context, partition, offsetContext); } @Override - protected void handleEvent(MySqlOffsetContext offsetContext, Event event) { - super.handleEvent(offsetContext, event); + protected void handleEvent( + MySqlPartition partition, MySqlOffsetContext offsetContext, Event event) { + super.handleEvent(partition, offsetContext, event); // check do we need to stop for fetch binlog for snapshot split. if (isBoundedRead()) { final BinlogOffset currentBinlogOffset = @@ -137,7 +145,7 @@ protected void handleEvent(MySqlOffsetContext offsetContext, Event event) { // send binlog end event try { dispatcher.dispatchWatermarkEvent( - offsetContext.getPartition(), + partition.getSourcePartition(), binlogSplit, currentBinlogOffset, WatermarkKind.END); diff --git a/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/utils/MySqlSchema.java b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/utils/MySqlSchema.java index abe69e4560..5ee98e98bd 100644 --- a/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/utils/MySqlSchema.java +++ b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/utils/MySqlSchema.java @@ -22,6 +22,7 @@ import io.debezium.connector.mysql.MySqlConnectorConfig; import io.debezium.connector.mysql.MySqlDatabaseSchema; import io.debezium.connector.mysql.MySqlOffsetContext; +import io.debezium.connector.mysql.MySqlPartition; import io.debezium.jdbc.JdbcConnection; import io.debezium.relational.TableId; import io.debezium.relational.history.TableChanges; @@ -76,9 +77,15 @@ private TableChanges.TableChange readTableSchema(JdbcConnection jdbc, TableId ta final String ddl = rs.getString(2); final MySqlOffsetContext offsetContext = MySqlOffsetContext.initial(connectorConfig); + final MySqlPartition partition = + new MySqlPartition(connectorConfig.getLogicalName()); List schemaChangeEvents = databaseSchema.parseSnapshotDdl( - ddl, tableId.catalog(), offsetContext, Instant.now()); + partition, + ddl, + tableId.catalog(), + offsetContext, + Instant.now()); for (SchemaChangeEvent schemaChangeEvent : schemaChangeEvents) { for (TableChanges.TableChange tableChange : schemaChangeEvent.getTableChanges()) { diff --git a/flink-cdc-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/OracleE2eITCase.java b/flink-cdc-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/OracleE2eITCase.java index 57fcb0bba4..a16e46ddae 100644 --- a/flink-cdc-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/OracleE2eITCase.java +++ b/flink-cdc-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/OracleE2eITCase.java @@ -81,6 +81,7 @@ public void after() { public void testOracleCDC() throws Exception { List sqlLines = Arrays.asList( + "SET 'execution.checkpointing.interval' = '3s';", "CREATE TABLE products_source (", " ID INT NOT NULL,", " NAME STRING,", diff --git a/flink-cdc-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/PostgresE2eITCase.java b/flink-cdc-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/PostgresE2eITCase.java index 111284e6bc..737573d41a 100644 --- a/flink-cdc-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/PostgresE2eITCase.java +++ b/flink-cdc-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/PostgresE2eITCase.java @@ -99,6 +99,7 @@ public void after() { List sourceSql = Arrays.asList( + "SET 'execution.checkpointing.interval' = '3s';", "CREATE TABLE products_source (", " `id` INT NOT NULL,", " name STRING,", diff --git a/flink-cdc-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/SqlServerE2eITCase.java b/flink-cdc-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/SqlServerE2eITCase.java index 6e2e744efb..2459a1402d 100644 --- a/flink-cdc-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/SqlServerE2eITCase.java +++ b/flink-cdc-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/SqlServerE2eITCase.java @@ -104,6 +104,7 @@ public void after() { public void testSqlServerCDC() throws Exception { List sqlLines = Arrays.asList( + "SET 'execution.checkpointing.interval' = '3s';", "CREATE TABLE products_source (", " `id` INT NOT NULL,", " name STRING,", diff --git a/flink-connector-debezium/src/main/java/com/ververica/cdc/debezium/history/FlinkJsonTableChangeSerializer.java b/flink-connector-debezium/src/main/java/com/ververica/cdc/debezium/history/FlinkJsonTableChangeSerializer.java index bbc5cc5ad1..b188477d6a 100644 --- a/flink-connector-debezium/src/main/java/com/ververica/cdc/debezium/history/FlinkJsonTableChangeSerializer.java +++ b/flink-connector-debezium/src/main/java/com/ververica/cdc/debezium/history/FlinkJsonTableChangeSerializer.java @@ -30,6 +30,7 @@ import io.debezium.relational.history.TableChanges.TableChangeType; import java.util.List; +import java.util.Optional; import java.util.stream.Collectors; import java.util.stream.StreamSupport; @@ -57,6 +58,7 @@ public Document toDocument(TableChange tableChange) { document.setString("type", tableChange.getType().name()); document.setString("id", tableChange.getId().toDoubleQuotedString()); document.setDocument("table", toDocument(tableChange.getTable())); + document.setString("comment", tableChange.getTable().comment()); return document; } @@ -98,10 +100,15 @@ private Document toDocument(Column column) { document.setBoolean("optional", column.isOptional()); document.setBoolean("autoIncremented", column.isAutoIncremented()); document.setBoolean("generated", column.isGenerated()); + document.setString("comment", column.comment()); + document.setBoolean("hasDefaultValue", column.hasDefaultValue()); - // BEGIN FLINK MODIFICATION - document.setArray("enumValues", column.enumValues().toArray()); - // END FLINK MODIFICATION + column.defaultValueExpression() + .ifPresent(d -> document.setString("defaultValueExpression", d)); + + Optional.ofNullable(column.enumValues()) + .map(List::toArray) + .ifPresent(enums -> document.setArray("enumValues", enums)); return document; } @@ -131,6 +138,9 @@ private static Table fromDocument(TableId id, Document document) { Table.editor() .tableId(id) .setDefaultCharsetName(document.getString("defaultCharsetName")); + if (document.getString("comment") != null) { + editor.setComment(document.getString("comment")); + } document.getArray("columns") .streamValues() @@ -161,22 +171,34 @@ private static Table fromDocument(TableId id, Document document) { columnEditor.scale(scale); } - columnEditor - .position(v.getInteger("position")) - .optional(v.getBoolean("optional")) - .autoIncremented(v.getBoolean("autoIncremented")) - .generated(v.getBoolean("generated")); + String columnComment = v.getString("comment"); + if (columnComment != null) { + columnEditor.comment(columnComment); + } + + Boolean hasDefaultValue = v.getBoolean("hasDefaultValue"); + String defaultValueExpression = v.getString("defaultValueExpression"); + if (defaultValueExpression != null) { + columnEditor.defaultValueExpression(defaultValueExpression); + } else if (Boolean.TRUE.equals(hasDefaultValue)) { + columnEditor.defaultValueExpression(null); + } - // BEGIN FLINK MODIFICATION Array enumValues = v.getArray("enumValues"); if (enumValues != null && !enumValues.isEmpty()) { - columnEditor.enumValues( + List enumValueList = enumValues .streamValues() .map(Value::asString) - .collect(Collectors.toList())); + .collect(Collectors.toList()); + columnEditor.enumValues(enumValueList); } - // END FLINK MODIFICATION + + columnEditor + .position(v.getInteger("position")) + .optional(v.getBoolean("optional")) + .autoIncremented(v.getBoolean("autoIncremented")) + .generated(v.getBoolean("generated")); return columnEditor.create(); }) diff --git a/flink-connector-debezium/src/main/java/com/ververica/cdc/debezium/internal/DebeziumChangeFetcher.java b/flink-connector-debezium/src/main/java/com/ververica/cdc/debezium/internal/DebeziumChangeFetcher.java index dd5cef4d56..24366c806b 100644 --- a/flink-connector-debezium/src/main/java/com/ververica/cdc/debezium/internal/DebeziumChangeFetcher.java +++ b/flink-connector-debezium/src/main/java/com/ververica/cdc/debezium/internal/DebeziumChangeFetcher.java @@ -235,7 +235,7 @@ private void handleBatch(List> changeEve deserialization.deserialize(record, debeziumCollector); - if (!isSnapshotRecord(record)) { + if (isInDbSnapshotPhase && !isSnapshotRecord(record)) { LOG.debug("Snapshot phase finishes."); isInDbSnapshotPhase = false; } diff --git a/flink-connector-debezium/src/main/java/com/ververica/cdc/debezium/internal/FlinkDatabaseSchemaHistory.java b/flink-connector-debezium/src/main/java/com/ververica/cdc/debezium/internal/FlinkDatabaseSchemaHistory.java index 51432bdca0..f9dbefdd09 100644 --- a/flink-connector-debezium/src/main/java/com/ververica/cdc/debezium/internal/FlinkDatabaseSchemaHistory.java +++ b/flink-connector-debezium/src/main/java/com/ververica/cdc/debezium/internal/FlinkDatabaseSchemaHistory.java @@ -148,6 +148,19 @@ public void recover( listener.recoveryStopped(); } + @Override + public void recover( + Map, Map> offsets, Tables schema, DdlParser ddlParser) { + listener.recoveryStarted(); + for (SchemaRecord record : latestTables.values()) { + TableChange tableChange = + FlinkJsonTableChangeSerializer.fromDocument( + record.getTableChangeDoc(), useCatalogBeforeSchema); + schema.overwriteTable(tableChange.getTable()); + } + listener.recoveryStopped(); + } + @Override public void stop() { if (instanceName != null) { diff --git a/flink-connector-debezium/src/main/java/com/ververica/cdc/debezium/internal/FlinkOffsetBackingStore.java b/flink-connector-debezium/src/main/java/com/ververica/cdc/debezium/internal/FlinkOffsetBackingStore.java index 469054defc..8147074eef 100644 --- a/flink-connector-debezium/src/main/java/com/ververica/cdc/debezium/internal/FlinkOffsetBackingStore.java +++ b/flink-connector-debezium/src/main/java/com/ververica/cdc/debezium/internal/FlinkOffsetBackingStore.java @@ -63,6 +63,7 @@ public class FlinkOffsetBackingStore implements OffsetBackingStore { protected Map data = new HashMap<>(); protected ExecutorService executor; + @SuppressWarnings("unchecked") @Override public void configure(WorkerConfig config) { // eagerly initialize the executor, because OffsetStorageWriter will use it later @@ -100,7 +101,9 @@ public void configure(WorkerConfig config) { keyConverter, valueConverter); - offsetWriter.offset(debeziumOffset.sourcePartition, debeziumOffset.sourceOffset); + offsetWriter.offset( + (Map) debeziumOffset.sourcePartition, + (Map) debeziumOffset.sourceOffset); // flush immediately if (!offsetWriter.beginFlush()) { diff --git a/flink-connector-debezium/src/main/java/io/debezium/relational/HistorizedRelationalDatabaseConnectorConfig.java b/flink-connector-debezium/src/main/java/io/debezium/relational/HistorizedRelationalDatabaseConnectorConfig.java new file mode 100644 index 0000000000..98d25b74f1 --- /dev/null +++ b/flink-connector-debezium/src/main/java/io/debezium/relational/HistorizedRelationalDatabaseConnectorConfig.java @@ -0,0 +1,175 @@ +/* + * Copyright Debezium Authors. + * + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 + */ + +package io.debezium.relational; + +import io.debezium.config.ConfigDefinition; +import io.debezium.config.Configuration; +import io.debezium.config.Field; +import io.debezium.relational.Selectors.TableIdToStringMapper; +import io.debezium.relational.Tables.TableFilter; +import io.debezium.relational.history.DatabaseHistory; +import io.debezium.relational.history.DatabaseHistoryListener; +import io.debezium.relational.history.DatabaseHistoryMetrics; +import io.debezium.relational.history.HistoryRecordComparator; +import io.debezium.relational.history.KafkaDatabaseHistory; +import org.apache.kafka.common.config.ConfigDef.Importance; +import org.apache.kafka.common.config.ConfigDef.Type; +import org.apache.kafka.common.config.ConfigDef.Width; +import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.source.SourceConnector; + +/** + * Copied from Debezium project. Configuration options shared across the relational CDC connectors + * which use a persistent database schema history. + * + *

Added JMX_METRICS_ENABLED option. + */ +public abstract class HistorizedRelationalDatabaseConnectorConfig + extends RelationalDatabaseConnectorConfig { + + protected static final int DEFAULT_SNAPSHOT_FETCH_SIZE = 2_000; + + private boolean useCatalogBeforeSchema; + private final String logicalName; + private final Class connectorClass; + private final boolean multiPartitionMode; + + /** + * The database history class is hidden in the {@link #configDef()} since that is designed to + * work with a user interface, and in these situations using Kafka is the only way to go. + */ + public static final Field DATABASE_HISTORY = + Field.create("database.history") + .withDisplayName("Database history class") + .withType(Type.CLASS) + .withWidth(Width.LONG) + .withImportance(Importance.LOW) + .withInvisibleRecommender() + .withDescription( + "The name of the DatabaseHistory class that should be used to store and recover database schema changes. " + + "The configuration properties for the history are prefixed with the '" + + DatabaseHistory.CONFIGURATION_FIELD_PREFIX_STRING + + "' string.") + .withDefault(KafkaDatabaseHistory.class.getName()); + + public static final Field JMX_METRICS_ENABLED = + Field.create(DatabaseHistory.CONFIGURATION_FIELD_PREFIX_STRING + "metrics.enabled") + .withDisplayName("Skip DDL statements that cannot be parsed") + .withType(Type.BOOLEAN) + .withImportance(Importance.LOW) + .withDescription("Whether to enable JMX history metrics") + .withDefault(false); + + protected static final ConfigDefinition CONFIG_DEFINITION = + RelationalDatabaseConnectorConfig.CONFIG_DEFINITION + .edit() + .history( + DATABASE_HISTORY, + DatabaseHistory.SKIP_UNPARSEABLE_DDL_STATEMENTS, + DatabaseHistory.STORE_ONLY_MONITORED_TABLES_DDL, + DatabaseHistory.STORE_ONLY_CAPTURED_TABLES_DDL, + KafkaDatabaseHistory.BOOTSTRAP_SERVERS, + KafkaDatabaseHistory.TOPIC, + KafkaDatabaseHistory.RECOVERY_POLL_ATTEMPTS, + KafkaDatabaseHistory.RECOVERY_POLL_INTERVAL_MS, + KafkaDatabaseHistory.KAFKA_QUERY_TIMEOUT_MS) + .create(); + + protected HistorizedRelationalDatabaseConnectorConfig( + Class connectorClass, + Configuration config, + String logicalName, + TableFilter systemTablesFilter, + boolean useCatalogBeforeSchema, + int defaultSnapshotFetchSize, + ColumnFilterMode columnFilterMode, + boolean multiPartitionMode) { + super( + config, + logicalName, + systemTablesFilter, + TableId::toString, + defaultSnapshotFetchSize, + columnFilterMode); + this.useCatalogBeforeSchema = useCatalogBeforeSchema; + this.logicalName = logicalName; + this.connectorClass = connectorClass; + this.multiPartitionMode = multiPartitionMode; + } + + protected HistorizedRelationalDatabaseConnectorConfig( + Class connectorClass, + Configuration config, + String logicalName, + TableFilter systemTablesFilter, + TableIdToStringMapper tableIdMapper, + boolean useCatalogBeforeSchema, + ColumnFilterMode columnFilterMode, + boolean multiPartitionMode) { + super( + config, + logicalName, + systemTablesFilter, + tableIdMapper, + DEFAULT_SNAPSHOT_FETCH_SIZE, + columnFilterMode); + this.useCatalogBeforeSchema = useCatalogBeforeSchema; + this.logicalName = logicalName; + this.connectorClass = connectorClass; + this.multiPartitionMode = multiPartitionMode; + } + + /** Returns a configured (but not yet started) instance of the database history. */ + public DatabaseHistory getDatabaseHistory() { + Configuration config = getConfig(); + + DatabaseHistory databaseHistory = + config.getInstance( + HistorizedRelationalDatabaseConnectorConfig.DATABASE_HISTORY, + DatabaseHistory.class); + if (databaseHistory == null) { + throw new ConnectException( + "Unable to instantiate the database history class " + + config.getString( + HistorizedRelationalDatabaseConnectorConfig.DATABASE_HISTORY)); + } + + // Do not remove the prefix from the subset of config properties ... + Configuration dbHistoryConfig = + config.subset(DatabaseHistory.CONFIGURATION_FIELD_PREFIX_STRING, false) + .edit() + .withDefault(DatabaseHistory.NAME, getLogicalName() + "-dbhistory") + .withDefault( + KafkaDatabaseHistory.INTERNAL_CONNECTOR_CLASS, + connectorClass.getName()) + .withDefault(KafkaDatabaseHistory.INTERNAL_CONNECTOR_ID, logicalName) + .build(); + + DatabaseHistoryListener listener = + config.getBoolean(JMX_METRICS_ENABLED) + ? new DatabaseHistoryMetrics(this, multiPartitionMode) + : DatabaseHistoryListener.NOOP; + + HistoryRecordComparator historyComparator = getHistoryRecordComparator(); + databaseHistory.configure( + dbHistoryConfig, historyComparator, listener, useCatalogBeforeSchema); // validates + + return databaseHistory; + } + + public boolean useCatalogBeforeSchema() { + return useCatalogBeforeSchema; + } + + /** + * Returns a comparator to be used when recovering records from the schema history, making sure + * no history entries newer than the offset we resume from are recovered (which could happen + * when restarting a connector after history records have been persisted but no new offset has + * been committed yet). + */ + protected abstract HistoryRecordComparator getHistoryRecordComparator(); +} diff --git a/flink-connector-mongodb-cdc/pom.xml b/flink-connector-mongodb-cdc/pom.xml index b1b1732492..9285b05481 100644 --- a/flink-connector-mongodb-cdc/pom.xml +++ b/flink-connector-mongodb-cdc/pom.xml @@ -159,6 +159,12 @@ under the License. test + + org.apache.commons + commons-lang3 + 3.7 + + \ No newline at end of file diff --git a/flink-connector-mongodb-cdc/src/main/java/com/ververica/cdc/connectors/mongodb/source/reader/fetch/MongoDBFetchTaskContext.java b/flink-connector-mongodb-cdc/src/main/java/com/ververica/cdc/connectors/mongodb/source/reader/fetch/MongoDBFetchTaskContext.java index c365808a06..e4b0977458 100644 --- a/flink-connector-mongodb-cdc/src/main/java/com/ververica/cdc/connectors/mongodb/source/reader/fetch/MongoDBFetchTaskContext.java +++ b/flink-connector-mongodb-cdc/src/main/java/com/ververica/cdc/connectors/mongodb/source/reader/fetch/MongoDBFetchTaskContext.java @@ -65,8 +65,11 @@ public MongoDBFetchTaskContext( } public void configure(SourceSplitBase sourceSplitBase) { - final int queueSize = - sourceSplitBase.isSnapshotSplit() ? Integer.MAX_VALUE : sourceConfig.getBatchSize(); + // we need to use small batch size instead of INT.MAX as earlier because + // now under the hood of debezium the ArrayDequeue was used as queue implementation + // TODO: replace getBatchSize with getSnapshotBatchSize + // when SNAPSHOT_BATCH_SIZE option will be added + final int queueSize = sourceConfig.getBatchSize(); this.changeEventQueue = new ChangeEventQueue.Builder() @@ -191,4 +194,7 @@ record -> { }) .collect(Collectors.toList()); } + + @Override + public void close() throws Exception {} } diff --git a/flink-connector-mongodb-cdc/src/main/java/com/ververica/cdc/connectors/mongodb/source/reader/fetch/MongoDBScanFetchTask.java b/flink-connector-mongodb-cdc/src/main/java/com/ververica/cdc/connectors/mongodb/source/reader/fetch/MongoDBScanFetchTask.java index 6a30ce96b0..26df23b387 100644 --- a/flink-connector-mongodb-cdc/src/main/java/com/ververica/cdc/connectors/mongodb/source/reader/fetch/MongoDBScanFetchTask.java +++ b/flink-connector-mongodb-cdc/src/main/java/com/ververica/cdc/connectors/mongodb/source/reader/fetch/MongoDBScanFetchTask.java @@ -209,6 +209,9 @@ public SnapshotSplit getSplit() { return snapshotSplit; } + @Override + public void close() {} + private StreamSplit createBackfillStreamSplit( ChangeStreamOffset lowWatermark, ChangeStreamOffset highWatermark) { return new StreamSplit( diff --git a/flink-connector-mongodb-cdc/src/main/java/com/ververica/cdc/connectors/mongodb/source/reader/fetch/MongoDBStreamFetchTask.java b/flink-connector-mongodb-cdc/src/main/java/com/ververica/cdc/connectors/mongodb/source/reader/fetch/MongoDBStreamFetchTask.java index 401a943aba..5045a8f620 100644 --- a/flink-connector-mongodb-cdc/src/main/java/com/ververica/cdc/connectors/mongodb/source/reader/fetch/MongoDBStreamFetchTask.java +++ b/flink-connector-mongodb-cdc/src/main/java/com/ververica/cdc/connectors/mongodb/source/reader/fetch/MongoDBStreamFetchTask.java @@ -221,6 +221,9 @@ public StreamSplit getSplit() { return streamSplit; } + @Override + public void close() {} + private MongoChangeStreamCursor openChangeStreamCursor( ChangeStreamDescriptor changeStreamDescriptor) { ChangeStreamOffset offset = diff --git a/flink-connector-mysql-cdc/pom.xml b/flink-connector-mysql-cdc/pom.xml index 8d6a92d397..8cdfa005a3 100644 --- a/flink-connector-mysql-cdc/pom.xml +++ b/flink-connector-mysql-cdc/pom.xml @@ -182,6 +182,12 @@ under the License. test + + org.apache.commons + commons-lang3 + 3.7 + + diff --git a/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/DebeziumUtils.java b/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/DebeziumUtils.java index 6db1925447..beef567be6 100644 --- a/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/DebeziumUtils.java +++ b/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/DebeziumUtils.java @@ -29,6 +29,7 @@ import io.debezium.connector.mysql.MySqlSystemVariables; import io.debezium.connector.mysql.MySqlTopicSelector; import io.debezium.connector.mysql.MySqlValueConverters; +import io.debezium.jdbc.JdbcConfiguration; import io.debezium.jdbc.JdbcConnection; import io.debezium.jdbc.JdbcValueConverters; import io.debezium.jdbc.TemporalPrecisionMode; @@ -51,6 +52,7 @@ /** Utilities related to Debezium. */ public class DebeziumUtils { + private static final String QUOTED_CHARACTER = "`"; private static final Logger LOG = LoggerFactory.getLogger(DebeziumUtils.class); @@ -58,8 +60,10 @@ public class DebeziumUtils { public static JdbcConnection openJdbcConnection(MySqlSourceConfig sourceConfig) { JdbcConnection jdbc = new JdbcConnection( - sourceConfig.getDbzConfiguration(), - new JdbcConnectionFactory(sourceConfig)); + JdbcConfiguration.adapt(sourceConfig.getDbzConfiguration()), + new JdbcConnectionFactory(sourceConfig), + QUOTED_CHARACTER, + QUOTED_CHARACTER); try { jdbc.connect(); } catch (Exception e) { diff --git a/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/EmbeddedFlinkDatabaseHistory.java b/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/EmbeddedFlinkDatabaseHistory.java index aec7d094af..73e8afc2e7 100644 --- a/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/EmbeddedFlinkDatabaseHistory.java +++ b/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/EmbeddedFlinkDatabaseHistory.java @@ -18,6 +18,7 @@ import com.ververica.cdc.connectors.mysql.source.split.MySqlSplitState; import io.debezium.config.Configuration; +import io.debezium.pipeline.spi.Offsets; import io.debezium.relational.TableId; import io.debezium.relational.Tables; import io.debezium.relational.ddl.DdlParser; @@ -97,9 +98,18 @@ public void record( listener.onChangeApplied(record); } + @Override + public void recover(Offsets offsets, Tables schema, DdlParser ddlParser) { + listener.recoveryStarted(); + for (TableChange tableChange : tableSchemas.values()) { + schema.overwriteTable(tableChange.getTable()); + } + listener.recoveryStopped(); + } + @Override public void recover( - Map source, Map position, Tables schema, DdlParser ddlParser) { + Map, Map> offsets, Tables schema, DdlParser ddlParser) { listener.recoveryStarted(); for (TableChange tableChange : tableSchemas.values()) { schema.overwriteTable(tableChange.getTable()); diff --git a/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/dispatcher/EventDispatcherImpl.java b/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/dispatcher/EventDispatcherImpl.java index 36354ceb99..b6c3c9755a 100644 --- a/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/dispatcher/EventDispatcherImpl.java +++ b/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/dispatcher/EventDispatcherImpl.java @@ -18,9 +18,11 @@ import io.debezium.config.CommonConnectorConfig; import io.debezium.connector.base.ChangeEventQueue; +import io.debezium.connector.mysql.MySqlPartition; import io.debezium.document.DocumentWriter; import io.debezium.pipeline.DataChangeEvent; import io.debezium.pipeline.EventDispatcher; +import io.debezium.pipeline.source.snapshot.incremental.IncrementalSnapshotChangeEventSource; import io.debezium.pipeline.source.spi.EventMetadataProvider; import io.debezium.pipeline.spi.ChangeEventCreator; import io.debezium.pipeline.spi.SchemaChangeEventEmitter; @@ -57,7 +59,8 @@ * this is useful for downstream to deserialize the {@link HistoryRecord} back. * */ -public class EventDispatcherImpl extends EventDispatcher { +public class EventDispatcherImpl + extends EventDispatcher { private static final Logger LOG = LoggerFactory.getLogger(EventDispatcherImpl.class); @@ -127,7 +130,9 @@ public ChangeEventQueue getQueue() { @Override public void dispatchSchemaChangeEvent( - T dataCollectionId, SchemaChangeEventEmitter schemaChangeEventEmitter) + MySqlPartition partition, + T dataCollectionId, + SchemaChangeEventEmitter schemaChangeEventEmitter) throws InterruptedException { if (dataCollectionId != null && !filter.isIncluded(dataCollectionId)) { if (historizedSchema == null || historizedSchema.storeOnlyCapturedTables()) { @@ -136,6 +141,11 @@ public void dispatchSchemaChangeEvent( } } schemaChangeEventEmitter.emitSchemaChangeEvent(new SchemaChangeEventReceiver()); + IncrementalSnapshotChangeEventSource incrementalEventSource = + getIncrementalSnapshotChangeEventSource(); + if (incrementalEventSource != null) { + incrementalEventSource.processSchemaChange(partition, dataCollectionId); + } } @Override diff --git a/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/reader/BinlogSplitReader.java b/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/reader/BinlogSplitReader.java index 288b333a99..a76c4d3908 100644 --- a/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/reader/BinlogSplitReader.java +++ b/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/reader/BinlogSplitReader.java @@ -125,6 +125,7 @@ public void submitSplit(MySqlSplit mySqlSplit) { try { binlogSplitReadTask.execute( new BinlogSplitChangeEventSourceContextImpl(), + statefulTaskContext.getMySqlPartition(), statefulTaskContext.getOffsetContext()); } catch (Exception e) { currentTaskRunning = false; @@ -201,6 +202,7 @@ public void close() { READER_CLOSE_TIMEOUT); } } + statefulTaskContext.getDatabaseSchema().close(); } catch (Exception e) { LOG.error("Close binlog reader error", e); } diff --git a/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/reader/SnapshotSplitReader.java b/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/reader/SnapshotSplitReader.java index e270a6fdf2..3e8594e715 100644 --- a/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/reader/SnapshotSplitReader.java +++ b/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/reader/SnapshotSplitReader.java @@ -106,6 +106,7 @@ public SnapshotSplitReader(StatefulTaskContext statefulTaskContext, int subtaskI this.reachEnd = new AtomicBoolean(false); } + @Override public void submitSplit(MySqlSplit mySqlSplit) { this.currentSnapshotSplit = mySqlSplit.asSnapshotSplit(); statefulTaskContext.configure(currentSnapshotSplit); @@ -131,9 +132,11 @@ public void submitSplit(MySqlSplit mySqlSplit) { // execute snapshot read task final SnapshotSplitChangeEventSourceContextImpl sourceContext = new SnapshotSplitChangeEventSourceContextImpl(); - SnapshotResult snapshotResult = + SnapshotResult snapshotResult = splitSnapshotReadTask.execute( - sourceContext, statefulTaskContext.getOffsetContext()); + sourceContext, + statefulTaskContext.getMySqlPartition(), + statefulTaskContext.getOffsetContext()); final MySqlBinlogSplit backfillBinlogSplit = createBackfillBinlogSplit(sourceContext); @@ -162,6 +165,7 @@ public void submitSplit(MySqlSplit mySqlSplit) { backfillBinlogReadTask.execute( new SnapshotBinlogSplitChangeEventSourceContextImpl(), + statefulTaskContext.getMySqlPartition(), mySqlOffsetContext); } else { setReadException( @@ -219,7 +223,7 @@ private void dispatchBinlogEndEvent(MySqlBinlogSplit backFillBinlogSplit) throws InterruptedException { final SignalEventDispatcher signalEventDispatcher = new SignalEventDispatcher( - statefulTaskContext.getOffsetContext().getPartition(), + statefulTaskContext.getOffsetContext().getOffset(), statefulTaskContext.getTopicSelector().getPrimaryTopic(), statefulTaskContext.getDispatcher().getQueue()); signalEventDispatcher.dispatchWatermarkEvent( @@ -355,6 +359,9 @@ public void close() { if (statefulTaskContext.getBinaryLogClient() != null) { statefulTaskContext.getBinaryLogClient().disconnect(); } + if (statefulTaskContext.getDatabaseSchema() != null) { + statefulTaskContext.getDatabaseSchema().close(); + } if (executorService != null) { executorService.shutdown(); if (!executorService.awaitTermination(READER_CLOSE_TIMEOUT, TimeUnit.SECONDS)) { @@ -377,7 +384,7 @@ public ExecutorService getExecutorService() { * {@link ChangeEventSource.ChangeEventSourceContext} implementation that keeps low/high * watermark for each {@link MySqlSnapshotSplit}. */ - public class SnapshotSplitChangeEventSourceContextImpl + public static class SnapshotSplitChangeEventSourceContextImpl implements ChangeEventSource.ChangeEventSourceContext { private BinlogOffset lowWatermark; diff --git a/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/task/MySqlBinlogSplitReadTask.java b/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/task/MySqlBinlogSplitReadTask.java index e7a81783f1..07fb8d4965 100644 --- a/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/task/MySqlBinlogSplitReadTask.java +++ b/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/task/MySqlBinlogSplitReadTask.java @@ -26,6 +26,7 @@ import io.debezium.connector.mysql.MySqlConnection; import io.debezium.connector.mysql.MySqlConnectorConfig; import io.debezium.connector.mysql.MySqlOffsetContext; +import io.debezium.connector.mysql.MySqlPartition; import io.debezium.connector.mysql.MySqlStreamingChangeEventSource; import io.debezium.connector.mysql.MySqlStreamingChangeEventSourceMetrics; import io.debezium.connector.mysql.MySqlTaskContext; @@ -74,18 +75,22 @@ public MySqlBinlogSplitReadTask( } @Override - public void execute(ChangeEventSourceContext context, MySqlOffsetContext offsetContext) + public void execute( + ChangeEventSourceContext context, + MySqlPartition partition, + MySqlOffsetContext offsetContext) throws InterruptedException { this.context = context; - super.execute(context, offsetContext); + super.execute(context, partition, offsetContext); } @Override - protected void handleEvent(MySqlOffsetContext offsetContext, Event event) { + protected void handleEvent( + MySqlPartition partition, MySqlOffsetContext offsetContext, Event event) { if (!eventFilter.test(event)) { return; } - super.handleEvent(offsetContext, event); + super.handleEvent(partition, offsetContext, event); // check do we need to stop for read binlog for snapshot split. if (isBoundedRead()) { final BinlogOffset currentBinlogOffset = getBinlogPosition(offsetContext.getOffset()); diff --git a/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/task/MySqlSnapshotSplitReadTask.java b/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/task/MySqlSnapshotSplitReadTask.java index c73ddb91a8..b2d0a5b222 100644 --- a/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/task/MySqlSnapshotSplitReadTask.java +++ b/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/task/MySqlSnapshotSplitReadTask.java @@ -27,6 +27,7 @@ import io.debezium.connector.mysql.MySqlConnectorConfig; import io.debezium.connector.mysql.MySqlDatabaseSchema; import io.debezium.connector.mysql.MySqlOffsetContext; +import io.debezium.connector.mysql.MySqlPartition; import io.debezium.connector.mysql.MySqlValueConverters; import io.debezium.pipeline.EventDispatcher; import io.debezium.pipeline.metrics.SnapshotChangeEventSourceMetrics; @@ -60,7 +61,7 @@ /** Task to read snapshot split of table. */ public class MySqlSnapshotSplitReadTask - extends AbstractSnapshotChangeEventSource { + extends AbstractSnapshotChangeEventSource { private static final Logger LOG = LoggerFactory.getLogger(MySqlSnapshotSplitReadTask.class); @@ -74,17 +75,17 @@ public class MySqlSnapshotSplitReadTask private final Clock clock; private final MySqlSnapshotSplit snapshotSplit; private final TopicSelector topicSelector; - private final EventDispatcher.SnapshotReceiver snapshotReceiver; - private final SnapshotChangeEventSourceMetrics snapshotChangeEventSourceMetrics; + private final EventDispatcher.SnapshotReceiver snapshotReceiver; + private final SnapshotChangeEventSourceMetrics snapshotChangeEventSourceMetrics; public MySqlSnapshotSplitReadTask( MySqlConnectorConfig connectorConfig, - SnapshotChangeEventSourceMetrics snapshotChangeEventSourceMetrics, + SnapshotChangeEventSourceMetrics snapshotChangeEventSourceMetrics, MySqlDatabaseSchema databaseSchema, MySqlConnection jdbcConnection, EventDispatcherImpl dispatcher, TopicSelector topicSelector, - EventDispatcher.SnapshotReceiver snapshotReceiver, + EventDispatcher.SnapshotReceiver snapshotReceiver, Clock clock, MySqlSnapshotSplit snapshotSplit) { super(connectorConfig, snapshotChangeEventSourceMetrics); @@ -101,12 +102,14 @@ public MySqlSnapshotSplitReadTask( @Override public SnapshotResult execute( - ChangeEventSourceContext context, MySqlOffsetContext previousOffset) + ChangeEventSourceContext context, + MySqlPartition partition, + MySqlOffsetContext previousOffset) throws InterruptedException { - SnapshottingTask snapshottingTask = getSnapshottingTask(previousOffset); - final SnapshotContext ctx; + SnapshottingTask snapshottingTask = getSnapshottingTask(partition, previousOffset); + final SnapshotContext ctx; try { - ctx = prepare(context); + ctx = prepare(partition); } catch (Exception e) { LOG.error("Failed to initialize snapshot context.", e); throw new RuntimeException(e); @@ -125,18 +128,14 @@ public SnapshotResult execute( protected SnapshotResult doExecute( ChangeEventSourceContext context, MySqlOffsetContext previousOffset, - SnapshotContext snapshotContext, + SnapshotContext snapshotContext, SnapshottingTask snapshottingTask) throws Exception { - final RelationalSnapshotChangeEventSource.RelationalSnapshotContext - ctx = - (RelationalSnapshotChangeEventSource.RelationalSnapshotContext< - MySqlOffsetContext>) - snapshotContext; + final MySqlSnapshotContext ctx = (MySqlSnapshotContext) snapshotContext; ctx.offset = previousOffset; final SignalEventDispatcher signalEventDispatcher = new SignalEventDispatcher( - previousOffset.getPartition(), + previousOffset.getOffset(), topicSelector.topicNameFor(snapshotSplit.getTableId()), dispatcher.getQueue()); @@ -167,27 +166,26 @@ protected SnapshotResult doExecute( } @Override - protected SnapshottingTask getSnapshottingTask(MySqlOffsetContext offsetContext) { + protected SnapshottingTask getSnapshottingTask( + MySqlPartition partition, MySqlOffsetContext previousOffset) { return new SnapshottingTask(false, true); } @Override - protected SnapshotContext prepare(ChangeEventSourceContext changeEventSourceContext) - throws Exception { - return new MySqlSnapshotContext(); + protected MySqlSnapshotContext prepare(MySqlPartition partition) throws Exception { + return new MySqlSnapshotContext(partition); } private static class MySqlSnapshotContext - extends RelationalSnapshotChangeEventSource.RelationalSnapshotContext { + extends RelationalSnapshotChangeEventSource.RelationalSnapshotContext< + MySqlPartition, MySqlOffsetContext> { - public MySqlSnapshotContext() throws SQLException { - super(""); + public MySqlSnapshotContext(MySqlPartition partition) throws SQLException { + super(partition, ""); } } - private void createDataEvents( - RelationalSnapshotChangeEventSource.RelationalSnapshotContext snapshotContext, - TableId tableId) + private void createDataEvents(MySqlSnapshotContext snapshotContext, TableId tableId) throws Exception { LOG.debug("Snapshotting table {}", tableId); createDataEventsForTable( @@ -197,8 +195,8 @@ private void createDataEvents( /** Dispatches the data change events for the records of a single table. */ private void createDataEventsForTable( - RelationalSnapshotChangeEventSource.RelationalSnapshotContext snapshotContext, - EventDispatcher.SnapshotReceiver snapshotReceiver, + MySqlSnapshotContext snapshotContext, + EventDispatcher.SnapshotReceiver snapshotReceiver, Table table) throws InterruptedException { @@ -248,10 +246,12 @@ private void createDataEventsForTable( rows, snapshotSplit.splitId(), Strings.duration(stop - exportStart)); - snapshotChangeEventSourceMetrics.rowsScanned(table.id(), rows); + snapshotChangeEventSourceMetrics.rowsScanned( + snapshotContext.partition, table.id(), rows); logTimer = getTableScanLogTimer(); } dispatcher.dispatchSnapshotEvent( + (MySqlPartition) snapshotContext.partition, table.id(), getChangeRecordEmitter(snapshotContext, table.id(), row), snapshotReceiver); @@ -266,10 +266,11 @@ private void createDataEventsForTable( } } - protected ChangeRecordEmitter getChangeRecordEmitter( - SnapshotContext snapshotContext, TableId tableId, Object[] row) { + protected ChangeRecordEmitter getChangeRecordEmitter( + MySqlSnapshotContext snapshotContext, TableId tableId, Object[] row) { snapshotContext.offset.event(tableId, clock.currentTime()); - return new SnapshotChangeRecordEmitter(snapshotContext.offset, row, clock); + return new SnapshotChangeRecordEmitter<>( + snapshotContext.partition, snapshotContext.offset, row, clock); } private Threads.Timer getTableScanLogTimer() { diff --git a/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/task/context/MySqlErrorHandler.java b/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/task/context/MySqlErrorHandler.java index 8579395d1d..efc88f2066 100644 --- a/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/task/context/MySqlErrorHandler.java +++ b/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/task/context/MySqlErrorHandler.java @@ -22,6 +22,7 @@ import io.debezium.DebeziumException; import io.debezium.connector.base.ChangeEventQueue; import io.debezium.connector.mysql.MySqlConnector; +import io.debezium.connector.mysql.MySqlConnectorConfig; import io.debezium.connector.mysql.MySqlTaskContext; import io.debezium.pipeline.ErrorHandler; import io.debezium.relational.TableId; @@ -47,20 +48,15 @@ public class MySqlErrorHandler extends ErrorHandler { private final MySqlSourceConfig sourceConfig; public MySqlErrorHandler( - String logicalName, + MySqlConnectorConfig mySqlConnectorConfig, ChangeEventQueue queue, MySqlTaskContext context, MySqlSourceConfig sourceConfig) { - super(MySqlConnector.class, logicalName, queue); + super(MySqlConnector.class, mySqlConnectorConfig, queue); this.context = context; this.sourceConfig = sourceConfig; } - @Override - protected boolean isRetriable(Throwable throwable) { - return false; - } - @Override public void setProducerThrowable(Throwable producerThrowable) { if (isTableNotFoundException(producerThrowable)) { diff --git a/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/task/context/StatefulTaskContext.java b/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/task/context/StatefulTaskContext.java index 41dd3501cb..4a3dfac0a6 100644 --- a/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/task/context/StatefulTaskContext.java +++ b/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/task/context/StatefulTaskContext.java @@ -32,6 +32,7 @@ import io.debezium.connector.mysql.MySqlConnectorConfig; import io.debezium.connector.mysql.MySqlDatabaseSchema; import io.debezium.connector.mysql.MySqlOffsetContext; +import io.debezium.connector.mysql.MySqlPartition; import io.debezium.connector.mysql.MySqlStreamingChangeEventSourceMetrics; import io.debezium.connector.mysql.MySqlTopicSelector; import io.debezium.data.Envelope; @@ -42,6 +43,7 @@ import io.debezium.pipeline.metrics.StreamingChangeEventSourceMetrics; import io.debezium.pipeline.source.spi.EventMetadataProvider; import io.debezium.pipeline.spi.OffsetContext; +import io.debezium.pipeline.spi.Offsets; import io.debezium.relational.TableId; import io.debezium.schema.DataCollectionId; import io.debezium.schema.TopicSelector; @@ -55,6 +57,7 @@ import java.time.Instant; import java.util.List; import java.util.Map; +import java.util.Optional; import static com.ververica.cdc.connectors.mysql.source.offset.BinlogOffset.BINLOG_FILENAME_OFFSET_KEY; import static com.ververica.cdc.connectors.mysql.source.offset.BinlogOffsetUtils.initializeEffectiveOffset; @@ -81,11 +84,12 @@ public class StatefulTaskContext { private MySqlDatabaseSchema databaseSchema; private MySqlTaskContextImpl taskContext; private MySqlOffsetContext offsetContext; + private MySqlPartition mySqlPartition; private TopicSelector topicSelector; - private SnapshotChangeEventSourceMetrics snapshotChangeEventSourceMetrics; - private StreamingChangeEventSourceMetrics streamingChangeEventSourceMetrics; + private SnapshotChangeEventSourceMetrics snapshotChangeEventSourceMetrics; + private StreamingChangeEventSourceMetrics streamingChangeEventSourceMetrics; private EventDispatcherImpl dispatcher; - private EventDispatcher.SnapshotReceiver snapshotReceiver; + private EventDispatcher.SnapshotReceiver snapshotReceiver; private SignalEventDispatcher signalEventDispatcher; private ChangeEventQueue queue; private ErrorHandler errorHandler; @@ -111,8 +115,13 @@ public void configure(MySqlSplit mySqlSplit) { .getDbzConfiguration() .getString(EmbeddedFlinkDatabaseHistory.DATABASE_HISTORY_INSTANCE_NAME), mySqlSplit.getTableSchemas().values()); + + Optional.ofNullable(databaseSchema).ifPresent(MySqlDatabaseSchema::close); this.databaseSchema = DebeziumUtils.createMySqlDatabaseSchema(connectorConfig, tableIdCaseInsensitive); + + this.mySqlPartition = new MySqlPartition(connectorConfig.getLogicalName()); + this.offsetContext = loadStartingOffsetState(new MySqlOffsetContext.Loader(connectorConfig), mySqlSplit); validateAndLoadDatabaseHistory(offsetContext, databaseSchema); @@ -152,7 +161,7 @@ public void configure(MySqlSplit mySqlSplit) { this.signalEventDispatcher = new SignalEventDispatcher( - offsetContext.getPartition(), topicSelector.getPrimaryTopic(), queue); + offsetContext.getOffset(), topicSelector.getPrimaryTopic(), queue); final MySqlChangeEventSourceMetricsFactory changeEventSourceMetricsFactory = new MySqlChangeEventSourceMetricsFactory( @@ -165,14 +174,13 @@ public void configure(MySqlSplit mySqlSplit) { changeEventSourceMetricsFactory.getStreamingMetrics( taskContext, queue, metadataProvider); this.errorHandler = - new MySqlErrorHandler( - connectorConfig.getLogicalName(), queue, taskContext, sourceConfig); + new MySqlErrorHandler(connectorConfig, queue, taskContext, sourceConfig); } private void validateAndLoadDatabaseHistory( MySqlOffsetContext offset, MySqlDatabaseSchema schema) { schema.initializeStorage(); - schema.recover(offset); + schema.recover(Offsets.of(mySqlPartition, offset)); } /** Loads the connector's persistent offset (if present) via the given loader. */ @@ -359,7 +367,7 @@ public EventDispatcherImpl getDispatcher() { return dispatcher; } - public EventDispatcher.SnapshotReceiver getSnapshotReceiver() { + public EventDispatcher.SnapshotReceiver getSnapshotReceiver() { return snapshotReceiver; } @@ -379,17 +387,20 @@ public MySqlOffsetContext getOffsetContext() { return offsetContext; } + public MySqlPartition getMySqlPartition() { + return mySqlPartition; + } + public TopicSelector getTopicSelector() { return topicSelector; } - public SnapshotChangeEventSourceMetrics getSnapshotChangeEventSourceMetrics() { - snapshotChangeEventSourceMetrics.reset(); + public SnapshotChangeEventSourceMetrics getSnapshotChangeEventSourceMetrics() { return snapshotChangeEventSourceMetrics; } - public StreamingChangeEventSourceMetrics getStreamingChangeEventSourceMetrics() { - streamingChangeEventSourceMetrics.reset(); + public StreamingChangeEventSourceMetrics + getStreamingChangeEventSourceMetrics() { return streamingChangeEventSourceMetrics; } diff --git a/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/schema/MySqlSchema.java b/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/schema/MySqlSchema.java index 10a734a6e1..ce355262d1 100644 --- a/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/schema/MySqlSchema.java +++ b/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/schema/MySqlSchema.java @@ -22,6 +22,7 @@ import io.debezium.connector.mysql.MySqlConnectorConfig; import io.debezium.connector.mysql.MySqlDatabaseSchema; import io.debezium.connector.mysql.MySqlOffsetContext; +import io.debezium.connector.mysql.MySqlPartition; import io.debezium.jdbc.JdbcConnection; import io.debezium.relational.TableId; import io.debezium.relational.history.TableChanges.TableChange; @@ -39,7 +40,7 @@ import static com.ververica.cdc.connectors.mysql.source.utils.StatementUtils.quote; /** A component used to get schema by table path. */ -public class MySqlSchema { +public class MySqlSchema implements AutoCloseable { private static final String SHOW_CREATE_TABLE = "SHOW CREATE TABLE "; private static final String DESC_TABLE = "DESC "; @@ -57,11 +58,12 @@ public MySqlSchema(MySqlSourceConfig sourceConfig, boolean isTableIdCaseSensitiv * Gets table schema for the given table path. It will request to MySQL server by running `SHOW * CREATE TABLE` if cache missed. */ - public TableChange getTableSchema(JdbcConnection jdbc, TableId tableId) { + public TableChange getTableSchema( + MySqlPartition partition, JdbcConnection jdbc, TableId tableId) { // read schema from cache first TableChange schema = schemasByTableId.get(tableId); if (schema == null) { - schema = buildTableSchema(jdbc, tableId); + schema = buildTableSchema(partition, jdbc, tableId); schemasByTableId.put(tableId, schema); } return schema; @@ -71,14 +73,15 @@ public TableChange getTableSchema(JdbcConnection jdbc, TableId tableId) { // Helpers // ------------------------------------------------------------------------------------------ - private TableChange buildTableSchema(JdbcConnection jdbc, TableId tableId) { + private TableChange buildTableSchema( + MySqlPartition partition, JdbcConnection jdbc, TableId tableId) { final Map tableChangeMap = new HashMap<>(); String showCreateTable = SHOW_CREATE_TABLE + quote(tableId); - buildSchemaByShowCreateTable(jdbc, tableId, tableChangeMap); + buildSchemaByShowCreateTable(partition, jdbc, tableId, tableChangeMap); if (!tableChangeMap.containsKey(tableId)) { // fallback to desc table String descTable = DESC_TABLE + quote(tableId); - buildSchemaByDescTable(jdbc, descTable, tableId, tableChangeMap); + buildSchemaByDescTable(partition, jdbc, descTable, tableId, tableChangeMap); if (!tableChangeMap.containsKey(tableId)) { throw new FlinkRuntimeException( String.format( @@ -90,7 +93,10 @@ private TableChange buildTableSchema(JdbcConnection jdbc, TableId tableId) { } private void buildSchemaByShowCreateTable( - JdbcConnection jdbc, TableId tableId, Map tableChangeMap) { + MySqlPartition partition, + JdbcConnection jdbc, + TableId tableId, + Map tableChangeMap) { final String sql = SHOW_CREATE_TABLE + quote(tableId); try { jdbc.query( @@ -98,7 +104,7 @@ private void buildSchemaByShowCreateTable( rs -> { if (rs.next()) { final String ddl = rs.getString(2); - parseSchemaByDdl(ddl, tableId, tableChangeMap); + parseSchemaByDdl(partition, ddl, tableId, tableChangeMap); } }); } catch (SQLException e) { @@ -109,11 +115,14 @@ private void buildSchemaByShowCreateTable( } private void parseSchemaByDdl( - String ddl, TableId tableId, Map tableChangeMap) { + MySqlPartition partition, + String ddl, + TableId tableId, + Map tableChangeMap) { final MySqlOffsetContext offsetContext = MySqlOffsetContext.initial(connectorConfig); List schemaChangeEvents = databaseSchema.parseSnapshotDdl( - ddl, tableId.catalog(), offsetContext, Instant.now()); + partition, ddl, tableId.catalog(), offsetContext, Instant.now()); for (SchemaChangeEvent schemaChangeEvent : schemaChangeEvents) { for (TableChange tableChange : schemaChangeEvent.getTableChanges()) { tableChangeMap.put(tableId, tableChange); @@ -122,6 +131,7 @@ private void parseSchemaByDdl( } private void buildSchemaByDescTable( + MySqlPartition partition, JdbcConnection jdbc, String descTable, TableId tableId, @@ -149,6 +159,7 @@ private void buildSchemaByDescTable( } }); parseSchemaByDdl( + partition, new MySqlTableDefinition(tableId, fieldMetas, primaryKeys).toDdl(), tableId, tableChangeMap); @@ -159,4 +170,9 @@ private void buildSchemaByDescTable( e); } } + + @Override + public void close() { + databaseSchema.close(); + } } diff --git a/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/MySqlSource.java b/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/MySqlSource.java index 8b00598074..8f33a717a5 100644 --- a/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/MySqlSource.java +++ b/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/MySqlSource.java @@ -96,6 +96,8 @@ public class MySqlSource private static final long serialVersionUID = 1L; + private static final String ENUMERATOR_SERVER_NAME = "mysql_source_split_enumerator"; + private final MySqlSourceConfigFactory configFactory; private final DebeziumDeserializationSchema deserializationSchema; @@ -164,7 +166,7 @@ public SourceReader createReader(SourceReaderContext readerContex @Override public SplitEnumerator createEnumerator( SplitEnumeratorContext enumContext) { - MySqlSourceConfig sourceConfig = configFactory.createConfig(0); + MySqlSourceConfig sourceConfig = configFactory.createConfig(0, ENUMERATOR_SERVER_NAME); final MySqlValidator validator = new MySqlValidator(sourceConfig); validator.validate(); @@ -193,7 +195,8 @@ public SplitEnumerator createEnumerator( @Override public SplitEnumerator restoreEnumerator( SplitEnumeratorContext enumContext, PendingSplitsState checkpoint) { - MySqlSourceConfig sourceConfig = configFactory.createConfig(0); + + MySqlSourceConfig sourceConfig = configFactory.createConfig(0, ENUMERATOR_SERVER_NAME); final MySqlSplitAssigner splitAssigner; if (checkpoint instanceof HybridPendingSplitsState) { diff --git a/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/assigners/ChunkSplitter.java b/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/assigners/ChunkSplitter.java index acba94c8a8..31e7629c21 100644 --- a/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/assigners/ChunkSplitter.java +++ b/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/assigners/ChunkSplitter.java @@ -20,6 +20,7 @@ import com.ververica.cdc.connectors.mysql.source.assigners.state.ChunkSplitterState; import com.ververica.cdc.connectors.mysql.source.split.MySqlSnapshotSplit; +import io.debezium.connector.mysql.MySqlPartition; import io.debezium.relational.TableId; import java.util.List; @@ -38,7 +39,8 @@ public interface ChunkSplitter { * Called to split chunks for a table, the assigner could invoke this method multiple times to * receive all the splits. */ - List splitChunks(TableId tableId) throws Exception; + List splitChunks(MySqlPartition partition, TableId tableId) + throws Exception; /** Get whether the splitter has more chunks for current table. */ boolean hasNextChunk(); diff --git a/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/assigners/MySqlChunkSplitter.java b/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/assigners/MySqlChunkSplitter.java index 8c284efd9d..c95b8f06f1 100644 --- a/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/assigners/MySqlChunkSplitter.java +++ b/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/assigners/MySqlChunkSplitter.java @@ -29,6 +29,7 @@ import com.ververica.cdc.connectors.mysql.source.split.MySqlSnapshotSplit; import com.ververica.cdc.connectors.mysql.source.utils.ChunkUtils; import com.ververica.cdc.connectors.mysql.source.utils.ObjectUtils; +import io.debezium.connector.mysql.MySqlPartition; import io.debezium.jdbc.JdbcConnection; import io.debezium.relational.Column; import io.debezium.relational.Table; @@ -112,11 +113,12 @@ public void open() { } @Override - public List splitChunks(TableId tableId) throws Exception { + public List splitChunks(MySqlPartition partition, TableId tableId) + throws Exception { if (!hasNextChunk()) { - analyzeTable(tableId); + analyzeTable(partition, tableId); Optional> evenlySplitChunks = - trySplitAllEvenlySizedChunks(tableId); + trySplitAllEvenlySizedChunks(partition, tableId); if (evenlySplitChunks.isPresent()) { return evenlySplitChunks.get(); } else { @@ -124,7 +126,8 @@ public List splitChunks(TableId tableId) throws Exception { this.currentSplittingTableId = tableId; this.nextChunkStart = ChunkSplitterState.ChunkBound.START_BOUND; this.nextChunkId = 0; - return Collections.singletonList(splitOneUnevenlySizedChunk(tableId)); + return Collections.singletonList( + splitOneUnevenlySizedChunk(partition, tableId)); } } } else { @@ -132,18 +135,19 @@ public List splitChunks(TableId tableId) throws Exception { currentSplittingTableId.equals(tableId), "Can not split a new table before the previous table splitting finish."); if (currentSplittingTable == null) { - analyzeTable(currentSplittingTableId); + analyzeTable(partition, currentSplittingTableId); } synchronized (lock) { - return Collections.singletonList(splitOneUnevenlySizedChunk(tableId)); + return Collections.singletonList(splitOneUnevenlySizedChunk(partition, tableId)); } } } /** Analyze the meta information for given table. */ - private void analyzeTable(TableId tableId) { + private void analyzeTable(MySqlPartition partition, TableId tableId) { try { - currentSplittingTable = mySqlSchema.getTableSchema(jdbcConnection, tableId).getTable(); + currentSplittingTable = + mySqlSchema.getTableSchema(partition, jdbcConnection, tableId).getTable(); splitColumn = ChunkUtils.getChunkKeyColumn( currentSplittingTable, sourceConfig.getChunkKeyColumns()); @@ -156,7 +160,8 @@ private void analyzeTable(TableId tableId) { } /** Generates one snapshot split (chunk) for the give table path. */ - private MySqlSnapshotSplit splitOneUnevenlySizedChunk(TableId tableId) throws SQLException { + private MySqlSnapshotSplit splitOneUnevenlySizedChunk(MySqlPartition partition, TableId tableId) + throws SQLException { final int chunkSize = sourceConfig.getSplitSize(); final Object chunkStartVal = nextChunkStart.getValue(); LOG.info( @@ -182,12 +187,24 @@ private MySqlSnapshotSplit splitOneUnevenlySizedChunk(TableId tableId) throws SQ if (chunkEnd != null && ObjectUtils.compare(chunkEnd, minMaxOfSplitColumn[1]) <= 0) { nextChunkStart = ChunkSplitterState.ChunkBound.middleOf(chunkEnd); return createSnapshotSplit( - jdbcConnection, tableId, nextChunkId++, splitType, chunkStartVal, chunkEnd); + jdbcConnection, + partition, + tableId, + nextChunkId++, + splitType, + chunkStartVal, + chunkEnd); } else { currentSplittingTableId = null; nextChunkStart = ChunkSplitterState.ChunkBound.END_BOUND; return createSnapshotSplit( - jdbcConnection, tableId, nextChunkId++, splitType, chunkStartVal, null); + jdbcConnection, + partition, + tableId, + nextChunkId++, + splitType, + chunkStartVal, + null); } } @@ -198,14 +215,16 @@ private MySqlSnapshotSplit splitOneUnevenlySizedChunk(TableId tableId) throws SQ * using evenly-sized chunks which is much efficient, using unevenly-sized chunks which will * request many queries and is not efficient. */ - private Optional> trySplitAllEvenlySizedChunks(TableId tableId) { + private Optional> trySplitAllEvenlySizedChunks( + MySqlPartition partition, TableId tableId) { LOG.debug("Try evenly splitting table {} into chunks", tableId); final Object min = minMaxOfSplitColumn[0]; final Object max = minMaxOfSplitColumn[1]; if (min == null || max == null || min.equals(max)) { // empty table, or only one row, return full table scan as a chunk return Optional.of( - generateSplits(tableId, Collections.singletonList(ChunkRange.all()))); + generateSplits( + partition, tableId, Collections.singletonList(ChunkRange.all()))); } final int chunkSize = sourceConfig.getSplitSize(); @@ -216,7 +235,7 @@ private Optional> trySplitAllEvenlySizedChunks(TableId List chunks = splitEvenlySizedChunks( tableId, min, max, approximateRowCnt, chunkSize, dynamicChunkSize); - return Optional.of(generateSplits(tableId, chunks)); + return Optional.of(generateSplits(partition, tableId, chunks)); } else { LOG.debug("beginning unevenly splitting table {} into chunks", tableId); return Optional.empty(); @@ -224,7 +243,8 @@ private Optional> trySplitAllEvenlySizedChunks(TableId } /** Generates all snapshot splits (chunks) from chunk ranges. */ - private List generateSplits(TableId tableId, List chunks) { + private List generateSplits( + MySqlPartition partition, TableId tableId, List chunks) { // convert chunks into splits List splits = new ArrayList<>(); for (int i = 0; i < chunks.size(); i++) { @@ -232,6 +252,7 @@ private List generateSplits(TableId tableId, List schema = new HashMap<>(); - schema.put(tableId, mySqlSchema.getTableSchema(jdbc, tableId)); + schema.put(tableId, mySqlSchema.getTableSchema(partition, jdbc, tableId)); return new MySqlSnapshotSplit( tableId, splitId(tableId, chunkId), @@ -462,5 +484,6 @@ public void close() throws Exception { if (jdbcConnection != null) { jdbcConnection.close(); } + mySqlSchema.close(); } } diff --git a/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/assigners/MySqlSnapshotSplitAssigner.java b/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/assigners/MySqlSnapshotSplitAssigner.java index d87455ff0e..47326e6241 100644 --- a/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/assigners/MySqlSnapshotSplitAssigner.java +++ b/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/assigners/MySqlSnapshotSplitAssigner.java @@ -32,6 +32,7 @@ import com.ververica.cdc.connectors.mysql.source.split.MySqlSchemalessSnapshotSplit; import com.ververica.cdc.connectors.mysql.source.split.MySqlSnapshotSplit; import com.ververica.cdc.connectors.mysql.source.split.MySqlSplit; +import io.debezium.connector.mysql.MySqlPartition; import io.debezium.jdbc.JdbcConnection; import io.debezium.relational.TableId; import io.debezium.relational.history.TableChanges; @@ -81,6 +82,8 @@ public class MySqlSnapshotSplitAssigner implements MySqlSplitAssigner { private final int currentParallelism; private final List remainingTables; private final boolean isRemainingTablesCheckpointed; + + private final MySqlPartition partition; private final Object lock = new Object(); private volatile Throwable uncaughtSplitterException; @@ -156,6 +159,8 @@ private MySqlSnapshotSplitAssigner( this.isTableIdCaseSensitive = isTableIdCaseSensitive; this.chunkSplitter = createChunkSplitter(sourceConfig, isTableIdCaseSensitive, chunkSplitterState); + this.partition = + new MySqlPartition(sourceConfig.getMySqlConnectorConfig().getLogicalName()); } @Override @@ -275,7 +280,7 @@ private void splitTable(TableId nextTable) { synchronized (lock) { List splits; try { - splits = chunkSplitter.splitChunks(nextTable); + splits = chunkSplitter.splitChunks(partition, nextTable); } catch (Exception e) { throw new IllegalStateException( "Error when splitting chunks for " + nextTable, e); diff --git a/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/config/MySqlSourceConfigFactory.java b/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/config/MySqlSourceConfigFactory.java index 843ce749e0..6e51357712 100644 --- a/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/config/MySqlSourceConfigFactory.java +++ b/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/config/MySqlSourceConfigFactory.java @@ -279,15 +279,20 @@ public MySqlSourceConfigFactory closeIdleReaders(boolean closeIdleReaders) { /** Creates a new {@link MySqlSourceConfig} for the given subtask {@code subtaskId}. */ public MySqlSourceConfig createConfig(int subtaskId) { - checkSupportCheckpointsAfterTasksFinished(closeIdleReaders); - Properties props = new Properties(); // hard code server name, because we don't need to distinguish it, docs: // Logical name that identifies and provides a namespace for the particular // MySQL database server/cluster being monitored. The logical name should be // unique across all other connectors, since it is used as a prefix for all // Kafka topic names emanating from this connector. // Only alphanumeric characters and underscores should be used. - props.setProperty("database.server.name", "mysql_binlog_source"); + return createConfig(subtaskId, "mysql_binlog_source"); + } + + /** Creates a new {@link MySqlSourceConfig} for the given subtask {@code subtaskId}. */ + public MySqlSourceConfig createConfig(int subtaskId, String serverName) { + checkSupportCheckpointsAfterTasksFinished(closeIdleReaders); + Properties props = new Properties(); + props.setProperty("database.server.name", serverName); props.setProperty("database.hostname", checkNotNull(hostname)); props.setProperty("database.user", checkNotNull(username)); props.setProperty("database.password", checkNotNull(password)); diff --git a/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/reader/MySqlSourceReader.java b/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/reader/MySqlSourceReader.java index 610c749677..dc7ccaf464 100644 --- a/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/reader/MySqlSourceReader.java +++ b/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/reader/MySqlSourceReader.java @@ -48,6 +48,7 @@ import com.ververica.cdc.connectors.mysql.source.split.SourceRecords; import com.ververica.cdc.connectors.mysql.source.utils.TableDiscoveryUtils; import io.debezium.connector.mysql.MySqlConnection; +import io.debezium.connector.mysql.MySqlPartition; import io.debezium.relational.TableId; import io.debezium.relational.history.TableChanges; import org.slf4j.Logger; @@ -81,6 +82,7 @@ public class MySqlSourceReader private final Map uncompletedBinlogSplits; private final int subtaskId; private final MySqlSourceReaderContext mySqlSourceReaderContext; + private final MySqlPartition partition; private volatile MySqlBinlogSplit suspendedBinlogSplit; public MySqlSourceReader( @@ -102,6 +104,8 @@ public MySqlSourceReader( this.subtaskId = context.getSourceReaderContext().getIndexOfSubtask(); this.mySqlSourceReaderContext = context; this.suspendedBinlogSplit = null; + this.partition = + new MySqlPartition(sourceConfig.getMySqlConnectorConfig().getLogicalName()); } @Override @@ -400,7 +404,8 @@ private MySqlBinlogSplit discoverTableSchemasForBinlogSplit( Map tableSchemas; if (split.getTableSchemas().isEmpty()) { tableSchemas = - TableDiscoveryUtils.discoverSchemaForCapturedTables(sourceConfig, jdbc); + TableDiscoveryUtils.discoverSchemaForCapturedTables( + partition, sourceConfig, jdbc); LOG.info( "Source reader {} discovers table schema for binlog split {} success", subtaskId, @@ -408,7 +413,8 @@ private MySqlBinlogSplit discoverTableSchemasForBinlogSplit( } else { List existedTables = new ArrayList<>(split.getTableSchemas().keySet()); tableSchemas = - discoverSchemaForNewAddedTables(existedTables, sourceConfig, jdbc); + discoverSchemaForNewAddedTables( + partition, existedTables, sourceConfig, jdbc); LOG.info( "Source reader {} discovers table schema for new added tables of binlog split {} success", subtaskId, diff --git a/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/utils/StatementUtils.java b/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/utils/StatementUtils.java index 8caeb10807..facfb820f2 100644 --- a/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/utils/StatementUtils.java +++ b/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/utils/StatementUtils.java @@ -61,7 +61,7 @@ public static long queryApproximateRowCnt(JdbcConnection jdbc, TableId tableId) // accurate than COUNT(*), but is more efficient for large table. final String useDatabaseStatement = String.format("USE %s;", quote(tableId.catalog())); final String rowCountQuery = String.format("SHOW TABLE STATUS LIKE '%s';", tableId.table()); - jdbc.executeWithoutCommitting(useDatabaseStatement); + jdbc.execute(useDatabaseStatement); return jdbc.queryAndMap( rowCountQuery, rs -> { diff --git a/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/utils/TableDiscoveryUtils.java b/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/utils/TableDiscoveryUtils.java index 3cb9f27631..32b33755b8 100644 --- a/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/utils/TableDiscoveryUtils.java +++ b/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/utils/TableDiscoveryUtils.java @@ -21,6 +21,7 @@ import com.ververica.cdc.connectors.mysql.schema.MySqlSchema; import com.ververica.cdc.connectors.mysql.source.config.MySqlSourceConfig; import io.debezium.connector.mysql.MySqlConnection; +import io.debezium.connector.mysql.MySqlPartition; import io.debezium.jdbc.JdbcConnection; import io.debezium.relational.RelationalTableFilters; import io.debezium.relational.TableId; @@ -96,18 +97,21 @@ public static List listTables(JdbcConnection jdbc, RelationalTableFilte } public static Map discoverSchemaForCapturedTables( - MySqlSourceConfig sourceConfig, MySqlConnection jdbc) { + MySqlPartition partition, MySqlSourceConfig sourceConfig, MySqlConnection jdbc) { final List capturedTableIds; try { capturedTableIds = listTables(jdbc, sourceConfig.getTableFilters()); } catch (SQLException e) { throw new FlinkRuntimeException("Failed to discover captured tables", e); } - return discoverSchemaForCapturedTables(capturedTableIds, sourceConfig, jdbc); + return discoverSchemaForCapturedTables(partition, capturedTableIds, sourceConfig, jdbc); } public static Map discoverSchemaForNewAddedTables( - List existedTables, MySqlSourceConfig sourceConfig, MySqlConnection jdbc) { + MySqlPartition partition, + List existedTables, + MySqlSourceConfig sourceConfig, + MySqlConnection jdbc) { final List capturedTableIds; try { capturedTableIds = @@ -119,24 +123,30 @@ public static Map discoverSchemaForNewAddedTables( } return capturedTableIds.isEmpty() ? new HashMap<>() - : discoverSchemaForCapturedTables(capturedTableIds, sourceConfig, jdbc); + : discoverSchemaForCapturedTables(partition, capturedTableIds, sourceConfig, jdbc); } public static Map discoverSchemaForCapturedTables( - List capturedTableIds, MySqlSourceConfig sourceConfig, MySqlConnection jdbc) { + MySqlPartition partition, + List capturedTableIds, + MySqlSourceConfig sourceConfig, + MySqlConnection jdbc) { 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())); } + // fetch table schemas - MySqlSchema mySqlSchema = new MySqlSchema(sourceConfig, jdbc.isTableIdCaseSensitive()); - Map tableSchemas = new HashMap<>(); - for (TableId tableId : capturedTableIds) { - TableChange tableSchema = mySqlSchema.getTableSchema(jdbc, tableId); - tableSchemas.put(tableId, tableSchema); + try (MySqlSchema mySqlSchema = + new MySqlSchema(sourceConfig, jdbc.isTableIdCaseSensitive())) { + Map tableSchemas = new HashMap<>(); + for (TableId tableId : capturedTableIds) { + TableChange tableSchema = mySqlSchema.getTableSchema(partition, jdbc, tableId); + tableSchemas.put(tableId, tableSchema); + } + return tableSchemas; } - return tableSchemas; } } diff --git a/flink-connector-mysql-cdc/src/main/java/io/debezium/connector/mysql/MySqlConnection.java b/flink-connector-mysql-cdc/src/main/java/io/debezium/connector/mysql/MySqlConnection.java index 1d68becd41..faa4997a87 100644 --- a/flink-connector-mysql-cdc/src/main/java/io/debezium/connector/mysql/MySqlConnection.java +++ b/flink-connector-mysql-cdc/src/main/java/io/debezium/connector/mysql/MySqlConnection.java @@ -6,6 +6,7 @@ package io.debezium.connector.mysql; +import com.mysql.cj.CharsetMapping; import io.debezium.DebeziumException; import io.debezium.config.CommonConnectorConfig; import io.debezium.config.CommonConnectorConfig.EventProcessingFailureHandlingMode; @@ -14,6 +15,7 @@ import io.debezium.config.Field; import io.debezium.connector.mysql.MySqlConnectorConfig.SecureConnectionMode; import io.debezium.connector.mysql.legacy.MySqlJdbcContext.DatabaseLocales; +import io.debezium.jdbc.JdbcConfiguration; import io.debezium.jdbc.JdbcConnection; import io.debezium.relational.Column; import io.debezium.relational.Table; @@ -24,7 +26,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.sql.Connection; import java.sql.ResultSet; import java.sql.SQLException; import java.time.Duration; @@ -37,23 +38,23 @@ import java.util.Properties; /** - * Copied from Debezium project(1.6.4.final) to add custom jdbc properties in the jdbc url. The new + * Copied from Debezium project(1.9.7.final) to add custom jdbc properties in the jdbc url. The new * parameter {@code jdbcProperties} in the constructor of {@link MySqlConnectionConfiguration} will * be used to generate the jdbc url pattern, and may overwrite the default value. * - *

Line 71: Add field {@code urlPattern} in {@link MySqlConnection} and remove old pattern. + *

Line 75: Add field {@code urlPattern} in {@link MySqlConnection} and remove old pattern. * - *

Line 84: Init {@code urlPattern} using the url pattern from {@link + *

Line 92: Init {@code urlPattern} using the url pattern from {@link * MySqlConnectionConfiguration}. * - *

Line 552: Generate the connection string by the new field {@code urlPattern}. + *

Line 544: Generate the connection string by the new field {@code urlPattern}. * - *

Line 566 ~ 577: Add new constant and field {@code urlPattern} to {@link + *

Line 569 ~ 574: Add new constant and field {@code urlPattern} to {@link * MySqlConnectionConfiguration}. * - *

Line 622 ~ 625: Init new field {@code urlPattern} in {@link MySqlConnectionConfiguration}. + *

Line 625: Init new field {@code urlPattern} in {@link MySqlConnectionConfiguration}. * - *

Line 686 ~ 716: Add some methods helping to generate the url pattern and add default values. + *

Line 715 ~ 741: Add some methods helping to generate the url pattern and add default values. */ public class MySqlConnection extends JdbcConnection { @@ -64,10 +65,12 @@ public class MySqlConnection extends JdbcConnection { "SHOW VARIABLES WHERE Variable_name IN ('character_set_server','collation_server')"; private static final String SQL_SHOW_SESSION_VARIABLE_SSL_VERSION = "SHOW SESSION STATUS LIKE 'Ssl_version'"; + private static final String QUOTED_CHARACTER = "`"; private final Map originalSystemProperties = new HashMap<>(); private final MySqlConnectionConfiguration connectionConfig; - private final MysqlFieldReader mysqlFieldReader; + private final MySqlFieldReader mysqlFieldReader; + private final String urlPattern; /** @@ -77,8 +80,12 @@ public class MySqlConnection extends JdbcConnection { * @param fieldReader binary or text protocol based readers */ public MySqlConnection( - MySqlConnectionConfiguration connectionConfig, MysqlFieldReader fieldReader) { - super(connectionConfig.config(), connectionConfig.factory()); + MySqlConnectionConfiguration connectionConfig, MySqlFieldReader fieldReader) { + super( + connectionConfig.jdbcConfig, + connectionConfig.factory(), + QUOTED_CHARACTER, + QUOTED_CHARACTER); this.connectionConfig = connectionConfig; this.mysqlFieldReader = fieldReader; this.urlPattern = connectionConfig.getUrlPattern(); @@ -90,27 +97,7 @@ public MySqlConnection( * @param connectionConfig {@link MySqlConnectionConfiguration} instance, may not be null. */ public MySqlConnection(MySqlConnectionConfiguration connectionConfig) { - this(connectionConfig, new MysqlTextProtocolFieldReader()); - } - - @Override - public synchronized Connection connection(boolean executeOnConnect) throws SQLException { - if (!isConnected() && connectionConfig.sslModeEnabled()) { - originalSystemProperties.clear(); - // Set the System properties for SSL for the MySQL driver ... - setSystemProperty("javax.net.ssl.keyStore", MySqlConnectorConfig.SSL_KEYSTORE, true); - setSystemProperty( - "javax.net.ssl.keyStorePassword", - MySqlConnectorConfig.SSL_KEYSTORE_PASSWORD, - false); - setSystemProperty( - "javax.net.ssl.trustStore", MySqlConnectorConfig.SSL_TRUSTSTORE, true); - setSystemProperty( - "javax.net.ssl.trustStorePassword", - MySqlConnectorConfig.SSL_TRUSTSTORE_PASSWORD, - false); - } - return super.connection(executeOnConnect); + this(connectionConfig, new MySqlTextProtocolFieldReader(null)); } @Override @@ -202,7 +189,7 @@ protected String setStatementFor(Map variables) { } protected void setSystemProperty(String property, Field field, boolean showValueInError) { - String value = connectionConfig.config().getString(field); + String value = connectionConfig.originalConfig().getString(field); if (value != null) { value = value.trim(); String existingValue = System.getProperty(property); @@ -548,10 +535,25 @@ protected Map readDatabaseCollations() { } } + public MySqlConnectionConfiguration connectionConfig() { + return connectionConfig; + } + public String connectionString() { return connectionString(urlPattern); } + public static String getJavaEncodingForMysqlCharSet(String mysqlCharsetName) { + return CharsetMappingWrapper.getJavaEncodingForMysqlCharSet(mysqlCharsetName); + } + + /** Helper to gain access to protected method. */ + private static final class CharsetMappingWrapper extends CharsetMapping { + static String getJavaEncodingForMysqlCharSet(String mySqlCharsetName) { + return CharsetMapping.getStaticJavaEncodingForMysqlCharset(mySqlCharsetName); + } + } + /** Connection configuration to create a {@link MySqlConnection}. */ public static class MySqlConnectionConfiguration { @@ -559,7 +561,7 @@ public static class MySqlConnectionConfiguration { protected static final String JDBC_PROPERTY_CONNECTION_TIME_ZONE = "connectionTimeZone"; protected static final String JDBC_PROPERTY_LEGACY_SERVER_TIME_ZONE = "serverTimezone"; - private final Configuration jdbcConfig; + private final JdbcConfiguration jdbcConfig; private final ConnectionFactory factory; private final Configuration config; @@ -617,7 +619,7 @@ public MySqlConnectionConfiguration(Configuration config, Properties jdbcPropert jdbcConfigBuilder.with( JDBC_PROPERTY_CONNECTION_TIME_ZONE, determineConnectionTimeZone(dbConfig)); - this.jdbcConfig = jdbcConfigBuilder.build(); + this.jdbcConfig = JdbcConfiguration.adapt(jdbcConfigBuilder.build()); String driverClassName = this.jdbcConfig.getString(MySqlConnectorConfig.JDBC_DRIVER); this.urlPattern = formatJdbcUrl(jdbcProperties); factory = @@ -646,10 +648,14 @@ private static String determineConnectionTimeZone(final Configuration dbConfig) return connectionTimeZone != null ? connectionTimeZone : "SERVER"; } - public Configuration config() { + public JdbcConfiguration config() { return jdbcConfig; } + public Configuration originalConfig() { + return config; + } + public ConnectionFactory factory() { return factory; } @@ -679,6 +685,24 @@ public boolean sslModeEnabled() { return sslMode() != SecureConnectionMode.DISABLED; } + public String sslKeyStore() { + return config.getString(MySqlConnectorConfig.SSL_KEYSTORE); + } + + public char[] sslKeyStorePassword() { + String password = config.getString(MySqlConnectorConfig.SSL_KEYSTORE_PASSWORD); + return Strings.isNullOrBlank(password) ? null : password.toCharArray(); + } + + public String sslTrustStore() { + return config.getString(MySqlConnectorConfig.SSL_TRUSTSTORE); + } + + public char[] sslTrustStorePassword() { + String password = config.getString(MySqlConnectorConfig.SSL_TRUSTSTORE_PASSWORD); + return Strings.isNullOrBlank(password) ? null : password.toCharArray(); + } + public Duration getConnectionTimeout() { return Duration.ofMillis(config.getLong(MySqlConnectorConfig.CONNECTION_TIMEOUT_MS)); } diff --git a/flink-connector-mysql-cdc/src/main/java/io/debezium/connector/mysql/MySqlDefaultValueConverter.java b/flink-connector-mysql-cdc/src/main/java/io/debezium/connector/mysql/MySqlDefaultValueConverter.java deleted file mode 100644 index 671782d5d2..0000000000 --- a/flink-connector-mysql-cdc/src/main/java/io/debezium/connector/mysql/MySqlDefaultValueConverter.java +++ /dev/null @@ -1,451 +0,0 @@ -/* - * Copyright Debezium Authors. - * - * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 - */ - -package io.debezium.connector.mysql; - -import io.debezium.annotation.Immutable; -import io.debezium.relational.Column; -import io.debezium.relational.ColumnEditor; -import io.debezium.relational.ValueConverter; -import org.apache.kafka.connect.data.Field; -import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.data.SchemaBuilder; - -import java.math.BigDecimal; -import java.math.RoundingMode; -import java.sql.Timestamp; -import java.sql.Types; -import java.time.Duration; -import java.time.Instant; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.ZoneId; -import java.time.format.DateTimeFormatter; -import java.time.format.DateTimeFormatterBuilder; -import java.time.temporal.ChronoField; -import java.util.regex.Pattern; - -/** - * Copied from Debezium project(v1.6.4.Final) to fix error when parsing the string default value for - * numeric types. This class should be deleted after https://issues.redhat.com/browse/DBZ-4150 - * included. - * - *

Line 87, Line 114~129: trim the default string value when the type is a numeric type. - */ -@Immutable -public class MySqlDefaultValueConverter { - - private static final Pattern EPOCH_EQUIVALENT_TIMESTAMP = - Pattern.compile( - "(\\d{4}-\\d{2}-00|\\d{4}-00-\\d{2}|0000-\\d{2}-\\d{2}) (00:00:00(\\.\\d{1,6})?)"); - - private static final Pattern EPOCH_EQUIVALENT_DATE = - Pattern.compile("\\d{4}-\\d{2}-00|\\d{4}-00-\\d{2}|0000-\\d{2}-\\d{2}"); - - private static final String EPOCH_TIMESTAMP = "1970-01-01 00:00:00"; - - private static final String EPOCH_DATE = "1970-01-01"; - - private static final DateTimeFormatter ISO_LOCAL_DATE_WITH_OPTIONAL_TIME = - new DateTimeFormatterBuilder() - .append(DateTimeFormatter.ISO_LOCAL_DATE) - .optionalStart() - .appendLiteral(" ") - .append(DateTimeFormatter.ISO_LOCAL_TIME) - .optionalEnd() - .toFormatter(); - - private final MySqlValueConverters converters; - - public MySqlDefaultValueConverter(MySqlValueConverters converters) { - this.converters = converters; - } - - /** - * Converts a default value from the expected format to a logical object acceptable by the main - * JDBC converter. - * - * @param column column definition - * @param value string formatted default value - * @return value converted to a Java type - */ - public Object convert(Column column, String value) { - if (value == null) { - return value; - } - - // boolean is also INT(1) or TINYINT(1) - if ("TINYINT".equals(column.typeName()) || "INT".equals(column.typeName())) { - if ("true".equalsIgnoreCase(value) || "false".equalsIgnoreCase(value)) { - return convertToBoolean(value); - } - } - - value = trimValueIfNeed(column, value); - switch (column.jdbcType()) { - case Types.DATE: - return convertToLocalDate(column, value); - case Types.TIMESTAMP: - return convertToLocalDateTime(column, value); - case Types.TIMESTAMP_WITH_TIMEZONE: - return convertToTimestamp(column, value); - case Types.TIME: - return convertToDuration(column, value); - case Types.BOOLEAN: - return convertToBoolean(value); - case Types.BIT: - return convertToBits(column, value); - - case Types.NUMERIC: - case Types.DECIMAL: - return convertToDecimal(column, value); - - case Types.FLOAT: - case Types.DOUBLE: - case Types.REAL: - return convertToDouble(value); - } - return value; - } - - private String trimValueIfNeed(Column column, String value) { - int jdbcType = column.jdbcType(); - if (jdbcType == Types.DECIMAL - || jdbcType == Types.INTEGER - || jdbcType == Types.TINYINT - || jdbcType == Types.BIGINT - || jdbcType == Types.SMALLINT - || jdbcType == Types.DOUBLE - || jdbcType == Types.FLOAT - || jdbcType == Types.BOOLEAN - || jdbcType == Types.NUMERIC - || jdbcType == Types.REAL) { - return value.trim(); - } - return value; - } - - /** - * Converts a string object for an object type of {@link LocalDate} or {@link LocalDateTime} in - * case of MySql Date type. If the column definition allows null and default value is - * 0000-00-00, we need return null; else 0000-00-00 will be replaced with 1970-01-01; - * - * @param column the column definition describing the {@code data} value; never null - * @param value the string object to be converted into a {@link LocalDate} type or {@link - * LocalDateTime} in case of MySql Date type; - * @return the converted value; - */ - private Object convertToLocalDate(Column column, String value) { - final boolean zero = - EPOCH_EQUIVALENT_DATE.matcher(value).matches() - || EPOCH_EQUIVALENT_TIMESTAMP.matcher(value).matches() - || "0".equals(value); - - if (zero && column.isOptional()) { - return null; - } - if (zero) { - value = EPOCH_DATE; - } - return LocalDate.from(ISO_LOCAL_DATE_WITH_OPTIONAL_TIME.parse(value)); - } - - /** - * Converts a string object for an object type of {@link LocalDateTime}. If the column - * definition allows null and default value is 0000-00-00 00:00:00, we need return null, else - * 0000-00-00 00:00:00 will be replaced with 1970-01-01 00:00:00; - * - * @param column the column definition describing the {@code data} value; never null - * @param value the string object to be converted into a {@link LocalDateTime} type; - * @return the converted value; - */ - private Object convertToLocalDateTime(Column column, String value) { - final boolean matches = - EPOCH_EQUIVALENT_TIMESTAMP.matcher(value).matches() || "0".equals(value); - if (matches) { - if (column.isOptional()) { - return null; - } - - value = EPOCH_TIMESTAMP; - } - - return LocalDateTime.from(timestampFormat(column.length()).parse(value)); - } - - /** - * Converts a string object for an object type of {@link Timestamp}. If the column definition - * allows null and default value is 0000-00-00 00:00:00, we need return null, else 0000-00-00 - * 00:00:00 will be replaced with 1970-01-01 00:00:00; - * - * @param column the column definition describing the {@code data} value; never null - * @param value the string object to be converted into a {@link Timestamp} type; - * @return the converted value; - */ - private Object convertToTimestamp(Column column, String value) { - final boolean matches = - EPOCH_EQUIVALENT_TIMESTAMP.matcher(value).matches() - || "0".equals(value) - || EPOCH_TIMESTAMP.equals(value); - if (matches) { - if (column.isOptional()) { - return null; - } - - return Timestamp.from(Instant.EPOCH); - } - value = cleanTimestamp(value); - return Timestamp.valueOf(value).toInstant().atZone(ZoneId.systemDefault()); - } - - /** - * Converts a string object for an object type of {@link Duration}. - * - * @param column the column definition describing the {@code data} value; never null - * @param value the string object to be converted into a {@link Duration} type; - * @return the converted value; - */ - private Object convertToDuration(Column column, String value) { - return MySqlValueConverters.stringToDuration(value); - } - - /** - * Converts a string object for an expected JDBC type of {@link Types#DOUBLE}. - * - * @param value the string object to be converted into a {@link Types#DOUBLE} type; - * @return the converted value; - */ - private Object convertToDouble(String value) { - return Double.parseDouble(value); - } - - /** - * Converts a string object for an expected JDBC type of {@link Types#DECIMAL}. - * - * @param column the column definition describing the {@code data} value; never null - * @param value the string object to be converted into a {@link Types#DECIMAL} type; - * @return the converted value; - */ - private Object convertToDecimal(Column column, String value) { - return column.scale().isPresent() - ? new BigDecimal(value).setScale(column.scale().get(), RoundingMode.HALF_UP) - : new BigDecimal(value); - } - - /** - * Converts a string object for an expected JDBC type of {@link Types#BIT}. - * - * @param column the column definition describing the {@code data} value; never null - * @param value the string object to be converted into a {@link Types#BIT} type; - * @return the converted value; - */ - private Object convertToBits(Column column, String value) { - if (column.length() > 1) { - return convertToBits(value); - } - return convertToBit(value); - } - - private Object convertToBit(String value) { - try { - return Short.parseShort(value) != 0; - } catch (NumberFormatException ignore) { - return Boolean.parseBoolean(value); - } - } - - private Object convertToBits(String value) { - int nums = value.length() / Byte.SIZE + (value.length() % Byte.SIZE == 0 ? 0 : 1); - byte[] bytes = new byte[nums]; - for (int i = 0; i < nums; i++) { - int s = value.length() - Byte.SIZE < 0 ? 0 : value.length() - Byte.SIZE; - int e = value.length(); - bytes[nums - i - 1] = (byte) Integer.parseInt(value.substring(s, e), 2); - value = value.substring(0, s); - } - return bytes; - } - - /** - * Converts a string object for an expected JDBC type of {@link Types#BOOLEAN}. - * - * @param value the string object to be converted into a {@link Types#BOOLEAN} type; - * @return the converted value; - */ - private Object convertToBoolean(String value) { - try { - return Integer.parseInt(value) != 0; - } catch (NumberFormatException ignore) { - return Boolean.parseBoolean(value); - } - } - - private DateTimeFormatter timestampFormat(int length) { - final DateTimeFormatterBuilder dtf = - new DateTimeFormatterBuilder() - .appendPattern("yyyy-MM-dd") - .optionalStart() - .appendLiteral(" ") - .append(DateTimeFormatter.ISO_LOCAL_TIME) - .optionalEnd() - .parseDefaulting(ChronoField.HOUR_OF_DAY, 0) - .parseDefaulting(ChronoField.MINUTE_OF_HOUR, 0) - .parseDefaulting(ChronoField.SECOND_OF_MINUTE, 0); - if (length > 0) { - dtf.appendFraction(ChronoField.MICRO_OF_SECOND, 0, length, true); - } - return dtf.toFormatter(); - } - - /** - * Clean input timestamp to yyyy-mm-dd hh:mm:ss[.fffffffff] format. - * - * @param s input timestamp - * @return cleaned timestamp - */ - private String cleanTimestamp(String s) { - if (s == null) { - throw new java.lang.IllegalArgumentException("null string"); - } - - s = s.trim(); - - // clean first dash - s = replaceFirstNonNumericSubstring(s, 0, '-'); - // clean second dash - s = replaceFirstNonNumericSubstring(s, s.indexOf('-') + 1, '-'); - // clean dividing space - s = replaceFirstNonNumericSubstring(s, s.indexOf('-', s.indexOf('-') + 1) + 1, ' '); - if (s.indexOf(' ') != -1) { - // clean first colon - s = replaceFirstNonNumericSubstring(s, s.indexOf(' ') + 1, ':'); - if (s.indexOf(':') != -1) { - // clean second colon - s = replaceFirstNonNumericSubstring(s, s.indexOf(':') + 1, ':'); - } - } - - final int maxMonth = 12; - final int maxDay = 31; - - // Parse the date - int firstDash = s.indexOf('-'); - int secondDash = s.indexOf('-', firstDash + 1); - int dividingSpace = s.indexOf(' '); - - // Parse the time - int firstColon = s.indexOf(':', dividingSpace + 1); - int secondColon = s.indexOf(':', firstColon + 1); - int period = s.indexOf('.', secondColon + 1); - - int year = 0; - int month = 0; - int day = 0; - int hour = 0; - int minute = 0; - int second = 0; - - // Get the date - int len = s.length(); - boolean parsedDate = false; - if (firstDash > 0 && secondDash > firstDash) { - year = Integer.parseInt(s.substring(0, firstDash)); - month = Integer.parseInt(s.substring(firstDash + 1, secondDash)); - if (dividingSpace != -1) { - day = Integer.parseInt(s.substring(secondDash + 1, dividingSpace)); - } else { - day = Integer.parseInt(s.substring(secondDash + 1, len)); - } - - if ((month >= 1 && month <= maxMonth) && (day >= 1 && day <= maxDay)) { - parsedDate = true; - } - } - if (!parsedDate) { - throw new java.lang.IllegalArgumentException("Cannot parse the date from " + s); - } - - // Get the time. Hour, minute, second and colons are all optional - if (dividingSpace != -1 && dividingSpace < len - 1) { - if (firstColon == -1) { - hour = Integer.parseInt(s.substring(dividingSpace + 1, len)); - } else { - hour = Integer.parseInt(s.substring(dividingSpace + 1, firstColon)); - if (firstColon < len - 1) { - if (secondColon == -1) { - minute = Integer.parseInt(s.substring(firstColon + 1, len)); - } else { - minute = Integer.parseInt(s.substring(firstColon + 1, secondColon)); - if (secondColon < len - 1) { - if (period == -1) { - second = Integer.parseInt(s.substring(secondColon + 1, len)); - } else { - second = Integer.parseInt(s.substring(secondColon + 1, period)); - } - } - } - } - } - } - - StringBuilder cleanedTimestamp = new StringBuilder(); - cleanedTimestamp = - cleanedTimestamp.append( - String.format( - "%04d-%02d-%02d %02d:%02d:%02d", - year, month, day, hour, minute, second)); - - if (period != -1 && period < len - 1) { - cleanedTimestamp = cleanedTimestamp.append(".").append(s.substring(period + 1)); - } - - return cleanedTimestamp.toString(); - } - - /** - * Replace the first non-numeric substring. - * - * @param s the original string - * @param startIndex the beginning index, inclusive - * @param c the new character - * @return - */ - private String replaceFirstNonNumericSubstring(String s, int startIndex, char c) { - StringBuilder sb = new StringBuilder(); - sb.append(s.substring(0, startIndex)); - - String rest = s.substring(startIndex); - sb.append(rest.replaceFirst("[^\\d]+", Character.toString(c))); - return sb.toString(); - } - - public ColumnEditor setColumnDefaultValue(ColumnEditor columnEditor) { - final Column column = columnEditor.create(); - - // if converters is not null and the default value is not null, we need to convert default - // value - if (converters != null && columnEditor.defaultValue() != null) { - Object defaultValue = columnEditor.defaultValue(); - final SchemaBuilder schemaBuilder = converters.schemaBuilder(column); - if (schemaBuilder == null) { - return columnEditor; - } - final Schema schema = schemaBuilder.build(); - // In order to get the valueConverter for this column, we have to create a field; - // The index value -1 in the field will never used when converting default value; - // So we can set any number here; - final Field field = new Field(columnEditor.name(), -1, schema); - final ValueConverter valueConverter = - converters.converter(columnEditor.create(), field); - if (defaultValue instanceof String) { - defaultValue = convert(column, (String) defaultValue); - } - defaultValue = valueConverter.convert(defaultValue); - columnEditor.defaultValue(defaultValue); - } - return columnEditor; - } -} diff --git a/flink-connector-mysql-cdc/src/main/java/io/debezium/connector/mysql/MySqlStreamingChangeEventSource.java b/flink-connector-mysql-cdc/src/main/java/io/debezium/connector/mysql/MySqlStreamingChangeEventSource.java index f7ac0bded8..70d2204900 100644 --- a/flink-connector-mysql-cdc/src/main/java/io/debezium/connector/mysql/MySqlStreamingChangeEventSource.java +++ b/flink-connector-mysql-cdc/src/main/java/io/debezium/connector/mysql/MySqlStreamingChangeEventSource.java @@ -56,9 +56,9 @@ import javax.net.ssl.KeyManagerFactory; import javax.net.ssl.SSLContext; import javax.net.ssl.TrustManager; +import javax.net.ssl.TrustManagerFactory; import javax.net.ssl.X509TrustManager; -import java.io.FileInputStream; import java.io.IOException; import java.security.GeneralSecurityException; import java.security.KeyStore; @@ -83,17 +83,9 @@ import static io.debezium.util.Strings.isNullOrEmpty; -/** - * Copied from Debezium project to fix https://github.com/ververica/flink-cdc-connectors/issues/939. - * - *

Line 268 ~ 270: Clean cache on rotate event to prevent it from growing indefinitely. We should - * remove this class after we bumped a higher debezium version where the - * https://issues.redhat.com/browse/DBZ-5126 has been fixed. - * - *

Line 1386 : Add more error details for some exceptions. - */ +/** Copied from Debezium project to add more error details for some exceptions in line 1465. */ public class MySqlStreamingChangeEventSource - implements StreamingChangeEventSource { + implements StreamingChangeEventSource { private static final Logger LOGGER = LoggerFactory.getLogger(MySqlStreamingChangeEventSource.class); @@ -121,7 +113,7 @@ public class MySqlStreamingChangeEventSource private final MySqlTaskContext taskContext; private final MySqlConnectorConfig connectorConfig; private final MySqlConnection connection; - private final EventDispatcher eventDispatcher; + private final EventDispatcher eventDispatcher; private final ErrorHandler errorHandler; @SingleThreadAccess("binlog client thread") @@ -191,7 +183,7 @@ private interface BinlogChangeEmitter { public MySqlStreamingChangeEventSource( MySqlConnectorConfig connectorConfig, MySqlConnection connection, - EventDispatcher dispatcher, + EventDispatcher dispatcher, ErrorHandler errorHandler, Clock clock, MySqlTaskContext taskContext, @@ -363,7 +355,8 @@ protected void ignoreEvent(MySqlOffsetContext offsetContext, Event event) { LOGGER.trace("Ignoring event due to missing handler: {}", event); } - protected void handleEvent(MySqlOffsetContext offsetContext, Event event) { + protected void handleEvent( + MySqlPartition partition, MySqlOffsetContext offsetContext, Event event) { if (event == null) { return; } @@ -378,7 +371,7 @@ protected void handleEvent(MySqlOffsetContext offsetContext, Event event) { !eventHeader.getEventType().equals(EventType.HEARTBEAT) ? Instant.ofEpochMilli(eventHeader.getTimestamp()) : null; - offsetContext.setBinlogThread(eventHeader.getServerId()); + offsetContext.setBinlogServerId(eventHeader.getServerId()); final EventType eventType = eventHeader.getEventType(); if (eventType == EventType.ROTATE) { @@ -407,11 +400,14 @@ protected void handleEvent(MySqlOffsetContext offsetContext, Event event) { .accept(event); // Generate heartbeat message if the time is right - eventDispatcher.dispatchHeartbeatEvent(offsetContext); + eventDispatcher.dispatchHeartbeatEvent(partition, offsetContext); // Capture that we've completed another event ... offsetContext.completeEvent(); + // update last offset used for logging + lastOffset = offsetContext.getOffset(); + if (skipEvent) { // We're in the mode of skipping events and we just skipped this one, so decrement // our skip count ... @@ -462,8 +458,11 @@ protected void handleServerStop(MySqlOffsetContext offsetContext, Event event) { * * @param event the server stopped event to be processed; may not be null */ - protected void handleServerHeartbeat(MySqlOffsetContext offsetContext, Event event) { + protected void handleServerHeartbeat( + MySqlPartition partition, MySqlOffsetContext offsetContext, Event event) + throws InterruptedException { LOGGER.trace("Server heartbeat: {}", event); + eventDispatcher.dispatchServerHeartbeatEvent(partition, offsetContext); } /** @@ -473,9 +472,10 @@ protected void handleServerHeartbeat(MySqlOffsetContext offsetContext, Event eve * * @param event the server stopped event to be processed; may not be null */ - protected void handleServerIncident(MySqlOffsetContext offsetContext, Event event) { + protected void handleServerIncident( + MySqlPartition partition, MySqlOffsetContext offsetContext, Event event) { if (event.getData() instanceof EventDataDeserializationExceptionData) { - metrics.onErroneousEvent("source = " + event.toString()); + metrics.onErroneousEvent(partition, "source = " + event); EventDataDeserializationExceptionData data = event.getData(); EventHeaderV4 eventHeader = @@ -579,10 +579,12 @@ protected void handleRowsQuery(MySqlOffsetContext offsetContext, Event event) { * Handle the supplied event with an {@link QueryEventData} by possibly recording the DDL * statements as changes in the MySQL schemas. * + * @param partition the partition in which the even occurred * @param event the database change data event to be processed; may not be null * @throws InterruptedException if this thread is interrupted while recording the DDL statements */ - protected void handleQueryEvent(MySqlOffsetContext offsetContext, Event event) + protected void handleQueryEvent( + MySqlPartition partition, MySqlOffsetContext offsetContext, Event event) throws InterruptedException { QueryEventData command = unwrapData(event); LOGGER.debug("Received query command: {}", event); @@ -591,7 +593,7 @@ protected void handleQueryEvent(MySqlOffsetContext offsetContext, Event event) // We are starting a new transaction ... offsetContext.startNextTransaction(); eventDispatcher.dispatchTransactionStartedEvent( - offsetContext.getTransactionId(), offsetContext); + partition, offsetContext.getTransactionId(), offsetContext); offsetContext.setBinlogThread(command.getThreadId()); if (initialEventsToSkip != 0) { LOGGER.debug( @@ -605,7 +607,7 @@ protected void handleQueryEvent(MySqlOffsetContext offsetContext, Event event) return; } if (sql.equalsIgnoreCase("COMMIT")) { - handleTransactionCompletion(offsetContext, event); + handleTransactionCompletion(partition, offsetContext, event); return; } @@ -622,22 +624,11 @@ protected void handleQueryEvent(MySqlOffsetContext offsetContext, Event event) if (upperCasedStatementBegin.equals("INSERT ") || upperCasedStatementBegin.equals("UPDATE ") || upperCasedStatementBegin.equals("DELETE ")) { - if (eventDeserializationFailureHandlingMode - == EventProcessingFailureHandlingMode.FAIL) { - throw new DebeziumException( - "Received DML '" - + sql - + "' for processing, binlog probably contains events generated with statement or mixed based replication format"); - } else if (eventDeserializationFailureHandlingMode - == EventProcessingFailureHandlingMode.WARN) { - LOGGER.warn( - "Warning only: Received DML '" - + sql - + "' for processing, binlog probably contains events generated with statement or mixed based replication format"); - return; - } else { - return; - } + LOGGER.warn( + "Received DML '" + + sql + + "' for processing, binlog probably contains events generated with statement or mixed based replication format"); + return; } if (sql.equalsIgnoreCase("ROLLBACK")) { // We have hit a ROLLBACK which is not supported @@ -650,6 +641,7 @@ protected void handleQueryEvent(MySqlOffsetContext offsetContext, Event event) taskContext .getSchema() .parseStreamingDdl( + partition, sql, command.getDatabase(), offsetContext, @@ -665,6 +657,7 @@ protected void handleQueryEvent(MySqlOffsetContext offsetContext, Event event) ? null : schemaChangeEvent.getTables().iterator().next().id(); eventDispatcher.dispatchSchemaChangeEvent( + partition, tableId, (receiver) -> { try { @@ -679,10 +672,11 @@ protected void handleQueryEvent(MySqlOffsetContext offsetContext, Event event) } } - private void handleTransactionCompletion(MySqlOffsetContext offsetContext, Event event) + private void handleTransactionCompletion( + MySqlPartition partition, MySqlOffsetContext offsetContext, Event event) throws InterruptedException { // We are completing the transaction ... - eventDispatcher.dispatchTransactionCommittedEvent(offsetContext); + eventDispatcher.dispatchTransactionCommittedEvent(partition, offsetContext); offsetContext.commitTransaction(); offsetContext.setBinlogThread(-1L); skipEvent = false; @@ -704,7 +698,9 @@ private void handleTransactionCompletion(MySqlOffsetContext offsetContext, Event * * @param event the update event; never null */ - protected void handleUpdateTableMetadata(MySqlOffsetContext offsetContext, Event event) { + protected void handleUpdateTableMetadata( + MySqlPartition partition, MySqlOffsetContext offsetContext, Event event) + throws InterruptedException { TableMapEventData metadata = unwrapData(event); long tableNumber = metadata.getTableId(); String databaseName = metadata.getDatabase(); @@ -713,8 +709,7 @@ protected void handleUpdateTableMetadata(MySqlOffsetContext offsetContext, Event if (taskContext.getSchema().assignTableNumber(tableNumber, tableId)) { LOGGER.debug("Received update table metadata event: {}", event); } else { - informAboutUnknownTableIfRequired( - offsetContext, event, tableId, "update table metadata"); + informAboutUnknownTableIfRequired(partition, offsetContext, event, tableId); } } @@ -724,10 +719,16 @@ protected void handleUpdateTableMetadata(MySqlOffsetContext offsetContext, Event * MySqlConnectorConfig#INCONSISTENT_SCHEMA_HANDLING_MODE} configuration. */ private void informAboutUnknownTableIfRequired( - MySqlOffsetContext offsetContext, Event event, TableId tableId, String typeToLog) { + MySqlPartition partition, + MySqlOffsetContext offsetContext, + Event event, + TableId tableId, + Operation operation) + throws InterruptedException { if (tableId != null && connectorConfig.getTableFilters().dataCollectionFilter().isIncluded(tableId)) { - metrics.onErroneousEvent("source = " + tableId + ", event " + event); + metrics.onErroneousEvent( + partition, "source = " + tableId + ", event " + event, operation); EventHeaderV4 eventHeader = event.getHeader(); if (inconsistentSchemaHandlingMode == EventProcessingFailureHandlingMode.FAIL) { @@ -773,53 +774,98 @@ private void informAboutUnknownTableIfRequired( offsetContext.getSource().binlogFilename()); } } else { - LOGGER.debug( - "Filtering {} event: {} for non-monitored table {}", typeToLog, event, tableId); - metrics.onFilteredEvent("source = " + tableId); + if (tableId == null) { + EventData eventData = unwrapData(event); + if (eventData instanceof WriteRowsEventData) { + tableId = + taskContext + .getSchema() + .getExcludeTableId( + ((WriteRowsEventData) eventData).getTableId()); + } else if (eventData instanceof UpdateRowsEventData) { + tableId = + taskContext + .getSchema() + .getExcludeTableId( + ((UpdateRowsEventData) eventData).getTableId()); + } else if (eventData instanceof DeleteRowsEventData) { + tableId = + taskContext + .getSchema() + .getExcludeTableId( + ((DeleteRowsEventData) eventData).getTableId()); + } + } + LOGGER.trace("Filtered {} event for {}", event.getHeader().getEventType(), tableId); + metrics.onFilteredEvent(partition, "source = " + tableId, operation); + eventDispatcher.dispatchFilteredEvent(partition, offsetContext); } } + private void informAboutUnknownTableIfRequired( + MySqlPartition partition, + MySqlOffsetContext offsetContext, + Event event, + TableId tableId) + throws InterruptedException { + informAboutUnknownTableIfRequired(partition, offsetContext, event, tableId, null); + } + /** * Generate source records for the supplied event with an {@link WriteRowsEventData}. * + * @param partition the partition in which the even occurred * @param event the database change data event to be processed; may not be null * @throws InterruptedException if this thread is interrupted while blocking */ - protected void handleInsert(MySqlOffsetContext offsetContext, Event event) + protected void handleInsert( + MySqlPartition partition, MySqlOffsetContext offsetContext, Event event) throws InterruptedException { handleChange( + partition, offsetContext, event, - "insert", + Operation.CREATE, WriteRowsEventData.class, x -> taskContext.getSchema().getTableId(x.getTableId()), WriteRowsEventData::getRows, (tableId, row) -> eventDispatcher.dispatchDataChangeEvent( + partition, tableId, new MySqlChangeRecordEmitter( - offsetContext, clock, Operation.CREATE, null, row))); + partition, + offsetContext, + clock, + Operation.CREATE, + null, + row))); } /** * Generate source records for the supplied event with an {@link UpdateRowsEventData}. * + * @param partition the partition in which the even occurred * @param event the database change data event to be processed; may not be null * @throws InterruptedException if this thread is interrupted while blocking */ - protected void handleUpdate(MySqlOffsetContext offsetContext, Event event) + protected void handleUpdate( + MySqlPartition partition, MySqlOffsetContext offsetContext, Event event) throws InterruptedException { handleChange( + partition, offsetContext, event, - "update", + Operation.UPDATE, UpdateRowsEventData.class, x -> taskContext.getSchema().getTableId(x.getTableId()), UpdateRowsEventData::getRows, (tableId, row) -> eventDispatcher.dispatchDataChangeEvent( + partition, tableId, new MySqlChangeRecordEmitter( + partition, offsetContext, clock, Operation.UPDATE, @@ -830,29 +876,39 @@ protected void handleUpdate(MySqlOffsetContext offsetContext, Event event) /** * Generate source records for the supplied event with an {@link DeleteRowsEventData}. * + * @param partition the partition in which the even occurred * @param event the database change data event to be processed; may not be null * @throws InterruptedException if this thread is interrupted while blocking */ - protected void handleDelete(MySqlOffsetContext offsetContext, Event event) + protected void handleDelete( + MySqlPartition partition, MySqlOffsetContext offsetContext, Event event) throws InterruptedException { handleChange( + partition, offsetContext, event, - "delete", + Operation.DELETE, DeleteRowsEventData.class, x -> taskContext.getSchema().getTableId(x.getTableId()), DeleteRowsEventData::getRows, (tableId, row) -> eventDispatcher.dispatchDataChangeEvent( + partition, tableId, new MySqlChangeRecordEmitter( - offsetContext, clock, Operation.DELETE, row, null))); + partition, + offsetContext, + clock, + Operation.DELETE, + row, + null))); } private void handleChange( + MySqlPartition partition, MySqlOffsetContext offsetContext, Event event, - String changeType, + Operation operation, Class eventDataClass, TableIdProvider tableIdProvider, RowsProvider rowsProvider, @@ -870,6 +926,7 @@ private void handleChange( final T data = unwrapData(event); final TableId tableId = tableIdProvider.getTableId(data); final List rows = rowsProvider.getRows(data); + String changeType = operation.name(); if (tableId != null && taskContext.getSchema().schemaFor(tableId) != null) { int count = 0; @@ -900,7 +957,7 @@ private void handleChange( LOGGER.debug("Skipping previously processed {} event: {}", changeType, event); } } else { - informAboutUnknownTableIfRequired(offsetContext, event, tableId, changeType + " row"); + informAboutUnknownTableIfRequired(partition, offsetContext, event, tableId, operation); } startingRowNumber = 0; } @@ -946,7 +1003,10 @@ private SSLMode sslModeFor(SecureConnectionMode mode) { } @Override - public void execute(ChangeEventSourceContext context, MySqlOffsetContext offsetContext) + public void execute( + ChangeEventSourceContext context, + MySqlPartition partition, + MySqlOffsetContext offsetContext) throws InterruptedException { if (!connectorConfig.getSnapshotMode().shouldStream()) { LOGGER.info( @@ -954,7 +1014,9 @@ public void execute(ChangeEventSourceContext context, MySqlOffsetContext offsetC connectorConfig.getSnapshotMode()); return; } - taskContext.getSchema().assureNonEmptySchema(); + if (connectorConfig.getSnapshotMode() != MySqlConnectorConfig.SnapshotMode.NEVER) { + taskContext.getSchema().assureNonEmptySchema(); + } final Set skippedOperations = connectorConfig.getSkippedOperations(); final MySqlOffsetContext effectiveOffsetContext = @@ -965,39 +1027,44 @@ public void execute(ChangeEventSourceContext context, MySqlOffsetContext offsetC EventType.STOP, (event) -> handleServerStop(effectiveOffsetContext, event)); eventHandlers.put( EventType.HEARTBEAT, - (event) -> handleServerHeartbeat(effectiveOffsetContext, event)); + (event) -> handleServerHeartbeat(partition, effectiveOffsetContext, event)); eventHandlers.put( - EventType.INCIDENT, (event) -> handleServerIncident(effectiveOffsetContext, event)); + EventType.INCIDENT, + (event) -> handleServerIncident(partition, effectiveOffsetContext, event)); eventHandlers.put( EventType.ROTATE, (event) -> handleRotateLogsEvent(effectiveOffsetContext, event)); eventHandlers.put( EventType.TABLE_MAP, - (event) -> handleUpdateTableMetadata(effectiveOffsetContext, event)); + (event) -> handleUpdateTableMetadata(partition, effectiveOffsetContext, event)); eventHandlers.put( - EventType.QUERY, (event) -> handleQueryEvent(effectiveOffsetContext, event)); + EventType.QUERY, + (event) -> handleQueryEvent(partition, effectiveOffsetContext, event)); if (!skippedOperations.contains(Operation.CREATE)) { eventHandlers.put( - EventType.WRITE_ROWS, (event) -> handleInsert(effectiveOffsetContext, event)); + EventType.WRITE_ROWS, + (event) -> handleInsert(partition, effectiveOffsetContext, event)); eventHandlers.put( EventType.EXT_WRITE_ROWS, - (event) -> handleInsert(effectiveOffsetContext, event)); + (event) -> handleInsert(partition, effectiveOffsetContext, event)); } if (!skippedOperations.contains(Operation.UPDATE)) { eventHandlers.put( - EventType.UPDATE_ROWS, (event) -> handleUpdate(effectiveOffsetContext, event)); + EventType.UPDATE_ROWS, + (event) -> handleUpdate(partition, effectiveOffsetContext, event)); eventHandlers.put( EventType.EXT_UPDATE_ROWS, - (event) -> handleUpdate(effectiveOffsetContext, event)); + (event) -> handleUpdate(partition, effectiveOffsetContext, event)); } if (!skippedOperations.contains(Operation.DELETE)) { eventHandlers.put( - EventType.DELETE_ROWS, (event) -> handleDelete(effectiveOffsetContext, event)); + EventType.DELETE_ROWS, + (event) -> handleDelete(partition, effectiveOffsetContext, event)); eventHandlers.put( EventType.EXT_DELETE_ROWS, - (event) -> handleDelete(effectiveOffsetContext, event)); + (event) -> handleDelete(partition, effectiveOffsetContext, event)); } eventHandlers.put( @@ -1006,7 +1073,7 @@ public void execute(ChangeEventSourceContext context, MySqlOffsetContext offsetC EventType.XA_PREPARE, (event) -> prepareTransaction(effectiveOffsetContext, event)); eventHandlers.put( EventType.XID, - (event) -> handleTransactionCompletion(effectiveOffsetContext, event)); + (event) -> handleTransactionCompletion(partition, effectiveOffsetContext, event)); // Conditionally register ROWS_QUERY handler to parse SQL statements. if (connectorConfig.includeSqlQuery()) { @@ -1017,14 +1084,14 @@ public void execute(ChangeEventSourceContext context, MySqlOffsetContext offsetC BinaryLogClient.EventListener listener; if (connectorConfig.bufferSizeForStreamingChangeEventSource() == 0) { - listener = (event) -> handleEvent(effectiveOffsetContext, event); + listener = (event) -> handleEvent(partition, effectiveOffsetContext, event); } else { EventBuffer buffer = new EventBuffer( connectorConfig.bufferSizeForStreamingChangeEventSource(), this, context); - listener = (event) -> buffer.add(effectiveOffsetContext, event); + listener = (event) -> buffer.add(partition, effectiveOffsetContext, event); } client.registerEventListener(listener); @@ -1181,65 +1248,79 @@ private SSLSocketFactory getBinlogSslSocketFactory( String acceptedTlsVersion = connection.getSessionVariableForSslVersion(); if (!isNullOrEmpty(acceptedTlsVersion)) { SSLMode sslMode = sslModeFor(connectorConfig.sslMode()); - - // Keystore settings can be passed via system properties too so we need to read them - final String password = System.getProperty("javax.net.ssl.keyStorePassword"); - final String keyFilename = System.getProperty("javax.net.ssl.keyStore"); + LOGGER.info( + "Enable ssl " + + sslMode + + " mode for connector " + + connectorConfig.getLogicalName()); + + final char[] keyPasswordArray = connection.connectionConfig().sslKeyStorePassword(); + final String keyFilename = connection.connectionConfig().sslKeyStore(); + final char[] trustPasswordArray = connection.connectionConfig().sslTrustStorePassword(); + final String trustFilename = connection.connectionConfig().sslTrustStore(); KeyManager[] keyManagers = null; if (keyFilename != null) { - final char[] passwordArray = (password == null) ? null : password.toCharArray(); try { - KeyStore ks = KeyStore.getInstance("JKS"); - ks.load(new FileInputStream(keyFilename), passwordArray); + KeyStore ks = connection.loadKeyStore(keyFilename, keyPasswordArray); KeyManagerFactory kmf = KeyManagerFactory.getInstance("NewSunX509"); - kmf.init(ks, passwordArray); + kmf.init(ks, keyPasswordArray); keyManagers = kmf.getKeyManagers(); } catch (KeyStoreException - | IOException - | CertificateException | NoSuchAlgorithmException | UnrecoverableKeyException e) { throw new DebeziumException("Could not load keystore", e); } } + TrustManager[] trustManagers; + try { + KeyStore ks = null; + if (trustFilename != null) { + ks = connection.loadKeyStore(trustFilename, trustPasswordArray); + } - // DBZ-1208 Resembles the logic from the upstream BinaryLogClient, only that - // the accepted TLS version is passed to the constructed factory - if (sslMode == SSLMode.PREFERRED || sslMode == SSLMode.REQUIRED) { - final KeyManager[] finalKMS = keyManagers; - return new DefaultSSLSocketFactory(acceptedTlsVersion) { + if (ks == null && (sslMode == SSLMode.PREFERRED || sslMode == SSLMode.REQUIRED)) { + trustManagers = + new TrustManager[] { + new X509TrustManager() { - @Override - protected void initSSLContext(SSLContext sc) throws GeneralSecurityException { - sc.init( - finalKMS, - new TrustManager[] { - new X509TrustManager() { - - @Override - public void checkClientTrusted( - X509Certificate[] x509Certificates, String s) - throws CertificateException {} - - @Override - public void checkServerTrusted( - X509Certificate[] x509Certificates, String s) - throws CertificateException {} - - @Override - public X509Certificate[] getAcceptedIssuers() { - return new X509Certificate[0]; - } + @Override + public void checkClientTrusted( + X509Certificate[] x509Certificates, String s) + throws CertificateException {} + + @Override + public void checkServerTrusted( + X509Certificate[] x509Certificates, String s) + throws CertificateException {} + + @Override + public X509Certificate[] getAcceptedIssuers() { + return new X509Certificate[0]; } - }, - null); - } - }; - } else { - return new DefaultSSLSocketFactory(acceptedTlsVersion); + } + }; + } else { + TrustManagerFactory tmf = + TrustManagerFactory.getInstance( + TrustManagerFactory.getDefaultAlgorithm()); + tmf.init(ks); + trustManagers = tmf.getTrustManagers(); + } + } catch (KeyStoreException | NoSuchAlgorithmException e) { + throw new DebeziumException("Could not load truststore", e); } + // DBZ-1208 Resembles the logic from the upstream BinaryLogClient, only that + // the accepted TLS version is passed to the constructed factory + final KeyManager[] finalKMS = keyManagers; + return new DefaultSSLSocketFactory(acceptedTlsVersion) { + + @Override + protected void initSSLContext(SSLContext sc) throws GeneralSecurityException { + sc.init(finalKMS, trustManagers, null); + } + }; } return null; diff --git a/flink-connector-mysql-cdc/src/main/java/io/debezium/connector/mysql/antlr/listener/DefaultValueParserListener.java b/flink-connector-mysql-cdc/src/main/java/io/debezium/connector/mysql/antlr/listener/DefaultValueParserListener.java index 5aa9390854..d96a7a5ebd 100644 --- a/flink-connector-mysql-cdc/src/main/java/io/debezium/connector/mysql/antlr/listener/DefaultValueParserListener.java +++ b/flink-connector-mysql-cdc/src/main/java/io/debezium/connector/mysql/antlr/listener/DefaultValueParserListener.java @@ -6,20 +6,17 @@ package io.debezium.connector.mysql.antlr.listener; -import io.debezium.connector.mysql.MySqlDefaultValueConverter; -import io.debezium.connector.mysql.MySqlValueConverters; -import io.debezium.ddl.parser.mysql.generated.MySqlParser.CurrentTimestampContext; -import io.debezium.ddl.parser.mysql.generated.MySqlParser.DefaultValueContext; +import io.debezium.ddl.parser.mysql.generated.MySqlParser; import io.debezium.ddl.parser.mysql.generated.MySqlParserBaseListener; import io.debezium.relational.ColumnEditor; import java.util.concurrent.atomic.AtomicReference; /** - * Copied from Debezium project(v1.6.4.Final) to fix + * Copied from Debezium project(v1.9.7.Final) to fix * https://github.com/ververica/flink-cdc-connectors/issues/1506. * - *

Line 66~77: use the actual default string value when the sql contains COLLATE. We should + *

Line 48~59: use the actual default string value when the sql contains COLLATE. We should * remove this class after we bumped a higher debezium version where the * https://issues.redhat.com/browse/DBZ-5587 has been fixed. */ @@ -28,32 +25,17 @@ public class DefaultValueParserListener extends MySqlParserBaseListener { private final ColumnEditor columnEditor; private final AtomicReference optionalColumn; - private final MySqlDefaultValueConverter defaultValueConverter; - - /** - * Whether to convert the column's default value into the corresponding schema type or not. This - * is done for column definitions of ALTER TABLE statements but not for CREATE TABLE. In case of - * the latter, the default value conversion is handled by the CREATE TABLE statement listener - * itself, as a default character set given at the table level might have to be applied. - */ - private final boolean convertDefault; - private boolean converted; public DefaultValueParserListener( - ColumnEditor columnEditor, - MySqlValueConverters converters, - AtomicReference optionalColumn, - boolean convertDefault) { + ColumnEditor columnEditor, AtomicReference optionalColumn) { this.columnEditor = columnEditor; - this.defaultValueConverter = new MySqlDefaultValueConverter(converters); this.optionalColumn = optionalColumn; - this.convertDefault = convertDefault; this.converted = false; } @Override - public void enterDefaultValue(DefaultValueContext ctx) { + public void enterDefaultValue(MySqlParser.DefaultValueContext ctx) { String sign = ""; if (ctx.NULL_LITERAL() != null) { return; @@ -64,10 +46,10 @@ public void enterDefaultValue(DefaultValueContext ctx) { if (ctx.constant() != null) { if (ctx.constant().stringLiteral() != null) { if (ctx.constant().stringLiteral().COLLATE() == null) { - columnEditor.defaultValue( + columnEditor.defaultValueExpression( sign + unquote(ctx.constant().stringLiteral().getText())); } else { - columnEditor.defaultValue( + columnEditor.defaultValueExpression( sign + unquote( ctx.constant() @@ -76,47 +58,42 @@ public void enterDefaultValue(DefaultValueContext ctx) { .getText())); } } else if (ctx.constant().decimalLiteral() != null) { - columnEditor.defaultValue(sign + ctx.constant().decimalLiteral().getText()); + columnEditor.defaultValueExpression( + sign + ctx.constant().decimalLiteral().getText()); } else if (ctx.constant().BIT_STRING() != null) { - columnEditor.defaultValue(unquoteBinary(ctx.constant().BIT_STRING().getText())); + columnEditor.defaultValueExpression( + unquoteBinary(ctx.constant().BIT_STRING().getText())); } else if (ctx.constant().booleanLiteral() != null) { - columnEditor.defaultValue(ctx.constant().booleanLiteral().getText()); + columnEditor.defaultValueExpression(ctx.constant().booleanLiteral().getText()); } else if (ctx.constant().REAL_LITERAL() != null) { - columnEditor.defaultValue(ctx.constant().REAL_LITERAL().getText()); + columnEditor.defaultValueExpression(ctx.constant().REAL_LITERAL().getText()); } } else if (ctx.currentTimestamp() != null && !ctx.currentTimestamp().isEmpty()) { if (ctx.currentTimestamp().size() > 1 || (ctx.ON() == null && ctx.UPDATE() == null)) { - final CurrentTimestampContext currentTimestamp = ctx.currentTimestamp(0); + final MySqlParser.CurrentTimestampContext currentTimestamp = + ctx.currentTimestamp(0); if (currentTimestamp.CURRENT_TIMESTAMP() != null || currentTimestamp.NOW() != null) { - columnEditor.defaultValue("1970-01-01 00:00:00"); + columnEditor.defaultValueExpression("1970-01-01 00:00:00"); } else { - columnEditor.defaultValue(currentTimestamp.getText()); + columnEditor.defaultValueExpression(currentTimestamp.getText()); } } } - convertDefaultValue(true); + exitDefaultValue(true); super.enterDefaultValue(ctx); } - public void convertDefaultValue(boolean skipIfUnknownOptional) { - // For CREATE TABLE are all column default values converted only after charset is known. - if (convertDefault) { - if (!converted && (optionalColumn.get() != null || !skipIfUnknownOptional)) { - convertDefaultValueToSchemaType(columnEditor); - converted = true; + public void exitDefaultValue(boolean skipIfUnknownOptional) { + boolean isOptionalColumn = optionalColumn.get() != null; + if (!converted && (isOptionalColumn || !skipIfUnknownOptional)) { + if (isOptionalColumn) { + columnEditor.optional(optionalColumn.get().booleanValue()); } + converted = true; } } - private void convertDefaultValueToSchemaType(ColumnEditor columnEditor) { - if (optionalColumn.get() != null) { - columnEditor.optional(optionalColumn.get().booleanValue()); - } - - defaultValueConverter.setColumnDefaultValue(columnEditor); - } - private String unquote(String stringLiteral) { return stringLiteral.substring(1, stringLiteral.length() - 1); } diff --git a/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/LegacyMySqlSourceTest.java b/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/LegacyMySqlSourceTest.java index 7ca528311d..8d55b2d304 100644 --- a/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/LegacyMySqlSourceTest.java +++ b/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/LegacyMySqlSourceTest.java @@ -805,6 +805,7 @@ public void go() throws Exception { if (useLegacyImplementation) { // should fail because user specifies to use the legacy implementation try { + source.close(); runThread.sync(); fail("Should fail."); } catch (Exception e) { @@ -906,6 +907,9 @@ public void go() throws Exception { "Retrieve schema history failed, the schema records for engine %s has been removed," + " this might because the debezium engine has been shutdown due to other errors.", engineInstanceName))); + } finally { + source.close(); + runThread.sync(); } } } @@ -1116,6 +1120,11 @@ private static class MockedTable implements Table { private MockedTable() {} + @Override + public String comment() { + return ""; + } + @Override public TableId id() { return TableId.parse("Test"); diff --git a/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/debezium/reader/BinlogSplitReaderTest.java b/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/debezium/reader/BinlogSplitReaderTest.java index e6eeecf566..5705a87d30 100644 --- a/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/debezium/reader/BinlogSplitReaderTest.java +++ b/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/debezium/reader/BinlogSplitReaderTest.java @@ -47,6 +47,7 @@ import io.debezium.connector.mysql.MySqlConnection; import io.debezium.connector.mysql.MySqlConnectorConfig; import io.debezium.connector.mysql.MySqlOffsetContext; +import io.debezium.connector.mysql.MySqlPartition; import io.debezium.jdbc.JdbcConnection; import io.debezium.pipeline.spi.OffsetContext; import io.debezium.relational.TableId; @@ -396,6 +397,9 @@ public void testReadBinlogFromEarliestOffset() throws Exception { "+I[2003, user_24, Shanghai, 123567891234]" }; List actual = readBinlogSplits(dataType, reader, expected.length); + + reader.close(); + assertEqualsInOrder(Arrays.asList(expected), actual); } @@ -428,6 +432,9 @@ public void testReadBinlogFromEarliestOffsetAfterSchemaChange() throws Exception assertThrows(Throwable.class, () -> readBinlogSplits(dataType, reader, 1)); Optional schemaOutOfSyncException = ExceptionUtils.findThrowable(throwable, SchemaOutOfSyncException.class); + + reader.close(); + assertTrue(schemaOutOfSyncException.isPresent()); assertEquals( "Internal schema representation is probably out of sync with real database schema. " @@ -485,6 +492,9 @@ public void testReadBinlogFromBinlogFilePosition() throws Exception { "+I[2003, user_24, Shanghai, 123567891234]" }; List actual = readBinlogSplits(dataType, reader, expected.length); + + reader.close(); + assertEqualsInOrder(Arrays.asList(expected), actual); } @@ -538,6 +548,9 @@ public void testSkippingEvents() throws Exception { "+U[109, user_4, Pittsburgh, 123567891234]" }; List actual = readBinlogSplits(dataType, reader, expected.length); + + reader.close(); + assertEqualsInOrder(Arrays.asList(expected), actual); } @@ -589,6 +602,9 @@ public void testReadBinlogFromGtidSet() throws Exception { "+I[2003, user_24, Shanghai, 123567891234]" }; List actual = readBinlogSplits(dataType, reader, expected.length); + + reader.close(); + assertEqualsInOrder(Arrays.asList(expected), actual); } @@ -642,6 +658,9 @@ public void testReadBinlogFromTimestamp() throws Exception { "+I[2003, user_24, Shanghai, 123567891234]" }; List actual = readBinlogSplits(dataType, reader, expected.length); + + reader.close(); + assertEqualsInOrder(Arrays.asList(expected), actual); } @@ -700,6 +719,9 @@ public void testReadBinlogFromTimestampAfterSchemaChange() throws Exception { "+U[103, user_3, Shanghai, 123567891234, 15213]", }; List actual = readBinlogSplits(dataType, reader, expected.length); + + reader.close(); + assertEqualsInOrder(Arrays.asList(expected), actual); } @@ -751,6 +773,7 @@ public void testHeartbeatEvent() throws Exception { }, DEFAULT_TIMEOUT, "Timeout waiting for heartbeat event"); + binlogReader.close(); } @Test @@ -829,7 +852,11 @@ private MySqlBinlogSplit createBinlogSplit(MySqlSourceConfig sourceConfig) throw binlogSplitAssigner.open(); try (MySqlConnection jdbc = DebeziumUtils.createMySqlConnection(sourceConfig)) { Map tableSchemas = - TableDiscoveryUtils.discoverSchemaForCapturedTables(sourceConfig, jdbc); + TableDiscoveryUtils.discoverSchemaForCapturedTables( + new MySqlPartition( + sourceConfig.getMySqlConnectorConfig().getLogicalName()), + sourceConfig, + jdbc); return MySqlBinlogSplit.fillTableSchemas( binlogSplitAssigner.getNext().get().asBinlogSplit(), tableSchemas); } diff --git a/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/debezium/reader/SnapshotSplitReaderTest.java b/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/debezium/reader/SnapshotSplitReaderTest.java index 2de61520b8..3a06e7eabc 100644 --- a/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/debezium/reader/SnapshotSplitReaderTest.java +++ b/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/debezium/reader/SnapshotSplitReaderTest.java @@ -33,6 +33,7 @@ import com.ververica.cdc.connectors.mysql.testutils.RecordsFormatter; import com.ververica.cdc.connectors.mysql.testutils.UniqueDatabase; import io.debezium.connector.mysql.MySqlConnection; +import io.debezium.connector.mysql.MySqlPartition; import io.debezium.data.Envelope; import io.debezium.jdbc.JdbcConnection; import io.debezium.pipeline.EventDispatcher; @@ -523,8 +524,7 @@ public static MySqlSourceConfig getConfig( } private boolean executeSql(MySqlSourceConfig sourceConfig, String[] sqlStatements) { - JdbcConnection connection = DebeziumUtils.openJdbcConnection(sourceConfig); - try { + try (JdbcConnection connection = DebeziumUtils.openJdbcConnection(sourceConfig)) { connection.setAutoCommit(false); connection.execute(sqlStatements); connection.commit(); @@ -535,7 +535,7 @@ private boolean executeSql(MySqlSourceConfig sourceConfig, String[] sqlStatement return true; } - class MakeBinlogEventTaskContext extends StatefulTaskContext { + static class MakeBinlogEventTaskContext extends StatefulTaskContext { private final Supplier makeBinlogFunction; @@ -549,12 +549,14 @@ public MakeBinlogEventTaskContext( } @Override - public EventDispatcher.SnapshotReceiver getSnapshotReceiver() { - EventDispatcher.SnapshotReceiver snapshotReceiver = super.getSnapshotReceiver(); - return new EventDispatcher.SnapshotReceiver() { + public EventDispatcher.SnapshotReceiver getSnapshotReceiver() { + EventDispatcher.SnapshotReceiver snapshotReceiver = + super.getSnapshotReceiver(); + return new EventDispatcher.SnapshotReceiver() { @Override public void changeRecord( + MySqlPartition partition, DataCollectionSchema schema, Envelope.Operation operation, Object key, @@ -562,7 +564,8 @@ public void changeRecord( OffsetContext offset, ConnectHeaders headers) throws InterruptedException { - snapshotReceiver.changeRecord(schema, operation, key, value, offset, headers); + snapshotReceiver.changeRecord( + partition, schema, operation, key, value, offset, headers); } @Override diff --git a/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/source/assigners/state/PendingSplitsStateSerializerTest.java b/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/source/assigners/state/PendingSplitsStateSerializerTest.java index 5acc4e185c..13b6027a48 100644 --- a/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/source/assigners/state/PendingSplitsStateSerializerTest.java +++ b/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/source/assigners/state/PendingSplitsStateSerializerTest.java @@ -255,6 +255,11 @@ public String defaultCharsetName() { return "UTF-8"; } + @Override + public String comment() { + return ""; + } + @Override public TableEditor edit() { throw new UnsupportedOperationException("Not implemented."); diff --git a/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/source/reader/MySqlRecordEmitterTest.java b/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/source/reader/MySqlRecordEmitterTest.java index 8e87b1af88..e647a2d6bb 100644 --- a/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/source/reader/MySqlRecordEmitterTest.java +++ b/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/source/reader/MySqlRecordEmitterTest.java @@ -27,13 +27,21 @@ import com.ververica.cdc.connectors.mysql.source.split.MySqlBinlogSplitState; import com.ververica.cdc.connectors.mysql.source.split.SourceRecords; import com.ververica.cdc.debezium.DebeziumDeserializationSchema; +import io.debezium.config.Configuration; +import io.debezium.connector.mysql.MySqlConnectorConfig; import io.debezium.heartbeat.Heartbeat; +import io.debezium.heartbeat.HeartbeatFactory; +import io.debezium.jdbc.JdbcConfiguration; +import io.debezium.relational.TableId; +import io.debezium.schema.TopicSelector; +import io.debezium.util.SchemaNameAdjuster; import org.apache.kafka.connect.source.SourceRecord; import org.junit.Test; -import java.time.Duration; import java.util.Collections; +import static io.debezium.config.CommonConnectorConfig.TRANSACTION_TOPIC; +import static io.debezium.connector.mysql.MySqlConnectorConfig.SERVER_NAME; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; @@ -42,7 +50,21 @@ public class MySqlRecordEmitterTest { @Test public void testHeartbeatEventHandling() throws Exception { - Heartbeat heartbeat = Heartbeat.create(Duration.ofMillis(100), "fake-topic", "fake-key"); + Configuration dezConf = + JdbcConfiguration.create() + .with(Heartbeat.HEARTBEAT_INTERVAL, 100) + .with(TRANSACTION_TOPIC, "fake-topic") + .with(SERVER_NAME, "mysql_binlog_source") + .build(); + + MySqlConnectorConfig mySqlConfig = new MySqlConnectorConfig(dezConf); + HeartbeatFactory heartbeatFactory = + new HeartbeatFactory<>( + new MySqlConnectorConfig(dezConf), + TopicSelector.defaultSelector( + mySqlConfig, (id, prefix, delimiter) -> "fake-topic"), + SchemaNameAdjuster.create()); + Heartbeat heartbeat = heartbeatFactory.createHeartbeat(); BinlogOffset fakeOffset = BinlogOffset.ofBinlogFilePosition("fake-file", 15213L); MySqlRecordEmitter recordEmitter = createRecordEmitter(); MySqlBinlogSplitState splitState = createBinlogSplitState(); diff --git a/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/source/reader/MySqlSourceReaderTest.java b/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/source/reader/MySqlSourceReaderTest.java index 5137883ebb..bb7570a7d5 100644 --- a/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/source/reader/MySqlSourceReaderTest.java +++ b/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/source/reader/MySqlSourceReaderTest.java @@ -53,6 +53,7 @@ import com.ververica.cdc.debezium.DebeziumDeserializationSchema; import com.ververica.cdc.debezium.history.FlinkJsonTableChangeSerializer; import io.debezium.connector.mysql.MySqlConnection; +import io.debezium.connector.mysql.MySqlPartition; import io.debezium.document.Array; import io.debezium.jdbc.JdbcConnection; import io.debezium.relational.TableId; @@ -119,7 +120,11 @@ public void testBinlogReadFailoverCrossTransaction() throws Exception { MySqlSplit binlogSplit; try (MySqlConnection jdbc = DebeziumUtils.createMySqlConnection(sourceConfig)) { Map tableSchemas = - TableDiscoveryUtils.discoverSchemaForCapturedTables(sourceConfig, jdbc); + TableDiscoveryUtils.discoverSchemaForCapturedTables( + new MySqlPartition( + sourceConfig.getMySqlConnectorConfig().getLogicalName()), + sourceConfig, + jdbc); binlogSplit = MySqlBinlogSplit.fillTableSchemas( createBinlogSplit(sourceConfig).asBinlogSplit(), tableSchemas); diff --git a/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/table/MySqlConnectorITCase.java b/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/table/MySqlConnectorITCase.java index a9f934574a..ab107ed89c 100644 --- a/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/table/MySqlConnectorITCase.java +++ b/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/table/MySqlConnectorITCase.java @@ -1262,6 +1262,7 @@ public void testReadingWithDotTableName() throws Exception { + " 'database-name' = '%s'," + " 'table-name' = '%s'," + " 'scan.incremental.snapshot.enabled' = '%s'," + + " 'server-time-zone' = 'UTC'," + " 'server-id' = '%s'," + " 'scan.incremental.snapshot.chunk.size' = '%s'" + ")", @@ -1971,6 +1972,7 @@ public void testReadingWithMultiMaxValue() throws Exception { + " 'database-name' = '%s'," + " 'table-name' = '%s'," + " 'server-id' = '%s'," + + " 'server-time-zone' = 'UTC'," + " 'scan.incremental.snapshot.chunk.size' = '%s'" + ")", MYSQL_CONTAINER.getHost(), diff --git a/flink-connector-oceanbase-cdc/src/main/java/com/ververica/cdc/connectors/oceanbase/source/OceanBaseConnection.java b/flink-connector-oceanbase-cdc/src/main/java/com/ververica/cdc/connectors/oceanbase/source/OceanBaseConnection.java index c0c179656c..913c3a0291 100644 --- a/flink-connector-oceanbase-cdc/src/main/java/com/ververica/cdc/connectors/oceanbase/source/OceanBaseConnection.java +++ b/flink-connector-oceanbase-cdc/src/main/java/com/ververica/cdc/connectors/oceanbase/source/OceanBaseConnection.java @@ -18,7 +18,7 @@ import org.apache.flink.util.FlinkRuntimeException; -import io.debezium.config.Configuration; +import io.debezium.jdbc.JdbcConfiguration; import io.debezium.jdbc.JdbcConnection; import java.sql.DatabaseMetaData; @@ -34,6 +34,7 @@ /** {@link JdbcConnection} extension to be used with OceanBase server. */ public class OceanBaseConnection extends JdbcConnection { + private static final String QUOTED_CHARACTER = "`"; private static final Properties DEFAULT_JDBC_PROPERTIES = initializeDefaultJdbcProperties(); private static final String MYSQL_URL_PATTERN = "jdbc:mysql://${hostname}:${port}/?connectTimeout=${connectTimeout}"; @@ -54,13 +55,15 @@ public OceanBaseConnection( ClassLoader classLoader) { super( config(hostname, port, user, password, timeout), - factory(jdbcDriver, jdbcProperties, classLoader)); + factory(jdbcDriver, jdbcProperties, classLoader), + QUOTED_CHARACTER, + QUOTED_CHARACTER); this.compatibleMode = compatibleMode; } - private static Configuration config( + private static JdbcConfiguration config( String hostname, Integer port, String user, String password, Duration timeout) { - return Configuration.create() + return JdbcConfiguration.create() .with("hostname", hostname) .with("port", port) .with("user", user) diff --git a/flink-connector-oracle-cdc/pom.xml b/flink-connector-oracle-cdc/pom.xml index 0e24731689..47217337c3 100644 --- a/flink-connector-oracle-cdc/pom.xml +++ b/flink-connector-oracle-cdc/pom.xml @@ -154,6 +154,11 @@ under the License. test + + org.apache.commons + commons-lang3 + 3.7 + diff --git a/flink-connector-oracle-cdc/src/main/java/com/ververica/cdc/connectors/oracle/OracleSource.java b/flink-connector-oracle-cdc/src/main/java/com/ververica/cdc/connectors/oracle/OracleSource.java index 183a5c8e96..c05f5cb6fb 100644 --- a/flink-connector-oracle-cdc/src/main/java/com/ververica/cdc/connectors/oracle/OracleSource.java +++ b/flink-connector-oracle-cdc/src/main/java/com/ververica/cdc/connectors/oracle/OracleSource.java @@ -165,6 +165,8 @@ public DebeziumSourceFunction build() { if (tableList != null) { props.setProperty("table.include.list", String.join(",", tableList)); } + // we need this in order not to lose any transaction during snapshot to streaming switch + props.setProperty("internal.log.mining.transaction.snapshot.boundary.mode", "all"); switch (startupOptions.startupMode) { case INITIAL: diff --git a/flink-connector-oracle-cdc/src/main/java/com/ververica/cdc/connectors/oracle/source/reader/fetch/OracleScanFetchTask.java b/flink-connector-oracle-cdc/src/main/java/com/ververica/cdc/connectors/oracle/source/reader/fetch/OracleScanFetchTask.java index 9905b5a12e..bc22eadeeb 100644 --- a/flink-connector-oracle-cdc/src/main/java/com/ververica/cdc/connectors/oracle/source/reader/fetch/OracleScanFetchTask.java +++ b/flink-connector-oracle-cdc/src/main/java/com/ververica/cdc/connectors/oracle/source/reader/fetch/OracleScanFetchTask.java @@ -29,7 +29,7 @@ import io.debezium.connector.oracle.OracleConnectorConfig; import io.debezium.connector.oracle.OracleDatabaseSchema; import io.debezium.connector.oracle.OracleOffsetContext; -import io.debezium.connector.oracle.OracleValueConverters; +import io.debezium.connector.oracle.OraclePartition; import io.debezium.connector.oracle.logminer.LogMinerOracleOffsetContextLoader; import io.debezium.heartbeat.Heartbeat; import io.debezium.pipeline.EventDispatcher; @@ -39,19 +39,14 @@ import io.debezium.pipeline.spi.ChangeRecordEmitter; import io.debezium.pipeline.spi.OffsetContext; import io.debezium.pipeline.spi.SnapshotResult; -import io.debezium.relational.Column; import io.debezium.relational.RelationalSnapshotChangeEventSource; import io.debezium.relational.SnapshotChangeRecordEmitter; import io.debezium.relational.Table; import io.debezium.relational.TableId; -import io.debezium.relational.ValueConverter; import io.debezium.util.Clock; import io.debezium.util.ColumnUtils; import io.debezium.util.Strings; import io.debezium.util.Threads; -import org.apache.kafka.connect.data.Field; -import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.data.SchemaBuilder; import org.apache.kafka.connect.errors.ConnectException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -86,6 +81,11 @@ public SnapshotSplit getSplit() { return split; } + @Override + public void close() { + taskRunning = false; + } + @Override public boolean isRunning() { return taskRunning; @@ -106,9 +106,11 @@ public void execute(Context context) throws Exception { split); SnapshotSplitChangeEventSourceContext changeEventSourceContext = new SnapshotSplitChangeEventSourceContext(); - SnapshotResult snapshotResult = + SnapshotResult snapshotResult = snapshotSplitReadTask.execute( - changeEventSourceContext, sourceFetchContext.getOffsetContext()); + changeEventSourceContext, + sourceFetchContext.getPartition(), + sourceFetchContext.getOffsetContext()); final StreamSplit backfillBinlogSplit = createBackfillRedoLogSplit(changeEventSourceContext); @@ -121,7 +123,7 @@ public void execute(Context context) throws Exception { if (!binlogBackfillRequired) { dispatchBinlogEndEvent( backfillBinlogSplit, - ((OracleSourceFetchTaskContext) context).getOffsetContext().getPartition(), + sourceFetchContext.getPartition().getSourcePartition(), ((OracleSourceFetchTaskContext) context).getDispatcher()); taskRunning = false; return; @@ -137,7 +139,9 @@ public void execute(Context context) throws Exception { final OracleOffsetContext oracleOffsetContext = loader.load(backfillBinlogSplit.getStartingOffset().getOffset()); backfillBinlogReadTask.execute( - new SnapshotBinlogSplitChangeEventSourceContext(), oracleOffsetContext); + new SnapshotBinlogSplitChangeEventSourceContext(), + sourceFetchContext.getPartition(), + oracleOffsetContext); taskRunning = false; } else { taskRunning = false; @@ -163,8 +167,6 @@ private RedoLogSplitReadTask createBackfillRedoLogReadTask( context.getSourceConfig().getDbzConnectorConfig(); final OffsetContext.Loader loader = new LogMinerOracleOffsetContextLoader(oracleConnectorConfig); - final OracleOffsetContext oracleOffsetContext = - loader.load(backfillBinlogSplit.getStartingOffset().getOffset()); // we should only capture events for the current table, // otherwise, we may can't find corresponding schema Configuration dezConf = @@ -190,7 +192,7 @@ private RedoLogSplitReadTask createBackfillRedoLogReadTask( private void dispatchBinlogEndEvent( StreamSplit backFillBinlogSplit, Map sourcePartition, - JdbcSourceEventDispatcher eventDispatcher) + JdbcSourceEventDispatcher eventDispatcher) throws InterruptedException { eventDispatcher.dispatchWatermarkEvent( sourcePartition, @@ -200,7 +202,8 @@ private void dispatchBinlogEndEvent( } /** A wrapped task to fetch snapshot split of table. */ - public static class OracleSnapshotSplitReadTask extends AbstractSnapshotChangeEventSource { + public static class OracleSnapshotSplitReadTask + extends AbstractSnapshotChangeEventSource { private static final Logger LOG = LoggerFactory.getLogger(OracleSnapshotSplitReadTask.class); @@ -211,19 +214,19 @@ public static class OracleSnapshotSplitReadTask extends AbstractSnapshotChangeEv private final OracleConnectorConfig connectorConfig; private final OracleDatabaseSchema databaseSchema; private final OracleConnection jdbcConnection; - private final JdbcSourceEventDispatcher dispatcher; + private final JdbcSourceEventDispatcher dispatcher; private final Clock clock; private final SnapshotSplit snapshotSplit; private final OracleOffsetContext offsetContext; - private final SnapshotProgressListener snapshotProgressListener; + private final SnapshotProgressListener snapshotProgressListener; public OracleSnapshotSplitReadTask( OracleConnectorConfig connectorConfig, OracleOffsetContext previousOffset, - SnapshotProgressListener snapshotProgressListener, + SnapshotProgressListener snapshotProgressListener, OracleDatabaseSchema databaseSchema, OracleConnection jdbcConnection, - JdbcSourceEventDispatcher dispatcher, + JdbcSourceEventDispatcher dispatcher, SnapshotSplit snapshotSplit) { super(connectorConfig, snapshotProgressListener); this.offsetContext = previousOffset; @@ -237,13 +240,15 @@ public OracleSnapshotSplitReadTask( } @Override - public SnapshotResult execute( - ChangeEventSourceContext context, OffsetContext previousOffset) + public SnapshotResult execute( + ChangeEventSourceContext context, + OraclePartition partition, + OracleOffsetContext previousOffset) throws InterruptedException { - SnapshottingTask snapshottingTask = getSnapshottingTask(previousOffset); - final SnapshotContext ctx; + SnapshottingTask snapshottingTask = getSnapshottingTask(partition, previousOffset); + final SnapshotContext ctx; try { - ctx = prepare(context); + ctx = prepare(partition); } catch (Exception e) { LOG.error("Failed to initialize snapshot context.", e); throw new RuntimeException(e); @@ -259,14 +264,13 @@ public SnapshotResult execute( } @Override - protected SnapshotResult doExecute( + protected SnapshotResult doExecute( ChangeEventSourceContext context, - OffsetContext previousOffset, + OracleOffsetContext previousOffset, SnapshotContext snapshotContext, SnapshottingTask snapshottingTask) throws Exception { - final RelationalSnapshotChangeEventSource.RelationalSnapshotContext ctx = - (RelationalSnapshotChangeEventSource.RelationalSnapshotContext) snapshotContext; + final OracleSnapshotContext ctx = (OracleSnapshotContext) snapshotContext; ctx.offset = offsetContext; final RedoLogOffset lowWatermark = currentRedoLogOffset(jdbcConnection); @@ -276,7 +280,10 @@ protected SnapshotResult doExecute( snapshotSplit); ((SnapshotSplitChangeEventSourceContext) (context)).setLowWatermark(lowWatermark); dispatcher.dispatchWatermarkEvent( - offsetContext.getPartition(), snapshotSplit, lowWatermark, WatermarkKind.LOW); + snapshotContext.partition.getSourcePartition(), + snapshotSplit, + lowWatermark, + WatermarkKind.LOW); LOG.info("Snapshot step 2 - Snapshotting data"); createDataEvents(ctx, snapshotSplit.getTableId()); @@ -288,34 +295,37 @@ protected SnapshotResult doExecute( snapshotSplit); ((SnapshotSplitChangeEventSourceContext) (context)).setHighWatermark(highWatermark); dispatcher.dispatchWatermarkEvent( - offsetContext.getPartition(), snapshotSplit, highWatermark, WatermarkKind.HIGH); + snapshotContext.partition.getSourcePartition(), + snapshotSplit, + highWatermark, + WatermarkKind.HIGH); return SnapshotResult.completed(ctx.offset); } @Override - protected SnapshottingTask getSnapshottingTask(OffsetContext previousOffset) { + protected SnapshottingTask getSnapshottingTask( + OraclePartition partition, OracleOffsetContext previousOffset) { return new SnapshottingTask(false, true); } @Override - protected SnapshotContext prepare(ChangeEventSourceContext changeEventSourceContext) - throws Exception { - return new MySqlSnapshotContext(); + protected SnapshotContext prepare( + OraclePartition partition) throws Exception { + return new OracleSnapshotContext(partition); } - private static class MySqlSnapshotContext - extends RelationalSnapshotChangeEventSource.RelationalSnapshotContext { + private static class OracleSnapshotContext + extends RelationalSnapshotChangeEventSource.RelationalSnapshotContext< + OraclePartition, OracleOffsetContext> { - public MySqlSnapshotContext() throws SQLException { - super(""); + public OracleSnapshotContext(OraclePartition partition) throws SQLException { + super(partition, ""); } } - private void createDataEvents( - RelationalSnapshotChangeEventSource.RelationalSnapshotContext snapshotContext, - TableId tableId) + private void createDataEvents(OracleSnapshotContext snapshotContext, TableId tableId) throws Exception { - EventDispatcher.SnapshotReceiver snapshotReceiver = + EventDispatcher.SnapshotReceiver snapshotReceiver = dispatcher.getSnapshotChangeEventReceiver(); LOG.debug("Snapshotting table {}", tableId); createDataEventsForTable( @@ -325,8 +335,8 @@ private void createDataEvents( /** Dispatches the data change events for the records of a single table. */ private void createDataEventsForTable( - RelationalSnapshotChangeEventSource.RelationalSnapshotContext snapshotContext, - EventDispatcher.SnapshotReceiver snapshotReceiver, + OracleSnapshotContext snapshotContext, + EventDispatcher.SnapshotReceiver snapshotReceiver, Table table) throws InterruptedException { @@ -366,12 +376,8 @@ private void createDataEventsForTable( 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, table); - } + final Object[] row = + jdbcConnection.rowToArray(table, databaseSchema, rs, columnArray); if (logTimer.expired()) { long stop = clock.currentTimeInMillis(); LOG.info( @@ -379,10 +385,12 @@ private void createDataEventsForTable( rows, snapshotSplit.splitId(), Strings.duration(stop - exportStart)); - snapshotProgressListener.rowsScanned(table.id(), rows); + snapshotProgressListener.rowsScanned( + snapshotContext.partition, table.id(), rows); logTimer = getTableScanLogTimer(); } dispatcher.dispatchSnapshotEvent( + snapshotContext.partition, table.id(), getChangeRecordEmitter(snapshotContext, table.id(), row), snapshotReceiver); @@ -397,43 +405,25 @@ private void createDataEventsForTable( } } - protected ChangeRecordEmitter getChangeRecordEmitter( - SnapshotContext snapshotContext, TableId tableId, Object[] row) { + protected ChangeRecordEmitter getChangeRecordEmitter( + SnapshotContext snapshotContext, + TableId tableId, + Object[] row) { snapshotContext.offset.event(tableId, clock.currentTime()); - return new SnapshotChangeRecordEmitter(snapshotContext.offset, row, clock); + return new SnapshotChangeRecordEmitter<>( + snapshotContext.partition, snapshotContext.offset, row, clock); } private Threads.Timer getTableScanLogTimer() { return Threads.timer(clock, LOG_INTERVAL); } - - /** - * copied from - * io.debezium.connector.oracle.antlr.listener.ParserUtils#convertValueToSchemaType. - */ - private Object readField(ResultSet rs, int fieldNo, Column actualColumn, Table actualTable) - throws SQLException { - - OracleValueConverters oracleValueConverters = - new OracleValueConverters(connectorConfig, jdbcConnection); - - final SchemaBuilder schemaBuilder = oracleValueConverters.schemaBuilder(actualColumn); - if (schemaBuilder == null) { - return null; - } - Schema schema = schemaBuilder.build(); - Field field = new Field(actualColumn.name(), 1, schema); - final ValueConverter valueConverter = - oracleValueConverters.converter(actualColumn, field); - return valueConverter.convert(rs.getObject(fieldNo)); - } } /** * {@link ChangeEventSource.ChangeEventSourceContext} implementation that keeps low/high * watermark for each {@link SnapshotSplit}. */ - public class SnapshotSplitChangeEventSourceContext + public static class SnapshotSplitChangeEventSourceContext implements ChangeEventSource.ChangeEventSourceContext { private RedoLogOffset lowWatermark; diff --git a/flink-connector-oracle-cdc/src/main/java/com/ververica/cdc/connectors/oracle/source/reader/fetch/OracleSourceFetchTaskContext.java b/flink-connector-oracle-cdc/src/main/java/com/ververica/cdc/connectors/oracle/source/reader/fetch/OracleSourceFetchTaskContext.java index 6cc097922c..55731c6e95 100644 --- a/flink-connector-oracle-cdc/src/main/java/com/ververica/cdc/connectors/oracle/source/reader/fetch/OracleSourceFetchTaskContext.java +++ b/flink-connector-oracle-cdc/src/main/java/com/ververica/cdc/connectors/oracle/source/reader/fetch/OracleSourceFetchTaskContext.java @@ -37,6 +37,7 @@ import io.debezium.connector.oracle.OracleDatabaseSchema; import io.debezium.connector.oracle.OracleErrorHandler; import io.debezium.connector.oracle.OracleOffsetContext; +import io.debezium.connector.oracle.OraclePartition; import io.debezium.connector.oracle.OracleStreamingChangeEventSourceMetrics; import io.debezium.connector.oracle.OracleTaskContext; import io.debezium.connector.oracle.OracleTopicSelector; @@ -48,6 +49,7 @@ import io.debezium.pipeline.metrics.SnapshotChangeEventSourceMetrics; import io.debezium.pipeline.source.spi.EventMetadataProvider; import io.debezium.pipeline.spi.OffsetContext; +import io.debezium.pipeline.spi.Offsets; import io.debezium.relational.Table; import io.debezium.relational.TableId; import io.debezium.relational.Tables; @@ -78,10 +80,12 @@ public class OracleSourceFetchTaskContext extends JdbcSourceFetchTaskContext { private OracleDatabaseSchema databaseSchema; private OracleTaskContext taskContext; private OracleOffsetContext offsetContext; - private SnapshotChangeEventSourceMetrics snapshotChangeEventSourceMetrics; + private OraclePartition partition; + + private SnapshotChangeEventSourceMetrics snapshotChangeEventSourceMetrics; private OracleStreamingChangeEventSourceMetrics streamingChangeEventSourceMetrics; private TopicSelector topicSelector; - private JdbcSourceEventDispatcher dispatcher; + private JdbcSourceEventDispatcher dispatcher; private ChangeEventQueue queue; private OracleErrorHandler errorHandler; @@ -109,12 +113,13 @@ public void configure(SourceSplitBase sourceSplitBase) { this.offsetContext = loadStartingOffsetState( new LogMinerOracleOffsetContextLoader(connectorConfig), sourceSplitBase); + this.partition = new OraclePartition(connectorConfig.getLogicalName()); validateAndLoadDatabaseHistory(offsetContext, databaseSchema); this.taskContext = new OracleTaskContext(connectorConfig, databaseSchema); final int queueSize = sourceSplitBase.isSnapshotSplit() - ? Integer.MAX_VALUE + ? getSourceConfig().getSplitSize() : getSourceConfig().getDbzConnectorConfig().getMaxQueueSize(); this.queue = new ChangeEventQueue.Builder() @@ -130,7 +135,7 @@ public void configure(SourceSplitBase sourceSplitBase) { // .buffering() .build(); this.dispatcher = - new JdbcSourceEventDispatcher( + new JdbcSourceEventDispatcher<>( connectorConfig, topicSelector, databaseSchema, @@ -151,7 +156,7 @@ public void configure(SourceSplitBase sourceSplitBase) { (OracleStreamingChangeEventSourceMetrics) changeEventSourceMetricsFactory.getStreamingMetrics( taskContext, queue, metadataProvider); - this.errorHandler = new OracleErrorHandler(connectorConfig.getLogicalName(), queue); + this.errorHandler = new OracleErrorHandler(connectorConfig, queue); } @Override @@ -173,7 +178,7 @@ public OracleOffsetContext getOffsetContext() { return offsetContext; } - public SnapshotChangeEventSourceMetrics getSnapshotChangeEventSourceMetrics() { + public SnapshotChangeEventSourceMetrics getSnapshotChangeEventSourceMetrics() { return snapshotChangeEventSourceMetrics; } @@ -233,7 +238,7 @@ public TableId getTableId(SourceRecord record) { } @Override - public JdbcSourceEventDispatcher getDispatcher() { + public JdbcSourceEventDispatcher getDispatcher() { return dispatcher; } @@ -242,6 +247,10 @@ public ChangeEventQueue getQueue() { return queue; } + public OraclePartition getPartition() { + return partition; + } + @Override public Tables.TableFilter getTableFilter() { return getDbzConnectorConfig().getTableFilters().dataCollectionFilter(); @@ -252,24 +261,26 @@ public Offset getStreamOffset(SourceRecord sourceRecord) { return OracleUtils.getRedoLogPosition(sourceRecord); } + @Override + public void close() throws Exception { + connection.close(); + } + /** Loads the connector's persistent offset (if present) via the given loader. */ private OracleOffsetContext loadStartingOffsetState( - OffsetContext.Loader loader, SourceSplitBase oracleSplit) { + OffsetContext.Loader loader, SourceSplitBase oracleSplit) { Offset offset = oracleSplit.isSnapshotSplit() ? RedoLogOffset.INITIAL_OFFSET : oracleSplit.asStreamSplit().getStartingOffset(); - OracleOffsetContext oracleOffsetContext = - (OracleOffsetContext) loader.load(offset.getOffset()); - - return oracleOffsetContext; + return loader.load(offset.getOffset()); } private void validateAndLoadDatabaseHistory( OracleOffsetContext offset, OracleDatabaseSchema schema) { schema.initializeStorage(); - schema.recover(offset); + schema.recover(Offsets.of(partition, offset)); } /** Copied from debezium for accessing here. */ diff --git a/flink-connector-oracle-cdc/src/main/java/com/ververica/cdc/connectors/oracle/source/reader/fetch/OracleStreamFetchTask.java b/flink-connector-oracle-cdc/src/main/java/com/ververica/cdc/connectors/oracle/source/reader/fetch/OracleStreamFetchTask.java index 973690888a..237b786a28 100644 --- a/flink-connector-oracle-cdc/src/main/java/com/ververica/cdc/connectors/oracle/source/reader/fetch/OracleStreamFetchTask.java +++ b/flink-connector-oracle-cdc/src/main/java/com/ververica/cdc/connectors/oracle/source/reader/fetch/OracleStreamFetchTask.java @@ -28,6 +28,7 @@ import io.debezium.connector.oracle.OracleConnectorConfig; import io.debezium.connector.oracle.OracleDatabaseSchema; import io.debezium.connector.oracle.OracleOffsetContext; +import io.debezium.connector.oracle.OraclePartition; import io.debezium.connector.oracle.OracleStreamingChangeEventSourceMetrics; import io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource; import io.debezium.pipeline.ErrorHandler; @@ -69,7 +70,9 @@ public void execute(Context context) throws Exception { RedoLogSplitChangeEventSourceContext changeEventSourceContext = new RedoLogSplitChangeEventSourceContext(); redoLogSplitReadTask.execute( - changeEventSourceContext, sourceFetchContext.getOffsetContext()); + changeEventSourceContext, + sourceFetchContext.getPartition(), + sourceFetchContext.getOffsetContext()); } @Override @@ -82,6 +85,11 @@ public StreamSplit getSplit() { return split; } + @Override + public void close() { + taskRunning = false; + } + /** * A wrapped task to read all binlog for table and also supports read bounded (from lowWatermark * to highWatermark) binlog. @@ -90,14 +98,14 @@ public static class RedoLogSplitReadTask extends LogMinerStreamingChangeEventSou private static final Logger LOG = LoggerFactory.getLogger(RedoLogSplitReadTask.class); private final StreamSplit redoLogSplit; - private final JdbcSourceEventDispatcher dispatcher; + private final JdbcSourceEventDispatcher dispatcher; private final ErrorHandler errorHandler; private ChangeEventSourceContext context; public RedoLogSplitReadTask( OracleConnectorConfig connectorConfig, OracleConnection connection, - JdbcSourceEventDispatcher dispatcher, + JdbcSourceEventDispatcher dispatcher, ErrorHandler errorHandler, OracleDatabaseSchema schema, Configuration jdbcConfig, @@ -118,14 +126,18 @@ public RedoLogSplitReadTask( } @Override - public void execute(ChangeEventSourceContext context, OracleOffsetContext offsetContext) { + public void execute( + ChangeEventSourceContext context, + OraclePartition partition, + OracleOffsetContext offsetContext) { this.context = context; - super.execute(context, offsetContext); + super.execute(context, partition, offsetContext); } @Override - public void afterHandleScn(OracleOffsetContext offsetContext) { - super.afterHandleScn(offsetContext); + protected void afterHandleScn( + OraclePartition partition, OracleOffsetContext offsetContext) { + super.afterHandleScn(partition, offsetContext); // check do we need to stop for fetch binlog for snapshot split. if (isBoundedRead()) { final RedoLogOffset currentRedoLogOffset = @@ -135,7 +147,7 @@ public void afterHandleScn(OracleOffsetContext offsetContext) { // send binlog end event try { dispatcher.dispatchWatermarkEvent( - offsetContext.getPartition(), + partition.getSourcePartition(), redoLogSplit, currentRedoLogOffset, WatermarkKind.END); diff --git a/flink-connector-oracle-cdc/src/main/java/com/ververica/cdc/connectors/oracle/source/utils/OracleConnectionUtils.java b/flink-connector-oracle-cdc/src/main/java/com/ververica/cdc/connectors/oracle/source/utils/OracleConnectionUtils.java index 7e926e9922..3f08d44816 100644 --- a/flink-connector-oracle-cdc/src/main/java/com/ververica/cdc/connectors/oracle/source/utils/OracleConnectionUtils.java +++ b/flink-connector-oracle-cdc/src/main/java/com/ververica/cdc/connectors/oracle/source/utils/OracleConnectionUtils.java @@ -22,6 +22,7 @@ import io.debezium.config.Configuration; import io.debezium.connector.oracle.OracleConnection; import io.debezium.connector.oracle.Scn; +import io.debezium.jdbc.JdbcConfiguration; import io.debezium.jdbc.JdbcConnection; import io.debezium.relational.RelationalTableFilters; import io.debezium.relational.TableId; @@ -48,10 +49,15 @@ public class OracleConnectionUtils { private static final String SHOW_CURRENT_SCN = "SELECT CURRENT_SCN FROM V$DATABASE"; /** Creates a new {@link OracleConnection}, but not open the connection. */ - public static OracleConnection createOracleConnection(Configuration dbzConfiguration) { + public static OracleConnection createOracleConnection(Configuration configuration) { + return createOracleConnection(JdbcConfiguration.adapt(configuration)); + } + + /** Creates a new {@link OracleConnection}, but not open the connection. */ + public static OracleConnection createOracleConnection(JdbcConfiguration dbzConfiguration) { Configuration configuration = dbzConfiguration.subset(DATABASE_CONFIG_PREFIX, true); return new OracleConnection( - configuration.isEmpty() ? dbzConfiguration : configuration, + configuration.isEmpty() ? dbzConfiguration : JdbcConfiguration.adapt(configuration), OracleConnectionUtils.class::getClassLoader); } diff --git a/flink-connector-oracle-cdc/src/main/java/com/ververica/cdc/connectors/oracle/source/utils/OracleUtils.java b/flink-connector-oracle-cdc/src/main/java/com/ververica/cdc/connectors/oracle/source/utils/OracleUtils.java index 56c1cc5c98..0430e57e75 100644 --- a/flink-connector-oracle-cdc/src/main/java/com/ververica/cdc/connectors/oracle/source/utils/OracleUtils.java +++ b/flink-connector-oracle-cdc/src/main/java/com/ververica/cdc/connectors/oracle/source/utils/OracleUtils.java @@ -23,9 +23,11 @@ import io.debezium.connector.oracle.OracleConnection; import io.debezium.connector.oracle.OracleConnectorConfig; import io.debezium.connector.oracle.OracleDatabaseSchema; +import io.debezium.connector.oracle.OracleDefaultValueConverter; import io.debezium.connector.oracle.OracleTopicSelector; import io.debezium.connector.oracle.OracleValueConverters; import io.debezium.connector.oracle.StreamingAdapter; +import io.debezium.jdbc.JdbcConfiguration; import io.debezium.jdbc.JdbcConnection; import io.debezium.relational.TableId; import io.debezium.schema.TopicSelector; @@ -249,11 +251,14 @@ public static OracleDatabaseSchema createOracleDatabaseSchema( // OracleConnectionUtils.createOracleConnection((Configuration) dbzOracleConfig); OracleValueConverters oracleValueConverters = new OracleValueConverters(dbzOracleConfig, oracleConnection); + OracleDefaultValueConverter defaultValueConverter = + new OracleDefaultValueConverter(oracleValueConverters, oracleConnection); StreamingAdapter.TableNameCaseSensitivity tableNameCaseSensitivity = dbzOracleConfig.getAdapter().getTableNameCaseSensitivity(oracleConnection); return new OracleDatabaseSchema( dbzOracleConfig, oracleValueConverters, + defaultValueConverter, schemaNameAdjuster, topicSelector, tableNameCaseSensitivity); @@ -265,9 +270,12 @@ public static OracleDatabaseSchema createOracleDatabaseSchema( TopicSelector topicSelector = OracleTopicSelector.defaultSelector(dbzOracleConfig); SchemaNameAdjuster schemaNameAdjuster = SchemaNameAdjuster.create(); OracleConnection oracleConnection = - OracleConnectionUtils.createOracleConnection((Configuration) dbzOracleConfig); + OracleConnectionUtils.createOracleConnection( + JdbcConfiguration.adapt((Configuration) dbzOracleConfig)); OracleValueConverters oracleValueConverters = new OracleValueConverters(dbzOracleConfig, oracleConnection); + OracleDefaultValueConverter defaultValueConverter = + new OracleDefaultValueConverter(oracleValueConverters, oracleConnection); StreamingAdapter.TableNameCaseSensitivity tableNameCaseSensitivity = tableIdCaseInsensitive ? StreamingAdapter.TableNameCaseSensitivity.SENSITIVE @@ -275,6 +283,7 @@ public static OracleDatabaseSchema createOracleDatabaseSchema( return new OracleDatabaseSchema( dbzOracleConfig, oracleValueConverters, + defaultValueConverter, schemaNameAdjuster, topicSelector, tableNameCaseSensitivity); diff --git a/flink-connector-oracle-cdc/src/main/java/io/debezium/connector/oracle/OracleErrorHandler.java b/flink-connector-oracle-cdc/src/main/java/io/debezium/connector/oracle/OracleErrorHandler.java deleted file mode 100644 index 01a696579f..0000000000 --- a/flink-connector-oracle-cdc/src/main/java/io/debezium/connector/oracle/OracleErrorHandler.java +++ /dev/null @@ -1,105 +0,0 @@ -/* - * Copyright 2022 Ververica Inc. - * - * Licensed 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 io.debezium.connector.oracle; - -import io.debezium.connector.base.ChangeEventQueue; -import io.debezium.pipeline.ErrorHandler; - -import java.io.IOException; -import java.sql.SQLRecoverableException; -import java.util.ArrayList; -import java.util.List; - -/** - * Copied from https://github.com/debezium/debezium project to fix - * https://issues.redhat.com/browse/DBZ-4536 for 1.6.4.Final version. - * - *

This file is override to fix logger mining session stopped due to 'No more data to read from - * socket' exception. please see more discussion under - * https://github.com/debezium/debezium/pull/3118, We should remove this class since we bumped - * higher debezium version after 1.8.1.Final where the issue has been fixed. - */ -public class OracleErrorHandler extends ErrorHandler { - - private static final List retryOracleErrors = new ArrayList<>(); - private static final List retryOracleMessageContainsTexts = new ArrayList<>(); - - static { - // Contents of this list should only be ORA-xxxxx errors - // The error check uses starts-with semantics - retryOracleErrors.add("ORA-03135"); // connection lost - retryOracleErrors.add("ORA-12543"); // TNS:destination host unreachable - retryOracleErrors.add("ORA-00604"); // error occurred at recursive SQL level 1 - retryOracleErrors.add("ORA-01089"); // Oracle immediate shutdown in progress - retryOracleErrors.add("ORA-01333"); // Failed to establish LogMiner dictionary - retryOracleErrors.add("ORA-01284"); // Redo/Archive log cannot be opened, likely locked - retryOracleErrors.add( - "ORA-26653"); // Apply DBZXOUT did not start properly and is currently in state - // INITIAL - retryOracleErrors.add("ORA-01291"); // missing logfile - retryOracleErrors.add( - "ORA-01327"); // failed to exclusively lock system dictionary as required BUILD - retryOracleErrors.add("ORA-04030"); // out of process memory - - // Contents of this list should be any type of error message text - // The error check uses case-insensitive contains semantics - retryOracleMessageContainsTexts.add("No more data to read from socket"); - } - - public OracleErrorHandler(String logicalName, ChangeEventQueue queue) { - super(OracleConnector.class, logicalName, queue); - } - - @Override - protected boolean isRetriable(Throwable throwable) { - while (throwable != null) { - // Always retry any recoverable error - if (throwable instanceof SQLRecoverableException) { - return true; - } - - // If message is provided, run checks against it - final String message = throwable.getMessage(); - if (message != null && message.length() > 0) { - // Check Oracle error codes - for (String errorCode : retryOracleErrors) { - if (message.startsWith(errorCode)) { - return true; - } - } - // Check Oracle error message texts - for (String messageText : retryOracleMessageContainsTexts) { - if (message.toUpperCase().contains(messageText.toUpperCase())) { - return true; - } - } - } - - if (throwable.getCause() != null) { - // We explicitly check this below the top-level error as we only want - // certain nested exceptions to be retried, not if they're at the top - final Throwable cause = throwable.getCause(); - if (cause instanceof IOException) { - return true; - } - } - - throwable = throwable.getCause(); - } - return false; - } -} diff --git a/flink-connector-oracle-cdc/src/main/java/io/debezium/connector/oracle/logminer/LogMinerAdapter.java b/flink-connector-oracle-cdc/src/main/java/io/debezium/connector/oracle/logminer/LogMinerAdapter.java new file mode 100644 index 0000000000..53387e3cdb --- /dev/null +++ b/flink-connector-oracle-cdc/src/main/java/io/debezium/connector/oracle/logminer/LogMinerAdapter.java @@ -0,0 +1,448 @@ +/* + * Copyright 2023 Ververica Inc. + * + * Licensed 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 io.debezium.connector.oracle.logminer; + +import io.debezium.DebeziumException; +import io.debezium.config.Configuration; +import io.debezium.connector.oracle.AbstractStreamingAdapter; +import io.debezium.connector.oracle.OracleConnection; +import io.debezium.connector.oracle.OracleConnectorConfig; +import io.debezium.connector.oracle.OracleConnectorConfig.TransactionSnapshotBoundaryMode; +import io.debezium.connector.oracle.OracleDatabaseSchema; +import io.debezium.connector.oracle.OracleOffsetContext; +import io.debezium.connector.oracle.OraclePartition; +import io.debezium.connector.oracle.OracleStreamingChangeEventSourceMetrics; +import io.debezium.connector.oracle.OracleTaskContext; +import io.debezium.connector.oracle.Scn; +import io.debezium.document.Document; +import io.debezium.pipeline.ErrorHandler; +import io.debezium.pipeline.EventDispatcher; +import io.debezium.pipeline.source.snapshot.incremental.SignalBasedIncrementalSnapshotContext; +import io.debezium.pipeline.source.spi.StreamingChangeEventSource; +import io.debezium.pipeline.spi.OffsetContext; +import io.debezium.pipeline.txmetadata.TransactionContext; +import io.debezium.relational.RelationalSnapshotChangeEventSource.RelationalSnapshotContext; +import io.debezium.relational.TableId; +import io.debezium.relational.history.HistoryRecordComparator; +import io.debezium.util.Clock; +import io.debezium.util.HexConverter; +import io.debezium.util.Strings; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; + +/** + * Copied from Debezium 1.9.7. + * + *

Line 356: Replace < condition with <= to be able to catch ongoing transactions during snapshot + * if current SCN points to START/INSERT/DELETE/UPDATE event. + */ +public class LogMinerAdapter extends AbstractStreamingAdapter { + + private static final Duration GET_TRANSACTION_SCN_PAUSE = Duration.ofSeconds(1); + + private static final int GET_TRANSACTION_SCN_ATTEMPTS = 5; + + private static final Logger LOGGER = LoggerFactory.getLogger(LogMinerAdapter.class); + + public static final String TYPE = "logminer"; + + public LogMinerAdapter(OracleConnectorConfig connectorConfig) { + super(connectorConfig); + } + + @Override + public String getType() { + return TYPE; + } + + @Override + public HistoryRecordComparator getHistoryRecordComparator() { + return new HistoryRecordComparator() { + @Override + protected boolean isPositionAtOrBefore(Document recorded, Document desired) { + return resolveScn(recorded).compareTo(resolveScn(desired)) < 1; + } + }; + } + + @Override + public OffsetContext.Loader getOffsetContextLoader() { + return new LogMinerOracleOffsetContextLoader(connectorConfig); + } + + @Override + public StreamingChangeEventSource getSource( + OracleConnection connection, + EventDispatcher dispatcher, + ErrorHandler errorHandler, + Clock clock, + OracleDatabaseSchema schema, + OracleTaskContext taskContext, + Configuration jdbcConfig, + OracleStreamingChangeEventSourceMetrics streamingMetrics) { + return new LogMinerStreamingChangeEventSource( + connectorConfig, + connection, + dispatcher, + errorHandler, + clock, + schema, + jdbcConfig, + streamingMetrics); + } + + @Override + public OracleOffsetContext determineSnapshotOffset( + RelationalSnapshotContext ctx, + OracleConnectorConfig connectorConfig, + OracleConnection connection) + throws SQLException { + + final Scn latestTableDdlScn = getLatestTableDdlScn(ctx, connection).orElse(null); + final String tableName = getTransactionTableName(connectorConfig); + + final Map pendingTransactions = new LinkedHashMap<>(); + + final Optional currentScn; + if (isPendingTransactionSkip(connectorConfig)) { + currentScn = getCurrentScn(latestTableDdlScn, connection); + } else { + currentScn = + getPendingTransactions( + latestTableDdlScn, connection, pendingTransactions, tableName); + } + + if (!currentScn.isPresent()) { + throw new DebeziumException("Failed to resolve current SCN"); + } + + // The provided snapshot connection already has an in-progress transaction with a save point + // that prevents switching from a PDB to the root CDB and if invoking the LogMiner APIs on + // such a connection, the use of commit/rollback by LogMiner will drop/invalidate the save + // point as well. A separate connection is necessary to preserve the save point. + try (OracleConnection conn = + new OracleConnection( + connection.config(), () -> getClass().getClassLoader(), false)) { + conn.setAutoCommit(false); + if (!Strings.isNullOrEmpty(connectorConfig.getPdbName())) { + // The next stage cannot be run within the PDB, reset the connection to the CDB. + conn.resetSessionToCdb(); + } + return determineSnapshotOffset( + connectorConfig, conn, currentScn.get(), pendingTransactions, tableName); + } + } + + private Optional getCurrentScn(Scn latestTableDdlScn, OracleConnection connection) + throws SQLException { + final String query = "SELECT CURRENT_SCN FROM V$DATABASE"; + + Scn currentScn; + do { + currentScn = + connection.queryAndMap( + query, rs -> rs.next() ? Scn.valueOf(rs.getString(1)) : Scn.NULL); + } while (areSameTimestamp(latestTableDdlScn, currentScn, connection)); + + return Optional.ofNullable(currentScn); + } + + private Optional getPendingTransactions( + Scn latestTableDdlScn, + OracleConnection connection, + Map transactions, + String transactionTableName) + throws SQLException { + final String query = + "SELECT d.CURRENT_SCN, t.XID, t.START_SCN " + + "FROM V$DATABASE d " + + "LEFT OUTER JOIN " + + transactionTableName + + " t " + + "ON t.START_SCN < d.CURRENT_SCN "; + + Scn currentScn = null; + do { + // Clear iterative state + currentScn = null; + transactions.clear(); + + try (Statement s = connection.connection().createStatement(); + ResultSet rs = s.executeQuery(query)) { + List results = new ArrayList<>(); + Statement s2 = connection.connection().createStatement(); + ResultSet rs2 = + s2.executeQuery( + "SELECT t.START_SCN, t.START_SCNB, t.DEPENDENT_SCN FROM V$TRANSACTION t"); + while (rs2.next()) { + results.add( + String.join( + " | ", rs2.getString(1), rs2.getString(2), rs2.getString(3))); + } + if (!results.isEmpty()) { + LOGGER.info("NOT EMPTY TRSNASSS: {}", results); + } + rs2.close(); + + while (rs.next()) { + if (currentScn == null) { + // Only need to set this once per iteration + currentScn = Scn.valueOf(rs.getString(1)); + } + final String pendingTxStartScn = rs.getString(3); + if (!Strings.isNullOrEmpty(pendingTxStartScn)) { + // There is a pending transaction, capture state + transactions.put( + HexConverter.convertToHexString(rs.getBytes(2)), + Scn.valueOf(pendingTxStartScn)); + } + } + } catch (SQLException e) { + LOGGER.warn( + "Could not query the {} view: {}", transactionTableName, e.getMessage(), e); + throw e; + } + + } while (areSameTimestamp(latestTableDdlScn, currentScn, connection)); + + for (Map.Entry transaction : transactions.entrySet()) { + LOGGER.trace( + "\tPending Transaction '{}' started at SCN {}", + transaction.getKey(), + transaction.getValue()); + } + + return Optional.ofNullable(currentScn); + } + + private OracleOffsetContext determineSnapshotOffset( + OracleConnectorConfig connectorConfig, + OracleConnection connection, + Scn currentScn, + Map pendingTransactions, + String transactionTableName) + throws SQLException { + + if (isPendingTransactionSkip(connectorConfig)) { + LOGGER.info("\tNo in-progress transactions will be captured."); + } else if (isPendingTransactionViewOnly(connectorConfig)) { + LOGGER.info( + "\tSkipping transaction logs for resolving snapshot offset, only using {}.", + transactionTableName); + } else { + LOGGER.info( + "\tConsulting {} and transaction logs for resolving snapshot offset.", + transactionTableName); + getPendingTransactionsFromLogs(connection, currentScn, pendingTransactions); + } + + if (!pendingTransactions.isEmpty()) { + for (Map.Entry entry : pendingTransactions.entrySet()) { + LOGGER.info( + "\tFound in-progress transaction {}, starting at SCN {}", + entry.getKey(), + entry.getValue()); + } + } else if (!isPendingTransactionSkip(connectorConfig)) { + LOGGER.info("\tFound no in-progress transactions."); + } + + return OracleOffsetContext.create() + .logicalName(connectorConfig) + .scn(currentScn) + .snapshotScn(currentScn) + .snapshotPendingTransactions(pendingTransactions) + .transactionContext(new TransactionContext()) + .incrementalSnapshotContext(new SignalBasedIncrementalSnapshotContext<>()) + .build(); + } + + private void addLogsToSession(List logs, OracleConnection connection) + throws SQLException { + for (LogFile logFile : logs) { + LOGGER.debug("\tAdding log: {}", logFile.getFileName()); + connection.executeWithoutCommitting( + SqlUtils.addLogFileStatement("DBMS_LOGMNR.ADDFILE", logFile.getFileName())); + } + } + + private void startSession(OracleConnection connection) throws SQLException { + // We explicitly use the ONLINE data dictionary mode here. + // Since we are only concerned about non-SQL columns, it is safe to always use this mode + final String query = + "BEGIN sys.dbms_logmnr.start_logmnr(" + + "OPTIONS => DBMS_LOGMNR.DICT_FROM_ONLINE_CATALOG + DBMS_LOGMNR.NO_ROWID_IN_STMT);" + + "END;"; + LOGGER.debug("\tStarting mining session"); + connection.executeWithoutCommitting(query); + } + + private void stopSession(OracleConnection connection) throws SQLException { + // stop the current mining session + try { + LOGGER.debug("\tStopping mining session"); + connection.executeWithoutCommitting("BEGIN SYS.DBMS_LOGMNR.END_LOGMNR(); END;"); + } catch (SQLException e) { + if (e.getMessage().toUpperCase().contains("ORA-01307")) { + LOGGER.debug("LogMiner mining session is already closed."); + } else { + throw e; + } + } + } + + private Scn getOldestScnAvailableInLogs( + OracleConnectorConfig config, OracleConnection connection) throws SQLException { + final Duration archiveLogRetention = config.getLogMiningArchiveLogRetention(); + final String archiveLogDestinationName = config.getLogMiningArchiveDestinationName(); + return connection.queryAndMap( + SqlUtils.oldestFirstChangeQuery(archiveLogRetention, archiveLogDestinationName), + rs -> { + if (rs.next()) { + final String value = rs.getString(1); + if (!Strings.isNullOrEmpty(value)) { + return Scn.valueOf(value); + } + } + return Scn.NULL; + }); + } + + private List getOrderedLogsFromScn( + OracleConnectorConfig config, Scn sinceScn, OracleConnection connection) + throws SQLException { + return LogMinerHelper.getLogFilesForOffsetScn( + connection, + sinceScn, + config.getLogMiningArchiveLogRetention(), + config.isArchiveLogOnlyMode(), + config.getLogMiningArchiveDestinationName()) + .stream() + .sorted(Comparator.comparing(LogFile::getSequence)) + .collect(Collectors.toList()); + } + + private void getPendingTransactionsFromLogs( + OracleConnection connection, Scn currentScn, Map pendingTransactions) + throws SQLException { + final Scn oldestScn = getOldestScnAvailableInLogs(connectorConfig, connection); + final List logFiles = + getOrderedLogsFromScn(connectorConfig, oldestScn, connection); + if (!logFiles.isEmpty()) { + try { + addLogsToSession(getMostRecentLogFilesForSearch(logFiles), connection); + startSession(connection); + + LOGGER.info("\tQuerying transaction logs, please wait..."); + connection.query( + "SELECT START_SCN, XID FROM V$LOGMNR_CONTENTS WHERE OPERATION_CODE=7 AND SCN >= " + + currentScn + + " AND START_SCN <= " + + currentScn, + rs -> { + while (rs.next()) { + final String transactionId = + HexConverter.convertToHexString(rs.getBytes("XID")); + final String startScnStr = rs.getString("START_SCN"); + if (!Strings.isNullOrBlank(startScnStr)) { + final Scn startScn = Scn.valueOf(rs.getString("START_SCN")); + if (!pendingTransactions.containsKey(transactionId)) { + LOGGER.info( + "\tTransaction '{}' started at SCN '{}'", + transactionId, + startScn); + pendingTransactions.put(transactionId, startScn); + } + } + } + }); + } catch (Exception e) { + throw new DebeziumException("Failed to resolve snapshot offset", e); + } finally { + stopSession(connection); + } + } + } + + private List getMostRecentLogFilesForSearch(List allLogFiles) { + Map> recentLogsPerThread = new HashMap<>(); + for (LogFile logFile : allLogFiles) { + if (!recentLogsPerThread.containsKey(logFile.getThread())) { + if (logFile.isCurrent()) { + recentLogsPerThread.put(logFile.getThread(), new ArrayList<>()); + recentLogsPerThread.get(logFile.getThread()).add(logFile); + final Optional maxArchiveLogFile = + allLogFiles.stream() + .filter( + f -> + logFile.getThread() == f.getThread() + && logFile.getSequence() + .compareTo( + f.getSequence()) + > 0) + .max(Comparator.comparing(LogFile::getSequence)); + maxArchiveLogFile.ifPresent( + file -> recentLogsPerThread.get(logFile.getThread()).add(file)); + } + } + } + + final List logs = new ArrayList<>(); + for (Map.Entry> entry : recentLogsPerThread.entrySet()) { + logs.addAll(entry.getValue()); + } + return logs; + } + + private boolean isPendingTransactionSkip(OracleConnectorConfig config) { + return config.getLogMiningTransactionSnapshotBoundaryMode() + == TransactionSnapshotBoundaryMode.SKIP; + } + + public boolean isPendingTransactionViewOnly(OracleConnectorConfig config) { + return config.getLogMiningTransactionSnapshotBoundaryMode() + == TransactionSnapshotBoundaryMode.TRANSACTION_VIEW_ONLY; + } + + /** + * Under Oracle RAC, the V$ tables are specific the node that the JDBC connection is established + * to and not every V$ is synchronized across the cluster. Therefore, when Oracle RAC is in + * play, we should use the GV$ tables instead. + * + * @param config the connector configuration, should not be {@code null} + * @return the pending transaction table name + */ + private static String getTransactionTableName(OracleConnectorConfig config) { + if (config.getRacNodes() == null || config.getRacNodes().isEmpty()) { + return "V$TRANSACTION"; + } + return "GV$TRANSACTION"; + } +} diff --git a/flink-connector-oracle-cdc/src/main/java/io/debezium/connector/oracle/logminer/LogMinerChangeRecordEmitter.java b/flink-connector-oracle-cdc/src/main/java/io/debezium/connector/oracle/logminer/LogMinerChangeRecordEmitter.java index 4165083ecf..468616e36e 100644 --- a/flink-connector-oracle-cdc/src/main/java/io/debezium/connector/oracle/logminer/LogMinerChangeRecordEmitter.java +++ b/flink-connector-oracle-cdc/src/main/java/io/debezium/connector/oracle/logminer/LogMinerChangeRecordEmitter.java @@ -8,261 +8,78 @@ import io.debezium.DebeziumException; import io.debezium.connector.oracle.BaseChangeRecordEmitter; -import io.debezium.connector.oracle.logminer.valueholder.LogMinerDmlEntry; +import io.debezium.connector.oracle.OracleConnectorConfig; +import io.debezium.connector.oracle.OracleDatabaseSchema; +import io.debezium.connector.oracle.logminer.events.EventType; import io.debezium.data.Envelope.Operation; import io.debezium.pipeline.spi.OffsetContext; +import io.debezium.pipeline.spi.Partition; import io.debezium.relational.Table; -import io.debezium.relational.TableSchema; -import io.debezium.schema.DataCollectionSchema; import io.debezium.util.Clock; import oracle.sql.ROWID; import org.apache.kafka.connect.data.SchemaAndValue; -import org.apache.kafka.connect.data.Struct; -import org.apache.kafka.connect.header.ConnectHeaders; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import java.util.Objects; - -/** - * Copied from Debezium 1.6.4.Final. - * - *

Emits change record based on a single {@link LogMinerDmlEntry} event. - * - *

This class overrides the emit methods to put the ROWID in the header. - * - *

Line 59 ~ 257: add ROWID and emit methods. - */ +/** Emits change records based on an event read from Oracle LogMiner. */ public class LogMinerChangeRecordEmitter extends BaseChangeRecordEmitter { - private static final Logger LOGGER = LoggerFactory.getLogger(LogMinerChangeRecordEmitter.class); - - private final int operation; - private final Object[] oldValues; - private final Object[] newValues; - private final String rowId; + private final Operation operation; public LogMinerChangeRecordEmitter( + OracleConnectorConfig connectorConfig, + Partition partition, OffsetContext offset, - int operation, + Operation operation, Object[] oldValues, Object[] newValues, Table table, - Clock clock) { - super(offset, table, clock); + OracleDatabaseSchema schema, + Clock clock, + String rowId) { + super(connectorConfig, partition, offset, schema, table, clock, oldValues, newValues); this.operation = operation; - this.oldValues = oldValues; - this.newValues = newValues; - this.rowId = null; + addStaticHeader(ROWID.class.getSimpleName(), new SchemaAndValue(null, rowId)); } public LogMinerChangeRecordEmitter( + OracleConnectorConfig connectorConfig, + Partition partition, OffsetContext offset, - int operation, + EventType eventType, Object[] oldValues, Object[] newValues, Table table, + OracleDatabaseSchema schema, Clock clock, String rowId) { - super(offset, table, clock); - this.operation = operation; - this.oldValues = oldValues; - this.newValues = newValues; - this.rowId = rowId; + this( + connectorConfig, + partition, + offset, + getOperation(eventType), + oldValues, + newValues, + table, + schema, + clock, + rowId); } - @Override - protected Operation getOperation() { - switch (operation) { - case RowMapper.INSERT: + private static Operation getOperation(EventType eventType) { + switch (eventType) { + case INSERT: return Operation.CREATE; - case RowMapper.UPDATE: - case RowMapper.SELECT_LOB_LOCATOR: - return Operation.UPDATE; - case RowMapper.DELETE: - return Operation.DELETE; - default: - throw new DebeziumException("Unsupported operation type: " + operation); - } - } - - @Override - public void emitChangeRecords(DataCollectionSchema schema, Receiver receiver) - throws InterruptedException { - TableSchema tableSchema = (TableSchema) schema; - Operation operation = getOperation(); - - switch (operation) { - case CREATE: - emitCreateRecord(receiver, tableSchema); - break; - case READ: - emitReadRecord(receiver, tableSchema); - break; case UPDATE: - emitUpdateRecord(receiver, tableSchema); - break; + case SELECT_LOB_LOCATOR: + return Operation.UPDATE; case DELETE: - emitDeleteRecord(receiver, tableSchema); - break; - case TRUNCATE: - emitTruncateRecord(receiver, tableSchema); - break; + return Operation.DELETE; default: - throw new IllegalArgumentException("Unsupported operation: " + operation); + throw new DebeziumException("Unsupported operation type: " + eventType); } } @Override - protected void emitCreateRecord(Receiver receiver, TableSchema tableSchema) - throws InterruptedException { - Object[] newColumnValues = getNewColumnValues(); - Struct newKey = tableSchema.keyFromColumnData(newColumnValues); - Struct newValue = tableSchema.valueFromColumnData(newColumnValues); - Struct envelope = - tableSchema - .getEnvelopeSchema() - .create( - newValue, - getOffset().getSourceInfo(), - getClock().currentTimeAsInstant()); - ConnectHeaders headers = new ConnectHeaders(); - headers.add(ROWID.class.getSimpleName(), new SchemaAndValue(null, rowId)); - - if (skipEmptyMessages() && (newColumnValues == null || newColumnValues.length == 0)) { - // This case can be hit on UPDATE / DELETE when there's no primary key defined while - // using certain decoders - LOGGER.warn( - "no new values found for table '{}' from create message at '{}'; skipping record", - tableSchema, - getOffset().getSourceInfo()); - return; - } - receiver.changeRecord( - tableSchema, Operation.CREATE, newKey, envelope, getOffset(), headers); - } - - @Override - protected void emitReadRecord(Receiver receiver, TableSchema tableSchema) - throws InterruptedException { - Object[] newColumnValues = getNewColumnValues(); - Struct newKey = tableSchema.keyFromColumnData(newColumnValues); - Struct newValue = tableSchema.valueFromColumnData(newColumnValues); - Struct envelope = - tableSchema - .getEnvelopeSchema() - .read( - newValue, - getOffset().getSourceInfo(), - getClock().currentTimeAsInstant()); - ConnectHeaders headers = new ConnectHeaders(); - headers.add(ROWID.class.getSimpleName(), new SchemaAndValue(null, rowId)); - - receiver.changeRecord(tableSchema, Operation.READ, newKey, envelope, getOffset(), headers); - } - - @Override - protected void emitUpdateRecord(Receiver receiver, TableSchema tableSchema) - throws InterruptedException { - Object[] oldColumnValues = getOldColumnValues(); - Object[] newColumnValues = getNewColumnValues(); - - Struct oldKey = tableSchema.keyFromColumnData(oldColumnValues); - Struct newKey = tableSchema.keyFromColumnData(newColumnValues); - - Struct newValue = tableSchema.valueFromColumnData(newColumnValues); - Struct oldValue = tableSchema.valueFromColumnData(oldColumnValues); - - if (skipEmptyMessages() && (newColumnValues == null || newColumnValues.length == 0)) { - LOGGER.warn( - "no new values found for table '{}' from update message at '{}'; skipping record", - tableSchema, - getOffset().getSourceInfo()); - return; - } - // some configurations does not provide old values in case of updates - // in this case we handle all updates as regular ones - if (oldKey == null || Objects.equals(oldKey, newKey)) { - Struct envelope = - tableSchema - .getEnvelopeSchema() - .update( - oldValue, - newValue, - getOffset().getSourceInfo(), - getClock().currentTimeAsInstant()); - ConnectHeaders headers = new ConnectHeaders(); - headers.add(ROWID.class.getSimpleName(), new SchemaAndValue(null, rowId)); - receiver.changeRecord( - tableSchema, Operation.UPDATE, newKey, envelope, getOffset(), headers); - } - // PK update -> emit as delete and re-insert with new key - else { - ConnectHeaders headers = new ConnectHeaders(); - headers.add(PK_UPDATE_NEWKEY_FIELD, newKey, tableSchema.keySchema()); - - Struct envelope = - tableSchema - .getEnvelopeSchema() - .delete( - oldValue, - getOffset().getSourceInfo(), - getClock().currentTimeAsInstant()); - receiver.changeRecord( - tableSchema, Operation.DELETE, oldKey, envelope, getOffset(), headers); - - headers = new ConnectHeaders(); - headers.add(PK_UPDATE_OLDKEY_FIELD, oldKey, tableSchema.keySchema()); - - headers.add(ROWID.class.getSimpleName(), new SchemaAndValue(null, rowId)); - - envelope = - tableSchema - .getEnvelopeSchema() - .create( - newValue, - getOffset().getSourceInfo(), - getClock().currentTimeAsInstant()); - receiver.changeRecord( - tableSchema, Operation.CREATE, newKey, envelope, getOffset(), headers); - } - } - - @Override - protected void emitDeleteRecord(Receiver receiver, TableSchema tableSchema) - throws InterruptedException { - Object[] oldColumnValues = getOldColumnValues(); - Struct oldKey = tableSchema.keyFromColumnData(oldColumnValues); - Struct oldValue = tableSchema.valueFromColumnData(oldColumnValues); - ConnectHeaders headers = new ConnectHeaders(); - headers.add(ROWID.class.getSimpleName(), new SchemaAndValue(null, rowId)); - - if (skipEmptyMessages() && (oldColumnValues == null || oldColumnValues.length == 0)) { - LOGGER.warn( - "no old values found for table '{}' from delete message at '{}'; skipping record", - tableSchema, - getOffset().getSourceInfo()); - return; - } - - Struct envelope = - tableSchema - .getEnvelopeSchema() - .delete( - oldValue, - getOffset().getSourceInfo(), - getClock().currentTimeAsInstant()); - receiver.changeRecord( - tableSchema, Operation.DELETE, oldKey, envelope, getOffset(), headers); - } - - @Override - protected Object[] getOldColumnValues() { - return oldValues; - } - - @Override - protected Object[] getNewColumnValues() { - return newValues; + public Operation getOperation() { + return operation; } } diff --git a/flink-connector-oracle-cdc/src/main/java/io/debezium/connector/oracle/logminer/LogMinerStreamingChangeEventSource.java b/flink-connector-oracle-cdc/src/main/java/io/debezium/connector/oracle/logminer/LogMinerStreamingChangeEventSource.java index e5519d9313..c813ab005d 100644 --- a/flink-connector-oracle-cdc/src/main/java/io/debezium/connector/oracle/logminer/LogMinerStreamingChangeEventSource.java +++ b/flink-connector-oracle-cdc/src/main/java/io/debezium/connector/oracle/logminer/LogMinerStreamingChangeEventSource.java @@ -22,12 +22,19 @@ import io.debezium.connector.oracle.OracleConnectorConfig; import io.debezium.connector.oracle.OracleDatabaseSchema; import io.debezium.connector.oracle.OracleOffsetContext; +import io.debezium.connector.oracle.OraclePartition; import io.debezium.connector.oracle.OracleStreamingChangeEventSourceMetrics; import io.debezium.connector.oracle.Scn; +import io.debezium.connector.oracle.logminer.logwriter.CommitLogWriterFlushStrategy; +import io.debezium.connector.oracle.logminer.logwriter.LogWriterFlushStrategy; +import io.debezium.connector.oracle.logminer.logwriter.RacCommitLogWriterFlushStrategy; +import io.debezium.connector.oracle.logminer.processor.LogMinerEventProcessor; import io.debezium.jdbc.JdbcConfiguration; import io.debezium.pipeline.ErrorHandler; import io.debezium.pipeline.EventDispatcher; import io.debezium.pipeline.source.spi.StreamingChangeEventSource; +import io.debezium.relational.Column; +import io.debezium.relational.Table; import io.debezium.relational.TableId; import io.debezium.util.Clock; import io.debezium.util.Metronome; @@ -36,12 +43,14 @@ import org.slf4j.LoggerFactory; import java.math.BigInteger; -import java.sql.PreparedStatement; -import java.sql.ResultSet; import java.sql.SQLException; +import java.text.DecimalFormat; import java.time.Duration; import java.time.Instant; +import java.time.OffsetDateTime; +import java.time.temporal.ChronoUnit; import java.util.ArrayList; +import java.util.Collections; import java.util.HashSet; import java.util.LinkedHashMap; import java.util.List; @@ -50,36 +59,27 @@ import java.util.Set; import java.util.stream.Collectors; -import static io.debezium.connector.oracle.logminer.LogMinerHelper.buildDataDictionary; -import static io.debezium.connector.oracle.logminer.LogMinerHelper.checkSupplementalLogging; -import static io.debezium.connector.oracle.logminer.LogMinerHelper.endMining; -import static io.debezium.connector.oracle.logminer.LogMinerHelper.getCurrentRedoLogFiles; -import static io.debezium.connector.oracle.logminer.LogMinerHelper.getEndScn; -import static io.debezium.connector.oracle.logminer.LogMinerHelper.getFirstOnlineLogScn; -import static io.debezium.connector.oracle.logminer.LogMinerHelper.getLastScnToAbandon; -import static io.debezium.connector.oracle.logminer.LogMinerHelper.getSystime; -import static io.debezium.connector.oracle.logminer.LogMinerHelper.instantiateFlushConnections; import static io.debezium.connector.oracle.logminer.LogMinerHelper.logError; import static io.debezium.connector.oracle.logminer.LogMinerHelper.setLogFilesForMining; -import static io.debezium.connector.oracle.logminer.LogMinerHelper.setNlsSessionParameters; -import static io.debezium.connector.oracle.logminer.LogMinerHelper.startLogMining; /** - * A {@link StreamingChangeEventSource} based on Oracle's LogMiner utility. The event handler loop - * is executed in a separate executor, and add method afterHandleScn. + * Copied from Debezium 1.9.7. Diff: added afterHandleScn() method. A {@link + * StreamingChangeEventSource} based on Oracle's LogMiner utility. The event handler loop is + * executed in a separate executor. */ public class LogMinerStreamingChangeEventSource - implements StreamingChangeEventSource { + implements StreamingChangeEventSource { private static final Logger LOGGER = LoggerFactory.getLogger(LogMinerStreamingChangeEventSource.class); + private static final int MAXIMUM_NAME_LENGTH = 30; + private static final String ALL_COLUMN_LOGGING = "ALL COLUMN LOGGING"; + private static final int MINING_START_RETRIES = 5; private final OracleConnection jdbcConnection; - private final EventDispatcher dispatcher; + private final EventDispatcher dispatcher; private final Clock clock; private final OracleDatabaseSchema schema; - private final boolean isRac; - private final Set racHosts = new HashSet<>(); private final JdbcConfiguration jdbcConfiguration; private final OracleConnectorConfig.LogMiningStrategy strategy; private final ErrorHandler errorHandler; @@ -89,15 +89,20 @@ public class LogMinerStreamingChangeEventSource private final Duration archiveLogRetention; private final boolean archiveLogOnlyMode; private final String archiveDestinationName; + private final int logFileQueryMaxRetries; + private final Duration initialDelay; + private final Duration maxDelay; - private Scn startScn; + private Scn startScn; // startScn is the **exclusive** lower bound for mining private Scn endScn; + private Scn snapshotScn; + private List currentLogFiles; private List currentRedoLogSequences; public LogMinerStreamingChangeEventSource( OracleConnectorConfig connectorConfig, OracleConnection jdbcConnection, - EventDispatcher dispatcher, + EventDispatcher dispatcher, ErrorHandler errorHandler, Clock clock, OracleDatabaseSchema schema, @@ -113,17 +118,12 @@ public LogMinerStreamingChangeEventSource( this.errorHandler = errorHandler; this.streamingMetrics = streamingMetrics; this.jdbcConfiguration = JdbcConfiguration.adapt(jdbcConfig); - this.isRac = connectorConfig.isRacSystem(); - if (this.isRac) { - this.racHosts.addAll( - connectorConfig.getRacNodes().stream() - .map(String::toUpperCase) - .collect(Collectors.toSet())); - instantiateFlushConnections(jdbcConfiguration, racHosts); - } this.archiveLogRetention = connectorConfig.getLogMiningArchiveLogRetention(); this.archiveLogOnlyMode = connectorConfig.isArchiveLogOnlyMode(); this.archiveDestinationName = connectorConfig.getLogMiningArchiveDestinationName(); + this.logFileQueryMaxRetries = connectorConfig.getMaximumNumberOfLogQueryRetries(); + this.initialDelay = connectorConfig.getLogMiningInitialDelay(); + this.maxDelay = connectorConfig.getLogMiningMaxDelay(); } /** @@ -132,20 +132,34 @@ public LogMinerStreamingChangeEventSource( * @param context change event source context */ @Override - public void execute(ChangeEventSourceContext context, OracleOffsetContext offsetContext) { - try (TransactionalBuffer transactionalBuffer = - new TransactionalBuffer( - connectorConfig, schema, clock, errorHandler, streamingMetrics)) { - try { - startScn = offsetContext.getScn(); - - if (!isContinuousMining - && startScn.compareTo( - getFirstOnlineLogScn( - jdbcConnection, - archiveLogRetention, - archiveDestinationName)) - < 0) { + public void execute( + ChangeEventSourceContext context, + OraclePartition partition, + OracleOffsetContext offsetContext) { + if (!connectorConfig.getSnapshotMode().shouldStream()) { + LOGGER.info("Streaming is not enabled in current configuration"); + return; + } + try { + // We explicitly expect auto-commit to be disabled + jdbcConnection.setAutoCommit(false); + + startScn = offsetContext.getScn(); + snapshotScn = offsetContext.getSnapshotScn(); + Scn firstScn = getFirstScnInLogs(jdbcConnection); + if (startScn.compareTo(snapshotScn) == 0) { + // This is the initial run of the streaming change event source. + // We need to compute the correct start offset for mining. That is not the snapshot + // offset, + // but the start offset of the oldest transaction that was still pending when the + // snapshot + // was taken. + computeStartScnForFirstMiningSession(offsetContext, firstScn); + } + + try (LogWriterFlushStrategy flushStrategy = resolveFlushStrategy()) { + if (!isContinuousMining && startScn.compareTo(firstScn.subtract(Scn.ONE)) < 0) { + // startScn is the exclusive lower bound, so must be >= (firstScn - 1) throw new DebeziumException( "Online REDO LOG files or archive log files do not contain the offset scn " + startScn @@ -153,206 +167,237 @@ public void execute(ChangeEventSourceContext context, OracleOffsetContext offset } setNlsSessionParameters(jdbcConnection); - checkSupplementalLogging(jdbcConnection, connectorConfig.getPdbName(), schema); + checkDatabaseAndTableState(jdbcConnection, connectorConfig.getPdbName(), schema); - if (archiveLogOnlyMode && !waitForStartScnInArchiveLogs(context, startScn)) { - return; - } + try (LogMinerEventProcessor processor = + createProcessor(context, partition, offsetContext)) { - initializeRedoLogsForMining(jdbcConnection, false, startScn); - - HistoryRecorder historyRecorder = connectorConfig.getLogMiningHistoryRecorder(); - - try { - // todo: why can't OracleConnection be used rather than a - // Factory+JdbcConfiguration? - historyRecorder.prepare( - streamingMetrics, - jdbcConfiguration, - connectorConfig.getLogMinerHistoryRetentionHours()); - - final LogMinerQueryResultProcessor processor = - new LogMinerQueryResultProcessor( - context, - connectorConfig, - streamingMetrics, - transactionalBuffer, - offsetContext, - schema, - dispatcher, - historyRecorder); - - final String query = - LogMinerQueryBuilder.build( - connectorConfig, schema, jdbcConnection.username()); - try (PreparedStatement miningView = - jdbcConnection - .connection() - .prepareStatement( - query, - ResultSet.TYPE_FORWARD_ONLY, - ResultSet.CONCUR_READ_ONLY, - ResultSet.HOLD_CURSORS_OVER_COMMIT)) { - - currentRedoLogSequences = getCurrentRedoLogSequences(); - Stopwatch stopwatch = Stopwatch.reusable(); - while (context.isRunning()) { - // Calculate time difference before each mining session to detect time - // zone offset changes (e.g. DST) on database server - streamingMetrics.calculateTimeDifference(getSystime(jdbcConnection)); - - if (archiveLogOnlyMode - && !waitForStartScnInArchiveLogs(context, startScn)) { - break; - } + if (archiveLogOnlyMode && !waitForStartScnInArchiveLogs(context, startScn)) { + return; + } - Instant start = Instant.now(); - endScn = - getEndScn( - jdbcConnection, - startScn, - endScn, - streamingMetrics, - connectorConfig.getLogMiningBatchSizeDefault(), - connectorConfig.isLobEnabled(), - connectorConfig.isArchiveLogOnlyMode(), - connectorConfig.getLogMiningArchiveDestinationName()); - - // This is a small window where when archive log only mode has - // completely caught up to the last - // record in the archive logs that both the start and end values are - // identical. In this use - // case we want to pause and restart the loop waiting for a new archive - // log before proceeding. - if (archiveLogOnlyMode && startScn.equals(endScn)) { - pauseBetweenMiningSessions(); - continue; - } + initializeRedoLogsForMining(jdbcConnection, false, startScn); - if (hasLogSwitchOccurred()) { - // This is the way to mitigate PGA leaks. - // With one mining session, it grows and maybe there is another way - // to flush PGA. - // At this point we use a new mining session - LOGGER.trace( - "Ending log mining startScn={}, endScn={}, offsetContext.getScn={}, strategy={}, continuous={}", - startScn, - endScn, - offsetContext.getScn(), - strategy, - isContinuousMining); - endMining(jdbcConnection); - - initializeRedoLogsForMining(jdbcConnection, true, startScn); - - abandonOldTransactionsIfExist( - jdbcConnection, offsetContext, transactionalBuffer); - - // This needs to be re-calculated because building the data - // dictionary will force the - // current redo log sequence to be advanced due to a complete log - // switch of all logs. - currentRedoLogSequences = getCurrentRedoLogSequences(); - } + int retryAttempts = 1; + Stopwatch sw = Stopwatch.accumulating().start(); + while (context.isRunning()) { + // Calculate time difference before each mining session to detect time zone + // offset changes (e.g. DST) on database server + streamingMetrics.calculateTimeDifference( + getDatabaseSystemTime(jdbcConnection)); - startLogMining( - jdbcConnection, - startScn, - endScn, - strategy, - isContinuousMining, - streamingMetrics); - - LOGGER.trace( - "Fetching LogMiner view results SCN {} to {}", - startScn, - endScn); - stopwatch.start(); - miningView.setFetchSize(connectorConfig.getMaxQueueSize()); - miningView.setFetchDirection(ResultSet.FETCH_FORWARD); - miningView.setString(1, startScn.toString()); - miningView.setString(2, endScn.toString()); - try (ResultSet rs = miningView.executeQuery()) { - Duration lastDurationOfBatchCapturing = - stopwatch.stop().durations().statistics().getTotal(); - streamingMetrics.setLastDurationOfBatchCapturing( - lastDurationOfBatchCapturing); - processor.processResult(rs); - if (connectorConfig.isLobEnabled()) { - startScn = - transactionalBuffer.updateOffsetContext( - offsetContext, dispatcher); - } else { - - final Scn lastProcessedScn = processor.getLastProcessedScn(); - if (!lastProcessedScn.isNull() - && lastProcessedScn.compareTo(endScn) < 0) { - // If the last processed SCN is before the endScn we need to - // use the last processed SCN as the - // next starting point as the LGWR buffer didn't flush all - // entries from memory to disk yet. - endScn = lastProcessedScn; - } - - if (transactionalBuffer.isEmpty()) { - LOGGER.debug( - "Buffer is empty, updating offset SCN to {}", - endScn); - offsetContext.setScn(endScn); - } else { - final Scn minStartScn = transactionalBuffer.getMinimumScn(); - if (!minStartScn.isNull()) { - offsetContext.setScn( - minStartScn.subtract(Scn.valueOf(1))); - dispatcher.dispatchHeartbeatEvent(offsetContext); - } - } - startScn = endScn; - } + if (archiveLogOnlyMode + && !waitForStartScnInArchiveLogs(context, startScn)) { + break; + } + + Instant start = Instant.now(); + endScn = calculateEndScn(jdbcConnection, startScn, endScn); + + // This is a small window where when archive log only mode has completely + // caught up to the last + // record in the archive logs that both the start and end values are + // identical. In this use + // case we want to pause and restart the loop waiting for a new archive log + // before proceeding. + if (archiveLogOnlyMode && startScn.equals(endScn)) { + pauseBetweenMiningSessions(); + continue; + } + + flushStrategy.flush(jdbcConnection.getCurrentScn()); + + boolean restartRequired = false; + if (connectorConfig.getLogMiningMaximumSession().isPresent()) { + final Duration totalDuration = + sw.stop().durations().statistics().getTotal(); + if (totalDuration.toMillis() + >= connectorConfig + .getLogMiningMaximumSession() + .get() + .toMillis()) { + LOGGER.info( + "LogMiner session has exceeded maximum session time of '{}', forcing restart.", + connectorConfig.getLogMiningMaximumSession()); + restartRequired = true; + } else { + // resume the existing stop watch, we haven't met the criteria yet + sw.start(); } + } - afterHandleScn(offsetContext); - streamingMetrics.setCurrentBatchProcessingTime( - Duration.between(start, Instant.now())); + if (restartRequired || hasLogSwitchOccurred()) { + // This is the way to mitigate PGA leaks. + // With one mining session, it grows and maybe there is another way to + // flush PGA. + // At this point we use a new mining session + endMiningSession(jdbcConnection, offsetContext); + initializeRedoLogsForMining(jdbcConnection, true, startScn); + + // log switch or restart required, re-create a new stop watch + sw = Stopwatch.accumulating().start(); + } + + if (context.isRunning()) { + if (!startMiningSession( + jdbcConnection, startScn, endScn, retryAttempts)) { + retryAttempts++; + } else { + retryAttempts = 1; + startScn = processor.process(partition, startScn, endScn); + streamingMetrics.setCurrentBatchProcessingTime( + Duration.between(start, Instant.now())); + captureSessionMemoryStatistics(jdbcConnection); + } pauseBetweenMiningSessions(); } + + afterHandleScn(partition, offsetContext); } - } finally { - historyRecorder.close(); } - } catch (Throwable t) { - logError(streamingMetrics, "Mining session stopped due to the {}", t); - errorHandler.setProducerThrowable(t); - } finally { + } + } catch (Throwable t) { + logError(streamingMetrics, "Mining session stopped due to the {}", t); + errorHandler.setProducerThrowable(t); + } finally { + LOGGER.info("startScn={}, endScn={}", startScn, endScn); + LOGGER.info("Streaming metrics dump: {}", streamingMetrics.toString()); + LOGGER.info("Offsets: {}", offsetContext); + } + } + + protected void afterHandleScn(OraclePartition partition, OracleOffsetContext offsetContext) {} + + /** + * Computes the start SCN for the first mining session. + * + *

Normally, this would be the snapshot SCN, but if there were pending transactions at the + * time the snapshot was taken, we'd miss the events in those transactions that have an SCN + * smaller than the snapshot SCN. + * + * @param offsetContext the offset context + * @param firstScn the oldest SCN still available in the REDO logs + */ + private void computeStartScnForFirstMiningSession( + OracleOffsetContext offsetContext, Scn firstScn) { + // This is the initial run of the streaming change event source. + // We need to compute the correct start offset for mining. That is not the snapshot offset, + // but the start offset of the oldest transaction that was still pending when the snapshot + // was taken. + Map snapshotPendingTransactions = + offsetContext.getSnapshotPendingTransactions(); + if (snapshotPendingTransactions == null || snapshotPendingTransactions.isEmpty()) { + // no pending transactions, we can start mining from the snapshot SCN + startScn = snapshotScn; + } else { + // find the oldest transaction we can still fully process, and start from there. + Scn minScn = snapshotScn; + for (Map.Entry entry : snapshotPendingTransactions.entrySet()) { + String transactionId = entry.getKey(); + Scn scn = entry.getValue(); + LOGGER.info( + "Transaction {} was pending across snapshot boundary. Start SCN = {}, snapshot SCN = {}", + transactionId, + scn, + startScn); + if (scn.compareTo(firstScn) < 0) { + LOGGER.warn( + "Transaction {} was still ongoing while snapshot was taken, but is no longer completely recorded in the archive logs. Events will be lost. Oldest SCN in logs = {}, TX start SCN = {}", + transactionId, + firstScn, + scn); + minScn = firstScn; + } else if (scn.compareTo(minScn) < 0) { + minScn = scn; + } + } + + // Make sure the commit SCN is at least the snapshot SCN - 1. + // This ensures we'll never emit events for transactions that were complete before the + // snapshot was + // taken. + if (offsetContext.getCommitScn().compareTo(snapshotScn) < 0) { + LOGGER.info( + "Setting commit SCN to {} (snapshot SCN - 1) to ensure we don't double-emit events from pre-snapshot transactions.", + snapshotScn.subtract(Scn.ONE)); + offsetContext + .getCommitScn() + .setCommitScnOnAllThreads(snapshotScn.subtract(Scn.ONE)); + } + + // set start SCN to minScn + if (minScn.compareTo(startScn) <= 0) { LOGGER.info( - "startScn={}, endScn={}, offsetContext.getScn()={}", + "Resetting start SCN from {} (snapshot SCN) to {} (start of oldest complete pending transaction)", startScn, - endScn, - offsetContext.getScn()); - LOGGER.info("Transactional buffer dump: {}", transactionalBuffer.toString()); - LOGGER.info("Streaming metrics dump: {}", streamingMetrics.toString()); + minScn); + startScn = minScn.subtract(Scn.ONE); } } + offsetContext.setScn(startScn); + } + + private void captureSessionMemoryStatistics(OracleConnection connection) throws SQLException { + long sessionUserGlobalAreaMemory = + connection.getSessionStatisticByName("session uga memory"); + long sessionUserGlobalAreaMaxMemory = + connection.getSessionStatisticByName("session uga memory max"); + streamingMetrics.setUserGlobalAreaMemory( + sessionUserGlobalAreaMemory, sessionUserGlobalAreaMaxMemory); + + long sessionProcessGlobalAreaMemory = + connection.getSessionStatisticByName("session pga memory"); + long sessionProcessGlobalAreaMaxMemory = + connection.getSessionStatisticByName("session pga memory max"); + streamingMetrics.setProcessGlobalAreaMemory( + sessionProcessGlobalAreaMemory, sessionProcessGlobalAreaMaxMemory); + + final DecimalFormat format = new DecimalFormat("#.##"); + LOGGER.debug( + "Oracle Session UGA {}MB (max = {}MB), PGA {}MB (max = {}MB)", + format.format(sessionUserGlobalAreaMemory / 1024.f / 1024.f), + format.format(sessionUserGlobalAreaMaxMemory / 1024.f / 1024.f), + format.format(sessionProcessGlobalAreaMemory / 1024.f / 1024.f), + format.format(sessionProcessGlobalAreaMaxMemory / 1024.f / 1024.f)); } - protected void afterHandleScn(OracleOffsetContext offsetContext) {} - - private void abandonOldTransactionsIfExist( - OracleConnection connection, - OracleOffsetContext offsetContext, - TransactionalBuffer transactionalBuffer) { - Duration transactionRetention = connectorConfig.getLogMiningTransactionRetention(); - if (!Duration.ZERO.equals(transactionRetention)) { - final Scn offsetScn = offsetContext.getScn(); - Optional lastScnToAbandonTransactions = - getLastScnToAbandon(connection, offsetScn, transactionRetention); - lastScnToAbandonTransactions.ifPresent( - thresholdScn -> { - transactionalBuffer.abandonLongTransactions(thresholdScn, offsetContext); - offsetContext.setScn(thresholdScn); - startScn = endScn; - }); + private LogMinerEventProcessor createProcessor( + ChangeEventSourceContext context, + OraclePartition partition, + OracleOffsetContext offsetContext) { + final OracleConnectorConfig.LogMiningBufferType bufferType = + connectorConfig.getLogMiningBufferType(); + return bufferType.createProcessor( + context, + connectorConfig, + jdbcConnection, + dispatcher, + partition, + offsetContext, + schema, + streamingMetrics); + } + + /** + * Gets the first system change number in both archive and redo logs. + * + * @param connection database connection, should not be {@code null} + * @return the oldest system change number + * @throws SQLException if a database exception occurred + * @throws DebeziumException if the oldest system change number cannot be found due to no logs + * available + */ + private Scn getFirstScnInLogs(OracleConnection connection) throws SQLException { + String oldestScn = + connection.singleOptionalValue( + SqlUtils.oldestFirstChangeQuery( + archiveLogRetention, archiveDestinationName), + rs -> rs.getString(1)); + if (oldestScn == null) { + throw new DebeziumException("Failed to calculate oldest SCN available in logs"); } + LOGGER.trace("Oldest SCN in logs is '{}'", oldestScn); + return Scn.valueOf(oldestScn); } private void initializeRedoLogsForMining( @@ -363,26 +408,110 @@ private void initializeRedoLogsForMining( buildDataDictionary(connection); } if (!isContinuousMining) { - setLogFilesForMining( - connection, - startScn, - archiveLogRetention, - archiveLogOnlyMode, - archiveDestinationName); + currentLogFiles = + setLogFilesForMining( + connection, + startScn, + archiveLogRetention, + archiveLogOnlyMode, + archiveDestinationName, + logFileQueryMaxRetries, + initialDelay, + maxDelay); + currentRedoLogSequences = getCurrentLogFileSequences(currentLogFiles); } } else { if (!isContinuousMining) { if (OracleConnectorConfig.LogMiningStrategy.CATALOG_IN_REDO.equals(strategy)) { buildDataDictionary(connection); } - setLogFilesForMining( - connection, - startScn, - archiveLogRetention, - archiveLogOnlyMode, - archiveDestinationName); + currentLogFiles = + setLogFilesForMining( + connection, + startScn, + archiveLogRetention, + archiveLogOnlyMode, + archiveDestinationName, + logFileQueryMaxRetries, + initialDelay, + maxDelay); + currentRedoLogSequences = getCurrentLogFileSequences(currentLogFiles); } } + + updateRedoLogMetrics(); + } + + /** + * Get the current log file sequences from the supplied list of log files. + * + * @param logFiles list of log files + * @return list of sequences for the logs that are marked "current" in the database. + */ + private List getCurrentLogFileSequences(List logFiles) { + if (logFiles == null || logFiles.isEmpty()) { + return Collections.emptyList(); + } + return logFiles.stream() + .filter(LogFile::isCurrent) + .map(LogFile::getSequence) + .collect(Collectors.toList()); + } + + /** + * Get the maximum archive log SCN. + * + * @param logFiles the current logs that are part of the mining session + * @return the maximum system change number from the archive logs + * @throws DebeziumException if no logs are provided or if the provided logs has no archive log + * types + */ + private Scn getMaxArchiveLogScn(List logFiles) { + if (logFiles == null || logFiles.isEmpty()) { + throw new DebeziumException( + "Cannot get maximum archive log SCN as no logs were available."); + } + + final List archiveLogs = + logFiles.stream() + .filter(log -> log.getType().equals(LogFile.Type.ARCHIVE)) + .collect(Collectors.toList()); + + if (archiveLogs.isEmpty()) { + throw new DebeziumException( + "Cannot get maximum archive log SCN as no archive logs are present."); + } + + Scn maxScn = archiveLogs.get(0).getNextScn(); + for (int i = 1; i < archiveLogs.size(); ++i) { + Scn nextScn = archiveLogs.get(i).getNextScn(); + if (nextScn.compareTo(maxScn) > 0) { + maxScn = nextScn; + } + } + + LOGGER.debug("Maximum archive log SCN resolved as {}", maxScn); + return maxScn; + } + + /** + * Requests Oracle to build the data dictionary. + * + *

During the build step, Oracle will perform an additional series of redo log switches. + * Additionally, this call may introduce a delay in delivering incremental changes since the + * dictionary will need to have statistics gathered, analyzed, and prepared by LogMiner before + * any redo entries can be mined. + * + *

This should only be used in conjunction with the mining strategy {@link + * io.debezium.connector.oracle.OracleConnectorConfig.LogMiningStrategy#CATALOG_IN_REDO}. + * + * @param connection database connection + * @throws SQLException if a database exception occurred + */ + private void buildDataDictionary(OracleConnection connection) throws SQLException { + LOGGER.trace("Building data dictionary"); + connection.executeWithoutCommitting( + "BEGIN DBMS_LOGMNR_D.BUILD (options => DBMS_LOGMNR_D.STORE_IN_REDO_LOGS); END;"); } /** @@ -401,17 +530,6 @@ private boolean hasLogSwitchOccurred() throws SQLException { currentRedoLogSequences = newSequences; - final Map logStatuses = - jdbcConnection.queryAndMap( - SqlUtils.redoLogStatusQuery(), - rs -> { - Map results = new LinkedHashMap<>(); - while (rs.next()) { - results.put(rs.getString(1), rs.getString(2)); - } - return results; - }); - final int logSwitchCount = jdbcConnection.queryAndMap( SqlUtils.switchHistoryQuery(archiveDestinationName), @@ -421,19 +539,56 @@ private boolean hasLogSwitchOccurred() throws SQLException { } return 0; }); - - final Set fileNames = getCurrentRedoLogFiles(jdbcConnection); - - streamingMetrics.setRedoLogStatus(logStatuses); streamingMetrics.setSwitchCount(logSwitchCount); - streamingMetrics.setCurrentLogFileName(fileNames); - return true; } return false; } + /** + * Updates the redo log names and statues in the streaming metrics. + * + * @throws SQLException if a database exception occurred + */ + private void updateRedoLogMetrics() throws SQLException { + final Map logStatuses = + jdbcConnection.queryAndMap( + SqlUtils.redoLogStatusQuery(), + rs -> { + Map results = new LinkedHashMap<>(); + while (rs.next()) { + results.put(rs.getString(1), rs.getString(2)); + } + return results; + }); + + final Set fileNames = getCurrentRedoLogFiles(jdbcConnection); + streamingMetrics.setCurrentLogFileName(fileNames); + streamingMetrics.setRedoLogStatus(logStatuses); + } + + /** + * Get a list of all the CURRENT redo log file names. For Oracle RAC clusters, multiple + * filenames will be returned, one for each node that participates in the cluster. + * + * @param connection database connection, should not be {@code null} + * @return unique set of all current redo log file names, with full paths, never {@code null} + * @throws SQLException if a database exception occurred + */ + private Set getCurrentRedoLogFiles(OracleConnection connection) throws SQLException { + final Set fileNames = new HashSet<>(); + connection.query( + SqlUtils.currentRedoNameQuery(), + rs -> { + while (rs.next()) { + fileNames.add(rs.getString(1)); + } + }); + LOGGER.trace("Current redo log filenames: {}", fileNames); + return fileNames; + } + /** * Get the current redo log sequence(s). * @@ -461,6 +616,410 @@ private void pauseBetweenMiningSessions() throws InterruptedException { Metronome.sleeper(period, clock).pause(); } + /** + * Sets the NLS parameters for the mining session. + * + * @param connection database connection, should not be {@code null} + * @throws SQLException if a database exception occurred + */ + private void setNlsSessionParameters(OracleConnection connection) throws SQLException { + final String nlsSessionParameters = + "ALTER SESSION SET " + + " NLS_DATE_FORMAT = 'YYYY-MM-DD HH24:MI:SS'" + + " NLS_TIMESTAMP_FORMAT = 'YYYY-MM-DD HH24:MI:SS.FF'" + + " NLS_TIMESTAMP_TZ_FORMAT = 'YYYY-MM-DD HH24:MI:SS.FF TZH:TZM'" + + " NLS_NUMERIC_CHARACTERS = '.,'"; + + connection.executeWithoutCommitting(nlsSessionParameters); + // This is necessary so that TIMESTAMP WITH LOCAL TIME ZONE is returned in UTC + connection.executeWithoutCommitting("ALTER SESSION SET TIME_ZONE = '00:00'"); + } + + /** + * Get the database system time in the database system's time zone. + * + * @param connection database connection, should not be {@code null} + * @return the database system time + * @throws SQLException if a database exception occurred + */ + private OffsetDateTime getDatabaseSystemTime(OracleConnection connection) throws SQLException { + return connection.singleOptionalValue( + "SELECT SYSTIMESTAMP FROM DUAL", rs -> rs.getObject(1, OffsetDateTime.class)); + } + + /** + * Starts a new Oracle LogMiner session. + * + *

When this is called, LogMiner prepares all the necessary state for an upcoming LogMiner + * view query. If the mining statement defines using DDL tracking, the data dictionary will be + * mined as a part of this call to prepare DDL tracking state for the upcoming LogMiner view + * query. + * + * @param connection database connection, should not be {@code null} + * @param startScn mining session's starting system change number (exclusive), should not be + * {@code null} + * @param endScn mining session's ending system change number (inclusive), can be {@code null} + * @param attempts the number of mining start attempts + * @return true if the session was started successfully, false if it should be retried + * @throws SQLException if mining session failed to start + */ + public boolean startMiningSession( + OracleConnection connection, Scn startScn, Scn endScn, int attempts) + throws SQLException { + LOGGER.trace( + "Starting mining session startScn={}, endScn={}, strategy={}, continuous={}", + startScn, + endScn, + strategy, + isContinuousMining); + try { + Instant start = Instant.now(); + // NOTE: we treat startSCN as the _exclusive_ lower bound for mining, + // whereas START_LOGMNR takes an _inclusive_ lower bound, hence the increment. + connection.executeWithoutCommitting( + SqlUtils.startLogMinerStatement( + startScn.add(Scn.ONE), endScn, strategy, isContinuousMining)); + streamingMetrics.addCurrentMiningSessionStart(Duration.between(start, Instant.now())); + return true; + } catch (SQLException e) { + if (e.getErrorCode() == 1291 || e.getMessage().startsWith("ORA-01291")) { + if (attempts <= MINING_START_RETRIES) { + LOGGER.warn("Failed to start Oracle LogMiner session, retrying..."); + return false; + } + LOGGER.error( + "Failed to start Oracle LogMiner after '{}' attempts.", + MINING_START_RETRIES, + e); + } + LOGGER.error("Got exception when starting mining session.", e); + // Capture the database state before throwing the exception up + LogMinerDatabaseStateWriter.write(connection); + throw e; + } + } + + /** + * End the current Oracle LogMiner session, if one is in progress. If the current session does + * not have an active mining session, a log message is recorded and the method is a no-op. + * + * @param connection database connection, should not be {@code null} + * @param offsetContext connector offset context, should not be {@code null} + * @throws SQLException if the current mining session cannot be ended gracefully + */ + public void endMiningSession(OracleConnection connection, OracleOffsetContext offsetContext) + throws SQLException { + try { + LOGGER.trace( + "Ending log mining startScn={}, endScn={}, offsetContext.getScn={}, strategy={}, continuous={}", + startScn, + endScn, + offsetContext.getScn(), + strategy, + isContinuousMining); + connection.executeWithoutCommitting("BEGIN SYS.DBMS_LOGMNR.END_LOGMNR(); END;"); + } catch (SQLException e) { + if (e.getMessage().toUpperCase().contains("ORA-01307")) { + LOGGER.info("LogMiner mining session is already closed."); + return; + } + // LogMiner failed to terminate properly, a restart of the connector will be required. + throw e; + } + } + + /** + * Calculates the mining session's end system change number. + * + *

This calculation is based upon a sliding window algorithm to where if the connector is + * falling behind, the mining session's end point will be calculated based on the batch size and + * either be increased up to the maximum batch size or reduced to as low as the minimum batch + * size. + * + *

Additionally, this method calculates and maintains a sliding algorithm for the sleep time + * between the mining sessions, increasing the pause up to the maximum sleep time if the + * connector is not behind or is mining too quick and reducing the pause down to the mimum sleep + * time if the connector has fallen behind and needs to catch-up faster. + * + * @param connection database connection, should not be {@code null} + * @param startScn upcoming mining session's starting change number, should not be {@code null} + * @param prevEndScn last mining session's ending system change number, can be {@code null} + * @return the ending system change number to be used for the upcoming mining session, never + * {@code null} + * @throws SQLException if the current max system change number cannot be obtained from the + * database + */ + private Scn calculateEndScn(OracleConnection connection, Scn startScn, Scn prevEndScn) + throws SQLException { + Scn currentScn = + archiveLogOnlyMode + ? getMaxArchiveLogScn(currentLogFiles) + : connection.getCurrentScn(); + streamingMetrics.setCurrentScn(currentScn); + + // Add the current batch size to the starting system change number + final Scn currentBatchSizeScn = Scn.valueOf(streamingMetrics.getBatchSize()); + Scn topScnToMine = startScn.add(currentBatchSizeScn); + + // Control adjusting batch size + boolean topMiningScnInFarFuture = false; + if (topScnToMine.subtract(currentScn).compareTo(currentBatchSizeScn) > 0) { + streamingMetrics.changeBatchSize(false, connectorConfig.isLobEnabled()); + topMiningScnInFarFuture = true; + } + if (currentScn.subtract(topScnToMine).compareTo(currentBatchSizeScn) > 0) { + streamingMetrics.changeBatchSize(true, connectorConfig.isLobEnabled()); + } + + // Control sleep time to reduce database impact + if (currentScn.compareTo(topScnToMine) < 0) { + if (!topMiningScnInFarFuture) { + streamingMetrics.changeSleepingTime(true); + } + LOGGER.debug("Using current SCN {} as end SCN.", currentScn); + return currentScn; + } else { + if (prevEndScn != null && topScnToMine.compareTo(prevEndScn) <= 0) { + LOGGER.debug( + "Max batch size too small, using current SCN {} as end SCN.", currentScn); + return currentScn; + } + streamingMetrics.changeSleepingTime(false); + if (topScnToMine.compareTo(startScn) < 0) { + LOGGER.debug( + "Top SCN calculation resulted in end before start SCN, using current SCN {} as end SCN.", + currentScn); + return currentScn; + } + + if (prevEndScn != null) { + final Scn deltaScn = currentScn.subtract(prevEndScn); + if (deltaScn.compareTo( + Scn.valueOf( + connectorConfig.getLogMiningScnGapDetectionGapSizeMin())) + > 0) { + Optional prevEndScnTimestamp = + connection.getScnToTimestamp(prevEndScn); + if (prevEndScnTimestamp.isPresent()) { + Optional currentScnTimestamp = + connection.getScnToTimestamp(currentScn); + if (currentScnTimestamp.isPresent()) { + long timeDeltaMs = + ChronoUnit.MILLIS.between( + prevEndScnTimestamp.get(), currentScnTimestamp.get()); + if (timeDeltaMs + < connectorConfig + .getLogMiningScnGapDetectionTimeIntervalMaxMs()) { + LOGGER.warn( + "Detected possible SCN gap, using current SCN, startSCN {}, prevEndScn {} timestamp {}, current SCN {} timestamp {}.", + startScn, + prevEndScn, + prevEndScnTimestamp.get(), + currentScn, + currentScnTimestamp.get()); + return currentScn; + } + } + } + } + } + + LOGGER.debug( + "Using Top SCN calculation {} as end SCN. currentScn {}, startScn {}", + topScnToMine, + currentScn, + startScn); + return topScnToMine; + } + } + + /** + * Checks and validates the database's supplemental logging configuration as well as the lengths + * of the table and column names that are part of the database schema. + * + * @param connection database connection, should not be {@code null} + * @param pdbName pluggable database name, can be {@code null} when not using pluggable + * databases + * @param schema connector's database schema, should not be {@code null} + * @throws SQLException if a database exception occurred + */ + private void checkDatabaseAndTableState( + OracleConnection connection, String pdbName, OracleDatabaseSchema schema) + throws SQLException { + final Instant start = Instant.now(); + LOGGER.trace( + "Checking database and table state, this may take time depending on the size of your schema."); + try { + if (pdbName != null) { + connection.setSessionToPdb(pdbName); + } + + // Check if ALL supplemental logging is enabled at the database + if (!isDatabaseAllSupplementalLoggingEnabled(connection)) { + // Check if MIN supplemental logging is enabled at the database + if (!isDatabaseMinSupplementalLoggingEnabled(connection)) { + throw new DebeziumException( + "Supplemental logging not properly configured. " + + "Use: ALTER DATABASE ADD SUPPLEMENTAL LOG DATA"); + } + + // Check if ALL COLUMNS supplemental logging is enabled for each captured table + for (TableId tableId : schema.tableIds()) { + if (!connection.isTableExists(tableId)) { + LOGGER.warn( + "Database table '{}' no longer exists, supplemental log check skipped", + tableId); + } else if (!isTableAllColumnsSupplementalLoggingEnabled(connection, tableId)) { + LOGGER.warn( + "Database table '{}' not configured with supplemental logging \"(ALL) COLUMNS\"; " + + "only explicitly changed columns will be captured. " + + "Use: ALTER TABLE {}.{} ADD SUPPLEMENTAL LOG DATA (ALL) COLUMNS", + tableId, + tableId.schema(), + tableId.table()); + } + final Table table = schema.tableFor(tableId); + if (table == null) { + // This should never happen; however in the event something would cause it + // we can + // at least get the table identifier thrown in the error to debug from + // rather + // than an erroneous NPE + throw new DebeziumException( + "Unable to find table in relational model: " + tableId); + } + checkTableColumnNameLengths(table); + } + } else { + // ALL supplemental logging is enabled, now check table/column lengths + for (TableId tableId : schema.tableIds()) { + final Table table = schema.tableFor(tableId); + if (table == null) { + // This should never happen; however in the event something would cause it + // we can + // at least get the table identifier thrown in the error to debug from + // rather + // than an erroneous NPE + throw new DebeziumException( + "Unable to find table in relational model: " + tableId); + } + checkTableColumnNameLengths(table); + } + } + } finally { + if (pdbName != null) { + connection.resetSessionToCdb(); + } + } + LOGGER.trace( + "Database and table state check finished after {} ms", + Duration.between(start, Instant.now()).toMillis()); + } + + /** + * Examines the table and column names and logs a warning if any name exceeds {@link + * #MAXIMUM_NAME_LENGTH}. + * + * @param table the table, should not be {@code null} + */ + private void checkTableColumnNameLengths(Table table) { + if (table.id().table().length() > MAXIMUM_NAME_LENGTH) { + LOGGER.warn( + "Table '{}' won't be captured by Oracle LogMiner because its name exceeds {} characters.", + table.id().table(), + MAXIMUM_NAME_LENGTH); + } + for (Column column : table.columns()) { + if (column.name().length() > MAXIMUM_NAME_LENGTH) { + LOGGER.warn( + "Table '{}' won't be captured by Oracle LogMiner because column '{}' exceeds {} characters.", + table.id().table(), + column.name(), + MAXIMUM_NAME_LENGTH); + } + } + } + + /** + * Returns whether the database is configured with ALL supplemental logging. + * + * @param connection database connection, must not be {@code null} + * @return true if all supplemental logging is enabled, false otherwise + * @throws SQLException if a database exception occurred + */ + private boolean isDatabaseAllSupplementalLoggingEnabled(OracleConnection connection) + throws SQLException { + return connection.queryAndMap( + SqlUtils.databaseSupplementalLoggingAllCheckQuery(), + rs -> { + while (rs.next()) { + if ("YES".equalsIgnoreCase(rs.getString(2))) { + return true; + } + } + return false; + }); + } + + /** + * Returns whether the database is configured with MIN supplemental logging. + * + * @param connection database connection, must not be {@code null} + * @return true if min supplemental logging is enabled, false otherwise + * @throws SQLException if a database exception occurred + */ + private boolean isDatabaseMinSupplementalLoggingEnabled(OracleConnection connection) + throws SQLException { + return connection.queryAndMap( + SqlUtils.databaseSupplementalLoggingMinCheckQuery(), + rs -> { + while (rs.next()) { + if ("YES".equalsIgnoreCase(rs.getString(2))) { + return true; + } + } + return false; + }); + } + + /** + * Return whether the table is configured with ALL COLUMN supplemental logging. + * + * @param connection database connection, must not be {@code null} + * @param tableId table identifier, must not be {@code null} + * @return true if all column supplemental logging is enabled, false otherwise + * @throws SQLException if a database exception occurred + */ + private boolean isTableAllColumnsSupplementalLoggingEnabled( + OracleConnection connection, TableId tableId) throws SQLException { + // A table can be defined with multiple logging groups, hence why this check needs to + // iterate + // multiple returned rows to see whether ALL_COLUMN_LOGGING is part of the set. + return connection.queryAndMap( + SqlUtils.tableSupplementalLoggingCheckQuery(tableId), + rs -> { + while (rs.next()) { + if (ALL_COLUMN_LOGGING.equals(rs.getString(2))) { + return true; + } + } + return false; + }); + } + + /** + * Resolves the Oracle LGWR buffer flushing strategy. + * + * @return the strategy to be used to flush Oracle's LGWR process, never {@code null}. + */ + private LogWriterFlushStrategy resolveFlushStrategy() { + if (connectorConfig.isRacSystem()) { + return new RacCommitLogWriterFlushStrategy( + connectorConfig, jdbcConfiguration, streamingMetrics); + } + return new CommitLogWriterFlushStrategy(jdbcConnection); + } + /** * Waits for the starting system change number to exist in the archive logs before returning. * diff --git a/flink-connector-oracle-cdc/src/main/java/io/debezium/connector/oracle/logminer/TransactionalBuffer.java b/flink-connector-oracle-cdc/src/main/java/io/debezium/connector/oracle/logminer/TransactionalBuffer.java deleted file mode 100644 index 101f941c93..0000000000 --- a/flink-connector-oracle-cdc/src/main/java/io/debezium/connector/oracle/logminer/TransactionalBuffer.java +++ /dev/null @@ -1,1322 +0,0 @@ -/* - * Copyright 2022 Ververica Inc. - * - * Licensed 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 io.debezium.connector.oracle.logminer; - -import io.debezium.DebeziumException; -import io.debezium.annotation.NotThreadSafe; -import io.debezium.connector.oracle.BlobChunkList; -import io.debezium.connector.oracle.OracleConnectorConfig; -import io.debezium.connector.oracle.OracleDatabaseSchema; -import io.debezium.connector.oracle.OracleOffsetContext; -import io.debezium.connector.oracle.OracleStreamingChangeEventSourceMetrics; -import io.debezium.connector.oracle.Scn; -import io.debezium.connector.oracle.logminer.parser.SelectLobParser; -import io.debezium.connector.oracle.logminer.valueholder.LogMinerDmlEntry; -import io.debezium.pipeline.ErrorHandler; -import io.debezium.pipeline.EventDispatcher; -import io.debezium.pipeline.source.spi.ChangeEventSource; -import io.debezium.relational.Table; -import io.debezium.relational.TableId; -import io.debezium.util.Clock; -import org.apache.kafka.connect.errors.DataException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.sql.Timestamp; -import java.time.Duration; -import java.time.Instant; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Set; -import java.util.function.Supplier; - -/** - * Copied from https://github.com/debezium/debezium project to fix - * https://issues.redhat.com/browse/DBZ-4936 for 1.6.4.Final version. - * - *

Line 370 : Use `offsetContext.getCommitScn().compareTo(scn) >= 0` instead of - * `offsetContext.getCommitScn().compareTo(scn) > 0`. We should remove this class since we bumped - * higher debezium version after 1.9.1.Final where the issue has been fixed. - */ -@NotThreadSafe -public final class TransactionalBuffer implements AutoCloseable { - - private static final Logger LOGGER = LoggerFactory.getLogger(TransactionalBuffer.class); - - private final OracleConnectorConfig connectorConfig; - private final Map transactions; - private final OracleDatabaseSchema schema; - private final Clock clock; - private final ErrorHandler errorHandler; - private final Set abandonedTransactionIds; - private final Set rolledBackTransactionIds; - private final Set recentlyCommittedTransactionIds; - private final Set recentlyEmittedDdls; - private final OracleStreamingChangeEventSourceMetrics streamingMetrics; - - private Scn lastCommittedScn; - private Scn maxCommittedScn; - - /** - * Constructor to create a new instance. - * - * @param connectorConfig connector configuration, should not be {@code null} - * @param schema database schema - * @param clock system clock - * @param errorHandler the connector error handler - * @param streamingMetrics the streaming metrics - */ - TransactionalBuffer( - OracleConnectorConfig connectorConfig, - OracleDatabaseSchema schema, - Clock clock, - ErrorHandler errorHandler, - OracleStreamingChangeEventSourceMetrics streamingMetrics) { - this.transactions = new HashMap<>(); - this.connectorConfig = connectorConfig; - this.schema = schema; - this.clock = clock; - this.errorHandler = errorHandler; - this.lastCommittedScn = Scn.NULL; - this.maxCommittedScn = Scn.NULL; - this.abandonedTransactionIds = new HashSet<>(); - this.rolledBackTransactionIds = new HashSet<>(); - this.recentlyCommittedTransactionIds = new HashSet<>(); - this.recentlyEmittedDdls = new HashSet<>(); - this.streamingMetrics = streamingMetrics; - } - - /** @return rolled back transactions */ - Set getRolledBackTransactionIds() { - return new HashSet<>(rolledBackTransactionIds); - } - - /** - * Registers a DDL operation with the buffer. - * - * @param scn the system change number - */ - void registerDdlOperation(Scn scn) { - if (connectorConfig.isLobEnabled()) { - recentlyEmittedDdls.add(scn); - } - } - - /** - * Returns whether the ddl operation has been registered. - * - * @param scn the system change number - * @return true if the ddl operation has been seen and processed, false otherwise. - */ - boolean isDdlOperationRegistered(Scn scn) { - return recentlyEmittedDdls.contains(scn); - } - - /** - * Register a DML operation with the transaction buffer. - * - * @param operation operation type - * @param transactionId unique transaction identifier - * @param scn system change number - * @param tableId table identifier - * @param entrySupplier parser entry provider - * @param changeTime time the DML operation occurred - * @param rowId unique row identifier - * @param rsId rollback sequence identifier - */ - void registerDmlOperation( - int operation, - String transactionId, - Scn scn, - TableId tableId, - Supplier entrySupplier, - Instant changeTime, - String rowId, - Object rsId) { - if (registerEvent( - transactionId, - scn, - changeTime, - () -> new DmlEvent(operation, entrySupplier.get(), scn, tableId, rowId, rsId))) { - streamingMetrics.incrementRegisteredDmlCount(); - } - } - - /** - * Register a {@code SEL_LOB_LOCATOR} operation with the transaction buffer. - * - * @param operation operation type - * @param transactionId unique transaction identifier - * @param scn system change number - * @param tableId table identifier - * @param changeTime time the operation occurred - * @param rowId unique row identifier - * @param rsId rollback sequence identifier - * @param segOwner table owner - * @param tableName table name - * @param redoSql the redo sql statement - * @param table the relational table - * @param selectLobParser the select LOB parser - */ - void registerSelectLobOperation( - int operation, - String transactionId, - Scn scn, - TableId tableId, - Instant changeTime, - String rowId, - Object rsId, - String segOwner, - String tableName, - String redoSql, - Table table, - SelectLobParser selectLobParser) { - registerEvent( - transactionId, - scn, - changeTime, - () -> { - final LogMinerDmlEntry entry = selectLobParser.parse(redoSql, table); - entry.setObjectOwner(segOwner); - entry.setObjectName(tableName); - return new SelectLobLocatorEvent( - operation, - entry, - selectLobParser.getColumnName(), - selectLobParser.isBinary(), - scn, - tableId, - rowId, - rsId); - }); - } - - /** - * Register a {@code LOB_WRITE} operation with the transaction buffer. - * - * @param operation operation type - * @param transactionId unique transaction identifier - * @param scn system change number - * @param tableId table identifier - * @param data data written by the LOB operation - * @param changeTime time the operation occurred - * @param rowId unique row identifier - * @param rsId rollback sequence identifier - */ - void registerLobWriteOperation( - int operation, - String transactionId, - Scn scn, - TableId tableId, - String data, - Instant changeTime, - String rowId, - Object rsId) { - if (data != null) { - registerEvent( - transactionId, - scn, - changeTime, - () -> - new LobWriteEvent( - operation, parseLobWriteSql(data), scn, tableId, rowId, rsId)); - } - } - - /** - * Register a {@code LOB_ERASE} operation with the transction buffer. - * - * @param operation operation type - * @param transactionId unique transaction identifier - * @param scn system change number - * @param tableId table identifier - * @param changeTime time the operation occurred - * @param rowId unique row identifier - * @param rsId rollback sequence identifier - */ - void registerLobEraseOperation( - int operation, - String transactionId, - Scn scn, - TableId tableId, - Instant changeTime, - String rowId, - Object rsId) { - registerEvent( - transactionId, - scn, - changeTime, - () -> new LobEraseEvent(operation, scn, tableId, rowId, rsId)); - } - - /** - * Undo a staged DML operation in the transaction buffer. - * - * @param transactionId unique transaction identifier - * @param undoRowId unique row identifier to be undone - * @param tableId table identifier - */ - void undoDmlOperation(String transactionId, String undoRowId, TableId tableId) { - Transaction transaction = transactions.get(transactionId); - if (transaction == null) { - LOGGER.warn( - "Cannot undo changes to {} with row id {} as transaction {} not found.", - tableId, - undoRowId, - transactionId); - return; - } - - transaction.events.removeIf( - o -> { - if (o.getRowId().equals(undoRowId)) { - LOGGER.trace( - "Undoing change to {} with row id {} in transaction {}", - tableId, - undoRowId, - transactionId); - return true; - } - return false; - }); - } - - /** - * Register a new transaction with the transaction buffer. - * - * @param transactionId unique transaction identifier - * @param scn starting SCN of the transaction - */ - void registerTransaction(String transactionId, Scn scn) { - Transaction transaction = transactions.get(transactionId); - if (transaction == null && !isRecentlyCommitted(transactionId)) { - transactions.put(transactionId, new Transaction(transactionId, scn)); - streamingMetrics.setActiveTransactions(transactions.size()); - } else if (transaction != null && !isRecentlyCommitted(transactionId)) { - LOGGER.trace( - "Transaction {} is not yet committed and START event detected, reset eventIds.", - transactionId); - // Since the transaction hasn't been committed and the START transaction was re-mined, - // reset the event id counter for the transaction so that any events pulled from the - // event stream are added at the right index offsets. - transaction.eventIds = 0; - } - } - - /** - * Commits a transaction by looking up the transaction in the buffer and if exists, all - * registered callbacks will be executed in chronological order, emitting events for each - * followed by a transaction commit event. - * - * @param transactionId transaction identifier - * @param scn SCN of the commit. - * @param offsetContext Oracle offset - * @param timestamp commit timestamp - * @param context context to check that source is running - * @param debugMessage message - * @param dispatcher event dispatcher - * @return true if committed transaction is in the buffer, was not processed yet and processed - * now - */ - boolean commit( - String transactionId, - Scn scn, - OracleOffsetContext offsetContext, - Timestamp timestamp, - ChangeEventSource.ChangeEventSourceContext context, - String debugMessage, - EventDispatcher dispatcher) { - - Instant start = Instant.now(); - Transaction transaction = transactions.remove(transactionId); - if (transaction == null) { - return false; - } - - Scn smallestScn = calculateSmallestScn(); - - abandonedTransactionIds.remove(transactionId); - - if (isRecentlyCommitted(transactionId)) { - return false; - } - - // On the restarting connector, we start from SCN in the offset. There is possibility to - // commit a transaction(s) which were already committed. - // Currently we cannot use ">=", because we may lose normal commit which may happen at the - // same time. TODO use audit table to prevent duplications - if ((offsetContext.getCommitScn() != null - && offsetContext.getCommitScn().compareTo(scn) >= 0) - || lastCommittedScn.compareTo(scn) > 0) { - LOGGER.debug( - "Transaction {} already processed, ignored. Committed SCN in offset is {}, commit SCN of the transaction is {}, last committed SCN is {}", - transactionId, - offsetContext.getCommitScn(), - scn, - lastCommittedScn); - streamingMetrics.setActiveTransactions(transactions.size()); - return false; - } - - reconcileTransaction(transaction); - - LOGGER.trace("COMMIT, {}, smallest SCN: {}", debugMessage, smallestScn); - try { - int counter = transaction.events.size(); - for (LogMinerEvent event : transaction.events) { - if (!context.isRunning()) { - return false; - } - - // Update SCN in offset context only if processed SCN less than SCN among other - // transactions - if (smallestScn == null || scn.compareTo(smallestScn) < 0) { - offsetContext.setScn(event.getScn()); - streamingMetrics.setOldestScn(event.getScn()); - } - - offsetContext.setTransactionId(transaction.transactionId); - offsetContext.setSourceTime(timestamp.toInstant()); - offsetContext.setTableId(event.getTableId()); - if (--counter == 0) { - offsetContext.setCommitScn(scn); - } - - LOGGER.trace("Processing event {}", event); - dispatcher.dispatchDataChangeEvent( - event.getTableId(), - new LogMinerChangeRecordEmitter( - offsetContext, - event.getOperation(), - event.getEntry().getOldValues(), - event.getEntry().getNewValues(), - schema.tableFor(event.getTableId()), - clock, - event.rowId)); - } - - lastCommittedScn = Scn.valueOf(scn.longValue()); - if (!transaction.events.isEmpty()) { - dispatcher.dispatchTransactionCommittedEvent(offsetContext); - } else { - dispatcher.dispatchHeartbeatEvent(offsetContext); - } - - streamingMetrics.calculateLagMetrics(timestamp.toInstant()); - - if (lastCommittedScn.compareTo(maxCommittedScn) > 0) { - LOGGER.trace("Updated transaction buffer max commit SCN to '{}'", lastCommittedScn); - maxCommittedScn = lastCommittedScn; - } - - if (connectorConfig.isLobEnabled()) { - // cache recent transaction and commit scn for handling offset updates - recentlyCommittedTransactionIds.add( - new RecentlyCommittedTransaction(transaction, scn)); - } - } catch (InterruptedException e) { - LogMinerHelper.logError(streamingMetrics, "Commit interrupted", e); - Thread.currentThread().interrupt(); - } catch (Exception e) { - errorHandler.setProducerThrowable(e); - } finally { - streamingMetrics.incrementCommittedTransactions(); - streamingMetrics.setActiveTransactions(transactions.size()); - streamingMetrics.incrementCommittedDmlCount(transaction.events.size()); - streamingMetrics.setCommittedScn(scn); - streamingMetrics.setOffsetScn(offsetContext.getScn()); - streamingMetrics.setLastCommitDuration(Duration.between(start, Instant.now())); - } - - return true; - } - - /** - * Update the offset context based on the current state of the transaction buffer. - * - * @param offsetContext offset context, should not be {@code null} - * @param dispatcher event dispatcher, should not be {@code null} - * @return offset context SCN, never {@code null} - * @throws InterruptedException thrown if dispatch of heartbeat event fails - */ - Scn updateOffsetContext(OracleOffsetContext offsetContext, EventDispatcher dispatcher) - throws InterruptedException { - if (transactions.isEmpty()) { - if (!maxCommittedScn.isNull()) { - LOGGER.trace( - "Transaction buffer is empty, updating offset SCN to '{}'", - maxCommittedScn); - offsetContext.setScn(maxCommittedScn); - dispatcher.dispatchHeartbeatEvent(offsetContext); - } else { - LOGGER.trace( - "No max committed SCN detected, offset SCN still '{}'", - offsetContext.getScn()); - } - } else { - Scn minStartScn = getMinimumScn(); - if (!minStartScn.isNull()) { - LOGGER.trace("Removing all commits up to SCN '{}'", minStartScn); - recentlyCommittedTransactionIds.removeIf( - t -> t.firstScn.compareTo(minStartScn) < 0); - LOGGER.trace("Removing all tracked DDL operations up to SCN '{}'", minStartScn); - recentlyEmittedDdls.removeIf(scn -> scn.compareTo(minStartScn) < 0); - offsetContext.setScn(minStartScn.subtract(Scn.valueOf(1))); - dispatcher.dispatchHeartbeatEvent(offsetContext); - } else { - LOGGER.trace("Minimum SCN in transaction buffer is still SCN '{}'", minStartScn); - } - } - return offsetContext.getScn(); - } - - Scn getMinimumScn() { - return transactions.values().stream() - .map(t -> t.firstScn) - .min(Scn::compareTo) - .orElse(Scn.NULL); - } - - /** - * Clears registered callbacks for given transaction identifier. - * - * @param transactionId transaction id - * @param debugMessage message - * @return true if the rollback is for a transaction in the buffer - */ - boolean rollback(String transactionId, String debugMessage) { - - Transaction transaction = transactions.get(transactionId); - if (transaction != null) { - LOGGER.debug("Transaction rolled back: {}", debugMessage); - - transactions.remove(transactionId); - abandonedTransactionIds.remove(transactionId); - rolledBackTransactionIds.add(transactionId); - - streamingMetrics.setActiveTransactions(transactions.size()); - streamingMetrics.incrementRolledBackTransactions(); - streamingMetrics.addRolledBackTransactionId(transactionId); - - return true; - } - - return false; - } - - /** - * If for some reason the connector got restarted, the offset will point to the beginning of the - * oldest captured transaction. If that transaction was lasted for a long time, let say > 4 - * hours, the offset might be not accessible after restart, Hence we have to address these cases - * manually. - * - *

In case of an abandonment, all DMLs/Commits/Rollbacks for this transaction will be ignored - * - * @param thresholdScn the smallest SVN of any transaction to keep in the buffer. All others - * will be removed. - * @param offsetContext the offset context - */ - void abandonLongTransactions(Scn thresholdScn, OracleOffsetContext offsetContext) { - LogMinerHelper.logWarn( - streamingMetrics, - "All transactions with first SCN <= {} will be abandoned, offset: {}", - thresholdScn, - offsetContext.getScn()); - Scn threshold = Scn.valueOf(thresholdScn.toString()); - Scn smallestScn = calculateSmallestScn(); - if (smallestScn == null) { - // no transactions in the buffer - return; - } - if (threshold.compareTo(smallestScn) < 0) { - threshold = smallestScn; - } - Iterator> iter = transactions.entrySet().iterator(); - while (iter.hasNext()) { - Map.Entry transaction = iter.next(); - if (transaction.getValue().firstScn.compareTo(threshold) <= 0) { - LogMinerHelper.logWarn( - streamingMetrics, - "Following long running transaction {} will be abandoned and ignored: {} ", - transaction.getKey(), - transaction.getValue().toString()); - abandonedTransactionIds.add(transaction.getKey()); - iter.remove(); - - streamingMetrics.addAbandonedTransactionId(transaction.getKey()); - streamingMetrics.setActiveTransactions(transactions.size()); - } - } - } - - boolean isTransactionRegistered(String txId) { - return transactions.get(txId) != null; - } - - private Scn calculateSmallestScn() { - Scn scn = - transactions.isEmpty() - ? null - : transactions.values().stream() - .map(transaction -> transaction.firstScn) - .min(Scn::compareTo) - .orElseThrow( - () -> new DataException("Cannot calculate smallest SCN")); - streamingMetrics.setOldestScn(scn == null ? Scn.valueOf(-1) : scn); - return scn; - } - - /** - * Returns {@code true} if buffer is empty, otherwise {@code false}. - * - * @return {@code true} if buffer is empty, otherwise {@code false} - */ - boolean isEmpty() { - return transactions.isEmpty(); - } - - @Override - public String toString() { - StringBuilder result = new StringBuilder(); - this.transactions.values().forEach(t -> result.append(t.toString())); - return result.toString(); - } - - @Override - public void close() { - transactions.clear(); - } - - /** - * Helper method to register a given {@link LogMinerEvent} implementation with the buffer. If - * the event is registered, the underlying metrics active transactions and lag will be - * re-calculated. - * - * @param transactionId transaction id that contained the given event - * @param scn system change number for the event - * @param changeTime the time the event occurred - * @param supplier supplier function to generate the event if validity checks pass - * @return true if the event was registered, false otherwise - */ - private boolean registerEvent( - String transactionId, Scn scn, Instant changeTime, Supplier supplier) { - if (abandonedTransactionIds.contains(transactionId)) { - LogMinerHelper.logWarn( - streamingMetrics, - "Event for abandoned transaction {}, ignored.", - transactionId); - return false; - } - if (rolledBackTransactionIds.contains(transactionId)) { - LogMinerHelper.logWarn( - streamingMetrics, - "Event for rolled back transaction {}, ignored.", - transactionId); - return false; - } - if (isRecentlyCommitted(transactionId)) { - LOGGER.trace( - "Event for transaction {} skipped, transaction already committed.", - transactionId); - return false; - } - - Transaction transaction = - transactions.computeIfAbsent( - transactionId, s -> new Transaction(transactionId, scn)); - streamingMetrics.setActiveTransactions(transactions.size()); - - int eventId = transaction.eventIds++; - if (transaction.events.size() > eventId) { - // only return true if new event is added, otherwise false - return false; - } else { - // Adding new event at eventId offset - LOGGER.trace( - "Transaction {}, adding event reference at index {}", transactionId, eventId); - transaction.events.add(supplier.get()); - streamingMetrics.calculateLagMetrics(changeTime); - return true; - } - } - - /** - * Returns whether the specified transaction has recently been committed. - * - * @param transactionId the transaction identifier - * @return true if the transaction has been recently committed (seen by the connector), - * otherwise false. - */ - private boolean isRecentlyCommitted(String transactionId) { - if (recentlyCommittedTransactionIds.isEmpty()) { - return false; - } - - for (RecentlyCommittedTransaction transaction : recentlyCommittedTransactionIds) { - if (transaction.transactionId.equals(transactionId)) { - return true; - } - } - return false; - } - - /** - * Parses a {@code LOB_WRITE} operation SQL fragment. - * - * @param sql sql statement - * @return the parsed statement - * @throws DebeziumException if an unexpected SQL fragment is provided that cannot be parsed - */ - private String parseLobWriteSql(String sql) { - if (sql == null) { - return null; - } - - int start = sql.indexOf(":= '"); - if (start != -1) { - // LOB_WRITE SQL is for a CLOB field - int end = sql.lastIndexOf("'"); - return sql.substring(start + 4, end); - } - - start = sql.indexOf(":= HEXTORAW"); - if (start != -1) { - // LOB_WRITE SQL is for a BLOB field - int end = sql.lastIndexOf("'") + 2; - return sql.substring(start + 3, end); - } - - throw new DebeziumException("Unable to parse unsupported LOB_WRITE SQL: " + sql); - } - - /** - * Reconcile the specified transaction by merging multiple events that should be emitted as a - * single logical event, such as changes made to LOB column types that involve multiple events. - * - * @param transaction transaction to be reconciled, never {@code null} - */ - private void reconcileTransaction(Transaction transaction) { - // Do not perform reconciliation if LOB support is not enabled. - if (!connectorConfig.isLobEnabled()) { - return; - } - - LOGGER.trace("Reconciling transaction {}", transaction.transactionId); - LogMinerEvent prevEvent = null; - - int prevEventSize = transaction.events.size(); - for (int i = 0; i < transaction.events.size(); ) { - - final LogMinerEvent event = transaction.events.get(i); - LOGGER.trace("Processing event {}", event); - - switch (event.getOperation()) { - case RowMapper.SELECT_LOB_LOCATOR: - if (shouldMergeSelectLobLocatorEvent( - transaction, i, (SelectLobLocatorEvent) event, prevEvent)) { - continue; - } - break; - case RowMapper.INSERT: - case RowMapper.UPDATE: - if (shouldMergeDmlEvent(transaction, i, (DmlEvent) event, prevEvent)) { - continue; - } - break; - } - - ++i; - prevEvent = event; - LOGGER.trace("Previous event is now {}", prevEvent); - } - - if (transaction.events.size() != prevEventSize) { - LOGGER.trace( - "Reconciled transaction {} from {} events to {}.", - transaction.transactionId, - prevEventSize, - transaction.events.size()); - } else { - LOGGER.trace("Transaction {} event queue was unmodified.", transaction.transactionId); - } - } - - /** - * Attempts to merge the provided SEL_LOB_LOCATOR event with the previous event in the - * transaction. - * - * @param transaction transaction being processed, never {@code null} - * @param index event index being processed - * @param event event being processed, never {@code null} - * @param prevEvent previous event in the transaction, can be {@code null} - * @return true if the event is merged, false if the event was not merged. - */ - private boolean shouldMergeSelectLobLocatorEvent( - Transaction transaction, - int index, - SelectLobLocatorEvent event, - LogMinerEvent prevEvent) { - LOGGER.trace("\tDetected SelectLobLocatorEvent for column '{}'", event.getColumnName()); - - final int columnIndex = - LogMinerHelper.getColumnIndexByName( - event.getColumnName(), schema.tableFor(event.getTableId())); - - // Read and combine all LOB_WRITE events that follow SEL_LOB_LOCATOR - Object lobData = null; - final List lobWrites = - readAndCombineLobWriteEvents(transaction, index, event.isBinaryData()); - if (!lobWrites.isEmpty()) { - if (event.isBinaryData()) { - // For BLOB we pass the list of string chunks as-is to the value converter - lobData = new BlobChunkList(lobWrites); - } else { - // For CLOB we go ahead and pre-process the List into a single string. - lobData = String.join("", lobWrites); - } - } - - // Read and consume all LOB_ERASE events that follow SEL_LOB_LOCATOR - final int lobEraseEvents = readAndConsumeLobEraseEvents(transaction, index); - if (lobEraseEvents > 0) { - LOGGER.warn( - "LOB_ERASE for table '{}' column '{}' is not supported, use DML operations to manipulate LOB columns only.", - event.getTableId(), - event.getColumnName()); - if (lobWrites.isEmpty()) { - // There are no write and only erase events, discard entire SEL_LOB_LOCATOR - // To simulate this, we treat this as a "merge" op so caller doesn't modify previous - // event - transaction.events.remove(index); - return true; - } - } else if (lobEraseEvents == 0 && lobWrites.isEmpty()) { - // There were no LOB operations present, discard entire SEL_LOB_LOCATOR - // To simulate this, we treat this as a "merge" op so caller doesn't modify previous - // event - transaction.events.remove(index); - return true; - } - - // SelectLobLocatorEvent can be treated as a parent DML operation where an update occurs on - // any - // LOB-based column. In this case, the event will be treated as an UPDATE event when - // emitted. - - if (prevEvent == null) { - // There is no prior event, add column to this SelectLobLocatorEvent and don't merge. - LOGGER.trace("\tAdding column '{}' to current event", event.getColumnName()); - event.getEntry().getNewValues()[columnIndex] = lobData; - return false; - } - - if (RowMapper.INSERT == prevEvent.getOperation()) { - // Previous event is an INSERT operation. - // Only merge the SEL_LOB_LOCATOR event if the previous INSERT is for the same table/row - // and if the INSERT's column value is EMPTY_CLOB() or EMPTY_BLOB() - if (isForSameTableOrScn(event, prevEvent)) { - LOGGER.trace("\tMerging SEL_LOB_LOCATOR with previous INSERT event"); - Object prevValue = prevEvent.getEntry().getNewValues()[columnIndex]; - if (!"EMPTY_CLOB()".equals(prevValue) && !"EMPTY_BLOB()".equals(prevValue)) { - throw new DebeziumException( - "Expected to find column '" - + event.getColumnName() - + "' in table '" - + prevEvent.getTableId() - + "' to be initialized as an empty LOB value.'"); - } - - prevEvent.getEntry().getNewValues()[columnIndex] = lobData; - - // Remove the SEL_LOB_LOCATOR event from event list and indicate merged. - transaction.events.remove(index); - return true; - } - } else if (RowMapper.UPDATE == prevEvent.getOperation()) { - // Previous event is an UPDATE operation. - // Only merge the SEL_LOB_LOCATOR event if the previous UPDATE is for the same table/row - if (isForSameTableOrScn(event, prevEvent) && isSameTableRow(event, prevEvent)) { - LOGGER.trace( - "\tUpdating SEL_LOB_LOCATOR column '{}' to previous UPDATE event", - event.getColumnName()); - prevEvent.getEntry().getNewValues()[columnIndex] = lobData; - - // Remove the SEL_LOB_LOCATOR event from event list and indicate merged. - transaction.events.remove(index); - return true; - } - } else if (RowMapper.SELECT_LOB_LOCATOR == prevEvent.getOperation()) { - // Previous event is a SEL_LOB_LOCATOR operation. - // Only merge the two SEL_LOB_LOCATOR events if they're for the same table/row - if (isForSameTableOrScn(event, prevEvent) && isSameTableRow(event, prevEvent)) { - LOGGER.trace( - "\tAdding column '{}' to previous SEL_LOB_LOCATOR event", - event.getColumnName()); - prevEvent.getEntry().getNewValues()[columnIndex] = lobData; - - // Remove the SEL_LOB_LOCATOR event from event list and indicate merged. - transaction.events.remove(index); - return true; - } - } else { - throw new DebeziumException( - "Unexpected previous event operation: " + prevEvent.getOperation()); - } - - LOGGER.trace("\tSEL_LOB_LOCATOR event is for different row, merge skipped."); - LOGGER.trace("\tAdding column '{}' to current event", event.getColumnName()); - event.getEntry().getNewValues()[columnIndex] = lobData; - return false; - } - - /** - * Attempts to merge the provided DML event with the previous event in the transaction. - * - * @param transaction transaction being processed, never {@code null} - * @param index event index being processed - * @param event event being processed, never {@code null} - * @param prevEvent previous event in the transaction, can be {@code null} - * @return true if the event is merged, false if the event was not merged - */ - private boolean shouldMergeDmlEvent( - Transaction transaction, int index, DmlEvent event, LogMinerEvent prevEvent) { - LOGGER.trace("\tDetected DmlEvent {}", event.getOperation()); - - if (prevEvent == null) { - // There is no prior event, therefore there is no reason to perform any merge. - return false; - } - - if (RowMapper.INSERT == prevEvent.getOperation()) { - // Previous event is an INSERT operation. - // The only valid combination here would be if the current event is an UPDATE since an - // INSERT cannot - // be merged with a prior INSERT with how LogMiner materializes the rows. - if (RowMapper.UPDATE == event.getOperation()) { - if (isForSameTableOrScn(event, prevEvent) && isSameTableRow(event, prevEvent)) { - LOGGER.trace("\tMerging UPDATE event with previous INSERT event"); - mergeNewColumns(event, prevEvent); - - // Remove the UPDATE event from event list and indicate merged. - transaction.events.remove(index); - return true; - } - } - } else if (RowMapper.UPDATE == prevEvent.getOperation()) { - // Previous event is an UPDATE operation. - // This will happen if there are non-CLOB and inline-CLOB fields updated in the same - // SQL. - // The inline-CLOB values should be merged with the previous UPDATE event. - if (RowMapper.UPDATE == event.getOperation()) { - if (isForSameTableOrScn(event, prevEvent) && isSameTableRow(event, prevEvent)) { - LOGGER.trace("\tMerging UPDATE event with previous UPDATE event"); - mergeNewColumns(event, prevEvent); - - // Remove the UPDATE event from event list and indicate merged. - transaction.events.remove(index); - return true; - } - } - } else if (RowMapper.SELECT_LOB_LOCATOR == prevEvent.getOperation()) { - // Previous event is a SEL_LOB_LOCATOR operation. - // SQL contained both non-inline CLOB and inline-CLOB field changes. - if (RowMapper.UPDATE == event.getOperation()) { - if (isForSameTableOrScn(event, prevEvent) && isSameTableRow(event, prevEvent)) { - LOGGER.trace("\tMerging UPDATE event with previous SEL_LOB_LOCATOR event"); - for (int i = 0; i < event.getEntry().getNewValues().length; ++i) { - Object value = event.getEntry().getNewValues()[i]; - Object prevValue = prevEvent.getEntry().getNewValues()[i]; - if (prevValue == null && value != null) { - LOGGER.trace( - "\tAdding column index {} to previous SEL_LOB_LOCATOR event", - i); - prevEvent.getEntry().getNewValues()[i] = value; - } - } - - // Remove the UPDATE event from event list and indicate merged. - transaction.events.remove(index); - return true; - } - } - } - - LOGGER.trace( - "\tDmlEvent {} event is for different row, merge skipped.", event.getOperation()); - return false; - } - - /** - * Reads the transaction event queue and combines all LOB_WRITE events starting at the provided - * index. for a SEL_LOB_LOCATOR event which is for binary data (BLOB) data types. - * - * @param transaction transaction being processed, never {@code null} - * @param index index to the first LOB_WRITE operation - * @return list of string-based values for each LOB_WRITE operation - */ - private List readAndCombineLobWriteEvents( - Transaction transaction, int index, boolean binaryData) { - List chunks = new ArrayList<>(); - for (int i = index + 1; i < transaction.events.size(); ++i) { - final LogMinerEvent event = transaction.events.get(i); - if (!(event instanceof LobWriteEvent)) { - break; - } - - final LobWriteEvent writeEvent = (LobWriteEvent) event; - if (binaryData - && !writeEvent.getData().startsWith("HEXTORAW('") - && !writeEvent.getData().endsWith("')")) { - throw new DebeziumException("Unexpected BLOB data chunk: " + writeEvent.getData()); - } - - chunks.add(writeEvent.getData()); - } - - if (!chunks.isEmpty()) { - LOGGER.trace("\tCombined {} LobWriteEvent events", chunks.size()); - // Remove events from the transaction queue queue - for (int i = 0; i < chunks.size(); ++i) { - transaction.events.remove(index + 1); - } - } - - return chunks; - } - - /** - * Read and remove all LobErase events detected in the transaction event queue. - * - * @param transaction transaction being processed, never {@code null} - * @param index index to the first LOB_ERASE operation - * @return number of LOB_ERASE events consumed and removed from the event queue - */ - private int readAndConsumeLobEraseEvents(Transaction transaction, int index) { - int events = 0; - for (int i = index + 1; i < transaction.events.size(); ++i) { - final LogMinerEvent event = transaction.events.get(i); - if (!(event instanceof LobEraseEvent)) { - break; - } - events++; - } - - if (events > 0) { - LOGGER.trace("\tConsumed {} LobErase events", events); - for (int i = 0; i < events; ++i) { - transaction.events.remove(index + 1); - } - } - - return events; - } - - /** - * Checks whether the two events are for the same table or participate in the same system - * change. - * - * @param event current event being processed, never {@code null} - * @param prevEvent previous/parent event that has been processed, may be {@code null} - * @return true if the two events are for the same table or system change number, false - * otherwise - */ - private boolean isForSameTableOrScn(LogMinerEvent event, LogMinerEvent prevEvent) { - if (prevEvent != null) { - if (event.getTableId().equals(prevEvent.getTableId())) { - return true; - } - return event.getScn().equals(prevEvent.getScn()) - && event.getRsId().equals(prevEvent.getRsId()); - } - return false; - } - - /** - * Checks whether the two events are for the same table row. - * - * @param event current event being processed, never {@code null} - * @param prevEvent previous/parent event that has been processed, never {@code null} - * @return true if the two events are for the same table row, false otherwise - */ - private boolean isSameTableRow(LogMinerEvent event, LogMinerEvent prevEvent) { - final Table table = schema.tableFor(event.getTableId()); - if (table == null) { - LOGGER.trace( - "Unable to locate table '{}' schema, unable to detect if same row.", - event.getTableId()); - return false; - } - for (String columnName : table.primaryKeyColumnNames()) { - int position = LogMinerHelper.getColumnIndexByName(columnName, table); - Object prevValue = prevEvent.getEntry().getNewValues()[position]; - if (prevValue == null) { - throw new DebeziumException( - "Could not find column " + columnName + " in previous event"); - } - Object value = event.getEntry().getNewValues()[position]; - if (value == null) { - throw new DebeziumException("Could not find column " + columnName + " in event"); - } - if (!Objects.equals(value, prevValue)) { - return false; - } - } - return true; - } - - /** - * Merge column values from {@code event} with {@code prevEvent}. - * - * @param event current event being processed, never {@code null} - * @param prevEvent previous/parent parent that has been processed, never {@code null} - */ - private void mergeNewColumns(LogMinerEvent event, LogMinerEvent prevEvent) { - final boolean prevEventIsInsert = RowMapper.INSERT == prevEvent.getOperation(); - - for (int i = 0; i < event.getEntry().getNewValues().length; ++i) { - Object value = event.getEntry().getNewValues()[i]; - Object prevValue = prevEvent.getEntry().getNewValues()[i]; - if (prevEventIsInsert && "EMPTY_CLOB()".equals(prevValue)) { - LOGGER.trace("\tAssigning column index {} with updated CLOB value.", i); - prevEvent.getEntry().getNewValues()[i] = value; - } else if (prevEventIsInsert && "EMPTY_BLOB()".equals(prevValue)) { - LOGGER.trace("\tAssigning column index {} with updated BLOB value.", i); - prevEvent.getEntry().getNewValues()[i] = value; - } else if (!prevEventIsInsert && value != null) { - LOGGER.trace("\tUpdating column index {} in previous event", i); - prevEvent.getEntry().getNewValues()[i] = value; - } - } - } - - /** - * Represents a transaction boundary that was recently committed. - * - *

This is used by the buffer to detect transactions read from overlapping mining sessions - * that can safely be ignored as the connector has already reconciled and emitted the event for - * it. - */ - private static final class RecentlyCommittedTransaction { - private final String transactionId; - private final Scn firstScn; - private final Scn commitScn; - - public RecentlyCommittedTransaction(Transaction transaction, Scn commitScn) { - this.transactionId = transaction.transactionId; - this.firstScn = transaction.firstScn; - this.commitScn = commitScn; - } - - public Scn getFirstScn() { - return firstScn; - } - - public Scn getCommitScn() { - return commitScn; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - RecentlyCommittedTransaction that = (RecentlyCommittedTransaction) o; - return Objects.equals(transactionId, that.transactionId) - && Objects.equals(firstScn, that.firstScn) - && Objects.equals(commitScn, that.commitScn); - } - - @Override - public int hashCode() { - return Objects.hash(transactionId, firstScn, commitScn); - } - } - - /** Represents a logical database transaction. */ - private static final class Transaction { - - private final String transactionId; - private final Scn firstScn; - private Scn lastScn; - private final List events; - private int eventIds; - - private Transaction(String transactionId, Scn firstScn) { - this.transactionId = transactionId; - this.firstScn = firstScn; - this.events = new ArrayList<>(); - this.lastScn = firstScn; - this.eventIds = 0; - } - - @Override - public String toString() { - return "Transaction{" - + "transactionId=" - + transactionId - + ", firstScn=" - + firstScn - + ", lastScn=" - + lastScn - + ", eventIds=" - + eventIds - + '}'; - } - } - - /** Base class for all possible LogMiner events. */ - private static class LogMinerEvent { - private final int operation; - private final LogMinerDmlEntry entry; - private final Scn scn; - private final TableId tableId; - private final String rowId; - private final Object rsId; - - public LogMinerEvent( - int operation, - LogMinerDmlEntry entry, - Scn scn, - TableId tableId, - String rowId, - Object rsId) { - this.operation = operation; - this.scn = scn; - this.tableId = tableId; - this.rowId = rowId; - this.rsId = rsId; - this.entry = entry; - } - - public int getOperation() { - return operation; - } - - public LogMinerDmlEntry getEntry() { - return entry; - } - - public Scn getScn() { - return scn; - } - - public TableId getTableId() { - return tableId; - } - - public String getRowId() { - return rowId; - } - - public Object getRsId() { - return rsId; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - LogMinerEvent event = (LogMinerEvent) o; - return operation == event.operation - && Objects.equals(entry, event.entry) - && Objects.equals(scn, event.scn) - && Objects.equals(tableId, event.tableId) - && Objects.equals(rowId, event.rowId) - && Objects.equals(rsId, event.rsId); - } - - @Override - public int hashCode() { - return Objects.hash(operation, entry, scn, tableId, rowId, rsId); - } - } - - /** Represents a DML event for a given table row. */ - private static class DmlEvent extends LogMinerEvent { - public DmlEvent( - int operation, - LogMinerDmlEntry entry, - Scn scn, - TableId tableId, - String rowId, - Object rsId) { - super(operation, entry, scn, tableId, rowId, rsId); - } - } - - /** Represents a SELECT_LOB_LOCATOR event. */ - private static class SelectLobLocatorEvent extends LogMinerEvent { - private final String columnName; - private final boolean binaryData; - - public SelectLobLocatorEvent( - int operation, - LogMinerDmlEntry entry, - String columnName, - boolean binaryData, - Scn scn, - TableId tableId, - String rowId, - Object rsId) { - super(operation, entry, scn, tableId, rowId, rsId); - this.columnName = columnName; - this.binaryData = binaryData; - } - - public String getColumnName() { - return columnName; - } - - public boolean isBinaryData() { - return binaryData; - } - } - - /** Represents a LOB_WRITE event. */ - private static class LobWriteEvent extends LogMinerEvent { - private final String data; - - public LobWriteEvent( - int operation, String data, Scn scn, TableId tableId, String rowId, Object rsId) { - super(operation, null, scn, tableId, rowId, rsId); - this.data = data; - } - - public String getData() { - return data; - } - } - - /** Represents a LOB_ERASE event. */ - private static class LobEraseEvent extends LogMinerEvent { - public LobEraseEvent(int operation, Scn scn, TableId tableId, String rowId, Object rsId) { - super(operation, null, scn, tableId, rowId, rsId); - } - } -} diff --git a/flink-connector-oracle-cdc/src/main/java/io/debezium/connector/oracle/logminer/processor/AbstractLogMinerEventProcessor.java b/flink-connector-oracle-cdc/src/main/java/io/debezium/connector/oracle/logminer/processor/AbstractLogMinerEventProcessor.java new file mode 100644 index 0000000000..af868c9bb9 --- /dev/null +++ b/flink-connector-oracle-cdc/src/main/java/io/debezium/connector/oracle/logminer/processor/AbstractLogMinerEventProcessor.java @@ -0,0 +1,1235 @@ +/* + * Copyright Debezium Authors. + * + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 + */ + +package io.debezium.connector.oracle.logminer.processor; + +import io.debezium.DebeziumException; +import io.debezium.connector.oracle.OracleConnection; +import io.debezium.connector.oracle.OracleConnection.NonRelationalTableException; +import io.debezium.connector.oracle.OracleConnectorConfig; +import io.debezium.connector.oracle.OracleDatabaseSchema; +import io.debezium.connector.oracle.OracleOffsetContext; +import io.debezium.connector.oracle.OraclePartition; +import io.debezium.connector.oracle.OracleSchemaChangeEventEmitter; +import io.debezium.connector.oracle.OracleStreamingChangeEventSourceMetrics; +import io.debezium.connector.oracle.Scn; +import io.debezium.connector.oracle.logminer.LogMinerChangeRecordEmitter; +import io.debezium.connector.oracle.logminer.events.DmlEvent; +import io.debezium.connector.oracle.logminer.events.EventType; +import io.debezium.connector.oracle.logminer.events.LobEraseEvent; +import io.debezium.connector.oracle.logminer.events.LobWriteEvent; +import io.debezium.connector.oracle.logminer.events.LogMinerEvent; +import io.debezium.connector.oracle.logminer.events.LogMinerEventRow; +import io.debezium.connector.oracle.logminer.events.SelectLobLocatorEvent; +import io.debezium.connector.oracle.logminer.events.TruncateEvent; +import io.debezium.connector.oracle.logminer.parser.DmlParserException; +import io.debezium.connector.oracle.logminer.parser.LogMinerDmlEntry; +import io.debezium.connector.oracle.logminer.parser.LogMinerDmlEntryImpl; +import io.debezium.connector.oracle.logminer.parser.LogMinerDmlParser; +import io.debezium.connector.oracle.logminer.parser.SelectLobParser; +import io.debezium.data.Envelope; +import io.debezium.pipeline.EventDispatcher; +import io.debezium.pipeline.source.spi.ChangeEventSource.ChangeEventSourceContext; +import io.debezium.relational.Table; +import io.debezium.relational.TableId; +import io.debezium.util.Clock; +import io.debezium.util.Strings; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.time.Duration; +import java.time.Instant; +import java.time.ZoneOffset; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; +import java.util.function.Supplier; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * Copied from Debezium 1.9.7.Final. + * + *

An abstract implementation of {@link LogMinerEventProcessor} that all processors should + * extend. + * + *

Lines 411, 423: pass ROWID to LogMinerChangeRecordEmitter. + */ +public abstract class AbstractLogMinerEventProcessor + implements LogMinerEventProcessor { + + private static final Logger LOGGER = + LoggerFactory.getLogger(AbstractLogMinerEventProcessor.class); + private static final String NO_SEQUENCE_TRX_ID_SUFFIX = "ffffffff"; + + private final ChangeEventSourceContext context; + private final OracleConnectorConfig connectorConfig; + private final OracleDatabaseSchema schema; + private final OraclePartition partition; + private final OracleOffsetContext offsetContext; + private final EventDispatcher dispatcher; + private final OracleStreamingChangeEventSourceMetrics metrics; + private final LogMinerDmlParser dmlParser; + private final SelectLobParser selectLobParser; + + protected final Counters counters; + + private Scn currentOffsetScn = Scn.NULL; + private Map currentOffsetCommitScns = new HashMap<>(); + private Scn lastProcessedScn = Scn.NULL; + private boolean sequenceUnavailable = false; + + public AbstractLogMinerEventProcessor( + ChangeEventSourceContext context, + OracleConnectorConfig connectorConfig, + OracleDatabaseSchema schema, + OraclePartition partition, + OracleOffsetContext offsetContext, + EventDispatcher dispatcher, + OracleStreamingChangeEventSourceMetrics metrics) { + this.context = context; + this.connectorConfig = connectorConfig; + this.schema = schema; + this.partition = partition; + this.offsetContext = offsetContext; + this.dispatcher = dispatcher; + this.metrics = metrics; + this.counters = new Counters(); + this.dmlParser = new LogMinerDmlParser(); + this.selectLobParser = new SelectLobParser(); + } + + protected OracleConnectorConfig getConfig() { + return connectorConfig; + } + + protected OracleDatabaseSchema getSchema() { + return schema; + } + + /** + * Check whether a transaction has been recently processed through either a commit or rollback. + * + * @param transactionId the unique transaction id + * @return true if the transaction has been recently processed, false otherwise + */ + protected boolean isRecentlyProcessed(String transactionId) { + return false; + } + + /** + * Checks whether the LogMinerEvent row for a schema change can be emitted. + * + * @param row the result set row + * @return true if the schema change has been seen, false otherwise. + */ + protected boolean hasSchemaChangeBeenSeen(LogMinerEventRow row) { + return false; + } + + /** + * Return the last processed system change number handled by the processor. + * + * @return the last processed system change number, never {@code null}. + */ + protected Scn getLastProcessedScn() { + return lastProcessedScn; + } + + /** + * Returns the {@code TransactionCache} implementation. + * + * @return the transaction cache, never {@code null} + */ + protected abstract Map getTransactionCache(); + + /** + * Creates a new transaction based on the supplied {@code START} event. + * + * @param row the event row, must not be {@code null} + * @return the implementation-specific {@link Transaction} instance + */ + protected abstract T createTransaction(LogMinerEventRow row); + + /** + * Removes a specific transaction event by database row identifier. + * + * @param row the event row that contains the row identifier, must not be {@code null} + */ + protected abstract void removeEventWithRowId(LogMinerEventRow row); + + /** + * Returns the number of events associated with the specified transaction. + * + * @param transaction the transaction, must not be {@code null} + * @return the number of events in the transaction + */ + protected abstract int getTransactionEventCount(T transaction); + + // todo: can this be removed in favor of a single implementation? + protected boolean isTrxIdRawValue() { + return true; + } + + @Override + public Scn process(OraclePartition partition, Scn startScn, Scn endScn) + throws SQLException, InterruptedException { + counters.reset(); + + try (PreparedStatement statement = createQueryStatement()) { + LOGGER.debug("Fetching results for SCN [{}, {}]", startScn, endScn); + statement.setFetchSize(getConfig().getLogMiningViewFetchSize()); + statement.setFetchDirection(ResultSet.FETCH_FORWARD); + statement.setString(1, startScn.toString()); + statement.setString(2, endScn.toString()); + + Instant queryStart = Instant.now(); + try (ResultSet resultSet = statement.executeQuery()) { + metrics.setLastDurationOfBatchCapturing( + Duration.between(queryStart, Instant.now())); + + Instant startProcessTime = Instant.now(); + processResults(this.partition, resultSet); + + Duration totalTime = Duration.between(startProcessTime, Instant.now()); + metrics.setLastCapturedDmlCount(counters.dmlCount); + + if (counters.dmlCount > 0 + || counters.commitCount > 0 + || counters.rollbackCount > 0) { + warnPotentiallyStuckScn(currentOffsetScn, currentOffsetCommitScns); + + currentOffsetScn = offsetContext.getScn(); + if (offsetContext.getCommitScn() != null) { + currentOffsetCommitScns = + offsetContext.getCommitScn().getCommitScnForAllRedoThreads(); + } + } + + LOGGER.debug("{}.", counters); + LOGGER.debug( + "Processed in {} ms. Lag: {}. Offset SCN: {}, Offset Commit SCN: {}, Active Transactions: {}, Sleep: {}", + totalTime.toMillis(), + metrics.getLagFromSourceInMilliseconds(), + offsetContext.getScn(), + offsetContext.getCommitScn(), + metrics.getNumberOfActiveTransactions(), + metrics.getMillisecondToSleepBetweenMiningQuery()); + + metrics.addProcessedRows(counters.rows); + return calculateNewStartScn( + endScn, offsetContext.getCommitScn().getMaxCommittedScn()); + } + } + } + + /** + * Create the JDBC query that will be used to fetch the mining result set. + * + * @return a prepared query statement, never {@code null} + * @throws SQLException if a database exception occurred creating the statement + */ + protected abstract PreparedStatement createQueryStatement() throws SQLException; + + /** + * Calculates the new starting system change number based on the current processing range. + * + * @param endScn the end system change number for the previously mined range, never {@code null} + * @param maxCommittedScn the maximum committed system change number, never {@code null} + * @return the system change number to start then next mining iteration from, never {@code null} + * @throws InterruptedException if the current thread is interrupted + */ + protected abstract Scn calculateNewStartScn(Scn endScn, Scn maxCommittedScn) + throws InterruptedException; + + /** + * Processes the LogMiner results. + * + * @param resultSet the result set from a LogMiner query + * @throws SQLException if a database exception occurred + * @throws InterruptedException if the dispatcher was interrupted sending an event + */ + protected void processResults(OraclePartition partition, ResultSet resultSet) + throws SQLException, InterruptedException { + while (context.isRunning() && hasNextWithMetricsUpdate(resultSet)) { + counters.rows++; + processRow( + partition, + LogMinerEventRow.fromResultSet( + resultSet, getConfig().getCatalogName(), isTrxIdRawValue())); + } + } + + /** + * Processes a single LogMinerEventRow. + * + * @param row the event row, must not be {@code null} + * @throws SQLException if a database exception occurred + * @throws InterruptedException if the dispatcher was interrupted sending an event + */ + protected void processRow(OraclePartition partition, LogMinerEventRow row) + throws SQLException, InterruptedException { + if (!row.getEventType().equals(EventType.MISSING_SCN)) { + lastProcessedScn = row.getScn(); + } + // filter out all events that are captured as part of the initial snapshot + if (row.getScn().compareTo(offsetContext.getSnapshotScn()) < 0) { + Map snapshotPendingTransactions = + offsetContext.getSnapshotPendingTransactions(); + if (snapshotPendingTransactions == null + || !snapshotPendingTransactions.containsKey(row.getTransactionId())) { + LOGGER.debug( + "Skipping event {} (SCN {}) because it is already encompassed by the initial snapshot", + row.getEventType(), + row.getScn()); + return; + } + } + switch (row.getEventType()) { + case MISSING_SCN: + handleMissingScn(row); + case START: + handleStart(row); + break; + case COMMIT: + handleCommit(partition, row); + break; + case ROLLBACK: + handleRollback(row); + break; + case DDL: + handleSchemaChange(row); + break; + case SELECT_LOB_LOCATOR: + handleSelectLobLocator(row); + break; + case LOB_WRITE: + handleLobWrite(row); + break; + case LOB_ERASE: + handleLobErase(row); + break; + case INSERT: + case UPDATE: + case DELETE: + handleDataEvent(row); + break; + case UNSUPPORTED: + handleUnsupportedEvent(row); + break; + } + } + + /** + * Handle processing a LogMinerEventRow for a {@code MISSING_SCN} event. + * + * @param row the result set row + */ + protected void handleMissingScn(LogMinerEventRow row) { + LOGGER.warn("Missing SCN detected. {}", row); + } + + /** + * Handle processing a LogMinerEventRow for a {@code START} event. + * + * @param row the result set row + */ + protected void handleStart(LogMinerEventRow row) { + final String transactionId = row.getTransactionId(); + final AbstractTransaction transaction = getTransactionCache().get(transactionId); + if (transaction == null && !isRecentlyProcessed(transactionId)) { + getTransactionCache().put(transactionId, createTransaction(row)); + metrics.setActiveTransactions(getTransactionCache().size()); + } else if (transaction != null && !isRecentlyProcessed(transactionId)) { + LOGGER.trace( + "Transaction {} is not yet committed and START event detected.", transactionId); + transaction.start(); + } + } + + /** + * Handle processing a LogMinerEventRow for a {@code COMMIT} event. + * + * @param row the result set row + * @throws InterruptedException if the event dispatcher was interrupted sending events + */ + protected void handleCommit(OraclePartition partition, LogMinerEventRow row) + throws InterruptedException { + final String transactionId = row.getTransactionId(); + if (isRecentlyProcessed(transactionId)) { + LOGGER.debug("\tTransaction is already committed, skipped."); + return; + } + + final T transaction = getAndRemoveTransactionFromCache(transactionId); + if (transaction == null) { + LOGGER.trace("Transaction {} not found, commit skipped.", transactionId); + return; + } + + // Calculate the smallest SCN that remains in the transaction cache + final Scn smallestScn = getTransactionCacheMinimumScn(); + metrics.setOldestScn(smallestScn.isNull() ? Scn.valueOf(-1) : smallestScn); + + final Scn commitScn = row.getScn(); + if (offsetContext.getCommitScn().hasCommitAlreadyBeenHandled(row)) { + final Scn lastCommittedScn = + offsetContext.getCommitScn().getCommitScnForRedoThread(row.getThread()); + LOGGER.debug( + "Transaction {} has already been processed. " + + "Offset Commit SCN {}, Transaction Commit SCN {}, Last Seen Commit SCN {}.", + transactionId, + offsetContext.getCommitScn(), + commitScn, + lastCommittedScn); + removeTransactionAndEventsFromCache(transaction); + metrics.setActiveTransactions(getTransactionCache().size()); + return; + } + + counters.commitCount++; + + int numEvents = getTransactionEventCount(transaction); + LOGGER.trace("Commit (smallest SCN {}) {}", smallestScn, row); + LOGGER.trace("Transaction {} has {} events", transactionId, numEvents); + + final ZoneOffset databaseOffset = metrics.getDatabaseOffset(); + + final boolean skipExcludedUserName = isTransactionUserExcluded(transaction); + TransactionCommitConsumer.Handler delegate = + new TransactionCommitConsumer.Handler() { + private int numEvents = getTransactionEventCount(transaction); + + @Override + public void accept(LogMinerEvent event, long eventsProcessed) + throws InterruptedException { + // Update SCN in offset context only if processed SCN less than SCN of other + // transactions + if (smallestScn.isNull() || commitScn.compareTo(smallestScn) < 0) { + offsetContext.setScn(event.getScn()); + metrics.setOldestScn(event.getScn()); + } + + offsetContext.setEventScn(event.getScn()); + offsetContext.setTransactionId(transactionId); + offsetContext.setSourceTime( + event.getChangeTime() + .minusSeconds(databaseOffset.getTotalSeconds())); + offsetContext.setTableId(event.getTableId()); + offsetContext.setRedoThread(row.getThread()); + if (eventsProcessed == numEvents) { + // reached the last event update the commit scn in the offsets + offsetContext.getCommitScn().recordCommit(row); + } + + final DmlEvent dmlEvent = (DmlEvent) event; + if (!skipExcludedUserName) { + LogMinerChangeRecordEmitter logMinerChangeRecordEmitter; + if (dmlEvent instanceof TruncateEvent) { + // a truncate event is seen by logminer as a DDL event type. + // So force this here to be a Truncate Operation. + logMinerChangeRecordEmitter = + new LogMinerChangeRecordEmitter( + connectorConfig, + partition, + offsetContext, + Envelope.Operation.TRUNCATE, + dmlEvent.getDmlEntry().getOldValues(), + dmlEvent.getDmlEntry().getNewValues(), + getSchema().tableFor(event.getTableId()), + getSchema(), + Clock.system(), + dmlEvent.getRowId()); + } else { + logMinerChangeRecordEmitter = + new LogMinerChangeRecordEmitter( + connectorConfig, + partition, + offsetContext, + dmlEvent.getEventType(), + dmlEvent.getDmlEntry().getOldValues(), + dmlEvent.getDmlEntry().getNewValues(), + getSchema().tableFor(event.getTableId()), + getSchema(), + Clock.system(), + dmlEvent.getRowId()); + } + dispatcher.dispatchDataChangeEvent( + partition, event.getTableId(), logMinerChangeRecordEmitter); + } + } + }; + + Instant start = Instant.now(); + int dispatchedEventCount = 0; + if (numEvents > 0) { + try (TransactionCommitConsumer commitConsumer = + new TransactionCommitConsumer(delegate, connectorConfig, schema)) { + final Iterator iterator = getTransactionEventIterator(transaction); + while (iterator.hasNext()) { + if (!context.isRunning()) { + return; + } + + final LogMinerEvent event = iterator.next(); + LOGGER.trace( + "Dispatching event {} {}", + ++dispatchedEventCount, + event.getEventType()); + commitConsumer.accept(event); + } + } + } + + offsetContext.setEventScn(commitScn); + if (getTransactionEventCount(transaction) > 0 && !skipExcludedUserName) { + dispatcher.dispatchTransactionCommittedEvent(partition, offsetContext); + } else { + dispatcher.dispatchHeartbeatEvent(partition, offsetContext); + } + + metrics.calculateLagMetrics(row.getChangeTime()); + + finalizeTransactionCommit(transactionId, commitScn); + removeTransactionAndEventsFromCache(transaction); + + metrics.incrementCommittedTransactions(); + metrics.setActiveTransactions(getTransactionCache().size()); + metrics.incrementCommittedDmlCount(dispatchedEventCount); + metrics.setCommittedScn(commitScn); + metrics.setOffsetScn(offsetContext.getScn()); + metrics.setLastCommitDuration(Duration.between(start, Instant.now())); + } + + /** + * Gets a transaction instance from the transaction cache while also removing its cache entry. + * + * @param transactionId the transaction's unique identifier, should not be {@code null} + * @return the transaction instance if found, {@code null} if the transaction wasn't found + */ + protected abstract T getAndRemoveTransactionFromCache(String transactionId); + + /** + * Removes the transaction and all its associated event entries from the connector's caches. + * + * @param transaction the transaction instance, should never be {@code null} + */ + protected abstract void removeTransactionAndEventsFromCache(T transaction); + + /** + * Get an iterator over the events that are part of the specified transaction. + * + * @param transaction the transaction instance, should never be {@code null} + * @return an iterator over the transaction's events, never {@code null} + */ + protected abstract Iterator getTransactionEventIterator(T transaction); + + /** + * Finalizes the commit of a transaction. + * + * @param transactionId the transaction's unique identifier, should not be {@code null} + * @param commitScn the transaction's system change number, should not be {@code null} + */ + protected abstract void finalizeTransactionCommit(String transactionId, Scn commitScn); + + /** + * Check whether the supplied username associated with the specified transaction is excluded. + * + * @param transaction the transaction, never {@code null} + * @return true if the transaction should be skipped; false if transaction should be emitted + */ + protected boolean isTransactionUserExcluded(T transaction) { + if (transaction != null) { + if (transaction.getUserName() == null && getTransactionEventCount(transaction) > 0) { + LOGGER.debug("Detected transaction with null username {}", transaction); + return false; + } else if (connectorConfig + .getLogMiningUsernameExcludes() + .contains(transaction.getUserName())) { + LOGGER.trace("Skipped transaction with excluded username {}", transaction); + return true; + } + } + return false; + } + + /** + * Handle processing a LogMinerEventRow for a {@code ROLLBACK} event. + * + * @param row the result set row + */ + protected void handleRollback(LogMinerEventRow row) { + if (getTransactionCache().containsKey(row.getTransactionId())) { + LOGGER.trace("Transaction {} was rolled back.", row.getTransactionId()); + finalizeTransactionRollback(row.getTransactionId(), row.getScn()); + metrics.setActiveTransactions(getTransactionCache().size()); + metrics.incrementRolledBackTransactions(); + metrics.addRolledBackTransactionId(row.getTransactionId()); + counters.rollbackCount++; + } else { + LOGGER.trace( + "Could not rollback transaction {}, was not found in cache.", + row.getTransactionId()); + } + } + + /** + * Finalizes the rollback the specified transaction. + * + * @param transactionId the unique transaction identifier, never {@code null} + * @param rollbackScn the rollback transaction's system change number, never {@code null} + */ + protected abstract void finalizeTransactionRollback(String transactionId, Scn rollbackScn); + + /** + * Handle processing a LogMinerEventRow for a {@code DDL} event. + * + * @param row the result set row + * @throws InterruptedException if the event dispatcher is interrupted sending the event + */ + protected void handleSchemaChange(LogMinerEventRow row) throws InterruptedException { + if (hasSchemaChangeBeenSeen(row)) { + LOGGER.trace( + "DDL: Scn {}, SQL '{}' has already been processed, skipped.", + row.getScn(), + row.getRedoSql()); + return; + } + + if (offsetContext.getCommitScn().hasCommitAlreadyBeenHandled(row)) { + final Scn commitScn = + offsetContext.getCommitScn().getCommitScnForRedoThread(row.getThread()); + LOGGER.trace( + "DDL: SQL '{}' skipped with {} (SCN) <= {} (commit SCN for redo thread {})", + row.getRedoSql(), + row.getScn(), + commitScn, + row.getThread()); + return; + } + + LOGGER.trace("DDL: '{}' {}", row.getRedoSql(), row); + if (row.getTableName() != null) { + counters.ddlCount++; + final TableId tableId = row.getTableId(); + + final int activeTransactions = getTransactionCache().size(); + boolean advanceLowerScnBoundary = false; + if (activeTransactions == 0) { + // The DDL isn't wrapped in a transaction, fast-forward the lower boundary + advanceLowerScnBoundary = true; + } else if (activeTransactions == 1) { + final String transactionId = getTransactionCache().keySet().iterator().next(); + if (transactionId.equals(row.getTransactionId())) { + // The row's transaction is the current and only active transaction. + advanceLowerScnBoundary = true; + } + } + + if (advanceLowerScnBoundary) { + LOGGER.debug("Schema change advanced offset SCN to {}", row.getScn()); + offsetContext.setScn(row.getScn()); + } + + // Should always advance the commit SCN point with schema changes + LOGGER.debug( + "Schema change advanced offset commit SCN to {} for thread {}", + row.getScn(), + row.getThread()); + offsetContext.getCommitScn().recordCommit(row); + + offsetContext.setEventScn(row.getScn()); + offsetContext.setRedoThread(row.getThread()); + dispatcher.dispatchSchemaChangeEvent( + partition, + tableId, + new OracleSchemaChangeEventEmitter( + getConfig(), + partition, + offsetContext, + tableId, + tableId.catalog(), + tableId.schema(), + row.getRedoSql(), + getSchema(), + row.getChangeTime(), + metrics, + () -> processTruncateEvent(row))); + } + } + + private void processTruncateEvent(LogMinerEventRow row) { + LOGGER.debug("Handling truncate event"); + addToTransaction( + row.getTransactionId(), + row, + () -> { + final LogMinerDmlEntry dmlEntry = LogMinerDmlEntryImpl.forValuelessDdl(); + dmlEntry.setObjectName(row.getTableName()); + dmlEntry.setObjectOwner(row.getTablespaceName()); + return new TruncateEvent(row, dmlEntry); + }); + } + + /** + * Handle processing a LogMinerEventRow for a {@code SEL_LOB_LOCATOR} event. + * + * @param row the result set row + */ + protected void handleSelectLobLocator(LogMinerEventRow row) { + if (!getConfig().isLobEnabled()) { + LOGGER.trace( + "LOB support is disabled, SEL_LOB_LOCATOR '{}' skipped.", row.getRedoSql()); + return; + } + + LOGGER.trace("SEL_LOB_LOCATOR: {}", row); + final TableId tableId = row.getTableId(); + final Table table = getSchema().tableFor(tableId); + if (table == null) { + LOGGER.warn("SEL_LOB_LOCATOR for table '{}' is not known, skipped.", tableId); + return; + } + + addToTransaction( + row.getTransactionId(), + row, + () -> { + final LogMinerDmlEntry dmlEntry = + selectLobParser.parse(row.getRedoSql(), table); + dmlEntry.setObjectName(row.getTableName()); + dmlEntry.setObjectOwner(row.getTablespaceName()); + + return new SelectLobLocatorEvent( + row, + dmlEntry, + selectLobParser.getColumnName(), + selectLobParser.isBinary()); + }); + + metrics.incrementRegisteredDmlCount(); + } + + /** + * Handle processing a LogMinerEventRow for a {@code LOB_WRITE} event. + * + * @param row the result set row + */ + protected void handleLobWrite(LogMinerEventRow row) { + if (!getConfig().isLobEnabled()) { + LOGGER.trace( + "LOB support is disabled, LOB_WRITE scn={}, tableId={} skipped", + row.getScn(), + row.getTableId()); + return; + } + + LOGGER.trace( + "LOB_WRITE: scn={}, tableId={}, changeTime={}, transactionId={}", + row.getScn(), + row.getTableId(), + row.getChangeTime(), + row.getTransactionId()); + + final TableId tableId = row.getTableId(); + final Table table = getSchema().tableFor(tableId); + if (table == null) { + LOGGER.warn("LOB_WRITE for table '{}' is not known, skipped", tableId); + return; + } + + if (row.getRedoSql() != null) { + addToTransaction( + row.getTransactionId(), + row, + () -> { + final ParsedLobWriteSql parsed = parseLobWriteSql(row.getRedoSql()); + return new LobWriteEvent(row, parsed.data, parsed.offset, parsed.length); + }); + } + } + + /** + * Handle processing a LogMinerEventRow for a {@code LOB_ERASE} event. + * + * @param row the result set row + */ + private void handleLobErase(LogMinerEventRow row) { + if (!getConfig().isLobEnabled()) { + LOGGER.trace("LOB support is disabled, LOB_ERASE '{}' skipped", row); + return; + } + + LOGGER.trace("LOB_ERASE: {}", row); + final TableId tableId = row.getTableId(); + final Table table = getSchema().tableFor(tableId); + if (table == null) { + LOGGER.warn("LOB_ERASE for table '{}' is not known, skipped", tableId); + return; + } + + addToTransaction(row.getTransactionId(), row, () -> new LobEraseEvent(row)); + } + + /** + * Handle processing a LogMinerEventRow for a {@code INSERT}, {@code UPDATE}, or {@code DELETE} + * event. + * + * @param row the result set row + * @throws SQLException if a database exception occurs + * @throws InterruptedException if the dispatch of an event is interrupted + */ + protected void handleDataEvent(LogMinerEventRow row) throws SQLException, InterruptedException { + if (row.getRedoSql() == null) { + return; + } + + LOGGER.trace("DML: {}", row); + LOGGER.trace("\t{}", row.getRedoSql()); + + // Oracle LogMiner reports LONG data types as STATUS=2 on UPDATE statements but there is no + // value in the INFO column, and the record can be managed by the connector successfully, + // so to be backward compatible, we only explicitly trigger this behavior if there is an + // error reason for STATUS=2 in the INFO column as well as STATUS=2. + if (row.getStatus() == 2 && !Strings.isNullOrBlank(row.getInfo())) { + // The SQL in the SQL_REDO column is not valid and cannot be parsed. + switch (connectorConfig.getEventProcessingFailureHandlingMode()) { + case FAIL: + LOGGER.error("Oracle LogMiner is unable to re-construct the SQL for '{}'", row); + throw new DebeziumException( + "Oracle failed to re-construct redo SQL '" + row.getRedoSql() + "'"); + case WARN: + LOGGER.warn( + "Oracle LogMiner event '{}' cannot be parsed. This event will be ignored and skipped.", + row); + return; + default: + // In this case, we explicitly log the situation in "debug" only and not as an + // error/warn. + LOGGER.debug( + "Oracle LogMiner event '{}' cannot be parsed. This event will be ignored and skipped.", + row); + return; + } + } + + counters.dmlCount++; + switch (row.getEventType()) { + case INSERT: + counters.insertCount++; + break; + case UPDATE: + counters.updateCount++; + break; + case DELETE: + counters.deleteCount++; + break; + } + + final Table table = getTableForDataEvent(row); + if (table == null) { + return; + } + + if (row.isRollbackFlag()) { + // There is a use case where a constraint violation will result in a DML event being + // written to the redo log subsequently followed by another DML event that is marked + // with a rollback flag to indicate that the prior event should be omitted. In this + // use case, the transaction can still be committed, so we need to manually rollback + // the previous DML event when this use case occurs. + removeEventWithRowId(row); + return; + } + + addToTransaction( + row.getTransactionId(), + row, + () -> { + final LogMinerDmlEntry dmlEntry = + parseDmlStatement(row.getRedoSql(), table, row.getTransactionId()); + dmlEntry.setObjectName(row.getTableName()); + dmlEntry.setObjectOwner(row.getTablespaceName()); + return new DmlEvent(row, dmlEntry); + }); + + metrics.incrementRegisteredDmlCount(); + } + + protected void handleUnsupportedEvent(LogMinerEventRow row) { + if (!Strings.isNullOrEmpty(row.getTableName())) { + LOGGER.warn( + "An unsupported operation detected for table '{}' in transaction {} with SCN {} on redo thread {}.", + row.getTableId(), + row.getTransactionId(), + row.getScn(), + row.getThread()); + } + } + + /** + * Checks to see whether the offset's {@code scn} is remaining the same across multiple mining + * sessions while the offset's {@code commit_scn} is changing between sessions. + * + * @param previousOffsetScn the previous offset system change number + * @param previousOffsetCommitScns the previous offset commit system change number + */ + protected void warnPotentiallyStuckScn( + Scn previousOffsetScn, Map previousOffsetCommitScns) { + if (offsetContext != null && offsetContext.getCommitScn() != null) { + final Scn scn = offsetContext.getScn(); + final Map commitScns = + offsetContext.getCommitScn().getCommitScnForAllRedoThreads(); + if (previousOffsetScn.equals(scn) && !previousOffsetCommitScns.equals(commitScns)) { + counters.stuckCount++; + if (counters.stuckCount == 25) { + LOGGER.warn( + "Offset SCN {} has not changed in 25 mining session iterations. " + + "This indicates long running transaction(s) are active. Commit SCNs {}.", + previousOffsetScn, + previousOffsetCommitScns); + metrics.incrementScnFreezeCount(); + } + } else { + counters.stuckCount = 0; + } + } + } + + private Table getTableForDataEvent(LogMinerEventRow row) + throws SQLException, InterruptedException { + final TableId tableId = row.getTableId(); + Table table = getSchema().tableFor(tableId); + if (table == null) { + if (!getConfig().getTableFilters().dataCollectionFilter().isIncluded(tableId)) { + return null; + } + table = + dispatchSchemaChangeEventAndGetTableForNewCapturedTable( + tableId, offsetContext, dispatcher); + } + return table; + } + + /** + * Checks whether the result-set has any more data available. When a new row is available, the + * streaming metrics is updated with the fetch timings. + * + * @param resultSet the result set to check if any more rows exist + * @return true if another row exists, false otherwise + * @throws SQLException if there was a database exception + */ + private boolean hasNextWithMetricsUpdate(ResultSet resultSet) throws SQLException { + Instant start = Instant.now(); + boolean result = false; + try { + if (resultSet.next()) { + metrics.addCurrentResultSetNext(Duration.between(start, Instant.now())); + result = true; + } + + // Reset sequence unavailability on successful read from the result set + if (sequenceUnavailable) { + LOGGER.debug("The previous batch's unavailable log problem has been cleared."); + sequenceUnavailable = false; + } + } catch (SQLException e) { + // Oracle's online redo logs can be defined with dynamic names using the instance + // configuration property LOG_ARCHIVE_FORMAT. + // + // Dynamically named online redo logs can lead to ORA-00310 errors if a log switch + // happens while the processor is iterating the LogMiner session's result set and + // LogMiner can no longer read the next batch of records from the log. + // + // LogMiner only validates that there are no gaps and that the logs are available + // when the session is first started and any change in the logs later will raise + // these types of errors. + // + // Catching the ORA-00310 and treating it as the end of the result set will allow + // the connector's outer loop to re-evaluate the log state and start a new LogMiner + // session with the new logs. The connector will then begin streaming from where + // it left off. If any other exception is caught here, it'll be thrown. + if (!e.getMessage().startsWith("ORA-00310")) { + // throw any non ORA-00310 error, old behavior + throw e; + } else if (sequenceUnavailable) { + // If an ORA-00310 error was raised on the previous iteration and wasn't cleared + // after re-evaluation of the log availability and the mining session, we will + // explicitly stop the connector to avoid an infinite loop. + LOGGER.error( + "The log availability error '{}' wasn't cleared, stop requested.", + e.getMessage()); + throw e; + } + + LOGGER.debug("A mined log is no longer available: {}", e.getMessage()); + LOGGER.warn("Restarting mining session after a log became unavailable."); + + // Track that we gracefully stopped due to a ORA-00310. + // Will be used to detect an infinite loop of this error across sequential iterations + sequenceUnavailable = true; + } + return result; + } + + /** + * Add a transaction to the transaction map if allowed. + * + * @param transactionId the unqiue transaction id + * @param row the LogMiner event row + * @param eventSupplier the supplier of the event to create if the event is allowed to be added + */ + protected abstract void addToTransaction( + String transactionId, LogMinerEventRow row, Supplier eventSupplier); + + /** + * Dispatch a schema change event for a new table and get the newly created relational table + * model. + * + * @param tableId the unique table identifier, must not be {@code null} + * @param offsetContext the offset context + * @param dispatcher the event dispatcher + * @return the relational table model + * @throws SQLException if a database exception occurred + * @throws InterruptedException if the event dispatch was interrupted + */ + private Table dispatchSchemaChangeEventAndGetTableForNewCapturedTable( + TableId tableId, + OracleOffsetContext offsetContext, + EventDispatcher dispatcher) + throws SQLException, InterruptedException { + + final String tableDdl; + try { + tableDdl = getTableMetadataDdl(tableId); + } catch (NonRelationalTableException e) { + LOGGER.warn("Table {} is not a relational table and will be skipped.", tableId); + metrics.incrementWarningCount(); + return null; + } + + LOGGER.info("Table '{}' is new and will now be captured.", tableId); + offsetContext.event(tableId, Instant.now()); + dispatcher.dispatchSchemaChangeEvent( + partition, + tableId, + new OracleSchemaChangeEventEmitter( + connectorConfig, + partition, + offsetContext, + tableId, + tableId.catalog(), + tableId.schema(), + tableDdl, + getSchema(), + Instant.now(), + metrics, + null)); + + return getSchema().tableFor(tableId); + } + + /** + * Get the specified table's create DDL statement. + * + * @param tableId the table identifier, must not be {@code null} + * @return the table's create DDL statement, never {@code null} + * @throws SQLException if an exception occurred obtaining the DDL statement + * @throws NonRelationalTableException if the table is not a relational table + */ + private String getTableMetadataDdl(TableId tableId) + throws SQLException, NonRelationalTableException { + counters.tableMetadataCount++; + LOGGER.info("Getting database metadata for table '{}'", tableId); + // A separate connection must be used for this out-of-bands query while processing LogMiner + // results. + // This should have negligible overhead since this use case should happen rarely. + try (OracleConnection connection = + new OracleConnection( + connectorConfig.getJdbcConfig(), + () -> getClass().getClassLoader(), + false)) { + connection.setAutoCommit(false); + final String pdbName = getConfig().getPdbName(); + if (pdbName != null) { + connection.setSessionToPdb(pdbName); + } + return connection.getTableMetadataDdl(tableId); + } + } + + /** + * Parse a DML redo SQL statement. + * + * @param redoSql the redo SQL statement + * @param table the table the SQL statement is for + * @param transactionId the associated transaction id for the SQL statement + * @return a parse object for the redo SQL statement + */ + private LogMinerDmlEntry parseDmlStatement(String redoSql, Table table, String transactionId) { + LogMinerDmlEntry dmlEntry; + try { + Instant parseStart = Instant.now(); + dmlEntry = dmlParser.parse(redoSql, table); + metrics.addCurrentParseTime(Duration.between(parseStart, Instant.now())); + } catch (DmlParserException e) { + String message = + "DML statement couldn't be parsed." + + " Please open a Jira issue with the statement '" + + redoSql + + "'."; + throw new DmlParserException(message, e); + } + + if (dmlEntry.getOldValues().length == 0) { + if (EventType.UPDATE == dmlEntry.getEventType() + || EventType.DELETE == dmlEntry.getEventType()) { + LOGGER.warn("The DML event '{}' contained no before state.", redoSql); + metrics.incrementWarningCount(); + } + } + + return dmlEntry; + } + + private static final Pattern LOB_WRITE_SQL_PATTERN = + Pattern.compile( + "(?s).* := ((?:HEXTORAW\\()?'.*'(?:\\))?);\\s*dbms_lob.write\\([^,]+,\\s*(\\d+)\\s*,\\s*(\\d+)\\s*,[^,]+\\);.*"); + + /** + * Parses a {@code LOB_WRITE} operation SQL fragment. + * + * @param sql sql statement + * @return the parsed statement + * @throws DebeziumException if an unexpected SQL fragment is provided that cannot be parsed + */ + private ParsedLobWriteSql parseLobWriteSql(String sql) { + if (sql == null) { + return null; + } + + Matcher m = LOB_WRITE_SQL_PATTERN.matcher(sql.trim()); + if (!m.matches()) { + throw new DebeziumException("Unable to parse unsupported LOB_WRITE SQL: " + sql); + } + + String data = m.group(1); + if (data.startsWith("'")) { + // string data; drop the quotes + data = data.substring(1, data.length() - 1); + } + int length = Integer.parseInt(m.group(2)); + int offset = Integer.parseInt(m.group(3)) - 1; // Oracle uses 1-based offsets + return new ParsedLobWriteSql(offset, length, data); + } + + private class ParsedLobWriteSql { + final int offset; + final int length; + final String data; + + ParsedLobWriteSql(int offset, int length, String data) { + this.offset = offset; + this.length = length; + this.data = data; + } + } + + /** + * Gets the minimum system change number stored in the transaction cache. + * + * @return the minimum system change number, never {@code null} but could be {@link Scn#NULL}. + */ + protected abstract Scn getTransactionCacheMinimumScn(); + + /** + * Returns whether the transaction id has no sequence number component. + * + *

Oracle transaction identifiers are a composite of: + * + *

    + *
  1. Undo segment number + *
  2. Slot numbber of the transaction that generated the change + *
  3. Sequence number of the transaction that generated the change + *
+ * + *

When Oracle LogMiner mines records, it is possible that when an undo operation is + * detected, often the product of a constraint violation, the LogMiner row will have the same + * explicit XID (transaction id) as the source operation that we should undo; however, if the + * record to be undone was mined in a prior iteration, Oracle LogMiner won't be able to make a + * link back to the full transaction's sequence number, therefore the XID value for the undo row + * will contain only the undo segment number and slot number, setting the sequence number to + * 4294967295 (aka -1 or 0xFFFFFFFF). + * + *

This method explicitly checks if the provided transaction id has the no sequence sentinel + * value and if so, returns {@code true}; otherwise returns {@code false}. + * + * @param transactionId the transaction identifier to check, should not be {@code null} + * @return true if the transaction has no sequence reference, false if it does + */ + protected boolean isTransactionIdWithNoSequence(String transactionId) { + return transactionId.endsWith(NO_SEQUENCE_TRX_ID_SUFFIX); + } + + protected String getTransactionIdPrefix(String transactionId) { + return transactionId.substring(0, 8); + } + + /** Wrapper for all counter variables. */ + protected class Counters { + public int stuckCount; + public int dmlCount; + public int ddlCount; + public int insertCount; + public int updateCount; + public int deleteCount; + public int commitCount; + public int rollbackCount; + public int tableMetadataCount; + public long rows; + + public void reset() { + stuckCount = 0; + dmlCount = 0; + ddlCount = 0; + insertCount = 0; + updateCount = 0; + deleteCount = 0; + commitCount = 0; + rollbackCount = 0; + tableMetadataCount = 0; + rows = 0; + } + + @Override + public String toString() { + return "Counters{" + + "rows=" + + rows + + ", stuckCount=" + + stuckCount + + ", dmlCount=" + + dmlCount + + ", ddlCount=" + + ddlCount + + ", insertCount=" + + insertCount + + ", updateCount=" + + updateCount + + ", deleteCount=" + + deleteCount + + ", commitCount=" + + commitCount + + ", rollbackCount=" + + rollbackCount + + ", tableMetadataCount=" + + tableMetadataCount + + '}'; + } + } +} diff --git a/flink-connector-oracle-cdc/src/main/java/io/debezium/relational/RelationalChangeRecordEmitter.java b/flink-connector-oracle-cdc/src/main/java/io/debezium/relational/RelationalChangeRecordEmitter.java new file mode 100644 index 0000000000..86bb2f6558 --- /dev/null +++ b/flink-connector-oracle-cdc/src/main/java/io/debezium/relational/RelationalChangeRecordEmitter.java @@ -0,0 +1,289 @@ +/* + * Copyright Debezium Authors. + * + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 + */ + +package io.debezium.relational; + +import io.debezium.data.Envelope.Operation; +import io.debezium.pipeline.AbstractChangeRecordEmitter; +import io.debezium.pipeline.spi.ChangeRecordEmitter; +import io.debezium.pipeline.spi.OffsetContext; +import io.debezium.pipeline.spi.Partition; +import io.debezium.schema.DataCollectionSchema; +import io.debezium.util.Clock; +import org.apache.kafka.connect.data.SchemaAndValue; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.header.ConnectHeaders; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; + +/** + * Copied from Debezium 1.9.7.Final. + * + *

Base class for {@link ChangeRecordEmitter} implementations based on a relational database. + * + *

This class overrides the emit methods to put the ROWID in the header. + * + *

Line 59 ~ 257: add ROWID and emit methods. + */ +public abstract class RelationalChangeRecordEmitter

+ extends AbstractChangeRecordEmitter { + + private static final Logger LOGGER = + LoggerFactory.getLogger(RelationalChangeRecordEmitter.class); + + public static final String PK_UPDATE_OLDKEY_FIELD = "__debezium.oldkey"; + public static final String PK_UPDATE_NEWKEY_FIELD = "__debezium.newkey"; + + private final Map staticHeaders = new HashMap<>(); + + public RelationalChangeRecordEmitter(P partition, OffsetContext offsetContext, Clock clock) { + super(partition, offsetContext, clock); + } + + @Override + public void emitChangeRecords(DataCollectionSchema schema, Receiver

receiver) + throws InterruptedException { + TableSchema tableSchema = (TableSchema) schema; + Operation operation = getOperation(); + + switch (operation) { + case CREATE: + emitCreateRecord(receiver, tableSchema); + break; + case READ: + emitReadRecord(receiver, tableSchema); + break; + case UPDATE: + emitUpdateRecord(receiver, tableSchema); + break; + case DELETE: + emitDeleteRecord(receiver, tableSchema); + break; + case TRUNCATE: + emitTruncateRecord(receiver, tableSchema); + break; + default: + throw new IllegalArgumentException("Unsupported operation: " + operation); + } + } + + public void addStaticHeader(String key, SchemaAndValue value) { + this.staticHeaders.put(key, value); + } + + @Override + protected void emitCreateRecord(Receiver

receiver, TableSchema tableSchema) + throws InterruptedException { + Object[] newColumnValues = getNewColumnValues(); + Struct newKey = tableSchema.keyFromColumnData(newColumnValues); + Struct newValue = tableSchema.valueFromColumnData(newColumnValues); + Struct envelope = + tableSchema + .getEnvelopeSchema() + .create( + newValue, + getOffset().getSourceInfo(), + getClock().currentTimeAsInstant()); + + if (skipEmptyMessages() && (newColumnValues == null || newColumnValues.length == 0)) { + // This case can be hit on UPDATE / DELETE when there's no primary key defined while + // using certain decoders + LOGGER.warn( + "no new values found for table '{}' from create message at '{}'; skipping record", + tableSchema, + getOffset().getSourceInfo()); + return; + } + receiver.changeRecord( + getPartition(), + tableSchema, + Operation.CREATE, + newKey, + envelope, + getOffset(), + getStaticConnectHeaders()); + } + + @Override + protected void emitReadRecord(Receiver

receiver, TableSchema tableSchema) + throws InterruptedException { + Object[] newColumnValues = getNewColumnValues(); + Struct newKey = tableSchema.keyFromColumnData(newColumnValues); + Struct newValue = tableSchema.valueFromColumnData(newColumnValues); + Struct envelope = + tableSchema + .getEnvelopeSchema() + .read( + newValue, + getOffset().getSourceInfo(), + getClock().currentTimeAsInstant()); + + receiver.changeRecord( + getPartition(), + tableSchema, + Operation.READ, + newKey, + envelope, + getOffset(), + getStaticConnectHeaders()); + } + + @Override + protected void emitUpdateRecord(Receiver

receiver, TableSchema tableSchema) + throws InterruptedException { + Object[] oldColumnValues = getOldColumnValues(); + Object[] newColumnValues = getNewColumnValues(); + + Struct oldKey = tableSchema.keyFromColumnData(oldColumnValues); + Struct newKey = tableSchema.keyFromColumnData(newColumnValues); + + Struct newValue = tableSchema.valueFromColumnData(newColumnValues); + Struct oldValue = tableSchema.valueFromColumnData(oldColumnValues); + + if (skipEmptyMessages() && (newColumnValues == null || newColumnValues.length == 0)) { + LOGGER.warn( + "no new values found for table '{}' from update message at '{}'; skipping record", + tableSchema, + getOffset().getSourceInfo()); + return; + } + // some configurations does not provide old values in case of updates + // in this case we handle all updates as regular ones + if (oldKey == null || Objects.equals(oldKey, newKey)) { + Struct envelope = + tableSchema + .getEnvelopeSchema() + .update( + oldValue, + newValue, + getOffset().getSourceInfo(), + getClock().currentTimeAsInstant()); + receiver.changeRecord( + getPartition(), + tableSchema, + Operation.UPDATE, + newKey, + envelope, + getOffset(), + getStaticConnectHeaders()); + } + // PK update -> emit as delete and re-insert with new key + else { + emitUpdateAsPrimaryKeyChangeRecord( + receiver, tableSchema, oldKey, newKey, oldValue, newValue); + } + } + + @Override + protected void emitDeleteRecord(Receiver

receiver, TableSchema tableSchema) + throws InterruptedException { + Object[] oldColumnValues = getOldColumnValues(); + Struct oldKey = tableSchema.keyFromColumnData(oldColumnValues); + Struct oldValue = tableSchema.valueFromColumnData(oldColumnValues); + + if (skipEmptyMessages() && (oldColumnValues == null || oldColumnValues.length == 0)) { + LOGGER.warn( + "no old values found for table '{}' from delete message at '{}'; skipping record", + tableSchema, + getOffset().getSourceInfo()); + return; + } + + Struct envelope = + tableSchema + .getEnvelopeSchema() + .delete( + oldValue, + getOffset().getSourceInfo(), + getClock().currentTimeAsInstant()); + receiver.changeRecord( + getPartition(), + tableSchema, + Operation.DELETE, + oldKey, + envelope, + getOffset(), + getStaticConnectHeaders()); + } + + protected void emitTruncateRecord(Receiver

receiver, TableSchema schema) + throws InterruptedException { + throw new UnsupportedOperationException("TRUNCATE not supported"); + } + + /** Returns the old row state in case of an UPDATE or DELETE. */ + protected abstract Object[] getOldColumnValues(); + + /** Returns the new row state in case of a CREATE or READ. */ + protected abstract Object[] getNewColumnValues(); + + /** + * Whether empty data messages should be ignored. + * + * @return true if empty data messages coming from data source should be ignored. Typical use + * case are PostgreSQL changes without FULL replica identity. + */ + protected boolean skipEmptyMessages() { + return false; + } + + protected void emitUpdateAsPrimaryKeyChangeRecord( + Receiver

receiver, + TableSchema tableSchema, + Struct oldKey, + Struct newKey, + Struct oldValue, + Struct newValue) + throws InterruptedException { + ConnectHeaders headers = getStaticConnectHeaders(); + headers.add(PK_UPDATE_NEWKEY_FIELD, newKey, tableSchema.keySchema()); + + Struct envelope = + tableSchema + .getEnvelopeSchema() + .delete( + oldValue, + getOffset().getSourceInfo(), + getClock().currentTimeAsInstant()); + receiver.changeRecord( + getPartition(), + tableSchema, + Operation.DELETE, + oldKey, + envelope, + getOffset(), + headers); + + headers = getStaticConnectHeaders(); + headers.add(PK_UPDATE_OLDKEY_FIELD, oldKey, tableSchema.keySchema()); + + envelope = + tableSchema + .getEnvelopeSchema() + .create( + newValue, + getOffset().getSourceInfo(), + getClock().currentTimeAsInstant()); + receiver.changeRecord( + getPartition(), + tableSchema, + Operation.CREATE, + newKey, + envelope, + getOffset(), + headers); + } + + private ConnectHeaders getStaticConnectHeaders() { + ConnectHeaders headers = new ConnectHeaders(); + staticHeaders.forEach(headers::add); + return headers; + } +} diff --git a/flink-connector-oracle-cdc/src/test/java/com/ververica/cdc/connectors/oracle/OracleSourceTest.java b/flink-connector-oracle-cdc/src/test/java/com/ververica/cdc/connectors/oracle/OracleSourceTest.java index 1de4b34e20..2dbbb50868 100644 --- a/flink-connector-oracle-cdc/src/test/java/com/ververica/cdc/connectors/oracle/OracleSourceTest.java +++ b/flink-connector-oracle-cdc/src/test/java/com/ververica/cdc/connectors/oracle/OracleSourceTest.java @@ -575,6 +575,8 @@ private OracleSource.Builder basicSourceBuilder(OracleContainer or Properties debeziumProperties = new Properties(); debeziumProperties.setProperty("debezium.log.mining.strategy", "online_catalog"); debeziumProperties.setProperty("debezium.log.mining.continuous.mine", "true"); + // ignore APEX XE system tables changes + debeziumProperties.setProperty("database.history.store.only.captured.tables.ddl", "true"); return OracleSource.builder() .hostname(oracleContainer.getHost()) .port(oracleContainer.getOraclePort()) diff --git a/flink-connector-oracle-cdc/src/test/java/com/ververica/cdc/connectors/oracle/source/OracleSourceITCase.java b/flink-connector-oracle-cdc/src/test/java/com/ververica/cdc/connectors/oracle/source/OracleSourceITCase.java index 1faad37538..49719fe776 100644 --- a/flink-connector-oracle-cdc/src/test/java/com/ververica/cdc/connectors/oracle/source/OracleSourceITCase.java +++ b/flink-connector-oracle-cdc/src/test/java/com/ververica/cdc/connectors/oracle/source/OracleSourceITCase.java @@ -146,7 +146,8 @@ private void testOracleParallelSource( + " 'table-name' = '%s'," + " 'scan.incremental.snapshot.enabled' = 'false'," + " 'debezium.log.mining.strategy' = 'online_catalog'," - + " 'debezium.log.mining.continuous.mine' = 'true'" + + " 'debezium.log.mining.continuous.mine' = 'true'," + + " 'debezium.database.history.store.only.captured.tables.ddl' = 'true'" + ")", ORACLE_CONTAINER.getHost(), ORACLE_CONTAINER.getOraclePort(), diff --git a/flink-connector-oracle-cdc/src/test/java/com/ververica/cdc/connectors/oracle/table/OracleConnectorITCase.java b/flink-connector-oracle-cdc/src/test/java/com/ververica/cdc/connectors/oracle/table/OracleConnectorITCase.java index cef8a41334..1ede929de6 100644 --- a/flink-connector-oracle-cdc/src/test/java/com/ververica/cdc/connectors/oracle/table/OracleConnectorITCase.java +++ b/flink-connector-oracle-cdc/src/test/java/com/ververica/cdc/connectors/oracle/table/OracleConnectorITCase.java @@ -23,8 +23,12 @@ import org.apache.flink.table.planner.factories.TestValuesTableFactory; import org.apache.flink.test.util.AbstractTestBase; +import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; +import org.apache.flink.shaded.guava30.com.google.common.util.concurrent.RateLimiter; + import com.ververica.cdc.connectors.oracle.utils.OracleTestUtils; import org.junit.After; +import org.junit.Assume; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -43,7 +47,15 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.IntStream; import java.util.stream.Stream; import static com.ververica.cdc.connectors.oracle.source.OracleSourceTestBase.assertEqualsInAnyOrder; @@ -56,6 +68,8 @@ /** Integration tests for Oracle binlog SQL source. */ @RunWith(Parameterized.class) public class OracleConnectorITCase extends AbstractTestBase { + private static final int RECORDS_COUNT = 10_000; + private static final int WORKERS_COUNT = 4; private static final Logger LOG = LoggerFactory.getLogger(OracleConnectorITCase.class); @@ -120,6 +134,7 @@ public void testConsumingAllEvents() + " 'scan.incremental.snapshot.enabled' = '%s'," + " 'debezium.log.mining.strategy' = 'online_catalog'," + " 'debezium.log.mining.continuous.mine' = 'true'," + + " 'debezium.database.history.store.only.captured.tables.ddl' = 'true'," + " 'scan.incremental.snapshot.chunk.size' = '2'," + " 'database-name' = 'XE'," + " 'schema-name' = '%s'," @@ -232,6 +247,7 @@ public void testConsumingAllEventsByChunkKeyColumn() + " 'scan.incremental.snapshot.chunk.key-column' = 'ID'," + " 'debezium.log.mining.strategy' = 'online_catalog'," + " 'debezium.log.mining.continuous.mine' = 'true'," + + " 'debezium.database.history.store.only.captured.tables.ddl' = 'true'," + " 'scan.incremental.snapshot.chunk.size' = '2'," + " 'database-name' = 'XE'," + " 'schema-name' = '%s'," @@ -322,6 +338,8 @@ public void testMetadataColumns() throws Throwable { + " 'scan.incremental.snapshot.enabled' = '%s'," + " 'debezium.log.mining.strategy' = 'online_catalog'," + " 'debezium.log.mining.continuous.mine' = 'true'," + // + " 'debezium.database.history.store.only.captured.tables.ddl' = + // 'true'," + " 'scan.incremental.snapshot.chunk.size' = '2'," + " 'database-name' = 'XE'," + " 'schema-name' = '%s'," @@ -417,6 +435,7 @@ public void testStartupFromLatestOffset() throws Exception { + " 'scan.incremental.snapshot.enabled' = '%s'," + " 'debezium.log.mining.strategy' = 'online_catalog'," + " 'debezium.log.mining.continuous.mine' = 'true'," + + " 'debezium.database.history.store.only.captured.tables.ddl' = 'true'," + " 'database-name' = 'XE'," + " 'schema-name' = '%s'," + " 'table-name' = '%s' ," @@ -516,6 +535,7 @@ public void testConsumingNumericColumns() throws Exception { + " 'scan.incremental.snapshot.enabled' = '%s'," + " 'debezium.log.mining.strategy' = 'online_catalog'," + " 'debezium.log.mining.continuous.mine' = 'true'," + + " 'debezium.database.history.store.only.captured.tables.ddl' = 'true'," + " 'database-name' = 'XE'," + " 'schema-name' = '%s'," + " 'table-name' = '%s'" @@ -598,6 +618,7 @@ public void testXmlType() throws Exception { + " 'scan.incremental.snapshot.enabled' = '%s'," + " 'debezium.log.mining.strategy' = 'online_catalog'," + " 'debezium.log.mining.continuous.mine' = 'true'," + + " 'debezium.database.history.store.only.captured.tables.ddl' = 'true'," + " 'scan.incremental.snapshot.chunk.size' = '2'," + " 'database-name' = 'XE'," + " 'schema-name' = '%s'," @@ -708,6 +729,7 @@ public void testAllDataTypes() throws Throwable { + " 'scan.incremental.snapshot.enabled' = '%s'," + " 'debezium.log.mining.strategy' = 'online_catalog'," + " 'debezium.log.mining.continuous.mine' = 'true'," + + " 'debezium.database.history.store.only.captured.tables.ddl' = 'true'," + " 'scan.incremental.snapshot.chunk.size' = '2'," + " 'database-name' = 'XE'," + " 'schema-name' = '%s'," @@ -765,6 +787,134 @@ public void testAllDataTypes() throws Throwable { result.getJobClient().get().cancel().get(); } + @Test + public void testSnapshotToStreamingSwitchPendingTransactions() throws Exception { + Assume.assumeFalse(parallelismSnapshot); + + CompletableFuture finishFuture = createRecordInserters(); + + String sourceDDL = + String.format( + "CREATE TABLE messages (" + + " ID INT NOT NULL," + + " CATEGORY_NAME STRING" + + ") WITH (" + + " 'connector' = 'oracle-cdc'," + + " 'hostname' = '%s'," + + " 'port' = '%s'," + + " 'username' = '%s'," + + " 'password' = '%s'," + + " 'database-name' = '%s'," + + " 'schema-name' = '%s'," + + " 'table-name' = 'category'," + + " 'scan.incremental.snapshot.enabled' = 'false'," + + " 'debezium.log.mining.strategy' = 'online_catalog'," + + " 'debezium.database.history.store.only.captured.tables.ddl' = 'true'," + + " 'debezium.log.mining.continuous.mine' = 'true'" + + ")", + oracleContainer.getHost(), + oracleContainer.getOraclePort(), + "dbzuser", + "dbz", + "XE", + "debezium"); + + String sinkDDL = + "CREATE TABLE sink (" + + " ID INT," + + " message STRING" + + ") WITH (" + + " 'connector' = 'values'," + + " 'sink-insert-only' = 'false'" + + ")"; + tEnv.executeSql(sourceDDL); + tEnv.executeSql(sinkDDL); + + TableResult result = tEnv.executeSql("INSERT INTO sink SELECT * FROM messages"); + + finishFuture.get(10, TimeUnit.MINUTES); + LOG.info("all async runners were finished"); + + waitForSinkSize("sink", RECORDS_COUNT); + + List actual = + TestValuesTableFactory.getResults("sink").stream() + .map(s -> s.replaceFirst("\\+I\\[(\\d+).+", "$1")) + .map(Integer::parseInt) + .sorted() + .collect(Collectors.toList()); + + List expected = + IntStream.range(0, RECORDS_COUNT).boxed().collect(Collectors.toList()); + + assertEquals(expected, actual); + result.getJobClient().get().cancel().get(); + } + + @SuppressWarnings("unchecked") + private CompletableFuture createRecordInserters() { + int requestPerSecondPerThread = 100; + int recordsChunkSize = RECORDS_COUNT / WORKERS_COUNT; + int recordsToCommit = recordsChunkSize / 4; + + List runners = + IntStream.range(0, WORKERS_COUNT) + .mapToObj( + i -> + createRecordInserter( + requestPerSecondPerThread, + recordsChunkSize * i, + recordsChunkSize, + recordsToCommit)) + .collect(Collectors.toList()); + + ExecutorService executor = Executors.newFixedThreadPool(WORKERS_COUNT); + CompletableFuture[] completableFutures = + runners.stream() + .map(runnable -> CompletableFuture.runAsync(runnable, executor)) + .toArray(CompletableFuture[]::new); + + return CompletableFuture.allOf(completableFutures); + } + + private Runnable createRecordInserter( + int requestPerSecond, int startIndex, int recordsCnt, int recordsToCommit) { + return () -> { + Supplier messageSupplier = + createRandomSupplier( + Lists.newArrayList("msg1", "msg2", "msg3", "msg4", "msg5", "msg6")); + + RateLimiter rateLimiter = RateLimiter.create(requestPerSecond); + + try (Connection connection = getJdbcConnection(); + Statement statement = connection.createStatement()) { + + connection.setAutoCommit(false); + for (long i = startIndex; i < startIndex + recordsCnt; i++) { + rateLimiter.acquire(); + statement.execute( + String.format( + "INSERT INTO %s.%s VALUES (%d,'%s')", + "debezium", "category", i, messageSupplier.get())); + if (i % recordsToCommit == 0) { + LOG.info("Committing at id {}", i); + connection.commit(); + } + } + + connection.commit(); + } catch (SQLException e) { + e.printStackTrace(); + throw new RuntimeException(e); + } + }; + } + + private Supplier createRandomSupplier(List possibleValues) { + int size = possibleValues.size(); + return () -> possibleValues.get(ThreadLocalRandom.current().nextInt(size)); + } + // ------------------------------------------------------------------------------------ private static void waitForSnapshotStarted(String sinkName) throws InterruptedException { diff --git a/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/PostgresConnectionPoolFactory.java b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/PostgresConnectionPoolFactory.java index 09ea548ad8..905ae31912 100644 --- a/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/PostgresConnectionPoolFactory.java +++ b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/PostgresConnectionPoolFactory.java @@ -19,15 +19,11 @@ import com.ververica.cdc.connectors.base.config.JdbcSourceConfig; import com.ververica.cdc.connectors.base.relational.connection.JdbcConnectionPoolFactory; import com.zaxxer.hikari.HikariDataSource; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** A connection pool factory to create pooled Postgres {@link HikariDataSource}. */ public class PostgresConnectionPoolFactory extends JdbcConnectionPoolFactory { public static final String JDBC_URL_PATTERN = "jdbc:postgresql://%s:%s/%s"; - private static final Logger LOG = LoggerFactory.getLogger(PostgresConnectionPoolFactory.class); - @Override public String getJdbcUrl(JdbcSourceConfig sourceConfig) { diff --git a/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/PostgresDialect.java b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/PostgresDialect.java index 4e018da2c0..fcac388075 100644 --- a/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/PostgresDialect.java +++ b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/PostgresDialect.java @@ -53,6 +53,8 @@ /** The dialect for Postgres. */ public class PostgresDialect implements JdbcDataSourceDialect { private static final long serialVersionUID = 1L; + + private static final String CONNECTION_NAME = "postgres-cdc-connector"; private final PostgresSourceConfig sourceConfig; private transient CustomPostgresSchema schema; @@ -74,6 +76,7 @@ public JdbcConnection openJdbcConnection(JdbcSourceConfig sourceConfig) { new PostgresConnection( dbzConfig.getJdbcConfig(), valueConverterBuilder, + CONNECTION_NAME, new JdbcConnectionFactory(sourceConfig, getPooledDataSourceFactory())); try { diff --git a/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/fetch/PostgresScanFetchTask.java b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/fetch/PostgresScanFetchTask.java index 2df2e7cbe8..0ae6aa466c 100644 --- a/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/fetch/PostgresScanFetchTask.java +++ b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/fetch/PostgresScanFetchTask.java @@ -31,6 +31,7 @@ import io.debezium.config.Configuration; import io.debezium.connector.postgresql.PostgresConnectorConfig; import io.debezium.connector.postgresql.PostgresOffsetContext; +import io.debezium.connector.postgresql.PostgresPartition; import io.debezium.connector.postgresql.PostgresSchema; import io.debezium.connector.postgresql.connection.PostgresConnection; import io.debezium.heartbeat.Heartbeat; @@ -38,7 +39,6 @@ import io.debezium.pipeline.source.AbstractSnapshotChangeEventSource; import io.debezium.pipeline.source.spi.ChangeEventSource; import io.debezium.pipeline.source.spi.SnapshotProgressListener; -import io.debezium.pipeline.spi.OffsetContext; import io.debezium.pipeline.spi.SnapshotResult; import io.debezium.relational.RelationalSnapshotChangeEventSource; import io.debezium.relational.SnapshotChangeRecordEmitter; @@ -59,7 +59,6 @@ import static io.debezium.connector.postgresql.Utils.currentOffset; import static io.debezium.connector.postgresql.Utils.refreshSchema; -import static io.debezium.relational.RelationalSnapshotChangeEventSource.LOG_INTERVAL; /** A {@link FetchTask} implementation for Postgres to read snapshot split. */ public class PostgresScanFetchTask implements FetchTask { @@ -78,6 +77,11 @@ public SourceSplitBase getSplit() { return split; } + @Override + public void close() { + taskRunning = false; + } + @Override public boolean isRunning() { return taskRunning; @@ -102,7 +106,8 @@ public void execute(Context context) throws Exception { SnapshotSplitChangeEventSourceContext changeEventSourceContext = new SnapshotSplitChangeEventSourceContext(); SnapshotResult snapshotResult = - snapshotSplitReadTask.execute(changeEventSourceContext, ctx.getOffsetContext()); + snapshotSplitReadTask.execute( + changeEventSourceContext, ctx.getPartition(), ctx.getOffsetContext()); if (!snapshotResult.isCompletedOrSkipped()) { taskRunning = false; @@ -136,7 +141,7 @@ private void executeBackfillTask( split); ctx.getDispatcher() .dispatchWatermarkEvent( - ctx.getOffsetContext().getPartition(), + ctx.getPartition().getSourcePartition(), backfillSplit, backfillSplit.getEndingOffset(), WatermarkKind.END); @@ -174,6 +179,7 @@ private void executeBackfillTask( ctx.getSnapShotter(), ctx.getConnection(), ctx.getDispatcher(), + ctx.getPostgresDispatcher(), ctx.getErrorHandler(), ctx.getTaskContext().getClock(), ctx.getDatabaseSchema(), @@ -182,7 +188,8 @@ private void executeBackfillTask( backfillSplit); LOG.info("Execute backfillReadTask for split {}", split); LOG.info("Slot name {}", dbzConf.getString("slot.name")); - backfillReadTask.execute(new PostgresChangeEventSourceContext(), postgresOffsetContext); + backfillReadTask.execute( + new PostgresChangeEventSourceContext(), ctx.getPartition(), postgresOffsetContext); } static class SnapshotSplitChangeEventSourceContext @@ -226,17 +233,18 @@ public boolean isRunning() { } /** A SnapshotChangeEventSource implementation for Postgres to read snapshot split. */ - public static class PostgresSnapshotSplitReadTask extends AbstractSnapshotChangeEventSource { + public static class PostgresSnapshotSplitReadTask + extends AbstractSnapshotChangeEventSource { private static final Logger LOG = LoggerFactory.getLogger(PostgresSnapshotSplitReadTask.class); private final PostgresConnection jdbcConnection; private final PostgresConnectorConfig connectorConfig; - private final JdbcSourceEventDispatcher dispatcher; + private final JdbcSourceEventDispatcher dispatcher; private final SnapshotSplit snapshotSplit; private final PostgresOffsetContext offsetContext; private final PostgresSchema databaseSchema; - private final SnapshotProgressListener snapshotProgressListener; + private final SnapshotProgressListener snapshotProgressListener; private final Clock clock; public PostgresSnapshotSplitReadTask( @@ -244,8 +252,8 @@ public PostgresSnapshotSplitReadTask( PostgresConnectorConfig connectorConfig, PostgresSchema databaseSchema, PostgresOffsetContext previousOffset, - JdbcSourceEventDispatcher dispatcher, - SnapshotProgressListener snapshotProgressListener, + JdbcSourceEventDispatcher dispatcher, + SnapshotProgressListener snapshotProgressListener, SnapshotSplit snapshotSplit) { super(connectorConfig, snapshotProgressListener); this.jdbcConnection = jdbcConnection; @@ -259,14 +267,13 @@ public PostgresSnapshotSplitReadTask( } @Override - protected SnapshotResult doExecute( + protected SnapshotResult doExecute( ChangeEventSourceContext context, - OffsetContext previousOffset, - SnapshotContext snapshotContext, + PostgresOffsetContext previousOffset, + SnapshotContext snapshotContext, SnapshottingTask snapshottingTask) throws Exception { - final RelationalSnapshotChangeEventSource.RelationalSnapshotContext ctx = - (RelationalSnapshotChangeEventSource.RelationalSnapshotContext) snapshotContext; + final PostgresSnapshotContext ctx = (PostgresSnapshotContext) snapshotContext; ctx.offset = offsetContext; refreshSchema(databaseSchema, jdbcConnection, false); @@ -277,7 +284,10 @@ protected SnapshotResult doExecute( snapshotSplit); ((SnapshotSplitChangeEventSourceContext) (context)).setLowWatermark(lowWatermark); dispatcher.dispatchWatermarkEvent( - offsetContext.getPartition(), snapshotSplit, lowWatermark, WatermarkKind.LOW); + snapshotContext.partition.getSourcePartition(), + snapshotSplit, + lowWatermark, + WatermarkKind.LOW); LOG.info("Snapshot step 2 - Snapshotting data"); createDataEvents(ctx, snapshotSplit.getTableId()); @@ -289,15 +299,16 @@ protected SnapshotResult doExecute( snapshotSplit); ((SnapshotSplitChangeEventSourceContext) (context)).setHighWatermark(highWatermark); dispatcher.dispatchWatermarkEvent( - offsetContext.getPartition(), snapshotSplit, highWatermark, WatermarkKind.HIGH); + snapshotContext.partition.getSourcePartition(), + snapshotSplit, + highWatermark, + WatermarkKind.HIGH); return SnapshotResult.completed(ctx.offset); } - private void createDataEvents( - RelationalSnapshotChangeEventSource.RelationalSnapshotContext snapshotContext, - TableId tableId) + private void createDataEvents(PostgresSnapshotContext snapshotContext, TableId tableId) throws InterruptedException { - EventDispatcher.SnapshotReceiver snapshotReceiver = + EventDispatcher.SnapshotReceiver snapshotReceiver = dispatcher.getSnapshotChangeEventReceiver(); LOG.info("Snapshotting table {}", tableId); createDataEventsForTable( @@ -309,8 +320,8 @@ private void createDataEvents( /** Dispatches the data change events for the records of a single table. */ private void createDataEventsForTable( - RelationalSnapshotChangeEventSource.RelationalSnapshotContext snapshotContext, - EventDispatcher.SnapshotReceiver snapshotReceiver, + PostgresSnapshotContext snapshotContext, + EventDispatcher.SnapshotReceiver snapshotReceiver, Table table) throws InterruptedException { @@ -361,13 +372,16 @@ private void createDataEventsForTable( rows, snapshotSplit.splitId(), Strings.duration(stop - exportStart)); - snapshotProgressListener.rowsScanned(table.id(), rows); + snapshotProgressListener.rowsScanned( + snapshotContext.partition, table.id(), rows); logTimer = getTableScanLogTimer(); } snapshotContext.offset.event(table.id(), clock.currentTime()); - SnapshotChangeRecordEmitter emitter = - new SnapshotChangeRecordEmitter(snapshotContext.offset, row, clock); - dispatcher.dispatchSnapshotEvent(table.id(), emitter, snapshotReceiver); + SnapshotChangeRecordEmitter emitter = + new SnapshotChangeRecordEmitter<>( + snapshotContext.partition, snapshotContext.offset, row, clock); + dispatcher.dispatchSnapshotEvent( + snapshotContext.partition, table.id(), emitter, snapshotReceiver); } LOG.info( "Finished exporting {} records for split '{}', total duration '{}'", @@ -385,21 +399,22 @@ private Threads.Timer getTableScanLogTimer() { } @Override - protected SnapshottingTask getSnapshottingTask(OffsetContext previousOffset) { + protected SnapshottingTask getSnapshottingTask( + PostgresPartition partition, PostgresOffsetContext previousOffset) { return new SnapshottingTask(false, true); } @Override - protected SnapshotContext prepare(ChangeEventSourceContext changeEventSourceContext) - throws Exception { - return new PostgresSnapshotContext(); + protected PostgresSnapshotContext prepare(PostgresPartition partition) throws Exception { + return new PostgresSnapshotContext(partition); } private static class PostgresSnapshotContext - extends RelationalSnapshotChangeEventSource.RelationalSnapshotContext { + extends RelationalSnapshotChangeEventSource.RelationalSnapshotContext< + PostgresPartition, PostgresOffsetContext> { - public PostgresSnapshotContext() throws SQLException { - super(""); + public PostgresSnapshotContext(PostgresPartition partition) throws SQLException { + super(partition, ""); } } } diff --git a/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/fetch/PostgresSourceFetchTaskContext.java b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/fetch/PostgresSourceFetchTaskContext.java index e9fd5cd7b5..3df7705cfb 100644 --- a/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/fetch/PostgresSourceFetchTaskContext.java +++ b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/fetch/PostgresSourceFetchTaskContext.java @@ -32,8 +32,10 @@ import io.debezium.connector.base.ChangeEventQueue; import io.debezium.connector.postgresql.PostgresConnectorConfig; import io.debezium.connector.postgresql.PostgresErrorHandler; +import io.debezium.connector.postgresql.PostgresEventDispatcher; import io.debezium.connector.postgresql.PostgresObjectUtils; import io.debezium.connector.postgresql.PostgresOffsetContext; +import io.debezium.connector.postgresql.PostgresPartition; import io.debezium.connector.postgresql.PostgresSchema; import io.debezium.connector.postgresql.PostgresTaskContext; import io.debezium.connector.postgresql.PostgresTopicSelector; @@ -45,7 +47,6 @@ import io.debezium.pipeline.ErrorHandler; import io.debezium.pipeline.metrics.DefaultChangeEventSourceMetricsFactory; import io.debezium.pipeline.metrics.SnapshotChangeEventSourceMetrics; -import io.debezium.pipeline.metrics.StreamingChangeEventSourceMetrics; import io.debezium.pipeline.metrics.spi.ChangeEventSourceMetricsFactory; import io.debezium.pipeline.source.spi.EventMetadataProvider; import io.debezium.relational.Column; @@ -69,6 +70,9 @@ /** The context of {@link PostgresScanFetchTask} and {@link PostgresStreamFetchTask}. */ public class PostgresSourceFetchTaskContext extends JdbcSourceFetchTaskContext { + + private static final String CONNECTION_NAME = "postgres-fetch-task-connection"; + private static final Logger LOG = LoggerFactory.getLogger(PostgresSourceFetchTaskContext.class); private PostgresTaskContext taskContext; @@ -77,12 +81,13 @@ public class PostgresSourceFetchTaskContext extends JdbcSourceFetchTaskContext { private final AtomicReference replicationConnection = new AtomicReference<>(); private PostgresOffsetContext offsetContext; + private PostgresPartition partition; private PostgresSchema schema; private ErrorHandler errorHandler; - private JdbcSourceEventDispatcher dispatcher; + private JdbcSourceEventDispatcher dispatcher; + private PostgresEventDispatcher postgresDispatcher; private EventMetadataProvider metadataProvider; - private SnapshotChangeEventSourceMetrics snapshotChangeEventSourceMetrics; - private StreamingChangeEventSourceMetrics streamingChangeEventSourceMetrics; + private SnapshotChangeEventSourceMetrics snapshotChangeEventSourceMetrics; private Snapshotter snapShotter; public PostgresSourceFetchTaskContext( @@ -119,7 +124,8 @@ public void configure(SourceSplitBase sourceSplitBase) { PostgresConnection.PostgresValueConverterBuilder valueConverterBuilder = newPostgresValueConverterBuilder(dbzConfig); this.jdbcConnection = - new PostgresConnection(dbzConfig.getJdbcConfig(), valueConverterBuilder); + new PostgresConnection( + dbzConfig.getJdbcConfig(), valueConverterBuilder, CONNECTION_NAME); TopicSelector topicSelector = PostgresTopicSelector.create(dbzConfig); EmbeddedFlinkDatabaseHistory.registerHistory( @@ -143,20 +149,22 @@ public void configure(SourceSplitBase sourceSplitBase) { this.offsetContext = loadStartingOffsetState( new PostgresOffsetContext.Loader(dbzConfig), sourceSplitBase); + this.partition = new PostgresPartition(dbzConfig.getLogicalName()); this.taskContext = PostgresObjectUtils.newTaskContext(dbzConfig, schema, topicSelector); this.replicationConnection.compareAndSet( null, createReplicationConnection( - this.taskContext, this.snapShotter.shouldSnapshot(), dbzConfig)); + this.taskContext, + jdbcConnection, + this.snapShotter.shouldSnapshot(), + dbzConfig)); - final int queueSize = - sourceSplitBase.isSnapshotSplit() ? Integer.MAX_VALUE : dbzConfig.getMaxQueueSize(); this.queue = new ChangeEventQueue.Builder() .pollInterval(dbzConfig.getPollInterval()) .maxBatchSize(dbzConfig.getMaxBatchSize()) - .maxQueueSize(queueSize) + .maxQueueSize(dbzConfig.getMaxQueueSize()) .maxQueueSizeInBytes(dbzConfig.getMaxQueueSizeInBytes()) .loggingContextSupplier( () -> @@ -166,10 +174,21 @@ public void configure(SourceSplitBase sourceSplitBase) { // .buffering() .build(); - this.errorHandler = new PostgresErrorHandler(dbzConnectorConfig.getLogicalName(), queue); + this.errorHandler = new PostgresErrorHandler(getDbzConnectorConfig(), queue); this.metadataProvider = PostgresObjectUtils.newEventMetadataProvider(); this.dispatcher = - new JdbcSourceEventDispatcher( + new JdbcSourceEventDispatcher<>( + dbzConfig, + topicSelector, + schema, + queue, + dbzConfig.getTableFilters().dataCollectionFilter(), + DataChangeEvent::new, + metadataProvider, + schemaNameAdjuster); + + this.postgresDispatcher = + new PostgresEventDispatcher<>( dbzConfig, topicSelector, schema, @@ -179,12 +198,10 @@ public void configure(SourceSplitBase sourceSplitBase) { metadataProvider, schemaNameAdjuster); - ChangeEventSourceMetricsFactory metricsFactory = - new DefaultChangeEventSourceMetricsFactory(); + ChangeEventSourceMetricsFactory metricsFactory = + new DefaultChangeEventSourceMetricsFactory<>(); this.snapshotChangeEventSourceMetrics = metricsFactory.getSnapshotMetrics(taskContext, queue, metadataProvider); - this.streamingChangeEventSourceMetrics = - metricsFactory.getStreamingMetrics(taskContext, queue, metadataProvider); } @Override @@ -204,15 +221,24 @@ public ErrorHandler getErrorHandler() { } @Override - public JdbcSourceEventDispatcher getDispatcher() { + public JdbcSourceEventDispatcher getDispatcher() { return dispatcher; } + public PostgresEventDispatcher getPostgresDispatcher() { + return postgresDispatcher; + } + @Override public PostgresOffsetContext getOffsetContext() { return offsetContext; } + @Override + public PostgresPartition getPartition() { + return partition; + } + @Override public ChangeEventQueue getQueue() { return queue; @@ -237,6 +263,11 @@ public Offset getStreamOffset(SourceRecord sourceRecord) { return PostgresOffset.of(sourceRecord); } + @Override + public void close() { + jdbcConnection.close(); + } + public PostgresConnection getConnection() { return jdbcConnection; } @@ -249,14 +280,11 @@ public ReplicationConnection getReplicationConnection() { return replicationConnection.get(); } - public SnapshotChangeEventSourceMetrics getSnapshotChangeEventSourceMetrics() { + public SnapshotChangeEventSourceMetrics + getSnapshotChangeEventSourceMetrics() { return snapshotChangeEventSourceMetrics; } - public StreamingChangeEventSourceMetrics getStreamingChangeEventSourceMetrics() { - return streamingChangeEventSourceMetrics; - } - public Snapshotter getSnapShotter() { return snapShotter; } diff --git a/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/fetch/PostgresStreamFetchTask.java b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/fetch/PostgresStreamFetchTask.java index 1c832e6274..fb5b56ec11 100644 --- a/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/fetch/PostgresStreamFetchTask.java +++ b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/fetch/PostgresStreamFetchTask.java @@ -25,7 +25,9 @@ import com.ververica.cdc.connectors.base.source.reader.external.FetchTask; import com.ververica.cdc.connectors.postgres.source.offset.PostgresOffset; import io.debezium.connector.postgresql.PostgresConnectorConfig; +import io.debezium.connector.postgresql.PostgresEventDispatcher; import io.debezium.connector.postgresql.PostgresOffsetContext; +import io.debezium.connector.postgresql.PostgresPartition; import io.debezium.connector.postgresql.PostgresSchema; import io.debezium.connector.postgresql.PostgresStreamingChangeEventSource; import io.debezium.connector.postgresql.PostgresTaskContext; @@ -35,6 +37,7 @@ import io.debezium.connector.postgresql.spi.Snapshotter; import io.debezium.pipeline.ErrorHandler; import io.debezium.pipeline.source.spi.ChangeEventSource; +import io.debezium.relational.TableId; import io.debezium.util.Clock; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -80,6 +83,7 @@ public void execute(Context context) throws Exception { sourceFetchContext.getSnapShotter(), sourceFetchContext.getConnection(), sourceFetchContext.getDispatcher(), + sourceFetchContext.getPostgresDispatcher(), sourceFetchContext.getErrorHandler(), sourceFetchContext.getTaskContext().getClock(), sourceFetchContext.getDatabaseSchema(), @@ -89,16 +93,19 @@ public void execute(Context context) throws Exception { StreamSplitChangeEventSourceContext changeEventSourceContext = new StreamSplitChangeEventSourceContext(); streamSplitReadTask.execute( - changeEventSourceContext, sourceFetchContext.getOffsetContext()); + changeEventSourceContext, + sourceFetchContext.getPartition(), + sourceFetchContext.getOffsetContext()); } @Override - public void stop() { + public void close() { LOG.debug("stopping StreamFetchTask for split: {}", split); if (streamSplitReadTask != null) { ((StreamSplitChangeEventSourceContext) streamSplitReadTask.context).finished(); } stopped = true; + taskRunning = false; } @Override @@ -154,7 +161,7 @@ public boolean isRunning() { public static class StreamSplitReadTask extends PostgresStreamingChangeEventSource { private static final Logger LOG = LoggerFactory.getLogger(StreamSplitReadTask.class); private final StreamSplit streamSplit; - private final JdbcSourceEventDispatcher dispatcher; + private final JdbcSourceEventDispatcher dispatcher; private final ErrorHandler errorHandler; public ChangeEventSourceContext context; @@ -164,7 +171,8 @@ public StreamSplitReadTask( PostgresConnectorConfig connectorConfig, Snapshotter snapshotter, PostgresConnection connection, - JdbcSourceEventDispatcher dispatcher, + JdbcSourceEventDispatcher dispatcher, + PostgresEventDispatcher postgresEventDispatcher, ErrorHandler errorHandler, Clock clock, PostgresSchema schema, @@ -176,7 +184,7 @@ public StreamSplitReadTask( connectorConfig, snapshotter, connection, - dispatcher, + postgresEventDispatcher, errorHandler, clock, schema, @@ -188,7 +196,10 @@ public StreamSplitReadTask( } @Override - public void execute(ChangeEventSourceContext context, PostgresOffsetContext offsetContext) + public void execute( + ChangeEventSourceContext context, + PostgresPartition partition, + PostgresOffsetContext offsetContext) throws InterruptedException { this.context = context; this.offsetContext = offsetContext; @@ -197,7 +208,7 @@ public void execute(ChangeEventSourceContext context, PostgresOffsetContext offs offsetContext.setStreamingStoppingLsn( ((PostgresOffset) streamSplit.getEndingOffset()).getLsn()); - super.execute(context, offsetContext); + super.execute(context, partition, offsetContext); if (isBoundedRead()) { LOG.debug("StreamSplit is bounded read: {}", streamSplit); @@ -208,7 +219,7 @@ public void execute(ChangeEventSourceContext context, PostgresOffsetContext offs try { dispatcher.dispatchWatermarkEvent( - offsetContext.getPartition(), + partition.getSourcePartition(), streamSplit, currentOffset, WatermarkKind.END); diff --git a/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/utils/CustomPostgresSchema.java b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/utils/CustomPostgresSchema.java index 6fa1579796..e594bf3816 100644 --- a/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/utils/CustomPostgresSchema.java +++ b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/utils/CustomPostgresSchema.java @@ -21,6 +21,7 @@ import com.ververica.cdc.connectors.postgres.source.config.PostgresSourceConfig; import io.debezium.connector.postgresql.PostgresConnectorConfig; import io.debezium.connector.postgresql.PostgresOffsetContext; +import io.debezium.connector.postgresql.PostgresPartition; import io.debezium.connector.postgresql.connection.PostgresConnection; import io.debezium.relational.Table; import io.debezium.relational.TableId; @@ -67,6 +68,8 @@ private TableChange readTableSchema(TableId tableId) throws SQLException { final PostgresOffsetContext offsetContext = PostgresOffsetContext.initialContext(dbzConfig, jdbcConnection, Clock.SYSTEM); + PostgresPartition partition = new PostgresPartition(dbzConfig.getLogicalName()); + // set the events to populate proper sourceInfo into offsetContext offsetContext.event(tableId, Instant.now()); @@ -87,15 +90,13 @@ private TableChange readTableSchema(TableId tableId) throws SQLException { // TODO: check whether we always set isFromSnapshot = true SchemaChangeEvent schemaChangeEvent = - new SchemaChangeEvent( - offsetContext.getPartition(), - offsetContext.getOffset(), - offsetContext.getSourceInfo(), + SchemaChangeEvent.ofCreate( + partition, + offsetContext, dbzConfig.databaseName(), tableId.schema(), null, table, - SchemaChangeEvent.SchemaChangeEventType.CREATE, true); for (TableChanges.TableChange tableChange : schemaChangeEvent.getTableChanges()) { diff --git a/flink-connector-postgres-cdc/src/main/java/io/debezium/connector/postgresql/PostgresObjectUtils.java b/flink-connector-postgres-cdc/src/main/java/io/debezium/connector/postgresql/PostgresObjectUtils.java index b6e064961b..8ad9ea8c12 100644 --- a/flink-connector-postgres-cdc/src/main/java/io/debezium/connector/postgresql/PostgresObjectUtils.java +++ b/flink-connector-postgres-cdc/src/main/java/io/debezium/connector/postgresql/PostgresObjectUtils.java @@ -48,7 +48,12 @@ public static PostgresSchema newSchema( PostgresValueConverter valueConverter) throws SQLException { PostgresSchema schema = - new PostgresSchema(config, typeRegistry, topicSelector, valueConverter); + new PostgresSchema( + config, + typeRegistry, + connection.getDefaultValueConverter(), + topicSelector, + valueConverter); schema.refresh(connection, false); return schema; } @@ -83,9 +88,11 @@ public static PostgresConnection.PostgresValueConverterBuilder newPostgresValueC // - old: ReplicationConnection createReplicationConnection(PostgresTaskContext taskContext, // boolean doSnapshot, int maxRetries, Duration retryDelay) // - new: ReplicationConnection createReplicationConnection(PostgresTaskContext taskContext, - // boolean doSnapshot, PostgresConnectorConfig connectorConfig) + // PostgresConnection postgresConnection, boolean doSnapshot, PostgresConnectorConfig + // connectorConfig) public static ReplicationConnection createReplicationConnection( PostgresTaskContext taskContext, + PostgresConnection postgresConnection, boolean doSnapshot, PostgresConnectorConfig connectorConfig) { int maxRetries = connectorConfig.maxRetries(); @@ -96,7 +103,7 @@ public static ReplicationConnection createReplicationConnection( while (retryCount <= maxRetries) { try { LOGGER.info("Creating a new replication connection for {}", taskContext); - return taskContext.createReplicationConnection(doSnapshot); + return taskContext.createReplicationConnection(doSnapshot, postgresConnection); } catch (SQLException ex) { retryCount++; if (retryCount > maxRetries) { diff --git a/flink-connector-postgres-cdc/src/main/java/io/debezium/connector/postgresql/connection/Lsn.java b/flink-connector-postgres-cdc/src/main/java/io/debezium/connector/postgresql/connection/Lsn.java new file mode 100644 index 0000000000..deb7bb7542 --- /dev/null +++ b/flink-connector-postgres-cdc/src/main/java/io/debezium/connector/postgresql/connection/Lsn.java @@ -0,0 +1,149 @@ +/* + * Copyright Debezium Authors. + * + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 + */ + +package io.debezium.connector.postgresql.connection; + +import org.postgresql.replication.LogSequenceNumber; + +import java.nio.ByteBuffer; + +/** + * Copied from Debezium 1.9.7 without changes due to the NoSuchMethodError, caused by the fact that + * current Debezium release java version is 11, so we need to compile this file by java 8 compiler. + * More + * info. Abstraction of PostgreSQL log sequence number, adapted from {@link LogSequenceNumber}. + */ +public class Lsn implements Comparable { + + /** + * Zero is used indicate an invalid pointer. Bootstrap skips the first possible WAL segment, + * initializing the first WAL page at XLOG_SEG_SIZE, so no XLOG record can begin at zero. + */ + public static final Lsn INVALID_LSN = new Lsn(0); + + private final long value; + + private Lsn(long value) { + this.value = value; + } + + /** + * @param value numeric represent position in the write-ahead log stream + * @return not null LSN instance + */ + public static Lsn valueOf(Long value) { + if (value == null) { + return null; + } + if (value == 0) { + return INVALID_LSN; + } + return new Lsn(value); + } + + /** + * @param value PostgreSQL JDBC driver domain type representing position in the write-ahead log + * stream + * @return not null LSN instance + */ + public static Lsn valueOf(LogSequenceNumber value) { + if (value.asLong() == 0) { + return INVALID_LSN; + } + return new Lsn(value.asLong()); + } + + /** + * Create LSN instance by string represent LSN. + * + * @param strValue not null string as two hexadecimal numbers of up to 8 digits each, separated + * by a slash. For example {@code 16/3002D50}, {@code 0/15D68C50} + * @return not null LSN instance where if specified string represent have not valid form {@link + * Lsn#INVALID_LSN} + */ + public static Lsn valueOf(String strValue) { + final int slashIndex = strValue.lastIndexOf('/'); + + if (slashIndex <= 0) { + return INVALID_LSN; + } + + final String logicalXLogStr = strValue.substring(0, slashIndex); + final int logicalXlog = (int) Long.parseLong(logicalXLogStr, 16); + final String segmentStr = strValue.substring(slashIndex + 1, strValue.length()); + final int segment = (int) Long.parseLong(segmentStr, 16); + + final ByteBuffer buf = ByteBuffer.allocate(8); + buf.putInt(logicalXlog); + buf.putInt(segment); + buf.position(0); + final long value = buf.getLong(); + + return Lsn.valueOf(value); + } + + /** @return Long represent position in the write-ahead log stream */ + public long asLong() { + return value; + } + + /** @return PostgreSQL JDBC driver representation of position in the write-ahead log stream */ + public LogSequenceNumber asLogSequenceNumber() { + return LogSequenceNumber.valueOf(value); + } + + /** + * @return String represent position in the write-ahead log stream as two hexadecimal numbers of + * up to 8 digits each, separated by a slash. For example {@code 16/3002D50}, {@code + * 0/15D68C50} + */ + public String asString() { + final ByteBuffer buf = ByteBuffer.allocate(8); + buf.putLong(value); + buf.position(0); + + final int logicalXlog = buf.getInt(); + final int segment = buf.getInt(); + return String.format("%X/%X", logicalXlog, segment); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + final Lsn that = (Lsn) o; + + return value == that.value; + } + + @Override + public int hashCode() { + return (int) (value ^ (value >>> 32)); + } + + public boolean isValid() { + return this != INVALID_LSN; + } + + @Override + public String toString() { + return "LSN{" + asString() + '}'; + } + + @Override + public int compareTo(Lsn o) { + if (value == o.value) { + return 0; + } + // Unsigned comparison + return value + Long.MIN_VALUE < o.value + Long.MIN_VALUE ? -1 : 1; + } +} diff --git a/flink-connector-postgres-cdc/src/main/java/io/debezium/connector/postgresql/connection/PostgresConnection.java b/flink-connector-postgres-cdc/src/main/java/io/debezium/connector/postgresql/connection/PostgresConnection.java index a64d9a117b..e728f3b474 100644 --- a/flink-connector-postgres-cdc/src/main/java/io/debezium/connector/postgresql/connection/PostgresConnection.java +++ b/flink-connector-postgres-cdc/src/main/java/io/debezium/connector/postgresql/connection/PostgresConnection.java @@ -30,7 +30,6 @@ import io.debezium.util.Metronome; import org.apache.kafka.connect.errors.ConnectException; import org.postgresql.core.BaseConnection; -import org.postgresql.core.ConnectionFactory; import org.postgresql.jdbc.PgConnection; import org.postgresql.jdbc.TimestampUtils; import org.postgresql.replication.LogSequenceNumber; @@ -40,7 +39,12 @@ import org.slf4j.LoggerFactory; import java.nio.charset.Charset; -import java.sql.*; +import java.sql.Connection; +import java.sql.DatabaseMetaData; +import java.sql.ResultSet; +import java.sql.ResultSetMetaData; +import java.sql.SQLException; +import java.sql.Statement; import java.time.Duration; import java.util.Objects; import java.util.Optional; @@ -50,17 +54,16 @@ * {@link JdbcConnection} connection extension used for connecting to Postgres instances. * * @author Horia Chiorean - *

Copied from Debezium 1.6.4-Final with two additional methods: - *

    - *
  • Constructor PostgresConnection( Configuration config, PostgresValueConverterBuilder - * valueConverterBuilder, ConnectionFactory factory) to allow passing a custom - * ConnectionFactory - *
  • override connection() to return a unwrapped PgConnection (otherwise, it will complain - * about HikariProxyConnection cannot be cast to class org.postgresql.core.BaseConnection) - *
*/ public class PostgresConnection extends JdbcConnection { + public static final String CONNECTION_STREAMING = "Debezium Streaming"; + public static final String CONNECTION_SLOT_INFO = "Debezium Slot Info"; + public static final String CONNECTION_DROP_SLOT = "Debezium Drop Slot"; + public static final String CONNECTION_VALIDATE_CONNECTION = "Debezium Validate Connection"; + public static final String CONNECTION_HEARTBEAT = "Debezium Heartbeat"; + public static final String CONNECTION_GENERAL = "Debezium General"; + private static Logger LOGGER = LoggerFactory.getLogger(PostgresConnection.class); private static final String URL_PATTERN = @@ -100,22 +103,38 @@ public class PostgresConnection extends JdbcConnection { * @param config {@link Configuration} instance, may not be null. * @param valueConverterBuilder supplies a configured {@link PostgresValueConverter} for a given * {@link TypeRegistry} + * @param connectionUsage a symbolic name of the connection to be tracked in monitoring tools */ public PostgresConnection( - Configuration config, PostgresValueConverterBuilder valueConverterBuilder) { - this(config, valueConverterBuilder, FACTORY); + JdbcConfiguration config, + PostgresValueConverterBuilder valueConverterBuilder, + String connectionUsage) { + this(config, valueConverterBuilder, connectionUsage, FACTORY); } - /** Creates a Postgres connection using the supplied configuration with customized factory */ + /** + * Creates a Postgres connection using the supplied configuration. If necessary this connection + * is able to resolve data type mappings. Such a connection requires a {@link + * PostgresValueConverter}, and will provide its own {@link TypeRegistry}. Usually only one such + * connection per connector is needed. + * + * @param config {@link Configuration} instance, may not be null. + * @param valueConverterBuilder supplies a configured {@link PostgresValueConverter} for a given + * {@link TypeRegistry} + * @param connectionUsage a symbolic name of the connection to be tracked in monitoring tools + */ public PostgresConnection( - Configuration config, + JdbcConfiguration config, PostgresValueConverterBuilder valueConverterBuilder, + String connectionUsage, ConnectionFactory factory) { super( - config, + addDefaultSettings(config, connectionUsage), factory, PostgresConnection::validateServerVersion, - PostgresConnection::defaultSettings); + null, + "\"", + "\""); if (Objects.isNull(valueConverterBuilder)) { this.typeRegistry = null; @@ -130,39 +149,29 @@ public PostgresConnection( } } - /** Return an unwrapped PgConnection instead of HikariProxyConnection */ - @Override - public synchronized Connection connection() throws SQLException { - Connection conn = connection(true); - if (conn instanceof HikariProxyConnection) { - // assuming HikariCP use org.postgresql.jdbc.PgConnection - return conn.unwrap(PgConnection.class); - } - return conn; - } - /** * Create a Postgres connection using the supplied configuration and {@link TypeRegistry} * * @param config {@link Configuration} instance, may not be null. * @param typeRegistry an existing/already-primed {@link TypeRegistry} instance + * @param connectionUsage a symbolic name of the connection to be tracked in monitoring tools */ - public PostgresConnection(Configuration config, TypeRegistry typeRegistry) { + public PostgresConnection( + PostgresConnectorConfig config, TypeRegistry typeRegistry, String connectionUsage) { super( - config, + addDefaultSettings(config.getJdbcConfig(), connectionUsage), FACTORY, PostgresConnection::validateServerVersion, - PostgresConnection::defaultSettings); + null, + "\"", + "\""); if (Objects.isNull(typeRegistry)) { this.typeRegistry = null; this.defaultValueConverter = null; } else { this.typeRegistry = typeRegistry; final PostgresValueConverter valueConverter = - PostgresValueConverter.of( - new PostgresConnectorConfig(config), - this.getDatabaseCharset(), - typeRegistry); + PostgresValueConverter.of(config, this.getDatabaseCharset(), typeRegistry); this.defaultValueConverter = new PostgresDefaultValueConverter(valueConverter, this.getTimestampUtils()); } @@ -173,9 +182,33 @@ public PostgresConnection(Configuration config, TypeRegistry typeRegistry) { * one without datatype resolution capabilities. * * @param config {@link Configuration} instance, may not be null. + * @param connectionUsage a symbolic name of the connection to be tracked in monitoring tools */ - public PostgresConnection(Configuration config) { - this(config, (TypeRegistry) null); + public PostgresConnection(JdbcConfiguration config, String connectionUsage) { + this(config, null, connectionUsage); + } + + /** Return an unwrapped PgConnection instead of HikariProxyConnection */ + @Override + public synchronized Connection connection() throws SQLException { + Connection conn = connection(true); + if (conn instanceof HikariProxyConnection) { + // assuming HikariCP use org.postgresql.jdbc.PgConnection + return conn.unwrap(PgConnection.class); + } + return conn; + } + + static JdbcConfiguration addDefaultSettings( + JdbcConfiguration configuration, String connectionUsage) { + // we require Postgres 9.4 as the minimum server version since that's where logical + // replication was first introduced + return JdbcConfiguration.adapt( + configuration + .edit() + .with("assumeMinServerVersion", "9.4") + .with("ApplicationName", connectionUsage) + .build()); } /** @@ -598,12 +631,6 @@ public TimestampUtils getTimestampUtils() { } } - protected static void defaultSettings(Configuration.Builder builder) { - // we require Postgres 9.4 as the minimum server version since that's where logical - // replication was first introduced - builder.with("assumeMinServerVersion", "9.4"); - } - private static void validateServerVersion(Statement statement) throws SQLException { DatabaseMetaData metaData = statement.getConnection().getMetaData(); int majorVersion = metaData.getDatabaseMajorVersion(); @@ -613,6 +640,15 @@ private static void validateServerVersion(Statement statement) throws SQLExcepti } } + @Override + public String quotedColumnIdString(String columnName) { + if (columnName.contains("\"")) { + columnName = columnName.replaceAll("\"", "\"\""); + } + + return super.quotedColumnIdString(columnName); + } + @Override protected int resolveNativeType(String typeName) { return getTypeRegistry().get(typeName).getRootType().getOid(); @@ -684,9 +720,10 @@ private Optional doReadTableColumn( column.scale(nativeType.getDefaultScale()); } - final String defaultValue = columnMetadata.getString(13); - if (defaultValue != null) { - getDefaultValue(column.create(), defaultValue).ifPresent(column::defaultValue); + final String defaultValueExpression = columnMetadata.getString(13); + if (defaultValueExpression != null + && getDefaultValueConverter().supportConversion(column.typeName())) { + column.defaultValueExpression(defaultValueExpression); } return Optional.of(column); @@ -695,9 +732,10 @@ private Optional doReadTableColumn( return Optional.empty(); } - @Override - protected Optional getDefaultValue(Column column, String defaultValue) { - return defaultValueConverter.parseDefaultValue(column, defaultValue); + public PostgresDefaultValueConverter getDefaultValueConverter() { + Objects.requireNonNull( + defaultValueConverter, "Connection does not provide default value converter"); + return defaultValueConverter; } public TypeRegistry getTypeRegistry() { @@ -774,6 +812,16 @@ public > Object getColumnValue( } } + @Override + protected String[] supportedTableTypes() { + return new String[] {"VIEW", "MATERIALIZED VIEW", "TABLE", "PARTITIONED TABLE"}; + } + + @Override + protected boolean isTableType(String tableType) { + return "TABLE".equals(tableType) || "PARTITIONED TABLE".equals(tableType); + } + @FunctionalInterface public interface PostgresValueConverterBuilder { PostgresValueConverter build(TypeRegistry registry); diff --git a/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/PostgresTestBase.java b/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/PostgresTestBase.java index 880b4d1ee5..8b496ea9dc 100644 --- a/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/PostgresTestBase.java +++ b/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/PostgresTestBase.java @@ -18,6 +18,9 @@ import org.apache.flink.test.util.AbstractTestBase; +import io.debezium.config.Configuration; +import io.debezium.connector.postgresql.connection.PostgresConnection; +import io.debezium.jdbc.JdbcConfiguration; import org.junit.BeforeClass; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -35,6 +38,7 @@ import java.sql.Statement; import java.util.Arrays; import java.util.List; +import java.util.Map; import java.util.regex.Matcher; import java.util.regex.Pattern; import java.util.stream.Collectors; @@ -125,4 +129,9 @@ protected void initializePostgresTable(String sqlFile) { throw new RuntimeException(e); } } + + protected PostgresConnection createConnection(Map properties) { + Configuration config = Configuration.from(properties); + return new PostgresConnection(JdbcConfiguration.adapt(config), "test-connection"); + } } diff --git a/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/source/PostgresSourceExampleTest.java b/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/source/PostgresSourceExampleTest.java index 2ac7ecf040..849e12ef13 100644 --- a/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/source/PostgresSourceExampleTest.java +++ b/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/source/PostgresSourceExampleTest.java @@ -37,7 +37,6 @@ import com.ververica.cdc.debezium.DebeziumDeserializationSchema; import com.ververica.cdc.debezium.JsonDebeziumDeserializationSchema; import com.ververica.cdc.debezium.table.RowDataDebeziumDeserializeSchema; -import io.debezium.config.Configuration; import io.debezium.connector.postgresql.connection.PostgresConnection; import io.debezium.connector.postgresql.spi.SlotState; import org.junit.Ignore; @@ -253,7 +252,7 @@ private PostgresConnection getConnection() throws SQLException { properties.put("dbname", inventoryDatabase.getDatabaseName()); properties.put("user", inventoryDatabase.getUsername()); properties.put("password", inventoryDatabase.getPassword()); - PostgresConnection connection = new PostgresConnection(Configuration.from(properties)); + PostgresConnection connection = createConnection(properties); connection.connect(); return connection; } diff --git a/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/source/PostgresSourceITCase.java b/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/source/PostgresSourceITCase.java index 701697d9df..0132c0b374 100644 --- a/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/source/PostgresSourceITCase.java +++ b/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/source/PostgresSourceITCase.java @@ -457,9 +457,7 @@ private PostgresConnection getConnection() { properties.put("user", customDatabase.getUsername()); properties.put("password", customDatabase.getPassword()); properties.put("dbname", customDatabase.getDatabaseName()); - io.debezium.config.Configuration configuration = - io.debezium.config.Configuration.from(properties); - return new PostgresConnection(configuration); + return createConnection(properties); } // ------------------------------------------------------------------------ diff --git a/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/source/fetch/PostgresSourceFetchTaskContextTest.java b/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/source/fetch/PostgresSourceFetchTaskContextTest.java index ec4457df09..f52ffae228 100644 --- a/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/source/fetch/PostgresSourceFetchTaskContextTest.java +++ b/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/source/fetch/PostgresSourceFetchTaskContextTest.java @@ -35,7 +35,7 @@ public class PostgresSourceFetchTaskContextTest { private PostgresConnectorConfig connectorConfig; - private OffsetContext.Loader offsetLoader; + private OffsetContext.Loader offsetLoader; @Before public void beforeEach() { @@ -44,14 +44,13 @@ public void beforeEach() { } @Test - public void shouldNotResetLsnWhenLastCommitLsnIsNull() throws Exception { + public void shouldNotResetLsnWhenLastCommitLsnIsNull() { final Map offsetValues = new HashMap<>(); offsetValues.put(SourceInfo.LSN_KEY, 12345L); offsetValues.put(SourceInfo.TIMESTAMP_USEC_KEY, 67890L); offsetValues.put(PostgresOffsetContext.LAST_COMMIT_LSN_KEY, null); - final PostgresOffsetContext offsetContext = - (PostgresOffsetContext) offsetLoader.load(offsetValues); + final PostgresOffsetContext offsetContext = offsetLoader.load(offsetValues); assertEquals(lastKnownLsn(offsetContext), Lsn.valueOf(12345L)); } } diff --git a/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/dialect/SqlServerDialect.java b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/dialect/SqlServerDialect.java index dc12b39f50..b524e5f88b 100644 --- a/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/dialect/SqlServerDialect.java +++ b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/dialect/SqlServerDialect.java @@ -83,7 +83,7 @@ public boolean isDataCollectionIdCaseSensitive(JdbcSourceConfig sourceConfig) { @Override public JdbcConnection openJdbcConnection(JdbcSourceConfig sourceConfig) { - return createSqlServerConnection(sourceConfig.getDbzConnectorConfig().getJdbcConfig()); + return createSqlServerConnection(sourceConfig.getDbzConnectorConfig()); } @Override @@ -112,7 +112,7 @@ public Map discoverDataCollectionSchemas(JdbcSourceConfig final List capturedTableIds = discoverDataCollections(sourceConfig); try (SqlServerConnection jdbc = - createSqlServerConnection(sourceConfig.getDbzConnectorConfig().getJdbcConfig())) { + createSqlServerConnection(sourceConfig.getDbzConnectorConfig())) { // fetch table schemas Map tableSchemas = new HashMap<>(); for (TableId tableId : capturedTableIds) { @@ -138,9 +138,9 @@ public TableChange queryTableSchema(JdbcConnection jdbc, TableId tableId) { public SqlServerSourceFetchTaskContext createFetchTaskContext( SourceSplitBase sourceSplitBase, JdbcSourceConfig taskSourceConfig) { final SqlServerConnection jdbcConnection = - createSqlServerConnection(sourceConfig.getDbzConnectorConfig().getJdbcConfig()); + createSqlServerConnection(sourceConfig.getDbzConnectorConfig()); final SqlServerConnection metaDataConnection = - createSqlServerConnection(sourceConfig.getDbzConnectorConfig().getJdbcConfig()); + createSqlServerConnection(sourceConfig.getDbzConnectorConfig()); return new SqlServerSourceFetchTaskContext( taskSourceConfig, this, jdbcConnection, metaDataConnection); } diff --git a/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/reader/fetch/SqlServerScanFetchTask.java b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/reader/fetch/SqlServerScanFetchTask.java index b60559c395..5f7fdea8e5 100644 --- a/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/reader/fetch/SqlServerScanFetchTask.java +++ b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/reader/fetch/SqlServerScanFetchTask.java @@ -30,15 +30,14 @@ import io.debezium.connector.sqlserver.SqlServerConnectorConfig; import io.debezium.connector.sqlserver.SqlServerDatabaseSchema; import io.debezium.connector.sqlserver.SqlServerOffsetContext; +import io.debezium.connector.sqlserver.SqlServerPartition; import io.debezium.heartbeat.Heartbeat; import io.debezium.pipeline.EventDispatcher; import io.debezium.pipeline.source.AbstractSnapshotChangeEventSource; import io.debezium.pipeline.source.spi.ChangeEventSource; import io.debezium.pipeline.source.spi.SnapshotProgressListener; import io.debezium.pipeline.spi.ChangeRecordEmitter; -import io.debezium.pipeline.spi.OffsetContext; import io.debezium.pipeline.spi.SnapshotResult; -import io.debezium.relational.Column; import io.debezium.relational.RelationalSnapshotChangeEventSource; import io.debezium.relational.SnapshotChangeRecordEmitter; import io.debezium.relational.Table; @@ -53,9 +52,7 @@ import java.sql.PreparedStatement; import java.sql.ResultSet; -import java.sql.ResultSetMetaData; import java.sql.SQLException; -import java.sql.Types; import java.time.Duration; import java.util.ArrayList; import java.util.Map; @@ -93,9 +90,11 @@ public void execute(Context context) throws Exception { split); SnapshotSplitChangeEventSourceContext changeEventSourceContext = new SnapshotSplitChangeEventSourceContext(); - SnapshotResult snapshotResult = + SnapshotResult snapshotResult = snapshotSplitReadTask.execute( - changeEventSourceContext, sourceFetchContext.getOffsetContext()); + changeEventSourceContext, + sourceFetchContext.getPartition(), + sourceFetchContext.getOffsetContext()); final StreamSplit backfillBinlogSplit = createBackFillLsnSplit(changeEventSourceContext); // optimization that skip the binlog read when the low watermark equals high @@ -107,8 +106,8 @@ public void execute(Context context) throws Exception { if (!binlogBackfillRequired) { dispatchLsnEndEvent( backfillBinlogSplit, - ((SqlServerSourceFetchTaskContext) context).getOffsetContext().getPartition(), - ((SqlServerSourceFetchTaskContext) context).getDispatcher()); + sourceFetchContext.getPartition().getSourcePartition(), + sourceFetchContext.getDispatcher()); taskRunning = false; return; } @@ -122,7 +121,9 @@ public void execute(Context context) throws Exception { final LsnSplitReadTask backfillBinlogReadTask = createBackFillLsnSplitReadTask(backfillBinlogSplit, sourceFetchContext); backfillBinlogReadTask.execute( - new SnapshotBinlogSplitChangeEventSourceContext(), streamOffsetContext); + new SnapshotBinlogSplitChangeEventSourceContext(), + sourceFetchContext.getPartition(), + streamOffsetContext); } else { taskRunning = false; throw new IllegalStateException( @@ -144,7 +145,7 @@ private StreamSplit createBackFillLsnSplit( private void dispatchLsnEndEvent( StreamSplit backFillBinlogSplit, Map sourcePartition, - JdbcSourceEventDispatcher eventDispatcher) + JdbcSourceEventDispatcher eventDispatcher) throws InterruptedException { eventDispatcher.dispatchWatermarkEvent( sourcePartition, @@ -186,8 +187,14 @@ public SourceSplitBase getSplit() { return split; } + @Override + public void close() { + taskRunning = false; + } + /** A wrapped task to fetch snapshot split of table. */ - public static class SqlServerSnapshotSplitReadTask extends AbstractSnapshotChangeEventSource { + public static class SqlServerSnapshotSplitReadTask + extends AbstractSnapshotChangeEventSource { private static final Logger LOG = LoggerFactory.getLogger(SqlServerSnapshotSplitReadTask.class); @@ -198,21 +205,21 @@ public static class SqlServerSnapshotSplitReadTask extends AbstractSnapshotChang private final SqlServerConnectorConfig connectorConfig; private final SqlServerDatabaseSchema databaseSchema; private final SqlServerConnection jdbcConnection; - private final JdbcSourceEventDispatcher dispatcher; + private final JdbcSourceEventDispatcher dispatcher; private final Clock clock; private final SnapshotSplit snapshotSplit; private final SqlServerOffsetContext offsetContext; - private final SnapshotProgressListener snapshotProgressListener; - private final EventDispatcher.SnapshotReceiver snapshotReceiver; + private final SnapshotProgressListener snapshotProgressListener; + private final EventDispatcher.SnapshotReceiver snapshotReceiver; public SqlServerSnapshotSplitReadTask( SqlServerConnectorConfig connectorConfig, SqlServerOffsetContext previousOffset, - SnapshotProgressListener snapshotProgressListener, + SnapshotProgressListener snapshotProgressListener, SqlServerDatabaseSchema databaseSchema, SqlServerConnection jdbcConnection, - JdbcSourceEventDispatcher dispatcher, - EventDispatcher.SnapshotReceiver snapshotReceiver, + JdbcSourceEventDispatcher dispatcher, + EventDispatcher.SnapshotReceiver snapshotReceiver, SnapshotSplit snapshotSplit) { super(connectorConfig, snapshotProgressListener); this.offsetContext = previousOffset; @@ -227,13 +234,15 @@ public SqlServerSnapshotSplitReadTask( } @Override - public SnapshotResult execute( - ChangeEventSourceContext context, OffsetContext previousOffset) + public SnapshotResult execute( + ChangeEventSourceContext context, + SqlServerPartition partition, + SqlServerOffsetContext previousOffset) throws InterruptedException { - SnapshottingTask snapshottingTask = getSnapshottingTask(previousOffset); - final SnapshotContext ctx; + SnapshottingTask snapshottingTask = getSnapshottingTask(partition, previousOffset); + final SqlSeverSnapshotContext ctx; try { - ctx = prepare(context); + ctx = prepare(partition); } catch (Exception e) { LOG.error("Failed to initialize snapshot context.", e); throw new RuntimeException(e); @@ -249,14 +258,13 @@ public SnapshotResult execute( } @Override - protected SnapshotResult doExecute( + protected SnapshotResult doExecute( ChangeEventSourceContext context, - OffsetContext previousOffset, - SnapshotContext snapshotContext, + SqlServerOffsetContext previousOffset, + SnapshotContext snapshotContext, SnapshottingTask snapshottingTask) throws Exception { - final RelationalSnapshotChangeEventSource.RelationalSnapshotContext ctx = - (RelationalSnapshotChangeEventSource.RelationalSnapshotContext) snapshotContext; + final SqlSeverSnapshotContext ctx = (SqlSeverSnapshotContext) snapshotContext; ctx.offset = offsetContext; final LsnOffset lowWatermark = currentLsn(jdbcConnection); @@ -266,7 +274,10 @@ protected SnapshotResult doExecute( snapshotSplit); ((SnapshotSplitChangeEventSourceContext) (context)).setLowWatermark(lowWatermark); dispatcher.dispatchWatermarkEvent( - offsetContext.getPartition(), snapshotSplit, lowWatermark, WatermarkKind.LOW); + snapshotContext.partition.getSourcePartition(), + snapshotSplit, + lowWatermark, + WatermarkKind.LOW); LOG.info("Snapshot step 2 - Snapshotting data"); createDataEvents(ctx, snapshotSplit.getTableId()); @@ -278,24 +289,25 @@ protected SnapshotResult doExecute( snapshotSplit); ((SnapshotSplitChangeEventSourceContext) (context)).setHighWatermark(highWatermark); dispatcher.dispatchWatermarkEvent( - offsetContext.getPartition(), snapshotSplit, highWatermark, WatermarkKind.HIGH); + snapshotContext.partition.getSourcePartition(), + snapshotSplit, + highWatermark, + WatermarkKind.HIGH); return SnapshotResult.completed(ctx.offset); } @Override - protected SnapshottingTask getSnapshottingTask(OffsetContext previousOffset) { + protected SnapshottingTask getSnapshottingTask( + SqlServerPartition partition, SqlServerOffsetContext previousOffset) { return new SnapshottingTask(false, true); } @Override - protected SnapshotContext prepare(ChangeEventSourceContext changeEventSourceContext) - throws Exception { - return new SqlSeverSnapshotContext(); + protected SqlSeverSnapshotContext prepare(SqlServerPartition partition) throws Exception { + return new SqlSeverSnapshotContext(partition); } - private void createDataEvents( - RelationalSnapshotChangeEventSource.RelationalSnapshotContext snapshotContext, - TableId tableId) + private void createDataEvents(SqlSeverSnapshotContext snapshotContext, TableId tableId) throws Exception { LOG.debug("Snapshotting table {}", tableId); createDataEventsForTable( @@ -305,8 +317,8 @@ private void createDataEvents( /** Dispatches the data change events for the records of a single table. */ private void createDataEventsForTable( - RelationalSnapshotChangeEventSource.RelationalSnapshotContext snapshotContext, - EventDispatcher.SnapshotReceiver snapshotReceiver, + SqlSeverSnapshotContext snapshotContext, + EventDispatcher.SnapshotReceiver snapshotReceiver, Table table) throws InterruptedException { @@ -346,12 +358,8 @@ private void createDataEventsForTable( 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, table); - } + final Object[] row = + jdbcConnection.rowToArray(table, databaseSchema, rs, columnArray); if (logTimer.expired()) { long stop = clock.currentTimeInMillis(); LOG.info( @@ -359,10 +367,12 @@ private void createDataEventsForTable( rows, snapshotSplit.splitId(), Strings.duration(stop - exportStart)); - snapshotProgressListener.rowsScanned(table.id(), rows); + snapshotProgressListener.rowsScanned( + snapshotContext.partition, table.id(), rows); logTimer = getTableScanLogTimer(); } dispatcher.dispatchSnapshotEvent( + snapshotContext.partition, table.id(), getChangeRecordEmitter(snapshotContext, table.id(), row), snapshotReceiver); @@ -377,38 +387,23 @@ private void createDataEventsForTable( } } - protected ChangeRecordEmitter getChangeRecordEmitter( - SnapshotContext snapshotContext, TableId tableId, Object[] row) { + protected ChangeRecordEmitter getChangeRecordEmitter( + SqlSeverSnapshotContext snapshotContext, TableId tableId, Object[] row) { snapshotContext.offset.event(tableId, clock.currentTime()); - return new SnapshotChangeRecordEmitter(snapshotContext.offset, row, clock); + return new SnapshotChangeRecordEmitter<>( + snapshotContext.partition, snapshotContext.offset, row, clock); } private Threads.Timer getTableScanLogTimer() { return Threads.timer(clock, LOG_INTERVAL); } - /** - * copied from - * io.debezium.connector.SqlServer.antlr.listener.ParserUtils#convertValueToSchemaType. - */ - private Object readField( - ResultSet rs, int columnIndex, Column actualColumn, Table actualTable) - throws SQLException { - final ResultSetMetaData metaData = rs.getMetaData(); - final int columnType = metaData.getColumnType(columnIndex); - - if (columnType == Types.TIME) { - return rs.getTimestamp(columnIndex); - } else { - return rs.getObject(columnIndex); - } - } - private static class SqlSeverSnapshotContext - extends RelationalSnapshotChangeEventSource.RelationalSnapshotContext { + extends RelationalSnapshotChangeEventSource.RelationalSnapshotContext< + SqlServerPartition, SqlServerOffsetContext> { - public SqlSeverSnapshotContext() throws SQLException { - super(""); + public SqlSeverSnapshotContext(SqlServerPartition partition) throws SQLException { + super(partition, ""); } } } @@ -417,7 +412,7 @@ public SqlSeverSnapshotContext() throws SQLException { * {@link ChangeEventSource.ChangeEventSourceContext} implementation that keeps low/high * watermark for each {@link SnapshotSplit}. */ - public class SnapshotSplitChangeEventSourceContext + public static class SnapshotSplitChangeEventSourceContext implements ChangeEventSource.ChangeEventSourceContext { private LsnOffset lowWatermark; diff --git a/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/reader/fetch/SqlServerSourceFetchTaskContext.java b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/reader/fetch/SqlServerSourceFetchTaskContext.java index 0ee2e87704..8e60a91c09 100644 --- a/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/reader/fetch/SqlServerSourceFetchTaskContext.java +++ b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/reader/fetch/SqlServerSourceFetchTaskContext.java @@ -38,6 +38,7 @@ import io.debezium.connector.sqlserver.SqlServerErrorHandler; import io.debezium.connector.sqlserver.SqlServerOffsetContext; import io.debezium.connector.sqlserver.SqlServerOffsetContext.Loader; +import io.debezium.connector.sqlserver.SqlServerPartition; import io.debezium.connector.sqlserver.SqlServerTaskContext; import io.debezium.connector.sqlserver.SqlServerTopicSelector; import io.debezium.data.Envelope.FieldName; @@ -58,12 +59,16 @@ import io.debezium.util.SchemaNameAdjuster; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.source.SourceRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.time.Instant; import java.util.Map; /** The context for fetch task that fetching data of snapshot split from SqlServer data source. */ public class SqlServerSourceFetchTaskContext extends JdbcSourceFetchTaskContext { + private static final Logger LOG = + LoggerFactory.getLogger(SqlServerSourceFetchTaskContext.class); /** Connection used for reading CDC tables. */ private final SqlServerConnection connection; @@ -79,15 +84,16 @@ public class SqlServerSourceFetchTaskContext extends JdbcSourceFetchTaskContext private final SqlServerEventMetadataProvider metadataProvider; private SqlServerOffsetContext offsetContext; + private SqlServerPartition partition; private SqlServerDatabaseSchema databaseSchema; - private JdbcSourceEventDispatcher dispatcher; + private JdbcSourceEventDispatcher dispatcher; private SqlServerErrorHandler errorHandler; private ChangeEventQueue queue; private SqlServerTaskContext taskContext; private TopicSelector topicSelector; - private EventDispatcher.SnapshotReceiver snapshotReceiver; - private SnapshotChangeEventSourceMetrics snapshotChangeEventSourceMetrics; - private StreamingChangeEventSourceMetrics streamingChangeEventSourceMetrics; + private EventDispatcher.SnapshotReceiver snapshotReceiver; + private SnapshotChangeEventSourceMetrics snapshotChangeEventSourceMetrics; + private StreamingChangeEventSourceMetrics streamingChangeEventSourceMetrics; public SqlServerSourceFetchTaskContext( JdbcSourceConfig sourceConfig, @@ -111,16 +117,19 @@ public void configure(SourceSplitBase sourceSplitBase) { .getString(EmbeddedFlinkDatabaseHistory.DATABASE_HISTORY_INSTANCE_NAME), sourceSplitBase.getTableSchemas().values()); - this.databaseSchema = SqlServerUtils.createSqlServerDatabaseSchema(connectorConfig); + this.databaseSchema = + SqlServerUtils.createSqlServerDatabaseSchema(connectorConfig, connection); this.offsetContext = loadStartingOffsetState(new Loader(connectorConfig), sourceSplitBase); + + String serverName = connectorConfig.getLogicalName(); + String dbName = connectorConfig.getJdbcConfig().getDatabase(); + this.partition = new SqlServerPartition(serverName, dbName, false); + validateAndLoadDatabaseHistory(offsetContext, databaseSchema); this.taskContext = new SqlServerTaskContext(connectorConfig, databaseSchema); - final int queueSize = - sourceSplitBase.isSnapshotSplit() - ? Integer.MAX_VALUE - : getSourceConfig().getDbzConnectorConfig().getMaxQueueSize(); + final int queueSize = getSourceConfig().getDbzConnectorConfig().getMaxQueueSize(); this.queue = new Builder() .pollInterval(connectorConfig.getPollInterval()) @@ -135,7 +144,7 @@ public void configure(SourceSplitBase sourceSplitBase) { // .buffering() .build(); this.dispatcher = - new JdbcSourceEventDispatcher( + new JdbcSourceEventDispatcher<>( connectorConfig, topicSelector, databaseSchema, @@ -147,15 +156,15 @@ public void configure(SourceSplitBase sourceSplitBase) { this.snapshotReceiver = dispatcher.getSnapshotChangeEventReceiver(); - final DefaultChangeEventSourceMetricsFactory changeEventSourceMetricsFactory = - new DefaultChangeEventSourceMetricsFactory(); + final DefaultChangeEventSourceMetricsFactory + changeEventSourceMetricsFactory = new DefaultChangeEventSourceMetricsFactory<>(); this.snapshotChangeEventSourceMetrics = changeEventSourceMetricsFactory.getSnapshotMetrics( taskContext, queue, metadataProvider); this.streamingChangeEventSourceMetrics = changeEventSourceMetricsFactory.getStreamingMetrics( taskContext, queue, metadataProvider); - this.errorHandler = new SqlServerErrorHandler(connectorConfig.getLogicalName(), queue); + this.errorHandler = new SqlServerErrorHandler(connectorConfig, queue); } /** Loads the connector's persistent offset (if present) via the given loader. */ @@ -166,14 +175,13 @@ private SqlServerOffsetContext loadStartingOffsetState( ? LsnOffset.INITIAL_OFFSET : sourceSplitBase.asStreamSplit().getStartingOffset(); - SqlServerOffsetContext sqlServerOffsetContext = loader.load(offset.getOffset()); - return sqlServerOffsetContext; + return loader.load(offset.getOffset()); } private void validateAndLoadDatabaseHistory( SqlServerOffsetContext offset, SqlServerDatabaseSchema schema) { schema.initializeStorage(); - schema.recover(offset); + schema.recover(partition, offset); } @Override @@ -217,11 +225,11 @@ public SqlServerSourceConfig getSourceConfig() { } @Override - public JdbcSourceEventDispatcher getDispatcher() { + public JdbcSourceEventDispatcher getDispatcher() { return dispatcher; } - public EventDispatcher.SnapshotReceiver getSnapshotReceiver() { + public EventDispatcher.SnapshotReceiver getSnapshotReceiver() { return snapshotReceiver; } @@ -230,6 +238,11 @@ public SqlServerOffsetContext getOffsetContext() { return offsetContext; } + @Override + public SqlServerPartition getPartition() { + return partition; + } + public SqlServerConnection getConnection() { return connection; } @@ -238,11 +251,13 @@ public SqlServerConnection getMetaDataConnection() { return metaDataConnection; } - public SnapshotChangeEventSourceMetrics getSnapshotChangeEventSourceMetrics() { + public SnapshotChangeEventSourceMetrics + getSnapshotChangeEventSourceMetrics() { return snapshotChangeEventSourceMetrics; } - public StreamingChangeEventSourceMetrics getStreamingChangeEventSourceMetrics() { + public StreamingChangeEventSourceMetrics + getStreamingChangeEventSourceMetrics() { return streamingChangeEventSourceMetrics; } @@ -256,6 +271,12 @@ public boolean isDataChangeRecord(SourceRecord record) { return SourceRecordUtils.isDataChangeRecord(record); } + @Override + public void close() throws Exception { + metaDataConnection.close(); + connection.close(); + } + @Override public SchemaNameAdjuster getSchemaNameAdjuster() { return schemaNameAdjuster; diff --git a/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/reader/fetch/SqlServerStreamFetchTask.java b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/reader/fetch/SqlServerStreamFetchTask.java index 1a901bbd83..22c7dd0f5f 100644 --- a/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/reader/fetch/SqlServerStreamFetchTask.java +++ b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/reader/fetch/SqlServerStreamFetchTask.java @@ -27,6 +27,7 @@ import io.debezium.connector.sqlserver.SqlServerConnectorConfig; import io.debezium.connector.sqlserver.SqlServerDatabaseSchema; import io.debezium.connector.sqlserver.SqlServerOffsetContext; +import io.debezium.connector.sqlserver.SqlServerPartition; import io.debezium.connector.sqlserver.SqlServerStreamingChangeEventSource; import io.debezium.pipeline.ErrorHandler; import io.debezium.pipeline.source.spi.ChangeEventSource; @@ -66,7 +67,9 @@ public void execute(Context context) throws Exception { RedoLogSplitChangeEventSourceContext changeEventSourceContext = new RedoLogSplitChangeEventSourceContext(); redoLogSplitReadTask.execute( - changeEventSourceContext, sourceFetchContext.getOffsetContext()); + changeEventSourceContext, + sourceFetchContext.getPartition(), + sourceFetchContext.getOffsetContext()); } @Override @@ -79,6 +82,11 @@ public StreamSplit getSplit() { return split; } + @Override + public void close() { + taskRunning = false; + } + /** * A wrapped task to read all binlog for table and also supports read bounded (from lowWatermark * to highWatermark) binlog. @@ -87,7 +95,7 @@ public static class LsnSplitReadTask extends SqlServerStreamingChangeEventSource private static final Logger LOG = LoggerFactory.getLogger(LsnSplitReadTask.class); private final StreamSplit lsnSplit; - private final JdbcSourceEventDispatcher dispatcher; + private final JdbcSourceEventDispatcher dispatcher; private final ErrorHandler errorHandler; private ChangeEventSourceContext context; @@ -95,7 +103,7 @@ public LsnSplitReadTask( SqlServerConnectorConfig connectorConfig, SqlServerConnection connection, SqlServerConnection metadataConnection, - JdbcSourceEventDispatcher dispatcher, + JdbcSourceEventDispatcher dispatcher, ErrorHandler errorHandler, SqlServerDatabaseSchema schema, StreamSplit lsnSplit) { @@ -113,7 +121,8 @@ public LsnSplitReadTask( } @Override - public void afterHandleLsn(SqlServerOffsetContext offsetContext) { + public void afterHandleLsn( + SqlServerPartition partition, SqlServerOffsetContext offsetContext) { // check do we need to stop for fetch binlog for snapshot split. if (isBoundedRead()) { final LsnOffset currentRedoLogOffset = getLsnPosition(offsetContext.getOffset()); @@ -122,7 +131,7 @@ public void afterHandleLsn(SqlServerOffsetContext offsetContext) { // send binlog end event try { dispatcher.dispatchWatermarkEvent( - offsetContext.getPartition(), + partition.getSourcePartition(), lsnSplit, currentRedoLogOffset, WatermarkKind.END); @@ -143,10 +152,13 @@ private boolean isBoundedRead() { } @Override - public void execute(ChangeEventSourceContext context, SqlServerOffsetContext offsetContext) + public void execute( + ChangeEventSourceContext context, + SqlServerPartition partition, + SqlServerOffsetContext offsetContext) throws InterruptedException { this.context = context; - super.execute(context, offsetContext); + super.execute(context, partition, offsetContext); } } diff --git a/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/utils/SqlServerConnectionUtils.java b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/utils/SqlServerConnectionUtils.java index f84a4d842e..1ed6a2e6a0 100644 --- a/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/utils/SqlServerConnectionUtils.java +++ b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/utils/SqlServerConnectionUtils.java @@ -20,10 +20,11 @@ import io.debezium.connector.sqlserver.SqlServerConnection; import io.debezium.connector.sqlserver.SqlServerConnectorConfig; import io.debezium.connector.sqlserver.SqlServerValueConverters; +import io.debezium.jdbc.JdbcConfiguration; import io.debezium.jdbc.JdbcConnection; +import io.debezium.relational.RelationalDatabaseConnectorConfig; import io.debezium.relational.RelationalTableFilters; import io.debezium.relational.TableId; -import io.debezium.util.Clock; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -35,21 +36,22 @@ public class SqlServerConnectionUtils { private static final Logger LOG = LoggerFactory.getLogger(SqlServerConnectionUtils.class); - public static SqlServerConnection createSqlServerConnection(Configuration dbzConfiguration) { - final SqlServerConnectorConfig connectorConfig = - new SqlServerConnectorConfig(dbzConfiguration); + public static SqlServerConnection createSqlServerConnection( + RelationalDatabaseConnectorConfig connectorConfig) { + Configuration dbzConnectorConfig = connectorConfig.getJdbcConfig(); + final SqlServerValueConverters valueConverters = new SqlServerValueConverters( connectorConfig.getDecimalMode(), connectorConfig.getTemporalPrecisionMode(), connectorConfig.binaryHandlingMode()); return new SqlServerConnection( - dbzConfiguration, - Clock.system(), - connectorConfig.getSourceTimestampMode(), + JdbcConfiguration.adapt(dbzConnectorConfig), + ((SqlServerConnectorConfig) connectorConfig).getSourceTimestampMode(), valueConverters, SqlServerConnectionUtils.class::getClassLoader, - connectorConfig.getSkippedOperations()); + connectorConfig.getSkippedOperations(), + false); } public static List listTables(JdbcConnection jdbc, RelationalTableFilters tableFilters) diff --git a/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/utils/SqlServerUtils.java b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/utils/SqlServerUtils.java index eaaac08e36..404336d7c9 100644 --- a/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/utils/SqlServerUtils.java +++ b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/utils/SqlServerUtils.java @@ -200,7 +200,7 @@ public static LsnOffset getLsnPosition(Map offset) { /** Fetch current largest log sequence number (LSN) of the database. */ public static LsnOffset currentLsn(SqlServerConnection connection) { try { - Lsn maxLsn = connection.getMaxLsn(); + Lsn maxLsn = connection.getMaxLsn(connection.database()); return new LsnOffset(maxLsn, maxLsn, null); } catch (SQLException e) { throw new FlinkRuntimeException(e.getMessage(), e); @@ -251,7 +251,7 @@ public static PreparedStatement readTableSplitDataStatement( } public static SqlServerDatabaseSchema createSqlServerDatabaseSchema( - SqlServerConnectorConfig connectorConfig) { + SqlServerConnectorConfig connectorConfig, SqlServerConnection connection) { TopicSelector topicSelector = SqlServerTopicSelector.defaultSelector(connectorConfig); SchemaNameAdjuster schemaNameAdjuster = SchemaNameAdjuster.create(); @@ -262,7 +262,11 @@ public static SqlServerDatabaseSchema createSqlServerDatabaseSchema( connectorConfig.binaryHandlingMode()); return new SqlServerDatabaseSchema( - connectorConfig, valueConverters, topicSelector, schemaNameAdjuster); + connectorConfig, + connection.getDefaultValueConverter(), + valueConverters, + topicSelector, + schemaNameAdjuster); } // --------------------------private method------------------------------- diff --git a/flink-connector-sqlserver-cdc/src/main/java/io/debezium/connector/sqlserver/SqlServerStreamingChangeEventSource.java b/flink-connector-sqlserver-cdc/src/main/java/io/debezium/connector/sqlserver/SqlServerStreamingChangeEventSource.java index d934979e72..9073b2844a 100644 --- a/flink-connector-sqlserver-cdc/src/main/java/io/debezium/connector/sqlserver/SqlServerStreamingChangeEventSource.java +++ b/flink-connector-sqlserver-cdc/src/main/java/io/debezium/connector/sqlserver/SqlServerStreamingChangeEventSource.java @@ -1,17 +1,7 @@ /* - * Copyright 2022 Ververica Inc. + * Copyright Debezium Authors. * - * Licensed 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. + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 */ package io.debezium.connector.sqlserver; @@ -20,6 +10,7 @@ import io.debezium.pipeline.ErrorHandler; import io.debezium.pipeline.EventDispatcher; import io.debezium.pipeline.source.spi.StreamingChangeEventSource; +import io.debezium.relational.ChangeTable; import io.debezium.relational.Table; import io.debezium.relational.TableId; import io.debezium.schema.SchemaChangeEvent.SchemaChangeEventType; @@ -34,11 +25,11 @@ import java.time.Instant; import java.util.ArrayList; import java.util.Arrays; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.PriorityQueue; import java.util.Queue; -import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; import java.util.regex.Matcher; @@ -46,10 +37,12 @@ import java.util.stream.Collectors; /** - * Copied from Debezium project(1.6.4.final) to add method {@link - * SqlServerStreamingChangeEventSource#afterHandleLsn(SqlServerOffsetContext)}. A {@link - * StreamingChangeEventSource} based on SQL Server change data capture functionality. A main loop - * polls database DDL change and change data tables and turns them into change events. + * Copied from Debezium project(1.9.7.final) to add method {@link + * SqlServerStreamingChangeEventSource#afterHandleLsn(SqlServerPartition, SqlServerOffsetContext)}. + * Also implemented {@link SqlServerStreamingChangeEventSource#execute( ChangeEventSourceContext, + * SqlServerPartition, SqlServerOffsetContext)}. A {@link StreamingChangeEventSource} based on SQL + * Server change data capture functionality. A main loop polls database DDL change and change data + * tables and turns them into change events. * *

The connector uses CDC functionality of SQL Server that is implemented as as a process that * monitors source table and write changes from the table into the change table. @@ -67,10 +60,10 @@ * old one. Then the change table is switched and streaming is executed from the new one. */ public class SqlServerStreamingChangeEventSource - implements StreamingChangeEventSource { + implements StreamingChangeEventSource { private static final Pattern MISSING_CDC_FUNCTION_CHANGES_ERROR = - Pattern.compile("Invalid object name 'cdc.fn_cdc_get_all_changes_(.*)'\\."); + Pattern.compile("Invalid object name '(.*)\\.cdc.fn_cdc_get_all_changes_(.*)'\\."); private static final Logger LOGGER = LoggerFactory.getLogger(SqlServerStreamingChangeEventSource.class); @@ -82,15 +75,15 @@ public class SqlServerStreamingChangeEventSource private final SqlServerConnection dataConnection; /** - * A separate connection for retrieving timestamps; without it, adaptive buffering will not - * work. + * A separate connection for retrieving details of the schema changes; without it, adaptive + * buffering will not work. * * @link * https://docs.microsoft.com/en-us/sql/connect/jdbc/using-adaptive-buffering?view=sql-server-2017#guidelines-for-using-adaptive-buffering */ private final SqlServerConnection metadataConnection; - private final EventDispatcher dispatcher; + private final EventDispatcher dispatcher; private final ErrorHandler errorHandler; private final Clock clock; private final SqlServerDatabaseSchema schema; @@ -98,12 +91,14 @@ public class SqlServerStreamingChangeEventSource private final SqlServerConnectorConfig connectorConfig; private final ElapsedTimeStrategy pauseBetweenCommits; + private final Map + streamingExecutionContexts; public SqlServerStreamingChangeEventSource( SqlServerConnectorConfig connectorConfig, SqlServerConnection dataConnection, SqlServerConnection metadataConnection, - EventDispatcher dispatcher, + EventDispatcher dispatcher, ErrorHandler errorHandler, Clock clock, SqlServerDatabaseSchema schema) { @@ -126,74 +121,121 @@ public SqlServerStreamingChangeEventSource( ? DEFAULT_INTERVAL_BETWEEN_COMMITS.toMillis() : intervalBetweenCommitsBasedOnPoll.toMillis()); this.pauseBetweenCommits.hasElapsed(); + this.streamingExecutionContexts = new HashMap<>(); + } + + @Override + public void execute( + ChangeEventSourceContext context, + SqlServerPartition partition, + SqlServerOffsetContext offsetContext) + throws InterruptedException { + final Metronome metronome = Metronome.sleeper(pollInterval, clock); + + LOGGER.info("Starting streaming"); + + while (context.isRunning()) { + boolean streamedEvents = executeIteration(context, partition, offsetContext); + + if (!streamedEvents) { + metronome.pause(); + } + } + + LOGGER.info("Finished streaming"); } @Override - public void execute(ChangeEventSourceContext context, SqlServerOffsetContext offsetContext) + public boolean executeIteration( + ChangeEventSourceContext context, + SqlServerPartition partition, + SqlServerOffsetContext offsetContext) throws InterruptedException { if (connectorConfig.getSnapshotMode().equals(SnapshotMode.INITIAL_ONLY)) { LOGGER.info("Streaming is not enabled in current configuration"); - return; + return false; } - final Metronome metronome = Metronome.sleeper(pollInterval, clock); - final Queue schemaChangeCheckpoints = - new PriorityQueue<>((x, y) -> x.getStopLsn().compareTo(y.getStopLsn())); + final String databaseName = partition.getDatabaseName(); + try { - final AtomicReference tablesSlot = - new AtomicReference(getCdcTablesToQuery(offsetContext)); + final SqlServerStreamingExecutionContext streamingExecutionContext = + streamingExecutionContexts.getOrDefault( + partition, + new SqlServerStreamingExecutionContext( + new PriorityQueue<>( + (x, y) -> x.getStopLsn().compareTo(y.getStopLsn())), + new AtomicReference<>(), + offsetContext.getChangePosition(), + new AtomicBoolean(false), + // LSN should be increased for the first run only immediately + // after snapshot completion + // otherwise we might skip an incomplete transaction after + // restart + offsetContext.isSnapshotCompleted())); + + if (!streamingExecutionContexts.containsKey(partition)) { + streamingExecutionContexts.put(partition, streamingExecutionContext); + LOGGER.info( + "Last position recorded in offsets is {}[{}]", + offsetContext.getChangePosition(), + offsetContext.getEventSerialNo()); + } + final Queue schemaChangeCheckpoints = + streamingExecutionContext.getSchemaChangeCheckpoints(); + final AtomicReference tablesSlot = + streamingExecutionContext.getTablesSlot(); final TxLogPosition lastProcessedPositionOnStart = offsetContext.getChangePosition(); final long lastProcessedEventSerialNoOnStart = offsetContext.getEventSerialNo(); - LOGGER.info( - "Last position recorded in offsets is {}[{}]", - lastProcessedPositionOnStart, - lastProcessedEventSerialNoOnStart); - final AtomicBoolean changesStoppedBeingMonotonic = new AtomicBoolean(false); + final AtomicBoolean changesStoppedBeingMonotonic = + streamingExecutionContext.getChangesStoppedBeingMonotonic(); final int maxTransactionsPerIteration = connectorConfig.getMaxTransactionsPerIteration(); - TxLogPosition lastProcessedPosition = lastProcessedPositionOnStart; + TxLogPosition lastProcessedPosition = + streamingExecutionContext.getLastProcessedPosition(); - // LSN should be increased for the first run only immediately after snapshot completion - // otherwise we might skip an incomplete transaction after restart - boolean shouldIncreaseFromLsn = offsetContext.isSnapshotCompleted(); - while (context.isRunning()) { + if (context.isRunning()) { commitTransaction(); - afterHandleLsn(offsetContext); + afterHandleLsn(partition, offsetContext); final Lsn toLsn = getToLsn( - dataConnection, lastProcessedPosition, maxTransactionsPerIteration); + dataConnection, + databaseName, + lastProcessedPosition, + maxTransactionsPerIteration); // Shouldn't happen if the agent is running, but it is better to guard against such // situation if (!toLsn.isAvailable()) { LOGGER.warn( "No maximum LSN recorded in the database; please ensure that the SQL Server Agent is running"); - metronome.pause(); - continue; + return false; } // There is no change in the database if (toLsn.compareTo(lastProcessedPosition.getCommitLsn()) <= 0 - && shouldIncreaseFromLsn) { + && streamingExecutionContext.getShouldIncreaseFromLsn()) { LOGGER.debug("No change in the database"); - metronome.pause(); - continue; + return false; } // Reading interval is inclusive so we need to move LSN forward but not for first // run as TX might not be streamed completely final Lsn fromLsn = - lastProcessedPosition.getCommitLsn().isAvailable() && shouldIncreaseFromLsn - ? dataConnection.incrementLsn(lastProcessedPosition.getCommitLsn()) + lastProcessedPosition.getCommitLsn().isAvailable() + && streamingExecutionContext.getShouldIncreaseFromLsn() + ? dataConnection.incrementLsn( + databaseName, lastProcessedPosition.getCommitLsn()) : lastProcessedPosition.getCommitLsn(); - shouldIncreaseFromLsn = true; + streamingExecutionContext.setShouldIncreaseFromLsn(true); while (!schemaChangeCheckpoints.isEmpty()) { - migrateTable(schemaChangeCheckpoints, offsetContext); + migrateTable(partition, schemaChangeCheckpoints, offsetContext); } - if (!dataConnection.listOfNewChangeTables(fromLsn, toLsn).isEmpty()) { - final SqlServerChangeTable[] tables = getCdcTablesToQuery(offsetContext); + if (!dataConnection.getNewChangeTables(databaseName, fromLsn, toLsn).isEmpty()) { + final SqlServerChangeTable[] tables = + getChangeTablesToQuery(partition, offsetContext, toLsn); tablesSlot.set(tables); for (SqlServerChangeTable table : tables) { if (table.getStartLsn().isBetween(fromLsn, toLsn)) { @@ -202,8 +244,12 @@ public void execute(ChangeEventSourceContext context, SqlServerOffsetContext off } } } + if (tablesSlot.get() == null) { + tablesSlot.set(getChangeTablesToQuery(partition, offsetContext, toLsn)); + } try { dataConnection.getChangesForTables( + databaseName, tablesSlot.get(), fromLsn, toLsn, @@ -333,7 +379,10 @@ public void execute(ChangeEventSourceContext context, SqlServerOffsetContext off .peek() .getStartLsn()) >= 0) { - migrateTable(schemaChangeCheckpoints, offsetContext); + migrateTable( + partition, + schemaChangeCheckpoints, + offsetContext); } } final TableId tableId = @@ -380,13 +429,14 @@ public void execute(ChangeEventSourceContext context, SqlServerOffsetContext off connectorConfig .getSourceTimestampMode() .getTimestamp( - metadataConnection, clock, tableWithSmallestLsn.getResultSet())); dispatcher.dispatchDataChangeEvent( + partition, tableId, new SqlServerChangeRecordEmitter( + partition, offsetContext, operation, data, @@ -395,16 +445,20 @@ public void execute(ChangeEventSourceContext context, SqlServerOffsetContext off tableWithSmallestLsn.next(); } }); - lastProcessedPosition = TxLogPosition.valueOf(toLsn); + streamingExecutionContext.setLastProcessedPosition( + TxLogPosition.valueOf(toLsn)); // Terminate the transaction otherwise CDC could not be disabled for tables dataConnection.rollback(); } catch (SQLException e) { - tablesSlot.set(processErrorFromChangeTableQuery(e, tablesSlot.get())); + tablesSlot.set( + processErrorFromChangeTableQuery(databaseName, e, tablesSlot.get())); } } } catch (Exception e) { errorHandler.setProducerThrowable(e); } + + return true; } private void commitTransaction() throws SQLException { @@ -415,47 +469,63 @@ private void commitTransaction() throws SQLException { // TempDB if (connectorConfig.isReadOnlyDatabaseConnection() || pauseBetweenCommits.hasElapsed()) { dataConnection.commit(); + metadataConnection.commit(); } } private void migrateTable( + SqlServerPartition partition, final Queue schemaChangeCheckpoints, SqlServerOffsetContext offsetContext) throws InterruptedException, SQLException { final SqlServerChangeTable newTable = schemaChangeCheckpoints.poll(); LOGGER.info("Migrating schema to {}", newTable); - Table tableSchema = metadataConnection.getTableSchemaFromTable(newTable); + Table oldTableSchema = schema.tableFor(newTable.getSourceTableId()); + Table tableSchema = + metadataConnection.getTableSchemaFromTable(partition.getDatabaseName(), newTable); + if (oldTableSchema.equals(tableSchema)) { + LOGGER.info("Migration skipped, no table schema changes detected."); + return; + } dispatcher.dispatchSchemaChangeEvent( + partition, newTable.getSourceTableId(), new SqlServerSchemaChangeEventEmitter( - offsetContext, newTable, tableSchema, SchemaChangeEventType.ALTER)); + partition, + offsetContext, + newTable, + tableSchema, + SchemaChangeEventType.ALTER)); newTable.setSourceTable(tableSchema); } private SqlServerChangeTable[] processErrorFromChangeTableQuery( - SQLException exception, SqlServerChangeTable[] currentChangeTables) throws Exception { + String databaseName, SQLException exception, SqlServerChangeTable[] currentChangeTables) + throws Exception { final Matcher m = MISSING_CDC_FUNCTION_CHANGES_ERROR.matcher(exception.getMessage()); - if (m.matches()) { - final String captureName = m.group(1); + if (m.matches() && m.group(1).equals(databaseName)) { + final String captureName = m.group(2); LOGGER.info("Table is no longer captured with capture instance {}", captureName); - return Arrays.asList(currentChangeTables).stream() + return Arrays.stream(currentChangeTables) .filter(x -> !x.getCaptureInstance().equals(captureName)) - .collect(Collectors.toList()) - .toArray(new SqlServerChangeTable[0]); + .toArray(SqlServerChangeTable[]::new); } throw exception; } - private SqlServerChangeTable[] getCdcTablesToQuery(SqlServerOffsetContext offsetContext) + private SqlServerChangeTable[] getChangeTablesToQuery( + SqlServerPartition partition, SqlServerOffsetContext offsetContext, Lsn toLsn) throws SQLException, InterruptedException { - final Set cdcEnabledTables = dataConnection.listOfChangeTables(); - if (cdcEnabledTables.isEmpty()) { + final String databaseName = partition.getDatabaseName(); + final List changeTables = + dataConnection.getChangeTables(databaseName, toLsn); + if (changeTables.isEmpty()) { LOGGER.warn( "No table has enabled CDC or security constraints prevents getting the list of change tables"); } - final Map> includeListCdcEnabledTables = - cdcEnabledTables.stream() + final Map> includeListChangeTables = + changeTables.stream() .filter( changeTable -> { if (connectorConfig @@ -470,15 +540,15 @@ private SqlServerChangeTable[] getCdcTablesToQuery(SqlServerOffsetContext offset return false; } }) - .collect(Collectors.groupingBy(x -> x.getSourceTableId())); + .collect(Collectors.groupingBy(ChangeTable::getSourceTableId)); - if (includeListCdcEnabledTables.isEmpty()) { + if (includeListChangeTables.isEmpty()) { LOGGER.warn( "No whitelisted table has enabled CDC, whitelisted table list does not contain any table with CDC enabled or no table match the white/blacklist filter(s)"); } final List tables = new ArrayList<>(); - for (List captures : includeListCdcEnabledTables.values()) { + for (List captures : includeListChangeTables.values()) { SqlServerChangeTable currentTable = captures.get(0); if (captures.size() > 1) { SqlServerChangeTable futureTable; @@ -489,7 +559,8 @@ private SqlServerChangeTable[] getCdcTablesToQuery(SqlServerOffsetContext offset futureTable = captures.get(0); } currentTable.setStopLsn(futureTable.getStartLsn()); - futureTable.setSourceTable(dataConnection.getTableSchemaFromTable(futureTable)); + futureTable.setSourceTable( + dataConnection.getTableSchemaFromTable(databaseName, futureTable)); tables.add(futureTable); LOGGER.info( "Multiple capture instances present for the same table: {} and {}", @@ -507,11 +578,13 @@ private SqlServerChangeTable[] getCdcTablesToQuery(SqlServerOffsetContext offset // timestamp is used offsetContext.event(currentTable.getSourceTableId(), Instant.now()); dispatcher.dispatchSchemaChangeEvent( + partition, currentTable.getSourceTableId(), new SqlServerSchemaChangeEventEmitter( + partition, offsetContext, currentTable, - dataConnection.getTableSchemaFromTable(currentTable), + dataConnection.getTableSchemaFromTable(databaseName, currentTable), SchemaChangeEventType.CREATE)); } @@ -531,25 +604,29 @@ private SqlServerChangeTable[] getCdcTablesToQuery(SqlServerOffsetContext offset */ private Lsn getToLsn( SqlServerConnection connection, + String databaseName, TxLogPosition lastProcessedPosition, int maxTransactionsPerIteration) throws SQLException { if (maxTransactionsPerIteration == 0) { - return connection.getMaxTransactionLsn(); + return connection.getMaxTransactionLsn(databaseName); } final Lsn fromLsn = lastProcessedPosition.getCommitLsn(); if (!fromLsn.isAvailable()) { - return connection.getNthTransactionLsnFromBeginning(maxTransactionsPerIteration); + return connection.getNthTransactionLsnFromBeginning( + databaseName, maxTransactionsPerIteration); } - return connection.getNthTransactionLsnFromLast(fromLsn, maxTransactionsPerIteration); + return connection.getNthTransactionLsnFromLast( + databaseName, fromLsn, maxTransactionsPerIteration); } /** expose control to the user to stop the connector. */ - protected void afterHandleLsn(SqlServerOffsetContext offsetContext) { + protected void afterHandleLsn( + SqlServerPartition partition, SqlServerOffsetContext offsetContext) { // do nothing } } diff --git a/flink-connector-sqlserver-cdc/src/test/java/com/ververica/cdc/connectors/sqlserver/source/read/fetch/SqlServerScanFetchTaskTest.java b/flink-connector-sqlserver-cdc/src/test/java/com/ververica/cdc/connectors/sqlserver/source/read/fetch/SqlServerScanFetchTaskTest.java index e5f6cbec63..5dac1c68b1 100644 --- a/flink-connector-sqlserver-cdc/src/test/java/com/ververica/cdc/connectors/sqlserver/source/read/fetch/SqlServerScanFetchTaskTest.java +++ b/flink-connector-sqlserver-cdc/src/test/java/com/ververica/cdc/connectors/sqlserver/source/read/fetch/SqlServerScanFetchTaskTest.java @@ -33,6 +33,7 @@ import com.ververica.cdc.connectors.sqlserver.source.reader.fetch.SqlServerSourceFetchTaskContext; import com.ververica.cdc.connectors.sqlserver.testutils.RecordsFormatter; import io.debezium.connector.sqlserver.SqlServerConnection; +import io.debezium.connector.sqlserver.SqlServerPartition; import io.debezium.data.Envelope; import io.debezium.jdbc.JdbcConnection; import io.debezium.pipeline.EventDispatcher; @@ -88,10 +89,8 @@ public void testChangingDataInSnapshotScan() throws Exception { new MakeChangeEventTaskContext( sourceConfig, sqlServerDialect, - createSqlServerConnection( - sourceConfig.getDbzConnectorConfig().getJdbcConfig()), - createSqlServerConnection( - sourceConfig.getDbzConnectorConfig().getJdbcConfig()), + createSqlServerConnection(sourceConfig.getDbzConnectorConfig()), + createSqlServerConnection(sourceConfig.getDbzConnectorConfig()), () -> executeSql(sourceConfig, changingDataSql)); final DataType dataType = @@ -143,10 +142,8 @@ public void testInsertDataInSnapshotScan() throws Exception { new MakeChangeEventTaskContext( sourceConfig, sqlServerDialect, - createSqlServerConnection( - sourceConfig.getDbzConnectorConfig().getJdbcConfig()), - createSqlServerConnection( - sourceConfig.getDbzConnectorConfig().getJdbcConfig()), + createSqlServerConnection(sourceConfig.getDbzConnectorConfig()), + createSqlServerConnection(sourceConfig.getDbzConnectorConfig()), () -> executeSql(sourceConfig, insertDataSql)); final DataType dataType = @@ -200,10 +197,8 @@ public void testDeleteDataInSnapshotScan() throws Exception { new MakeChangeEventTaskContext( sourceConfig, sqlServerDialect, - createSqlServerConnection( - sourceConfig.getDbzConnectorConfig().getJdbcConfig()), - createSqlServerConnection( - sourceConfig.getDbzConnectorConfig().getJdbcConfig()), + createSqlServerConnection(sourceConfig.getDbzConnectorConfig()), + createSqlServerConnection(sourceConfig.getDbzConnectorConfig()), () -> executeSql(sourceConfig, deleteDataSql)); final DataType dataType = @@ -299,9 +294,8 @@ public static SqlServerSourceConfigFactory getConfigFactory( } private boolean executeSql(SqlServerSourceConfig sourceConfig, String[] sqlStatements) { - JdbcConnection connection = - createSqlServerConnection(sourceConfig.getDbzConnectorConfig().getJdbcConfig()); - try { + try (JdbcConnection connection = + createSqlServerConnection(sourceConfig.getDbzConnectorConfig())) { connection.setAutoCommit(false); connection.execute(sqlStatements); connection.commit(); @@ -312,9 +306,9 @@ private boolean executeSql(SqlServerSourceConfig sourceConfig, String[] sqlState return true; } - class MakeChangeEventTaskContext extends SqlServerSourceFetchTaskContext { + static class MakeChangeEventTaskContext extends SqlServerSourceFetchTaskContext { - private Supplier makeChangeEventFunction; + private final Supplier makeChangeEventFunction; public MakeChangeEventTaskContext( JdbcSourceConfig jdbcSourceConfig, @@ -327,12 +321,14 @@ public MakeChangeEventTaskContext( } @Override - public EventDispatcher.SnapshotReceiver getSnapshotReceiver() { - EventDispatcher.SnapshotReceiver snapshotReceiver = super.getSnapshotReceiver(); - return new EventDispatcher.SnapshotReceiver() { + public EventDispatcher.SnapshotReceiver getSnapshotReceiver() { + EventDispatcher.SnapshotReceiver snapshotReceiver = + super.getSnapshotReceiver(); + return new EventDispatcher.SnapshotReceiver() { @Override public void changeRecord( + SqlServerPartition partition, DataCollectionSchema schema, Envelope.Operation operation, Object key, @@ -340,7 +336,8 @@ public void changeRecord( OffsetContext offset, ConnectHeaders headers) throws InterruptedException { - snapshotReceiver.changeRecord(schema, operation, key, value, offset, headers); + snapshotReceiver.changeRecord( + partition, schema, operation, key, value, offset, headers); } @Override diff --git a/flink-connector-sqlserver-cdc/src/test/java/com/ververica/cdc/connectors/sqlserver/table/SqlServerConnectorITCase.java b/flink-connector-sqlserver-cdc/src/test/java/com/ververica/cdc/connectors/sqlserver/table/SqlServerConnectorITCase.java index 3680066abc..b6e19ad011 100644 --- a/flink-connector-sqlserver-cdc/src/test/java/com/ververica/cdc/connectors/sqlserver/table/SqlServerConnectorITCase.java +++ b/flink-connector-sqlserver-cdc/src/test/java/com/ververica/cdc/connectors/sqlserver/table/SqlServerConnectorITCase.java @@ -65,7 +65,7 @@ public SqlServerConnectorITCase(boolean parallelismSnapshot) { @Parameterized.Parameters(name = "parallelismSnapshot: {0}") public static Object[] parameters() { - return new Object[][] {new Object[] {false}, new Object[] {true}}; + return new Object[] {false, true}; } @Before diff --git a/pom.xml b/pom.xml index 0701fa807f..f57c86cad3 100644 --- a/pom.xml +++ b/pom.xml @@ -51,7 +51,7 @@ under the License. 1.17.0 - 1.6.4.Final + 1.9.7.Final 3.2.0 2.2.0