Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -42,11 +42,9 @@ public class MySqlConstantOffsetBackingStore extends MemoryOffsetBackingStore {
static final String ROW = "row";
static final String EVENT = "event";
static final String GTID_SET = "gtids";
static final String REPLICATION_CONNECTOR_NAME = "replication.connector.name";

private static final Gson GSON = new Gson();
// The key is hardcoded here
private static final ByteBuffer KEY =
StandardCharsets.UTF_8.encode("{\"schema\":null,\"payload\":[\"delta\",{\"server\":\"dummy\"}]}");

@Override
public void configure(WorkerConfig config) {
Expand All @@ -57,7 +55,12 @@ public void configure(WorkerConfig config) {
String rowStr = originalConfig.get(ROW);
String eventStr = originalConfig.get(EVENT);
String gtidSetStr = originalConfig.get(GTID_SET);
String replicationConnectorName = originalConfig.get(REPLICATION_CONNECTOR_NAME);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

good catch


ByteBuffer key =
StandardCharsets.UTF_8.encode("{\"schema\":null,\"payload\":[\""
+ replicationConnectorName
+ "\",{\"server\":\"dummy\"}]}");
Map<String, Object> offset = new HashMap<>();
if (!Strings.isNullOrEmpty(fileStr)) {
offset.put(FILE, fileStr);
Expand All @@ -82,6 +85,6 @@ public void configure(WorkerConfig config) {
return;
}

data.put(KEY, StandardCharsets.UTF_8.encode(GSON.toJson(offset)));
data.put(key, StandardCharsets.UTF_8.encode(GSON.toJson(offset)));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -49,7 +49,6 @@
import java.time.temporal.Temporal;
import java.util.Map;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
Expand Down Expand Up @@ -99,6 +98,7 @@ public void start(Offset offset) {
Collectors.toMap(t -> config.getDatabase() + "." + t.getTable(), t -> t));
Map<String, String> state = offset.get(); // state map is always not null
String isSnapshot = state.getOrDefault(MySqlConstantOffsetBackingStore.SNAPSHOT, "");
String replicationConnectorName = "delta" + context.getInstanceId();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit : "delta-" ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Since its internal name it is ok to keep it like the way we have. Also some properties from debezium don't accept special chars like -.

Configuration.Builder configBuilder = Configuration.create()
.with("connector.class", MySqlConnector.class.getName())
.with("offset.storage", MySqlConstantOffsetBackingStore.class.getName())
Expand All @@ -111,7 +111,7 @@ public void start(Offset offset) {
.with("event", state.getOrDefault(MySqlConstantOffsetBackingStore.EVENT, ""))
.with("gtids", state.getOrDefault(MySqlConstantOffsetBackingStore.GTID_SET, ""))
/* begin connector properties */
.with("name", "delta" + UUID.randomUUID().toString().replace("-", ""))
.with("name", replicationConnectorName)
.with("database.hostname", config.getHost())
.with("database.port", config.getPort())
.with("database.user", config.getUser())
Expand All @@ -121,7 +121,8 @@ public void start(Offset offset) {
.with("database.server.name", "dummy") // this is the kafka topic for hosted debezium - it doesn't matter
.with("database.serverTimezone", config.getServerTimezone())
.with("table.whitelist", String.join(",", sourceTableMap.keySet()))
.with("snapshot.mode", config.getReplicateExistingData() ? "initial" : "schema_only");
.with("snapshot.mode", config.getReplicateExistingData() ? "initial" : "schema_only")
.with(MySqlConstantOffsetBackingStore.REPLICATION_CONNECTOR_NAME, replicationConnectorName);;

if (config.getConsumerID() != null) {
// If not provided debezium will randomly pick integer between 5400 and 6400.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,8 +33,9 @@
*/
public class SqlServerConstantOffsetBackingStore extends MemoryOffsetBackingStore {
private static final Gson GSON = new Gson();
private static final String KEY = "{\"schema\":null,\"payload\":[\"delta\",{\"server\":\"dummy\"}]}";

static final String SNAPSHOT_COMPLETED = "snapshot_completed";
static final String REPLICATION_CONNECTOR_NAME = "replication.connector.name";

@Override
public void configure(WorkerConfig config) {
Expand All @@ -45,7 +46,8 @@ public void configure(WorkerConfig config) {
String commitStr = originalConfig.get(SourceInfo.COMMIT_LSN_KEY);
String snapshot = originalConfig.get(SourceInfo.SNAPSHOT_KEY);
String snapshotCompleted = originalConfig.get(SNAPSHOT_COMPLETED);

String replicationConnectorName = originalConfig.get(REPLICATION_CONNECTOR_NAME);
String key = "{\"schema\":null,\"payload\":[\"" + replicationConnectorName + "\",{\"server\":\"dummy\"}]}";
Map<String, Object> offset = new HashMap<>();
if (!changeStr.isEmpty()) {
offset.put(SourceInfo.CHANGE_LSN_KEY, changeStr);
Expand All @@ -67,6 +69,6 @@ public void configure(WorkerConfig config) {
}
byte[] offsetBytes = Bytes.toBytes(GSON.toJson(offset));

data.put(ByteBuffer.wrap(Bytes.toBytes(KEY)), ByteBuffer.wrap(offsetBytes));
data.put(ByteBuffer.wrap(Bytes.toBytes(key)), ByteBuffer.wrap(offsetBytes));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,6 @@
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
Expand Down Expand Up @@ -107,6 +106,7 @@ public void start(Offset offset) {
Map<String, String> state = offset.get(); // this will never be null
// offset config
String isSnapshotCompleted = state.getOrDefault(SqlServerConstantOffsetBackingStore.SNAPSHOT_COMPLETED, "");
String replicationConnectorName = "delta" + context.getInstanceId();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: "delta-" ?

Configuration.Builder configBuilder = Configuration.create()
.with("connector.class", SqlServerConnector.class.getName())
.with("offset.storage", SqlServerConstantOffsetBackingStore.class.getName())
Expand All @@ -117,7 +117,7 @@ public void start(Offset offset) {
.with("snapshot", state.getOrDefault(SourceInfo.SNAPSHOT_KEY, ""))
.with("snapshot_completed", isSnapshotCompleted)
/* begin connector properties */
.with("name", "delta" + UUID.randomUUID().toString().replace("-", ""))
.with("name", replicationConnectorName)
.with("database.hostname", config.getHost())
.with("database.port", config.getPort())
.with("database.user", config.getUser())
Expand All @@ -127,7 +127,8 @@ public void start(Offset offset) {
.with("table.whitelist", String.join(",", sourceTableMap.keySet()))
.with("database.server.name", "dummy") // this is the kafka topic for hosted debezium - it doesn't matter
.with("database.serverTimezone", config.getServerTimezone())
.with("snapshot.mode", config.getReplicateExistingData() ? "initial" : "schema_only");
.with("snapshot.mode", config.getReplicateExistingData() ? "initial" : "schema_only")
.with(SqlServerConstantOffsetBackingStore.REPLICATION_CONNECTOR_NAME, replicationConnectorName);

LOG.info("Overriding sql server connector configs with arguments {}", debeziumConnectorConfigs);
for (Map.Entry<String, String> entry: debeziumConnectorConfigs.entrySet()) {
Expand Down