DBZ-4027 Make signalling channel configurable

DBZ-4027 Add an Incremental snapshot test with kafka signaling

DBZ-4027 Add an Incremental snapshot test with kafka signaling

DBZ-4027 Add an Incremental snapshot test with kafka signaling

DBZ-4027 Code style

DBZ-4027 Make SignalPayload more generic and extensible

DBZ-4027 Rename DatabaseSignalChannel to SourceSignalChannel

DBZ-4027 Improve logging

DBZ-4027 Moved SPI file definition to debezium-core

DBZ-4027 Move SignalProcessor synchronization point to be processed only when a signal cdc event arrives.

DBZ-4027 Add EventDispatcher constructor without signalProcessor for spanner connector

DBZ-4027 Fix NPE

DBZ-4027 Fix NPE

DBZ-4027 Formatting

DBZ-4027 Correctly manage signal on not supported connector

DBZ-4027 Use the correct MongoDbOffset

DBZ-4027 Correctly initialize offset for Oracle and SqlServer connectors

DBZ-4027 Register SPI implementations

DBZ-4027 Improve SignalProcessor instantiation

DBZ-4027 Pass source info in case of SchemaChanges action

DBZ-4027 Manage close event in a synchronous way

DBZ-4027 Correctly init offset context also in case of snapshot mode 'never'

DBZ-4027 Fix MySqlMetricsIT test

DBZ-4027 Move KafkaSignalChannel to core

DBZ-4027 Move KafkaSignalChannel to core

DBZ-4027 Set pass offset context after initial snapshot to SignalProcessor

DBZ-4027 Pass OffsetContext to signal processor

DBZ-4027 Pass CommonConnectorConfig to SignalChannelReader init method

DBZ-4027 Move Incremental snapshot window actions to dedicated package

DBZ-4027 Align SignalsIT test with new code

DBZ-4027 Fix SignalsIT test

DBZ-4027 Fix SignalProcessor scheduling

DBZ-4027 Moved DatabaseSignalChannel and SignalChannelReader to dedicated package

DBZ-4027 Start SignalProcessor from ChangeEventSourceCoordinator

DBZ-4027 Create SignalProcessor and renamed Signal to DatabaseSignalChannel

DBZ-4027 Initial refactoring of signal feature
This commit is contained in:
mfvitale 2023-03-31 12:28:39 +02:00 committed by Jiri Pechanec
parent 58ef4f0b98
commit 952340286e
66 changed files with 1708 additions and 868 deletions

View File

