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
@@ -0,0 +1,54 @@
/*
* 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.Function;
import javax.annotation.Nonnull;

/**
* Interface to support validating snapshot ancestry during the commit process.
Copy link
Contributor

Choose a reason for hiding this comment

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

minor/optional : This won't be called per commit process ? as not all updates produce a new snapshot, do we wanna say commits that produce snapshots ?

*
* <p>Validation will be called after the table metadata is refreshed to pick up any changes to the
* table state.
*/
@FunctionalInterface
public interface SnapshotAncestryValidator extends Function<Iterable<Snapshot>, Boolean> {

SnapshotAncestryValidator NON_VALIDATING = baseSnapshots -> true;

/**
* Validate the snapshots based on the refreshed table state.
*
* @param baseSnapshots ancestry of the base table metadata snapshots
* @return boolean for whether the update is valid
*/
@Override
Boolean apply(Iterable<Snapshot> baseSnapshots);
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.

There's no need to pass the full Snapshot history to fix the Kafka Connect issue. This method can accept only new Snapshots as implemented in #14515.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Unfortunately, I don't think what you have in #14515 is quite correct. There's no guarantee (and it's very commonly not the case) that the offsets will be in the prior commit.

Copy link
Contributor

@aiborodin aiborodin Nov 7, 2025

Choose a reason for hiding this comment

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

There's no guarantee (and it's very commonly not the case) that the offsets will be in the prior commit.

#14515 checks all new commits when configured with the starting snapshot id.
Wouldn't the current approach result in unnecessary checks of the whole Snapshot history on every validation run?

Copy link
Contributor

@aiborodin aiborodin Nov 7, 2025

Choose a reason for hiding this comment

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

I double-checked the implementation of SnapshotUtil::ancestorsOf() - the current approach is also efficient because it walks backwards from the latest snapshot, so we can short-cut early.

Copy link
Contributor

Choose a reason for hiding this comment

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

nit: wonder if just the primitive type boolean is fine ? this way it will always to true / false ?


/**
* Validation message that will be included when throwing {@link
* org.apache.iceberg.exceptions.ValidationException}
*
* @return message
*/
@Nonnull
default String errorMessage() {
return "error message not provided";
}
}
5 changes: 5 additions & 0 deletions api/src/main/java/org/apache/iceberg/SnapshotUpdate.java
Original file line number Diff line number Diff line change
Expand Up @@ -71,4 +71,9 @@ default ThisT toBranch(String branch) {
"Cannot commit to branch %s: %s does not support branch commits",
branch, this.getClass().getName()));
}

