Skip to content
Closed
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
10 changes: 8 additions & 2 deletions core/src/main/java/org/apache/iceberg/SnapshotProducer.java
Original file line number Diff line number Diff line change
Expand Up @@ -87,7 +87,7 @@
* and {@link TableProperties#COMMIT_NUM_RETRIES_DEFAULT} properties.
*/
@SuppressWarnings("UnnecessaryAnonymousClass")
abstract class SnapshotProducer<ThisT> implements SnapshotUpdate<ThisT> {
abstract class SnapshotProducer<ThisT> implements SnapshotUpdate<ThisT>, SupportsCommitValidation {
private static final Logger LOG = LoggerFactory.getLogger(SnapshotProducer.class);
static final int MIN_FILE_GROUP_SIZE = 10_000;
static final Set<ManifestFile> EMPTY_SET = Sets.newHashSet();
Expand Down Expand Up @@ -421,8 +421,13 @@ protected TableMetadata refresh() {
}

@Override
@SuppressWarnings("checkstyle:CyclomaticComplexity")
public void commit() {
commit(NON_VALIDATING);
}

@Override
@SuppressWarnings("checkstyle:CyclomaticComplexity")
public void commit(CommitValidator validator) {
// this is always set to the latest commit attempt's snapshot
AtomicReference<Snapshot> stagedSnapshot = new AtomicReference<>();
try (Timed ignore = commitMetrics().totalDuration().start()) {
Expand Down Expand Up @@ -464,6 +469,7 @@ public void commit() {
// this operation retries
// to ensure that if a concurrent operation assigns the UUID, this operation will
// not fail.
validator.accept(base, updated);
taskOps.commit(base, updated.withUUID());
});

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,33 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.iceberg;

import java.util.function.BiConsumer;

public interface SupportsCommitValidation {
CommitValidator NON_VALIDATING = (base, metadata) -> {};

@FunctionalInterface
interface CommitValidator extends BiConsumer<TableMetadata, TableMetadata> {
@Override
void accept(TableMetadata snapshot, TableMetadata snapshot2);
}

void commit(CommitValidator validator);
}
98 changes: 97 additions & 1 deletion core/src/test/java/org/apache/iceberg/TestSnapshotProducer.java
Original file line number Diff line number Diff line change
Expand Up @@ -19,10 +19,18 @@
package org.apache.iceberg;

import static org.assertj.core.api.Assertions.assertThat;
import static org.assertj.core.api.Assertions.assertThatThrownBy;

import java.io.IOException;
import org.apache.iceberg.SupportsCommitValidation.CommitValidator;
import org.apache.iceberg.exceptions.CommitFailedException;
import org.apache.iceberg.util.PropertyUtil;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.TestTemplate;
import org.junit.jupiter.api.extension.ExtendWith;

public class TestSnapshotProducer {
@ExtendWith(ParameterizedTestExtension.class)
public class TestSnapshotProducer extends TestBase {

@Test
public void testManifestFileGroupSize() {
Expand Down Expand Up @@ -74,4 +82,92 @@ private void assertManifestWriterCount(
int writerCount = SnapshotProducer.manifestWriterCount(workerPoolSize, fileCount);
assertThat(writerCount).as(errMsg).isEqualTo(expectedManifestWriterCount);
}

@TestTemplate
public void testCommitValidationPreventingCommit() throws IOException {
// Commit the first file
table.newAppend().appendFile(FILE_A).commit();

// Create a file with no records for testing
DataFile fileNoRecords =
DataFiles.builder(SPEC)
.withPath("/path/to/data-no-records.parquet")
.withFileSizeInBytes(100)
.withRecordCount(0) // File with no records
.build();

// Create a CommitValidator that will reject commits based on snapshot summary
CommitValidator validator =
(baseMetadata, newMetadata) -> {
long addedRecords =
PropertyUtil.propertyAsInt(
newMetadata.currentSnapshot().summary(), SnapshotSummary.ADDED_RECORDS_PROP, 0);
long addedFiles =
PropertyUtil.propertyAsInt(
newMetadata.currentSnapshot().summary(), SnapshotSummary.ADDED_FILES_PROP, 0);
// Reject if no records are added (empty file)
if (addedFiles >= 1 && addedRecords == 0) {
throw new CommitFailedException("Cannot add files with no records");
}
};

// Test that the validator rejects commits with no records
AppendFiles append1 = table.newAppend().appendFile(fileNoRecords);
assertThatThrownBy(() -> ((SupportsCommitValidation) append1).commit(validator))
.isInstanceOf(CommitFailedException.class)
.hasMessage("Cannot add files with no records");

// Verify the file was not committed
assertThat(table.currentSnapshot().allManifests(table.io())).hasSize(1);
assertThat(table.currentSnapshot().summary().get(SnapshotSummary.TOTAL_DATA_FILES_PROP))
.isEqualTo("1");

// Verify files were not committed
assertThat(table.currentSnapshot().summary().get(SnapshotSummary.TOTAL_DATA_FILES_PROP))
.isEqualTo("1");

// Test that a valid commit passes the validator (FILE_B has only 1 record)
AppendFiles append4 = table.newFastAppend().appendFile(FILE_B);
((SupportsCommitValidation) append4).commit(validator);

// Verify the file was committed successfully
assertThat(table.currentSnapshot().summary().get(SnapshotSummary.TOTAL_DATA_FILES_PROP))
.isEqualTo("2");
assertThat(table.currentSnapshot().summary().get(SnapshotSummary.ADDED_FILES_PROP))
.isEqualTo("1");
assertThat(table.currentSnapshot().summary().get(SnapshotSummary.ADDED_RECORDS_PROP))
.isEqualTo("1");
}

@TestTemplate
public void testCommitValidationWithCustomSummaryProperties() throws IOException {
// Create a validator that checks custom summary properties
CommitValidator customPropertyValidator =
(baseMetadata, newMetadata) -> {
String operationType = newMetadata.currentSnapshot().summary().get("operation-type");
if ("restricted".equals(operationType)) {
throw new CommitFailedException("Restricted operation type not allowed");
}
};

// Add a file with a custom summary property that will be rejected
AppendFiles append1 =
table.newFastAppend().appendFile(FILE_A).set("operation-type", "restricted");

assertThatThrownBy(() -> ((SupportsCommitValidation) append1).commit(customPropertyValidator))
.isInstanceOf(CommitFailedException.class)
.hasMessage("Restricted operation type not allowed");

// Verify no snapshot was created
assertThat(table.currentSnapshot()).isNull();

// Add the file with an allowed operation type
AppendFiles append2 = table.newFastAppend().appendFile(FILE_A).set("operation-type", "allowed");

((SupportsCommitValidation) append2).commit(customPropertyValidator);

// Verify the snapshot was created with the custom property
assertThat(table.currentSnapshot()).isNotNull();
assertThat(table.currentSnapshot().summary().get("operation-type")).isEqualTo("allowed");
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -38,9 +38,13 @@
import org.apache.iceberg.ContentFile;
import org.apache.iceberg.DataFile;
import org.apache.iceberg.DeleteFile;
import org.apache.iceberg.PendingUpdate;
import org.apache.iceberg.RowDelta;
import org.apache.iceberg.Snapshot;
import org.apache.iceberg.SupportsCommitValidation;
import org.apache.iceberg.SupportsCommitValidation.CommitValidator;
import org.apache.iceberg.Table;
import org.apache.iceberg.TableMetadata;
import org.apache.iceberg.catalog.Catalog;
import org.apache.iceberg.catalog.TableIdentifier;
import org.apache.iceberg.connect.IcebergSinkConfig;
Expand All @@ -50,6 +54,7 @@
import org.apache.iceberg.connect.events.Event;
import org.apache.iceberg.connect.events.StartCommit;
import org.apache.iceberg.connect.events.TableReference;
import org.apache.iceberg.exceptions.CommitFailedException;
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;
Expand Down Expand Up @@ -251,7 +256,7 @@ private void commitToTable(
appendOp.set(VALID_THROUGH_TS_SNAPSHOT_PROP, validThroughTs.toString());
}
dataFiles.forEach(appendOp::appendFile);
appendOp.commit();
validateAndCommit(appendOp, branch, committedOffsets);
} else {
RowDelta deltaOp = table.newRowDelta();
if (branch != null) {
Expand All @@ -264,7 +269,7 @@ private void commitToTable(
}
dataFiles.forEach(deltaOp::addRows);
deleteFiles.forEach(deltaOp::addDeletes);
deltaOp.commit();
validateAndCommit(deltaOp, branch, committedOffsets);
}

Long snapshotId = latestSnapshot(table, branch).snapshotId();
Expand All @@ -284,6 +289,29 @@ private void commitToTable(
}
}

private void validateAndCommit(
PendingUpdate<?> pendingUpdate, String branch, Map<Integer, Long> expectedOffsets) {
CommitValidator validator =
(base, metadata) -> {
Map<Integer, Long> lastCommittedOffsets = lastCommittedOffsetsForTable(base, branch);

if (expectedOffsets == null || expectedOffsets.isEmpty()) {
return; // there are no stored offsets, so assume we're starting with new offsets
}

if (!expectedOffsets.equals(lastCommittedOffsets)) {
throw new CommitFailedException(
"Latest offsets do not match expected offsets for this commit.");
}
};
Comment on lines +296 to +306
Copy link
Contributor

Choose a reason for hiding this comment

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

[doubt] wouldn't this be lastCommittedOffset ?
if expectedOffset is null and lastCommittedOffset is non empty then we should fail ?

Suggested change
Map<Integer, Long> lastCommittedOffsets = lastCommittedOffsetsForTable(base, branch);
if (expectedOffsets == null || expectedOffsets.isEmpty()) {
return; // there are no stored offsets, so assume we're starting with new offsets
}
if (!expectedOffsets.equals(lastCommittedOffsets)) {
throw new CommitFailedException(
"Latest offsets do not match expected offsets for this commit.");
}
};
Map<Integer, Long> lastCommittedOffsets = lastCommittedOffsetsForTable(base, branch);
if (lastCommittedOffsets == null || lastCommittedOffsets.isEmpty()) {
return; // there are no stored offsets, so assume we're starting with new offsets
}
// handle case for expectedOffsets being null too
if (!lastCommittedOffsets.equals(expectedOffsets)) {
throw new CommitFailedException(
"Committed offsets do not match expected offsets for this commit.");
}
};

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I debated this, but I'm not sure what the expected behavior should be in that case. It seems likely that we want to error in that case as well, but the scenario around it is less clear to me.


if (pendingUpdate instanceof SupportsCommitValidation) {
((SupportsCommitValidation) pendingUpdate).commit(validator);
} else {
pendingUpdate.commit();
}
}

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 @@ -296,6 +324,13 @@ private Snapshot latestSnapshot(Table table, String branch) {
return table.snapshot(branch);
}

private Snapshot latestSnapshot(TableMetadata metadata, String branch) {
if (branch == null) {
return metadata.currentSnapshot();
}
return metadata.snapshot(metadata.ref(branch).snapshotId());
}
Comment on lines +327 to +332
Copy link
Contributor

Choose a reason for hiding this comment

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

probably we can move this to SnapshotUtil


private Map<Integer, Long> lastCommittedOffsetsForTable(Table table, String branch) {
Snapshot snapshot = latestSnapshot(table, branch);
while (snapshot != null) {
Expand All @@ -315,6 +350,25 @@ private Map<Integer, Long> lastCommittedOffsetsForTable(Table table, String bran
return ImmutableMap.of();
}

private Map<Integer, Long> lastCommittedOffsetsForTable(TableMetadata metadata, String branch) {
Snapshot snapshot = latestSnapshot(metadata, branch);
while (snapshot != null) {
Copy link
Contributor

Choose a reason for hiding this comment

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

[optional] we can get an iterable of ancestors from the SnapshotUtils

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 ? metadata.snapshot(parentSnapshotId) : null;
}
return ImmutableMap.of();
}

void terminate() {
this.terminated = true;

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 @@ -229,4 +230,45 @@ public void testCoordinatorRunning() {
sourceConsumer.rebalance(ImmutableList.of(tp1));
assertThat(mockIcebergSinkTask.isCoordinatorRunning()).isFalse();
}

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