@ -91,9 +91,13 @@ public StreamingChangeEventSource<MongoDbPartition, MongoDbOffsetContext> getStr
MongoDbOffsetContext offsetContext, MongoDbOffsetContext offsetContext,
SnapshotProgressListener<MongoDbPartition> snapshotProgressListener, SnapshotProgressListener<MongoDbPartition> snapshotProgressListener,
DataChangeEventListener<MongoDbPartition> dataChangeEventListener) { DataChangeEventListener<MongoDbPartition> dataChangeEventListener) {
if (replicaSets.size() > 1) {
LOGGER.info("Only ReplicaSet deployments and Sharded Cluster with connection.mode=sharded are supported by incremental snapshot");
return Optional.empty();
}
final MongoDbIncrementalSnapshotChangeEventSource incrementalSnapshotChangeEventSource = new MongoDbIncrementalSnapshotChangeEventSource( final MongoDbIncrementalSnapshotChangeEventSource incrementalSnapshotChangeEventSource = new MongoDbIncrementalSnapshotChangeEventSource(
configuration, configuration,
taskContext,
connections, connections,
replicaSets, replicaSets,
dispatcher, dispatcher,

View File

@ -24,10 +24,12 @@
import io.debezium.connector.base.ChangeEventQueue; import io.debezium.connector.base.ChangeEventQueue;
import io.debezium.connector.common.BaseSourceTask; import io.debezium.connector.common.BaseSourceTask;
import io.debezium.connector.mongodb.metrics.MongoDbChangeEventSourceMetricsFactory; import io.debezium.connector.mongodb.metrics.MongoDbChangeEventSourceMetricsFactory;
import io.debezium.document.DocumentReader;
import io.debezium.pipeline.ChangeEventSourceCoordinator; import io.debezium.pipeline.ChangeEventSourceCoordinator;
import io.debezium.pipeline.DataChangeEvent; import io.debezium.pipeline.DataChangeEvent;
import io.debezium.pipeline.ErrorHandler; import io.debezium.pipeline.ErrorHandler;
import io.debezium.pipeline.EventDispatcher; import io.debezium.pipeline.EventDispatcher;
import io.debezium.pipeline.signal.SignalProcessor;
import io.debezium.pipeline.spi.Offsets; import io.debezium.pipeline.spi.Offsets;
import io.debezium.schema.SchemaNameAdjuster; import io.debezium.schema.SchemaNameAdjuster;
import io.debezium.util.Clock; import io.debezium.util.Clock;
@ -97,6 +99,12 @@ public ChangeEventSourceCoordinator<MongoDbPartition, MongoDbOffsetContext> star
final MongoDbEventMetadataProvider metadataProvider = new MongoDbEventMetadataProvider(); final MongoDbEventMetadataProvider metadataProvider = new MongoDbEventMetadataProvider();
SignalProcessor<MongoDbPartition, MongoDbOffsetContext> signalProcessor = new SignalProcessor<>(
MongoDbConnector.class, connectorConfig, Map.of(),
getAvailableSignalChannels(),
DocumentReader.defaultReader(),
Offsets.of(Collections.singletonMap(new MongoDbPartition(), previousOffset)));
final EventDispatcher<MongoDbPartition, CollectionId> dispatcher = new EventDispatcher<>( final EventDispatcher<MongoDbPartition, CollectionId> dispatcher = new EventDispatcher<>(
connectorConfig, connectorConfig,
taskContext.topicNamingStrategy(), taskContext.topicNamingStrategy(),
@ -105,7 +113,10 @@ public ChangeEventSourceCoordinator<MongoDbPartition, MongoDbOffsetContext> star
taskContext.filters().collectionFilter()::test, taskContext.filters().collectionFilter()::test,
DataChangeEvent::new, DataChangeEvent::new,
metadataProvider, metadataProvider,
schemaNameAdjuster); schemaNameAdjuster,
signalProcessor);
dispatcher.getSignalingActions().forEach(signalProcessor::registerSignalAction);
ChangeEventSourceCoordinator<MongoDbPartition, MongoDbOffsetContext> coordinator = new ChangeEventSourceCoordinator<>( ChangeEventSourceCoordinator<MongoDbPartition, MongoDbOffsetContext> coordinator = new ChangeEventSourceCoordinator<>(
// TODO pass offsets from all the partitions // TODO pass offsets from all the partitions
@ -123,7 +134,8 @@ public ChangeEventSourceCoordinator<MongoDbPartition, MongoDbOffsetContext> star
schema), schema),
new MongoDbChangeEventSourceMetricsFactory(), new MongoDbChangeEventSourceMetricsFactory(),
dispatcher, dispatcher,
schema); schema,
signalProcessor);
coordinator.start(taskContext, this.queue, metadataProvider); coordinator.start(taskContext, this.queue, metadataProvider);

View File

@ -24,16 +24,16 @@
import io.debezium.DebeziumException; import io.debezium.DebeziumException;
import io.debezium.annotation.NotThreadSafe; import io.debezium.annotation.NotThreadSafe;
import io.debezium.connector.mongodb.connection.ConnectionContext;
import io.debezium.connector.mongodb.connection.MongoDbConnection; import io.debezium.connector.mongodb.connection.MongoDbConnection;
import io.debezium.connector.mongodb.connection.ReplicaSet;
import io.debezium.connector.mongodb.recordemitter.MongoDbSnapshotRecordEmitter; import io.debezium.connector.mongodb.recordemitter.MongoDbSnapshotRecordEmitter;
import io.debezium.pipeline.EventDispatcher; import io.debezium.pipeline.EventDispatcher;
import io.debezium.pipeline.signal.actions.snapshotting.CloseIncrementalSnapshotWindow;
import io.debezium.pipeline.signal.actions.snapshotting.OpenIncrementalSnapshotWindow;
import io.debezium.pipeline.source.AbstractSnapshotChangeEventSource; import io.debezium.pipeline.source.AbstractSnapshotChangeEventSource;
import io.debezium.pipeline.source.snapshot.incremental.CloseIncrementalSnapshotWindow;
import io.debezium.pipeline.source.snapshot.incremental.DataCollection; import io.debezium.pipeline.source.snapshot.incremental.DataCollection;
import io.debezium.pipeline.source.snapshot.incremental.IncrementalSnapshotChangeEventSource; import io.debezium.pipeline.source.snapshot.incremental.IncrementalSnapshotChangeEventSource;
import io.debezium.pipeline.source.snapshot.incremental.IncrementalSnapshotContext; import io.debezium.pipeline.source.snapshot.incremental.IncrementalSnapshotContext;
import io.debezium.pipeline.source.snapshot.incremental.OpenIncrementalSnapshotWindow;
import io.debezium.pipeline.source.spi.DataChangeEventListener; import io.debezium.pipeline.source.spi.DataChangeEventListener;
import io.debezium.pipeline.source.spi.SnapshotProgressListener; import io.debezium.pipeline.source.spi.SnapshotProgressListener;
import io.debezium.pipeline.spi.ChangeRecordEmitter; import io.debezium.pipeline.spi.ChangeRecordEmitter;
@ -61,8 +61,6 @@ public class MongoDbIncrementalSnapshotChangeEventSource
private final DataChangeEventListener<MongoDbPartition> dataListener; private final DataChangeEventListener<MongoDbPartition> dataListener;
private long totalRowsScanned = 0; private long totalRowsScanned = 0;
private final ReplicaSets replicaSets; private final ReplicaSets replicaSets;
private final ConnectionContext connectionContext;
private final MongoDbTaskContext taskContext;
private final MongoDbConnection.ChangeEventSourceConnectionFactory connections; private final MongoDbConnection.ChangeEventSourceConnectionFactory connections;
private MongoDbCollectionSchema currentCollection; private MongoDbCollectionSchema currentCollection;
@ -72,10 +70,9 @@ public class MongoDbIncrementalSnapshotChangeEventSource
protected final Map<Struct, Object[]> window = new LinkedHashMap<>(); protected final Map<Struct, Object[]> window = new LinkedHashMap<>();
private MongoDbConnection mongo; private MongoDbConnection mongo;
private CollectionId signallingCollectionId; private final CollectionId signallingCollectionId;
public MongoDbIncrementalSnapshotChangeEventSource(MongoDbConnectorConfig config, public MongoDbIncrementalSnapshotChangeEventSource(MongoDbConnectorConfig config,
MongoDbTaskContext taskContext,
MongoDbConnection.ChangeEventSourceConnectionFactory connections, ReplicaSets replicaSets, MongoDbConnection.ChangeEventSourceConnectionFactory connections, ReplicaSets replicaSets,
EventDispatcher<MongoDbPartition, CollectionId> dispatcher, EventDispatcher<MongoDbPartition, CollectionId> dispatcher,
MongoDbSchema collectionSchema, MongoDbSchema collectionSchema,
@ -84,9 +81,7 @@ public MongoDbIncrementalSnapshotChangeEventSource(MongoDbConnectorConfig config
DataChangeEventListener<MongoDbPartition> dataChangeEventListener) { DataChangeEventListener<MongoDbPartition> dataChangeEventListener) {
this.connectorConfig = config; this.connectorConfig = config;
this.replicaSets = replicaSets; this.replicaSets = replicaSets;
this.taskContext = taskContext;
this.connections = connections; this.connections = connections;
this.connectionContext = taskContext.getConnectionContext();
this.dispatcher = dispatcher; this.dispatcher = dispatcher;
this.collectionSchema = collectionSchema; this.collectionSchema = collectionSchema;
this.clock = clock; this.clock = clock;
@ -103,7 +98,7 @@ public void closeWindow(MongoDbPartition partition, String id, OffsetContext off
if (!context.closeWindow(id)) { if (!context.closeWindow(id)) {
return; return;
} }
sendWindowEvents(partition, offsetContext); sendWindowEvents(offsetContext);
readChunk(partition); readChunk(partition);
} }
@ -132,26 +127,38 @@ protected String getSignalCollectionName(String dataCollectionId) {
return dataCollectionId; return dataCollectionId;
} }
protected void sendWindowEvents(MongoDbPartition partition, OffsetContext offsetContext) throws InterruptedException { protected void sendWindowEvents(OffsetContext offsetContext) throws InterruptedException {
LOGGER.debug("Sending {} events from window buffer", window.size()); LOGGER.debug("Sending {} events from window buffer", window.size());
offsetContext.incrementalSnapshotEvents(); offsetContext.incrementalSnapshotEvents();
for (Object[] row : window.values()) { for (Object[] row : window.values()) {
sendEvent(partition, dispatcher, offsetContext, row); sendEvent(dispatcher, offsetContext, row);
} }
offsetContext.postSnapshotCompletion(); offsetContext.postSnapshotCompletion();
window.clear(); window.clear();
} }
// TODO Used typed dispatcher and offset context // TODO Used typed dispatcher and offset context
protected void sendEvent(MongoDbPartition partition, EventDispatcher<MongoDbPartition, CollectionId> dispatcher, OffsetContext offsetContext, Object[] row) protected void sendEvent(EventDispatcher<MongoDbPartition, CollectionId> dispatcher, OffsetContext offsetContext, Object[] row)
throws InterruptedException { throws InterruptedException {
context.sendEvent(keyFromRow(row)); context.sendEvent(keyFromRow(row));
((ReplicaSetOffsetContext) offsetContext).readEvent(context.currentDataCollectionId().getId(), clock.currentTimeAsInstant());
dispatcher.dispatchSnapshotEvent(partition, context.currentDataCollectionId().getId(), MongoDbOffsetContext mongoDbOffsetContext = getMongoDbOffsetContext(offsetContext);
getChangeRecordEmitter(partition, offsetContext, row), ReplicaSet replicaSet = replicaSets.getIncrementalSnapshotReplicaSet();
ReplicaSetOffsetContext replicaSetOffsetContext = mongoDbOffsetContext.getReplicaSetOffsetContext(replicaSet);
ReplicaSetPartition replicaSetPartition = mongoDbOffsetContext.getReplicaSetPartition(replicaSet);
replicaSetOffsetContext.readEvent(context.currentDataCollectionId().getId(), clock.currentTimeAsInstant());
dispatcher.dispatchSnapshotEvent(
replicaSetPartition, context.currentDataCollectionId().getId(),
getChangeRecordEmitter(
replicaSetPartition, replicaSetOffsetContext, row),
dispatcher.getIncrementalSnapshotChangeEventReceiver(dataListener)); dispatcher.getIncrementalSnapshotChangeEventReceiver(dataListener));
} }
private static MongoDbOffsetContext getMongoDbOffsetContext(OffsetContext offsetContext) {
return (MongoDbOffsetContext) offsetContext;
}
/** /**
* Returns a {@link ChangeRecordEmitter} producing the change records for * Returns a {@link ChangeRecordEmitter} producing the change records for
* the given table row. * the given table row.
@ -336,8 +343,9 @@ private Object[] readMaximumKey() throws InterruptedException {
@Override @Override
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
public void addDataCollectionNamesToSnapshot(MongoDbPartition partition, List<String> dataCollectionIds, public void addDataCollectionNamesToSnapshot(MongoDbPartition partition, OffsetContext offsetContext, Map<String, Object> additionalData,
Optional<String> additionalCondition, Optional<String> surrogateKey, OffsetContext offsetContext) List<String> dataCollectionIds,
Optional<String> additionalCondition, Optional<String> surrogateKey)
throws InterruptedException { throws InterruptedException {
if (additionalCondition != null && additionalCondition.isPresent()) { if (additionalCondition != null && additionalCondition.isPresent()) {
throw new UnsupportedOperationException("Additional condition not supported for MongoDB"); throw new UnsupportedOperationException("Additional condition not supported for MongoDB");
@ -365,7 +373,7 @@ public void addDataCollectionNamesToSnapshot(MongoDbPartition partition, List<St
@Override @Override
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
public void stopSnapshot(MongoDbPartition partition, List<String> dataCollectionIds, OffsetContext offsetContext) { public void stopSnapshot(MongoDbPartition partition, OffsetContext offsetContext, Map<String, Object> additionalData, List<String> dataCollectionIds) {
context = (IncrementalSnapshotContext<CollectionId>) offsetContext.getIncrementalSnapshotContext(); context = (IncrementalSnapshotContext<CollectionId>) offsetContext.getIncrementalSnapshotContext();
if (context.snapshotRunning()) { if (context.snapshotRunning()) {
if (dataCollectionIds == null || dataCollectionIds.isEmpty()) { if (dataCollectionIds == null || dataCollectionIds.isEmpty()) {

View File

@ -50,6 +50,7 @@ public class MongoDbStreamingChangeEventSource implements StreamingChangeEventSo
private final ReplicaSets replicaSets; private final ReplicaSets replicaSets;
private final MongoDbTaskContext taskContext; private final MongoDbTaskContext taskContext;
private final MongoDbConnection.ChangeEventSourceConnectionFactory connections; private final MongoDbConnection.ChangeEventSourceConnectionFactory connections;
private MongoDbOffsetContext effectiveOffset;
public MongoDbStreamingChangeEventSource(MongoDbConnectorConfig connectorConfig, MongoDbTaskContext taskContext, public MongoDbStreamingChangeEventSource(MongoDbConnectorConfig connectorConfig, MongoDbTaskContext taskContext,
MongoDbConnection.ChangeEventSourceConnectionFactory connections, ReplicaSets replicaSets, MongoDbConnection.ChangeEventSourceConnectionFactory connections, ReplicaSets replicaSets,
@ -65,15 +66,17 @@ public MongoDbStreamingChangeEventSource(MongoDbConnectorConfig connectorConfig,
this.connections = connections; this.connections = connections;
} }
@Override
public void init(MongoDbOffsetContext offsetContext) {
this.effectiveOffset = offsetContext == null ? emptyOffsets(connectorConfig) : offsetContext;
}
@Override @Override
public void execute(ChangeEventSourceContext context, MongoDbPartition partition, MongoDbOffsetContext offsetContext) public void execute(ChangeEventSourceContext context, MongoDbPartition partition, MongoDbOffsetContext offsetContext)
throws InterruptedException { throws InterruptedException {
final List<ReplicaSet> validReplicaSets = replicaSets.all(); final List<ReplicaSet> validReplicaSets = replicaSets.all();
if (offsetContext == null) {
offsetContext = emptyOffsets(connectorConfig);
}
if (validReplicaSets.size() == 1) { if (validReplicaSets.size() == 1) {
// Streams the replica-set changes in the current thread // Streams the replica-set changes in the current thread
streamChangesForReplicaSet(context, partition, validReplicaSets.get(0), offsetContext); streamChangesForReplicaSet(context, partition, validReplicaSets.get(0), offsetContext);
@ -84,6 +87,11 @@ else if (validReplicaSets.size() > 1) {
} }
} }
@Override
public MongoDbOffsetContext getOffsetContext() {
return effectiveOffset;
}
private void streamChangesForReplicaSet(ChangeEventSourceContext context, MongoDbPartition partition, private void streamChangesForReplicaSet(ChangeEventSourceContext context, MongoDbPartition partition,
ReplicaSet replicaSet, MongoDbOffsetContext offsetContext) { ReplicaSet replicaSet, MongoDbOffsetContext offsetContext) {
try (MongoDbConnection mongo = connections.get(replicaSet, partition)) { try (MongoDbConnection mongo = connections.get(replicaSet, partition)) {
@ -129,8 +137,8 @@ private void streamChangesForReplicaSets(ChangeEventSourceContext context, Mongo
private void readChangeStream(MongoClient client, ReplicaSet replicaSet, ChangeEventSourceContext context, private void readChangeStream(MongoClient client, ReplicaSet replicaSet, ChangeEventSourceContext context,
MongoDbOffsetContext offsetContext) { MongoDbOffsetContext offsetContext) {
final ReplicaSetPartition rsPartition = offsetContext.getReplicaSetPartition(replicaSet); final ReplicaSetPartition rsPartition = effectiveOffset.getReplicaSetPartition(replicaSet);
final ReplicaSetOffsetContext rsOffsetContext = offsetContext.getReplicaSetOffsetContext(replicaSet); final ReplicaSetOffsetContext rsOffsetContext = effectiveOffset.getReplicaSetOffsetContext(replicaSet);
LOGGER.info("Reading change stream for '{}'", replicaSet); LOGGER.info("Reading change stream for '{}'", replicaSet);

View File

@ -93,6 +93,16 @@ public List<ReplicaSet> all() {
return new ArrayList<>(this.replicaSets); return new ArrayList<>(this.replicaSets);
} }
/**
* Get the ReplicaSet for the incremental snapshot
*
* @return in case of a ReplicaSet deployments return the only ReplicaSet available.
* In case of a Sharded Cluster, for incremental snapshot, only the connection.mode=sharded is supported. In this case only one ReplicaSet is present.
*/
public ReplicaSet getIncrementalSnapshotReplicaSet() {
return all().get(0);
}
/** /**
* Determine if one or more replica sets has been added or removed since the prior state. * Determine if one or more replica sets has been added or removed since the prior state.
* *

View File

@ -45,7 +45,7 @@
import io.debezium.doc.FixFor; import io.debezium.doc.FixFor;
import io.debezium.engine.DebeziumEngine; import io.debezium.engine.DebeziumEngine;
import io.debezium.junit.logging.LogInterceptor; import io.debezium.junit.logging.LogInterceptor;
import io.debezium.pipeline.signal.StopSnapshot; import io.debezium.pipeline.signal.actions.snapshotting.StopSnapshot;
/** /**
* Test to verify incremental snapshotting for MongoDB. * Test to verify incremental snapshotting for MongoDB.
@ -89,6 +89,7 @@ protected Configuration.Builder config() {
.with(MongoDbConnectorConfig.DATABASE_INCLUDE_LIST, DATABASE_NAME) .with(MongoDbConnectorConfig.DATABASE_INCLUDE_LIST, DATABASE_NAME)
.with(MongoDbConnectorConfig.COLLECTION_INCLUDE_LIST, fullDataCollectionName() + ",dbA.c1,dbA.c2") .with(MongoDbConnectorConfig.COLLECTION_INCLUDE_LIST, fullDataCollectionName() + ",dbA.c1,dbA.c2")
.with(MongoDbConnectorConfig.SIGNAL_DATA_COLLECTION, SIGNAL_COLLECTION_NAME) .with(MongoDbConnectorConfig.SIGNAL_DATA_COLLECTION, SIGNAL_COLLECTION_NAME)
.with(MongoDbConnectorConfig.SIGNAL_POLL_INTERVAL_MS, 5)
.with(MongoDbConnectorConfig.INCREMENTAL_SNAPSHOT_CHUNK_SIZE, 10) .with(MongoDbConnectorConfig.INCREMENTAL_SNAPSHOT_CHUNK_SIZE, 10)
.with(MongoDbConnectorConfig.SNAPSHOT_MODE, SnapshotMode.NEVER); .with(MongoDbConnectorConfig.SNAPSHOT_MODE, SnapshotMode.NEVER);
} }
@ -418,6 +419,8 @@ public void invalidTablesInTheList() throws Exception {
public void snapshotOnlyWithRestart() throws Exception { public void snapshotOnlyWithRestart() throws Exception {
// Testing.Print.enable(); // Testing.Print.enable();
LogInterceptor interceptor = new LogInterceptor(MongoDbIncrementalSnapshotChangeEventSource.class);
populateDataCollection(); populateDataCollection();
final Configuration config = config().build(); final Configuration config = config().build();
startAndConsumeTillEnd(connectorClass(), config); startAndConsumeTillEnd(connectorClass(), config);
@ -429,6 +432,10 @@ public void snapshotOnlyWithRestart() throws Exception {
sendAdHocSnapshotSignal(); sendAdHocSnapshotSignal();
Awaitility.await().atMost(60, TimeUnit.SECONDS)
.until(() -> interceptor
.containsMessage("No data returned by the query, incremental snapshotting of table '" + "rs0." + fullDataCollectionName() + "' finished"));
final int expectedRecordCount = ROW_COUNT; final int expectedRecordCount = ROW_COUNT;
final AtomicInteger recordCounter = new AtomicInteger(); final AtomicInteger recordCounter = new AtomicInteger();
final AtomicBoolean restarted = new AtomicBoolean(); final AtomicBoolean restarted = new AtomicBoolean();

View File

@ -23,7 +23,7 @@
import io.debezium.connector.mongodb.MongoDbConnectorConfig.ConnectionMode; import io.debezium.connector.mongodb.MongoDbConnectorConfig.ConnectionMode;
import io.debezium.data.Envelope; import io.debezium.data.Envelope;
public class ShardedMongoConnectorIT extends AbstractShardedMongoConnectorIT { public class ShardedMongoDbConnectorIT extends AbstractShardedMongoConnectorIT {
public static final String TOPIC_PREFIX = "mongo"; public static final String TOPIC_PREFIX = "mongo";
private static final int INIT_DOCUMENT_COUNT = 1000; private static final int INIT_DOCUMENT_COUNT = 1000;

View File

@ -7,6 +7,7 @@
import java.sql.SQLException; import java.sql.SQLException;
import java.util.List; import java.util.List;
import java.util.Map;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import org.apache.kafka.connect.source.SourceRecord; import org.apache.kafka.connect.source.SourceRecord;
@ -21,6 +22,7 @@
import io.debezium.connector.mysql.MySqlConnection.MySqlConnectionConfiguration; import io.debezium.connector.mysql.MySqlConnection.MySqlConnectionConfiguration;
import io.debezium.connector.mysql.MySqlConnectorConfig.BigIntUnsignedHandlingMode; import io.debezium.connector.mysql.MySqlConnectorConfig.BigIntUnsignedHandlingMode;
import io.debezium.connector.mysql.MySqlConnectorConfig.SnapshotMode; import io.debezium.connector.mysql.MySqlConnectorConfig.SnapshotMode;
import io.debezium.document.DocumentReader;
import io.debezium.jdbc.DefaultMainConnectionProvidingConnectionFactory; import io.debezium.jdbc.DefaultMainConnectionProvidingConnectionFactory;
import io.debezium.jdbc.JdbcValueConverters.BigIntUnsignedMode; import io.debezium.jdbc.JdbcValueConverters.BigIntUnsignedMode;
import io.debezium.jdbc.JdbcValueConverters.DecimalMode; import io.debezium.jdbc.JdbcValueConverters.DecimalMode;
@ -30,6 +32,7 @@
import io.debezium.pipeline.DataChangeEvent; import io.debezium.pipeline.DataChangeEvent;
import io.debezium.pipeline.ErrorHandler; import io.debezium.pipeline.ErrorHandler;
import io.debezium.pipeline.EventDispatcher; import io.debezium.pipeline.EventDispatcher;
import io.debezium.pipeline.signal.SignalProcessor;
import io.debezium.pipeline.spi.Offsets; import io.debezium.pipeline.spi.Offsets;
import io.debezium.relational.TableId; import io.debezium.relational.TableId;
import io.debezium.schema.SchemaNameAdjuster; import io.debezium.schema.SchemaNameAdjuster;
@ -135,6 +138,12 @@ public ChangeEventSourceCoordinator<MySqlPartition, MySqlOffsetContext> start(Co
final MySqlEventMetadataProvider metadataProvider = new MySqlEventMetadataProvider(); final MySqlEventMetadataProvider metadataProvider = new MySqlEventMetadataProvider();
SignalProcessor<MySqlPartition, MySqlOffsetContext> signalProcessor = new SignalProcessor<>(
MySqlConnector.class, connectorConfig, Map.of(),
getAvailableSignalChannels(),
DocumentReader.defaultReader(),
previousOffsets);
final Configuration heartbeatConfig = config; final Configuration heartbeatConfig = config;
final EventDispatcher<MySqlPartition, TableId> dispatcher = new EventDispatcher<>( final EventDispatcher<MySqlPartition, TableId> dispatcher = new EventDispatcher<>(
connectorConfig, connectorConfig,
@ -164,10 +173,13 @@ public ChangeEventSourceCoordinator<MySqlPartition, MySqlOffsetContext> start(Co
break; break;
} }
}), }),
schemaNameAdjuster); schemaNameAdjuster,
signalProcessor);
final MySqlStreamingChangeEventSourceMetrics streamingMetrics = new MySqlStreamingChangeEventSourceMetrics(taskContext, queue, metadataProvider); final MySqlStreamingChangeEventSourceMetrics streamingMetrics = new MySqlStreamingChangeEventSourceMetrics(taskContext, queue, metadataProvider);
dispatcher.getSignalingActions().forEach(signalProcessor::registerSignalAction);
ChangeEventSourceCoordinator<MySqlPartition, MySqlOffsetContext> coordinator = new ChangeEventSourceCoordinator<>( ChangeEventSourceCoordinator<MySqlPartition, MySqlOffsetContext> coordinator = new ChangeEventSourceCoordinator<>(
previousOffsets, previousOffsets,
errorHandler, errorHandler,
@ -176,7 +188,8 @@ public ChangeEventSourceCoordinator<MySqlPartition, MySqlOffsetContext> start(Co
new MySqlChangeEventSourceFactory(connectorConfig, connectionFactory, errorHandler, dispatcher, clock, schema, taskContext, streamingMetrics, queue), new MySqlChangeEventSourceFactory(connectorConfig, connectionFactory, errorHandler, dispatcher, clock, schema, taskContext, streamingMetrics, queue),
new MySqlChangeEventSourceMetricsFactory(streamingMetrics), new MySqlChangeEventSourceMetricsFactory(streamingMetrics),
dispatcher, dispatcher,
schema); schema,
signalProcessor);
coordinator.start(taskContext, this.queue, metadataProvider); coordinator.start(taskContext, this.queue, metadataProvider);

View File

@ -7,6 +7,7 @@
import java.sql.SQLException; import java.sql.SQLException;
import java.util.List; import java.util.List;
import java.util.Map;
import java.util.Optional; import java.util.Optional;
import java.util.function.Consumer; import java.util.function.Consumer;
@ -14,14 +15,9 @@
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import io.debezium.DebeziumException; import io.debezium.DebeziumException;
import io.debezium.connector.mysql.signal.ExecuteSnapshotKafkaSignal;
import io.debezium.connector.mysql.signal.KafkaSignal;
import io.debezium.connector.mysql.signal.KafkaSignalThread;
import io.debezium.connector.mysql.signal.PauseSnapshotKafkaSignal;
import io.debezium.connector.mysql.signal.ResumeSnapshotKafkaSignal;
import io.debezium.connector.mysql.signal.StopSnapshotKafkaSignal;
import io.debezium.jdbc.JdbcConnection; import io.debezium.jdbc.JdbcConnection;
import io.debezium.pipeline.EventDispatcher; import io.debezium.pipeline.EventDispatcher;
import io.debezium.pipeline.signal.channels.KafkaSignalChannel;
import io.debezium.pipeline.source.snapshot.incremental.AbstractIncrementalSnapshotChangeEventSource; import io.debezium.pipeline.source.snapshot.incremental.AbstractIncrementalSnapshotChangeEventSource;
import io.debezium.pipeline.source.spi.DataChangeEventListener; import io.debezium.pipeline.source.spi.DataChangeEventListener;
import io.debezium.pipeline.source.spi.SnapshotProgressListener; import io.debezium.pipeline.source.spi.SnapshotProgressListener;
@ -83,10 +79,7 @@ public class MySqlReadOnlyIncrementalSnapshotChangeEventSource<T extends DataCol
private static final Logger LOGGER = LoggerFactory.getLogger(MySqlReadOnlyIncrementalSnapshotChangeEventSource.class); private static final Logger LOGGER = LoggerFactory.getLogger(MySqlReadOnlyIncrementalSnapshotChangeEventSource.class);
private final String showMasterStmt = "SHOW MASTER STATUS"; private final String showMasterStmt = "SHOW MASTER STATUS";
private final KafkaSignalThread<T> kafkaSignal; private final KafkaSignalChannel kafkaSignal;
private MySqlPartition partition;
private OffsetContext offsetContext;
public MySqlReadOnlyIncrementalSnapshotChangeEventSource(RelationalDatabaseConnectorConfig config, public MySqlReadOnlyIncrementalSnapshotChangeEventSource(RelationalDatabaseConnectorConfig config,
JdbcConnection jdbcConnection, JdbcConnection jdbcConnection,
@ -96,22 +89,17 @@ public MySqlReadOnlyIncrementalSnapshotChangeEventSource(RelationalDatabaseConne
SnapshotProgressListener<MySqlPartition> progressListener, SnapshotProgressListener<MySqlPartition> progressListener,
DataChangeEventListener<MySqlPartition> dataChangeEventListener) { DataChangeEventListener<MySqlPartition> dataChangeEventListener) {
super(config, jdbcConnection, dispatcher, databaseSchema, clock, progressListener, dataChangeEventListener); super(config, jdbcConnection, dispatcher, databaseSchema, clock, progressListener, dataChangeEventListener);
kafkaSignal = new KafkaSignalThread<>(MySqlConnector.class, config, this); kafkaSignal = new KafkaSignalChannel();
} }
@Override @Override
public void init(MySqlPartition partition, OffsetContext offsetContext) { public void init(MySqlPartition partition, OffsetContext offsetContext) {
super.init(partition, offsetContext); super.init(partition, offsetContext);
// cache these for stop signals
this.partition = partition;
this.offsetContext = offsetContext;
Long signalOffset = getContext().getSignalOffset(); Long signalOffset = getContext().getSignalOffset();
if (signalOffset != null) { if (signalOffset != null) {
kafkaSignal.seek(signalOffset); kafkaSignal.seek(signalOffset);
} }
kafkaSignal.start();
} }
@Override @Override
@ -120,7 +108,6 @@ public void processMessage(MySqlPartition partition, DataCollectionId dataCollec
LOGGER.warn("Context is null, skipping message processing"); LOGGER.warn("Context is null, skipping message processing");
return; return;
} }
checkEnqueuedSnapshotSignals(partition, offsetContext);
LOGGER.trace("Checking window for table '{}', key '{}', window contains '{}'", dataCollectionId, key, window); LOGGER.trace("Checking window for table '{}', key '{}', window contains '{}'", dataCollectionId, key, window);
boolean windowClosed = getContext().updateWindowState(offsetContext); boolean windowClosed = getContext().updateWindowState(offsetContext);
if (windowClosed) { if (windowClosed) {
@ -138,7 +125,6 @@ public void processHeartbeat(MySqlPartition partition, OffsetContext offsetConte
LOGGER.warn("Context is null, skipping message processing"); LOGGER.warn("Context is null, skipping message processing");
return; return;
} }
checkEnqueuedSnapshotSignals(partition, offsetContext);
readUntilGtidChange(partition, offsetContext); readUntilGtidChange(partition, offsetContext);
} }
@ -160,7 +146,6 @@ public void processFilteredEvent(MySqlPartition partition, OffsetContext offsetC
LOGGER.warn("Context is null, skipping message processing"); LOGGER.warn("Context is null, skipping message processing");
return; return;
} }
checkEnqueuedSnapshotSignals(partition, offsetContext);
boolean windowClosed = getContext().updateWindowState(offsetContext); boolean windowClosed = getContext().updateWindowState(offsetContext);
if (windowClosed) { if (windowClosed) {
sendWindowEvents(partition, offsetContext); sendWindowEvents(partition, offsetContext);
@ -168,25 +153,6 @@ public void processFilteredEvent(MySqlPartition partition, OffsetContext offsetC
} }
} }
public void stopSnapshot(List<String> dataCollectionIds, long signalOffset) {
// We explicitly do not use the queue here for stop signals on purpose, for immediate processing
final StopSnapshotKafkaSignal signal = new StopSnapshotKafkaSignal(dataCollectionIds, signalOffset);
removeDataCollectionsFromSnapshot(signal, partition, offsetContext);
}
public void enqueueDataCollectionNamesToSnapshot(List<String> dataCollectionIds, long signalOffset, Optional<String> additionalCondition,
Optional<String> surrogateKey) {
getContext().enqueueKafkaSignal(new ExecuteSnapshotKafkaSignal(dataCollectionIds, signalOffset, additionalCondition, surrogateKey));
}
public void enqueuePauseSnapshot() {
getContext().enqueueKafkaSignal(new PauseSnapshotKafkaSignal());
}
public void enqueueResumeSnapshot() {
getContext().enqueueKafkaSignal(new ResumeSnapshotKafkaSignal());
}
@Override @Override
public void processTransactionStartedEvent(MySqlPartition partition, OffsetContext offsetContext) throws InterruptedException { public void processTransactionStartedEvent(MySqlPartition partition, OffsetContext offsetContext) throws InterruptedException {
if (getContext() == null) { if (getContext() == null) {
@ -262,39 +228,22 @@ protected void sendEvent(MySqlPartition partition, EventDispatcher<MySqlPartitio
sourceInfo.setQuery(query); sourceInfo.setQuery(query);
} }
private void checkEnqueuedSnapshotSignals(MySqlPartition partition, OffsetContext offsetContext) throws InterruptedException { @Override
while (getContext().hasKafkaSignals()) { public void addDataCollectionNamesToSnapshot(MySqlPartition partition, OffsetContext offsetContext, Map<String, Object> additionalData,
KafkaSignal signal = getContext().getKafkaSignals(); List<String> dataCollectionIds,
if (signal instanceof ExecuteSnapshotKafkaSignal) { Optional<String> additionalCondition, Optional<String> surrogateKey)
addDataCollectionNamesToSnapshot((ExecuteSnapshotKafkaSignal) signal, partition, offsetContext);
}
else if (signal instanceof StopSnapshotKafkaSignal) {
// If a stop signal gets passed here, write a log entry rather than throw an exception
LOGGER.warn("Stop signal skipped, this should never be processed via an enqueued signal");
}
else if (signal instanceof PauseSnapshotKafkaSignal) {
pauseSnapshot(partition, offsetContext);
}
else if (signal instanceof ResumeSnapshotKafkaSignal) {
resumeSnapshot(partition, offsetContext);
}
else {
throw new IllegalArgumentException("Unknown Kafka signal " + signal);
}
}
}
private void addDataCollectionNamesToSnapshot(ExecuteSnapshotKafkaSignal executeSnapshotSignal, MySqlPartition partition, OffsetContext offsetContext)
throws InterruptedException { throws InterruptedException {
super.addDataCollectionNamesToSnapshot(partition, executeSnapshotSignal.getDataCollections(), executeSnapshotSignal.getAdditionalCondition(), super.addDataCollectionNamesToSnapshot(partition, offsetContext, additionalData, dataCollectionIds, additionalCondition, surrogateKey);
executeSnapshotSignal.getSurrogateKey(),
offsetContext); getContext().setSignalOffset((Long) additionalData.get(KafkaSignalChannel.CHANNEL_OFFSET));
getContext().setSignalOffset(executeSnapshotSignal.getSignalOffset());
} }
private void removeDataCollectionsFromSnapshot(StopSnapshotKafkaSignal stopSnapshotKafkaSignal, MySqlPartition partition, OffsetContext offsetContext) { @Override
super.stopSnapshot(partition, stopSnapshotKafkaSignal.getDataCollections(), offsetContext); public void stopSnapshot(MySqlPartition partition, OffsetContext offsetContext, Map<String, Object> additionalData, List<String> dataCollectionIds) {
getContext().setSignalOffset(stopSnapshotKafkaSignal.getSignalOffset()); {
super.stopSnapshot(partition, offsetContext, additionalData, dataCollectionIds);
getContext().setSignalOffset((Long) additionalData.get(KafkaSignalChannel.CHANNEL_OFFSET));
}
} }
private MySqlReadOnlyIncrementalSnapshotContext<T> getContext() { private MySqlReadOnlyIncrementalSnapshotContext<T> getContext() {

View File

@ -8,14 +8,11 @@
import static io.debezium.connector.mysql.GtidSet.GTID_DELIMITER; import static io.debezium.connector.mysql.GtidSet.GTID_DELIMITER;
import java.util.Map; import java.util.Map;
import java.util.Queue;
import java.util.concurrent.ConcurrentLinkedQueue;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import io.debezium.annotation.NotThreadSafe; import io.debezium.annotation.NotThreadSafe;
import io.debezium.connector.mysql.signal.KafkaSignal;
import io.debezium.pipeline.source.snapshot.incremental.AbstractIncrementalSnapshotContext; import io.debezium.pipeline.source.snapshot.incremental.AbstractIncrementalSnapshotContext;
import io.debezium.pipeline.source.snapshot.incremental.IncrementalSnapshotContext; import io.debezium.pipeline.source.snapshot.incremental.IncrementalSnapshotContext;
import io.debezium.pipeline.spi.OffsetContext; import io.debezium.pipeline.spi.OffsetContext;
@ -29,7 +26,6 @@ public class MySqlReadOnlyIncrementalSnapshotContext<T> extends AbstractIncremen
private GtidSet lowWatermark; private GtidSet lowWatermark;
private GtidSet highWatermark; private GtidSet highWatermark;
private Long signalOffset; private Long signalOffset;
private final Queue<KafkaSignal> kafkaSignals = new ConcurrentLinkedQueue<>();
public static final String SIGNAL_OFFSET = INCREMENTAL_SNAPSHOT_KEY + "_signal_offset"; public static final String SIGNAL_OFFSET = INCREMENTAL_SNAPSHOT_KEY + "_signal_offset";
public MySqlReadOnlyIncrementalSnapshotContext() { public MySqlReadOnlyIncrementalSnapshotContext() {
@ -141,18 +137,6 @@ public Map<String, Object> store(Map<String, Object> offset) {
return snapshotOffset; return snapshotOffset;
} }
public void enqueueKafkaSignal(KafkaSignal signal) {
kafkaSignals.add(signal);
}
public KafkaSignal getKafkaSignals() {
return kafkaSignals.poll();
}
public boolean hasKafkaSignals() {
return !kafkaSignals.isEmpty();
}
public boolean watermarksChanged() { public boolean watermarksChanged() {
return !previousLowWatermark.equals(lowWatermark) || !previousHighWatermark.equals(highWatermark); return !previousLowWatermark.equals(lowWatermark) || !previousHighWatermark.equals(highWatermark);
} }

View File

@ -120,6 +120,7 @@ public class MySqlStreamingChangeEventSource implements StreamingChangeEventSour
@SingleThreadAccess("binlog client thread") @SingleThreadAccess("binlog client thread")
private Instant eventTimestamp; private Instant eventTimestamp;
private MySqlOffsetContext effectiveOffsetContext;
public static class BinlogPosition { public static class BinlogPosition {
final String filename; final String filename;
@ -870,6 +871,14 @@ private SSLMode sslModeFor(SecureConnectionMode mode) {
return null; return null;
} }
@Override
public void init(MySqlOffsetContext offsetContext) {
this.effectiveOffsetContext = offsetContext != null
? offsetContext
: MySqlOffsetContext.initial(connectorConfig);
}
@Override @Override
public void execute(ChangeEventSourceContext context, MySqlPartition partition, MySqlOffsetContext offsetContext) throws InterruptedException { public void execute(ChangeEventSourceContext context, MySqlPartition partition, MySqlOffsetContext offsetContext) throws InterruptedException {
if (!connectorConfig.getSnapshotMode().shouldStream()) { if (!connectorConfig.getSnapshotMode().shouldStream()) {
@ -881,10 +890,6 @@ public void execute(ChangeEventSourceContext context, MySqlPartition partition,
} }
final Set<Operation> skippedOperations = connectorConfig.getSkippedOperations(); final Set<Operation> skippedOperations = connectorConfig.getSkippedOperations();
final MySqlOffsetContext effectiveOffsetContext = offsetContext != null
? offsetContext
: MySqlOffsetContext.initial(connectorConfig);
// Register our event handlers ... // Register our event handlers ...
eventHandlers.put(EventType.STOP, (event) -> handleServerStop(effectiveOffsetContext, event)); eventHandlers.put(EventType.STOP, (event) -> handleServerStop(effectiveOffsetContext, event));
eventHandlers.put(EventType.HEARTBEAT, (event) -> handleServerHeartbeat(partition, effectiveOffsetContext, event)); eventHandlers.put(EventType.HEARTBEAT, (event) -> handleServerHeartbeat(partition, effectiveOffsetContext, event));
@ -1045,6 +1050,11 @@ public void execute(ChangeEventSourceContext context, MySqlPartition partition,
} }
} }
@Override
public MySqlOffsetContext getOffsetContext() {
return effectiveOffsetContext;
}
private SSLSocketFactory getBinlogSslSocketFactory(MySqlConnectorConfig connectorConfig, MySqlConnection connection) { private SSLSocketFactory getBinlogSslSocketFactory(MySqlConnectorConfig connectorConfig, MySqlConnection connection) {
String acceptedTlsVersion = connection.getSessionVariableForSslVersion(); String acceptedTlsVersion = connection.getSessionVariableForSslVersion();
if (!isNullOrEmpty(acceptedTlsVersion)) { if (!isNullOrEmpty(acceptedTlsVersion)) {

View File

@ -1,39 +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.signal;
import java.util.List;
import java.util.Optional;
public class ExecuteSnapshotKafkaSignal implements KafkaSignal {
private final List<String> dataCollections;
private final long signalOffset;
private final Optional<String> additionalCondition;
private final Optional<String> surrogateKey;
public ExecuteSnapshotKafkaSignal(List<String> dataCollections, long signalOffset, Optional<String> additionalCondition, Optional<String> surrogateKey) {
this.dataCollections = dataCollections;
this.signalOffset = signalOffset;
this.additionalCondition = additionalCondition;
this.surrogateKey = surrogateKey;
}
public List<String> getDataCollections() {
return dataCollections;
}
public long getSignalOffset() {
return signalOffset;
}
public Optional<String> getAdditionalCondition() {
return additionalCondition;
}
public Optional<String> getSurrogateKey() {
return surrogateKey;
}
}

View File

@ -1,12 +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.signal;
/**
* Signal send to Debezium via Kafka.
*/
public interface KafkaSignal {
}

View File

@ -1,221 +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.signal;
import java.io.IOException;
import java.time.Duration;
import java.util.List;
import java.util.Optional;
import java.util.UUID;
import java.util.concurrent.ExecutorService;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.consumer.ConsumerRecords;
import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.common.serialization.StringDeserializer;
import org.apache.kafka.connect.source.SourceConnector;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import io.debezium.config.CommonConnectorConfig;
import io.debezium.config.Configuration;
import io.debezium.config.Field;
import io.debezium.connector.mysql.MySqlReadOnlyIncrementalSnapshotChangeEventSource;
import io.debezium.document.Document;
import io.debezium.document.DocumentReader;
import io.debezium.pipeline.signal.AbstractSnapshotSignal;
import io.debezium.pipeline.signal.ExecuteSnapshot;
import io.debezium.pipeline.signal.PauseIncrementalSnapshot;
import io.debezium.pipeline.signal.ResumeIncrementalSnapshot;
import io.debezium.pipeline.signal.StopSnapshot;
import io.debezium.spi.schema.DataCollectionId;
import io.debezium.util.Collect;
import io.debezium.util.Loggings;
import io.debezium.util.Threads;
/**
* The class responsible for processing of signals delivered to Debezium via a dedicated Kafka topic.
* The signal message must have the following structure:
* <ul>
* <li>{@code id STRING} - the unique identifier of the signal sent, usually UUID, can be used for deduplication</li>
* <li>{@code type STRING} - the unique logical name of the code executing the signal</li>
* <li>{@code data STRING} - the data in JSON format that are passed to the signal code
* </ul>
*/
public class KafkaSignalThread<T extends DataCollectionId> {
private static final Logger LOGGER = LoggerFactory.getLogger(KafkaSignalThread.class);
private final ExecutorService signalTopicListenerExecutor;
private final String topicName;
private final String connectorName;
private final Duration pollTimeoutMs;
private final MySqlReadOnlyIncrementalSnapshotChangeEventSource<T> eventSource;
private final KafkaConsumer<String, String> signalsConsumer;
public static final String CONFIGURATION_FIELD_PREFIX_STRING = "signal.";
private static final String CONSUMER_PREFIX = CONFIGURATION_FIELD_PREFIX_STRING + "consumer.";
public static final Field SIGNAL_TOPIC = Field.create(CONFIGURATION_FIELD_PREFIX_STRING + "kafka.topic")
.withDisplayName("Signal topic name")
.withType(ConfigDef.Type.STRING)
.withWidth(ConfigDef.Width.LONG)
.withImportance(ConfigDef.Importance.HIGH)
.withDescription("The name of the topic for the signals to the connector")
.withValidation(Field::isRequired);
public static final Field BOOTSTRAP_SERVERS = Field.create(CONFIGURATION_FIELD_PREFIX_STRING + "kafka.bootstrap.servers")
.withDisplayName("Kafka broker addresses")
.withType(ConfigDef.Type.STRING)
.withWidth(ConfigDef.Width.LONG)
.withImportance(ConfigDef.Importance.HIGH)
.withDescription("A list of host/port pairs that the connector will use for establishing the initial "
+ "connection to the Kafka cluster for retrieving signals to the connector."
+ "This should point to the same Kafka cluster used by the Kafka Connect process.")
.withValidation(Field::isRequired);
public static final Field SIGNAL_POLL_TIMEOUT_MS = Field.create(CONFIGURATION_FIELD_PREFIX_STRING
+ "kafka.poll.timeout.ms")
.withDisplayName("Poll timeout for kafka signals (ms)")
.withType(ConfigDef.Type.INT)
.withWidth(ConfigDef.Width.SHORT)
.withImportance(ConfigDef.Importance.LOW)
.withDescription("The number of milliseconds to wait while polling signals.")
.withDefault(100)
.withValidation(Field::isNonNegativeInteger);
public KafkaSignalThread(Class<? extends SourceConnector> connectorType, CommonConnectorConfig connectorConfig,
MySqlReadOnlyIncrementalSnapshotChangeEventSource<T> eventSource) {
String signalName = "kafka-signal";
connectorName = connectorConfig.getLogicalName();
signalTopicListenerExecutor = Threads.newSingleThreadExecutor(connectorType, connectorName, signalName, true);
Configuration signalConfig = connectorConfig.getConfig().subset(CONFIGURATION_FIELD_PREFIX_STRING, false)
.edit()
.withDefault(KafkaSignalThread.SIGNAL_TOPIC, connectorName + "-signal")
.build();
this.eventSource = eventSource;
this.topicName = signalConfig.getString(SIGNAL_TOPIC);
this.pollTimeoutMs = Duration.ofMillis(signalConfig.getInteger(SIGNAL_POLL_TIMEOUT_MS));
String bootstrapServers = signalConfig.getString(BOOTSTRAP_SERVERS);
Configuration consumerConfig = signalConfig.subset(CONSUMER_PREFIX, true).edit()
.withDefault(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers)
.withDefault(ConsumerConfig.CLIENT_ID_CONFIG, UUID.randomUUID().toString())
.withDefault(ConsumerConfig.GROUP_ID_CONFIG, signalName)
.withDefault(ConsumerConfig.FETCH_MIN_BYTES_CONFIG, 1) // get even smallest message
.withDefault(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false)
.withDefault(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, 10000) // readjusted since 0.10.1.0
.withDefault(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class)
.withDefault(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class)
.build();
signalsConsumer = new KafkaConsumer<>(consumerConfig.asProperties());
LOGGER.info("Subscribing to signals topic '{}'", topicName);
signalsConsumer.assign(Collect.arrayListOf(new TopicPartition(topicName, 0)));
}
public void start() {
signalTopicListenerExecutor.submit(this::monitorSignals);
}
private void monitorSignals() {
while (true) {
// DBZ-1361 not using poll(Duration) to keep compatibility with AK 1.x
ConsumerRecords<String, String> recoveredRecords = signalsConsumer.poll(pollTimeoutMs.toMillis());
for (ConsumerRecord<String, String> record : recoveredRecords) {
try {
processSignal(record);
}
catch (final InterruptedException e) {
LOGGER.error("Signals processing was interrupted", e);
signalsConsumer.close();
return;
}
catch (final Exception e) {
Loggings.logErrorAndTraceRecord(LOGGER, record, "Skipped signal due to an error", e);
}
}
}
}
private void processSignal(ConsumerRecord<String, String> record) throws IOException, InterruptedException {
if (!connectorName.equals(record.key())) {
LOGGER.info("Signal key '{}' doesn't match the connector's name '{}'", record.key(), connectorName);
return;
}
String value = record.value();
LOGGER.trace("Processing signal: {}", value);
final Document jsonData = (value == null || value.isEmpty()) ? Document.create()
: DocumentReader.defaultReader().read(value);
String type = jsonData.getString("type");
Document data = jsonData.getDocument("data");
switch (type) {
case ExecuteSnapshot.NAME:
executeSnapshot(data, record.offset());
break;
case StopSnapshot.NAME:
executeStopSnapshot(data, record.offset());
break;
case PauseIncrementalSnapshot.NAME:
executePause(data);
break;
case ResumeIncrementalSnapshot.NAME:
executeResume(data);
break;
default:
LOGGER.warn("Unknown signal type {}", type);
}
}
private void executeSnapshot(Document data, long signalOffset) {
final List<String> dataCollections = ExecuteSnapshot.getDataCollections(data);
if (dataCollections != null) {
ExecuteSnapshot.SnapshotType snapshotType = ExecuteSnapshot.getSnapshotType(data);
Optional<String> additionalCondition = ExecuteSnapshot.getAdditionalCondition(data);
Optional<String> surrogateKey = ExecuteSnapshot.getSurrogateKey(data);
LOGGER.info("Requested '{}' snapshot of data collections '{}' with additional condition '{}'", snapshotType, dataCollections,
additionalCondition.orElse("No condition passed"));
if (snapshotType == ExecuteSnapshot.SnapshotType.INCREMENTAL) {
eventSource.enqueueDataCollectionNamesToSnapshot(dataCollections, signalOffset, additionalCondition, surrogateKey);
}
}
}
private void executeStopSnapshot(Document data, long signalOffset) {
final List<String> dataCollections = StopSnapshot.getDataCollections(data);
final AbstractSnapshotSignal.SnapshotType snapshotType = StopSnapshot.getSnapshotType(data);
if (dataCollections == null || dataCollections.isEmpty()) {
LOGGER.info("Requested stop of '{}' snapshot", snapshotType);
}
else {
LOGGER.info("Requested stop of '{}' snapshot of data collections '{}'", snapshotType, dataCollections);
}
if (snapshotType == AbstractSnapshotSignal.SnapshotType.INCREMENTAL) {
eventSource.stopSnapshot(dataCollections, signalOffset);
}
}
private void executePause(Document data) {
PauseIncrementalSnapshot.SnapshotType snapshotType = ExecuteSnapshot.getSnapshotType(data);
LOGGER.info("Requested snapshot pause");
if (snapshotType == PauseIncrementalSnapshot.SnapshotType.INCREMENTAL) {
eventSource.enqueuePauseSnapshot();
}
}
private void executeResume(Document data) {
ResumeIncrementalSnapshot.SnapshotType snapshotType = ExecuteSnapshot.getSnapshotType(data);
LOGGER.info("Requested snapshot resume");
if (snapshotType == ResumeIncrementalSnapshot.SnapshotType.INCREMENTAL) {
eventSource.enqueueResumeSnapshot();
}
}
public void seek(long signalOffset) {
signalsConsumer.seek(new TopicPartition(topicName, 0), signalOffset + 1);
}
}

View File

@ -1,9 +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.signal;
public class PauseSnapshotKafkaSignal implements KafkaSignal {
}

View File

@ -1,9 +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.signal;
public class ResumeSnapshotKafkaSignal implements KafkaSignal {
}

View File

@ -1,31 +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.signal;
import java.util.List;
/**
* A {@link KafkaSignal} implementation to stop a current executing snapshot.
*
* @author Chris Cranford
*/
public class StopSnapshotKafkaSignal implements KafkaSignal {
private final List<String> dataCollections;
private final long signalOffset;
public StopSnapshotKafkaSignal(List<String> dataCollections, long signalOffset) {
this.dataCollections = dataCollections;
this.signalOffset = signalOffset;
}
public List<String> getDataCollections() {
return dataCollections;
}
public long getSignalOffset() {
return signalOffset;
}
}

View File

@ -72,6 +72,7 @@ protected Configuration.Builder config() {
.with(MySqlConnectorConfig.SNAPSHOT_MODE, SnapshotMode.SCHEMA_ONLY.getValue()) .with(MySqlConnectorConfig.SNAPSHOT_MODE, SnapshotMode.SCHEMA_ONLY.getValue())
.with(MySqlConnectorConfig.INCLUDE_SCHEMA_CHANGES, false) .with(MySqlConnectorConfig.INCLUDE_SCHEMA_CHANGES, false)
.with(MySqlConnectorConfig.SIGNAL_DATA_COLLECTION, DATABASE.qualifiedTableName("debezium_signal")) .with(MySqlConnectorConfig.SIGNAL_DATA_COLLECTION, DATABASE.qualifiedTableName("debezium_signal"))
.with(CommonConnectorConfig.SIGNAL_POLL_INTERVAL_MS, 1)
.with(MySqlConnectorConfig.INCREMENTAL_SNAPSHOT_CHUNK_SIZE, 10) .with(MySqlConnectorConfig.INCREMENTAL_SNAPSHOT_CHUNK_SIZE, 10)
.with(MySqlConnectorConfig.INCREMENTAL_SNAPSHOT_ALLOW_SCHEMA_CHANGES, true) .with(MySqlConnectorConfig.INCREMENTAL_SNAPSHOT_ALLOW_SCHEMA_CHANGES, true)
.with(CommonConnectorConfig.SCHEMA_NAME_ADJUSTMENT_MODE, SchemaNameAdjustmentMode.AVRO); .with(CommonConnectorConfig.SCHEMA_NAME_ADJUSTMENT_MODE, SchemaNameAdjustmentMode.AVRO);
@ -93,6 +94,7 @@ protected Configuration.Builder mutableConfig(boolean signalTableOnly, boolean s
.with(MySqlConnectorConfig.SNAPSHOT_MODE, SnapshotMode.INITIAL.getValue()) .with(MySqlConnectorConfig.SNAPSHOT_MODE, SnapshotMode.INITIAL.getValue())
.with(MySqlConnectorConfig.INCLUDE_SCHEMA_CHANGES, false) .with(MySqlConnectorConfig.INCLUDE_SCHEMA_CHANGES, false)
.with(MySqlConnectorConfig.SIGNAL_DATA_COLLECTION, DATABASE.qualifiedTableName("debezium_signal")) .with(MySqlConnectorConfig.SIGNAL_DATA_COLLECTION, DATABASE.qualifiedTableName("debezium_signal"))
.with(CommonConnectorConfig.SIGNAL_POLL_INTERVAL_MS, 5)
.with(MySqlConnectorConfig.TABLE_INCLUDE_LIST, tableIncludeList) .with(MySqlConnectorConfig.TABLE_INCLUDE_LIST, tableIncludeList)
.with(MySqlConnectorConfig.INCREMENTAL_SNAPSHOT_CHUNK_SIZE, 10) .with(MySqlConnectorConfig.INCREMENTAL_SNAPSHOT_CHUNK_SIZE, 10)
.with(MySqlConnectorConfig.INCREMENTAL_SNAPSHOT_ALLOW_SCHEMA_CHANGES, true) .with(MySqlConnectorConfig.INCREMENTAL_SNAPSHOT_ALLOW_SCHEMA_CHANGES, true)

View File

@ -152,6 +152,7 @@ public void testPauseResumeSnapshotMetrics() throws Exception {
.with(MySqlConnectorConfig.TABLE_INCLUDE_LIST, String.format("%s", TABLE_NAME)) .with(MySqlConnectorConfig.TABLE_INCLUDE_LIST, String.format("%s", TABLE_NAME))
.with(SchemaHistory.STORE_ONLY_CAPTURED_TABLES_DDL, Boolean.TRUE) .with(SchemaHistory.STORE_ONLY_CAPTURED_TABLES_DDL, Boolean.TRUE)
.with(CommonConnectorConfig.INCREMENTAL_SNAPSHOT_CHUNK_SIZE, 1) .with(CommonConnectorConfig.INCREMENTAL_SNAPSHOT_CHUNK_SIZE, 1)
.with(CommonConnectorConfig.SIGNAL_POLL_INTERVAL_MS, 5)
.with(MySqlConnectorConfig.SIGNAL_DATA_COLLECTION, SIGNAL_TABLE_NAME) .with(MySqlConnectorConfig.SIGNAL_DATA_COLLECTION, SIGNAL_TABLE_NAME)
.build()); .build());

View File

@ -37,11 +37,11 @@
import io.debezium.config.Configuration; import io.debezium.config.Configuration;
import io.debezium.connector.mysql.junit.SkipTestDependingOnGtidModeRule; import io.debezium.connector.mysql.junit.SkipTestDependingOnGtidModeRule;
import io.debezium.connector.mysql.junit.SkipWhenGtidModeIs; import io.debezium.connector.mysql.junit.SkipWhenGtidModeIs;
import io.debezium.connector.mysql.signal.KafkaSignalThread;
import io.debezium.doc.FixFor; import io.debezium.doc.FixFor;
import io.debezium.jdbc.JdbcConnection; import io.debezium.jdbc.JdbcConnection;
import io.debezium.junit.logging.LogInterceptor; import io.debezium.junit.logging.LogInterceptor;
import io.debezium.kafka.KafkaCluster; import io.debezium.kafka.KafkaCluster;
import io.debezium.pipeline.signal.channels.KafkaSignalChannel;
import io.debezium.pipeline.source.snapshot.incremental.AbstractIncrementalSnapshotChangeEventSource; import io.debezium.pipeline.source.snapshot.incremental.AbstractIncrementalSnapshotChangeEventSource;
import io.debezium.relational.RelationalDatabaseConnectorConfig; import io.debezium.relational.RelationalDatabaseConnectorConfig;
import io.debezium.util.Collect; import io.debezium.util.Collect;
@ -87,13 +87,14 @@ protected Configuration.Builder config() {
return super.config() return super.config()
.with(MySqlConnectorConfig.TABLE_EXCLUDE_LIST, DATABASE.getDatabaseName() + "." + EXCLUDED_TABLE) .with(MySqlConnectorConfig.TABLE_EXCLUDE_LIST, DATABASE.getDatabaseName() + "." + EXCLUDED_TABLE)
.with(MySqlConnectorConfig.READ_ONLY_CONNECTION, true) .with(MySqlConnectorConfig.READ_ONLY_CONNECTION, true)
.with(KafkaSignalThread.SIGNAL_TOPIC, getSignalsTopic()) .with(KafkaSignalChannel.SIGNAL_TOPIC, getSignalsTopic())
.with(KafkaSignalThread.BOOTSTRAP_SERVERS, kafka.brokerList()) .with(KafkaSignalChannel.BOOTSTRAP_SERVERS, kafka.brokerList())
.with(CommonConnectorConfig.SIGNAL_ENABLED_CHANNELS, "source,kafka")
.with(MySqlConnectorConfig.INCLUDE_SQL_QUERY, true) .with(MySqlConnectorConfig.INCLUDE_SQL_QUERY, true)
.with(RelationalDatabaseConnectorConfig.MSG_KEY_COLUMNS, String.format("%s:%s", DATABASE.qualifiedTableName("a42"), "pk1,pk2,pk3,pk4")); .with(RelationalDatabaseConnectorConfig.MSG_KEY_COLUMNS, String.format("%s:%s", DATABASE.qualifiedTableName("a42"), "pk1,pk2,pk3,pk4"));
} }
private String getSignalsTopic() { protected String getSignalsTopic() {
return DATABASE.getDatabaseName() + "signals_topic"; return DATABASE.getDatabaseName() + "signals_topic";
} }

View File

@ -7,6 +7,7 @@
import java.sql.SQLException; import java.sql.SQLException;
import java.util.List; import java.util.List;
import java.util.Map;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import org.apache.kafka.connect.source.SourceRecord; import org.apache.kafka.connect.source.SourceRecord;
@ -20,6 +21,7 @@
import io.debezium.connector.base.ChangeEventQueue; import io.debezium.connector.base.ChangeEventQueue;
import io.debezium.connector.common.BaseSourceTask; import io.debezium.connector.common.BaseSourceTask;
import io.debezium.connector.oracle.StreamingAdapter.TableNameCaseSensitivity; import io.debezium.connector.oracle.StreamingAdapter.TableNameCaseSensitivity;
import io.debezium.document.DocumentReader;
import io.debezium.jdbc.DefaultMainConnectionProvidingConnectionFactory; import io.debezium.jdbc.DefaultMainConnectionProvidingConnectionFactory;
import io.debezium.jdbc.JdbcConfiguration; import io.debezium.jdbc.JdbcConfiguration;
import io.debezium.jdbc.MainConnectionProvidingConnectionFactory; import io.debezium.jdbc.MainConnectionProvidingConnectionFactory;
@ -27,6 +29,7 @@
import io.debezium.pipeline.DataChangeEvent; import io.debezium.pipeline.DataChangeEvent;
import io.debezium.pipeline.ErrorHandler; import io.debezium.pipeline.ErrorHandler;
import io.debezium.pipeline.EventDispatcher; import io.debezium.pipeline.EventDispatcher;
import io.debezium.pipeline.signal.SignalProcessor;
import io.debezium.pipeline.spi.Offsets; import io.debezium.pipeline.spi.Offsets;
import io.debezium.relational.TableId; import io.debezium.relational.TableId;
import io.debezium.schema.SchemaNameAdjuster; import io.debezium.schema.SchemaNameAdjuster;
@ -94,6 +97,12 @@ public ChangeEventSourceCoordinator<OraclePartition, OracleOffsetContext> start(
final OracleEventMetadataProvider metadataProvider = new OracleEventMetadataProvider(); final OracleEventMetadataProvider metadataProvider = new OracleEventMetadataProvider();
SignalProcessor<OraclePartition, OracleOffsetContext> signalProcessor = new SignalProcessor<>(
OracleConnector.class, connectorConfig, Map.of(),
getAvailableSignalChannels(),
DocumentReader.defaultReader(),
previousOffsets);
EventDispatcher<OraclePartition, TableId> dispatcher = new EventDispatcher<>( EventDispatcher<OraclePartition, TableId> dispatcher = new EventDispatcher<>(
connectorConfig, connectorConfig,
topicNamingStrategy, topicNamingStrategy,
@ -110,11 +119,14 @@ public ChangeEventSourceCoordinator<OraclePartition, OracleOffsetContext> start(
final String sqlErrorId = exception.getMessage(); final String sqlErrorId = exception.getMessage();
throw new DebeziumException("Could not execute heartbeat action query (Error: " + sqlErrorId + ")", exception); throw new DebeziumException("Could not execute heartbeat action query (Error: " + sqlErrorId + ")", exception);
}), }),
schemaNameAdjuster); schemaNameAdjuster,
signalProcessor);
final OracleStreamingChangeEventSourceMetrics streamingMetrics = new OracleStreamingChangeEventSourceMetrics(taskContext, queue, metadataProvider, final OracleStreamingChangeEventSourceMetrics streamingMetrics = new OracleStreamingChangeEventSourceMetrics(taskContext, queue, metadataProvider,
connectorConfig); connectorConfig);
dispatcher.getSignalingActions().forEach(signalProcessor::registerSignalAction);
ChangeEventSourceCoordinator<OraclePartition, OracleOffsetContext> coordinator = new ChangeEventSourceCoordinator<>( ChangeEventSourceCoordinator<OraclePartition, OracleOffsetContext> coordinator = new ChangeEventSourceCoordinator<>(
previousOffsets, previousOffsets,
errorHandler, errorHandler,
@ -124,7 +136,7 @@ public ChangeEventSourceCoordinator<OraclePartition, OracleOffsetContext> start(
streamingMetrics), streamingMetrics),
new OracleChangeEventSourceMetricsFactory(streamingMetrics), new OracleChangeEventSourceMetricsFactory(streamingMetrics),
dispatcher, dispatcher,
schema); schema, signalProcessor);
coordinator.start(taskContext, this.queue, metadataProvider); coordinator.start(taskContext, this.queue, metadataProvider);

