-
Notifications
You must be signed in to change notification settings - Fork 3k
Kafka Connect: validate offsets for refreshed table state on commit #14510
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
Changes from 1 commit
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -27,6 +27,7 @@ | |
| import java.util.Collection; | ||
| import java.util.List; | ||
| import java.util.Map; | ||
| import java.util.Objects; | ||
| import java.util.concurrent.ExecutorService; | ||
| import java.util.concurrent.LinkedBlockingQueue; | ||
| import java.util.concurrent.ThreadPoolExecutor; | ||
|
|
@@ -41,6 +42,7 @@ | |
| import org.apache.iceberg.DeleteFile; | ||
| import org.apache.iceberg.RowDelta; | ||
| import org.apache.iceberg.Snapshot; | ||
| import org.apache.iceberg.SnapshotAncestryValidator; | ||
| import org.apache.iceberg.Table; | ||
| import org.apache.iceberg.catalog.Catalog; | ||
| import org.apache.iceberg.catalog.TableIdentifier; | ||
|
|
@@ -52,9 +54,10 @@ | |
| import org.apache.iceberg.connect.events.StartCommit; | ||
| import org.apache.iceberg.connect.events.TableReference; | ||
| import org.apache.iceberg.exceptions.NoSuchTableException; | ||
| import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; | ||
| import org.apache.iceberg.relocated.com.google.common.collect.Maps; | ||
| import org.apache.iceberg.relocated.com.google.common.collect.Streams; | ||
| import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder; | ||
| import org.apache.iceberg.util.SnapshotUtil; | ||
| import org.apache.iceberg.util.Tasks; | ||
| import org.apache.kafka.clients.admin.MemberDescription; | ||
| import org.apache.kafka.connect.errors.ConnectException; | ||
|
|
@@ -251,7 +254,8 @@ private void commitToTable( | |
| } else { | ||
| String taskId = String.format("%s-%s", config.connectorName(), config.taskId()); | ||
| if (deleteFiles.isEmpty()) { | ||
| AppendFiles appendOp = table.newAppend(); | ||
| AppendFiles appendOp = | ||
| table.newAppend().validateWith(offsetValidator(tableIdentifier, committedOffsets)); | ||
| if (branch != null) { | ||
| appendOp.toBranch(branch); | ||
| } | ||
|
|
@@ -264,7 +268,8 @@ private void commitToTable( | |
| dataFiles.forEach(appendOp::appendFile); | ||
| appendOp.commit(); | ||
| } else { | ||
| RowDelta deltaOp = table.newRowDelta(); | ||
| RowDelta deltaOp = | ||
| table.newRowDelta().validateWith(offsetValidator(tableIdentifier, committedOffsets)); | ||
| if (branch != null) { | ||
| deltaOp.toBranch(branch); | ||
| } | ||
|
|
@@ -296,6 +301,32 @@ private void commitToTable( | |
| } | ||
| } | ||
|
|
||
| private SnapshotAncestryValidator offsetValidator( | ||
| TableIdentifier tableIdentifier, Map<Integer, Long> expectedOffsets) { | ||
|
|
||
| return new SnapshotAncestryValidator() { | ||
| private Map<Integer, Long> lastCommittedOffsets; | ||
|
|
||
| @Override | ||
| public Boolean apply(Iterable<Snapshot> baseSnapshots) { | ||
| lastCommittedOffsets = lastCommittedOffsets(baseSnapshots); | ||
|
|
||
| if (expectedOffsets.isEmpty() && lastCommittedOffsets.isEmpty()) { | ||
| return true; // there are no stored offsets, so assume we're starting with new offsets | ||
| } | ||
|
|
||
| return expectedOffsets.equals(lastCommittedOffsets); | ||
| } | ||
|
|
||
| @Override | ||
| public String errorMessage() { | ||
| return String.format( | ||
| "Latest offsets do not match expected offsets for this commit. Table: %s, Expected: %s, Last Committed: %s", | ||
|
||
| tableIdentifier, expectedOffsets, lastCommittedOffsets); | ||
| } | ||
| }; | ||
| } | ||
|
|
||
| private <T> Predicate<T> distinctByKey(Function<? super T, ?> keyExtractor) { | ||
| Map<Object, Boolean> seen = Maps.newConcurrentMap(); | ||
| return t -> seen.putIfAbsent(keyExtractor.apply(t), Boolean.TRUE) == null; | ||
|
|
@@ -310,21 +341,37 @@ private Snapshot latestSnapshot(Table table, String branch) { | |
|
|
||
| private Map<Integer, Long> lastCommittedOffsetsForTable(Table table, String branch) { | ||
| Snapshot snapshot = latestSnapshot(table, branch); | ||
| while (snapshot != null) { | ||
| Map<String, String> summary = snapshot.summary(); | ||
| String value = summary.get(snapshotOffsetsProp); | ||
| if (value != null) { | ||
| TypeReference<Map<Integer, Long>> typeRef = new TypeReference<Map<Integer, Long>>() {}; | ||
| try { | ||
| return MAPPER.readValue(value, typeRef); | ||
| } catch (IOException e) { | ||
| throw new UncheckedIOException(e); | ||
| } | ||
| } | ||
| Long parentSnapshotId = snapshot.parentId(); | ||
| snapshot = parentSnapshotId != null ? table.snapshot(parentSnapshotId) : null; | ||
|
|
||
| if (snapshot == null) { | ||
| return Map.of(); | ||
| } | ||
|
|
||
| Iterable<Snapshot> branchAncestry = | ||
| SnapshotUtil.ancestorsOf(snapshot.snapshotId(), table::snapshot); | ||
| return lastCommittedOffsets(branchAncestry); | ||
| } | ||
|
|
||
| private Map<Integer, Long> lastCommittedOffsets(Iterable<Snapshot> snapshots) { | ||
| return Streams.stream(snapshots) | ||
| .filter(Objects::nonNull) | ||
danielcweeks marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| .filter(snapshot -> snapshot.summary().containsKey(snapshotOffsetsProp)) | ||
| .map(snapshot -> snapshot.summary().get(snapshotOffsetsProp)) | ||
| .map(this::parseOffsets) | ||
| .findFirst() | ||
| .orElseGet(Map::of); | ||
| } | ||
|
|
||
| private Map<Integer, Long> parseOffsets(String value) { | ||
| if (value == null) { | ||
| return Map.of(); | ||
| } | ||
danielcweeks marked this conversation as resolved.
Show resolved
Hide resolved
|
||
|
|
||
| TypeReference<Map<Integer, Long>> typeRef = new TypeReference<Map<Integer, Long>>() {}; | ||
| try { | ||
| return MAPPER.readValue(value, typeRef); | ||
| } catch (IOException e) { | ||
| throw new UncheckedIOException(e); | ||
| } | ||
| return ImmutableMap.of(); | ||
| } | ||
|
|
||
| void terminate() { | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -32,6 +32,7 @@ | |
| import org.apache.iceberg.FileFormat; | ||
| import org.apache.iceberg.PartitionSpec; | ||
| import org.apache.iceberg.Snapshot; | ||
| import org.apache.iceberg.Table; | ||
| import org.apache.iceberg.connect.events.AvroUtil; | ||
| import org.apache.iceberg.connect.events.CommitComplete; | ||
| import org.apache.iceberg.connect.events.CommitToTable; | ||
|
|
@@ -253,4 +254,45 @@ public void testCoordinatorCommittedOffsetMerging() { | |
| assertThat(table.currentSnapshot().summary()) | ||
| .containsEntry(OFFSETS_SNAPSHOT_PROP, "{\"0\":3,\"1\":7}"); | ||
| } | ||
|
|
||
| @Test | ||
| public void testCoordinatorCommittedOffsetValidation() { | ||
| // This test demonstrates that the Coordinator's validateAndCommit method | ||
| // prevents commits when another independent commit has updated the offsets | ||
| // during the commit process | ||
|
|
||
| // Set the initial offsets | ||
| table | ||
| .newAppend() | ||
| .appendFile(EventTestUtil.createDataFile()) | ||
| .set(OFFSETS_SNAPSHOT_PROP, "{\"0\":1}") | ||
| .commit(); | ||
|
|
||
| Table frozenTable = catalog.loadTable(TABLE_IDENTIFIER); | ||
|
|
||
| // return the original table state on the first load, so that the update will happen | ||
| // during the commit refresh | ||
| when(catalog.loadTable(TABLE_IDENTIFIER)).thenReturn(frozenTable).thenCallRealMethod(); | ||
|
|
||
| // Independently update the offsets | ||
| table | ||
| .newAppend() | ||
| .appendFile(EventTestUtil.createDataFile()) | ||
| .set(OFFSETS_SNAPSHOT_PROP, "{\"0\":7}") | ||
| .commit(); | ||
|
|
||
| table.refresh(); | ||
|
||
| assertThat(table.snapshots()).hasSize(2); | ||
| Snapshot firstSnapshot = table.currentSnapshot(); | ||
| assertThat(firstSnapshot.summary()).containsEntry(OFFSETS_SNAPSHOT_PROP, "{\"0\":7}"); | ||
|
|
||
| // Trigger commit to the table | ||
| coordinatorTest( | ||
| ImmutableList.of(EventTestUtil.createDataFile()), ImmutableList.of(), EventTestUtil.now()); | ||
|
|
||
| // Assert that the table was not updated and offsets remain | ||
| table.refresh(); | ||
| assertThat(table.snapshots()).hasSize(2); | ||
singhpk234 marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| assertThat(firstSnapshot.summary()).containsEntry(OFFSETS_SNAPSHOT_PROP, "{\"0\":7}"); | ||
| } | ||
| } | ||
Uh oh!
There was an error while loading. Please reload this page.