diff --git a/debezium-connector-sqlserver/pom.xml b/debezium-connector-sqlserver/pom.xml
index 99cad6442..4b2e1ad2a 100644
--- a/debezium-connector-sqlserver/pom.xml
+++ b/debezium-connector-sqlserver/pom.xml
@@ -19,7 +19,7 @@
1433saPassword!
- master
+ testDBmicrosoft/mssql-server-linux:2017-latestfalsetrue
@@ -171,7 +171,6 @@
${sqlserver.port}${sqlserver.user}${sqlserver.password}
- ${sqlserver.dbname}${skipLongRunningTests}
diff --git a/debezium-connector-sqlserver/src/main/java/io/debezium/connector/sqlserver/Lsn.java b/debezium-connector-sqlserver/src/main/java/io/debezium/connector/sqlserver/Lsn.java
index dc0324031..644e35af9 100644
--- a/debezium-connector-sqlserver/src/main/java/io/debezium/connector/sqlserver/Lsn.java
+++ b/debezium-connector-sqlserver/src/main/java/io/debezium/connector/sqlserver/Lsn.java
@@ -7,6 +7,8 @@
import java.util.Arrays;
+import io.debezium.util.Strings;
+
/**
* A logical representation of SQL Server LSN (log sequence number) position.
*
@@ -14,6 +16,8 @@
*
*/
public class Lsn implements Comparable {
+ public static final Lsn NULL = new Lsn(null);
+
private final byte[] binary;
private int[] unsignedBinary;
@@ -66,6 +70,10 @@ public String toString() {
return string;
}
+ public static Lsn valueOf(String lsnString) {
+ return (lsnString == null) ? NULL : new Lsn(Strings.hexStringToByteArray(lsnString.replace(":", "")));
+ }
+
@Override
public int hashCode() {
final int prime = 31;
diff --git a/debezium-connector-sqlserver/src/main/java/io/debezium/connector/sqlserver/SnapshotChangeRecordEmitter.java b/debezium-connector-sqlserver/src/main/java/io/debezium/connector/sqlserver/SnapshotChangeRecordEmitter.java
new file mode 100644
index 000000000..14364733b
--- /dev/null
+++ b/debezium-connector-sqlserver/src/main/java/io/debezium/connector/sqlserver/SnapshotChangeRecordEmitter.java
@@ -0,0 +1,42 @@
+/*
+ * 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.sqlserver;
+
+import io.debezium.data.Envelope.Operation;
+import io.debezium.pipeline.spi.OffsetContext;
+import io.debezium.relational.RelationalChangeRecordEmitter;
+import io.debezium.util.Clock;
+
+/**
+ * Emits change data based on a single row read via JDBC.
+ *
+ * @author Gunnar Morling
+ */
+public class SnapshotChangeRecordEmitter extends RelationalChangeRecordEmitter {
+
+ private final Object[] row;
+
+ public SnapshotChangeRecordEmitter(OffsetContext offset, Object[] row, Clock clock) {
+ super(offset, clock);
+
+ this.row = row;
+ }
+
+ @Override
+ protected Operation getOperation() {
+ return Operation.READ;
+ }
+
+ @Override
+ protected Object[] getOldColumnValues() {
+ throw new UnsupportedOperationException("Can't get old row values for READ record");
+ }
+
+ @Override
+ protected Object[] getNewColumnValues() {
+ return row;
+ }
+}
diff --git a/debezium-connector-sqlserver/src/main/java/io/debezium/connector/sqlserver/SourceInfo.java b/debezium-connector-sqlserver/src/main/java/io/debezium/connector/sqlserver/SourceInfo.java
index 95933951e..8fbaaf0b7 100644
--- a/debezium-connector-sqlserver/src/main/java/io/debezium/connector/sqlserver/SourceInfo.java
+++ b/debezium-connector-sqlserver/src/main/java/io/debezium/connector/sqlserver/SourceInfo.java
@@ -1,3 +1,8 @@
+/*
+ * 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.sqlserver;
import java.time.Instant;
@@ -19,6 +24,7 @@ public class SourceInfo extends AbstractSourceInfo {
public static final String SERVER_NAME_KEY = "name";
public static final String LOG_TIMESTAMP_KEY = "ts_ms";
public static final String CHANGE_LSN_KEY = "change_lsn";
+ public static final String COMMIT_LSN_KEY = "commit_lsn";
public static final String SNAPSHOT_KEY = "snapshot";
public static final Schema SCHEMA = schemaBuilder()
@@ -26,11 +32,14 @@ public class SourceInfo extends AbstractSourceInfo {
.field(SERVER_NAME_KEY, Schema.STRING_SCHEMA)
.field(LOG_TIMESTAMP_KEY, Schema.OPTIONAL_INT64_SCHEMA)
.field(CHANGE_LSN_KEY, Schema.OPTIONAL_STRING_SCHEMA)
+ .field(COMMIT_LSN_KEY, Schema.OPTIONAL_STRING_SCHEMA)
.field(SNAPSHOT_KEY, Schema.OPTIONAL_BOOLEAN_SCHEMA)
.build();
private final String serverName;
private Lsn changeLsn;
+ private Lsn commitLsn;
+ private boolean snapshot;
private Instant sourceTime;
protected SourceInfo(String serverName) {
@@ -46,10 +55,26 @@ public Lsn getChangeLsn() {
return changeLsn;
}
+ public Lsn getCommitLsn() {
+ return commitLsn;
+ }
+
+ public void setCommitLsn(Lsn commitLsn) {
+ this.commitLsn = commitLsn;
+ }
+
public void setSourceTime(Instant instant) {
sourceTime = instant;
}
+ public boolean isSnapshot() {
+ return snapshot;
+ }
+
+ public void setSnapshot(boolean snapshot) {
+ this.snapshot = snapshot;
+ }
+
@Override
protected Schema schema() {
return SCHEMA;
@@ -57,10 +82,15 @@ protected Schema schema() {
@Override
public Struct struct() {
- return super.struct()
+ final Struct ret = super.struct()
.put(SERVER_NAME_KEY, serverName)
.put(LOG_TIMESTAMP_KEY, sourceTime == null ? null : sourceTime.toEpochMilli())
.put(CHANGE_LSN_KEY, changeLsn.toString())
- .put(SNAPSHOT_KEY, false);
+ .put(SNAPSHOT_KEY, snapshot);
+
+ if (commitLsn != null) {
+ ret.put(COMMIT_LSN_KEY, commitLsn.toString());
+ }
+ return ret;
}
}
diff --git a/debezium-connector-sqlserver/src/main/java/io/debezium/connector/sqlserver/SqlServerChangeEventSourceFactory.java b/debezium-connector-sqlserver/src/main/java/io/debezium/connector/sqlserver/SqlServerChangeEventSourceFactory.java
index e2f6f9070..36adaa6f6 100644
--- a/debezium-connector-sqlserver/src/main/java/io/debezium/connector/sqlserver/SqlServerChangeEventSourceFactory.java
+++ b/debezium-connector-sqlserver/src/main/java/io/debezium/connector/sqlserver/SqlServerChangeEventSourceFactory.java
@@ -11,6 +11,7 @@
import io.debezium.pipeline.source.spi.SnapshotChangeEventSource;
import io.debezium.pipeline.source.spi.StreamingChangeEventSource;
import io.debezium.pipeline.spi.OffsetContext;
+import io.debezium.relational.TableId;
import io.debezium.util.Clock;
public class SqlServerChangeEventSourceFactory implements ChangeEventSourceFactory {
@@ -18,12 +19,12 @@ public class SqlServerChangeEventSourceFactory implements ChangeEventSourceFacto
private final SqlServerConnectorConfig configuration;
private final SqlServerConnection jdbcConnection;
private final ErrorHandler errorHandler;
- private final EventDispatcher dispatcher;
+ private final EventDispatcher dispatcher;
private final Clock clock;
private final SqlServerDatabaseSchema schema;
public SqlServerChangeEventSourceFactory(SqlServerConnectorConfig configuration, SqlServerConnection jdbcConnection,
- ErrorHandler errorHandler, EventDispatcher dispatcher, Clock clock, SqlServerDatabaseSchema schema) {
+ ErrorHandler errorHandler, EventDispatcher dispatcher, Clock clock, SqlServerDatabaseSchema schema) {
this.configuration = configuration;
this.jdbcConnection = jdbcConnection;
this.errorHandler = errorHandler;
@@ -34,7 +35,7 @@ public SqlServerChangeEventSourceFactory(SqlServerConnectorConfig configuration,
@Override
public SnapshotChangeEventSource getSnapshotChangeEventSource(OffsetContext offsetContext) {
- return new SqlServerSnapshotChangeEventSource(configuration, (SqlServerOffsetContext) offsetContext, jdbcConnection, schema);
+ return new SqlServerSnapshotChangeEventSource(configuration, (SqlServerOffsetContext) offsetContext, jdbcConnection, schema, dispatcher, clock);
}
@Override
diff --git a/debezium-connector-sqlserver/src/main/java/io/debezium/connector/sqlserver/SqlServerConnection.java b/debezium-connector-sqlserver/src/main/java/io/debezium/connector/sqlserver/SqlServerConnection.java
index 48f8a6765..3685b8b0b 100644
--- a/debezium-connector-sqlserver/src/main/java/io/debezium/connector/sqlserver/SqlServerConnection.java
+++ b/debezium-connector-sqlserver/src/main/java/io/debezium/connector/sqlserver/SqlServerConnection.java
@@ -36,6 +36,7 @@ public class SqlServerConnection extends JdbcConnection {
private static final String ENABLE_TABLE_CDC;
private static final String CDC_WRAPPERS_DML;
private static final String GET_MAX_LSN;
+ private static final String LOCK_TABLE;
static {
try {
@@ -46,6 +47,7 @@ public class SqlServerConnection extends JdbcConnection {
DISABLE_DB_CDC = statements.getProperty("disable_cdc_for_db");
ENABLE_TABLE_CDC = statements.getProperty("enable_cdc_for_table");
GET_MAX_LSN = statements.getProperty("get_max_lsn");
+ LOCK_TABLE = statements.getProperty("lock_table");
CDC_WRAPPERS_DML = IoUtil.read(classLoader.getResourceAsStream("generate_cdc_wrappers.sql"));
}
catch (Exception e) {
@@ -183,6 +185,11 @@ public Instant timestampOfLsn(Lsn lsn) throws SQLException {
});
}
+ public void lockTable(TableId tableId) throws SQLException {
+ final String lockTableStmt = LOCK_TABLE.replace(STATEMENTS_PLACEHOLDER, tableId.table());
+ execute(lockTableStmt);
+ }
+
private String cdcNameForTable(TableId tableId) {
return tableId.schema() + '_' + tableId.table();
}
diff --git a/debezium-connector-sqlserver/src/main/java/io/debezium/connector/sqlserver/SqlServerConnectorConfig.java b/debezium-connector-sqlserver/src/main/java/io/debezium/connector/sqlserver/SqlServerConnectorConfig.java
index a740edc26..60db4d130 100644
--- a/debezium-connector-sqlserver/src/main/java/io/debezium/connector/sqlserver/SqlServerConnectorConfig.java
+++ b/debezium-connector-sqlserver/src/main/java/io/debezium/connector/sqlserver/SqlServerConnectorConfig.java
@@ -13,8 +13,11 @@
import io.debezium.config.CommonConnectorConfig;
import io.debezium.config.Configuration;
+import io.debezium.config.EnumeratedValue;
import io.debezium.config.Field;
+import io.debezium.config.Field.ValidationOutput;
import io.debezium.document.Document;
+import io.debezium.heartbeat.Heartbeat;
import io.debezium.jdbc.JdbcConfiguration;
import io.debezium.relational.RelationalDatabaseConnectorConfig;
import io.debezium.relational.TableId;
@@ -31,8 +34,136 @@
*/
public class SqlServerConnectorConfig extends RelationalDatabaseConnectorConfig {
- // TODO pull up to RelationalConnectorConfig
- public static final String DATABASE_CONFIG_PREFIX = "database.";
+ /**
+ * The set of predefined SnapshotMode options or aliases.
+ */
+ public static enum SnapshotMode implements EnumeratedValue {
+
+ /**
+ * Perform a snapshot of data and schema upon initial startup of a connector.
+ */
+ INITIAL("initial", true),
+
+ /**
+ * Perform a snapshot of data and schema upon initial startup of a connector.
+ */
+ INITIAL_SCHEMA_ONLY("initial_schema_only", false);
+
+ private final String value;
+ private final boolean includeData;
+
+ private SnapshotMode(String value, boolean includeData) {
+ this.value = value;
+ this.includeData = includeData;
+ }
+
+ @Override
+ public String getValue() {
+ return value;
+ }
+
+ /**
+ * Whether this snapshotting mode should include the actual data or just the
+ * schema of captured tables.
+ */
+ public boolean includeData() {
+ return includeData;
+ }
+
+ /**
+ * Determine if the supplied value is one of the predefined options.
+ *
+ * @param value the configuration property value; may not be null
+ * @return the matching option, or null if no match is found
+ */
+ public static SnapshotMode parse(String value) {
+ if (value == null) {
+ return null;
+ }
+ value = value.trim();
+
+ for (SnapshotMode option : SnapshotMode.values()) {
+ if (option.getValue().equalsIgnoreCase(value)) return option;
+ }
+
+ return null;
+ }
+
+ /**
+ * Determine if the supplied value is one of the predefined options.
+ *
+ * @param value the configuration property value; may not be null
+ * @param defaultValue the default value; may be null
+ * @return the matching option, or null if no match is found and the non-null default is invalid
+ */
+ public static SnapshotMode parse(String value, String defaultValue) {
+ SnapshotMode mode = parse(value);
+
+ if (mode == null && defaultValue != null) {
+ mode = parse(defaultValue);
+ }
+
+ return mode;
+ }
+ }
+
+ /**
+ * The set of predefined Snapshot Locking Mode options.
+ */
+ public static enum SnapshotLockingMode implements EnumeratedValue {
+
+ /**
+ * This mode will block all reads and writes for the entire duration of the snapshot.
+ *
+ * The connector will execute {@code SELECT * FROM .. WITH (TABLOCKX)}
+ */
+ EXCLUSIVE("exclusive"),
+
+ /**
+ * This mode will avoid using ANY table locks during the snapshot process. This mode can only be used with SnapShotMode
+ * set to schema_only or schema_only_recovery.
+ */
+ NONE("none");
+
+ private final String value;
+
+ private SnapshotLockingMode(String value) {
+ this.value = value;
+ }
+
+ @Override
+ public String getValue() {
+ return value;
+ }
+
+ /**
+ * Determine if the supplied value is one of the predefined options.
+ *
+ * @param value the configuration property value; may not be null
+ * @return the matching option, or null if no match is found
+ */
+ public static SnapshotLockingMode parse(String value) {
+ if (value == null) return null;
+ value = value.trim();
+ for (SnapshotLockingMode option : SnapshotLockingMode.values()) {
+ if (option.getValue().equalsIgnoreCase(value)) return option;
+ }
+ return null;
+ }
+
+ /**
+ * Determine if the supplied value is one of the predefined options.
+ *
+ * @param value the configuration property value; may not be null
+ * @param defaultValue the default value; may be null
+ * @return the matching option, or null if no match is found and the non-null default is invalid
+ */
+ public static SnapshotLockingMode parse(String value, String defaultValue) {
+ SnapshotLockingMode mode = parse(value);
+ if (mode == null && defaultValue != null) mode = parse(defaultValue);
+ return mode;
+ }
+ }
public static final Field LOGICAL_NAME = Field.create("database.server.name")
.withDisplayName("Namespace")
@@ -40,11 +171,10 @@ public class SqlServerConnectorConfig extends RelationalDatabaseConnectorConfig
.withWidth(Width.MEDIUM)
.withImportance(Importance.HIGH)
.withValidation(Field::isRequired)
- // TODO
- //.withValidation(Field::isRequired, MySqlConnectorConfig::validateServerNameIsDifferentFromHistoryTopicName)
+ .withValidation(Field::isRequired, CommonConnectorConfig::validateServerNameIsDifferentFromHistoryTopicName)
.withDescription("Unique name that identifies the database server and all recorded offsets, and"
+ "that is used as a prefix for all schemas and topics. "
- + "Each distinct MySQL installation should have a separate namespace and monitored by "
+ + "Each distinct SQL Server installation should have a separate namespace and monitored by "
+ "at most one Debezium connector.");
public static final Field DATABASE_NAME = Field.create(DATABASE_CONFIG_PREFIX + JdbcConfiguration.DATABASE)
@@ -71,39 +201,68 @@ public class SqlServerConnectorConfig extends RelationalDatabaseConnectorConfig
+ DatabaseHistory.CONFIGURATION_FIELD_PREFIX_STRING + "' string.")
.withDefault(KafkaDatabaseHistory.class.getName());
+ public static final Field SNAPSHOT_MODE = Field.create("snapshot.mode")
+ .withDisplayName("Snapshot mode")
+ .withEnum(SnapshotMode.class, SnapshotMode.INITIAL)
+ .withWidth(Width.SHORT)
+ .withImportance(Importance.LOW)
+ .withDescription("The criteria for running a snapshot upon startup of the connector. "
+ + "Options include: "
+ + "'initial' (the default) to specify the connector should run a snapshot only when no offsets are available for the logical server name; "
+ + "'initial_schema_only' to specify the connector should run a snapshot of the schema when no offsets are available for the logical server name. ");
+
+ public static final Field SNAPSHOT_LOCKING_MODE = Field.create("snapshot.locking.mode")
+ .withDisplayName("Snapshot locking mode")
+ .withEnum(SnapshotLockingMode.class, SnapshotLockingMode.EXCLUSIVE)
+ .withWidth(Width.SHORT)
+ .withImportance(Importance.LOW)
+ .withDescription("Controls how long the connector locks the montiored tables for snapshot execution. The default is '" + SnapshotLockingMode.EXCLUSIVE.getValue() + "', "
+ + "which means that the connector holds the exlusive lock (and thus prevents any reads and updates) for all monitored tables "
+ + "while the database schemas, other metadata and the data itself are being read. Using a value of '" + SnapshotLockingMode.NONE.getValue() + "' will prevent the connector from acquiring any "
+ + "table locks during the snapshot process. This mode can only be used in combination with snapshot.mode values of '" + SnapshotMode.INITIAL_SCHEMA_ONLY.getValue() + "' or "
+ + "'schema_only_recovery' and is only safe to use if no schema changes are happening while the snapshot is taken.")
+ .withValidation(SqlServerConnectorConfig::validateSnapshotLockingMode);
+
/**
* The set of {@link Field}s defined as part of this configuration.
*/
public static Field.Set ALL_FIELDS = Field.setOf(
LOGICAL_NAME,
DATABASE_NAME,
+ SNAPSHOT_MODE,
RelationalDatabaseConnectorConfig.TABLE_WHITELIST,
RelationalDatabaseConnectorConfig.TABLE_BLACKLIST,
RelationalDatabaseConnectorConfig.TABLE_IGNORE_BUILTIN,
CommonConnectorConfig.POLL_INTERVAL_MS,
CommonConnectorConfig.MAX_BATCH_SIZE,
- CommonConnectorConfig.MAX_QUEUE_SIZE
+ CommonConnectorConfig.MAX_QUEUE_SIZE,
+ Heartbeat.HEARTBEAT_INTERVAL, Heartbeat.HEARTBEAT_TOPICS_PREFIX
);
+ public static ConfigDef configDef() {
+ ConfigDef config = new ConfigDef();
+
+ Field.group(config, "SQL Server", LOGICAL_NAME, DATABASE_NAME, SNAPSHOT_MODE);
+ Field.group(config, "Events", RelationalDatabaseConnectorConfig.TABLE_WHITELIST,
+ RelationalDatabaseConnectorConfig.TABLE_BLACKLIST,
+ RelationalDatabaseConnectorConfig.TABLE_IGNORE_BUILTIN,
+ Heartbeat.HEARTBEAT_INTERVAL, Heartbeat.HEARTBEAT_TOPICS_PREFIX
+ );
+ Field.group(config, "Connector", CommonConnectorConfig.POLL_INTERVAL_MS, CommonConnectorConfig.MAX_BATCH_SIZE, CommonConnectorConfig.MAX_QUEUE_SIZE);
+
+ return config;
+ }
+
private final String databaseName;
+ private final SnapshotMode snapshotMode;
+ private final SnapshotLockingMode snapshotLockingMode;
public SqlServerConnectorConfig(Configuration config) {
super(config, config.getString(LOGICAL_NAME), new SystemTablesPredicate());
this.databaseName = config.getString(DATABASE_NAME);
- }
-
- public static ConfigDef configDef() {
- ConfigDef config = new ConfigDef();
-
- Field.group(config, "Oracle", LOGICAL_NAME, DATABASE_NAME);
- Field.group(config, "Events", RelationalDatabaseConnectorConfig.TABLE_WHITELIST,
- RelationalDatabaseConnectorConfig.TABLE_BLACKLIST,
- RelationalDatabaseConnectorConfig.TABLE_IGNORE_BUILTIN
- );
- Field.group(config, "Connector", CommonConnectorConfig.POLL_INTERVAL_MS, CommonConnectorConfig.MAX_BATCH_SIZE, CommonConnectorConfig.MAX_QUEUE_SIZE);
-
- return config;
+ this.snapshotMode = SnapshotMode.parse(config.getString(SNAPSHOT_MODE), SNAPSHOT_MODE.defaultValueAsString());
+ this.snapshotLockingMode = SnapshotLockingMode.parse(config.getString(SNAPSHOT_LOCKING_MODE), SNAPSHOT_LOCKING_MODE.defaultValueAsString());
}
public String getDatabaseName() {
@@ -139,6 +298,41 @@ protected boolean isPositionAtOrBefore(Document recorded, Document desired) {
return databaseHistory;
}
+ public SnapshotLockingMode getSnapshotLockingMode() {
+ return this.snapshotLockingMode;
+ }
+
+ public SnapshotMode getSnapshotMode() {
+ return snapshotMode;
+ }
+
+ /**
+ * Validate the snapshot.locking.mode configuration
+ * The {@link SnapshotLockingMode.NONE} is allowed only for snapshot mode {@link SnapshotMode.INITIAL_SCHEMA_ONLY}
+ *
+ * @param config connector configuration
+ * @param field validated field (snapshot locking mode)
+ * @param problems the list of violated validations
+ *
+ * @return 0 for valid configuration
+ */
+ private static int validateSnapshotLockingMode(Configuration config, Field field, ValidationOutput problems) {
+ final SnapshotMode snapshotMode = SnapshotMode.parse(config.getString(SNAPSHOT_MODE), SNAPSHOT_MODE.defaultValueAsString());
+ final SnapshotLockingMode snapshotLockingMode = SnapshotLockingMode.parse(config.getString(SNAPSHOT_LOCKING_MODE), SNAPSHOT_LOCKING_MODE.defaultValueAsString());
+
+ if (snapshotLockingMode == SnapshotLockingMode.NONE) {
+ if (snapshotMode != SnapshotMode.INITIAL_SCHEMA_ONLY) {
+ problems.accept(
+ field,
+ snapshotLockingMode,
+ "Snapshot locking mode '" + snapshotLockingMode.getValue() + "' is not allowed for snapshot mode '" + snapshotMode.getValue() + "'"
+ );
+ }
+ }
+ // Everything checks out ok.
+ return 0;
+ }
+
private static class SystemTablesPredicate implements TableFilter {
@Override
diff --git a/debezium-connector-sqlserver/src/main/java/io/debezium/connector/sqlserver/SqlServerConnectorTask.java b/debezium-connector-sqlserver/src/main/java/io/debezium/connector/sqlserver/SqlServerConnectorTask.java
index 2cd003a50..27f68801e 100644
--- a/debezium-connector-sqlserver/src/main/java/io/debezium/connector/sqlserver/SqlServerConnectorTask.java
+++ b/debezium-connector-sqlserver/src/main/java/io/debezium/connector/sqlserver/SqlServerConnectorTask.java
@@ -6,6 +6,7 @@
package io.debezium.connector.sqlserver;
import java.sql.SQLException;
+import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicReference;
@@ -24,7 +25,9 @@
import io.debezium.pipeline.DataChangeEvent;
import io.debezium.pipeline.ErrorHandler;
import io.debezium.pipeline.EventDispatcher;
+import io.debezium.pipeline.spi.OffsetContext;
import io.debezium.relational.TableId;
+import io.debezium.schema.TopicSelector;
import io.debezium.util.Clock;
import io.debezium.util.SchemaNameAdjuster;
@@ -47,7 +50,7 @@ private static enum State {
private final AtomicReference state = new AtomicReference(State.STOPPED);
private volatile SqlServerTaskContext taskContext;
- private volatile ChangeEventQueue