View File

@ -45,7 +45,9 @@
import io.debezium.jdbc.JdbcConfiguration; import io.debezium.jdbc.JdbcConfiguration;
import io.debezium.pipeline.ErrorHandler; import io.debezium.pipeline.ErrorHandler;
import io.debezium.pipeline.EventDispatcher; import io.debezium.pipeline.EventDispatcher;
import io.debezium.pipeline.source.snapshot.incremental.SignalBasedIncrementalSnapshotContext;
import io.debezium.pipeline.source.spi.StreamingChangeEventSource; import io.debezium.pipeline.source.spi.StreamingChangeEventSource;
import io.debezium.pipeline.txmetadata.TransactionContext;
import io.debezium.relational.Column; import io.debezium.relational.Column;
import io.debezium.relational.Table; import io.debezium.relational.Table;
import io.debezium.relational.TableId; import io.debezium.relational.TableId;
@ -87,6 +89,7 @@ public class LogMinerStreamingChangeEventSource implements StreamingChangeEventS
private Scn snapshotScn; private Scn snapshotScn;
private List<LogFile> currentLogFiles; private List<LogFile> currentLogFiles;
private List<BigInteger> currentRedoLogSequences; private List<BigInteger> currentRedoLogSequences;
private OracleOffsetContext effectiveOffset;
public LogMinerStreamingChangeEventSource(OracleConnectorConfig connectorConfig, public LogMinerStreamingChangeEventSource(OracleConnectorConfig connectorConfig,
OracleConnection jdbcConnection, EventDispatcher<OraclePartition, TableId> dispatcher, OracleConnection jdbcConnection, EventDispatcher<OraclePartition, TableId> dispatcher,
@ -110,6 +113,18 @@ public LogMinerStreamingChangeEventSource(OracleConnectorConfig connectorConfig,
this.maxDelay = connectorConfig.getLogMiningMaxDelay(); this.maxDelay = connectorConfig.getLogMiningMaxDelay();
} }
@Override
public void init(OracleOffsetContext offsetContext) throws InterruptedException {
this.effectiveOffset = offsetContext == null ? emptyContext() : offsetContext;
}
private OracleOffsetContext emptyContext() {
return OracleOffsetContext.create().logicalName(connectorConfig)
.snapshotPendingTransactions(Collections.emptyMap())
.transactionContext(new TransactionContext())
.incrementalSnapshotContext(new SignalBasedIncrementalSnapshotContext<>()).build();
}
/** /**
* This is the loop to get changes from LogMiner * This is the loop to get changes from LogMiner
* *
@ -126,6 +141,7 @@ public void execute(ChangeEventSourceContext context, OraclePartition partition,
// We explicitly expect auto-commit to be disabled // We explicitly expect auto-commit to be disabled
jdbcConnection.setAutoCommit(false); jdbcConnection.setAutoCommit(false);
this.effectiveOffset = offsetContext;
startScn = offsetContext.getScn(); startScn = offsetContext.getScn();
snapshotScn = offsetContext.getSnapshotScn(); snapshotScn = offsetContext.getSnapshotScn();
Scn firstScn = getFirstScnInLogs(jdbcConnection); Scn firstScn = getFirstScnInLogs(jdbcConnection);
@ -907,4 +923,9 @@ private boolean isStartScnInArchiveLogs(Scn startScn) throws SQLException {
public void commitOffset(Map<String, ?> partition, Map<String, ?> offset) { public void commitOffset(Map<String, ?> partition, Map<String, ?> offset) {
// nothing to do // nothing to do
} }
@Override
public OracleOffsetContext getOffsetContext() {
return effectiveOffset;
}
} }

View File

@ -6,6 +6,7 @@
package io.debezium.connector.oracle.xstream; package io.debezium.connector.oracle.xstream;
import java.sql.SQLException; import java.sql.SQLException;
import java.util.Collections;
import java.util.Map; import java.util.Map;
import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.atomic.AtomicReference;
@ -24,7 +25,9 @@
import io.debezium.connector.oracle.StreamingAdapter.TableNameCaseSensitivity; import io.debezium.connector.oracle.StreamingAdapter.TableNameCaseSensitivity;
import io.debezium.pipeline.ErrorHandler; import io.debezium.pipeline.ErrorHandler;
import io.debezium.pipeline.EventDispatcher; import io.debezium.pipeline.EventDispatcher;
import io.debezium.pipeline.source.snapshot.incremental.SignalBasedIncrementalSnapshotContext;
import io.debezium.pipeline.source.spi.StreamingChangeEventSource; import io.debezium.pipeline.source.spi.StreamingChangeEventSource;
import io.debezium.pipeline.txmetadata.TransactionContext;
import io.debezium.relational.TableId; import io.debezium.relational.TableId;
import io.debezium.util.Clock; import io.debezium.util.Clock;
@ -61,6 +64,7 @@ public class XstreamStreamingChangeEventSource implements StreamingChangeEventSo
* internal Oracle code locking. * internal Oracle code locking.
*/ */
private final AtomicReference<PositionAndScn> lcrMessage = new AtomicReference<>(); private final AtomicReference<PositionAndScn> lcrMessage = new AtomicReference<>();
private OracleOffsetContext effectiveOffset;
public XstreamStreamingChangeEventSource(OracleConnectorConfig connectorConfig, OracleConnection jdbcConnection, public XstreamStreamingChangeEventSource(OracleConnectorConfig connectorConfig, OracleConnection jdbcConnection,
EventDispatcher<OraclePartition, TableId> dispatcher, ErrorHandler errorHandler, EventDispatcher<OraclePartition, TableId> dispatcher, ErrorHandler errorHandler,
@ -77,10 +81,24 @@ public XstreamStreamingChangeEventSource(OracleConnectorConfig connectorConfig,
this.posVersion = resolvePosVersion(jdbcConnection, connectorConfig); this.posVersion = resolvePosVersion(jdbcConnection, connectorConfig);
} }
@Override
public void init(OracleOffsetContext offsetContext) throws InterruptedException {
this.effectiveOffset = offsetContext == null ? emptyContext() : offsetContext;
}
private OracleOffsetContext emptyContext() {
return OracleOffsetContext.create().logicalName(connectorConfig)
.snapshotPendingTransactions(Collections.emptyMap())
.transactionContext(new TransactionContext())
.incrementalSnapshotContext(new SignalBasedIncrementalSnapshotContext<>()).build();
}
@Override @Override
public void execute(ChangeEventSourceContext context, OraclePartition partition, OracleOffsetContext offsetContext) public void execute(ChangeEventSourceContext context, OraclePartition partition, OracleOffsetContext offsetContext)
throws InterruptedException { throws InterruptedException {
this.effectiveOffset = offsetContext;
LcrEventHandler eventHandler = new LcrEventHandler(connectorConfig, errorHandler, dispatcher, clock, schema, LcrEventHandler eventHandler = new LcrEventHandler(connectorConfig, errorHandler, dispatcher, clock, schema,
partition, offsetContext, partition, offsetContext,
TableNameCaseSensitivity.INSENSITIVE.equals(connectorConfig.getAdapter().getTableNameCaseSensitivity(jdbcConnection)), TableNameCaseSensitivity.INSENSITIVE.equals(connectorConfig.getAdapter().getTableNameCaseSensitivity(jdbcConnection)),
@ -139,6 +157,11 @@ public void commitOffset(Map<String, ?> partition, Map<String, ?> offset) {
} }
} }
@Override
public OracleOffsetContext getOffsetContext() {
return effectiveOffset;
}
private byte[] convertScnToPosition(Scn scn) { private byte[] convertScnToPosition(Scn scn) {
try { try {
return XStreamUtility.convertSCNToPosition(new NUMBER(scn.toString(), 0), this.posVersion); return XStreamUtility.convertSCNToPosition(new NUMBER(scn.toString(), 0), this.posVersion);

View File

@ -124,6 +124,13 @@
<artifactId>apicurio-registry-utils-converter</artifactId> <artifactId>apicurio-registry-utils-converter</artifactId>
<scope>test</scope> <scope>test</scope>
</dependency> </dependency>
<!-- Used for unit testing with Kafka -->
<dependency>
<groupId>org.apache.kafka</groupId>
<artifactId>kafka_${version.kafka.scala}</artifactId>
<scope>test</scope>
</dependency>
</dependencies> </dependencies>
<build> <build>

View File

@ -19,6 +19,7 @@
import io.debezium.pipeline.ErrorHandler; import io.debezium.pipeline.ErrorHandler;
import io.debezium.pipeline.EventDispatcher; import io.debezium.pipeline.EventDispatcher;
import io.debezium.pipeline.metrics.spi.ChangeEventSourceMetricsFactory; import io.debezium.pipeline.metrics.spi.ChangeEventSourceMetricsFactory;
import io.debezium.pipeline.signal.SignalProcessor;
import io.debezium.pipeline.source.spi.ChangeEventSource; import io.debezium.pipeline.source.spi.ChangeEventSource;
import io.debezium.pipeline.source.spi.ChangeEventSource.ChangeEventSourceContext; import io.debezium.pipeline.source.spi.ChangeEventSource.ChangeEventSourceContext;
import io.debezium.pipeline.source.spi.SnapshotChangeEventSource; import io.debezium.pipeline.source.spi.SnapshotChangeEventSource;
@ -43,9 +44,10 @@ public PostgresChangeEventSourceCoordinator(Offsets<PostgresPartition, PostgresO
PostgresChangeEventSourceFactory changeEventSourceFactory, PostgresChangeEventSourceFactory changeEventSourceFactory,
ChangeEventSourceMetricsFactory<PostgresPartition> changeEventSourceMetricsFactory, ChangeEventSourceMetricsFactory<PostgresPartition> changeEventSourceMetricsFactory,
EventDispatcher<PostgresPartition, ?> eventDispatcher, DatabaseSchema<?> schema, EventDispatcher<PostgresPartition, ?> eventDispatcher, DatabaseSchema<?> schema,
Snapshotter snapshotter, SlotState slotInfo) { Snapshotter snapshotter, SlotState slotInfo,
SignalProcessor<PostgresPartition, PostgresOffsetContext> signalProcessor) {
super(previousOffsets, errorHandler, connectorType, connectorConfig, changeEventSourceFactory, super(previousOffsets, errorHandler, connectorType, connectorConfig, changeEventSourceFactory,
changeEventSourceMetricsFactory, eventDispatcher, schema); changeEventSourceMetricsFactory, eventDispatcher, schema, signalProcessor);
this.snapshotter = snapshotter; this.snapshotter = snapshotter;
this.slotInfo = slotInfo; this.slotInfo = slotInfo;
} }

View File

@ -10,6 +10,7 @@
import java.sql.SQLException; import java.sql.SQLException;
import java.time.Duration; import java.time.Duration;
import java.util.List; import java.util.List;
import java.util.Map;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.errors.ConnectException;
@ -31,12 +32,14 @@
import io.debezium.connector.postgresql.spi.SlotCreationResult; import io.debezium.connector.postgresql.spi.SlotCreationResult;
import io.debezium.connector.postgresql.spi.SlotState; import io.debezium.connector.postgresql.spi.SlotState;
import io.debezium.connector.postgresql.spi.Snapshotter; import io.debezium.connector.postgresql.spi.Snapshotter;
import io.debezium.document.DocumentReader;
import io.debezium.jdbc.DefaultMainConnectionProvidingConnectionFactory; import io.debezium.jdbc.DefaultMainConnectionProvidingConnectionFactory;
import io.debezium.jdbc.MainConnectionProvidingConnectionFactory; import io.debezium.jdbc.MainConnectionProvidingConnectionFactory;
import io.debezium.pipeline.ChangeEventSourceCoordinator; import io.debezium.pipeline.ChangeEventSourceCoordinator;
import io.debezium.pipeline.DataChangeEvent; import io.debezium.pipeline.DataChangeEvent;
import io.debezium.pipeline.ErrorHandler; import io.debezium.pipeline.ErrorHandler;
import io.debezium.pipeline.metrics.DefaultChangeEventSourceMetricsFactory; import io.debezium.pipeline.metrics.DefaultChangeEventSourceMetricsFactory;
import io.debezium.pipeline.signal.SignalProcessor;
import io.debezium.pipeline.spi.Offsets; import io.debezium.pipeline.spi.Offsets;
import io.debezium.relational.TableId; import io.debezium.relational.TableId;
import io.debezium.schema.SchemaNameAdjuster; import io.debezium.schema.SchemaNameAdjuster;
@ -171,6 +174,12 @@ public ChangeEventSourceCoordinator<PostgresPartition, PostgresOffsetContext> st
final PostgresEventMetadataProvider metadataProvider = new PostgresEventMetadataProvider(); final PostgresEventMetadataProvider metadataProvider = new PostgresEventMetadataProvider();
SignalProcessor<PostgresPartition, PostgresOffsetContext> signalProcessor = new SignalProcessor<>(
PostgresConnector.class, connectorConfig, Map.of(),
getAvailableSignalChannels(),
DocumentReader.defaultReader(),
previousOffsets);
final PostgresEventDispatcher<TableId> dispatcher = new PostgresEventDispatcher<>( final PostgresEventDispatcher<TableId> dispatcher = new PostgresEventDispatcher<>(
connectorConfig, connectorConfig,
topicNamingStrategy, topicNamingStrategy,
@ -197,7 +206,10 @@ public ChangeEventSourceCoordinator<PostgresPartition, PostgresOffsetContext> st
break; break;
} }
}), }),
schemaNameAdjuster); schemaNameAdjuster,
signalProcessor);
dispatcher.getSignalingActions().forEach(signalProcessor::registerSignalAction);
ChangeEventSourceCoordinator<PostgresPartition, PostgresOffsetContext> coordinator = new PostgresChangeEventSourceCoordinator( ChangeEventSourceCoordinator<PostgresPartition, PostgresOffsetContext> coordinator = new PostgresChangeEventSourceCoordinator(
previousOffsets, previousOffsets,
@ -216,11 +228,12 @@ public ChangeEventSourceCoordinator<PostgresPartition, PostgresOffsetContext> st
replicationConnection, replicationConnection,
slotCreatedInfo, slotCreatedInfo,
slotInfo), slotInfo),
new DefaultChangeEventSourceMetricsFactory<PostgresPartition>(), new DefaultChangeEventSourceMetricsFactory<>(),
dispatcher, dispatcher,
schema, schema,
snapshotter, snapshotter,
slotInfo); slotInfo,
signalProcessor);
coordinator.start(taskContext, this.queue, metadataProvider); coordinator.start(taskContext, this.queue, metadataProvider);

View File

