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 @@ -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;
Expand All @@ -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;
Expand All @@ -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;
Expand Down Expand Up @@ -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);
}
Expand All @@ -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);
}
Expand Down Expand Up @@ -296,6 +301,28 @@ 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);

return expectedOffsets.equals(lastCommittedOffsets);
}

@Override
public String errorMessage() {
return String.format(
"Cannot commit to %s, stale offsets: Expected: %s 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;
Expand All @@ -310,21 +337,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)
.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();
}

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() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyCollection;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.when;

import java.io.IOException;
Expand Down Expand Up @@ -92,7 +93,7 @@ private InMemoryCatalog initInMemoryCatalog() {
@BeforeEach
@SuppressWarnings("deprecation")
public void before() {
catalog = initInMemoryCatalog();
catalog = spy(initInMemoryCatalog());
catalog.createNamespace(NAMESPACE);
table = catalog.createTable(TABLE_IDENTIFIER, SCHEMA);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -253,4 +254,44 @@ 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();

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);
assertThat(firstSnapshot.summary()).containsEntry(OFFSETS_SNAPSHOT_PROP, "{\"0\":7}");
}
}