Skip to content
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

Core: Add update event for rewrite manifests #12627

Merged
merged 1 commit into from
Mar 24, 2025
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
14 changes: 13 additions & 1 deletion core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@
import java.util.function.Predicate;
import java.util.stream.Collectors;
import org.apache.iceberg.encryption.EncryptedOutputFile;
import org.apache.iceberg.events.CreateSnapshotEvent;
import org.apache.iceberg.exceptions.RuntimeIOException;
import org.apache.iceberg.exceptions.ValidationException;
import org.apache.iceberg.io.InputFile;
Expand All @@ -52,6 +53,7 @@ public class BaseRewriteManifests extends SnapshotProducer<RewriteManifests>
private static final String REPLACED_MANIFESTS_COUNT = "manifests-replaced";
private static final String PROCESSED_ENTRY_COUNT = "entries-processed";

private final String tableName;
private final Map<Integer, PartitionSpec> specsById;
private final long manifestTargetSizeBytes;

Expand All @@ -71,8 +73,9 @@ public class BaseRewriteManifests extends SnapshotProducer<RewriteManifests>

private final SnapshotSummary.Builder summaryBuilder = SnapshotSummary.builder();

BaseRewriteManifests(TableOperations ops) {
BaseRewriteManifests(String tableName, TableOperations ops) {
super(ops);
this.tableName = tableName;
this.specsById = ops().current().specsById();
this.manifestTargetSizeBytes =
ops()
Expand Down Expand Up @@ -193,6 +196,15 @@ public List<ManifestFile> apply(TableMetadata base, Snapshot snapshot) {
return apply;
}

@Override
public Object updateEvent() {
long snapshotId = snapshotId();
Snapshot snapshot = ops().current().snapshot(snapshotId);
long sequenceNumber = snapshot.sequenceNumber();
return new CreateSnapshotEvent(
tableName, operation(), snapshotId, sequenceNumber, snapshot.summary());
}

private boolean requiresRewrite(Set<ManifestFile> currentManifests) {
if (clusterByFunc == null) {
// manifests are deleted and added directly so don't perform a rewrite
Expand Down
2 changes: 1 addition & 1 deletion core/src/main/java/org/apache/iceberg/BaseTable.java
Original file line number Diff line number Diff line change
Expand Up @@ -192,7 +192,7 @@ public RewriteFiles newRewrite() {

@Override
public RewriteManifests rewriteManifests() {
return new BaseRewriteManifests(ops).reportWith(reporter);
return new BaseRewriteManifests(name, ops).reportWith(reporter);
}

@Override
Expand Down
3 changes: 2 additions & 1 deletion core/src/main/java/org/apache/iceberg/BaseTransaction.java
Original file line number Diff line number Diff line change
Expand Up @@ -196,7 +196,8 @@ public RewriteFiles newRewrite() {
@Override
public RewriteManifests rewriteManifests() {
checkLastOperationCommitted("RewriteManifests");
RewriteManifests rewrite = new BaseRewriteManifests(transactionOps).reportWith(reporter);
RewriteManifests rewrite =
new BaseRewriteManifests(tableName, transactionOps).reportWith(reporter);
rewrite.deleteWith(enqueueDelete);
updates.add(rewrite);
return rewrite;
Expand Down
11 changes: 3 additions & 8 deletions core/src/test/java/org/apache/iceberg/TestCommitReporting.java
Original file line number Diff line number Diff line change
Expand Up @@ -196,14 +196,9 @@ public void addAndDeleteManifests() throws IOException {

CommitReport report = reporter.lastCommitReport();
assertThat(report).isNotNull();
assertThat(report.operation()).isEqualTo("append");
assertThat(report.snapshotId()).isEqualTo(2L);
assertThat(report.sequenceNumber()).isEqualTo(2L);
assertThat(report.operation()).isEqualTo("replace");
assertThat(report.snapshotId()).isEqualTo(3L);
assertThat(report.sequenceNumber()).isEqualTo(3L);
assertThat(report.tableName()).isEqualTo(tableName);

CommitMetricsResult metrics = report.commitMetrics();
Copy link
Contributor

Choose a reason for hiding this comment

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

I'd probably change this to

CommitMetricsResult metrics = report.commitMetrics();
assertThat(metrics.totalDataFiles().value()).isEqualTo(2L);
assertThat(metrics.totalRecords().value()).isEqualTo(2L);
assertThat(metrics.totalFilesSizeInBytes().value()).isEqualTo(20L);

so that we at least check that some metrics were recorded

Copy link
Contributor

Choose a reason for hiding this comment

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

I'm fine with the PR as-is. I'll follow up in a separate PR to add proper commit metrics for manifests kept/created/replaced

Copy link
Contributor

Choose a reason for hiding this comment

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

@bryanck I've opened #12630 to address ^

assertThat(metrics.addedDataFiles().value()).isEqualTo(1L);
assertThat(metrics.addedRecords().value()).isEqualTo(1L);
assertThat(metrics.addedFilesSizeInBytes().value()).isEqualTo(10L);
}
}