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);
Copy link
Contributor

Choose a reason for hiding this comment

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

[optional] may be move this above comment since the comment is for line below

Copy link
Contributor

Choose a reason for hiding this comment

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

Should this be close to the other validate(TableMetadata currentMetadata, Snapshot snapshot) call on line 260?

taskOps.commit(base, updated.withUUID());
});

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,43 @@
/*
* 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;
import org.apache.iceberg.exceptions.CommitFailedException;

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

@FunctionalInterface
interface CommitValidator extends BiConsumer<TableMetadata, TableMetadata> {
@Override
void accept(TableMetadata base, TableMetadata current);
Copy link
Contributor

@aiborodin aiborodin Nov 5, 2025

Choose a reason for hiding this comment

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

Should this accept Snapshot as a parameter to avoid depending on core classes?
Or match the existing validation signature in SnapshotProducer:

 /**
   * Validate the current metadata.
   *
   * <p>Child operations can override this to add custom validation.
   *
   * @param currentMetadata current table metadata to validate
   * @param snapshot ending snapshot on the lineage which is being validated
   */
  protected void validate(TableMetadata currentMetadata, Snapshot snapshot) {}

}

/**
* Commits the updated metadata to the table taking a validator that will be called with the
* refreshed metadata and pending metadata.
* <p>
* The validator should throw a {@link CommitFailedException} if validation fails. Retries will still
Copy link
Contributor

Choose a reason for hiding this comment

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

Why not throw a ValidationException like the rest of the code, overriding SnapshotProducer::validate(TableMetadata currentMetadata, Snapshot snapshot)?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

We retry on CommitFailedException, so I think there's a question as to whether there's any scenario where a retry could result in a successful commit. I'm not clear whether that is a reasonable assumption.

Copy link
Contributor

@aiborodin aiborodin Nov 6, 2025

Choose a reason for hiding this comment

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

I think there's a question as to whether there's any scenario where a retry could result in a successful commit. I'm not clear whether that is a reasonable assumption.

It makes sense to retry a CommitFailedException in SnapshotProducer::commit() due to the nature of concurrent commits in Iceberg. But why would we want to retry logical validation failures?

I think the CommitValidator should throw a non-retryable ValidationException and the client (Kafka Connect, Flink) should then decide whether to retry or skip the commit.

* apply so subsequent attempts to commit will be validated until retries are exhausted.
*
* @param validator
*/
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");
}
}
Loading