Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

DBZ-8207 Use whole composite key in incremental snapshot notifications #5831

Draft
wants to merge 1 commit into
base: main
Choose a base branch
from
Draft
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 @@ -17,6 +17,7 @@
import java.time.LocalDateTime;
import java.time.LocalTime;
import java.time.ZoneOffset;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
Expand All @@ -25,6 +26,8 @@
import org.apache.kafka.connect.data.Struct;
import org.apache.kafka.connect.source.SourceConnector;
import org.apache.kafka.connect.source.SourceRecord;
import org.assertj.core.api.Assertions;
import org.awaitility.Awaitility;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
Expand All @@ -39,6 +42,8 @@
import io.debezium.doc.FixFor;
import io.debezium.jdbc.JdbcConnection;
import io.debezium.junit.logging.LogInterceptor;
import io.debezium.pipeline.notification.channels.SinkNotificationChannel;
import io.debezium.pipeline.signal.channels.FileSignalChannel;
import io.debezium.pipeline.source.snapshot.incremental.AbstractIncrementalSnapshotWithSchemaChangesSupportTest;
import io.debezium.relational.RelationalDatabaseConnectorConfig;
import io.debezium.relational.TableId;
Expand Down Expand Up @@ -389,4 +394,46 @@ public void incrementalSnapshotOnly() throws Exception {
assertThat(dbChanges).contains(entry(i + 1, i));
}
}

@Test
@FixFor("DBZ-8207")
public void all4PksAreInNotification() throws Exception {
// Testing.Print.enable();

populate4PkTable();
startConnector(c -> c.with(FileSignalChannel.SIGNAL_FILE, signalsFile.toString())
.with(SinkNotificationChannel.NOTIFICATION_TOPIC, "io.debezium.notification")
.with(CommonConnectorConfig.NOTIFICATION_ENABLED_CHANNELS, "sink")
.with(CommonConnectorConfig.INCREMENTAL_SNAPSHOT_CHUNK_SIZE, 1),
loggingCompletion(), false);
waitForConnectorToStart();

sendAdHocSnapshotSignal(DATABASE.qualifiedTableName("a4"));

List<SourceRecord> inProgressNotifications = new ArrayList<>();
Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> {
consumeAvailableRecords(r -> {
if (r.topic().equals("io.debezium.notification")
&& ((Struct) r.value()).getString("type").equals("IN_PROGRESS")
&& ((Struct) r.value()).getString("aggregate_type").equals("Incremental Snapshot")) {
inProgressNotifications.add(r);
}
});
return inProgressNotifications.size() > 0;
});

assertThat(inProgressNotifications.size()).isGreaterThan(0);
SourceRecord sourceRecord = inProgressNotifications.get(0);
Assertions.assertThat(sourceRecord.topic()).isEqualTo("io.debezium.notification");
Assertions.assertThat(((Struct) sourceRecord.value()).getString("aggregate_type")).isEqualTo("Incremental Snapshot");
Assertions.assertThat(((Struct) sourceRecord.value()).getString("type")).isEqualTo("IN_PROGRESS");
Assertions.assertThat(((String) ((Struct) sourceRecord.value()).getMap("additional_data").get("last_processed_key"))).contains(",");
Assertions.assertThat(((String) ((Struct) sourceRecord.value()).getMap("additional_data").get("last_processed_key")).split(",").length).isEqualTo(4);
}

protected void populate4PkTable() throws SQLException {
try (JdbcConnection connection = databaseConnection()) {
populate4PkTable(connection, "a4");
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -346,10 +346,4 @@ public void insertInsertWatermarkingStrategy() throws Exception {
public void insertDeleteWatermarkingStrategy() throws Exception {
// test has not to be executed on read only
}

protected void populate4PkTable() throws SQLException {
try (JdbcConnection connection = databaseConnection()) {
populate4PkTable(connection, "a4");
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -7,6 +7,7 @@

import java.time.Clock;
import java.time.Instant;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
Expand Down Expand Up @@ -131,8 +132,11 @@ public <T extends DataCollectionId> void notifyInProgress(IncrementalSnapshotCon
Map.of(
DATA_COLLECTIONS, dataCollections,
CURRENT_COLLECTION_IN_PROGRESS, incrementalSnapshotContext.currentDataCollectionId().getId().identifier(),
MAXIMUM_KEY, Objects.toString(incrementalSnapshotContext.maximumKey().orElse(new Object[0])[0], "<null>"),
LAST_PROCESSED_KEY, Objects.toString(incrementalSnapshotContext.chunkEndPosititon()[0], "<null>")),
MAXIMUM_KEY, incrementalSnapshotContext.maximumKey().isPresent() ? Arrays.stream(incrementalSnapshotContext.maximumKey().get())
.map(x -> Objects.toString(x, "<null>")).collect(Collectors.joining(","))
: "\"<null>\"",
LAST_PROCESSED_KEY, Arrays.stream(incrementalSnapshotContext.chunkEndPosititon())
.map(x -> Objects.toString(x, "<null>")).collect(Collectors.joining(","))),
offsetContext),
Offsets.of(partition, offsetContext));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -147,8 +147,8 @@ public void notifyInProgress() {
"connector_name", "connector-test",
"data_collections", "db.inventory.product,db.inventory.customer",
"current_collection_in_progress", "db.inventory.product",
"maximum_key", "100",
"last_processed_key", "50"), clock.millis());
"maximum_key", "100,0,0",
"last_processed_key", "50,0,0"), clock.millis());

verify(notificationService).notify(eq(expectedNotification), any(Offsets.class));
}
Expand Down
Loading