@ -16,6 +16,7 @@
import io.debezium.heartbeat.Heartbeat; import io.debezium.heartbeat.Heartbeat;
import io.debezium.pipeline.DataChangeEvent; import io.debezium.pipeline.DataChangeEvent;
import io.debezium.pipeline.EventDispatcher; import io.debezium.pipeline.EventDispatcher;
import io.debezium.pipeline.signal.SignalProcessor;
import io.debezium.pipeline.source.spi.EventMetadataProvider; import io.debezium.pipeline.source.spi.EventMetadataProvider;
import io.debezium.pipeline.spi.ChangeEventCreator; import io.debezium.pipeline.spi.ChangeEventCreator;
import io.debezium.pipeline.spi.OffsetContext; import io.debezium.pipeline.spi.OffsetContext;
@ -40,7 +41,8 @@ public class PostgresEventDispatcher<T extends DataCollectionId> extends EventDi
public PostgresEventDispatcher(PostgresConnectorConfig connectorConfig, TopicNamingStrategy<T> topicNamingStrategy, public PostgresEventDispatcher(PostgresConnectorConfig connectorConfig, TopicNamingStrategy<T> topicNamingStrategy,
DatabaseSchema<T> schema, ChangeEventQueue<DataChangeEvent> queue, DataCollectionFilters.DataCollectionFilter<T> filter, DatabaseSchema<T> schema, ChangeEventQueue<DataChangeEvent> queue, DataCollectionFilters.DataCollectionFilter<T> filter,
ChangeEventCreator changeEventCreator, InconsistentSchemaHandler<PostgresPartition, T> inconsistentSchemaHandler, ChangeEventCreator changeEventCreator, InconsistentSchemaHandler<PostgresPartition, T> inconsistentSchemaHandler,
EventMetadataProvider metadataProvider, Heartbeat heartbeat, SchemaNameAdjuster schemaNameAdjuster) { EventMetadataProvider metadataProvider, Heartbeat heartbeat, SchemaNameAdjuster schemaNameAdjuster,
SignalProcessor<PostgresPartition, PostgresOffsetContext> signalProcessor) {
super(connectorConfig, topicNamingStrategy, schema, queue, filter, changeEventCreator, inconsistentSchemaHandler, heartbeat, schemaNameAdjuster, super(connectorConfig, topicNamingStrategy, schema, queue, filter, changeEventCreator, inconsistentSchemaHandler, heartbeat, schemaNameAdjuster,
new PostgresTransactionMonitor( new PostgresTransactionMonitor(
connectorConfig, connectorConfig,
@ -49,7 +51,8 @@ public PostgresEventDispatcher(PostgresConnectorConfig connectorConfig, TopicNam
(record) -> { (record) -> {
queue.enqueue(new DataChangeEvent(record)); queue.enqueue(new DataChangeEvent(record));
}, },
topicNamingStrategy.transactionTopic())); topicNamingStrategy.transactionTopic()),
signalProcessor);
this.queue = queue; this.queue = queue;
this.logicalDecodingMessageMonitor = new LogicalDecodingMessageMonitor(connectorConfig, this::enqueueLogicalDecodingMessage); this.logicalDecodingMessageMonitor = new LogicalDecodingMessageMonitor(connectorConfig, this::enqueueLogicalDecodingMessage);
this.messageFilter = connectorConfig.getMessageFilter(); this.messageFilter = connectorConfig.getMessageFilter();

View File

@ -79,6 +79,7 @@ public class PostgresStreamingChangeEventSource implements StreamingChangeEventS
*/ */
private long numberOfEventsSinceLastEventSentOrWalGrowingWarning = 0; private long numberOfEventsSinceLastEventSentOrWalGrowingWarning = 0;
private Lsn lastCompletelyProcessedLsn; private Lsn lastCompletelyProcessedLsn;
private PostgresOffsetContext effectiveOffset;
public PostgresStreamingChangeEventSource(PostgresConnectorConfig connectorConfig, Snapshotter snapshotter, public PostgresStreamingChangeEventSource(PostgresConnectorConfig connectorConfig, Snapshotter snapshotter,
PostgresConnection connection, PostgresEventDispatcher<TableId> dispatcher, ErrorHandler errorHandler, Clock clock, PostgresConnection connection, PostgresEventDispatcher<TableId> dispatcher, ErrorHandler errorHandler, Clock clock,
@ -98,8 +99,14 @@ public PostgresStreamingChangeEventSource(PostgresConnectorConfig connectorConfi
} }
@Override @Override
public void init() { public void init(PostgresOffsetContext offsetContext) {
this.effectiveOffset = offsetContext == null ? PostgresOffsetContext.initialContext(connectorConfig, connection, clock) : offsetContext;
// refresh the schema so we have a latest view of the DB tables // refresh the schema so we have a latest view of the DB tables
initSchema();
}
private void initSchema() {
try { try {
taskContext.refreshSchema(connection, true); taskContext.refreshSchema(connection, true);
} }
@ -122,19 +129,16 @@ public void execute(ChangeEventSourceContext context, PostgresPartition partitio
// instead of the last position in the database // instead of the last position in the database
boolean hasStartLsnStoredInContext = offsetContext != null; boolean hasStartLsnStoredInContext = offsetContext != null;
if (!hasStartLsnStoredInContext) {
offsetContext = PostgresOffsetContext.initialContext(connectorConfig, connection, clock);
}
try { try {
final WalPositionLocator walPosition; final WalPositionLocator walPosition;
if (hasStartLsnStoredInContext) { if (hasStartLsnStoredInContext) {
// start streaming from the last recorded position in the offset // start streaming from the last recorded position in the offset
final Lsn lsn = offsetContext.lastCompletelyProcessedLsn() != null ? offsetContext.lastCompletelyProcessedLsn() : offsetContext.lsn(); final Lsn lsn = this.effectiveOffset.lastCompletelyProcessedLsn() != null ? this.effectiveOffset.lastCompletelyProcessedLsn()
final Operation lastProcessedMessageType = offsetContext.lastProcessedMessageType(); : this.effectiveOffset.lsn();
final Operation lastProcessedMessageType = this.effectiveOffset.lastProcessedMessageType();
LOGGER.info("Retrieved latest position from stored offset '{}'", lsn); LOGGER.info("Retrieved latest position from stored offset '{}'", lsn);
walPosition = new WalPositionLocator(offsetContext.lastCommitLsn(), lsn, lastProcessedMessageType); walPosition = new WalPositionLocator(this.effectiveOffset.lastCommitLsn(), lsn, lastProcessedMessageType);
replicationStream.compareAndSet(null, replicationConnection.startStreaming(lsn, walPosition)); replicationStream.compareAndSet(null, replicationConnection.startStreaming(lsn, walPosition));
} }
else { else {
@ -148,11 +152,11 @@ public void execute(ChangeEventSourceContext context, PostgresPartition partitio
ReplicationStream stream = this.replicationStream.get(); ReplicationStream stream = this.replicationStream.get();
stream.startKeepAlive(Threads.newSingleThreadExecutor(PostgresConnector.class, connectorConfig.getLogicalName(), KEEP_ALIVE_THREAD_NAME)); stream.startKeepAlive(Threads.newSingleThreadExecutor(PostgresConnector.class, connectorConfig.getLogicalName(), KEEP_ALIVE_THREAD_NAME));
init(); initSchema();
// If we need to do a pre-snapshot streaming catch up, we should allow the snapshot transaction to persist // If we need to do a pre-snapshot streaming catch up, we should allow the snapshot transaction to persist
// but normally we want to start streaming without any open transactions. // but normally we want to start streaming without any open transactions.
if (!isInPreSnapshotCatchUpStreaming(offsetContext)) { if (!isInPreSnapshotCatchUpStreaming(this.effectiveOffset)) {
connection.commit(); connection.commit();
} }
@ -161,7 +165,7 @@ public void execute(ChangeEventSourceContext context, PostgresPartition partitio
if (walPosition.searchingEnabled()) { if (walPosition.searchingEnabled()) {
searchWalPosition(context, stream, walPosition); searchWalPosition(context, stream, walPosition);
try { try {
if (!isInPreSnapshotCatchUpStreaming(offsetContext)) { if (!isInPreSnapshotCatchUpStreaming(this.effectiveOffset)) {
connection.commit(); connection.commit();
} }
} }
@ -175,7 +179,7 @@ public void execute(ChangeEventSourceContext context, PostgresPartition partitio
stream = this.replicationStream.get(); stream = this.replicationStream.get();
stream.startKeepAlive(Threads.newSingleThreadExecutor(PostgresConnector.class, connectorConfig.getLogicalName(), KEEP_ALIVE_THREAD_NAME)); stream.startKeepAlive(Threads.newSingleThreadExecutor(PostgresConnector.class, connectorConfig.getLogicalName(), KEEP_ALIVE_THREAD_NAME));
} }
processMessages(context, partition, offsetContext, stream); processMessages(context, partition, this.effectiveOffset, stream);
} }
catch (Throwable e) { catch (Throwable e) {
errorHandler.setProducerThrowable(e); errorHandler.setProducerThrowable(e);
@ -428,6 +432,11 @@ public void commitOffset(Map<String, ?> partition, Map<String, ?> offset) {
} }
} }
@Override
public PostgresOffsetContext getOffsetContext() {
return effectiveOffset;
}
/** /**
* Returns whether the current streaming phase is running a catch up streaming * Returns whether the current streaming phase is running a catch up streaming
* phase that runs before a snapshot. This is useful for transaction * phase that runs before a snapshot. This is useful for transaction

View File

@ -10,6 +10,7 @@
import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThat;
import static org.assertj.core.api.Assertions.entry; import static org.assertj.core.api.Assertions.entry;
import java.io.File;
import java.sql.SQLException; import java.sql.SQLException;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -17,18 +18,25 @@
import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.data.Struct;
import org.junit.After; import org.junit.After;
import org.junit.AfterClass;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Before; import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test; import org.junit.Test;
import io.debezium.config.CommonConnectorConfig;
import io.debezium.config.Configuration; import io.debezium.config.Configuration;
import io.debezium.connector.postgresql.PostgresConnectorConfig.SnapshotMode; import io.debezium.connector.postgresql.PostgresConnectorConfig.SnapshotMode;
import io.debezium.data.VariableScaleDecimal; import io.debezium.data.VariableScaleDecimal;
import io.debezium.doc.FixFor; import io.debezium.doc.FixFor;
import io.debezium.jdbc.JdbcConnection; import io.debezium.jdbc.JdbcConnection;
import io.debezium.junit.SkipWhenDatabaseVersion; import io.debezium.junit.SkipWhenDatabaseVersion;
import io.debezium.kafka.KafkaCluster;
import io.debezium.pipeline.signal.channels.KafkaSignalChannel;
import io.debezium.pipeline.source.snapshot.incremental.AbstractIncrementalSnapshotTest; import io.debezium.pipeline.source.snapshot.incremental.AbstractIncrementalSnapshotTest;
import io.debezium.relational.RelationalDatabaseConnectorConfig; import io.debezium.relational.RelationalDatabaseConnectorConfig;
import io.debezium.util.Collect;
import io.debezium.util.Testing;
public class IncrementalSnapshotIT extends AbstractIncrementalSnapshotTest<PostgresConnector> { public class IncrementalSnapshotIT extends AbstractIncrementalSnapshotTest<PostgresConnector> {
@ -51,11 +59,35 @@ public void before() throws SQLException {
TestHelper.execute(SETUP_TABLES_STMT); TestHelper.execute(SETUP_TABLES_STMT);
} }
@BeforeClass
public static void startKafka() throws Exception {
File dataDir = Testing.Files.createTestingDirectory("signal_cluster");
Testing.Files.delete(dataDir);
kafka = new KafkaCluster().usingDirectory(dataDir)
.deleteDataPriorToStartup(true)
.deleteDataUponShutdown(true)
.addBrokers(1)
.withKafkaConfiguration(Collect.propertiesOf(
"auto.create.topics.enable", "false",
"zookeeper.session.timeout.ms", "20000"))
.startup();
kafka.createTopic("signals_topic", 1, 1);
}
@AfterClass
public static void stopKafka() {
if (kafka != null) {
kafka.shutdown();
}
}
@After @After
public void after() { public void after() {
stopConnector(); stopConnector();
TestHelper.dropDefaultReplicationSlot(); TestHelper.dropDefaultReplicationSlot();
TestHelper.dropPublication(); TestHelper.dropPublication();
} }
protected Configuration.Builder config() { protected Configuration.Builder config() {
@ -65,6 +97,10 @@ protected Configuration.Builder config() {
.with(PostgresConnectorConfig.SIGNAL_DATA_COLLECTION, "s1.debezium_signal") .with(PostgresConnectorConfig.SIGNAL_DATA_COLLECTION, "s1.debezium_signal")
.with(PostgresConnectorConfig.INCREMENTAL_SNAPSHOT_CHUNK_SIZE, 10) .with(PostgresConnectorConfig.INCREMENTAL_SNAPSHOT_CHUNK_SIZE, 10)
.with(PostgresConnectorConfig.SCHEMA_INCLUDE_LIST, "s1") .with(PostgresConnectorConfig.SCHEMA_INCLUDE_LIST, "s1")
.with(CommonConnectorConfig.SIGNAL_ENABLED_CHANNELS, "source,kafka")
.with(KafkaSignalChannel.SIGNAL_TOPIC, getSignalsTopic())
.with(KafkaSignalChannel.BOOTSTRAP_SERVERS, kafka.brokerList())
.with(CommonConnectorConfig.SIGNAL_POLL_INTERVAL_MS, 5)
.with(RelationalDatabaseConnectorConfig.MSG_KEY_COLUMNS, "s1.a42:pk1,pk2,pk3,pk4") .with(RelationalDatabaseConnectorConfig.MSG_KEY_COLUMNS, "s1.a42:pk1,pk2,pk3,pk4")
// DBZ-4272 required to allow dropping columns just before an incremental snapshot // DBZ-4272 required to allow dropping columns just before an incremental snapshot
.with("database.autosave", "conservative"); .with("database.autosave", "conservative");
@ -83,6 +119,7 @@ protected Configuration.Builder mutableConfig(boolean signalTableOnly, boolean s
.with(PostgresConnectorConfig.SNAPSHOT_MODE, SnapshotMode.NEVER.getValue()) .with(PostgresConnectorConfig.SNAPSHOT_MODE, SnapshotMode.NEVER.getValue())
.with(PostgresConnectorConfig.DROP_SLOT_ON_STOP, Boolean.FALSE) .with(PostgresConnectorConfig.DROP_SLOT_ON_STOP, Boolean.FALSE)
.with(PostgresConnectorConfig.SIGNAL_DATA_COLLECTION, "s1.debezium_signal") .with(PostgresConnectorConfig.SIGNAL_DATA_COLLECTION, "s1.debezium_signal")
.with(CommonConnectorConfig.SIGNAL_POLL_INTERVAL_MS, 5)
.with(PostgresConnectorConfig.INCREMENTAL_SNAPSHOT_CHUNK_SIZE, 10) .with(PostgresConnectorConfig.INCREMENTAL_SNAPSHOT_CHUNK_SIZE, 10)
.with(PostgresConnectorConfig.SCHEMA_INCLUDE_LIST, "s1") .with(PostgresConnectorConfig.SCHEMA_INCLUDE_LIST, "s1")
.with(RelationalDatabaseConnectorConfig.MSG_KEY_COLUMNS, "s1.a42:pk1,pk2,pk3,pk4") .with(RelationalDatabaseConnectorConfig.MSG_KEY_COLUMNS, "s1.a42:pk1,pk2,pk3,pk4")
@ -174,6 +211,48 @@ record -> ((Struct) record.value()).getStruct("after").getInt32(valueFieldName()
} }
} }
@Test
public void inserts4PksWithKafkaSignal() throws Exception {
// Testing.Print.enable();
populate4PkTable();
startConnector();
sendExecuteSnapshotKafkaSignal("s1.a4");
Thread.sleep(5000);
try (JdbcConnection connection = databaseConnection()) {
connection.setAutoCommit(false);
for (int i = 0; i < ROW_COUNT; i++) {
final int id = i + ROW_COUNT + 1;
final int pk1 = id / 1000;
final int pk2 = (id / 100) % 10;
final int pk3 = (id / 10) % 10;
final int pk4 = id % 10;
connection.executeWithoutCommitting(String.format("INSERT INTO %s (pk1, pk2, pk3, pk4, aa) VALUES (%s, %s, %s, %s, %s)",
"s1.a4",
pk1,
pk2,
pk3,
pk4,
i + ROW_COUNT));
}
connection.commit();
}
final int expectedRecordCount = ROW_COUNT * 2;
final Map<Integer, Integer> dbChanges = consumeMixedWithIncrementalSnapshot(
expectedRecordCount,
x -> true,
k -> k.getInt32("pk1") * 1_000 + k.getInt32("pk2") * 100 + k.getInt32("pk3") * 10 + k.getInt32("pk4"),
record -> ((Struct) record.value()).getStruct("after").getInt32(valueFieldName()),
"test_server.s1.a4",
null);
for (int i = 0; i < expectedRecordCount; i++) {
assertThat(dbChanges).contains(entry(i + 1, i));
}
}
@Test @Test
public void insertsWithoutPks() throws Exception { public void insertsWithoutPks() throws Exception {
// Testing.Print.enable(); // Testing.Print.enable();

View File

@ -13,15 +13,17 @@
import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.data.Struct;
import org.apache.kafka.connect.source.SourceRecord; import org.apache.kafka.connect.source.SourceRecord;
import org.awaitility.Awaitility;
import org.junit.After; import org.junit.After;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import io.debezium.config.CommonConnectorConfig;
import io.debezium.config.Configuration; import io.debezium.config.Configuration;
import io.debezium.connector.postgresql.PostgresConnectorConfig.SnapshotMode; import io.debezium.connector.postgresql.PostgresConnectorConfig.SnapshotMode;
import io.debezium.embedded.AbstractConnectorTest; import io.debezium.embedded.AbstractConnectorTest;
import io.debezium.junit.logging.LogInterceptor; import io.debezium.junit.logging.LogInterceptor;
import io.debezium.pipeline.signal.Signal; import io.debezium.pipeline.signal.actions.Log;
public class SignalsIT extends AbstractConnectorTest { public class SignalsIT extends AbstractConnectorTest {
@ -49,7 +51,7 @@ public void after() {
@Test @Test
public void signalLog() throws InterruptedException { public void signalLog() throws InterruptedException {
// Testing.Print.enable(); // Testing.Print.enable();
final LogInterceptor logInterceptor = new LogInterceptor(Signal.class); final LogInterceptor logInterceptor = new LogInterceptor(Log.class);
TestHelper.dropDefaultReplicationSlot(); TestHelper.dropDefaultReplicationSlot();
TestHelper.execute(SETUP_TABLES_STMT); TestHelper.execute(SETUP_TABLES_STMT);
@ -57,6 +59,7 @@ public void signalLog() throws InterruptedException {
.with(PostgresConnectorConfig.SNAPSHOT_MODE, SnapshotMode.NEVER.getValue()) .with(PostgresConnectorConfig.SNAPSHOT_MODE, SnapshotMode.NEVER.getValue())
.with(PostgresConnectorConfig.DROP_SLOT_ON_STOP, Boolean.TRUE) .with(PostgresConnectorConfig.DROP_SLOT_ON_STOP, Boolean.TRUE)
.with(PostgresConnectorConfig.SIGNAL_DATA_COLLECTION, "s1.debezium_signal") .with(PostgresConnectorConfig.SIGNAL_DATA_COLLECTION, "s1.debezium_signal")
.with(CommonConnectorConfig.SIGNAL_POLL_INTERVAL_MS, "500")
.build(); .build();
start(PostgresConnector.class, config); start(PostgresConnector.class, config);
assertConnectorIsRunning(); assertConnectorIsRunning();
@ -72,21 +75,26 @@ public void signalLog() throws InterruptedException {
// Insert the signal record // Insert the signal record
TestHelper.execute("INSERT INTO s1.debezium_signal VALUES('1', 'log', '{\"message\": \"Signal message at offset ''{}''\"}')"); TestHelper.execute("INSERT INTO s1.debezium_signal VALUES('1', 'log', '{\"message\": \"Signal message at offset ''{}''\"}')");
waitForAvailableRecords(800, TimeUnit.MILLISECONDS);
final SourceRecords records = consumeRecordsByTopic(2); final SourceRecords records = consumeRecordsByTopic(2);
assertThat(records.allRecordsInOrder()).hasSize(2); assertThat(records.allRecordsInOrder()).hasSize(2);
assertThat(logInterceptor.containsMessage("Received signal")).isTrue(); assertThat(logInterceptor.containsMessage("Signal message at offset")).isTrue();
} }
@Test @Test
public void signalingDisabled() throws InterruptedException { public void signalingDisabled() throws InterruptedException {
// Testing.Print.enable(); // Testing.Print.enable();
final LogInterceptor logInterceptor = new LogInterceptor(Signal.class); final LogInterceptor logInterceptor = new LogInterceptor(Log.class);
TestHelper.dropDefaultReplicationSlot(); TestHelper.dropDefaultReplicationSlot();
TestHelper.execute(SETUP_TABLES_STMT); TestHelper.execute(SETUP_TABLES_STMT);
Configuration config = TestHelper.defaultConfig() Configuration config = TestHelper.defaultConfig()
.with(PostgresConnectorConfig.SNAPSHOT_MODE, SnapshotMode.NEVER.getValue()) .with(PostgresConnectorConfig.SNAPSHOT_MODE, SnapshotMode.NEVER.getValue())
.with(PostgresConnectorConfig.DROP_SLOT_ON_STOP, Boolean.TRUE) .with(PostgresConnectorConfig.DROP_SLOT_ON_STOP, Boolean.TRUE)
.with(PostgresConnectorConfig.SIGNAL_DATA_COLLECTION, "s1.debezium_signal")
.with(CommonConnectorConfig.SIGNAL_POLL_INTERVAL_MS, "500")
.with(CommonConnectorConfig.SIGNAL_ENABLED_CHANNELS, "")
.build(); .build();
start(PostgresConnector.class, config); start(PostgresConnector.class, config);
assertConnectorIsRunning(); assertConnectorIsRunning();
@ -99,12 +107,14 @@ public void signalingDisabled() throws InterruptedException {
// Insert the signal record // Insert the signal record
TestHelper.execute("INSERT INTO s1.debezium_signal VALUES('1', 'log', '{\"message\": \"Signal message\"}')"); TestHelper.execute("INSERT INTO s1.debezium_signal VALUES('1', 'log', '{\"message\": \"Signal message\"}')");
Awaitility.await().pollDelay(2000, TimeUnit.MILLISECONDS).until(() -> true);
// insert and verify a new record // insert and verify a new record
TestHelper.execute(INSERT_STMT); TestHelper.execute(INSERT_STMT);
final SourceRecords records = consumeRecordsByTopic(2); final SourceRecords records = consumeRecordsByTopic(2);
assertThat(records.allRecordsInOrder()).hasSize(2); assertThat(records.allRecordsInOrder()).hasSize(2);
assertThat(logInterceptor.containsMessage("Received signal")).isFalse(); assertThat(logInterceptor.containsMessage("Signal message")).isFalse();
} }
@Test @Test
@ -117,6 +127,7 @@ public void signalSchemaChange() throws InterruptedException {
.with(PostgresConnectorConfig.SNAPSHOT_MODE, SnapshotMode.NEVER.getValue()) .with(PostgresConnectorConfig.SNAPSHOT_MODE, SnapshotMode.NEVER.getValue())
.with(PostgresConnectorConfig.DROP_SLOT_ON_STOP, Boolean.TRUE) .with(PostgresConnectorConfig.DROP_SLOT_ON_STOP, Boolean.TRUE)
.with(PostgresConnectorConfig.SIGNAL_DATA_COLLECTION, "s1.debezium_signal") .with(PostgresConnectorConfig.SIGNAL_DATA_COLLECTION, "s1.debezium_signal")
.with(CommonConnectorConfig.SIGNAL_POLL_INTERVAL_MS, "500")
.build(); .build();
start(PostgresConnector.class, config); start(PostgresConnector.class, config);
assertConnectorIsRunning(); assertConnectorIsRunning();
@ -166,6 +177,8 @@ public void signalSchemaChange() throws InterruptedException {
+ " }\n" + " }\n"
+ "}]}')"); + "}]}')");
Awaitility.await().pollDelay(2000, TimeUnit.MILLISECONDS).until(() -> true);
TestHelper.execute(INSERT_STMT); TestHelper.execute(INSERT_STMT);
final SourceRecords records = consumeRecordsByTopic(3); final SourceRecords records = consumeRecordsByTopic(3);

View File

@ -41,7 +41,7 @@
</logger> </logger>
<!-- Needed for SignalsIT --> <!-- Needed for SignalsIT -->
<logger <logger
name="io.debezium.pipeline.signal.Signal" name="io.debezium.pipeline.signal.channels.SourceSignalChannel"
level="debug" additivity="false"> level="debug" additivity="false">
<appender-ref ref="CONSOLE" /> <appender-ref ref="CONSOLE" />
</logger> </logger>

View File

@ -21,6 +21,7 @@
import io.debezium.pipeline.ErrorHandler; import io.debezium.pipeline.ErrorHandler;
import io.debezium.pipeline.EventDispatcher; import io.debezium.pipeline.EventDispatcher;
import io.debezium.pipeline.metrics.spi.ChangeEventSourceMetricsFactory; import io.debezium.pipeline.metrics.spi.ChangeEventSourceMetricsFactory;
import io.debezium.pipeline.signal.SignalProcessor;
import io.debezium.pipeline.source.spi.ChangeEventSource; import io.debezium.pipeline.source.spi.ChangeEventSource;
import io.debezium.pipeline.source.spi.ChangeEventSource.ChangeEventSourceContext; import io.debezium.pipeline.source.spi.ChangeEventSource.ChangeEventSourceContext;
import io.debezium.pipeline.source.spi.ChangeEventSourceFactory; import io.debezium.pipeline.source.spi.ChangeEventSourceFactory;
@ -51,9 +52,10 @@ public SqlServerChangeEventSourceCoordinator(Offsets<SqlServerPartition, SqlServ
ChangeEventSourceMetricsFactory<SqlServerPartition> changeEventSourceMetricsFactory, ChangeEventSourceMetricsFactory<SqlServerPartition> changeEventSourceMetricsFactory,
EventDispatcher<SqlServerPartition, ?> eventDispatcher, EventDispatcher<SqlServerPartition, ?> eventDispatcher,
DatabaseSchema<?> schema, DatabaseSchema<?> schema,
Clock clock) { Clock clock,
SignalProcessor<SqlServerPartition, SqlServerOffsetContext> signalProcessor) {
super(previousOffsets, errorHandler, connectorType, connectorConfig, changeEventSourceFactory, super(previousOffsets, errorHandler, connectorType, connectorConfig, changeEventSourceFactory,
changeEventSourceMetricsFactory, eventDispatcher, schema); changeEventSourceMetricsFactory, eventDispatcher, schema, signalProcessor);
this.clock = clock; this.clock = clock;
this.pollInterval = connectorConfig.getPollInterval(); this.pollInterval = connectorConfig.getPollInterval();
} }
@ -79,6 +81,9 @@ protected void executeChangeEventSources(CdcSourceTaskContext taskContext, Snaps
if (snapshotResult.isCompletedOrSkipped()) { if (snapshotResult.isCompletedOrSkipped()) {
streamingOffsets.getOffsets().put(partition, snapshotResult.getOffset()); streamingOffsets.getOffsets().put(partition, snapshotResult.getOffset());
if (previousOffsets.getOffsets().size() == 1) {
signalProcessor.setContext(snapshotResult.getOffset());
}
} }
} }

View File

@ -7,6 +7,7 @@
import java.sql.SQLException; import java.sql.SQLException;
import java.util.List; import java.util.List;
import java.util.Map;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import org.apache.kafka.connect.source.SourceRecord; import org.apache.kafka.connect.source.SourceRecord;
@ -19,11 +20,13 @@
import io.debezium.connector.base.ChangeEventQueue; import io.debezium.connector.base.ChangeEventQueue;
import io.debezium.connector.common.BaseSourceTask; import io.debezium.connector.common.BaseSourceTask;
import io.debezium.connector.sqlserver.metrics.SqlServerMetricsFactory; import io.debezium.connector.sqlserver.metrics.SqlServerMetricsFactory;
import io.debezium.document.DocumentReader;
import io.debezium.jdbc.DefaultMainConnectionProvidingConnectionFactory; import io.debezium.jdbc.DefaultMainConnectionProvidingConnectionFactory;
import io.debezium.jdbc.MainConnectionProvidingConnectionFactory; import io.debezium.jdbc.MainConnectionProvidingConnectionFactory;
import io.debezium.pipeline.ChangeEventSourceCoordinator; import io.debezium.pipeline.ChangeEventSourceCoordinator;
import io.debezium.pipeline.DataChangeEvent; import io.debezium.pipeline.DataChangeEvent;
import io.debezium.pipeline.EventDispatcher; import io.debezium.pipeline.EventDispatcher;
import io.debezium.pipeline.signal.SignalProcessor;
import io.debezium.pipeline.spi.Offsets; import io.debezium.pipeline.spi.Offsets;
import io.debezium.relational.TableId; import io.debezium.relational.TableId;
import io.debezium.schema.SchemaNameAdjuster; import io.debezium.schema.SchemaNameAdjuster;
@ -107,6 +110,12 @@ public ChangeEventSourceCoordinator<SqlServerPartition, SqlServerOffsetContext>
final SqlServerEventMetadataProvider metadataProvider = new SqlServerEventMetadataProvider(); final SqlServerEventMetadataProvider metadataProvider = new SqlServerEventMetadataProvider();
SignalProcessor<SqlServerPartition, SqlServerOffsetContext> signalProcessor = new SignalProcessor<>(
SqlServerConnector.class, connectorConfig, Map.of(),
getAvailableSignalChannels(),
DocumentReader.defaultReader(),
offsets);
final EventDispatcher<SqlServerPartition, TableId> dispatcher = new EventDispatcher<>( final EventDispatcher<SqlServerPartition, TableId> dispatcher = new EventDispatcher<>(
connectorConfig, connectorConfig,
topicNamingStrategy, topicNamingStrategy,
@ -115,7 +124,10 @@ public ChangeEventSourceCoordinator<SqlServerPartition, SqlServerOffsetContext>
connectorConfig.getTableFilters().dataCollectionFilter(), connectorConfig.getTableFilters().dataCollectionFilter(),
DataChangeEvent::new, DataChangeEvent::new,
metadataProvider, metadataProvider,
schemaNameAdjuster); schemaNameAdjuster,
signalProcessor);
dispatcher.getSignalingActions().forEach(signalProcessor::registerSignalAction);
ChangeEventSourceCoordinator<SqlServerPartition, SqlServerOffsetContext> coordinator = new SqlServerChangeEventSourceCoordinator( ChangeEventSourceCoordinator<SqlServerPartition, SqlServerOffsetContext> coordinator = new SqlServerChangeEventSourceCoordinator(
offsets, offsets,
@ -126,7 +138,8 @@ public ChangeEventSourceCoordinator<SqlServerPartition, SqlServerOffsetContext>
new SqlServerMetricsFactory(offsets.getPartitions()), new SqlServerMetricsFactory(offsets.getPartitions()),
dispatcher, dispatcher,
schema, schema,
clock); clock,
signalProcessor);
coordinator.start(taskContext, this.queue, metadataProvider); coordinator.start(taskContext, this.queue, metadataProvider);

View File

@ -89,6 +89,7 @@ public class SqlServerStreamingChangeEventSource implements StreamingChangeEvent
private final Map<SqlServerPartition, SqlServerStreamingExecutionContext> streamingExecutionContexts; private final Map<SqlServerPartition, SqlServerStreamingExecutionContext> streamingExecutionContexts;
private boolean checkAgent; private boolean checkAgent;
private SqlServerOffsetContext effectiveOffset;
public SqlServerStreamingChangeEventSource(SqlServerConnectorConfig connectorConfig, SqlServerConnection dataConnection, public SqlServerStreamingChangeEventSource(SqlServerConnectorConfig connectorConfig, SqlServerConnection dataConnection,
SqlServerConnection metadataConnection, SqlServerConnection metadataConnection,
@ -112,6 +113,11 @@ public SqlServerStreamingChangeEventSource(SqlServerConnectorConfig connectorCon
this.checkAgent = true; this.checkAgent = true;
} }
@Override
public void init(SqlServerOffsetContext offsetContext) throws InterruptedException {
this.effectiveOffset = offsetContext == null ? new SqlServerOffsetContext(connectorConfig, TxLogPosition.NULL, false, false) : offsetContext;
}
@Override @Override
public void execute(ChangeEventSourceContext context, SqlServerPartition partition, SqlServerOffsetContext offsetContext) throws InterruptedException { public void execute(ChangeEventSourceContext context, SqlServerPartition partition, SqlServerOffsetContext offsetContext) throws InterruptedException {
throw new UnsupportedOperationException("Currently unsupported by the SQL Server connector"); throw new UnsupportedOperationException("Currently unsupported by the SQL Server connector");
@ -127,6 +133,8 @@ public boolean executeIteration(ChangeEventSourceContext context, SqlServerParti
final String databaseName = partition.getDatabaseName(); final String databaseName = partition.getDatabaseName();
this.effectiveOffset = offsetContext;
try { try {
final SqlServerStreamingExecutionContext streamingExecutionContext = streamingExecutionContexts.getOrDefault(partition, final SqlServerStreamingExecutionContext streamingExecutionContext = streamingExecutionContexts.getOrDefault(partition,
new SqlServerStreamingExecutionContext( new SqlServerStreamingExecutionContext(
@ -337,6 +345,11 @@ else if (!checkAgent) {
return true; return true;
} }
@Override
public SqlServerOffsetContext getOffsetContext() {
return effectiveOffset;
}
private void commitTransaction() throws SQLException { private void commitTransaction() throws SQLException {
// When reading from read-only Always On replica the default and only transaction isolation // When reading from read-only Always On replica the default and only transaction isolation
// is snapshot. This means that CDC metadata are not visible for long-running transactions. // is snapshot. This means that CDC metadata are not visible for long-running transactions.

View File

@ -123,6 +123,11 @@
<artifactId>apicurio-registry-utils-converter</artifactId> <artifactId>apicurio-registry-utils-converter</artifactId>
<scope>test</scope> <scope>test</scope>
</dependency> </dependency>
<dependency>
<groupId>org.awaitility</groupId>
<artifactId>awaitility</artifactId>
<scope>test</scope>
</dependency>
</dependencies> </dependencies>
<build> <build>
<resources> <resources>
@ -132,6 +137,7 @@
<directory>src/main/resources</directory> <directory>src/main/resources</directory>
<includes> <includes>
<include>**/build.properties</include> <include>**/build.properties</include>
<include>**/*</include>
</includes> </includes>
</resource> </resource>
</resources> </resources>

View File

@ -11,6 +11,7 @@
import java.util.Collections; import java.util.Collections;
import java.util.EnumSet; import java.util.EnumSet;
import java.util.List; import java.util.List;
import java.util.Objects;
import java.util.Optional; import java.util.Optional;
import java.util.Properties; import java.util.Properties;
import java.util.Set; import java.util.Set;
@ -617,9 +618,28 @@ public static FieldNameAdjustmentMode parse(String value) {
.withImportance(Importance.MEDIUM) .withImportance(Importance.MEDIUM)
.withDescription("The name of the data collection that is used to send signals/commands to Debezium. Signaling is disabled when not set."); .withDescription("The name of the data collection that is used to send signals/commands to Debezium. Signaling is disabled when not set.");
public static final Field SIGNAL_POLL_INTERVAL_MS = Field.create("signal.poll.interval.ms")
.withDisplayName("Signal processor poll interval")
.withGroup(Field.createGroupEntry(Field.Group.ADVANCED, 21))
.withType(Type.LONG)
.withWidth(Width.SHORT)
.withImportance(Importance.MEDIUM)
.withDefault(5L)
.withValidation(Field::isPositiveInteger)
.withDescription("Interval for looking for new signals in registered channels, given in milliseconds. Defaults to 30 seconds (30,000 ms).");
public static final Field SIGNAL_ENABLED_CHANNELS = Field.create("signal.enabled.channels")
.withDisplayName("Enabled channels names")
.withGroup(Field.createGroupEntry(Field.Group.ADVANCED, 22))
.withType(Type.LIST)
.withWidth(Width.LONG)
.withImportance(Importance.MEDIUM)
.withDefault("source")
.withDescription("List of channels names that are enabled. Source channel is enabled by default");
public static final Field TOPIC_NAMING_STRATEGY = Field.create("topic.naming.strategy") public static final Field TOPIC_NAMING_STRATEGY = Field.create("topic.naming.strategy")
.withDisplayName("Topic naming strategy class") .withDisplayName("Topic naming strategy class")
.withGroup(Field.createGroupEntry(Field.Group.ADVANCED, 21)) .withGroup(Field.createGroupEntry(Field.Group.ADVANCED, 23))
.withType(Type.CLASS) .withType(Type.CLASS)
.withWidth(Width.MEDIUM) .withWidth(Width.MEDIUM)
.withImportance(Importance.MEDIUM) .withImportance(Importance.MEDIUM)
@ -659,6 +679,8 @@ public static FieldNameAdjustmentMode parse(String value) {
Heartbeat.HEARTBEAT_INTERVAL, Heartbeat.HEARTBEAT_INTERVAL,
Heartbeat.HEARTBEAT_TOPICS_PREFIX, Heartbeat.HEARTBEAT_TOPICS_PREFIX,
SIGNAL_DATA_COLLECTION, SIGNAL_DATA_COLLECTION,
SIGNAL_POLL_INTERVAL_MS,
SIGNAL_ENABLED_CHANNELS,
TOPIC_NAMING_STRATEGY) TOPIC_NAMING_STRATEGY)
.create(); .create();
@ -686,6 +708,10 @@ public static FieldNameAdjustmentMode parse(String value) {
private final SchemaNameAdjustmentMode schemaNameAdjustmentMode; private final SchemaNameAdjustmentMode schemaNameAdjustmentMode;
private final FieldNameAdjustmentMode fieldNameAdjustmentMode; private final FieldNameAdjustmentMode fieldNameAdjustmentMode;
private final String signalingDataCollection; private final String signalingDataCollection;
private final Duration signalPollInterval;
private final List<String> signalEnabledChannels;
private final EnumSet<Operation> skippedOperations; private final EnumSet<Operation> skippedOperations;
private final String taskId; private final String taskId;
@ -714,10 +740,22 @@ protected CommonConnectorConfig(Configuration config, int defaultSnapshotFetchSi
this.customConverterRegistry = new CustomConverterRegistry(getCustomConverters()); this.customConverterRegistry = new CustomConverterRegistry(getCustomConverters());
this.binaryHandlingMode = BinaryHandlingMode.parse(config.getString(BINARY_HANDLING_MODE)); this.binaryHandlingMode = BinaryHandlingMode.parse(config.getString(BINARY_HANDLING_MODE));
this.signalingDataCollection = config.getString(SIGNAL_DATA_COLLECTION); this.signalingDataCollection = config.getString(SIGNAL_DATA_COLLECTION);
this.signalPollInterval = Duration.ofMillis(config.getLong(SIGNAL_POLL_INTERVAL_MS));
this.signalEnabledChannels = getSignalEnabledChannels(config);
this.skippedOperations = determineSkippedOperations(config); this.skippedOperations = determineSkippedOperations(config);
this.taskId = config.getString(TASK_ID); this.taskId = config.getString(TASK_ID);
} }
private static List<String> getSignalEnabledChannels(Configuration config) {
if (config.hasKey(SIGNAL_ENABLED_CHANNELS)) {
return config.getList(SIGNAL_ENABLED_CHANNELS);
}
return Arrays.stream(Objects.requireNonNull(SIGNAL_ENABLED_CHANNELS.defaultValueAsString()).split(","))
.map(String::trim)
.collect(Collectors.toList());
}
private static EnumSet<Envelope.Operation> determineSkippedOperations(Configuration config) { private static EnumSet<Envelope.Operation> determineSkippedOperations(Configuration config) {
String operations = config.getString(SKIPPED_OPERATIONS); String operations = config.getString(SKIPPED_OPERATIONS);
@ -1000,6 +1038,14 @@ public String getSignalingDataCollectionId() {
return signalingDataCollection; return signalingDataCollection;
} }
public Duration getSignalPollInterval() {
return signalPollInterval;
}
public List<String> getEnabledChannels() {
return signalEnabledChannels;
}
public Optional<String[]> parseSignallingMessage(Struct value) { public Optional<String[]> parseSignallingMessage(Struct value) {
final Struct after = value.getStruct(Envelope.FieldName.AFTER); final Struct after = value.getStruct(Envelope.FieldName.AFTER);
if (after == null) { if (after == null) {

View File

@ -14,10 +14,12 @@
import java.util.LinkedHashMap; import java.util.LinkedHashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.ServiceLoader;
import java.util.Set; import java.util.Set;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.atomic.AtomicReference;
import java.util.concurrent.locks.ReentrantLock; import java.util.concurrent.locks.ReentrantLock;
import java.util.stream.Collectors;
import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.errors.ConnectException;
import org.apache.kafka.connect.errors.RetriableException; import org.apache.kafka.connect.errors.RetriableException;
@ -32,6 +34,7 @@
import io.debezium.config.Configuration; import io.debezium.config.Configuration;
import io.debezium.config.Field; import io.debezium.config.Field;
import io.debezium.pipeline.ChangeEventSourceCoordinator; import io.debezium.pipeline.ChangeEventSourceCoordinator;
import io.debezium.pipeline.signal.channels.SignalChannelReader;
import io.debezium.pipeline.spi.OffsetContext; import io.debezium.pipeline.spi.OffsetContext;
import io.debezium.pipeline.spi.Offsets; import io.debezium.pipeline.spi.Offsets;
import io.debezium.pipeline.spi.Partition; import io.debezium.pipeline.spi.Partition;
@ -57,7 +60,7 @@ public enum State {
RESTARTING, RESTARTING,
RUNNING, RUNNING,
INITIAL, INITIAL,
STOPPED STOPPED;
} }
private final AtomicReference<State> state = new AtomicReference<>(State.INITIAL); private final AtomicReference<State> state = new AtomicReference<>(State.INITIAL);
@ -85,14 +88,16 @@ public enum State {
private Duration retriableRestartWait; private Duration retriableRestartWait;
private final ElapsedTimeStrategy pollOutputDelay; private final ElapsedTimeStrategy pollOutputDelay;
private final Clock clock = Clock.system();
private final Clock clock = Clock.system();
@SingleThreadAccess("polling thread") @SingleThreadAccess("polling thread")
private Instant previousOutputInstant; private Instant previousOutputInstant;
@SingleThreadAccess("polling thread") @SingleThreadAccess("polling thread")
private int previousOutputBatchSize; private int previousOutputBatchSize;
private final ServiceLoader<SignalChannelReader> availableSignalChannels = ServiceLoader.load(SignalChannelReader.class);
protected BaseSourceTask() { protected BaseSourceTask() {
// Use exponential delay to log the progress frequently at first, but the quickly tapering off to once an hour... // Use exponential delay to log the progress frequently at first, but the quickly tapering off to once an hour...
pollOutputDelay = ElapsedTimeStrategy.exponential(clock, INITIAL_POLL_PERIOD_IN_MILLIS, MAX_POLL_PERIOD_IN_MILLIS); pollOutputDelay = ElapsedTimeStrategy.exponential(clock, INITIAL_POLL_PERIOD_IN_MILLIS, MAX_POLL_PERIOD_IN_MILLIS);
@ -132,6 +137,10 @@ public final void start(Map<String, String> props) {
} }
} }
public List<SignalChannelReader> getAvailableSignalChannels() {
return availableSignalChannels.stream().map(ServiceLoader.Provider::get).collect(Collectors.toList());
}
protected Configuration withMaskedSensitiveOptions(Configuration config) { protected Configuration withMaskedSensitiveOptions(Configuration config) {
return config.withMaskedPasswords(); return config.withMaskedPasswords();
} }

View File

@ -24,6 +24,7 @@
import io.debezium.pipeline.metrics.SnapshotChangeEventSourceMetrics; import io.debezium.pipeline.metrics.SnapshotChangeEventSourceMetrics;
import io.debezium.pipeline.metrics.StreamingChangeEventSourceMetrics; import io.debezium.pipeline.metrics.StreamingChangeEventSourceMetrics;
import io.debezium.pipeline.metrics.spi.ChangeEventSourceMetricsFactory; import io.debezium.pipeline.metrics.spi.ChangeEventSourceMetricsFactory;
import io.debezium.pipeline.signal.SignalProcessor;
import io.debezium.pipeline.source.snapshot.incremental.IncrementalSnapshotChangeEventSource; import io.debezium.pipeline.source.snapshot.incremental.IncrementalSnapshotChangeEventSource;
import io.debezium.pipeline.source.spi.ChangeEventSource; import io.debezium.pipeline.source.spi.ChangeEventSource;
import io.debezium.pipeline.source.spi.ChangeEventSource.ChangeEventSourceContext; import io.debezium.pipeline.source.spi.ChangeEventSource.ChangeEventSourceContext;
@ -63,6 +64,7 @@ public class ChangeEventSourceCoordinator<P extends Partition, O extends OffsetC
protected final ExecutorService executor; protected final ExecutorService executor;
protected final EventDispatcher<P, ?> eventDispatcher; protected final EventDispatcher<P, ?> eventDispatcher;
protected final DatabaseSchema<?> schema; protected final DatabaseSchema<?> schema;
protected final SignalProcessor<P, O> signalProcessor;
private volatile boolean running; private volatile boolean running;
protected volatile StreamingChangeEventSource<P, O> streamingSource; protected volatile StreamingChangeEventSource<P, O> streamingSource;
@ -75,7 +77,8 @@ public ChangeEventSourceCoordinator(Offsets<P, O> previousOffsets, ErrorHandler
CommonConnectorConfig connectorConfig, CommonConnectorConfig connectorConfig,
ChangeEventSourceFactory<P, O> changeEventSourceFactory, ChangeEventSourceFactory<P, O> changeEventSourceFactory,
ChangeEventSourceMetricsFactory<P> changeEventSourceMetricsFactory, EventDispatcher<P, ?> eventDispatcher, ChangeEventSourceMetricsFactory<P> changeEventSourceMetricsFactory, EventDispatcher<P, ?> eventDispatcher,
DatabaseSchema<?> schema) { DatabaseSchema<?> schema,
SignalProcessor<P, O> signalProcessor) {
this.previousOffsets = previousOffsets; this.previousOffsets = previousOffsets;
this.errorHandler = errorHandler; this.errorHandler = errorHandler;
this.changeEventSourceFactory = changeEventSourceFactory; this.changeEventSourceFactory = changeEventSourceFactory;
@ -83,6 +86,15 @@ public ChangeEventSourceCoordinator(Offsets<P, O> previousOffsets, ErrorHandler
this.executor = Threads.newSingleThreadExecutor(connectorType, connectorConfig.getLogicalName(), "change-event-source-coordinator"); this.executor = Threads.newSingleThreadExecutor(connectorType, connectorConfig.getLogicalName(), "change-event-source-coordinator");
this.eventDispatcher = eventDispatcher; this.eventDispatcher = eventDispatcher;
this.schema = schema; this.schema = schema;
this.signalProcessor = signalProcessor;
}
public ChangeEventSourceCoordinator(Offsets<P, O> previousOffsets, ErrorHandler errorHandler, Class<? extends SourceConnector> connectorType,
CommonConnectorConfig connectorConfig,
ChangeEventSourceFactory<P, O> changeEventSourceFactory,
ChangeEventSourceMetricsFactory<P> changeEventSourceMetricsFactory, EventDispatcher<P, ?> eventDispatcher,
DatabaseSchema<?> schema) {
this(previousOffsets, errorHandler, connectorType, connectorConfig, changeEventSourceFactory, changeEventSourceMetricsFactory, eventDispatcher, schema, null);
} }
public synchronized void start(CdcSourceTaskContext taskContext, ChangeEventQueueMetrics changeEventQueueMetrics, public synchronized void start(CdcSourceTaskContext taskContext, ChangeEventQueueMetrics changeEventQueueMetrics,
@ -119,6 +131,8 @@ public synchronized void start(CdcSourceTaskContext taskContext, ChangeEventQueu
streamingConnected(false); streamingConnected(false);
} }
}); });
getSignalProcessor(previousOffsets).ifPresent(SignalProcessor::start); // this will run on a separate thread
} }
finally { finally {
if (previousLogContext.get() != null) { if (previousLogContext.get() != null) {
@ -127,6 +141,10 @@ public synchronized void start(CdcSourceTaskContext taskContext, ChangeEventQueu
} }
} }
public Optional<SignalProcessor<P, O>> getSignalProcessor(Offsets<P, O> previousOffset) { // Signal processing only work with one partition
return previousOffset == null || previousOffset.getOffsets().size() == 1 ? Optional.ofNullable(signalProcessor) : Optional.empty();
}
protected void executeChangeEventSources(CdcSourceTaskContext taskContext, SnapshotChangeEventSource<P, O> snapshotSource, Offsets<P, O> previousOffsets, protected void executeChangeEventSources(CdcSourceTaskContext taskContext, SnapshotChangeEventSource<P, O> snapshotSource, Offsets<P, O> previousOffsets,
AtomicReference<LoggingContext.PreviousContext> previousLogContext, ChangeEventSourceContext context) AtomicReference<LoggingContext.PreviousContext> previousLogContext, ChangeEventSourceContext context)
throws InterruptedException { throws InterruptedException {
@ -136,6 +154,10 @@ protected void executeChangeEventSources(CdcSourceTaskContext taskContext, Snaps
previousLogContext.set(taskContext.configureLoggingContext("snapshot", partition)); previousLogContext.set(taskContext.configureLoggingContext("snapshot", partition));
SnapshotResult<O> snapshotResult = doSnapshot(snapshotSource, context, partition, previousOffset); SnapshotResult<O> snapshotResult = doSnapshot(snapshotSource, context, partition, previousOffset);
getSignalProcessor(previousOffsets).ifPresent(s -> s.setContext(snapshotResult.getOffset()));
LOGGER.debug("Snapshot result {}", snapshotResult);
if (running && snapshotResult.isCompletedOrSkipped()) { if (running && snapshotResult.isCompletedOrSkipped()) {
previousLogContext.set(taskContext.configureLoggingContext("streaming", partition)); previousLogContext.set(taskContext.configureLoggingContext("streaming", partition));
streamEvents(context, partition, snapshotResult.getOffset()); streamEvents(context, partition, snapshotResult.getOffset());
@ -179,7 +201,8 @@ protected void initStreamEvents(P partition, O offsetContext) throws Interrupted
streamingSource = changeEventSourceFactory.getStreamingChangeEventSource(); streamingSource = changeEventSourceFactory.getStreamingChangeEventSource();
eventDispatcher.setEventListener(streamingMetrics); eventDispatcher.setEventListener(streamingMetrics);
streamingConnected(true); streamingConnected(true);
streamingSource.init(); streamingSource.init(offsetContext);
getSignalProcessor(previousOffsets).ifPresent(s -> s.setContext(streamingSource.getOffsetContext()));
final Optional<IncrementalSnapshotChangeEventSource<P, ? extends DataCollectionId>> incrementalSnapshotChangeEventSource = changeEventSourceFactory final Optional<IncrementalSnapshotChangeEventSource<P, ? extends DataCollectionId>> incrementalSnapshotChangeEventSource = changeEventSourceFactory
.getIncrementalSnapshotChangeEventSource(offsetContext, snapshotMetrics, snapshotMetrics); .getIncrementalSnapshotChangeEventSource(offsetContext, snapshotMetrics, snapshotMetrics);
@ -214,6 +237,11 @@ public synchronized void stop() throws InterruptedException {
executor.awaitTermination(SHUTDOWN_WAIT_TIMEOUT.toMillis(), TimeUnit.MILLISECONDS); executor.awaitTermination(SHUTDOWN_WAIT_TIMEOUT.toMillis(), TimeUnit.MILLISECONDS);
} }
Optional<SignalProcessor<P, O>> processor = getSignalProcessor(previousOffsets);
if (processor.isPresent()) {
processor.get().stop();
}
eventDispatcher.close(); eventDispatcher.close();
} }
finally { finally {

View File

@ -27,7 +27,17 @@
import io.debezium.data.Envelope; import io.debezium.data.Envelope;
import io.debezium.data.Envelope.Operation; import io.debezium.data.Envelope.Operation;
import io.debezium.heartbeat.Heartbeat; import io.debezium.heartbeat.Heartbeat;
import io.debezium.pipeline.signal.Signal; import io.debezium.pipeline.signal.SignalProcessor;
import io.debezium.pipeline.signal.actions.Log;
import io.debezium.pipeline.signal.actions.SchemaChanges;
import io.debezium.pipeline.signal.actions.SignalAction;
import io.debezium.pipeline.signal.actions.snapshotting.CloseIncrementalSnapshotWindow;
import io.debezium.pipeline.signal.actions.snapshotting.ExecuteSnapshot;
import io.debezium.pipeline.signal.actions.snapshotting.OpenIncrementalSnapshotWindow;
import io.debezium.pipeline.signal.actions.snapshotting.PauseIncrementalSnapshot;
import io.debezium.pipeline.signal.actions.snapshotting.ResumeIncrementalSnapshot;
import io.debezium.pipeline.signal.actions.snapshotting.StopSnapshot;
import io.debezium.pipeline.signal.channels.SourceSignalChannel;
import io.debezium.pipeline.source.snapshot.incremental.IncrementalSnapshotChangeEventSource; import io.debezium.pipeline.source.snapshot.incremental.IncrementalSnapshotChangeEventSource;
import io.debezium.pipeline.source.spi.DataChangeEventListener; import io.debezium.pipeline.source.spi.DataChangeEventListener;
import io.debezium.pipeline.source.spi.EventMetadataProvider; import io.debezium.pipeline.source.spi.EventMetadataProvider;
@ -38,6 +48,7 @@
import io.debezium.pipeline.spi.Partition; import io.debezium.pipeline.spi.Partition;
import io.debezium.pipeline.spi.SchemaChangeEventEmitter; import io.debezium.pipeline.spi.SchemaChangeEventEmitter;
import io.debezium.pipeline.txmetadata.TransactionMonitor; import io.debezium.pipeline.txmetadata.TransactionMonitor;
import io.debezium.relational.HistorizedRelationalDatabaseConnectorConfig;
import io.debezium.relational.history.ConnectTableChangeSerializer; import io.debezium.relational.history.ConnectTableChangeSerializer;
import io.debezium.relational.history.HistoryRecord.Fields; import io.debezium.relational.history.HistoryRecord.Fields;
import io.debezium.schema.DataCollectionFilters.DataCollectionFilter; import io.debezium.schema.DataCollectionFilters.DataCollectionFilter;
@ -81,7 +92,7 @@ public class EventDispatcher<P extends Partition, T extends DataCollectionId> im
private final Schema schemaChangeKeySchema; private final Schema schemaChangeKeySchema;
private final Schema schemaChangeValueSchema; private final Schema schemaChangeValueSchema;
private final ConnectTableChangeSerializer tableChangesSerializer; private final ConnectTableChangeSerializer tableChangesSerializer;
private final Signal<P> signal; private final SourceSignalChannel sourceSignalChannel;
private IncrementalSnapshotChangeEventSource<P, T> incrementalSnapshotChangeEventSource; private IncrementalSnapshotChangeEventSource<P, T> incrementalSnapshotChangeEventSource;
/** /**
@ -89,11 +100,22 @@ public class EventDispatcher<P extends Partition, T extends DataCollectionId> im
*/ */
private final StreamingChangeRecordReceiver streamingReceiver; private final StreamingChangeRecordReceiver streamingReceiver;
private final SignalProcessor<P, ?> signalProcessor;
public EventDispatcher(CommonConnectorConfig connectorConfig, TopicNamingStrategy<T> topicNamingStrategy, public EventDispatcher(CommonConnectorConfig connectorConfig, TopicNamingStrategy<T> topicNamingStrategy,
DatabaseSchema<T> schema, ChangeEventQueue<DataChangeEvent> queue, DataCollectionFilter<T> filter, DatabaseSchema<T> schema, ChangeEventQueue<DataChangeEvent> queue, DataCollectionFilter<T> filter,
ChangeEventCreator changeEventCreator, EventMetadataProvider metadataProvider, SchemaNameAdjuster schemaNameAdjuster) { ChangeEventCreator changeEventCreator, EventMetadataProvider metadataProvider, SchemaNameAdjuster schemaNameAdjuster,
SignalProcessor<P, ?> signalProcessor) {
this(connectorConfig, topicNamingStrategy, schema, queue, filter, changeEventCreator, null, metadataProvider, this(connectorConfig, topicNamingStrategy, schema, queue, filter, changeEventCreator, null, metadataProvider,
connectorConfig.createHeartbeat(topicNamingStrategy, schemaNameAdjuster, null, null), schemaNameAdjuster); connectorConfig.createHeartbeat(topicNamingStrategy, schemaNameAdjuster, null, null), schemaNameAdjuster, signalProcessor);
}
public EventDispatcher(CommonConnectorConfig connectorConfig, TopicNamingStrategy<T> topicNamingStrategy,
DatabaseSchema<T> schema, ChangeEventQueue<DataChangeEvent> queue, DataCollectionFilter<T> filter,
ChangeEventCreator changeEventCreator, EventMetadataProvider metadataProvider,
Heartbeat heartbeat, SchemaNameAdjuster schemaNameAdjuster, SignalProcessor<P, ?> signalProcessor) {
this(connectorConfig, topicNamingStrategy, schema, queue, filter, changeEventCreator, null, metadataProvider,
heartbeat, schemaNameAdjuster, signalProcessor);
} }
public EventDispatcher(CommonConnectorConfig connectorConfig, TopicNamingStrategy<T> topicNamingStrategy, public EventDispatcher(CommonConnectorConfig connectorConfig, TopicNamingStrategy<T> topicNamingStrategy,
@ -101,13 +123,21 @@ public EventDispatcher(CommonConnectorConfig connectorConfig, TopicNamingStrateg
ChangeEventCreator changeEventCreator, EventMetadataProvider metadataProvider, ChangeEventCreator changeEventCreator, EventMetadataProvider metadataProvider,
Heartbeat heartbeat, SchemaNameAdjuster schemaNameAdjuster) { Heartbeat heartbeat, SchemaNameAdjuster schemaNameAdjuster) {
this(connectorConfig, topicNamingStrategy, schema, queue, filter, changeEventCreator, null, metadataProvider, this(connectorConfig, topicNamingStrategy, schema, queue, filter, changeEventCreator, null, metadataProvider,
heartbeat, schemaNameAdjuster); heartbeat, schemaNameAdjuster, null);
}
public EventDispatcher(CommonConnectorConfig connectorConfig, TopicNamingStrategy<T> topicNamingStrategy,
DatabaseSchema<T> schema, ChangeEventQueue<DataChangeEvent> queue, DataCollectionFilter<T> filter,
ChangeEventCreator changeEventCreator, EventMetadataProvider metadataProvider, SchemaNameAdjuster schemaNameAdjuster) {
this(connectorConfig, topicNamingStrategy, schema, queue, filter, changeEventCreator, null, metadataProvider,
connectorConfig.createHeartbeat(topicNamingStrategy, schemaNameAdjuster, null, null), schemaNameAdjuster, null);
} }
public EventDispatcher(CommonConnectorConfig connectorConfig, TopicNamingStrategy<T> topicNamingStrategy, public EventDispatcher(CommonConnectorConfig connectorConfig, TopicNamingStrategy<T> topicNamingStrategy,
DatabaseSchema<T> schema, ChangeEventQueue<DataChangeEvent> queue, DataCollectionFilter<T> filter, DatabaseSchema<T> schema, ChangeEventQueue<DataChangeEvent> queue, DataCollectionFilter<T> filter,
ChangeEventCreator changeEventCreator, InconsistentSchemaHandler<P, T> inconsistentSchemaHandler, ChangeEventCreator changeEventCreator, InconsistentSchemaHandler<P, T> inconsistentSchemaHandler,
EventMetadataProvider metadataProvider, Heartbeat heartbeat, SchemaNameAdjuster schemaNameAdjuster) { EventMetadataProvider metadataProvider, Heartbeat heartbeat, SchemaNameAdjuster schemaNameAdjuster,
SignalProcessor<P, ?> signalProcessor) {
this.tableChangesSerializer = new ConnectTableChangeSerializer(schemaNameAdjuster); this.tableChangesSerializer = new ConnectTableChangeSerializer(schemaNameAdjuster);
this.connectorConfig = connectorConfig; this.connectorConfig = connectorConfig;
this.topicNamingStrategy = topicNamingStrategy; this.topicNamingStrategy = topicNamingStrategy;
@ -124,7 +154,14 @@ public EventDispatcher(CommonConnectorConfig connectorConfig, TopicNamingStrateg
this.transactionMonitor = new TransactionMonitor(connectorConfig, metadataProvider, schemaNameAdjuster, this.transactionMonitor = new TransactionMonitor(connectorConfig, metadataProvider, schemaNameAdjuster,
this::enqueueTransactionMessage, topicNamingStrategy.transactionTopic()); this::enqueueTransactionMessage, topicNamingStrategy.transactionTopic());
this.signal = new Signal<>(connectorConfig, this); this.signalProcessor = signalProcessor;
if (signalProcessor != null) {
this.sourceSignalChannel = signalProcessor.getSourceSignalChannel();
this.sourceSignalChannel.init(connectorConfig);
}
else {
this.sourceSignalChannel = null;
}
this.heartbeat = heartbeat; this.heartbeat = heartbeat;
schemaChangeKeySchema = SchemaFactory.get().schemaHistoryConnectorKeySchema(schemaNameAdjuster, connectorConfig); schemaChangeKeySchema = SchemaFactory.get().schemaHistoryConnectorKeySchema(schemaNameAdjuster, connectorConfig);
@ -135,7 +172,8 @@ public EventDispatcher(CommonConnectorConfig connectorConfig, TopicNamingStrateg
public EventDispatcher(CommonConnectorConfig connectorConfig, TopicNamingStrategy<T> topicNamingStrategy, public EventDispatcher(CommonConnectorConfig connectorConfig, TopicNamingStrategy<T> topicNamingStrategy,
DatabaseSchema<T> schema, ChangeEventQueue<DataChangeEvent> queue, DataCollectionFilter<T> filter, DatabaseSchema<T> schema, ChangeEventQueue<DataChangeEvent> queue, DataCollectionFilter<T> filter,
ChangeEventCreator changeEventCreator, InconsistentSchemaHandler<P, T> inconsistentSchemaHandler, Heartbeat heartbeat, ChangeEventCreator changeEventCreator, InconsistentSchemaHandler<P, T> inconsistentSchemaHandler, Heartbeat heartbeat,
SchemaNameAdjuster schemaNameAdjuster, TransactionMonitor transactionMonitor) { SchemaNameAdjuster schemaNameAdjuster, TransactionMonitor transactionMonitor,
SignalProcessor<P, ?> signalProcessor) {
this.tableChangesSerializer = new ConnectTableChangeSerializer(schemaNameAdjuster); this.tableChangesSerializer = new ConnectTableChangeSerializer(schemaNameAdjuster);
this.connectorConfig = connectorConfig; this.connectorConfig = connectorConfig;
this.topicNamingStrategy = topicNamingStrategy; this.topicNamingStrategy = topicNamingStrategy;
@ -150,12 +188,38 @@ public EventDispatcher(CommonConnectorConfig connectorConfig, TopicNamingStrateg
this.skippedOperations = connectorConfig.getSkippedOperations(); this.skippedOperations = connectorConfig.getSkippedOperations();
this.neverSkip = connectorConfig.supportsOperationFiltering() || this.skippedOperations.isEmpty(); this.neverSkip = connectorConfig.supportsOperationFiltering() || this.skippedOperations.isEmpty();
this.transactionMonitor = transactionMonitor; this.transactionMonitor = transactionMonitor;
this.signal = new Signal<>(connectorConfig, this); this.signalProcessor = signalProcessor;
if (signalProcessor != null) {
this.sourceSignalChannel = signalProcessor.getSourceSignalChannel();
this.sourceSignalChannel.init(connectorConfig);
}
else {
this.sourceSignalChannel = null;
}
this.heartbeat = heartbeat; this.heartbeat = heartbeat;
schemaChangeKeySchema = SchemaFactory.get().schemaHistoryConnectorKeySchema(schemaNameAdjuster, connectorConfig); schemaChangeKeySchema = SchemaFactory.get().schemaHistoryConnectorKeySchema(schemaNameAdjuster, connectorConfig);
schemaChangeValueSchema = SchemaFactory.get().schemaHistoryConnectorValueSchema(schemaNameAdjuster, connectorConfig, tableChangesSerializer); schemaChangeValueSchema = SchemaFactory.get().schemaHistoryConnectorValueSchema(schemaNameAdjuster, connectorConfig, tableChangesSerializer);
} }
public Map<String, SignalAction<P>> getSignalingActions() {
return Map.of(Log.NAME, new Log<>(),
SchemaChanges.NAME, getSchemaChangeAction(this.connectorConfig),
ExecuteSnapshot.NAME, new ExecuteSnapshot<>(this),
StopSnapshot.NAME, new StopSnapshot<>(this),
OpenIncrementalSnapshotWindow.NAME, new OpenIncrementalSnapshotWindow<>(),
CloseIncrementalSnapshotWindow.NAME, new CloseIncrementalSnapshotWindow<>(this),
PauseIncrementalSnapshot.NAME, new PauseIncrementalSnapshot<>(this),
ResumeIncrementalSnapshot.NAME, new ResumeIncrementalSnapshot<>(this));
}
private SignalAction<P> getSchemaChangeAction(CommonConnectorConfig connectorConfig) {
return connectorConfig instanceof HistorizedRelationalDatabaseConnectorConfig
? new SchemaChanges<>(this, ((HistorizedRelationalDatabaseConnectorConfig) connectorConfig).useCatalogBeforeSchema())
: new SchemaChanges<>(this, false);
}
public void dispatchSnapshotEvent(P partition, T dataCollectionId, ChangeRecordEmitter<P> changeRecordEmitter, public void dispatchSnapshotEvent(P partition, T dataCollectionId, ChangeRecordEmitter<P> changeRecordEmitter,
SnapshotReceiver<P> receiver) SnapshotReceiver<P> receiver)
throws InterruptedException { throws InterruptedException {
@ -232,8 +296,14 @@ public void changeRecord(P partition,
OffsetContext offset, OffsetContext offset,
ConnectHeaders headers) ConnectHeaders headers)
throws InterruptedException { throws InterruptedException {
if (operation == Operation.CREATE && signal.isSignal(dataCollectionId)) { if (operation == Operation.CREATE && connectorConfig.isSignalDataCollection(dataCollectionId) && sourceSignalChannel != null) {
signal.process(partition, value, offset); sourceSignalChannel.process(value);
if (signalProcessor != null) {
// This is a synchronization point to immediately execute an eventual stop signal, just before emitting the CDC event
// in this way the offset context updated by signaling will be correctly saved
signalProcessor.process();
}
} }
if (neverSkip || !skippedOperations.contains(operation)) { if (neverSkip || !skippedOperations.contains(operation)) {
@ -402,7 +472,7 @@ public void changeRecord(P partition,
Objects.requireNonNull(value, "value must not be null"); Objects.requireNonNull(value, "value must not be null");
LOGGER.trace("Received change record for {} operation on key {}", operation, key); LOGGER.trace("Received change record {} for {} operation on key {} with context {}", value, operation, key, offsetContext);
// Truncate events must have null key schema as they are sent to table topics without keys // Truncate events must have null key schema as they are sent to table topics without keys
Schema keySchema = (key == null && operation == Operation.TRUNCATE) ? null Schema keySchema = (key == null && operation == Operation.TRUNCATE) ? null

View File

@ -1,178 +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.pipeline.signal;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import java.util.Optional;
import org.apache.kafka.connect.data.Struct;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import io.debezium.annotation.NotThreadSafe;
import io.debezium.config.CommonConnectorConfig;
import io.debezium.data.Envelope;
import io.debezium.document.Document;
import io.debezium.document.DocumentReader;
import io.debezium.pipeline.EventDispatcher;
import io.debezium.pipeline.source.snapshot.incremental.CloseIncrementalSnapshotWindow;
import io.debezium.pipeline.source.snapshot.incremental.OpenIncrementalSnapshotWindow;
import io.debezium.pipeline.spi.OffsetContext;
import io.debezium.pipeline.spi.Partition;
import io.debezium.relational.HistorizedRelationalDatabaseConnectorConfig;
import io.debezium.spi.schema.DataCollectionId;
/**
* The class responsible for processing of signals delivered to Debezium via a dedicated signaling table.
* The processor supports a common set of signals that it can process and every connector can register its own
* additional signals.
* The signalling table must conform to the structure
* <ul>
* <li>{@code id STRING} - the unique identifier of the signal sent, usually UUID, can be used for deduplication</li>
* <li>{@code type STRING} - the unique logical name of the code executing the signal</li>
* <li>{@code data STRING} - the data in JSON format that are passed to the signal code
* </ul>
*
* @author Jiri Pechanec
*
*/
@NotThreadSafe
public class Signal<P extends Partition> {
@FunctionalInterface
public interface Action<P extends Partition> {
/**
* @param signalPayload the content of the signal
* @return true if the signal was processed
*/
boolean arrived(Payload<P> signalPayload) throws InterruptedException;
}
public static class Payload<P extends Partition> {
public final String id;
public final String type;
public final Document data;
public final P partition;
public final OffsetContext offsetContext;
public final Struct source;
/**
* @param partition partition from which the signal was sent
* @param id identifier of the signal intended for deduplication, usually ignored by the signal
* @param type of the signal, usually ignored by the signal, should be used only when a signal code is shared for mutlple signals
* @param data data specific for given signal instance
* @param offsetContext offset at what the signal was sent
* @param source source info about position at what the signal was sent
*/
public Payload(P partition, String id, String type, Document data, OffsetContext offsetContext, Struct source) {
super();
this.partition = partition;
this.id = id;
this.type = type;
this.data = data;
this.offsetContext = offsetContext;
this.source = source;
}
@Override
public String toString() {
return "Payload [id=" + id + ", type=" + type + ", data=" + data + ", offsetContext=" + offsetContext
+ ", source=" + source + "]";
}
}
private static final Logger LOGGER = LoggerFactory.getLogger(Signal.class);
private final CommonConnectorConfig connectorConfig;
private final Map<String, Action<P>> signalActions = new HashMap<>();
public Signal(CommonConnectorConfig connectorConfig, EventDispatcher<P, ? extends DataCollectionId> eventDispatcher) {
this.connectorConfig = connectorConfig;
registerSignalAction(Log.NAME, new Log<>());
if (connectorConfig instanceof HistorizedRelationalDatabaseConnectorConfig) {
registerSignalAction(SchemaChanges.NAME,
new SchemaChanges<>(eventDispatcher, ((HistorizedRelationalDatabaseConnectorConfig) connectorConfig).useCatalogBeforeSchema()));
}
else {
registerSignalAction(SchemaChanges.NAME, new SchemaChanges<>(eventDispatcher, false));
}
registerSignalAction(ExecuteSnapshot.NAME, new ExecuteSnapshot<>(eventDispatcher));
registerSignalAction(StopSnapshot.NAME, new StopSnapshot<>(eventDispatcher));
registerSignalAction(OpenIncrementalSnapshotWindow.NAME, new OpenIncrementalSnapshotWindow<>());
registerSignalAction(CloseIncrementalSnapshotWindow.NAME, new CloseIncrementalSnapshotWindow<>(eventDispatcher));
registerSignalAction(PauseIncrementalSnapshot.NAME, new PauseIncrementalSnapshot<>(eventDispatcher));
registerSignalAction(ResumeIncrementalSnapshot.NAME, new ResumeIncrementalSnapshot<>(eventDispatcher));
}
Signal(CommonConnectorConfig connectorConfig) {
this(connectorConfig, null);
}
public boolean isSignal(DataCollectionId dataCollectionId) {
return connectorConfig.isSignalDataCollection(dataCollectionId);
}
public void registerSignalAction(String id, Action<P> signal) {
LOGGER.debug("Registering signal '{}' using class '{}'", id, signal.getClass().getName());
signalActions.put(id, signal);
}
public boolean process(P partition, String id, String type, String data, OffsetContext offset, Struct source) throws InterruptedException {
LOGGER.debug("Received signal id = '{}', type = '{}', data = '{}'", id, type, data);
final Action<P> action = signalActions.get(type);
if (action == null) {
LOGGER.warn("Signal '{}' has been received but the type '{}' is not recognized", id, type);
return false;
}
try {
final Document jsonData = (data == null || data.isEmpty()) ? Document.create()
: DocumentReader.defaultReader().read(data);
return action.arrived(new Payload<>(partition, id, type, jsonData, offset, source));
}
catch (IOException e) {
LOGGER.warn("Signal '{}' has been received but the data '{}' cannot be parsed", id, data, e);
return false;
}
}
public boolean process(P partition, String id, String type, String data) throws InterruptedException {
return process(partition, id, type, data, null, null);
}
/**
*
* @param value Envelope with change from signaling table
* @param offset offset of the incoming signal
* @return true if the signal was processed
*/
public boolean process(P partition, Struct value, OffsetContext offset) throws InterruptedException {
String id = null;
String type = null;
String data = null;
Struct source = null;
try {
final Optional<String[]> parseSignal = connectorConfig.parseSignallingMessage(value);
if (value.schema().field(Envelope.FieldName.SOURCE) != null) {
source = value.getStruct(Envelope.FieldName.SOURCE);
}
if (!parseSignal.isPresent()) {
return false;
}
id = parseSignal.get()[0];
type = parseSignal.get()[1];
data = parseSignal.get()[2];
}
catch (Exception e) {
LOGGER.warn("Exception while preparing to process the signal '{}'", value, e);
}
return process(partition, id, type, data, offset, source);
}
}

View File

@ -0,0 +1,52 @@
/*
* 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.pipeline.signal;
import java.util.Map;
import io.debezium.document.Document;
import io.debezium.pipeline.spi.OffsetContext;
import io.debezium.pipeline.spi.Partition;
public class SignalPayload<P extends Partition> {
public final String id;
public final String type;
public final Document data;
public final P partition;
public final OffsetContext offsetContext;
public final Map<String, Object> additionalData;
/**
* @param partition partition from which the signal was sent
* @param id identifier of the signal intended for deduplication, usually ignored by the signal
* @param type of the signal, usually ignored by the signal, should be used only when a signal code is shared for multiple signals
* @param data data specific for given signal instance
* @param offsetContext offset at what the signal was sent
* @param additionalData additional data specific to the channel
*/
public SignalPayload(P partition, String id, String type, Document data, OffsetContext offsetContext, Map<String, Object> additionalData) {
super();
this.partition = partition;
this.id = id;
this.type = type;
this.data = data;
this.offsetContext = offsetContext;
this.additionalData = additionalData;
}
@Override
public String toString() {
return "SignalPayload{" +
"id='" + id + '\'' +
", type='" + type + '\'' +
", data=" + data +
", partition=" + partition +
", offsetContext=" + offsetContext +
", additionalData=" + additionalData +
'}';
}
}

View File

@ -0,0 +1,164 @@
/*
* 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.pipeline.signal;
import java.io.IOException;
import java.time.Duration;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.function.Predicate;
import org.apache.kafka.connect.source.SourceConnector;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import io.debezium.config.CommonConnectorConfig;
import io.debezium.document.Document;
import io.debezium.document.DocumentReader;
import io.debezium.pipeline.signal.actions.SignalAction;
import io.debezium.pipeline.signal.channels.SignalChannelReader;
import io.debezium.pipeline.signal.channels.SourceSignalChannel;
import io.debezium.pipeline.spi.OffsetContext;
import io.debezium.pipeline.spi.Offsets;
import io.debezium.pipeline.spi.Partition;
import io.debezium.util.Threads;
/**
* This class permits to process signals coming from the different channels.
*
* @author Mario Fiore Vitale
*/
public class SignalProcessor<P extends Partition, O extends OffsetContext> {
private static final Logger LOGGER = LoggerFactory.getLogger(SignalProcessor.class);
/**
* Waiting period for the polling loop to finish. Will be applied twice, once gracefully, once forcefully.
*/
public static final Duration SHUTDOWN_WAIT_TIMEOUT = Duration.ofSeconds(90);
private final Map<String, SignalAction<P>> signalActions = new HashMap<>();
private final CommonConnectorConfig connectorConfig;
private final List<SignalChannelReader> signalChannelReaders;
private final ScheduledExecutorService signalProcessorExecutor;
private final DocumentReader documentReader;
private Offsets<P, O> previousOffsets;
public SignalProcessor(Class<? extends SourceConnector> connector,
CommonConnectorConfig config,
Map<String, SignalAction<P>> signalActions,
List<SignalChannelReader> signalChannelReaders, DocumentReader documentReader,
Offsets<P, O> previousOffsets) {
this.connectorConfig = config;
this.signalChannelReaders = signalChannelReaders;
this.documentReader = documentReader;
this.previousOffsets = previousOffsets;
this.signalProcessorExecutor = Threads.newSingleThreadScheduledExecutor(connector, config.getLogicalName(), SignalProcessor.class.getSimpleName(), false);
signalChannelReaders.stream()
.filter(isEnabled())
.forEach(signalChannelReader -> signalChannelReader.init(connectorConfig));
this.signalActions.putAll(signalActions);
}
private Predicate<SignalChannelReader> isEnabled() {
return reader -> connectorConfig.getEnabledChannels().contains(reader.name());
}
public void setContext(O offset) {
previousOffsets = Offsets.of(Collections.singletonMap(previousOffsets.getTheOnlyPartition(), offset));
}
public void start() {
LOGGER.info("SignalProcessor started. Scheduling it every {}ms", connectorConfig.getSignalPollInterval().toMillis());
signalProcessorExecutor.scheduleAtFixedRate(this::process, 0, 1, TimeUnit.NANOSECONDS);
}
public void stop() throws InterruptedException {
// The close must run with same thread of the read otherwise Kafka client will detect multi-thread and throw and exception
signalProcessorExecutor.submit(() -> signalChannelReaders.stream()
.filter(isEnabled())
.forEach(SignalChannelReader::close));
signalProcessorExecutor.shutdown();
boolean isShutdown = signalProcessorExecutor.awaitTermination(SHUTDOWN_WAIT_TIMEOUT.toMillis(), TimeUnit.MILLISECONDS);
if (!isShutdown) {
LOGGER.warn("SignalProcessor didn't stop in the expected time, shutting down executor now");
// Clear interrupt flag so the forced termination is always attempted
Thread.interrupted();
signalProcessorExecutor.shutdownNow();
signalProcessorExecutor.awaitTermination(SHUTDOWN_WAIT_TIMEOUT.toMillis(), TimeUnit.MILLISECONDS);
}
LOGGER.info("SignalProcessor stopped");
}
public void registerSignalAction(String id, SignalAction<P> signal) {
LOGGER.debug("Registering signal '{}' using class '{}'", id, signal.getClass().getName());
signalActions.put(id, signal);
}
public synchronized void process() {
LOGGER.trace("SignalProcessor processing");
signalChannelReaders.parallelStream()
.filter(isEnabled())
.map(SignalChannelReader::read)
.flatMap(Collection::stream)
.forEach(this::processSignal);
}
private void processSignal(SignalRecord signalRecord) {
LOGGER.debug("Signal Processor offset context {}", previousOffsets.getOffsets());
LOGGER.debug("Received signal id = '{}', type = '{}', data = '{}'", signalRecord.getId(), signalRecord.getType(), signalRecord.getData());
final SignalAction<P> action = signalActions.get(signalRecord.getType());
if (action == null) {
LOGGER.warn("Signal '{}' has been received but the type '{}' is not recognized", signalRecord.getId(), signalRecord.getType());
return;
}
try {
final Document jsonData = (signalRecord.getData() == null || signalRecord.getData().isEmpty()) ? Document.create()
: documentReader.read(signalRecord.getData());
action.arrived(new SignalPayload<>(previousOffsets.getTheOnlyPartition(), signalRecord.getId(), signalRecord.getType(), jsonData,
previousOffsets.getTheOnlyOffset(), signalRecord.getAdditionalData()));
}
catch (IOException e) {
LOGGER.warn("Signal '{}' has been received but the data '{}' cannot be parsed", signalRecord.getId(), signalRecord.getData(), e);
}
catch (InterruptedException e) {
LOGGER.warn("Action {} has been interrupted. The signal {} may not have been processed.", signalRecord.getType(), signalRecord);
Thread.currentThread().interrupt();
}
catch (Exception e) {
LOGGER.warn("Action {} failed. The signal {} may not have been processed.", signalRecord.getType(), signalRecord, e);
}
}
public SourceSignalChannel getSourceSignalChannel() {
return (SourceSignalChannel) signalChannelReaders.stream()
.filter(channel -> channel.name().equals(SourceSignalChannel.CHANNEL_NAME))
.findFirst().get();
}
}

View File

@ -0,0 +1,92 @@
/*
* 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.pipeline.signal;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import org.apache.kafka.connect.data.Struct;
import io.debezium.config.CommonConnectorConfig;
/**
* The class represent the signal sent on a channel:
* <ul>
* <li>{@code id STRING} - the unique identifier of the signal sent, usually UUID, can be used for deduplication</li>
* <li>{@code type STRING} - the unique logical name of the code executing the signal</li>
* <li>{@code data STRING} - the data in JSON format that are passed to the signal code
* </ul>
*
* @author Mario Fiore Vitale
*/
public class SignalRecord {
private final String id;
private final String type;
private final String data;
private final Map<String, Object> additionalData;
public SignalRecord(String id, String type, String data, Long channelOffset, Map<String, Object> additionalData) {
this.id = id;
this.type = type;
this.data = data;
this.additionalData = additionalData;
}
public static Optional<SignalRecord> buildSignalRecordFromChangeEventSource(Struct value, CommonConnectorConfig config) {
final Optional<String[]> parseSignal = config.parseSignallingMessage(value);
return parseSignal.map(signalMessage -> new SignalRecord(signalMessage[0], signalMessage[1], signalMessage[2], null, Map.of()));
}
public String getId() {
return id;
}
public String getType() {
return type;
}
public String getData() {
return data;
}
public <T> T getAdditionalDataProperty(String property, Class<T> type) {
return type.cast(additionalData.get(property));
}
public Map<String, Object> getAdditionalData() {
return additionalData;
}
@Override
public String toString() {
return "SignalRecord{" +
"id='" + id + '\'' +
", type='" + type + '\'' +
", data='" + data + '\'' +
", additionalData=" + additionalData +
'}';
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
SignalRecord that = (SignalRecord) o;
return Objects.equals(id, that.id);
}
@Override
public int hashCode() {
return Objects.hash(id);
}
}

View File

@ -3,7 +3,7 @@
* *
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/ */
package io.debezium.pipeline.signal; package io.debezium.pipeline.signal.actions;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -14,7 +14,7 @@
/** /**
* @author Chris Cranford * @author Chris Cranford
*/ */
public abstract class AbstractSnapshotSignal<P extends Partition> implements Signal.Action<P> { public abstract class AbstractSnapshotSignal<P extends Partition> implements SignalAction<P> {
private static final Logger LOGGER = LoggerFactory.getLogger(AbstractSnapshotSignal.class); private static final Logger LOGGER = LoggerFactory.getLogger(AbstractSnapshotSignal.class);
protected static final String FIELD_DATA_COLLECTIONS = "data-collections"; protected static final String FIELD_DATA_COLLECTIONS = "data-collections";

View File

@ -3,15 +3,15 @@
* *
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/ */
package io.debezium.pipeline.signal; package io.debezium.pipeline.signal.actions;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import io.debezium.pipeline.signal.Signal.Payload; import io.debezium.pipeline.signal.SignalPayload;
import io.debezium.pipeline.spi.Partition; import io.debezium.pipeline.spi.Partition;
public class Log<P extends Partition> implements Signal.Action<P> { public class Log<P extends Partition> implements SignalAction<P> {
private static final Logger LOGGER = LoggerFactory.getLogger(Log.class); private static final Logger LOGGER = LoggerFactory.getLogger(Log.class);
private static final String FIELD_MESSAGE = "message"; private static final String FIELD_MESSAGE = "message";
@ -19,7 +19,7 @@ public class Log<P extends Partition> implements Signal.Action<P> {
public static final String NAME = "log"; public static final String NAME = "log";
@Override @Override
public boolean arrived(Payload<P> signalPayload) { public boolean arrived(SignalPayload<P> signalPayload) {
final String message = signalPayload.data.getString(FIELD_MESSAGE); final String message = signalPayload.data.getString(FIELD_MESSAGE);
if (message == null || message.isEmpty()) { if (message == null || message.isEmpty()) {
LOGGER.warn("Logging signal '{}' has arrived but the requested field '{}' is missing from data", signalPayload, FIELD_MESSAGE); LOGGER.warn("Logging signal '{}' has arrived but the requested field '{}' is missing from data", signalPayload, FIELD_MESSAGE);

View File

@ -3,14 +3,14 @@
* *
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/ */
package io.debezium.pipeline.signal; package io.debezium.pipeline.signal.actions;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import io.debezium.document.Array; import io.debezium.document.Array;
import io.debezium.pipeline.EventDispatcher; import io.debezium.pipeline.EventDispatcher;
import io.debezium.pipeline.signal.Signal.Payload; import io.debezium.pipeline.signal.SignalPayload;
import io.debezium.pipeline.spi.Partition; import io.debezium.pipeline.spi.Partition;
import io.debezium.relational.RelationalDatabaseSchema; import io.debezium.relational.RelationalDatabaseSchema;
import io.debezium.relational.TableId; import io.debezium.relational.TableId;
@ -20,7 +20,7 @@
import io.debezium.schema.SchemaChangeEvent; import io.debezium.schema.SchemaChangeEvent;
import io.debezium.spi.schema.DataCollectionId; import io.debezium.spi.schema.DataCollectionId;
public class SchemaChanges<P extends Partition> implements Signal.Action<P> { public class SchemaChanges<P extends Partition> implements SignalAction<P> {
private static final Logger LOGGER = LoggerFactory.getLogger(SchemaChanges.class); private static final Logger LOGGER = LoggerFactory.getLogger(SchemaChanges.class);
@ -42,7 +42,7 @@ public SchemaChanges(EventDispatcher<P, ? extends DataCollectionId> dispatcher,
} }
@Override @Override
public boolean arrived(Payload<P> signalPayload) throws InterruptedException { public boolean arrived(SignalPayload<P> signalPayload) throws InterruptedException {
final Array changes = signalPayload.data.getArray(FIELD_CHANGES); final Array changes = signalPayload.data.getArray(FIELD_CHANGES);
final String database = signalPayload.data.getString(FIELD_DATABASE); final String database = signalPayload.data.getString(FIELD_DATABASE);
final String schema = signalPayload.data.getString(FIELD_SCHEMA); final String schema = signalPayload.data.getString(FIELD_SCHEMA);
@ -63,7 +63,7 @@ public boolean arrived(Payload<P> signalPayload) throws InterruptedException {
tableChange, tableChange,
signalPayload.partition.getSourcePartition(), signalPayload.partition.getSourcePartition(),
signalPayload.offsetContext.getOffset(), signalPayload.offsetContext.getOffset(),
signalPayload.source, signalPayload.offsetContext.getSourceInfo(),
database, database,
schema)); schema));
}); });

View File

@ -0,0 +1,19 @@
/*
* 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.pipeline.signal.actions;
import io.debezium.pipeline.signal.SignalPayload;
import io.debezium.pipeline.spi.Partition;
@FunctionalInterface
public interface SignalAction<P extends Partition> {
/**
* @param signalPayload the content of the signal
* @return true if the signal was processed
*/
boolean arrived(SignalPayload<P> signalPayload) throws InterruptedException;
}

View File

@ -3,18 +3,18 @@
* *
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/ */
package io.debezium.pipeline.source.snapshot.incremental; package io.debezium.pipeline.signal.actions.snapshotting;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import io.debezium.pipeline.EventDispatcher; import io.debezium.pipeline.EventDispatcher;
import io.debezium.pipeline.signal.Signal; import io.debezium.pipeline.signal.SignalPayload;
import io.debezium.pipeline.signal.Signal.Payload; import io.debezium.pipeline.signal.actions.SignalAction;
import io.debezium.pipeline.spi.Partition; import io.debezium.pipeline.spi.Partition;
import io.debezium.spi.schema.DataCollectionId; import io.debezium.spi.schema.DataCollectionId;
public class CloseIncrementalSnapshotWindow<P extends Partition> implements Signal.Action<P> { public class CloseIncrementalSnapshotWindow<P extends Partition> implements SignalAction<P> {
private static final Logger LOGGER = LoggerFactory.getLogger(CloseIncrementalSnapshotWindow.class); private static final Logger LOGGER = LoggerFactory.getLogger(CloseIncrementalSnapshotWindow.class);
@ -28,7 +28,7 @@ public CloseIncrementalSnapshotWindow(EventDispatcher<P, ? extends DataCollectio
@SuppressWarnings({ "rawtypes", "unchecked" }) @SuppressWarnings({ "rawtypes", "unchecked" })
@Override @Override
public boolean arrived(Payload<P> signalPayload) throws InterruptedException { public boolean arrived(SignalPayload<P> signalPayload) throws InterruptedException {
dispatcher.getIncrementalSnapshotChangeEventSource().closeWindow(signalPayload.partition, signalPayload.id, dispatcher.getIncrementalSnapshotChangeEventSource().closeWindow(signalPayload.partition, signalPayload.id,
signalPayload.offsetContext); signalPayload.offsetContext);
return true; return true;

View File

@ -3,7 +3,7 @@
* *
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/ */
package io.debezium.pipeline.signal; package io.debezium.pipeline.signal.actions.snapshotting;
import java.util.List; import java.util.List;
import java.util.Optional; import java.util.Optional;
@ -15,7 +15,8 @@
import io.debezium.document.Array; import io.debezium.document.Array;
import io.debezium.document.Document; import io.debezium.document.Document;
import io.debezium.pipeline.EventDispatcher; import io.debezium.pipeline.EventDispatcher;
import io.debezium.pipeline.signal.Signal.Payload; import io.debezium.pipeline.signal.SignalPayload;
import io.debezium.pipeline.signal.actions.AbstractSnapshotSignal;
import io.debezium.pipeline.spi.Partition; import io.debezium.pipeline.spi.Partition;
import io.debezium.spi.schema.DataCollectionId; import io.debezium.spi.schema.DataCollectionId;
import io.debezium.util.Strings; import io.debezium.util.Strings;
@ -41,7 +42,7 @@ public ExecuteSnapshot(EventDispatcher<P, ? extends DataCollectionId> dispatcher
} }
@Override @Override
public boolean arrived(Payload<P> signalPayload) throws InterruptedException { public boolean arrived(SignalPayload<P> signalPayload) throws InterruptedException {
final List<String> dataCollections = getDataCollections(signalPayload.data); final List<String> dataCollections = getDataCollections(signalPayload.data);
if (dataCollections == null) { if (dataCollections == null) {
return false; return false;
@ -54,7 +55,7 @@ public boolean arrived(Payload<P> signalPayload) throws InterruptedException {
switch (type) { switch (type) {
case INCREMENTAL: case INCREMENTAL:
dispatcher.getIncrementalSnapshotChangeEventSource().addDataCollectionNamesToSnapshot( dispatcher.getIncrementalSnapshotChangeEventSource().addDataCollectionNamesToSnapshot(
signalPayload.partition, dataCollections, additionalCondition, surrogateKey, signalPayload.offsetContext); signalPayload.partition, signalPayload.offsetContext, signalPayload.additionalData, dataCollections, additionalCondition, surrogateKey);
break; break;
} }
return true; return true;

View File

@ -3,16 +3,16 @@
* *
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/ */
package io.debezium.pipeline.source.snapshot.incremental; package io.debezium.pipeline.signal.actions.snapshotting;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import io.debezium.pipeline.signal.Signal; import io.debezium.pipeline.signal.SignalPayload;
import io.debezium.pipeline.signal.Signal.Payload; import io.debezium.pipeline.signal.actions.SignalAction;
import io.debezium.pipeline.spi.Partition; import io.debezium.pipeline.spi.Partition;
public class OpenIncrementalSnapshotWindow<P extends Partition> implements Signal.Action<P> { public class OpenIncrementalSnapshotWindow<P extends Partition> implements SignalAction<P> {
private static final Logger LOGGER = LoggerFactory.getLogger(OpenIncrementalSnapshotWindow.class); private static final Logger LOGGER = LoggerFactory.getLogger(OpenIncrementalSnapshotWindow.class);
@ -22,7 +22,7 @@ public OpenIncrementalSnapshotWindow() {
} }
@Override @Override
public boolean arrived(Payload<P> signalPayload) { public boolean arrived(SignalPayload<P> signalPayload) {
signalPayload.offsetContext.getIncrementalSnapshotContext().openWindow(signalPayload.id); signalPayload.offsetContext.getIncrementalSnapshotContext().openWindow(signalPayload.id);
return true; return true;
} }

View File

@ -3,9 +3,11 @@
* *
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/ */
package io.debezium.pipeline.signal; package io.debezium.pipeline.signal.actions.snapshotting;
import io.debezium.pipeline.EventDispatcher; import io.debezium.pipeline.EventDispatcher;
import io.debezium.pipeline.signal.SignalPayload;
import io.debezium.pipeline.signal.actions.AbstractSnapshotSignal;
import io.debezium.pipeline.spi.Partition; import io.debezium.pipeline.spi.Partition;
import io.debezium.spi.schema.DataCollectionId; import io.debezium.spi.schema.DataCollectionId;
@ -20,7 +22,7 @@ public PauseIncrementalSnapshot(EventDispatcher<P, ? extends DataCollectionId> d
} }
@Override @Override
public boolean arrived(Signal.Payload<P> signalPayload) throws InterruptedException { public boolean arrived(SignalPayload<P> signalPayload) throws InterruptedException {
dispatcher.getIncrementalSnapshotChangeEventSource().pauseSnapshot(signalPayload.partition, signalPayload.offsetContext); dispatcher.getIncrementalSnapshotChangeEventSource().pauseSnapshot(signalPayload.partition, signalPayload.offsetContext);
return true; return true;
} }

View File

@ -3,9 +3,11 @@
* *
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/ */
package io.debezium.pipeline.signal; package io.debezium.pipeline.signal.actions.snapshotting;
import io.debezium.pipeline.EventDispatcher; import io.debezium.pipeline.EventDispatcher;
import io.debezium.pipeline.signal.SignalPayload;
import io.debezium.pipeline.signal.actions.AbstractSnapshotSignal;
import io.debezium.pipeline.spi.Partition; import io.debezium.pipeline.spi.Partition;
import io.debezium.spi.schema.DataCollectionId; import io.debezium.spi.schema.DataCollectionId;
@ -20,7 +22,7 @@ public ResumeIncrementalSnapshot(EventDispatcher<P, ? extends DataCollectionId>
} }
@Override @Override
public boolean arrived(Signal.Payload<P> signalPayload) throws InterruptedException { public boolean arrived(SignalPayload<P> signalPayload) throws InterruptedException {
dispatcher.getIncrementalSnapshotChangeEventSource().resumeSnapshot( dispatcher.getIncrementalSnapshotChangeEventSource().resumeSnapshot(
signalPayload.partition, signalPayload.offsetContext); signalPayload.partition, signalPayload.offsetContext);
return true; return true;

View File

@ -3,7 +3,7 @@
* *
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/ */
package io.debezium.pipeline.signal; package io.debezium.pipeline.signal.actions.snapshotting;
import java.util.List; import java.util.List;
import java.util.stream.Collectors; import java.util.stream.Collectors;
@ -14,6 +14,8 @@
import io.debezium.document.Array; import io.debezium.document.Array;
import io.debezium.document.Document; import io.debezium.document.Document;
import io.debezium.pipeline.EventDispatcher; import io.debezium.pipeline.EventDispatcher;
import io.debezium.pipeline.signal.SignalPayload;
import io.debezium.pipeline.signal.actions.AbstractSnapshotSignal;
import io.debezium.pipeline.spi.Partition; import io.debezium.pipeline.spi.Partition;
import io.debezium.spi.schema.DataCollectionId; import io.debezium.spi.schema.DataCollectionId;
@ -39,7 +41,7 @@ public StopSnapshot(EventDispatcher<P, ? extends DataCollectionId> dispatcher) {
} }
@Override @Override
public boolean arrived(Signal.Payload<P> signalPayload) throws InterruptedException { public boolean arrived(SignalPayload<P> signalPayload) throws InterruptedException {
final List<String> dataCollections = getDataCollections(signalPayload.data); final List<String> dataCollections = getDataCollections(signalPayload.data);
final SnapshotType type = getSnapshotType(signalPayload.data); final SnapshotType type = getSnapshotType(signalPayload.data);
@ -47,7 +49,7 @@ public boolean arrived(Signal.Payload<P> signalPayload) throws InterruptedExcept
switch (type) { switch (type) {
case INCREMENTAL: case INCREMENTAL:
dispatcher.getIncrementalSnapshotChangeEventSource() dispatcher.getIncrementalSnapshotChangeEventSource()
.stopSnapshot(signalPayload.partition, dataCollections, signalPayload.offsetContext); .stopSnapshot(signalPayload.partition, signalPayload.offsetContext, signalPayload.additionalData, dataCollections);
break; break;
} }

View File

@ -0,0 +1,180 @@
/*
* 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.pipeline.signal.channels;
import java.io.IOException;
import java.time.Duration;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.UUID;
import java.util.stream.Collectors;
import java.util.stream.StreamSupport;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.consumer.ConsumerRecords;
import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.common.serialization.StringDeserializer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import io.debezium.config.CommonConnectorConfig;
import io.debezium.config.Configuration;
import io.debezium.config.Field;
import io.debezium.document.Document;
import io.debezium.document.DocumentReader;
import io.debezium.pipeline.signal.SignalRecord;
import io.debezium.util.Collect;
import io.debezium.util.Loggings;
/**
* The class responsible for processing of signals delivered to Debezium via a dedicated Kafka topic.
* The signal message must have the following structure:
* <ul>
* <li>{@code id STRING} - the unique identifier of the signal sent, usually UUID, can be used for deduplication</li>
* <li>{@code type STRING} - the unique logical name of the code executing the signal</li>
* <li>{@code data STRING} - the data in JSON format that are passed to the signal code
* </ul>
*/
public class KafkaSignalChannel implements SignalChannelReader {
private static final Logger LOGGER = LoggerFactory.getLogger(KafkaSignalChannel.class);
public static final String CONFIGURATION_FIELD_PREFIX_STRING = "signal.";
private static final String CONSUMER_PREFIX = CONFIGURATION_FIELD_PREFIX_STRING + "consumer.";
public static final String CHANNEL_OFFSET = "channelOffset";
private String topicName;
private String connectorName;
private Duration pollTimeoutMs;
private KafkaConsumer<String, String> signalsConsumer;
public static final Field SIGNAL_TOPIC = Field.create(CONFIGURATION_FIELD_PREFIX_STRING + "kafka.topic")
.withDisplayName("Signal topic name")
.withType(ConfigDef.Type.STRING)
.withWidth(ConfigDef.Width.LONG)
.withImportance(ConfigDef.Importance.HIGH)
.withDescription("The name of the topic for the signals to the connector")
.withValidation(Field::isRequired);
public static final Field BOOTSTRAP_SERVERS = Field.create(CONFIGURATION_FIELD_PREFIX_STRING + "kafka.bootstrap.servers")
.withDisplayName("Kafka broker addresses")
.withType(ConfigDef.Type.STRING)
.withWidth(ConfigDef.Width.LONG)
.withImportance(ConfigDef.Importance.HIGH)
.withDescription("A list of host/port pairs that the connector will use for establishing the initial "
+ "connection to the Kafka cluster for retrieving signals to the connector."
+ "This should point to the same Kafka cluster used by the Kafka Connect process.")
.withValidation(Field::isRequired);
public static final Field SIGNAL_POLL_TIMEOUT_MS = Field.create(CONFIGURATION_FIELD_PREFIX_STRING
+ "kafka.poll.timeout.ms")
.withDisplayName("Poll timeout for kafka signals (ms)")
.withType(ConfigDef.Type.INT)
.withWidth(ConfigDef.Width.SHORT)
.withImportance(ConfigDef.Importance.LOW)
.withDescription("The number of milliseconds to wait while polling signals.")
.withDefault(100)
.withValidation(Field::isNonNegativeInteger);
private Optional<SignalRecord> processSignal(ConsumerRecord<String, String> record) {
if (!connectorName.equals(record.key())) {
LOGGER.info("Signal key '{}' doesn't match the connector's name '{}'", record.key(), connectorName);
return Optional.empty();
}
String value = record.value();
LOGGER.trace("Processing signal: {}", value);
if (value == null || value.isEmpty()) {
return Optional.empty();
}
final Optional<Document> jsonData = parseJson(value);
if (jsonData.isEmpty()) {
return Optional.empty();
}
Document document = jsonData.get();
String id = document.getString("id");
String type = document.getString("type");
Document data = document.getDocument("data");
return Optional.of(new SignalRecord(id, type, data.toString(), record.offset(), Map.of(CHANNEL_OFFSET, record.offset())));
}
private static Optional<Document> parseJson(String value) {
final Document jsonData;
try {
jsonData = DocumentReader.defaultReader().read(value);
}
catch (IOException e) {
Loggings.logErrorAndTraceRecord(LOGGER, value, "Skipped signal due to an error", e);
return Optional.empty();
}
return Optional.of(jsonData);
}
public void seek(long signalOffset) {
signalsConsumer.seek(new TopicPartition(topicName, 0), signalOffset + 1);
}
@Override
public String name() {
return "kafka";
}
@Override
public void init(CommonConnectorConfig connectorConfig) {
String signalName = "kafka-signal";
this.connectorName = connectorConfig.getLogicalName();
Configuration signalConfig = connectorConfig.getConfig().subset(CONFIGURATION_FIELD_PREFIX_STRING, false)
.edit()
.withDefault(KafkaSignalChannel.SIGNAL_TOPIC, connectorName + "-signal")
.build();
this.topicName = signalConfig.getString(SIGNAL_TOPIC);
this.pollTimeoutMs = Duration.ofMillis(signalConfig.getInteger(SIGNAL_POLL_TIMEOUT_MS));
Configuration consumerConfig = buildKafkaConfiguration(signalName, signalConfig);
this.signalsConsumer = new KafkaConsumer<>(consumerConfig.asProperties());
LOGGER.info("Subscribing to signals topic '{}'", topicName);
signalsConsumer.assign(Collect.arrayListOf(new TopicPartition(topicName, 0)));
}
private static Configuration buildKafkaConfiguration(String signalName, Configuration signalConfig) {
return signalConfig.subset(CONSUMER_PREFIX, true).edit()
.withDefault(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, signalConfig.getString(BOOTSTRAP_SERVERS))
.withDefault(ConsumerConfig.CLIENT_ID_CONFIG, UUID.randomUUID().toString())
.withDefault(ConsumerConfig.GROUP_ID_CONFIG, signalName)
.withDefault(ConsumerConfig.FETCH_MIN_BYTES_CONFIG, 1) // get even the smallest message
.withDefault(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false)
.withDefault(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, 10000) // readjusted since 0.10.1.0
.withDefault(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class)
.withDefault(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class)
.build();
}
@Override
public List<SignalRecord> read() {
// DBZ-1361 not using poll(Duration) to keep compatibility with AK 1.x
ConsumerRecords<String, String> recoveredRecords = signalsConsumer.poll(pollTimeoutMs.toMillis());
return StreamSupport.stream(recoveredRecords.spliterator(), false)
.map(this::processSignal)
.filter(Optional::isPresent)
.map(Optional::get)
.collect(Collectors.toList());
}
@Override
public void close() {
signalsConsumer.close();
}
}

View File

@ -0,0 +1,33 @@
/*
* 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.pipeline.signal.channels;
import java.util.List;
import io.debezium.config.CommonConnectorConfig;
import io.debezium.pipeline.signal.SignalProcessor;
import io.debezium.pipeline.signal.SignalRecord;
/**
* This interface is used to provide custom read channels for the Debezium signaling feature:
*
* Implementations must:
* define the name of the reader in {@link #name()},
* initialize specific configuration/variables/connections in the {@link #init(CommonConnectorConfig connectorConfig)} method,
* provide a list of signal record in the {@link #read()} method. It is called by {@link SignalProcessor} in a thread loop
* Close all allocated resources int the {@link #close()} method.
*
* @author Mario Fiore Vitale
*/
public interface SignalChannelReader {
String name();
void init(CommonConnectorConfig connectorConfig);
List<SignalRecord> read();
void close();
}

View File

@ -0,0 +1,99 @@
/*
* 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.pipeline.signal.channels;
import java.util.List;
import java.util.Optional;
import java.util.Queue;
import java.util.concurrent.ConcurrentLinkedQueue;
import org.apache.kafka.connect.data.Struct;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import io.debezium.annotation.NotThreadSafe;
import io.debezium.config.CommonConnectorConfig;
import io.debezium.pipeline.signal.SignalRecord;
/**
* The class responsible for processing of signals delivered to Debezium via a dedicated signaling table.
* The processor supports a common set of signals that it can process and every connector can register its own
* additional signals.
* The signalling table must conform to the structure
* <ul>
* <li>{@code id STRING} - the unique identifier of the signal sent, usually UUID, can be used for deduplication</li>
* <li>{@code type STRING} - the unique logical name of the code executing the signal</li>
* <li>{@code data STRING} - the data in JSON format that are passed to the signal code
* </ul>
*
* @author Jiri Pechanec
*
*/
@NotThreadSafe
public class SourceSignalChannel implements SignalChannelReader {
private static final Logger LOGGER = LoggerFactory.getLogger(SourceSignalChannel.class);
public static final String CHANNEL_NAME = "source";
public static final Queue<SignalRecord> SIGNALS = new ConcurrentLinkedQueue<>();
public CommonConnectorConfig connectorConfig;
@Override
public String name() {
return CHANNEL_NAME;
}
@Override
public void init(CommonConnectorConfig connectorConfig) {
this.connectorConfig = connectorConfig;
}
@Override
public List<SignalRecord> read() {
LOGGER.trace("Reading signaling events from queue");
SignalRecord signalRecord = SIGNALS.poll();
if (signalRecord == null) {
return List.of();
}
return List.of(signalRecord);
}
@Override
public void close() {
SIGNALS.clear();
}
/** Used in streaming flow to add signals from signaling table
*
* @param value Envelope with change from signaling table
* @return true if the signal was processed
*/
public boolean process(Struct value) throws InterruptedException {
LOGGER.trace("Received event from signaling table. Enqueue for process");
try {
Optional<SignalRecord> result = SignalRecord.buildSignalRecordFromChangeEventSource(value, connectorConfig);
if (result.isEmpty()) {
return false;
}
final SignalRecord signalRecord = result.get();
SIGNALS.add(signalRecord);
return true;
}
catch (Exception e) {
LOGGER.warn("Exception while preparing to process the signal '{}'", value, e);
return false;
}
}
}

View File

@ -62,7 +62,7 @@ public abstract class AbstractIncrementalSnapshotChangeEventSource<P extends Par
private static final Logger LOGGER = LoggerFactory.getLogger(AbstractIncrementalSnapshotChangeEventSource.class); private static final Logger LOGGER = LoggerFactory.getLogger(AbstractIncrementalSnapshotChangeEventSource.class);
private final RelationalDatabaseConnectorConfig connectorConfig; protected final RelationalDatabaseConnectorConfig connectorConfig;
private final Clock clock; private final Clock clock;
private final RelationalDatabaseSchema databaseSchema; private final RelationalDatabaseSchema databaseSchema;
private final SnapshotProgressListener<P> progressListener; private final SnapshotProgressListener<P> progressListener;
@ -98,6 +98,7 @@ public AbstractIncrementalSnapshotChangeEventSource(RelationalDatabaseConnectorC
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
public void closeWindow(P partition, String id, OffsetContext offsetContext) throws InterruptedException { public void closeWindow(P partition, String id, OffsetContext offsetContext) throws InterruptedException {
context = (IncrementalSnapshotContext<T>) offsetContext.getIncrementalSnapshotContext(); context = (IncrementalSnapshotContext<T>) offsetContext.getIncrementalSnapshotContext();
LOGGER.trace("Closing Window {}", context.toString());
if (!context.closeWindow(id)) { if (!context.closeWindow(id)) {
return; return;
} }
@ -461,8 +462,8 @@ private void nextDataCollection(P partition) {
@Override @Override
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
public void addDataCollectionNamesToSnapshot(P partition, List<String> dataCollectionIds, Optional<String> additionalCondition, Optional<String> surrogateKey, public void addDataCollectionNamesToSnapshot(P partition, OffsetContext offsetContext, Map<String, Object> additionalData, List<String> dataCollectionIds,
OffsetContext offsetContext) Optional<String> additionalCondition, Optional<String> surrogateKey)
throws InterruptedException { throws InterruptedException {
context = (IncrementalSnapshotContext<T>) offsetContext.getIncrementalSnapshotContext(); context = (IncrementalSnapshotContext<T>) offsetContext.getIncrementalSnapshotContext();
boolean shouldReadChunk = !context.snapshotRunning(); boolean shouldReadChunk = !context.snapshotRunning();
@ -483,8 +484,9 @@ public void addDataCollectionNamesToSnapshot(P partition, List<String> dataColle
@Override @Override
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
public void stopSnapshot(P partition, List<String> dataCollectionIds, OffsetContext offsetContext) { public void stopSnapshot(P partition, OffsetContext offsetContext, Map<String, Object> additionalData, List<String> dataCollectionIds) {
context = (IncrementalSnapshotContext<T>) offsetContext.getIncrementalSnapshotContext(); context = (IncrementalSnapshotContext<T>) offsetContext.getIncrementalSnapshotContext();
LOGGER.trace("Stopping incremental snapshot with context {}", context);
if (context.snapshotRunning()) { if (context.snapshotRunning()) {
if (dataCollectionIds == null || dataCollectionIds.isEmpty()) { if (dataCollectionIds == null || dataCollectionIds.isEmpty()) {
LOGGER.info("Stopping incremental snapshot."); LOGGER.info("Stopping incremental snapshot.");

View File

@ -6,6 +6,7 @@
package io.debezium.pipeline.source.snapshot.incremental; package io.debezium.pipeline.source.snapshot.incremental;
import java.util.List; import java.util.List;
import java.util.Map;
import java.util.Optional; import java.util.Optional;
import io.debezium.pipeline.spi.OffsetContext; import io.debezium.pipeline.spi.OffsetContext;
@ -31,11 +32,11 @@ public interface IncrementalSnapshotChangeEventSource<P extends Partition, T ext
void init(P partition, OffsetContext offsetContext); void init(P partition, OffsetContext offsetContext);
void addDataCollectionNamesToSnapshot(P partition, List<String> dataCollectionIds, Optional<String> additionalCondition, Optional<String> surrogateKey, void addDataCollectionNamesToSnapshot(P partition, OffsetContext offsetContext, Map<String, Object> additionalData, List<String> dataCollectionIds,
OffsetContext offsetContext) Optional<String> additionalCondition, Optional<String> surrogateKey)
throws InterruptedException; throws InterruptedException;
void stopSnapshot(P partition, List<String> dataCollectionIds, OffsetContext offsetContext); void stopSnapshot(P partition, OffsetContext offsetContext, Map<String, Object> additionalData, List<String> dataCollectionIds);
default void processHeartbeat(P partition, OffsetContext offsetContext) throws InterruptedException { default void processHeartbeat(P partition, OffsetContext offsetContext) throws InterruptedException {
} }

View File

@ -13,6 +13,8 @@
import io.debezium.annotation.NotThreadSafe; import io.debezium.annotation.NotThreadSafe;
import io.debezium.jdbc.JdbcConnection; import io.debezium.jdbc.JdbcConnection;
import io.debezium.pipeline.EventDispatcher; import io.debezium.pipeline.EventDispatcher;
import io.debezium.pipeline.signal.actions.snapshotting.CloseIncrementalSnapshotWindow;
import io.debezium.pipeline.signal.actions.snapshotting.OpenIncrementalSnapshotWindow;
import io.debezium.pipeline.source.spi.DataChangeEventListener; import io.debezium.pipeline.source.spi.DataChangeEventListener;
import io.debezium.pipeline.source.spi.SnapshotProgressListener; import io.debezium.pipeline.source.spi.SnapshotProgressListener;
import io.debezium.pipeline.spi.OffsetContext; import io.debezium.pipeline.spi.OffsetContext;

View File

@ -23,7 +23,7 @@ public interface StreamingChangeEventSource<P extends Partition, O extends Offse
* *
* @throws InterruptedException * @throws InterruptedException
*/ */
default void init() throws InterruptedException { default void init(O offsetContext) throws InterruptedException {
} }
/** /**
@ -69,4 +69,8 @@ default boolean executeIteration(ChangeEventSourceContext context, P partition,
*/ */
default void commitOffset(Map<String, ?> partition, Map<String, ?> offset) { default void commitOffset(Map<String, ?> partition, Map<String, ?> offset) {
} }
default O getOffsetContext() {
return null;
}
} }

View File

@ -9,6 +9,7 @@
import java.time.temporal.ChronoUnit; import java.time.temporal.ChronoUnit;
import java.util.concurrent.ExecutorService; import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors; import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ThreadFactory; import java.util.concurrent.ThreadFactory;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
@ -306,4 +307,8 @@ public static ExecutorService newFixedThreadPool(Class<? extends SourceConnector
public static ExecutorService newSingleThreadExecutor(Class<? extends SourceConnector> connector, String connectorId, String name) { public static ExecutorService newSingleThreadExecutor(Class<? extends SourceConnector> connector, String connectorId, String name) {
return newSingleThreadExecutor(connector, connectorId, name, false); return newSingleThreadExecutor(connector, connectorId, name, false);
} }
public static ScheduledExecutorService newSingleThreadScheduledExecutor(Class<? extends SourceConnector> connector, String connectorId, String name, boolean daemon) {
return Executors.newSingleThreadScheduledExecutor(threadFactory(connector, connectorId, name, false, daemon));
}
} }

View File

@ -0,0 +1,2 @@
io.debezium.pipeline.signal.channels.SourceSignalChannel
io.debezium.pipeline.signal.channels.KafkaSignalChannel

View File

@ -0,0 +1,277 @@
/*
* 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.pipeline.signal;
import static org.assertj.core.api.Assertions.assertThat;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import java.time.Instant;
import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.connect.source.SourceConnector;
import org.awaitility.Awaitility;
import org.junit.Before;
import org.junit.Test;
import io.debezium.config.CommonConnectorConfig;
import io.debezium.config.Configuration;
import io.debezium.connector.SourceInfoStructMaker;
import io.debezium.connector.common.BaseSourceInfo;
import io.debezium.document.DocumentReader;
import io.debezium.junit.logging.LogInterceptor;
import io.debezium.pipeline.CommonOffsetContext;
import io.debezium.pipeline.signal.actions.Log;
import io.debezium.pipeline.signal.actions.SignalAction;
import io.debezium.pipeline.signal.channels.SignalChannelReader;
import io.debezium.pipeline.spi.OffsetContext;
import io.debezium.pipeline.spi.Offsets;
import io.debezium.pipeline.spi.Partition;
import io.debezium.pipeline.txmetadata.TransactionContext;
import io.debezium.spi.schema.DataCollectionId;
public class SignalProcessorTest {
private SignalProcessor<TestPartition, OffsetContext> signalProcess;
private final DocumentReader documentReader = DocumentReader.defaultReader();
private Offsets<TestPartition, OffsetContext> initialOffset;
@Before
public void setUp() {
TestOffset testOffset = new TestOffset(new BaseSourceInfo(baseConfig()) {
@Override
protected Instant timestamp() {
return Instant.now();
}
@Override
protected String database() {
return "test_db";
}
});
initialOffset = Offsets.of(new TestPartition(), testOffset);
}
@Test
public void shouldExecuteLog() throws InterruptedException {
final SignalChannelReader genericChannel = mock(SignalChannelReader.class);
when(genericChannel.name()).thenReturn("generic");
when(genericChannel.read()).thenReturn(
List.of(new SignalRecord("log1", "log", "{\"message\": \"signallog {}\"}", -1L, Map.of("channelOffset", -1L))),
List.of());
final LogInterceptor log = new LogInterceptor(Log.class);
signalProcess = new SignalProcessor<>(SourceConnector.class,
baseConfig(),
Map.of(Log.NAME, new Log<>()),
List.of(genericChannel), documentReader, initialOffset);
signalProcess.start();
Awaitility.await()
.atMost(200, TimeUnit.MILLISECONDS)
.untilAsserted(() -> log.containsMessage("signallog {LSN=12345}"));
signalProcess.stop();
assertThat(log.containsMessage("signallog {LSN=12345}")).isTrue();
}
@Test
public void onlyEnabledConnectorShouldExecute() throws InterruptedException {
final SignalChannelReader genericChannel1 = mock(SignalChannelReader.class);
when(genericChannel1.name()).thenReturn("generic1");
when(genericChannel1.read()).thenReturn(
List.of(new SignalRecord("log1", "log", "{\"message\": \"signallog {}\"}", -1L, Map.of("channelOffset", -1L))),
List.of());
final SignalChannelReader genericChannel2 = mock(SignalChannelReader.class);
when(genericChannel2.name()).thenReturn("generic2");
when(genericChannel2.read()).thenReturn(
List.of(new SignalRecord("log1", "log", "{\"message\": \"signallog {}\"}", -1L, Map.of("channelOffset", -1L))),
List.of());
final LogInterceptor log = new LogInterceptor(Log.class);
signalProcess = new SignalProcessor<>(SourceConnector.class,
baseConfig(Map.of(CommonConnectorConfig.SIGNAL_ENABLED_CHANNELS.name(), "generic1")),
Map.of(Log.NAME, new Log<>()),
List.of(genericChannel1, genericChannel2), documentReader, initialOffset);
signalProcess.start();
Awaitility.await()
.atMost(300, TimeUnit.MILLISECONDS)
.until(() -> log.containsMessage("signallog {LSN=12345}"));
signalProcess.stop();
assertThat(log.countOccurrences("signallog {}")).isEqualTo(1);
}
@Test
public void shouldIgnoreInvalidSignalType() throws InterruptedException {
final SignalChannelReader genericChannel = mock(SignalChannelReader.class);
when(genericChannel.name()).thenReturn("generic");
when(genericChannel.read()).thenReturn(
List.of(new SignalRecord("log1", "invalidType", "{\"message\": \"signallog {}\"}", -1L, Map.of("channelOffset", -1L))),
List.of());
final LogInterceptor log = new LogInterceptor(SignalProcessor.class);
signalProcess = new SignalProcessor<>(SourceConnector.class, baseConfig(), Map.of(), List.of(genericChannel), documentReader, initialOffset);
signalProcess.start();
Awaitility.await()
.atMost(200, TimeUnit.SECONDS)
.untilAsserted(() -> assertThat(log.containsMessage("Signal 'log1' has been received but the type 'invalidType' is not recognized")).isTrue());
signalProcess.stop();
}
@Test
public void shouldIgnoreUnparseableData() {
final SignalChannelReader genericChannel = mock(SignalChannelReader.class);
when(genericChannel.name()).thenReturn("generic");
when(genericChannel.read()).thenReturn(
List.of(new SignalRecord("log1", "log", "{\"message: \"signallog\"}", -1L, Map.of("channelOffset", -1L))),
List.of());
final LogInterceptor log = new LogInterceptor(SignalProcessor.class);
signalProcess = new SignalProcessor<>(SourceConnector.class, baseConfig(), Map.of(Log.NAME, new Log<>()), List.of(genericChannel), documentReader, initialOffset);
signalProcess.start();
Awaitility.await()
.atMost(40, TimeUnit.SECONDS)
.untilAsserted(
() -> assertThat(log.containsMessage("Signal 'log1' has been received but the data '{\"message: \"signallog\"}' cannot be parsed")).isTrue());
}
@Test
public void shouldRegisterAdditionalAction() {
final SignalChannelReader genericChannel = mock(SignalChannelReader.class);
when(genericChannel.name()).thenReturn("generic");
when(genericChannel.read()).thenReturn(
List.of(new SignalRecord("log1", "custom", "{\"v\": 5}", -1L, Map.of("channelOffset", -1L))),
List.of());
final AtomicInteger called = new AtomicInteger();
final SignalAction<TestPartition> testAction = signalPayload -> {
called.set(signalPayload.data.getInteger("v"));
return true;
};
signalProcess = new SignalProcessor<>(SourceConnector.class, baseConfig(), Map.of(), List.of(genericChannel), documentReader, initialOffset);
signalProcess.registerSignalAction("custom", testAction);
signalProcess.start();
Awaitility.await()
.atMost(40, TimeUnit.SECONDS)
.untilAsserted(() -> assertThat(called.intValue()).isEqualTo(5));
}
protected CommonConnectorConfig baseConfig() {
return baseConfig(Map.of());
}
protected CommonConnectorConfig baseConfig(Map<String, Object> additionalConfig) {
Configuration.Builder confBuilder = Configuration.create()
.with(CommonConnectorConfig.SIGNAL_DATA_COLLECTION, "debezium.signal")
.with(CommonConnectorConfig.TOPIC_PREFIX, "core")
.with(CommonConnectorConfig.SIGNAL_POLL_INTERVAL_MS, 100)
.with(CommonConnectorConfig.SIGNAL_ENABLED_CHANNELS, "source,generic");
additionalConfig.forEach(confBuilder::with);
return new CommonConnectorConfig(confBuilder.build(), 0) {
@Override
protected SourceInfoStructMaker<?> getSourceInfoStructMaker(Version version) {
return null;
}
@Override
public String getContextName() {
return null;
}
@Override
public String getConnectorName() {
return null;
}
};
}
private static class TestPartition implements Partition {
@Override
public Map<String, String> getSourcePartition() {
throw new UnsupportedOperationException();
}
}
private static class TestOffset extends CommonOffsetContext {
TestOffset(BaseSourceInfo sourceInfo) {
super(sourceInfo);
}
@Override
public Map<String, ?> getOffset() {
return Map.of("LSN", 12345);
}
@Override
public Schema getSourceInfoSchema() {
return null;
}
@Override
public boolean isSnapshotRunning() {
return false;
}
@Override
public void preSnapshotStart() {
}
@Override
public void preSnapshotCompletion() {
}
@Override
public void event(DataCollectionId collectionId, Instant timestamp) {
}
@Override
public TransactionContext getTransactionContext() {
return null;
}
}
}

View File

@ -1,156 +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.pipeline.signal;
import static org.assertj.core.api.Assertions.assertThat;
import java.util.Map;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.connect.data.SchemaBuilder;
import org.apache.kafka.connect.data.Struct;
import org.junit.Test;
import io.debezium.config.CommonConnectorConfig;
import io.debezium.config.Configuration;
import io.debezium.connector.SourceInfoStructMaker;
import io.debezium.data.Envelope;
import io.debezium.junit.logging.LogInterceptor;
import io.debezium.pipeline.spi.Partition;
import io.debezium.relational.TableId;
/**
* @author Jiri Pechanec
*
*/
public class SignalTest {
@Test
public void shouldDetectSignal() {
final Signal<TestPartition> signal = new Signal<>(config());
assertThat(signal.isSignal(new TableId("dbo", null, "mytable"))).isFalse();
assertThat(signal.isSignal(new TableId("debezium", null, "signal"))).isTrue();
}
@Test
public void shouldExecuteLog() throws Exception {
final Signal<TestPartition> signal = new Signal<>(config());
final LogInterceptor log = new LogInterceptor(io.debezium.pipeline.signal.Log.class);
assertThat(signal.process(new TestPartition(), "log1", "log", "{\"message\": \"signallog {}\"}")).isTrue();
assertThat(log.containsMessage("signallog <none>")).isTrue();
}
@Test
public void shouldIgnoreInvalidSignalType() throws Exception {
final Signal<TestPartition> signal = new Signal<>(config());
assertThat(signal.process(new TestPartition(), "log1", "log1", "{\"message\": \"signallog\"}")).isFalse();
}
@Test
public void shouldIgnoreUnparseableData() throws Exception {
final Signal<TestPartition> signal = new Signal<>(config());
assertThat(signal.process(new TestPartition(), "log1", "log", "{\"message: \"signallog\"}")).isFalse();
}
@Test
public void shouldRegisterAdditionalAction() throws Exception {
final Signal<TestPartition> signal = new Signal<>(config());
final AtomicInteger called = new AtomicInteger();
final Signal.Action<TestPartition> testAction = signalPayload -> {
called.set(signalPayload.data.getInteger("v"));
return true;
};
signal.registerSignalAction("custom", testAction);
assertThat(signal.process(new TestPartition(), "log1", "custom", "{\"v\": 5}")).isTrue();
assertThat(called.intValue()).isEqualTo(5);
}
@Test
public void shouldExecuteFromEnvelope() throws Exception {
final Signal<TestPartition> signal = new Signal<>(config());
final Schema afterSchema = SchemaBuilder.struct().name("signal")
.field("col1", Schema.OPTIONAL_STRING_SCHEMA)
.field("col2", Schema.OPTIONAL_STRING_SCHEMA)
.field("col3", Schema.OPTIONAL_STRING_SCHEMA)
.build();
final Envelope env = Envelope.defineSchema()
.withName("someName")
.withRecord(afterSchema)
.withSource(SchemaBuilder.struct().name("source").build())
.build();
final Struct record = new Struct(afterSchema);
record.put("col1", "log1");
record.put("col2", "custom");
record.put("col3", "{\"v\": 5}");
final AtomicInteger called = new AtomicInteger();
final Signal.Action<TestPartition> testAction = signalPayload -> {
called.set(signalPayload.data.getInteger("v"));
return true;
};
signal.registerSignalAction("custom", testAction);
assertThat(signal.process(new TestPartition(), env.create(record, null, null), null)).isTrue();
assertThat(called.intValue()).isEqualTo(5);
}
@Test
public void shouldIgnoreInvalidEnvelope() throws Exception {
final Signal<TestPartition> signal = new Signal<>(config());
final Schema afterSchema = SchemaBuilder.struct().name("signal")
.field("col1", Schema.OPTIONAL_STRING_SCHEMA)
.field("col2", Schema.OPTIONAL_STRING_SCHEMA)
.build();
final Envelope env = Envelope.defineSchema()
.withName("someName")
.withRecord(afterSchema)
.withSource(SchemaBuilder.struct().name("source").build())
.build();
final Struct record = new Struct(afterSchema);
record.put("col1", "log1");
record.put("col2", "custom");
final AtomicInteger called = new AtomicInteger();
final Signal.Action<TestPartition> testAction = signalPayload -> {
called.set(signalPayload.data.getInteger("v"));
return true;
};
signal.registerSignalAction("custom", testAction);
assertThat(signal.process(new TestPartition(), env.create(record, null, null), null)).isFalse();
assertThat(called.intValue()).isEqualTo(0);
assertThat(signal.process(new TestPartition(), record, null)).isFalse();
assertThat(called.intValue()).isEqualTo(0);
}
protected CommonConnectorConfig config() {
return new CommonConnectorConfig(Configuration.create().with(CommonConnectorConfig.SIGNAL_DATA_COLLECTION, "debezium.signal")
.with(CommonConnectorConfig.TOPIC_PREFIX, "core").build(), 0) {
@Override
protected SourceInfoStructMaker<?> getSourceInfoStructMaker(Version version) {
return null;
}
@Override
public String getContextName() {
return null;
}
@Override
public String getConnectorName() {
return null;
}
};
}
private static class TestPartition implements Partition {
@Override
public Map<String, String> getSourcePartition() {
throw new UnsupportedOperationException();
}
}
}

View File

@ -0,0 +1,102 @@
/*
* 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.pipeline.signal;
import static org.assertj.core.api.Assertions.assertThat;
import java.util.List;
import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.connect.data.SchemaBuilder;
import org.apache.kafka.connect.data.Struct;
import org.junit.Test;
import io.debezium.config.CommonConnectorConfig;
import io.debezium.config.Configuration;
import io.debezium.connector.SourceInfoStructMaker;
import io.debezium.data.Envelope;
import io.debezium.pipeline.signal.channels.SourceSignalChannel;
/**
* @author Jiri Pechanec
*
*/
public class SourceSignalChannelTest {
@Test
public void shouldExecuteFromEnvelope() throws Exception {
final SourceSignalChannel sourceSignalChannel = new SourceSignalChannel();
sourceSignalChannel.init(config());
final Schema afterSchema = SchemaBuilder.struct().name("signal")
.field("col1", Schema.OPTIONAL_STRING_SCHEMA)
.field("col2", Schema.OPTIONAL_STRING_SCHEMA)
.field("col3", Schema.OPTIONAL_STRING_SCHEMA)
.build();
final Envelope env = Envelope.defineSchema()
.withName("someName")
.withRecord(afterSchema)
.withSource(SchemaBuilder.struct().name("source").build())
.build();
final Struct record = new Struct(afterSchema);
record.put("col1", "log1");
record.put("col2", "custom");
record.put("col3", "{\"v\": 5}");
sourceSignalChannel.process(env.create(record, null, null));
List<SignalRecord> signalRecords = sourceSignalChannel.read();
assertThat(signalRecords).hasSize(1);
assertThat(signalRecords.get(0).getData()).isEqualTo("{\"v\": 5}");
}
@Test
public void shouldIgnoreInvalidEnvelope() throws Exception {
final SourceSignalChannel sourceSignalChannel = new SourceSignalChannel();
sourceSignalChannel.init(config());
final Schema afterSchema = SchemaBuilder.struct().name("signal")
.field("col1", Schema.OPTIONAL_STRING_SCHEMA)
.field("col2", Schema.OPTIONAL_STRING_SCHEMA)
.build();
final Envelope env = Envelope.defineSchema()
.withName("someName")
.withRecord(afterSchema)
.withSource(SchemaBuilder.struct().name("source").build())
.build();
final Struct record = new Struct(afterSchema);
record.put("col1", "log1");
record.put("col2", "custom");
sourceSignalChannel.process(env.create(record, null, null));
assertThat(sourceSignalChannel.read()).hasSize(0);
sourceSignalChannel.process(record);
assertThat(sourceSignalChannel.read()).hasSize(0);
}
protected CommonConnectorConfig config() {
return new CommonConnectorConfig(Configuration.create()
.with(CommonConnectorConfig.SIGNAL_DATA_COLLECTION, "debezium.signal")
.with(CommonConnectorConfig.TOPIC_PREFIX, "core")
.build(), 0) {
@Override
protected SourceInfoStructMaker<?> getSourceInfoStructMaker(Version version) {
return null;
}
@Override
public String getContextName() {
return null;
}
@Override
public String getConnectorName() {
return null;
}
};
}
}

View File

@ -18,6 +18,7 @@
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Optional; import java.util.Optional;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
@ -26,6 +27,10 @@
import java.util.function.Predicate; import java.util.function.Predicate;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.common.serialization.StringSerializer;
import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.data.Struct;
import org.apache.kafka.connect.source.SourceConnector; import org.apache.kafka.connect.source.SourceConnector;
import org.apache.kafka.connect.source.SourceRecord; import org.apache.kafka.connect.source.SourceRecord;
@ -43,7 +48,8 @@
import io.debezium.junit.SkipWhenConnectorUnderTest; import io.debezium.junit.SkipWhenConnectorUnderTest;
import io.debezium.junit.SkipWhenConnectorUnderTest.Connector; import io.debezium.junit.SkipWhenConnectorUnderTest.Connector;
import io.debezium.junit.logging.LogInterceptor; import io.debezium.junit.logging.LogInterceptor;
import io.debezium.pipeline.signal.StopSnapshot; import io.debezium.kafka.KafkaCluster;
import io.debezium.pipeline.signal.actions.snapshotting.StopSnapshot;
import io.debezium.util.Testing; import io.debezium.util.Testing;
public abstract class AbstractIncrementalSnapshotTest<T extends SourceConnector> extends AbstractConnectorTest { public abstract class AbstractIncrementalSnapshotTest<T extends SourceConnector> extends AbstractConnectorTest {
@ -53,6 +59,10 @@ public abstract class AbstractIncrementalSnapshotTest<T extends SourceConnector>
protected static final Path SCHEMA_HISTORY_PATH = Testing.Files.createTestingPath("file-schema-history-is.txt") protected static final Path SCHEMA_HISTORY_PATH = Testing.Files.createTestingPath("file-schema-history-is.txt")
.toAbsolutePath(); .toAbsolutePath();
private static final int PARTITION_NO = 0;
private static final String SERVER_NAME = "test_server";
protected static KafkaCluster kafka;
protected abstract Class<T> connectorClass(); protected abstract Class<T> connectorClass();
@ -313,6 +323,35 @@ protected void sendAdHocSnapshotSignal() throws SQLException {
sendAdHocSnapshotSignal(tableDataCollectionId()); sendAdHocSnapshotSignal(tableDataCollectionId());
} }
protected void sendAdHocKafkaSnapshotSignal() throws ExecutionException, InterruptedException {
sendExecuteSnapshotKafkaSignal(tableDataCollectionId());
}
protected void sendExecuteSnapshotKafkaSignal(String fullTableNames) throws ExecutionException, InterruptedException {
String signalValue = String.format(
"{\"type\":\"execute-snapshot\",\"data\": {\"data-collections\": [\"%s\"], \"type\": \"INCREMENTAL\"}}",
fullTableNames);
sendKafkaSignal(signalValue);
}
protected String getSignalsTopic() {
return "signals_topic";
}
protected void sendKafkaSignal(String signalValue) throws ExecutionException, InterruptedException {
final ProducerRecord<String, String> executeSnapshotSignal = new ProducerRecord<>(getSignalsTopic(), PARTITION_NO, SERVER_NAME, signalValue);
final Configuration signalProducerConfig = Configuration.create()
.withDefault(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka.brokerList())
.withDefault(ProducerConfig.CLIENT_ID_CONFIG, "signals")
.withDefault(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class)
.withDefault(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class)
.build();
try (KafkaProducer<String, String> producer = new KafkaProducer<>(signalProducerConfig.asProperties())) {
producer.send(executeSnapshotSignal).get();
}
}
protected void sendPauseSignal() { protected void sendPauseSignal() {
try (JdbcConnection connection = databaseConnection()) { try (JdbcConnection connection = databaseConnection()) {
String query = String.format("INSERT INTO %s VALUES('test-pause', 'pause-snapshot', '')", signalTableName()); String query = String.format("INSERT INTO %s VALUES('test-pause', 'pause-snapshot', '')", signalTableName());
@ -428,6 +467,34 @@ public void inserts() throws Exception {
} }
} }
@Test
public void insertsWithKafkaSnapshotSignal() throws Exception {
// Testing.Print.enable();
populateTable();
startConnector();
sendAdHocSnapshotSignal();
try (JdbcConnection connection = databaseConnection()) {
connection.setAutoCommit(false);
for (int i = 0; i < ROW_COUNT; i++) {
connection.executeWithoutCommitting(String.format("INSERT INTO %s (%s, aa) VALUES (%s, %s)",
tableName(),
connection.quotedColumnIdString(pkFieldName()),
i + ROW_COUNT + 1,
i + ROW_COUNT));
}
connection.commit();
}
final int expectedRecordCount = ROW_COUNT * 2;
final Map<Integer, Integer> dbChanges = consumeMixedWithIncrementalSnapshot(expectedRecordCount);
for (int i = 0; i < expectedRecordCount; i++) {
assertThat(dbChanges).contains(entry(i + 1, i));
}
}
@Test @Test
public void updates() throws Exception { public void updates() throws Exception {
// Testing.Print.enable(); // Testing.Print.enable();
@ -624,6 +691,9 @@ public void snapshotWithRegexDataCollections() throws Exception {
@Test @Test
@FixFor("DBZ-4271") @FixFor("DBZ-4271")
public void stopCurrentIncrementalSnapshotWithoutCollectionsAndTakeNewNewIncrementalSnapshotAfterRestart() throws Exception { public void stopCurrentIncrementalSnapshotWithoutCollectionsAndTakeNewNewIncrementalSnapshotAfterRestart() throws Exception {
// Testing.Print.enable();
final LogInterceptor interceptor = new LogInterceptor(AbstractIncrementalSnapshotChangeEventSource.class); final LogInterceptor interceptor = new LogInterceptor(AbstractIncrementalSnapshotChangeEventSource.class);
// We will use chunk size of 1 to have very small batches to guarantee that when we stop // We will use chunk size of 1 to have very small batches to guarantee that when we stop
@ -676,6 +746,9 @@ public void stopCurrentIncrementalSnapshotWithoutCollectionsAndTakeNewNewIncreme
@Test @Test
@FixFor("DBZ-4271") @FixFor("DBZ-4271")
public void stopCurrentIncrementalSnapshotWithAllCollectionsAndTakeNewNewIncrementalSnapshotAfterRestart() throws Exception { public void stopCurrentIncrementalSnapshotWithAllCollectionsAndTakeNewNewIncrementalSnapshotAfterRestart() throws Exception {
// Testing.Print.enable();
final LogInterceptor interceptor = new LogInterceptor(AbstractIncrementalSnapshotChangeEventSource.class); final LogInterceptor interceptor = new LogInterceptor(AbstractIncrementalSnapshotChangeEventSource.class);
// We will use chunk size of 1 to have very small batches to guarantee that when we stop // We will use chunk size of 1 to have very small batches to guarantee that when we stop