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:
parent
58ef4f0b98
commit
952340286e
@ -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,
|
||||||
|
@ -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);
|
||||||
|
|
||||||
|
@ -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()) {
|
||||||
|
@ -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);
|
||||||
|
|
||||||
|
@ -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.
|
||||||
*
|
*
|
||||||
|
@ -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();
|
||||||
|
@ -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;
|
@ -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);
|
||||||
|
|
||||||
|
@ -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() {
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
|
@ -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)) {
|
||||||
|
@ -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;
|
|
||||||
}
|
|
||||||
}
|
|
@ -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 {
|
|
||||||
}
|
|
@ -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);
|
|
||||||
}
|
|
||||||
}
|
|
@ -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 {
|
|
||||||
}
|
|
@ -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 {
|
|
||||||
}
|
|
@ -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;
|
|
||||||
}
|
|
||||||
}
|
|
@ -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)
|
||||||
|
@ -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());
|
||||||
|
|
||||||
|
@ -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";
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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);
|
||||||
|
|
||||||
|
@ -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;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -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);
|
||||||
|
@ -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>
|
||||||
|
@ -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;
|
||||||
}
|
}
|
||||||
|
@ -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);
|
||||||
|
|
||||||
|
@ -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();
|
||||||
|
@ -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
|
||||||
|
@ -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();
|
||||||
|
@ -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);
|
||||||
|
@ -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>
|
||||||
|
@ -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());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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);
|
||||||
|
|
||||||
|
@ -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.
|
||||||
|
@ -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>
|
||||||
|
@ -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) {
|
||||||
|
@ -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();
|
||||||
}
|
}
|
||||||
|
@ -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 {
|
||||||
|
@ -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
|
||||||
|
@ -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);
|
|
||||||
}
|
|
||||||
}
|
|
@ -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 +
|
||||||
|
'}';
|
||||||
|
}
|
||||||
|
}
|
@ -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();
|
||||||
|
}
|
||||||
|
}
|
@ -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);
|
||||||
|
}
|
||||||
|
}
|
@ -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";
|
@ -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);
|
@ -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));
|
||||||
});
|
});
|
@ -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;
|
||||||
|
}
|
@ -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;
|
@ -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;
|
@ -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;
|
||||||
}
|
}
|
@ -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;
|
||||||
}
|
}
|
@ -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;
|
@ -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;
|
||||||
}
|
}
|
||||||
|
|
@ -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();
|
||||||
|
}
|
||||||
|
}
|
@ -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();
|
||||||
|
}
|
@ -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;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
@ -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.");
|
||||||
|
@ -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 {
|
||||||
}
|
}
|
||||||
|
@ -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;
|
||||||
|
@ -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;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -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));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -0,0 +1,2 @@
|
|||||||
|
io.debezium.pipeline.signal.channels.SourceSignalChannel
|
||||||
|
io.debezium.pipeline.signal.channels.KafkaSignalChannel
|
@ -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;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -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();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
@ -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;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -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
|
||||||
|
Loading…
Reference in New Issue
Block a user