default ThisT validateWith(SnapshotAncestryValidator validator) {
throw new UnsupportedOperationException(
"Snapshot validation not supported by " + this.getClass().getName());
}
}
34 changes: 33 additions & 1 deletion core/src/main/java/org/apache/iceberg/SnapshotProducer.java
Original file line number Diff line number Diff line change
Expand Up @@ -56,6 +56,7 @@
import org.apache.iceberg.exceptions.CommitFailedException;
import org.apache.iceberg.exceptions.CommitStateUnknownException;
import org.apache.iceberg.exceptions.RuntimeIOException;
import org.apache.iceberg.exceptions.ValidationException;
import org.apache.iceberg.io.OutputFile;
import org.apache.iceberg.metrics.CommitMetrics;
import org.apache.iceberg.metrics.CommitMetricsResult;
Expand Down Expand Up @@ -117,6 +118,8 @@ public void accept(String file) {
private TableMetadata base;
private boolean stageOnly = false;
private Consumer<String> deleteFunc = defaultDelete;
private SnapshotAncestryValidator snapshotAncestryValidator =
SnapshotAncestryValidator.NON_VALIDATING;

private ExecutorService workerPool;
private String targetBranch = SnapshotRef.MAIN_BRANCH;
Expand Down Expand Up @@ -159,6 +162,20 @@ public ThisT scanManifestsWith(ExecutorService executorService) {
return self();
}

/**
* Set a validator to check snapshot ancestry before committing changes.
*
* <p>If there is no parent snapshot, an empty iterable will be supplied to the validator.
*
* @param validator a validator to check snapshot ancestry validity
* @return this for method chaining
*/
@Override
public ThisT validateWith(SnapshotAncestryValidator validator) {
this.snapshotAncestryValidator = validator;
return self();
}

protected TableOperations ops() {
return ops;
}
Expand Down Expand Up @@ -257,7 +274,8 @@ public Snapshot apply() {
long sequenceNumber = base.nextSequenceNumber();
Long parentSnapshotId = parentSnapshot == null ? null : parentSnapshot.snapshotId();

validate(base, parentSnapshot);
runValidations(parentSnapshot);

List<ManifestFile> manifests = apply(base, parentSnapshot);

OutputFile manifestList = manifestListPath();
Expand Down Expand Up @@ -327,6 +345,20 @@ public Snapshot apply() {
writer.toManifestListFile().encryptionKeyID());
}

private void runValidations(Snapshot parentSnapshot) {
validate(base, parentSnapshot);

// Validate snapshot ancestry
Iterable<Snapshot> snapshotAncestry =
parentSnapshot != null
? SnapshotUtil.ancestorsOf(parentSnapshot.snapshotId(), base::snapshot)
: List.of();

boolean valid = snapshotAncestryValidator.apply(snapshotAncestry);
ValidationException.check(
valid, "Snapshot ancestry validation failed: %s", snapshotAncestryValidator.errorMessage());
}

protected abstract Map<String, String> summary();

/** Returns the snapshot summary from the implementation and updates totals. */
Expand Down
86 changes: 85 additions & 1 deletion core/src/test/java/org/apache/iceberg/TestSnapshotProducer.java
Original file line number Diff line number Diff line change
Expand Up @@ -18,11 +18,22 @@
*/
package org.apache.iceberg;

import static org.apache.iceberg.SnapshotSummary.PUBLISHED_WAP_ID_PROP;
import static org.assertj.core.api.Assertions.assertThat;
import static org.assertj.core.api.Assertions.assertThatThrownBy;

import java.io.IOException;
import java.util.List;
import java.util.stream.Collectors;
import javax.annotation.Nonnull;
import org.apache.iceberg.exceptions.ValidationException;
import org.apache.iceberg.relocated.com.google.common.collect.Streams;
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 +85,77 @@ private void assertManifestWriterCount(
int writerCount = SnapshotProducer.manifestWriterCount(workerPoolSize, fileCount);
assertThat(writerCount).as(errMsg).isEqualTo(expectedManifestWriterCount);
}

@TestTemplate
public void testCommitValidationPreventsCommit() throws IOException {
table.newAppend().commit();
String validationMessage = "Validation force failed";

// Create a CommitValidator that will reject commits
SnapshotAncestryValidator validator =
new SnapshotAncestryValidator() {
@Override
public Boolean apply(Iterable<Snapshot> baseSnapshots) {
return false;
}

@Nonnull
@Override
public String errorMessage() {
return validationMessage;
}
};

// Test that the validator rejects commit
AppendFiles append1 = table.newAppend().validateWith(validator).appendFile(FILE_A);
assertThatThrownBy(append1::commit)
.isInstanceOf(ValidationException.class)
.hasMessage("Snapshot ancestry validation failed: " + validationMessage);

// Verify the file was not committed
assertThat(table.currentSnapshot().allManifests(table.io())).hasSize(0);
}

@TestTemplate
public void testCommitValidationWithCustomSummaryProperties() throws IOException {
String wapId = "wap-12345-staging-audit";

// Create a validator that checks custom summary properties
SnapshotAncestryValidator customPropertyValidator =
baseSnapshots -> {
List<String> publishedWapIds =
Streams.stream(baseSnapshots)
.filter(snapshot -> snapshot.summary().containsKey(PUBLISHED_WAP_ID_PROP))
.map(snapshot -> snapshot.summary().get(PUBLISHED_WAP_ID_PROP))
.collect(Collectors.toList());

return !publishedWapIds.contains(wapId);
};

// Add a file with and set a published WAP id
table
.newFastAppend()
.validateWith(customPropertyValidator)
.appendFile(FILE_A)
.set(PUBLISHED_WAP_ID_PROP, wapId)
.commit();

// Verify the current state of the table
assertThat(table.currentSnapshot().summary().get(PUBLISHED_WAP_ID_PROP)).isEqualTo(wapId);

// Attempt to add the same published WAP id
AppendFiles append2 =
table
.newFastAppend()
.validateWith(customPropertyValidator)
.appendFile(FILE_A)
.set(PUBLISHED_WAP_ID_PROP, wapId);

assertThatThrownBy(append2::commit)
.isInstanceOf(ValidationException.class)
.hasMessageContaining("Snapshot ancestry validation failed");

// Verify the table wasn't updated
assertThat(table.snapshots()).hasSize(1);
}
}