From 715dbbac1e650f4bba593e44ad32ba25d346ebc7 Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Thu, 9 Jan 2025 12:05:27 -0600 Subject: [PATCH 01/22] Core: Allow setting Row Lineage and Last-Row-ID in TableMetadata --- .../apache/iceberg/RewriteTablePathUtil.java | 4 +- .../org/apache/iceberg/TableMetadata.java | 59 +++++++++++++++++- .../apache/iceberg/TableMetadataParser.java | 20 +++++- .../org/apache/iceberg/util/JsonUtil.java | 7 +++ .../org/apache/iceberg/TestRowLineage.java | 57 +++++++++++++++++ .../org/apache/iceberg/TestTableMetadata.java | 61 ++++++++++++++----- 6 files changed, 188 insertions(+), 20 deletions(-) create mode 100644 core/src/test/java/org/apache/iceberg/TestRowLineage.java diff --git a/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java b/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java index f250d2e12289..1cf490421238 100644 --- a/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java +++ b/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java @@ -129,7 +129,9 @@ public static TableMetadata replacePaths( // TODO: update statistic file paths metadata.statisticsFiles(), metadata.partitionStatisticsFiles(), - metadata.changes()); + metadata.changes(), + metadata.rowLineage(), + metadata.lastRowId()); } private static Map updateProperties( diff --git a/core/src/main/java/org/apache/iceberg/TableMetadata.java b/core/src/main/java/org/apache/iceberg/TableMetadata.java index 4ba3bdf8d737..99cd8a054ffa 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadata.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadata.java @@ -56,6 +56,9 @@ public class TableMetadata implements Serializable { static final int INITIAL_SPEC_ID = 0; static final int INITIAL_SORT_ORDER_ID = 1; static final int INITIAL_SCHEMA_ID = 0; + static final int INITIAL_ROW_ID = 0; + static final boolean DEFAULT_ROW_LINEAGE = false; + static final int MIN_FORMAT_VERSION_ROW_LINEAGE = 3; private static final long ONE_MINUTE = TimeUnit.MINUTES.toMillis(1); @@ -262,6 +265,8 @@ public String toString() { private volatile Map snapshotsById; private volatile Map refs; private volatile boolean snapshotsLoaded; + private final Boolean rowLineage; + private final long lastRowId; @SuppressWarnings("checkstyle:CyclomaticComplexity") TableMetadata( @@ -288,7 +293,9 @@ public String toString() { Map refs, List statisticsFiles, List partitionStatisticsFiles, - List changes) { + List changes, + boolean rowLineage, + long lastRowId) { Preconditions.checkArgument( specs != null && !specs.isEmpty(), "Partition specs cannot be null or empty"); Preconditions.checkArgument( @@ -307,6 +314,10 @@ public String toString() { Preconditions.checkArgument( metadataFileLocation == null || changes.isEmpty(), "Cannot create TableMetadata with a metadata location and changes"); + Preconditions.checkArgument( + formatVersion >= MIN_FORMAT_VERSION_ROW_LINEAGE || !rowLineage, + "Cannot enable row lineage when Table Version is less than V3. Table Version is %d", + formatVersion); this.metadataFileLocation = metadataFileLocation; this.formatVersion = formatVersion; @@ -341,6 +352,10 @@ public String toString() { this.statisticsFiles = ImmutableList.copyOf(statisticsFiles); this.partitionStatisticsFiles = ImmutableList.copyOf(partitionStatisticsFiles); + // row lineage + this.rowLineage = rowLineage; + this.lastRowId = lastRowId; + HistoryEntry last = null; for (HistoryEntry logEntry : snapshotLog) { if (last != null) { @@ -563,6 +578,14 @@ public TableMetadata withUUID() { return new Builder(this).assignUUID().build(); } + public boolean rowLineage() { + return rowLineage; + } + + public long lastRowId() { + return lastRowId; + } + /** * Updates the schema * @@ -903,6 +926,8 @@ public static class Builder { private final Map> statisticsFiles; private final Map> partitionStatisticsFiles; private boolean suppressHistoricalSnapshots = false; + private boolean rowLineage; + private long lastRowId; // change tracking private final List changes; @@ -949,6 +974,8 @@ private Builder(int formatVersion) { this.schemasById = Maps.newHashMap(); this.specsById = Maps.newHashMap(); this.sortOrdersById = Maps.newHashMap(); + this.rowLineage = DEFAULT_ROW_LINEAGE; + this.lastRowId = INITIAL_ROW_ID; } private Builder(TableMetadata base) { @@ -982,6 +1009,9 @@ private Builder(TableMetadata base) { this.schemasById = Maps.newHashMap(base.schemasById); this.specsById = Maps.newHashMap(base.specsById); this.sortOrdersById = Maps.newHashMap(base.sortOrdersById); + + this.rowLineage = base.rowLineage; + this.lastRowId = base.lastRowId; } public Builder withMetadataLocation(String newMetadataLocation) { @@ -1468,6 +1498,29 @@ public Builder setPreviousFileLocation(String previousFileLocation) { return this; } + public Builder enableRowLineage() { + Preconditions.checkArgument( + formatVersion >= 3, + "Cannot use row lineage with format version %s. Only format version 3 or higher support row lineage", + formatVersion); + this.rowLineage = true; + return this; + } + + public Builder withLastRowId(long lastRowId) { + Preconditions.checkArgument( + this.rowLineage, "Cannot set last-row-id if row lineage is not enabled"); + Preconditions.checkArgument( + this.lastRowId <= lastRowId, + "Cannot decrease last-row-id, last-row-id must increase monotonically." + + " Current last-row-id (%s) is greater that the new last-row-id (%s)", + this.lastRowId, + lastRowId); + + this.lastRowId = lastRowId; + return this; + } + private boolean hasChanges() { return changes.size() != startingChangeCount || (discardChanges && !changes.isEmpty()) @@ -1535,7 +1588,9 @@ public TableMetadata build() { partitionStatisticsFiles.values().stream() .flatMap(List::stream) .collect(Collectors.toList()), - discardChanges ? ImmutableList.of() : ImmutableList.copyOf(changes)); + discardChanges ? ImmutableList.of() : ImmutableList.copyOf(changes), + rowLineage, + lastRowId); } private int addSchemaInternal(Schema schema, int newLastColumnId) { diff --git a/core/src/main/java/org/apache/iceberg/TableMetadataParser.java b/core/src/main/java/org/apache/iceberg/TableMetadataParser.java index d7f2b29be75a..33a794852651 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadataParser.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadataParser.java @@ -110,6 +110,8 @@ private TableMetadataParser() {} static final String METADATA_LOG = "metadata-log"; static final String STATISTICS = "statistics"; static final String PARTITION_STATISTICS = "partition-statistics"; + static final String ROW_LINEAGE = "row-lineage"; + static final String LAST_ROW_ID = "last-row-id"; public static void overwrite(TableMetadata metadata, OutputFile outputFile) { internalWrite(metadata, outputFile, true); @@ -220,6 +222,11 @@ public static void toJson(TableMetadata metadata, JsonGenerator generator) throw generator.writeNullField(CURRENT_SNAPSHOT_ID); } + if (metadata.rowLineage()) { + generator.writeBooleanField(ROW_LINEAGE, metadata.rowLineage()); + generator.writeNumberField(LAST_ROW_ID, metadata.lastRowId()); + } + toJson(metadata.refs(), generator); generator.writeArrayFieldStart(SNAPSHOTS); @@ -454,6 +461,15 @@ public static TableMetadata fromJson(String metadataLocation, JsonNode node) { currentSnapshotId = -1L; } + Boolean rowLineage = JsonUtil.getBoolOrNull(ROW_LINEAGE, node); + long lastRowId; + if (rowLineage != null && rowLineage) { + lastRowId = JsonUtil.getLong(LAST_ROW_ID, node); + } else { + rowLineage = TableMetadata.DEFAULT_ROW_LINEAGE; + lastRowId = TableMetadata.INITIAL_ROW_ID; + } + long lastUpdatedMillis = JsonUtil.getLong(LAST_UPDATED_MILLIS, node); Map refs; @@ -545,7 +561,9 @@ public static TableMetadata fromJson(String metadataLocation, JsonNode node) { refs, statisticsFiles, partitionStatisticsFiles, - ImmutableList.of() /* no changes from the file */); + ImmutableList.of() /* no changes from the file */, + rowLineage, + lastRowId); } private static Map refsFromJson(JsonNode refMap) { diff --git a/core/src/main/java/org/apache/iceberg/util/JsonUtil.java b/core/src/main/java/org/apache/iceberg/util/JsonUtil.java index 2810ff5f23c0..972b89c56717 100644 --- a/core/src/main/java/org/apache/iceberg/util/JsonUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/JsonUtil.java @@ -144,6 +144,13 @@ public static long getLong(String property, JsonNode node) { return pNode.asLong(); } + public static Boolean getBoolOrNull(String property, JsonNode node) { + if (!node.hasNonNull(property)) { + return null; + } + return getBool(property, node); + } + public static boolean getBool(String property, JsonNode node) { Preconditions.checkArgument(node.has(property), "Cannot parse missing boolean: %s", property); JsonNode pNode = node.get(property); diff --git a/core/src/test/java/org/apache/iceberg/TestRowLineage.java b/core/src/test/java/org/apache/iceberg/TestRowLineage.java new file mode 100644 index 000000000000..9d949de8dc8f --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/TestRowLineage.java @@ -0,0 +1,57 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assertions.assertThrows; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.FieldSource; + +public class TestRowLineage { + + private TableMetadata.Builder testMetadataBuilder() { + return TableMetadata.buildFromEmpty(TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE) + .enableRowLineage(); + } + + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + public void testRowLineageSupported(int formatVersion) { + if (formatVersion == TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE) { + assertThat(TableMetadata.buildFromEmpty(formatVersion).enableRowLineage()).isNotNull(); + } else { + IllegalArgumentException notSupported = + assertThrows( + IllegalArgumentException.class, + () -> TableMetadata.buildFromEmpty(formatVersion).enableRowLineage()); + assertThat(notSupported.getMessage()).contains("Cannot use row lineage"); + } + } + + @Test + public void testLastRowIdMustIncrease() { + assertThat(testMetadataBuilder().withLastRowId(TableMetadata.INITIAL_ROW_ID + 5)).isNotNull(); + IllegalArgumentException noDecrease = assertThrows( + IllegalArgumentException.class, + () -> testMetadataBuilder().withLastRowId(TableMetadata.INITIAL_ROW_ID - 5)); + assertThat(noDecrease.getMessage()).contains("Cannot decrease last-row-id"); + } +} diff --git a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java index 45aa211e5187..f46c54539498 100644 --- a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java +++ b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java @@ -28,6 +28,7 @@ import static org.apache.iceberg.TableMetadataParser.PROPERTIES; import static org.apache.iceberg.TableMetadataParser.SCHEMA; import static org.apache.iceberg.TableMetadataParser.SNAPSHOTS; +import static org.apache.iceberg.TestHelpers.MAX_FORMAT_VERSION; import static org.apache.iceberg.TestHelpers.assertSameSchemaList; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -162,7 +163,7 @@ public void testJsonConversion() throws Exception { TableMetadata expected = new TableMetadata( null, - 2, + MAX_FORMAT_VERSION, UUID.randomUUID().toString(), TEST_LOCATION, SEQ_NO, @@ -184,7 +185,9 @@ public void testJsonConversion() throws Exception { refs, statisticsFiles, partitionStatisticsFiles, - ImmutableList.of()); + ImmutableList.of(), + true, + 40); String asJson = TableMetadataParser.toJson(expected); TableMetadata metadata = TableMetadataParser.fromJson(asJson); @@ -217,6 +220,8 @@ public void testJsonConversion() throws Exception { assertThat(metadata.statisticsFiles()).isEqualTo(statisticsFiles); assertThat(metadata.partitionStatisticsFiles()).isEqualTo(partitionStatisticsFiles); assertThat(metadata.refs()).isEqualTo(refs); + assertThat(metadata.rowLineage()).isEqualTo(expected.rowLineage()); + assertThat(metadata.lastRowId()).isEqualTo(expected.lastRowId()); } @Test @@ -273,7 +278,9 @@ public void testBackwardCompat() throws Exception { ImmutableMap.of(), ImmutableList.of(), ImmutableList.of(), - ImmutableList.of()); + ImmutableList.of(), + false, + 0); String asJson = toJsonWithoutSpecAndSchemaList(expected); TableMetadata metadata = TableMetadataParser.fromJson(asJson); @@ -312,6 +319,8 @@ public void testBackwardCompat() throws Exception { .isEqualTo(previousSnapshot.allManifests(ops.io())); assertThat(metadata.previousFiles()).isEqualTo(expected.previousFiles()); assertThat(metadata.snapshot(previousSnapshotId).schemaId()).isNull(); + assertThat(metadata.rowLineage()).isEqualTo(expected.rowLineage()); + assertThat(metadata.lastRowId()).isEqualTo(expected.lastRowId()); } @Test @@ -359,7 +368,7 @@ public void testInvalidMainBranch() throws IOException { () -> new TableMetadata( null, - 2, + MAX_FORMAT_VERSION, UUID.randomUUID().toString(), TEST_LOCATION, SEQ_NO, @@ -381,7 +390,9 @@ public void testInvalidMainBranch() throws IOException { refs, ImmutableList.of(), ImmutableList.of(), - ImmutableList.of())) + ImmutableList.of(), + false, + 0L)) .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith("Current snapshot ID does not match main branch"); } @@ -404,7 +415,7 @@ public void testMainWithoutCurrent() throws IOException { () -> new TableMetadata( null, - 2, + MAX_FORMAT_VERSION, UUID.randomUUID().toString(), TEST_LOCATION, SEQ_NO, @@ -426,7 +437,9 @@ public void testMainWithoutCurrent() throws IOException { refs, ImmutableList.of(), ImmutableList.of(), - ImmutableList.of())) + ImmutableList.of(), + false, + 0L)) .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith("Current snapshot is not set, but main branch exists"); } @@ -444,7 +457,7 @@ public void testBranchSnapshotMissing() { () -> new TableMetadata( null, - 2, + MAX_FORMAT_VERSION, UUID.randomUUID().toString(), TEST_LOCATION, SEQ_NO, @@ -466,7 +479,9 @@ public void testBranchSnapshotMissing() { refs, ImmutableList.of(), ImmutableList.of(), - ImmutableList.of())) + ImmutableList.of(), + false, + 0L)) .isInstanceOf(IllegalArgumentException.class) .hasMessageEndingWith("does not exist in the existing snapshots list"); } @@ -570,7 +585,9 @@ public void testJsonWithPreviousMetadataLog() throws Exception { ImmutableMap.of(), ImmutableList.of(), ImmutableList.of(), - ImmutableList.of()); + ImmutableList.of(), + false, + 0L); String asJson = TableMetadataParser.toJson(base); TableMetadata metadataFromJson = TableMetadataParser.fromJson(asJson); @@ -646,7 +663,9 @@ public void testAddPreviousMetadataRemoveNone() throws IOException { ImmutableMap.of(), ImmutableList.of(), ImmutableList.of(), - ImmutableList.of()); + ImmutableList.of(), + false, + 0L); previousMetadataLog.add(latestPreviousMetadata); @@ -737,7 +756,9 @@ public void testAddPreviousMetadataRemoveOne() throws IOException { ImmutableMap.of(), ImmutableList.of(), ImmutableList.of(), - ImmutableList.of()); + ImmutableList.of(), + false, + 0L); previousMetadataLog.add(latestPreviousMetadata); @@ -832,7 +853,9 @@ public void testAddPreviousMetadataRemoveMultiple() throws IOException { ImmutableMap.of(), ImmutableList.of(), ImmutableList.of(), - ImmutableList.of()); + ImmutableList.of(), + false, + 0L); previousMetadataLog.add(latestPreviousMetadata); @@ -878,7 +901,9 @@ public void testV2UUIDValidation() { ImmutableMap.of(), ImmutableList.of(), ImmutableList.of(), - ImmutableList.of())) + ImmutableList.of(), + false, + 0L)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("UUID is required in format v2"); } @@ -913,7 +938,9 @@ public void testVersionValidation() { ImmutableMap.of(), ImmutableList.of(), ImmutableList.of(), - ImmutableList.of())) + ImmutableList.of(), + false, + 0L)) .isInstanceOf(IllegalArgumentException.class) .hasMessage( "Unsupported format version: v%s (supported: v%s)", @@ -959,7 +986,9 @@ public void testVersionValidation() { ImmutableMap.of(), ImmutableList.of(), ImmutableList.of(), - ImmutableList.of())) + ImmutableList.of(), + false, + 0L)) .isNotNull(); assertThat( From 4039869cfa7f165a4c81a35728262896f582656b Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Fri, 10 Jan 2025 16:14:52 -0600 Subject: [PATCH 02/22] API, CORE, SPEC: Adds Snapshot Fields for Row Lineage --- .../java/org/apache/iceberg/Snapshot.java | 23 +++ .../java/org/apache/iceberg/BaseSnapshot.java | 20 ++- .../apache/iceberg/RewriteTablePathUtil.java | 4 +- .../org/apache/iceberg/SnapshotParser.java | 17 +- .../org/apache/iceberg/SnapshotProducer.java | 15 +- .../org/apache/iceberg/TableMetadata.java | 24 ++- .../apache/iceberg/TestDataTaskParser.java | 22 ++- .../iceberg/TestMetadataUpdateParser.java | 12 +- .../org/apache/iceberg/TestRowLineage.java | 57 ------- .../iceberg/TestRowLineageMetadata.java | 157 ++++++++++++++++++ .../org/apache/iceberg/TestSnapshotJson.java | 65 +++++++- .../org/apache/iceberg/TestTableMetadata.java | 108 ++++++++++-- 12 files changed, 434 insertions(+), 90 deletions(-) delete mode 100644 core/src/test/java/org/apache/iceberg/TestRowLineage.java create mode 100644 core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java diff --git a/api/src/main/java/org/apache/iceberg/Snapshot.java b/api/src/main/java/org/apache/iceberg/Snapshot.java index c035259e0e2c..4c9401894889 100644 --- a/api/src/main/java/org/apache/iceberg/Snapshot.java +++ b/api/src/main/java/org/apache/iceberg/Snapshot.java @@ -171,4 +171,27 @@ default Iterable removedDeleteFiles(FileIO io) { default Integer schemaId() { return null; } + + /** + * The row-id of the first newly added row in this snapshot. The all rows added in this snapshot + * will have a row-id value of greater than or equal to this value. Rows with a smaller value were + * not added in this snapshot. + * + * @return the first row-id to be used in this snapshot or null if row lineage was not enabled + * when the table was created. + */ + default Long firstRowId() { + return null; + } + + /** + * The total number of newly added rows in this snapshot. It should be the summation of {@link + * ManifestFile#ADDED_ROWS_COUNT} for every manifest added in this snapshot. This is optionally + * present but required if row lineage is enabled. + * + * @return the total number of new rows in this snapshot or null if it was not stored. + */ + default Long addedRows() { + return null; + } } diff --git a/core/src/main/java/org/apache/iceberg/BaseSnapshot.java b/core/src/main/java/org/apache/iceberg/BaseSnapshot.java index 58dec570d1fb..c3c1159ef8df 100644 --- a/core/src/main/java/org/apache/iceberg/BaseSnapshot.java +++ b/core/src/main/java/org/apache/iceberg/BaseSnapshot.java @@ -43,6 +43,8 @@ class BaseSnapshot implements Snapshot { private final Map summary; private final Integer schemaId; private final String[] v1ManifestLocations; + private final Long firstRowId; + private final Long addedRows; // lazily initialized private transient List allManifests = null; @@ -61,7 +63,9 @@ class BaseSnapshot implements Snapshot { String operation, Map summary, Integer schemaId, - String manifestList) { + String manifestList, + Long firstRowId, + Long addedRows) { this.sequenceNumber = sequenceNumber; this.snapshotId = snapshotId; this.parentId = parentId; @@ -71,6 +75,8 @@ class BaseSnapshot implements Snapshot { this.schemaId = schemaId; this.manifestListLocation = manifestList; this.v1ManifestLocations = null; + this.firstRowId = firstRowId; + this.addedRows = addedRows; } BaseSnapshot( @@ -91,6 +97,8 @@ class BaseSnapshot implements Snapshot { this.schemaId = schemaId; this.manifestListLocation = null; this.v1ManifestLocations = v1ManifestLocations; + this.firstRowId = null; + this.addedRows = null; } @Override @@ -128,6 +136,16 @@ public Integer schemaId() { return schemaId; } + @Override + public Long firstRowId() { + return firstRowId; + } + + @Override + public Long addedRows() { + return addedRows; + } + private void cacheManifests(FileIO fileIO) { if (fileIO == null) { throw new IllegalArgumentException("Cannot cache changes: FileIO is null"); diff --git a/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java b/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java index 1cf490421238..ab298bd6d733 100644 --- a/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java +++ b/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java @@ -188,7 +188,9 @@ private static List updatePathInSnapshots( snapshot.operation(), snapshot.summary(), snapshot.schemaId(), - newManifestListLocation); + newManifestListLocation, + snapshot.firstRowId(), + snapshot.addedRows()); newSnapshots.add(newSnapshot); } return newSnapshots; diff --git a/core/src/main/java/org/apache/iceberg/SnapshotParser.java b/core/src/main/java/org/apache/iceberg/SnapshotParser.java index b5ac3ec718ac..85fa71756e71 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotParser.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotParser.java @@ -51,6 +51,8 @@ private SnapshotParser() {} private static final String MANIFESTS = "manifests"; private static final String MANIFEST_LIST = "manifest-list"; private static final String SCHEMA_ID = "schema-id"; + private static final String FIRST_ROW_ID = "first-row-id"; + private static final String ADDED_ROWS = "added-rows"; static void toJson(Snapshot snapshot, JsonGenerator generator) throws IOException { generator.writeStartObject(); @@ -96,6 +98,14 @@ static void toJson(Snapshot snapshot, JsonGenerator generator) throws IOExceptio generator.writeNumberField(SCHEMA_ID, snapshot.schemaId()); } + if (snapshot.firstRowId() != null) { + generator.writeNumberField(FIRST_ROW_ID, snapshot.firstRowId()); + } + + if (snapshot.addedRows() != null) { + generator.writeNumberField(ADDED_ROWS, snapshot.addedRows()); + } + generator.writeEndObject(); } @@ -158,6 +168,9 @@ static Snapshot fromJson(JsonNode node) { Integer schemaId = JsonUtil.getIntOrNull(SCHEMA_ID, node); + Long firstRowId = JsonUtil.getLongOrNull(FIRST_ROW_ID, node); + Long addedRows = JsonUtil.getLongOrNull(ADDED_ROWS, node); + if (node.has(MANIFEST_LIST)) { // the manifest list is stored in a manifest list file String manifestList = JsonUtil.getString(MANIFEST_LIST, node); @@ -169,7 +182,9 @@ static Snapshot fromJson(JsonNode node) { operation, summary, schemaId, - manifestList); + manifestList, + firstRowId, + addedRows); } else { // fall back to an embedded manifest list. pass in the manifest's InputFile so length can be diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java index bc65e90eaeae..5ca81a2e9c2f 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java @@ -41,6 +41,7 @@ import java.util.List; import java.util.Locale; import java.util.Map; +import java.util.Objects; import java.util.Queue; import java.util.Set; import java.util.UUID; @@ -282,6 +283,16 @@ public Snapshot apply() { throw new RuntimeIOException(e, "Failed to write manifest list file"); } + Long addedRows = + manifests.stream() + .filter( + manifest -> + manifest.snapshotId() == null + || Objects.equals(manifest.snapshotId(), this.snapshotId)) + .mapToLong(ManifestFile::addedRowsCount) + .sum(); + Long lastRowId = base.rowLineage() ? base.lastRowId() : null; + return new BaseSnapshot( sequenceNumber, snapshotId(), @@ -290,7 +301,9 @@ public Snapshot apply() { operation(), summary(base), base.currentSchemaId(), - manifestList.location()); + manifestList.location(), + lastRowId, + addedRows); } protected abstract Map summary(); diff --git a/core/src/main/java/org/apache/iceberg/TableMetadata.java b/core/src/main/java/org/apache/iceberg/TableMetadata.java index 99cd8a054ffa..ce85ac227b59 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadata.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadata.java @@ -1260,6 +1260,18 @@ public Builder addSnapshot(Snapshot snapshot) { snapshotsById.put(snapshot.snapshotId(), snapshot); changes.add(new MetadataUpdate.AddSnapshot(snapshot)); + if (rowLineage) { + ValidationException.check( + snapshot.firstRowId() >= lastRowId, + "Cannot add a snapshot whose first-row-id (%s) is less than the metadata `last-used-id` (%s) because this will end up generating duplicate row_ids.", + snapshot.firstRowId(), + lastRowId); + ValidationException.check( + snapshot.addedRows() != null, + "Cannot add a snapshot with a null `addedRows` field when row lineage is enabled"); + this.incrementLastRowId(snapshot.addedRows()); + } + return this; } @@ -1507,17 +1519,15 @@ public Builder enableRowLineage() { return this; } - public Builder withLastRowId(long lastRowId) { + public Builder incrementLastRowId(long delta) { Preconditions.checkArgument( this.rowLineage, "Cannot set last-row-id if row lineage is not enabled"); Preconditions.checkArgument( - this.lastRowId <= lastRowId, - "Cannot decrease last-row-id, last-row-id must increase monotonically." - + " Current last-row-id (%s) is greater that the new last-row-id (%s)", - this.lastRowId, - lastRowId); + delta >= 0, + "Cannot decrease last-row-id, last-row-id must increase monotonically. Delta was %s", + delta); - this.lastRowId = lastRowId; + this.lastRowId += delta; return this; } diff --git a/core/src/test/java/org/apache/iceberg/TestDataTaskParser.java b/core/src/test/java/org/apache/iceberg/TestDataTaskParser.java index 5a3d119046f5..6e0b4325e4b5 100644 --- a/core/src/test/java/org/apache/iceberg/TestDataTaskParser.java +++ b/core/src/test/java/org/apache/iceberg/TestDataTaskParser.java @@ -194,9 +194,27 @@ private DataTask createDataTask() { List snapshots = Arrays.asList( new BaseSnapshot( - 1L, 1L, null, 1234567890000L, "append", summary1, 1, "file:/tmp/manifest1.avro"), + 1L, + 1L, + null, + 1234567890000L, + "append", + summary1, + 1, + "file:/tmp/manifest1.avro", + null, + null), new BaseSnapshot( - 2L, 2L, 1L, 9876543210000L, "append", summary2, 1, "file:/tmp/manifest2.avro")); + 2L, + 2L, + 1L, + 9876543210000L, + "append", + summary2, + 1, + "file:/tmp/manifest2.avro", + null, + null)); return StaticDataTask.of( Files.localInput("file:/tmp/metadata2.json"), diff --git a/core/src/test/java/org/apache/iceberg/TestMetadataUpdateParser.java b/core/src/test/java/org/apache/iceberg/TestMetadataUpdateParser.java index 6a65bf762880..f19c4a975c77 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetadataUpdateParser.java +++ b/core/src/test/java/org/apache/iceberg/TestMetadataUpdateParser.java @@ -360,6 +360,8 @@ public void testAddSnapshotToJson() throws IOException { long parentId = 1; long snapshotId = 2; int schemaId = 3; + Long firstRowId = 4L; + Long addedRows = 10L; String manifestList = createManifestListWithManifestFiles(snapshotId, parentId); @@ -372,7 +374,9 @@ public void testAddSnapshotToJson() throws IOException { DataOperations.REPLACE, ImmutableMap.of("files-added", "4", "files-deleted", "100"), schemaId, - manifestList); + manifestList, + firstRowId, + addedRows); String snapshotJson = SnapshotParser.toJson(snapshot, /* pretty */ false); String expected = String.format("{\"action\":\"%s\",\"snapshot\":%s}", action, snapshotJson); MetadataUpdate update = new MetadataUpdate.AddSnapshot(snapshot); @@ -388,6 +392,8 @@ public void testAddSnapshotFromJson() throws IOException { long parentId = 1; long snapshotId = 2; int schemaId = 3; + Long lastRowId = 4L; + Long addedRows = 5L; Map summary = ImmutableMap.of("files-added", "4", "files-deleted", "100"); String manifestList = createManifestListWithManifestFiles(snapshotId, parentId); @@ -400,7 +406,9 @@ public void testAddSnapshotFromJson() throws IOException { DataOperations.REPLACE, summary, schemaId, - manifestList); + manifestList, + lastRowId, + addedRows); String snapshotJson = SnapshotParser.toJson(snapshot, /* pretty */ false); String json = String.format("{\"action\":\"%s\",\"snapshot\":%s}", action, snapshotJson); MetadataUpdate expected = new MetadataUpdate.AddSnapshot(snapshot); diff --git a/core/src/test/java/org/apache/iceberg/TestRowLineage.java b/core/src/test/java/org/apache/iceberg/TestRowLineage.java deleted file mode 100644 index 9d949de8dc8f..000000000000 --- a/core/src/test/java/org/apache/iceberg/TestRowLineage.java +++ /dev/null @@ -1,57 +0,0 @@ -/* - * 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 static org.assertj.core.api.Assertions.assertThat; -import static org.junit.jupiter.api.Assertions.assertThrows; - -import org.junit.jupiter.api.Test; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.FieldSource; - -public class TestRowLineage { - - private TableMetadata.Builder testMetadataBuilder() { - return TableMetadata.buildFromEmpty(TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE) - .enableRowLineage(); - } - - @ParameterizedTest - @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") - public void testRowLineageSupported(int formatVersion) { - if (formatVersion == TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE) { - assertThat(TableMetadata.buildFromEmpty(formatVersion).enableRowLineage()).isNotNull(); - } else { - IllegalArgumentException notSupported = - assertThrows( - IllegalArgumentException.class, - () -> TableMetadata.buildFromEmpty(formatVersion).enableRowLineage()); - assertThat(notSupported.getMessage()).contains("Cannot use row lineage"); - } - } - - @Test - public void testLastRowIdMustIncrease() { - assertThat(testMetadataBuilder().withLastRowId(TableMetadata.INITIAL_ROW_ID + 5)).isNotNull(); - IllegalArgumentException noDecrease = assertThrows( - IllegalArgumentException.class, - () -> testMetadataBuilder().withLastRowId(TableMetadata.INITIAL_ROW_ID - 5)); - assertThat(noDecrease.getMessage()).contains("Cannot decrease last-row-id"); - } -} diff --git a/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java b/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java new file mode 100644 index 000000000000..41567b26de51 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java @@ -0,0 +1,157 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; +import static org.junit.Assume.assumeTrue; +import static org.junit.jupiter.api.Assertions.assertThrows; + +import java.nio.file.Path; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.FieldSource; + +public class TestRowLineageMetadata { + + private static final String TEST_LOCATION = "s3://bucket/test/location"; + + private static final Schema TEST_SCHEMA = + new Schema( + 7, + Types.NestedField.required(1, "x", Types.LongType.get()), + Types.NestedField.required(2, "y", Types.LongType.get(), "comment"), + Types.NestedField.required(3, "z", Types.LongType.get())); + + private static final long SEQ_NO = 34; + private static final int LAST_ASSIGNED_COLUMN_ID = 3; + + private static final PartitionSpec SPEC_5 = + PartitionSpec.builderFor(TEST_SCHEMA).withSpecId(5).build(); + private static final SortOrder SORT_ORDER_3 = + SortOrder.builderFor(TEST_SCHEMA) + .withOrderId(3) + .asc("y", NullOrder.NULLS_FIRST) + .desc(Expressions.bucket("z", 4), NullOrder.NULLS_LAST) + .build(); + + @TempDir private Path temp; + + public TableOperations ops = new LocalTableOperations(temp); + + private TableMetadata.Builder builderFor(int formatVersion) { + return TableMetadata.buildFromEmpty(TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE) + .enableRowLineage(); + } + + private TableMetadata baseMetadata(int formatVersion) { + return builderFor(formatVersion) + .addSchema(TEST_SCHEMA) + .setLocation(TEST_LOCATION) + .addPartitionSpec(PartitionSpec.unpartitioned()) + .addSortOrder(SortOrder.unsorted()) + .build(); + } + + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + public void testRowLineageSupported(int formatVersion) { + if (formatVersion == TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE) { + assertThat(builderFor(formatVersion)).isNotNull(); + } else { + IllegalArgumentException notSupported = + assertThrows( + IllegalArgumentException.class, + () -> TableMetadata.buildFromEmpty(formatVersion).enableRowLineage()); + assertThat(notSupported.getMessage()).contains("Cannot use row lineage"); + } + } + + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + public void testLastRowIdMustIncrease(int formatVersion) { + assumeTrue(formatVersion >= TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); + assertThat(builderFor(formatVersion).incrementLastRowId(5)).isNotNull(); + IllegalArgumentException noDecrease = + assertThrows( + IllegalArgumentException.class, () -> builderFor(formatVersion).incrementLastRowId(-5)); + assertThat(noDecrease.getMessage()).contains("Cannot decrease last-row-id"); + } + + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + public void testSnapshotAddition(int formatVersion) { + assumeTrue(formatVersion >= TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); + + Long newRows = 30L; + + TableMetadata base = baseMetadata(formatVersion); + + Snapshot addRows = + new BaseSnapshot( + 0, 1, null, 0, DataOperations.APPEND, null, 1, "foo", base.lastRowId(), newRows); + + TableMetadata firstAddition = TableMetadata.buildFrom(base).addSnapshot(addRows).build(); + + assertThat(firstAddition.lastRowId()).isEqualTo(newRows); + + Snapshot addMoreRows = + new BaseSnapshot( + 1, 2, 1L, 0, DataOperations.APPEND, null, 1, "foo", firstAddition.lastRowId(), newRows); + + TableMetadata secondAddition = + TableMetadata.buildFrom(firstAddition).addSnapshot(addMoreRows).build(); + + assertThat(secondAddition.lastRowId()).isEqualTo(newRows * 2); + } + + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + public void testInvalidSnapshotAddition(int formatVersion) { + assumeTrue(formatVersion >= TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); + + Long newRows = 30L; + + TableMetadata base = baseMetadata(formatVersion); + + Snapshot invalidLastRow = + new BaseSnapshot( + 0, 1, null, 0, DataOperations.APPEND, null, 1, "foo", base.lastRowId() - 3, newRows); + + ValidationException invalidLastRowId = + assertThrows( + ValidationException.class, + () -> TableMetadata.buildFrom(base).addSnapshot(invalidLastRow)); + assertThat(invalidLastRowId.getMessage()).contains("Cannot add a snapshot whose first-row-id"); + + Snapshot invalidNewRows = + new BaseSnapshot( + 0, 1, null, 0, DataOperations.APPEND, null, 1, "foo", base.lastRowId(), null); + + ValidationException nullNewRows = + assertThrows( + ValidationException.class, + () -> TableMetadata.buildFrom(base).addSnapshot(invalidNewRows)); + assertThat(nullNewRows.getMessage()) + .contains( + "Cannot add a snapshot with a null `addedRows` field when row lineage is enabled"); + } +} diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java b/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java index e4c2ba5ec2df..3fda738e4435 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java @@ -43,7 +43,16 @@ public void testJsonConversion() throws IOException { Snapshot expected = new BaseSnapshot( - 0, snapshotId, parentId, System.currentTimeMillis(), null, null, 1, manifestList); + 0, + snapshotId, + parentId, + System.currentTimeMillis(), + null, + null, + 1, + manifestList, + null, + null); String json = SnapshotParser.toJson(expected); Snapshot snapshot = SnapshotParser.fromJson(json); @@ -52,6 +61,8 @@ public void testJsonConversion() throws IOException { assertThat(snapshot.operation()).isNull(); assertThat(snapshot.summary()).isNull(); assertThat(snapshot.schemaId()).isEqualTo(1); + assertThat(snapshot.firstRowId()).isNull(); + assertThat(snapshot.addedRows()).isNull(); } @Test @@ -62,7 +73,16 @@ public void testJsonConversionWithoutSchemaId() throws IOException { Snapshot expected = new BaseSnapshot( - 0, snapshotId, parentId, System.currentTimeMillis(), null, null, null, manifestList); + 0, + snapshotId, + parentId, + System.currentTimeMillis(), + null, + null, + null, + manifestList, + null, + null); String json = SnapshotParser.toJson(expected); Snapshot snapshot = SnapshotParser.fromJson(json); @@ -71,6 +91,8 @@ public void testJsonConversionWithoutSchemaId() throws IOException { assertThat(snapshot.operation()).isNull(); assertThat(snapshot.summary()).isNull(); assertThat(snapshot.schemaId()).isNull(); + assertThat(snapshot.firstRowId()).isNull(); + assertThat(snapshot.addedRows()).isNull(); } @Test @@ -89,7 +111,9 @@ public void testJsonConversionWithOperation() throws IOException { DataOperations.REPLACE, ImmutableMap.of("files-added", "4", "files-deleted", "100"), 3, - manifestList); + manifestList, + null, + null); String json = SnapshotParser.toJson(expected); Snapshot snapshot = SnapshotParser.fromJson(json); @@ -105,6 +129,40 @@ public void testJsonConversionWithOperation() throws IOException { assertThat(snapshot.operation()).isEqualTo(expected.operation()); assertThat(snapshot.summary()).isEqualTo(expected.summary()); assertThat(snapshot.schemaId()).isEqualTo(expected.schemaId()); + assertThat(snapshot.firstRowId()).isNull(); + assertThat(snapshot.addedRows()).isNull(); + } + + @Test + public void testJsonConversionWithRowLineage() throws IOException { + int snapshotId = 23; + Long parentId = null; + Long firstRowId = 20L; + Long addedRows = 30L; + String manifestList = createManifestListWithManifestFiles(snapshotId, parentId); + + Snapshot expected = + new BaseSnapshot( + 0, + snapshotId, + parentId, + System.currentTimeMillis(), + null, + null, + null, + manifestList, + firstRowId, + addedRows); + String json = SnapshotParser.toJson(expected); + Snapshot snapshot = SnapshotParser.fromJson(json); + + assertThat(snapshot.snapshotId()).isEqualTo(expected.snapshotId()); + assertThat(snapshot.allManifests(ops.io())).isEqualTo(expected.allManifests(ops.io())); + assertThat(snapshot.operation()).isNull(); + assertThat(snapshot.summary()).isNull(); + assertThat(snapshot.schemaId()).isNull(); + assertThat(snapshot.firstRowId()).isEqualTo(firstRowId); + assertThat(snapshot.addedRows()).isEqualTo(addedRows); } @Test @@ -157,6 +215,7 @@ public void testJsonConversionWithV1Manifests() { assertThat(snapshot.operation()).isEqualTo(expected.operation()); assertThat(snapshot.summary()).isEqualTo(expected.summary()); assertThat(snapshot.schemaId()).isEqualTo(expected.schemaId()); + assertThat(snapshot.firstRowId()).isNull(); } private String createManifestListWithManifestFiles(long snapshotId, Long parentSnapshotId) diff --git a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java index f46c54539498..9f7ad05688d5 100644 --- a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java +++ b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java @@ -106,7 +106,16 @@ public void testJsonConversion() throws Exception { createManifestListWithManifestFile(previousSnapshotId, null, "file:/tmp/manifest1.avro"); Snapshot previousSnapshot = new BaseSnapshot( - 0, previousSnapshotId, null, previousSnapshotId, null, null, null, manifestList); + 0, + previousSnapshotId, + null, + previousSnapshotId, + null, + null, + null, + manifestList, + null, + null); long currentSnapshotId = System.currentTimeMillis(); manifestList = @@ -121,7 +130,9 @@ public void testJsonConversion() throws Exception { null, null, 7, - manifestList); + manifestList, + null, + null); List snapshotLog = ImmutableList.builder() @@ -236,7 +247,16 @@ public void testBackwardCompat() throws Exception { createManifestListWithManifestFile(previousSnapshotId, null, "file:/tmp/manifest1.avro"); Snapshot previousSnapshot = new BaseSnapshot( - 0, previousSnapshotId, null, previousSnapshotId, null, null, null, manifestList); + 0, + previousSnapshotId, + null, + previousSnapshotId, + null, + null, + null, + manifestList, + null, + null); long currentSnapshotId = System.currentTimeMillis(); manifestList = @@ -251,7 +271,9 @@ public void testBackwardCompat() throws Exception { null, null, null, - manifestList); + manifestList, + null, + null); TableMetadata expected = new TableMetadata( @@ -331,7 +353,16 @@ public void testInvalidMainBranch() throws IOException { createManifestListWithManifestFile(previousSnapshotId, null, "file:/tmp/manifest1.avro"); Snapshot previousSnapshot = new BaseSnapshot( - 0, previousSnapshotId, null, previousSnapshotId, null, null, null, manifestList); + 0, + previousSnapshotId, + null, + previousSnapshotId, + null, + null, + null, + manifestList, + null, + null); long currentSnapshotId = System.currentTimeMillis(); manifestList = @@ -347,7 +378,9 @@ public void testInvalidMainBranch() throws IOException { null, null, 7, - manifestList); + manifestList, + null, + null); List snapshotLog = ImmutableList.builder() @@ -404,7 +437,8 @@ public void testMainWithoutCurrent() throws IOException { String manifestList = createManifestListWithManifestFile(snapshotId, null, "file:/tmp/manifest1.avro"); Snapshot snapshot = - new BaseSnapshot(0, snapshotId, null, snapshotId, null, null, null, manifestList); + new BaseSnapshot( + 0, snapshotId, null, snapshotId, null, null, null, manifestList, null, null); Schema schema = new Schema(6, Types.NestedField.required(10, "x", Types.StringType.get())); @@ -536,7 +570,16 @@ public void testJsonWithPreviousMetadataLog() throws Exception { createManifestListWithManifestFile(previousSnapshotId, null, "file:/tmp/manifest1.avro"); Snapshot previousSnapshot = new BaseSnapshot( - 0, previousSnapshotId, null, previousSnapshotId, null, null, null, manifestList); + 0, + previousSnapshotId, + null, + previousSnapshotId, + null, + null, + null, + manifestList, + null, + null); long currentSnapshotId = System.currentTimeMillis(); manifestList = @@ -551,7 +594,9 @@ public void testJsonWithPreviousMetadataLog() throws Exception { null, null, null, - manifestList); + manifestList, + null, + null); List reversedSnapshotLog = Lists.newArrayList(); long currentTimestamp = System.currentTimeMillis(); @@ -603,7 +648,16 @@ public void testAddPreviousMetadataRemoveNone() throws IOException { createManifestListWithManifestFile(previousSnapshotId, null, "file:/tmp/manifest1.avro"); Snapshot previousSnapshot = new BaseSnapshot( - 0, previousSnapshotId, null, previousSnapshotId, null, null, null, manifestList); + 0, + previousSnapshotId, + null, + previousSnapshotId, + null, + null, + null, + manifestList, + null, + null); long currentSnapshotId = System.currentTimeMillis(); manifestList = @@ -618,7 +672,9 @@ public void testAddPreviousMetadataRemoveNone() throws IOException { null, null, null, - manifestList); + manifestList, + null, + null); List reversedSnapshotLog = Lists.newArrayList(); reversedSnapshotLog.add( @@ -687,7 +743,16 @@ public void testAddPreviousMetadataRemoveOne() throws IOException { createManifestListWithManifestFile(previousSnapshotId, null, "file:/tmp/manifest1.avro"); Snapshot previousSnapshot = new BaseSnapshot( - 0, previousSnapshotId, null, previousSnapshotId, null, null, null, manifestList); + 0, + previousSnapshotId, + null, + previousSnapshotId, + null, + null, + null, + manifestList, + null, + null); long currentSnapshotId = System.currentTimeMillis(); manifestList = @@ -702,7 +767,9 @@ public void testAddPreviousMetadataRemoveOne() throws IOException { null, null, null, - manifestList); + manifestList, + null, + null); List reversedSnapshotLog = Lists.newArrayList(); reversedSnapshotLog.add( @@ -784,7 +851,16 @@ public void testAddPreviousMetadataRemoveMultiple() throws IOException { createManifestListWithManifestFile(previousSnapshotId, null, "file:/tmp/manifest1.avro"); Snapshot previousSnapshot = new BaseSnapshot( - 0, previousSnapshotId, null, previousSnapshotId, null, null, null, manifestList); + 0, + previousSnapshotId, + null, + previousSnapshotId, + null, + null, + null, + manifestList, + null, + null); long currentSnapshotId = System.currentTimeMillis(); manifestList = @@ -799,7 +875,9 @@ public void testAddPreviousMetadataRemoveMultiple() throws IOException { null, null, null, - manifestList); + manifestList, + null, + null); List reversedSnapshotLog = Lists.newArrayList(); reversedSnapshotLog.add( From c416acf5b2e3488d60d3acaa2c16f37474c452a4 Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Fri, 10 Jan 2025 16:21:10 -0600 Subject: [PATCH 03/22] Cleanup Unused Test Variables --- .../iceberg/TestRowLineageMetadata.java | 20 +------------------ 1 file changed, 1 insertion(+), 19 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java b/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java index 41567b26de51..fbe8d3ccaecb 100644 --- a/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java +++ b/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java @@ -22,11 +22,8 @@ import static org.junit.Assume.assumeTrue; import static org.junit.jupiter.api.Assertions.assertThrows; -import java.nio.file.Path; import org.apache.iceberg.exceptions.ValidationException; -import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.types.Types; -import org.junit.jupiter.api.io.TempDir; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.FieldSource; @@ -41,24 +38,9 @@ public class TestRowLineageMetadata { Types.NestedField.required(2, "y", Types.LongType.get(), "comment"), Types.NestedField.required(3, "z", Types.LongType.get())); - private static final long SEQ_NO = 34; - private static final int LAST_ASSIGNED_COLUMN_ID = 3; - - private static final PartitionSpec SPEC_5 = - PartitionSpec.builderFor(TEST_SCHEMA).withSpecId(5).build(); - private static final SortOrder SORT_ORDER_3 = - SortOrder.builderFor(TEST_SCHEMA) - .withOrderId(3) - .asc("y", NullOrder.NULLS_FIRST) - .desc(Expressions.bucket("z", 4), NullOrder.NULLS_LAST) - .build(); - - @TempDir private Path temp; - - public TableOperations ops = new LocalTableOperations(temp); private TableMetadata.Builder builderFor(int formatVersion) { - return TableMetadata.buildFromEmpty(TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE) + return TableMetadata.buildFromEmpty(formatVersion) .enableRowLineage(); } From 1b4f0f66286dd2fd9e92abe2971c0f80d597e203 Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Fri, 10 Jan 2025 17:11:08 -0600 Subject: [PATCH 04/22] Core: Add tests for Append Pathways --- .../org/apache/iceberg/MetadataUpdate.java | 7 ++ .../org/apache/iceberg/TableMetadata.java | 3 +- .../iceberg/TestRowLineageMetadata.java | 74 +++++++++++++++++++ 3 files changed, 83 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/iceberg/MetadataUpdate.java b/core/src/main/java/org/apache/iceberg/MetadataUpdate.java index ff008c9d43e2..c24b2d42b291 100644 --- a/core/src/main/java/org/apache/iceberg/MetadataUpdate.java +++ b/core/src/main/java/org/apache/iceberg/MetadataUpdate.java @@ -517,4 +517,11 @@ public void applyTo(ViewMetadata.Builder viewMetadataBuilder) { viewMetadataBuilder.setCurrentVersionId(versionId); } } + + class EnableRowLineage implements MetadataUpdate { + @Override + public void applyTo(TableMetadata.Builder metadataBuilder) { + MetadataUpdate.super.applyTo(metadataBuilder.enableRowLineage()); + } + } } diff --git a/core/src/main/java/org/apache/iceberg/TableMetadata.java b/core/src/main/java/org/apache/iceberg/TableMetadata.java index ce85ac227b59..fa4475005cfd 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadata.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadata.java @@ -1516,10 +1516,11 @@ public Builder enableRowLineage() { "Cannot use row lineage with format version %s. Only format version 3 or higher support row lineage", formatVersion); this.rowLineage = true; + changes.add(new MetadataUpdate.EnableRowLineage()); return this; } - public Builder incrementLastRowId(long delta) { + Builder incrementLastRowId(long delta) { Preconditions.checkArgument( this.rowLineage, "Cannot set last-row-id if row lineage is not enabled"); Preconditions.checkArgument( diff --git a/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java b/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java index fbe8d3ccaecb..78b971bf8a71 100644 --- a/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java +++ b/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java @@ -22,8 +22,12 @@ import static org.junit.Assume.assumeTrue; import static org.junit.jupiter.api.Assertions.assertThrows; +import java.io.File; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.io.TempDir; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.FieldSource; @@ -31,6 +35,9 @@ public class TestRowLineageMetadata { private static final String TEST_LOCATION = "s3://bucket/test/location"; + @TempDir + protected File tableDir = null; + private static final Schema TEST_SCHEMA = new Schema( 7, @@ -53,6 +60,11 @@ private TableMetadata baseMetadata(int formatVersion) { .build(); } + @AfterEach + public void cleanup() { + TestTables.clearTables(); + } + @ParameterizedTest @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") public void testRowLineageSupported(int formatVersion) { @@ -136,4 +148,66 @@ public void testInvalidSnapshotAddition(int formatVersion) { .contains( "Cannot add a snapshot with a null `addedRows` field when row lineage is enabled"); } + + AtomicInteger fileNum = new AtomicInteger(0); + private DataFile fileWithRows(int numRows) { + return DataFiles.builder(PartitionSpec.unpartitioned()) + .withRecordCount(numRows) + .withFileSizeInBytes(numRows * 100) + .withPath("file://file_" + fileNum.incrementAndGet() + ".parquet") + .build(); + } + + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + public void testFastAppend(int formatVersion) { + assumeTrue(formatVersion >= TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); + TestTables.TestTable table = TestTables.create(tableDir, "test", TEST_SCHEMA, PartitionSpec.unpartitioned(), formatVersion); + TableMetadata base = table.ops().current(); + table.ops().commit(base, TableMetadata.buildFrom(base).enableRowLineage().build()); + + assertThat(table.ops().current().rowLineage()).isTrue(); + assertThat(table.ops().current().lastRowId()).isEqualTo(0L); + + + table.newFastAppend().appendFile(fileWithRows(30)).commit(); + + assertThat(table.ops().current().rowLineage()).isTrue(); + assertThat(table.currentSnapshot().firstRowId()).isEqualTo(0); + assertThat(table.ops().current().lastRowId()).isEqualTo(30); + + table.newFastAppend().appendFile(fileWithRows(17)).appendFile(fileWithRows(11)).commit(); + + assertThat(table.ops().current().rowLineage()).isTrue(); + assertThat(table.currentSnapshot().firstRowId()).isEqualTo(30); + assertThat(table.ops().current().lastRowId()).isEqualTo(30 + 17 + 11); + } + + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + public void testAppend(int formatVersion) { + assumeTrue(formatVersion >= TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); + TestTables.TestTable table = TestTables.create(tableDir, "test", TEST_SCHEMA, PartitionSpec.unpartitioned(), formatVersion); + TableMetadata base = table.ops().current(); + table.ops().commit(base, TableMetadata.buildFrom(base).enableRowLineage().build()); + + assertThat(table.ops().current().rowLineage()).isTrue(); + assertThat(table.ops().current().lastRowId()).isEqualTo(0L); + + + table.newAppend().appendFile(fileWithRows(30)).commit(); + + assertThat(table.ops().current().rowLineage()).isTrue(); + assertThat(table.currentSnapshot().firstRowId()).isEqualTo(0); + assertThat(table.ops().current().lastRowId()).isEqualTo(30); + + table.newAppend().appendFile(fileWithRows(17)).appendFile(fileWithRows(11)).commit(); + + assertThat(table.ops().current().rowLineage()).isTrue(); + assertThat(table.currentSnapshot().firstRowId()).isEqualTo(30); + assertThat(table.ops().current().lastRowId()).isEqualTo(30 + 17 + 11); + } + + + } From 75c1ad4f94a0e4bada96025d7df1d1661c96f6c4 Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Mon, 13 Jan 2025 14:13:53 -0600 Subject: [PATCH 05/22] Review Comments --- .../java/org/apache/iceberg/Snapshot.java | 15 ++++--- .../org/apache/iceberg/SnapshotProducer.java | 31 ++++++++++---- .../org/apache/iceberg/TableMetadata.java | 14 +++---- .../iceberg/TestRowLineageMetadata.java | 42 +++++++------------ 4 files changed, 52 insertions(+), 50 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/Snapshot.java b/api/src/main/java/org/apache/iceberg/Snapshot.java index 4c9401894889..74503fdb8b86 100644 --- a/api/src/main/java/org/apache/iceberg/Snapshot.java +++ b/api/src/main/java/org/apache/iceberg/Snapshot.java @@ -173,9 +173,10 @@ default Integer schemaId() { } /** - * The row-id of the first newly added row in this snapshot. The all rows added in this snapshot - * will have a row-id value of greater than or equal to this value. Rows with a smaller value were - * not added in this snapshot. + * The row-id of the first newly added row in this snapshot. All rows added in this snapshot will + * have a row-id assigned to them greater than this value. All rows with a row-id less than this + * value were created in a snapshot that was added to the table (but not necessarily commited to + * this branch) in the past. * * @return the first row-id to be used in this snapshot or null if row lineage was not enabled * when the table was created. @@ -186,10 +187,12 @@ default Long firstRowId() { /** * The total number of newly added rows in this snapshot. It should be the summation of {@link - * ManifestFile#ADDED_ROWS_COUNT} for every manifest added in this snapshot. This is optionally - * present but required if row lineage is enabled. + * ManifestFile#ADDED_ROWS_COUNT} for every manifest added in this snapshot. From the definition + * of ADDED_ROWS_COUNT this is the total number of rows in every added {@link DataFile} * - * @return the total number of new rows in this snapshot or null if it was not stored. + *

This is optionally present but is required if row lineage is enabled. + * + * @return the total number of new rows in this snapshot or null if the value was not stored. */ default Long addedRows() { return null; diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java index 5ca81a2e9c2f..5c6884d79c9e 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java @@ -283,15 +283,28 @@ public Snapshot apply() { throw new RuntimeIOException(e, "Failed to write manifest list file"); } - Long addedRows = - manifests.stream() - .filter( - manifest -> - manifest.snapshotId() == null - || Objects.equals(manifest.snapshotId(), this.snapshotId)) - .mapToLong(ManifestFile::addedRowsCount) - .sum(); - Long lastRowId = base.rowLineage() ? base.lastRowId() : null; + Long addedRows = null; + Long lastRowId = null; + if (base.rowLineage()) { + addedRows = + manifests.stream() + .filter( + manifest -> + manifest.snapshotId() == null + || Objects.equals(manifest.snapshotId(), this.snapshotId)) + .mapToLong( + manifest -> { + Preconditions.checkArgument( + manifest.addedRowsCount() != null, + "Cannot determine number of added rows in snapshot because" + + " the entry for manifest %s is missing the field `added-rows-count`", + manifest.path()); + return manifest.addedRowsCount(); + }) + .sum(); + + lastRowId = base.lastRowId(); + } return new BaseSnapshot( sequenceNumber, diff --git a/core/src/main/java/org/apache/iceberg/TableMetadata.java b/core/src/main/java/org/apache/iceberg/TableMetadata.java index fa4475005cfd..b7f795a957b3 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadata.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadata.java @@ -1269,7 +1269,7 @@ public Builder addSnapshot(Snapshot snapshot) { ValidationException.check( snapshot.addedRows() != null, "Cannot add a snapshot with a null `addedRows` field when row lineage is enabled"); - this.incrementLastRowId(snapshot.addedRows()); + incrementLastRowId(snapshot.addedRows()); } return this; @@ -1512,24 +1512,24 @@ public Builder setPreviousFileLocation(String previousFileLocation) { public Builder enableRowLineage() { Preconditions.checkArgument( - formatVersion >= 3, - "Cannot use row lineage with format version %s. Only format version 3 or higher support row lineage", - formatVersion); + formatVersion >= MIN_FORMAT_VERSION_ROW_LINEAGE, + "Cannot use row lineage with format version %s. Only format version %s or higher support row lineage", + formatVersion, + MIN_FORMAT_VERSION_ROW_LINEAGE); this.rowLineage = true; changes.add(new MetadataUpdate.EnableRowLineage()); return this; } - Builder incrementLastRowId(long delta) { + private void incrementLastRowId(long delta) { Preconditions.checkArgument( - this.rowLineage, "Cannot set last-row-id if row lineage is not enabled"); + rowLineage, "Cannot set last-row-id if row lineage is not enabled"); Preconditions.checkArgument( delta >= 0, "Cannot decrease last-row-id, last-row-id must increase monotonically. Delta was %s", delta); this.lastRowId += delta; - return this; } private boolean hasChanges() { diff --git a/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java b/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java index 78b971bf8a71..19ce14e35232 100644 --- a/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java +++ b/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java @@ -35,8 +35,7 @@ public class TestRowLineageMetadata { private static final String TEST_LOCATION = "s3://bucket/test/location"; - @TempDir - protected File tableDir = null; + @TempDir protected File tableDir = null; private static final Schema TEST_SCHEMA = new Schema( @@ -45,10 +44,8 @@ public class TestRowLineageMetadata { Types.NestedField.required(2, "y", Types.LongType.get(), "comment"), Types.NestedField.required(3, "z", Types.LongType.get())); - private TableMetadata.Builder builderFor(int formatVersion) { - return TableMetadata.buildFromEmpty(formatVersion) - .enableRowLineage(); + return TableMetadata.buildFromEmpty(formatVersion).enableRowLineage(); } private TableMetadata baseMetadata(int formatVersion) { @@ -79,17 +76,6 @@ public void testRowLineageSupported(int formatVersion) { } } - @ParameterizedTest - @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") - public void testLastRowIdMustIncrease(int formatVersion) { - assumeTrue(formatVersion >= TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); - assertThat(builderFor(formatVersion).incrementLastRowId(5)).isNotNull(); - IllegalArgumentException noDecrease = - assertThrows( - IllegalArgumentException.class, () -> builderFor(formatVersion).incrementLastRowId(-5)); - assertThat(noDecrease.getMessage()).contains("Cannot decrease last-row-id"); - } - @ParameterizedTest @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") public void testSnapshotAddition(int formatVersion) { @@ -149,27 +135,29 @@ public void testInvalidSnapshotAddition(int formatVersion) { "Cannot add a snapshot with a null `addedRows` field when row lineage is enabled"); } - AtomicInteger fileNum = new AtomicInteger(0); + private AtomicInteger fileNum = new AtomicInteger(0); + private DataFile fileWithRows(int numRows) { return DataFiles.builder(PartitionSpec.unpartitioned()) - .withRecordCount(numRows) - .withFileSizeInBytes(numRows * 100) - .withPath("file://file_" + fileNum.incrementAndGet() + ".parquet") - .build(); + .withRecordCount(numRows) + .withFileSizeInBytes(numRows * 100) + .withPath("file://file_" + fileNum.incrementAndGet() + ".parquet") + .build(); } @ParameterizedTest @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") public void testFastAppend(int formatVersion) { assumeTrue(formatVersion >= TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); - TestTables.TestTable table = TestTables.create(tableDir, "test", TEST_SCHEMA, PartitionSpec.unpartitioned(), formatVersion); + TestTables.TestTable table = + TestTables.create( + tableDir, "test", TEST_SCHEMA, PartitionSpec.unpartitioned(), formatVersion); TableMetadata base = table.ops().current(); table.ops().commit(base, TableMetadata.buildFrom(base).enableRowLineage().build()); assertThat(table.ops().current().rowLineage()).isTrue(); assertThat(table.ops().current().lastRowId()).isEqualTo(0L); - table.newFastAppend().appendFile(fileWithRows(30)).commit(); assertThat(table.ops().current().rowLineage()).isTrue(); @@ -187,14 +175,15 @@ public void testFastAppend(int formatVersion) { @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") public void testAppend(int formatVersion) { assumeTrue(formatVersion >= TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); - TestTables.TestTable table = TestTables.create(tableDir, "test", TEST_SCHEMA, PartitionSpec.unpartitioned(), formatVersion); + TestTables.TestTable table = + TestTables.create( + tableDir, "test", TEST_SCHEMA, PartitionSpec.unpartitioned(), formatVersion); TableMetadata base = table.ops().current(); table.ops().commit(base, TableMetadata.buildFrom(base).enableRowLineage().build()); assertThat(table.ops().current().rowLineage()).isTrue(); assertThat(table.ops().current().lastRowId()).isEqualTo(0L); - table.newAppend().appendFile(fileWithRows(30)).commit(); assertThat(table.ops().current().rowLineage()).isTrue(); @@ -207,7 +196,4 @@ public void testAppend(int formatVersion) { assertThat(table.currentSnapshot().firstRowId()).isEqualTo(30); assertThat(table.ops().current().lastRowId()).isEqualTo(30 + 17 + 11); } - - - } From 3ed2844af6b8761a45610a21aa385a2d066b7387 Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Mon, 13 Jan 2025 15:10:09 -0600 Subject: [PATCH 06/22] Adds some additional testing --- .../java/org/apache/iceberg/Snapshot.java | 5 +- .../iceberg/TestRowLineageMetadata.java | 115 +++++++++++++++++- 2 files changed, 116 insertions(+), 4 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/Snapshot.java b/api/src/main/java/org/apache/iceberg/Snapshot.java index 74503fdb8b86..52280a41620f 100644 --- a/api/src/main/java/org/apache/iceberg/Snapshot.java +++ b/api/src/main/java/org/apache/iceberg/Snapshot.java @@ -187,10 +187,9 @@ default Long firstRowId() { /** * The total number of newly added rows in this snapshot. It should be the summation of {@link - * ManifestFile#ADDED_ROWS_COUNT} for every manifest added in this snapshot. From the definition - * of ADDED_ROWS_COUNT this is the total number of rows in every added {@link DataFile} + * ManifestFile#ADDED_ROWS_COUNT} for every manifest added in this snapshot. * - *

This is optionally present but is required if row lineage is enabled. + *

This field is optional but is required when row lineage is enabled. * * @return the total number of new rows in this snapshot or null if the value was not stored. */ diff --git a/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java b/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java index 19ce14e35232..17e06ce68a14 100644 --- a/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java +++ b/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java @@ -19,8 +19,8 @@ package org.apache.iceberg; import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.Assume.assumeTrue; import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assumptions.assumeTrue; import java.io.File; import java.util.concurrent.atomic.AtomicInteger; @@ -149,6 +149,7 @@ private DataFile fileWithRows(int numRows) { @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") public void testFastAppend(int formatVersion) { assumeTrue(formatVersion >= TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); + TestTables.TestTable table = TestTables.create( tableDir, "test", TEST_SCHEMA, PartitionSpec.unpartitioned(), formatVersion); @@ -175,6 +176,7 @@ public void testFastAppend(int formatVersion) { @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") public void testAppend(int formatVersion) { assumeTrue(formatVersion >= TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); + TestTables.TestTable table = TestTables.create( tableDir, "test", TEST_SCHEMA, PartitionSpec.unpartitioned(), formatVersion); @@ -196,4 +198,115 @@ public void testAppend(int formatVersion) { assertThat(table.currentSnapshot().firstRowId()).isEqualTo(30); assertThat(table.ops().current().lastRowId()).isEqualTo(30 + 17 + 11); } + + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + public void testAppendBranch(int formatVersion) { + assumeTrue(formatVersion >= TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); + // Appends to a branch should still change last-row-id even if not on main, these changes + // should also affect commits to main + + String branch = "some_branch"; + + TestTables.TestTable table = + TestTables.create( + tableDir, "test", TEST_SCHEMA, PartitionSpec.unpartitioned(), formatVersion); + + TableMetadata base = table.ops().current(); + table.ops().commit(base, TableMetadata.buildFrom(base).enableRowLineage().build()); + + assertThat(table.ops().current().rowLineage()).isTrue(); + assertThat(table.ops().current().lastRowId()).isEqualTo(0L); + + // Write to Branch + table.newAppend().appendFile(fileWithRows(30)).toBranch(branch).commit(); + + assertThat(table.ops().current().rowLineage()).isTrue(); + assertThat(table.currentSnapshot()).isNull(); + assertThat(table.snapshot(branch).firstRowId()).isEqualTo(0L); + assertThat(table.ops().current().lastRowId()).isEqualTo(30); + + // Write to Main + table.newAppend().appendFile(fileWithRows(17)).appendFile(fileWithRows(11)).commit(); + + assertThat(table.ops().current().rowLineage()).isTrue(); + assertThat(table.currentSnapshot().firstRowId()).isEqualTo(30); + assertThat(table.ops().current().lastRowId()).isEqualTo(30 + 17 + 11); + + // Write again to branch + table.newAppend().appendFile(fileWithRows(21)).toBranch(branch).commit(); + assertThat(table.snapshot(branch).firstRowId()).isEqualTo(30 + 17 + 11); + assertThat(table.ops().current().lastRowId()).isEqualTo(30 + 17 + 11 + 21); + } + + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + public void testDeletes(int formatVersion) { + assumeTrue(formatVersion >= TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); + + TestTables.TestTable table = + TestTables.create( + tableDir, "test", TEST_SCHEMA, PartitionSpec.unpartitioned(), formatVersion); + TableMetadata base = table.ops().current(); + table.ops().commit(base, TableMetadata.buildFrom(base).enableRowLineage().build()); + + assertThat(table.ops().current().rowLineage()).isTrue(); + assertThat(table.ops().current().lastRowId()).isEqualTo(0L); + + DataFile file = fileWithRows(30); + + table.newAppend().appendFile(file).commit(); + + assertThat(table.ops().current().rowLineage()).isTrue(); + assertThat(table.currentSnapshot().firstRowId()).isEqualTo(0); + assertThat(table.ops().current().lastRowId()).isEqualTo(30); + + table.newDelete().deleteFile(file).commit(); + + // Deleting a file should create a new snapshot which should inherit last-row-id from the + // previous metadata and not + // change last-row-id for this metadata. + assertThat(table.ops().current().rowLineage()).isTrue(); + assertThat(table.currentSnapshot().firstRowId()).isEqualTo(30); + assertThat(table.currentSnapshot().addedRows()).isEqualTo(0); + assertThat(table.ops().current().lastRowId()).isEqualTo(30); + } + + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + public void testReplace(int formatVersion) { + assumeTrue(formatVersion >= TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); + + TestTables.TestTable table = + TestTables.create( + tableDir, "test", TEST_SCHEMA, PartitionSpec.unpartitioned(), formatVersion); + TableMetadata base = table.ops().current(); + + table.ops().commit(base, TableMetadata.buildFrom(base).enableRowLineage().build()); + + assertThat(table.ops().current().rowLineage()).isTrue(); + assertThat(table.ops().current().lastRowId()).isEqualTo(0L); + + DataFile file_part1 = fileWithRows(30); + DataFile file_part2 = fileWithRows(30); + DataFile file_compacted = fileWithRows(60); + + table.newAppend().appendFile(file_part1).appendFile(file_part2).commit(); + + assertThat(table.ops().current().rowLineage()).isTrue(); + assertThat(table.currentSnapshot().firstRowId()).isEqualTo(0); + assertThat(table.ops().current().lastRowId()).isEqualTo(60); + + table + .newRewrite() + .deleteFile(file_part1) + .deleteFile(file_part2) + .addFile(file_compacted) + .commit(); + + // Rewrites are currently just treated as appends. In the future we could treat these as no-ops + assertThat(table.ops().current().rowLineage()).isTrue(); + assertThat(table.currentSnapshot().firstRowId()).isEqualTo(60); + assertThat(table.ops().current().lastRowId()).isEqualTo(120); + } } From a58f7b47e5ab87a859a7b81410f8d9cf2ce1a812 Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Mon, 13 Jan 2025 15:26:10 -0600 Subject: [PATCH 07/22] Fix Checkstyle --- .../java/org/apache/iceberg/TableMetadata.java | 2 +- .../apache/iceberg/TestRowLineageMetadata.java | 16 ++++++++-------- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/TableMetadata.java b/core/src/main/java/org/apache/iceberg/TableMetadata.java index b7f795a957b3..1ad98184378f 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadata.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadata.java @@ -316,7 +316,7 @@ public String toString() { "Cannot create TableMetadata with a metadata location and changes"); Preconditions.checkArgument( formatVersion >= MIN_FORMAT_VERSION_ROW_LINEAGE || !rowLineage, - "Cannot enable row lineage when Table Version is less than V3. Table Version is %d", + "Cannot enable row lineage when Table Version is less than V3. Table Version is %s", formatVersion); this.metadataFileLocation = metadataFileLocation; diff --git a/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java b/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java index 17e06ce68a14..4dc0780e32d9 100644 --- a/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java +++ b/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java @@ -19,7 +19,7 @@ package org.apache.iceberg; import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.Assert.assertThrows; import static org.junit.jupiter.api.Assumptions.assumeTrue; import java.io.File; @@ -287,11 +287,11 @@ public void testReplace(int formatVersion) { assertThat(table.ops().current().rowLineage()).isTrue(); assertThat(table.ops().current().lastRowId()).isEqualTo(0L); - DataFile file_part1 = fileWithRows(30); - DataFile file_part2 = fileWithRows(30); - DataFile file_compacted = fileWithRows(60); + DataFile filePart1 = fileWithRows(30); + DataFile filePart2 = fileWithRows(30); + DataFile fileCompacted = fileWithRows(60); - table.newAppend().appendFile(file_part1).appendFile(file_part2).commit(); + table.newAppend().appendFile(filePart1).appendFile(filePart2).commit(); assertThat(table.ops().current().rowLineage()).isTrue(); assertThat(table.currentSnapshot().firstRowId()).isEqualTo(0); @@ -299,9 +299,9 @@ public void testReplace(int formatVersion) { table .newRewrite() - .deleteFile(file_part1) - .deleteFile(file_part2) - .addFile(file_compacted) + .deleteFile(filePart1) + .deleteFile(filePart2) + .addFile(fileCompacted) .commit(); // Rewrites are currently just treated as appends. In the future we could treat these as no-ops From 5811521192528b7348ce58ae2684458940d881f3 Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Mon, 13 Jan 2025 15:40:31 -0600 Subject: [PATCH 08/22] Reorder Imports --- .../java/org/apache/iceberg/TestRowLineageMetadata.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java b/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java index 4dc0780e32d9..d549694b1aa7 100644 --- a/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java +++ b/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java @@ -18,10 +18,6 @@ */ package org.apache.iceberg; -import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.Assert.assertThrows; -import static org.junit.jupiter.api.Assumptions.assumeTrue; - import java.io.File; import java.util.concurrent.atomic.AtomicInteger; import org.apache.iceberg.exceptions.ValidationException; @@ -31,6 +27,10 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.FieldSource; +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.Assert.assertThrows; +import static org.junit.jupiter.api.Assumptions.assumeTrue; + public class TestRowLineageMetadata { private static final String TEST_LOCATION = "s3://bucket/test/location"; From f472497127a24a5d561be012e076df11e9eed02c Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Mon, 13 Jan 2025 17:00:18 -0600 Subject: [PATCH 09/22] Add ability to set row-lineage via table property --- .../org/apache/iceberg/TableMetadata.java | 19 ++++++++++++ .../org/apache/iceberg/TableProperties.java | 2 ++ .../iceberg/TestRowLineageMetadata.java | 29 +++++++++++++++++++ 3 files changed, 50 insertions(+) diff --git a/core/src/main/java/org/apache/iceberg/TableMetadata.java b/core/src/main/java/org/apache/iceberg/TableMetadata.java index 1ad98184378f..b2e6cdd1e5d1 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadata.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadata.java @@ -638,10 +638,13 @@ public TableMetadata replaceProperties(Map rawProperties) { int newFormatVersion = PropertyUtil.propertyAsInt(rawProperties, TableProperties.FORMAT_VERSION, formatVersion); + Boolean newRowLineage = PropertyUtil.propertyAsBoolean(rawProperties, TableProperties.ROW_LINEAGE, rowLineage); + return new Builder(this) .setProperties(updated) .removeProperties(removed) .upgradeFormatVersion(newFormatVersion) + .setRowLineage(newRowLineage) .build(); } @@ -1510,6 +1513,22 @@ public Builder setPreviousFileLocation(String previousFileLocation) { return this; } + private Builder setRowLineage(Boolean newRowLineage) { + if (newRowLineage == null) { + return this; + } + + boolean disablingRowLineage = rowLineage && !newRowLineage; + + Preconditions.checkArgument(!disablingRowLineage, "Cannot disable row lineage once it has been enabled"); + + if (!rowLineage && newRowLineage) { + return enableRowLineage(); + } else { + return this; + } + } + public Builder enableRowLineage() { Preconditions.checkArgument( formatVersion >= MIN_FORMAT_VERSION_ROW_LINEAGE, diff --git a/core/src/main/java/org/apache/iceberg/TableProperties.java b/core/src/main/java/org/apache/iceberg/TableProperties.java index c137bcd3a2c3..cd7cda23c2d3 100644 --- a/core/src/main/java/org/apache/iceberg/TableProperties.java +++ b/core/src/main/java/org/apache/iceberg/TableProperties.java @@ -388,4 +388,6 @@ private TableProperties() {} public static final int ENCRYPTION_DEK_LENGTH_DEFAULT = 16; public static final int ENCRYPTION_AAD_LENGTH_DEFAULT = 16; + + public static final String ROW_LINEAGE = "row-lineage"; } diff --git a/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java b/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java index d549694b1aa7..25bbd8681e61 100644 --- a/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java +++ b/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java @@ -309,4 +309,33 @@ public void testReplace(int formatVersion) { assertThat(table.currentSnapshot().firstRowId()).isEqualTo(60); assertThat(table.ops().current().lastRowId()).isEqualTo(120); } + + + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + public void testEnableRowLineageViaProperty(int formatVersion) { + assumeTrue(formatVersion >= TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); + + TestTables.TestTable table = + TestTables.create( + tableDir, "test", TEST_SCHEMA, PartitionSpec.unpartitioned(), formatVersion); + + assertThat(table.ops().current().rowLineage()).isFalse(); + + // No-op + table.updateProperties().set(TableProperties.ROW_LINEAGE, "false").commit(); + assertThat(table.ops().current().rowLineage()).isFalse(); + + // Enable row lineage + table.updateProperties().set(TableProperties.ROW_LINEAGE, "true").commit(); + assertThat(table.ops().current().rowLineage()).isTrue(); + + // Disabling row lineage is not allowed + IllegalArgumentException cannotDisable = assertThrows(IllegalArgumentException.class, () -> table.updateProperties().set(TableProperties.ROW_LINEAGE, "false").commit()); + assertThat(cannotDisable.getMessage()).contains("Cannot disable row lineage once it has been enabled"); + + // No-op + table.updateProperties().set(TableProperties.ROW_LINEAGE, "true").commit(); + assertThat(table.ops().current().rowLineage()).isTrue(); + } } From 1023d81252ace0392e17cfeec95d8ac0407511bf Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Tue, 14 Jan 2025 08:49:30 -0600 Subject: [PATCH 10/22] Spotless Apply --- .../org/apache/iceberg/TableMetadata.java | 6 ++-- .../iceberg/TestRowLineageMetadata.java | 28 +++++++++---------- 2 files changed, 17 insertions(+), 17 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/TableMetadata.java b/core/src/main/java/org/apache/iceberg/TableMetadata.java index b2e6cdd1e5d1..0f7a700b3b44 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadata.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadata.java @@ -638,7 +638,8 @@ public TableMetadata replaceProperties(Map rawProperties) { int newFormatVersion = PropertyUtil.propertyAsInt(rawProperties, TableProperties.FORMAT_VERSION, formatVersion); - Boolean newRowLineage = PropertyUtil.propertyAsBoolean(rawProperties, TableProperties.ROW_LINEAGE, rowLineage); + Boolean newRowLineage = + PropertyUtil.propertyAsBoolean(rawProperties, TableProperties.ROW_LINEAGE, rowLineage); return new Builder(this) .setProperties(updated) @@ -1520,7 +1521,8 @@ private Builder setRowLineage(Boolean newRowLineage) { boolean disablingRowLineage = rowLineage && !newRowLineage; - Preconditions.checkArgument(!disablingRowLineage, "Cannot disable row lineage once it has been enabled"); + Preconditions.checkArgument( + !disablingRowLineage, "Cannot disable row lineage once it has been enabled"); if (!rowLineage && newRowLineage) { return enableRowLineage(); diff --git a/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java b/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java index 25bbd8681e61..c680b219eadf 100644 --- a/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java +++ b/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java @@ -18,6 +18,10 @@ */ package org.apache.iceberg; +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.Assert.assertThrows; +import static org.junit.jupiter.api.Assumptions.assumeTrue; + import java.io.File; import java.util.concurrent.atomic.AtomicInteger; import org.apache.iceberg.exceptions.ValidationException; @@ -27,10 +31,6 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.FieldSource; -import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.Assert.assertThrows; -import static org.junit.jupiter.api.Assumptions.assumeTrue; - public class TestRowLineageMetadata { private static final String TEST_LOCATION = "s3://bucket/test/location"; @@ -297,12 +297,7 @@ public void testReplace(int formatVersion) { assertThat(table.currentSnapshot().firstRowId()).isEqualTo(0); assertThat(table.ops().current().lastRowId()).isEqualTo(60); - table - .newRewrite() - .deleteFile(filePart1) - .deleteFile(filePart2) - .addFile(fileCompacted) - .commit(); + table.newRewrite().deleteFile(filePart1).deleteFile(filePart2).addFile(fileCompacted).commit(); // Rewrites are currently just treated as appends. In the future we could treat these as no-ops assertThat(table.ops().current().rowLineage()).isTrue(); @@ -310,15 +305,14 @@ public void testReplace(int formatVersion) { assertThat(table.ops().current().lastRowId()).isEqualTo(120); } - @ParameterizedTest @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") public void testEnableRowLineageViaProperty(int formatVersion) { assumeTrue(formatVersion >= TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); TestTables.TestTable table = - TestTables.create( - tableDir, "test", TEST_SCHEMA, PartitionSpec.unpartitioned(), formatVersion); + TestTables.create( + tableDir, "test", TEST_SCHEMA, PartitionSpec.unpartitioned(), formatVersion); assertThat(table.ops().current().rowLineage()).isFalse(); @@ -331,8 +325,12 @@ public void testEnableRowLineageViaProperty(int formatVersion) { assertThat(table.ops().current().rowLineage()).isTrue(); // Disabling row lineage is not allowed - IllegalArgumentException cannotDisable = assertThrows(IllegalArgumentException.class, () -> table.updateProperties().set(TableProperties.ROW_LINEAGE, "false").commit()); - assertThat(cannotDisable.getMessage()).contains("Cannot disable row lineage once it has been enabled"); + IllegalArgumentException cannotDisable = + assertThrows( + IllegalArgumentException.class, + () -> table.updateProperties().set(TableProperties.ROW_LINEAGE, "false").commit()); + assertThat(cannotDisable.getMessage()) + .contains("Cannot disable row lineage once it has been enabled"); // No-op table.updateProperties().set(TableProperties.ROW_LINEAGE, "true").commit(); From d7c54165da992a78b6040b274f3c193d4839f156 Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Wed, 15 Jan 2025 15:56:51 -0600 Subject: [PATCH 11/22] Review Comments --- .../org/apache/iceberg/MetadataUpdate.java | 2 +- .../apache/iceberg/RewriteTablePathUtil.java | 2 +- .../org/apache/iceberg/SnapshotProducer.java | 2 +- .../org/apache/iceberg/TableMetadata.java | 17 +-- .../apache/iceberg/TableMetadataParser.java | 4 +- .../iceberg/TestRowLineageMetadata.java | 125 ++++++++---------- .../org/apache/iceberg/TestTableMetadata.java | 4 +- .../org/apache/iceberg/util/TestJsonUtil.java | 20 +++ 8 files changed, 92 insertions(+), 84 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/MetadataUpdate.java b/core/src/main/java/org/apache/iceberg/MetadataUpdate.java index c24b2d42b291..03904ed55ab9 100644 --- a/core/src/main/java/org/apache/iceberg/MetadataUpdate.java +++ b/core/src/main/java/org/apache/iceberg/MetadataUpdate.java @@ -521,7 +521,7 @@ public void applyTo(ViewMetadata.Builder viewMetadataBuilder) { class EnableRowLineage implements MetadataUpdate { @Override public void applyTo(TableMetadata.Builder metadataBuilder) { - MetadataUpdate.super.applyTo(metadataBuilder.enableRowLineage()); + metadataBuilder.enableRowLineage(); } } } diff --git a/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java b/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java index ab298bd6d733..ba151833fe85 100644 --- a/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java +++ b/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java @@ -130,7 +130,7 @@ public static TableMetadata replacePaths( metadata.statisticsFiles(), metadata.partitionStatisticsFiles(), metadata.changes(), - metadata.rowLineage(), + metadata.rowLineageEnabled(), metadata.lastRowId()); } diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java index 5c6884d79c9e..05c3c406cbc0 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java @@ -285,7 +285,7 @@ public Snapshot apply() { Long addedRows = null; Long lastRowId = null; - if (base.rowLineage()) { + if (base.rowLineageEnabled()) { addedRows = manifests.stream() .filter( diff --git a/core/src/main/java/org/apache/iceberg/TableMetadata.java b/core/src/main/java/org/apache/iceberg/TableMetadata.java index 0f7a700b3b44..e090cec12d40 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadata.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadata.java @@ -265,7 +265,7 @@ public String toString() { private volatile Map snapshotsById; private volatile Map refs; private volatile boolean snapshotsLoaded; - private final Boolean rowLineage; + private final Boolean rowLineageEnabled; private final long lastRowId; @SuppressWarnings("checkstyle:CyclomaticComplexity") @@ -294,7 +294,7 @@ public String toString() { List statisticsFiles, List partitionStatisticsFiles, List changes, - boolean rowLineage, + boolean rowLineageEnabled, long lastRowId) { Preconditions.checkArgument( specs != null && !specs.isEmpty(), "Partition specs cannot be null or empty"); @@ -315,7 +315,7 @@ public String toString() { metadataFileLocation == null || changes.isEmpty(), "Cannot create TableMetadata with a metadata location and changes"); Preconditions.checkArgument( - formatVersion >= MIN_FORMAT_VERSION_ROW_LINEAGE || !rowLineage, + formatVersion >= MIN_FORMAT_VERSION_ROW_LINEAGE || !rowLineageEnabled, "Cannot enable row lineage when Table Version is less than V3. Table Version is %s", formatVersion); @@ -353,7 +353,7 @@ public String toString() { this.partitionStatisticsFiles = ImmutableList.copyOf(partitionStatisticsFiles); // row lineage - this.rowLineage = rowLineage; + this.rowLineageEnabled = rowLineageEnabled; this.lastRowId = lastRowId; HistoryEntry last = null; @@ -578,8 +578,8 @@ public TableMetadata withUUID() { return new Builder(this).assignUUID().build(); } - public boolean rowLineage() { - return rowLineage; + public boolean rowLineageEnabled() { + return rowLineageEnabled; } public long lastRowId() { @@ -639,7 +639,8 @@ public TableMetadata replaceProperties(Map rawProperties) { PropertyUtil.propertyAsInt(rawProperties, TableProperties.FORMAT_VERSION, formatVersion); Boolean newRowLineage = - PropertyUtil.propertyAsBoolean(rawProperties, TableProperties.ROW_LINEAGE, rowLineage); + PropertyUtil.propertyAsBoolean( + rawProperties, TableProperties.ROW_LINEAGE, rowLineageEnabled); return new Builder(this) .setProperties(updated) @@ -1014,7 +1015,7 @@ private Builder(TableMetadata base) { this.specsById = Maps.newHashMap(base.specsById); this.sortOrdersById = Maps.newHashMap(base.sortOrdersById); - this.rowLineage = base.rowLineage; + this.rowLineage = base.rowLineageEnabled; this.lastRowId = base.lastRowId; } diff --git a/core/src/main/java/org/apache/iceberg/TableMetadataParser.java b/core/src/main/java/org/apache/iceberg/TableMetadataParser.java index 33a794852651..eb6e9f331542 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadataParser.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadataParser.java @@ -222,8 +222,8 @@ public static void toJson(TableMetadata metadata, JsonGenerator generator) throw generator.writeNullField(CURRENT_SNAPSHOT_ID); } - if (metadata.rowLineage()) { - generator.writeBooleanField(ROW_LINEAGE, metadata.rowLineage()); + if (metadata.rowLineageEnabled()) { + generator.writeBooleanField(ROW_LINEAGE, metadata.rowLineageEnabled()); generator.writeNumberField(LAST_ROW_ID, metadata.lastRowId()); } diff --git a/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java b/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java index c680b219eadf..3476cc9ddb9d 100644 --- a/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java +++ b/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java @@ -19,8 +19,8 @@ package org.apache.iceberg; import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.Assert.assertThrows; -import static org.junit.jupiter.api.Assumptions.assumeTrue; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assumptions.assumeThat; import java.io.File; import java.util.concurrent.atomic.AtomicInteger; @@ -33,9 +33,9 @@ public class TestRowLineageMetadata { - private static final String TEST_LOCATION = "s3://bucket/test/location"; + @TempDir private File tableDir = null; - @TempDir protected File tableDir = null; + private static final String TEST_LOCATION = "s3://bucket/test/location"; private static final Schema TEST_SCHEMA = new Schema( @@ -44,12 +44,8 @@ public class TestRowLineageMetadata { Types.NestedField.required(2, "y", Types.LongType.get(), "comment"), Types.NestedField.required(3, "z", Types.LongType.get())); - private TableMetadata.Builder builderFor(int formatVersion) { - return TableMetadata.buildFromEmpty(formatVersion).enableRowLineage(); - } - private TableMetadata baseMetadata(int formatVersion) { - return builderFor(formatVersion) + return TableMetadata.buildFromEmpty(formatVersion) .addSchema(TEST_SCHEMA) .setLocation(TEST_LOCATION) .addPartitionSpec(PartitionSpec.unpartitioned()) @@ -66,20 +62,18 @@ public void cleanup() { @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") public void testRowLineageSupported(int formatVersion) { if (formatVersion == TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE) { - assertThat(builderFor(formatVersion)).isNotNull(); + assertThat(TableMetadata.buildFromEmpty(formatVersion)).isNotNull(); } else { - IllegalArgumentException notSupported = - assertThrows( - IllegalArgumentException.class, - () -> TableMetadata.buildFromEmpty(formatVersion).enableRowLineage()); - assertThat(notSupported.getMessage()).contains("Cannot use row lineage"); + assertThatThrownBy(() -> TableMetadata.buildFromEmpty(formatVersion).enableRowLineage()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot use row lineage"); } } @ParameterizedTest @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") public void testSnapshotAddition(int formatVersion) { - assumeTrue(formatVersion >= TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); + assumeThat(formatVersion).isGreaterThanOrEqualTo(TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); Long newRows = 30L; @@ -106,7 +100,7 @@ public void testSnapshotAddition(int formatVersion) { @ParameterizedTest @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") public void testInvalidSnapshotAddition(int formatVersion) { - assumeTrue(formatVersion >= TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); + assumeThat(formatVersion).isGreaterThanOrEqualTo(TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); Long newRows = 30L; @@ -116,39 +110,24 @@ public void testInvalidSnapshotAddition(int formatVersion) { new BaseSnapshot( 0, 1, null, 0, DataOperations.APPEND, null, 1, "foo", base.lastRowId() - 3, newRows); - ValidationException invalidLastRowId = - assertThrows( - ValidationException.class, - () -> TableMetadata.buildFrom(base).addSnapshot(invalidLastRow)); - assertThat(invalidLastRowId.getMessage()).contains("Cannot add a snapshot whose first-row-id"); + assertThatThrownBy(() -> TableMetadata.buildFrom(base).addSnapshot(invalidLastRow)) + .isInstanceOf(ValidationException.class) + .hasMessageContaining("Cannot add a snapshot whose first-row-id"); Snapshot invalidNewRows = new BaseSnapshot( 0, 1, null, 0, DataOperations.APPEND, null, 1, "foo", base.lastRowId(), null); - ValidationException nullNewRows = - assertThrows( - ValidationException.class, - () -> TableMetadata.buildFrom(base).addSnapshot(invalidNewRows)); - assertThat(nullNewRows.getMessage()) - .contains( + assertThatThrownBy(() -> TableMetadata.buildFrom(base).addSnapshot(invalidNewRows)) + .isInstanceOf(ValidationException.class) + .hasMessageContaining( "Cannot add a snapshot with a null `addedRows` field when row lineage is enabled"); } - private AtomicInteger fileNum = new AtomicInteger(0); - - private DataFile fileWithRows(int numRows) { - return DataFiles.builder(PartitionSpec.unpartitioned()) - .withRecordCount(numRows) - .withFileSizeInBytes(numRows * 100) - .withPath("file://file_" + fileNum.incrementAndGet() + ".parquet") - .build(); - } - @ParameterizedTest @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") public void testFastAppend(int formatVersion) { - assumeTrue(formatVersion >= TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); + assumeThat(formatVersion).isGreaterThanOrEqualTo(TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); TestTables.TestTable table = TestTables.create( @@ -156,18 +135,18 @@ public void testFastAppend(int formatVersion) { TableMetadata base = table.ops().current(); table.ops().commit(base, TableMetadata.buildFrom(base).enableRowLineage().build()); - assertThat(table.ops().current().rowLineage()).isTrue(); + assertThat(table.ops().current().rowLineageEnabled()).isTrue(); assertThat(table.ops().current().lastRowId()).isEqualTo(0L); table.newFastAppend().appendFile(fileWithRows(30)).commit(); - assertThat(table.ops().current().rowLineage()).isTrue(); + assertThat(table.ops().current().rowLineageEnabled()).isTrue(); assertThat(table.currentSnapshot().firstRowId()).isEqualTo(0); assertThat(table.ops().current().lastRowId()).isEqualTo(30); table.newFastAppend().appendFile(fileWithRows(17)).appendFile(fileWithRows(11)).commit(); - assertThat(table.ops().current().rowLineage()).isTrue(); + assertThat(table.ops().current().rowLineageEnabled()).isTrue(); assertThat(table.currentSnapshot().firstRowId()).isEqualTo(30); assertThat(table.ops().current().lastRowId()).isEqualTo(30 + 17 + 11); } @@ -175,7 +154,7 @@ public void testFastAppend(int formatVersion) { @ParameterizedTest @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") public void testAppend(int formatVersion) { - assumeTrue(formatVersion >= TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); + assumeThat(formatVersion).isGreaterThanOrEqualTo(TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); TestTables.TestTable table = TestTables.create( @@ -183,18 +162,18 @@ public void testAppend(int formatVersion) { TableMetadata base = table.ops().current(); table.ops().commit(base, TableMetadata.buildFrom(base).enableRowLineage().build()); - assertThat(table.ops().current().rowLineage()).isTrue(); + assertThat(table.ops().current().rowLineageEnabled()).isTrue(); assertThat(table.ops().current().lastRowId()).isEqualTo(0L); table.newAppend().appendFile(fileWithRows(30)).commit(); - assertThat(table.ops().current().rowLineage()).isTrue(); + assertThat(table.ops().current().rowLineageEnabled()).isTrue(); assertThat(table.currentSnapshot().firstRowId()).isEqualTo(0); assertThat(table.ops().current().lastRowId()).isEqualTo(30); table.newAppend().appendFile(fileWithRows(17)).appendFile(fileWithRows(11)).commit(); - assertThat(table.ops().current().rowLineage()).isTrue(); + assertThat(table.ops().current().rowLineageEnabled()).isTrue(); assertThat(table.currentSnapshot().firstRowId()).isEqualTo(30); assertThat(table.ops().current().lastRowId()).isEqualTo(30 + 17 + 11); } @@ -202,7 +181,7 @@ public void testAppend(int formatVersion) { @ParameterizedTest @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") public void testAppendBranch(int formatVersion) { - assumeTrue(formatVersion >= TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); + assumeThat(formatVersion).isGreaterThanOrEqualTo(TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); // Appends to a branch should still change last-row-id even if not on main, these changes // should also affect commits to main @@ -215,13 +194,13 @@ public void testAppendBranch(int formatVersion) { TableMetadata base = table.ops().current(); table.ops().commit(base, TableMetadata.buildFrom(base).enableRowLineage().build()); - assertThat(table.ops().current().rowLineage()).isTrue(); + assertThat(table.ops().current().rowLineageEnabled()).isTrue(); assertThat(table.ops().current().lastRowId()).isEqualTo(0L); // Write to Branch table.newAppend().appendFile(fileWithRows(30)).toBranch(branch).commit(); - assertThat(table.ops().current().rowLineage()).isTrue(); + assertThat(table.ops().current().rowLineageEnabled()).isTrue(); assertThat(table.currentSnapshot()).isNull(); assertThat(table.snapshot(branch).firstRowId()).isEqualTo(0L); assertThat(table.ops().current().lastRowId()).isEqualTo(30); @@ -229,7 +208,7 @@ public void testAppendBranch(int formatVersion) { // Write to Main table.newAppend().appendFile(fileWithRows(17)).appendFile(fileWithRows(11)).commit(); - assertThat(table.ops().current().rowLineage()).isTrue(); + assertThat(table.ops().current().rowLineageEnabled()).isTrue(); assertThat(table.currentSnapshot().firstRowId()).isEqualTo(30); assertThat(table.ops().current().lastRowId()).isEqualTo(30 + 17 + 11); @@ -242,7 +221,7 @@ public void testAppendBranch(int formatVersion) { @ParameterizedTest @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") public void testDeletes(int formatVersion) { - assumeTrue(formatVersion >= TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); + assumeThat(formatVersion).isGreaterThanOrEqualTo(TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); TestTables.TestTable table = TestTables.create( @@ -250,14 +229,14 @@ public void testDeletes(int formatVersion) { TableMetadata base = table.ops().current(); table.ops().commit(base, TableMetadata.buildFrom(base).enableRowLineage().build()); - assertThat(table.ops().current().rowLineage()).isTrue(); + assertThat(table.ops().current().rowLineageEnabled()).isTrue(); assertThat(table.ops().current().lastRowId()).isEqualTo(0L); DataFile file = fileWithRows(30); table.newAppend().appendFile(file).commit(); - assertThat(table.ops().current().rowLineage()).isTrue(); + assertThat(table.ops().current().rowLineageEnabled()).isTrue(); assertThat(table.currentSnapshot().firstRowId()).isEqualTo(0); assertThat(table.ops().current().lastRowId()).isEqualTo(30); @@ -266,7 +245,7 @@ public void testDeletes(int formatVersion) { // Deleting a file should create a new snapshot which should inherit last-row-id from the // previous metadata and not // change last-row-id for this metadata. - assertThat(table.ops().current().rowLineage()).isTrue(); + assertThat(table.ops().current().rowLineageEnabled()).isTrue(); assertThat(table.currentSnapshot().firstRowId()).isEqualTo(30); assertThat(table.currentSnapshot().addedRows()).isEqualTo(0); assertThat(table.ops().current().lastRowId()).isEqualTo(30); @@ -275,7 +254,7 @@ public void testDeletes(int formatVersion) { @ParameterizedTest @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") public void testReplace(int formatVersion) { - assumeTrue(formatVersion >= TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); + assumeThat(formatVersion).isGreaterThanOrEqualTo(TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); TestTables.TestTable table = TestTables.create( @@ -284,7 +263,7 @@ public void testReplace(int formatVersion) { table.ops().commit(base, TableMetadata.buildFrom(base).enableRowLineage().build()); - assertThat(table.ops().current().rowLineage()).isTrue(); + assertThat(table.ops().current().rowLineageEnabled()).isTrue(); assertThat(table.ops().current().lastRowId()).isEqualTo(0L); DataFile filePart1 = fileWithRows(30); @@ -293,14 +272,14 @@ public void testReplace(int formatVersion) { table.newAppend().appendFile(filePart1).appendFile(filePart2).commit(); - assertThat(table.ops().current().rowLineage()).isTrue(); + assertThat(table.ops().current().rowLineageEnabled()).isTrue(); assertThat(table.currentSnapshot().firstRowId()).isEqualTo(0); assertThat(table.ops().current().lastRowId()).isEqualTo(60); table.newRewrite().deleteFile(filePart1).deleteFile(filePart2).addFile(fileCompacted).commit(); // Rewrites are currently just treated as appends. In the future we could treat these as no-ops - assertThat(table.ops().current().rowLineage()).isTrue(); + assertThat(table.ops().current().rowLineageEnabled()).isTrue(); assertThat(table.currentSnapshot().firstRowId()).isEqualTo(60); assertThat(table.ops().current().lastRowId()).isEqualTo(120); } @@ -308,32 +287,40 @@ public void testReplace(int formatVersion) { @ParameterizedTest @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") public void testEnableRowLineageViaProperty(int formatVersion) { - assumeTrue(formatVersion >= TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); + assumeThat(formatVersion).isGreaterThanOrEqualTo(TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); TestTables.TestTable table = TestTables.create( tableDir, "test", TEST_SCHEMA, PartitionSpec.unpartitioned(), formatVersion); - assertThat(table.ops().current().rowLineage()).isFalse(); + assertThat(table.ops().current().rowLineageEnabled()).isFalse(); // No-op table.updateProperties().set(TableProperties.ROW_LINEAGE, "false").commit(); - assertThat(table.ops().current().rowLineage()).isFalse(); + assertThat(table.ops().current().rowLineageEnabled()).isFalse(); // Enable row lineage table.updateProperties().set(TableProperties.ROW_LINEAGE, "true").commit(); - assertThat(table.ops().current().rowLineage()).isTrue(); + assertThat(table.ops().current().rowLineageEnabled()).isTrue(); // Disabling row lineage is not allowed - IllegalArgumentException cannotDisable = - assertThrows( - IllegalArgumentException.class, - () -> table.updateProperties().set(TableProperties.ROW_LINEAGE, "false").commit()); - assertThat(cannotDisable.getMessage()) - .contains("Cannot disable row lineage once it has been enabled"); + assertThatThrownBy( + () -> table.updateProperties().set(TableProperties.ROW_LINEAGE, "false").commit()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot disable row lineage once it has been enabled"); // No-op table.updateProperties().set(TableProperties.ROW_LINEAGE, "true").commit(); - assertThat(table.ops().current().rowLineage()).isTrue(); + assertThat(table.ops().current().rowLineageEnabled()).isTrue(); + } + + private final AtomicInteger fileNum = new AtomicInteger(0); + + private DataFile fileWithRows(long numRows) { + return DataFiles.builder(PartitionSpec.unpartitioned()) + .withRecordCount(numRows) + .withFileSizeInBytes(numRows * 100) + .withPath("file://file_" + fileNum.incrementAndGet() + ".parquet") + .build(); } } diff --git a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java index 9f7ad05688d5..ce3e799c5312 100644 --- a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java +++ b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java @@ -231,7 +231,7 @@ public void testJsonConversion() throws Exception { assertThat(metadata.statisticsFiles()).isEqualTo(statisticsFiles); assertThat(metadata.partitionStatisticsFiles()).isEqualTo(partitionStatisticsFiles); assertThat(metadata.refs()).isEqualTo(refs); - assertThat(metadata.rowLineage()).isEqualTo(expected.rowLineage()); + assertThat(metadata.rowLineageEnabled()).isEqualTo(expected.rowLineageEnabled()); assertThat(metadata.lastRowId()).isEqualTo(expected.lastRowId()); } @@ -341,7 +341,7 @@ public void testBackwardCompat() throws Exception { .isEqualTo(previousSnapshot.allManifests(ops.io())); assertThat(metadata.previousFiles()).isEqualTo(expected.previousFiles()); assertThat(metadata.snapshot(previousSnapshotId).schemaId()).isNull(); - assertThat(metadata.rowLineage()).isEqualTo(expected.rowLineage()); + assertThat(metadata.rowLineageEnabled()).isEqualTo(expected.rowLineageEnabled()); assertThat(metadata.lastRowId()).isEqualTo(expected.lastRowId()); } diff --git a/core/src/test/java/org/apache/iceberg/util/TestJsonUtil.java b/core/src/test/java/org/apache/iceberg/util/TestJsonUtil.java index 7702d691afd0..4bac5677a856 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestJsonUtil.java +++ b/core/src/test/java/org/apache/iceberg/util/TestJsonUtil.java @@ -195,6 +195,26 @@ public void getBool() throws JsonProcessingException { assertThat(JsonUtil.getBool("x", JsonUtil.mapper().readTree("{\"x\": false}"))).isFalse(); } + @Test + public void getBoolOrNull() throws JsonProcessingException { + assertThat(JsonUtil.getBoolOrNull("x", JsonUtil.mapper().readTree("{}"))).isNull(); + + assertThat(JsonUtil.getBoolOrNull("x", JsonUtil.mapper().readTree("{\"x\": null}"))).isNull(); + + assertThatThrownBy( + () -> JsonUtil.getBoolOrNull("x", JsonUtil.mapper().readTree("{\"x\": \"23\"}"))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse to a boolean value: x: \"23\""); + + assertThatThrownBy( + () -> JsonUtil.getBoolOrNull("x", JsonUtil.mapper().readTree("{\"x\": \"true\"}"))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse to a boolean value: x: \"true\""); + + assertThat(JsonUtil.getBoolOrNull("x", JsonUtil.mapper().readTree("{\"x\": true}"))).isTrue(); + assertThat(JsonUtil.getBoolOrNull("x", JsonUtil.mapper().readTree("{\"x\": false}"))).isFalse(); + } + @Test public void getIntArrayOrNull() throws JsonProcessingException { assertThat(JsonUtil.getIntArrayOrNull("items", JsonUtil.mapper().readTree("{}"))).isNull(); From 2c0546d4359908271e6d151778aade0cb9c3b569 Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Wed, 15 Jan 2025 16:27:27 -0600 Subject: [PATCH 12/22] Test Fix --- .../src/test/java/org/apache/iceberg/TestRowLineageMetadata.java | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java b/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java index 3476cc9ddb9d..b7ff0e1a26b0 100644 --- a/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java +++ b/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java @@ -46,6 +46,7 @@ public class TestRowLineageMetadata { private TableMetadata baseMetadata(int formatVersion) { return TableMetadata.buildFromEmpty(formatVersion) + .enableRowLineage() .addSchema(TEST_SCHEMA) .setLocation(TEST_LOCATION) .addPartitionSpec(PartitionSpec.unpartitioned()) From f048cc34067017c7bb8be51ed32590975e521656 Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Fri, 17 Jan 2025 16:57:00 -0600 Subject: [PATCH 13/22] Reviewer Comments --- .../apache/iceberg/MetadataUpdateParser.java | 6 +++ .../apache/iceberg/RewriteTablePathUtil.java | 2 +- .../org/apache/iceberg/SnapshotProducer.java | 38 ++++++------- .../org/apache/iceberg/TableMetadata.java | 41 ++++++-------- .../apache/iceberg/TableMetadataParser.java | 6 +-- .../iceberg/TestMetadataUpdateParser.java | 14 +++++ .../iceberg/TestRowLineageMetadata.java | 54 +++++++++---------- .../org/apache/iceberg/TestTableMetadata.java | 4 +- 8 files changed, 90 insertions(+), 75 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/MetadataUpdateParser.java b/core/src/main/java/org/apache/iceberg/MetadataUpdateParser.java index 08d4b3398f10..20aff178957e 100644 --- a/core/src/main/java/org/apache/iceberg/MetadataUpdateParser.java +++ b/core/src/main/java/org/apache/iceberg/MetadataUpdateParser.java @@ -60,6 +60,7 @@ private MetadataUpdateParser() {} static final String SET_PARTITION_STATISTICS = "set-partition-statistics"; static final String REMOVE_PARTITION_STATISTICS = "remove-partition-statistics"; static final String REMOVE_PARTITION_SPECS = "remove-partition-specs"; + static final String ENABLE_ROW_LINEAGE = "enable-row-lineage"; // AssignUUID private static final String UUID = "uuid"; @@ -154,6 +155,7 @@ private MetadataUpdateParser() {} .put(MetadataUpdate.AddViewVersion.class, ADD_VIEW_VERSION) .put(MetadataUpdate.SetCurrentViewVersion.class, SET_CURRENT_VIEW_VERSION) .put(MetadataUpdate.RemovePartitionSpecs.class, REMOVE_PARTITION_SPECS) + .put(MetadataUpdate.EnableRowLineage.class, ENABLE_ROW_LINEAGE) .buildOrThrow(); public static String toJson(MetadataUpdate metadataUpdate) { @@ -249,6 +251,8 @@ public static void toJson(MetadataUpdate metadataUpdate, JsonGenerator generator case REMOVE_PARTITION_SPECS: writeRemovePartitionSpecs((MetadataUpdate.RemovePartitionSpecs) metadataUpdate, generator); break; + case ENABLE_ROW_LINEAGE: + break; default: throw new IllegalArgumentException( String.format( @@ -322,6 +326,8 @@ public static MetadataUpdate fromJson(JsonNode jsonNode) { return readCurrentViewVersionId(jsonNode); case REMOVE_PARTITION_SPECS: return readRemovePartitionSpecs(jsonNode); + case ENABLE_ROW_LINEAGE: + return new MetadataUpdate.EnableRowLineage(); default: throw new UnsupportedOperationException( String.format("Cannot convert metadata update action to json: %s", action)); diff --git a/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java b/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java index ba151833fe85..d0a5d4aa941a 100644 --- a/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java +++ b/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java @@ -131,7 +131,7 @@ public static TableMetadata replacePaths( metadata.partitionStatisticsFiles(), metadata.changes(), metadata.rowLineageEnabled(), - metadata.lastRowId()); + metadata.nextRowId()); } private static Map updateProperties( diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java index 05c3c406cbc0..0abbd27ae78e 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java @@ -286,24 +286,8 @@ public Snapshot apply() { Long addedRows = null; Long lastRowId = null; if (base.rowLineageEnabled()) { - addedRows = - manifests.stream() - .filter( - manifest -> - manifest.snapshotId() == null - || Objects.equals(manifest.snapshotId(), this.snapshotId)) - .mapToLong( - manifest -> { - Preconditions.checkArgument( - manifest.addedRowsCount() != null, - "Cannot determine number of added rows in snapshot because" - + " the entry for manifest %s is missing the field `added-rows-count`", - manifest.path()); - return manifest.addedRowsCount(); - }) - .sum(); - - lastRowId = base.lastRowId(); + addedRows = calculateAddedRows(manifests); + lastRowId = base.nextRowId(); } return new BaseSnapshot( @@ -319,6 +303,24 @@ public Snapshot apply() { addedRows); } + private Long calculateAddedRows(List manifests) { + return manifests.stream() + .filter( + manifest -> + manifest.snapshotId() == null + || Objects.equals(manifest.snapshotId(), this.snapshotId)) + .mapToLong( + manifest -> { + Preconditions.checkArgument( + manifest.addedRowsCount() != null, + "Cannot determine number of added rows in snapshot because" + + " the entry for manifest %s is missing the field `added-rows-count`", + manifest.path()); + return manifest.addedRowsCount(); + }) + .sum(); + } + protected abstract Map summary(); /** Returns the snapshot summary from the implementation and updates totals. */ diff --git a/core/src/main/java/org/apache/iceberg/TableMetadata.java b/core/src/main/java/org/apache/iceberg/TableMetadata.java index e090cec12d40..56caaf54861e 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadata.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadata.java @@ -266,7 +266,7 @@ public String toString() { private volatile Map refs; private volatile boolean snapshotsLoaded; private final Boolean rowLineageEnabled; - private final long lastRowId; + private final long nextRowId; @SuppressWarnings("checkstyle:CyclomaticComplexity") TableMetadata( @@ -295,7 +295,7 @@ public String toString() { List partitionStatisticsFiles, List changes, boolean rowLineageEnabled, - long lastRowId) { + long nextRowId) { Preconditions.checkArgument( specs != null && !specs.isEmpty(), "Partition specs cannot be null or empty"); Preconditions.checkArgument( @@ -354,7 +354,7 @@ public String toString() { // row lineage this.rowLineageEnabled = rowLineageEnabled; - this.lastRowId = lastRowId; + this.nextRowId = nextRowId; HistoryEntry last = null; for (HistoryEntry logEntry : snapshotLog) { @@ -582,8 +582,8 @@ public boolean rowLineageEnabled() { return rowLineageEnabled; } - public long lastRowId() { - return lastRowId; + public long nextRowId() { + return nextRowId; } /** @@ -932,7 +932,7 @@ public static class Builder { private final Map> partitionStatisticsFiles; private boolean suppressHistoricalSnapshots = false; private boolean rowLineage; - private long lastRowId; + private long nextRowId; // change tracking private final List changes; @@ -980,7 +980,7 @@ private Builder(int formatVersion) { this.specsById = Maps.newHashMap(); this.sortOrdersById = Maps.newHashMap(); this.rowLineage = DEFAULT_ROW_LINEAGE; - this.lastRowId = INITIAL_ROW_ID; + this.nextRowId = INITIAL_ROW_ID; } private Builder(TableMetadata base) { @@ -1016,7 +1016,7 @@ private Builder(TableMetadata base) { this.sortOrdersById = Maps.newHashMap(base.sortOrdersById); this.rowLineage = base.rowLineageEnabled; - this.lastRowId = base.lastRowId; + this.nextRowId = base.nextRowId; } public Builder withMetadataLocation(String newMetadataLocation) { @@ -1267,14 +1267,18 @@ public Builder addSnapshot(Snapshot snapshot) { if (rowLineage) { ValidationException.check( - snapshot.firstRowId() >= lastRowId, + snapshot.firstRowId() >= nextRowId, "Cannot add a snapshot whose first-row-id (%s) is less than the metadata `last-used-id` (%s) because this will end up generating duplicate row_ids.", snapshot.firstRowId(), - lastRowId); + nextRowId); ValidationException.check( snapshot.addedRows() != null, - "Cannot add a snapshot with a null `addedRows` field when row lineage is enabled"); - incrementLastRowId(snapshot.addedRows()); + "Cannot add a snapshot with a null `added-rows` field when row lineage is enabled"); + Preconditions.checkArgument( + snapshot.addedRows() >= 0, + "Cannot decrease last-row-id, last-row-id must increase monotonically. Snapshot reports %s added rows"); + + this.nextRowId += snapshot.addedRows(); } return this; @@ -1543,17 +1547,6 @@ public Builder enableRowLineage() { return this; } - private void incrementLastRowId(long delta) { - Preconditions.checkArgument( - rowLineage, "Cannot set last-row-id if row lineage is not enabled"); - Preconditions.checkArgument( - delta >= 0, - "Cannot decrease last-row-id, last-row-id must increase monotonically. Delta was %s", - delta); - - this.lastRowId += delta; - } - private boolean hasChanges() { return changes.size() != startingChangeCount || (discardChanges && !changes.isEmpty()) @@ -1623,7 +1616,7 @@ public TableMetadata build() { .collect(Collectors.toList()), discardChanges ? ImmutableList.of() : ImmutableList.copyOf(changes), rowLineage, - lastRowId); + nextRowId); } private int addSchemaInternal(Schema schema, int newLastColumnId) { diff --git a/core/src/main/java/org/apache/iceberg/TableMetadataParser.java b/core/src/main/java/org/apache/iceberg/TableMetadataParser.java index eb6e9f331542..ba8f2f78c5e9 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadataParser.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadataParser.java @@ -111,7 +111,7 @@ private TableMetadataParser() {} static final String STATISTICS = "statistics"; static final String PARTITION_STATISTICS = "partition-statistics"; static final String ROW_LINEAGE = "row-lineage"; - static final String LAST_ROW_ID = "last-row-id"; + static final String NEXT_ROW_ID = "next-row-id"; public static void overwrite(TableMetadata metadata, OutputFile outputFile) { internalWrite(metadata, outputFile, true); @@ -224,7 +224,7 @@ public static void toJson(TableMetadata metadata, JsonGenerator generator) throw if (metadata.rowLineageEnabled()) { generator.writeBooleanField(ROW_LINEAGE, metadata.rowLineageEnabled()); - generator.writeNumberField(LAST_ROW_ID, metadata.lastRowId()); + generator.writeNumberField(NEXT_ROW_ID, metadata.nextRowId()); } toJson(metadata.refs(), generator); @@ -464,7 +464,7 @@ public static TableMetadata fromJson(String metadataLocation, JsonNode node) { Boolean rowLineage = JsonUtil.getBoolOrNull(ROW_LINEAGE, node); long lastRowId; if (rowLineage != null && rowLineage) { - lastRowId = JsonUtil.getLong(LAST_ROW_ID, node); + lastRowId = JsonUtil.getLong(NEXT_ROW_ID, node); } else { rowLineage = TableMetadata.DEFAULT_ROW_LINEAGE; lastRowId = TableMetadata.INITIAL_ROW_ID; diff --git a/core/src/test/java/org/apache/iceberg/TestMetadataUpdateParser.java b/core/src/test/java/org/apache/iceberg/TestMetadataUpdateParser.java index f19c4a975c77..27bf5f26eda2 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetadataUpdateParser.java +++ b/core/src/test/java/org/apache/iceberg/TestMetadataUpdateParser.java @@ -931,6 +931,17 @@ public void testRemovePartitionSpec() { .isEqualTo(json); } + @Test + public void testEnableRowLineage() { + String action = MetadataUpdateParser.ENABLE_ROW_LINEAGE; + String json = "{\"action\":\"enable-row-lineage\"}"; + MetadataUpdate expected = new MetadataUpdate.EnableRowLineage(); + assertEquals(action, expected, MetadataUpdateParser.fromJson(json)); + assertThat(MetadataUpdateParser.toJson(expected)) + .as("Enable row lineage should convert to the correct JSON value") + .isEqualTo(json); + } + public void assertEquals( String action, MetadataUpdate expectedUpdate, MetadataUpdate actualUpdate) { switch (action) { @@ -1040,6 +1051,9 @@ public void assertEquals( (MetadataUpdate.RemovePartitionSpecs) expectedUpdate, (MetadataUpdate.RemovePartitionSpecs) actualUpdate); break; + case MetadataUpdateParser.ENABLE_ROW_LINEAGE: + assertThat(actualUpdate).isInstanceOf(MetadataUpdate.EnableRowLineage.class); + break; default: fail("Unrecognized metadata update action: " + action); } diff --git a/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java b/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java index b7ff0e1a26b0..86ce02112bdf 100644 --- a/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java +++ b/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java @@ -33,8 +33,6 @@ public class TestRowLineageMetadata { - @TempDir private File tableDir = null; - private static final String TEST_LOCATION = "s3://bucket/test/location"; private static final Schema TEST_SCHEMA = @@ -54,6 +52,8 @@ private TableMetadata baseMetadata(int formatVersion) { .build(); } + @TempDir private File tableDir = null; + @AfterEach public void cleanup() { TestTables.clearTables(); @@ -62,8 +62,8 @@ public void cleanup() { @ParameterizedTest @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") public void testRowLineageSupported(int formatVersion) { - if (formatVersion == TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE) { - assertThat(TableMetadata.buildFromEmpty(formatVersion)).isNotNull(); + if (formatVersion >= TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE) { + assertThat(TableMetadata.buildFromEmpty(formatVersion).enableRowLineage()).isNotNull(); } else { assertThatThrownBy(() -> TableMetadata.buildFromEmpty(formatVersion).enableRowLineage()) .isInstanceOf(IllegalArgumentException.class) @@ -76,26 +76,26 @@ public void testRowLineageSupported(int formatVersion) { public void testSnapshotAddition(int formatVersion) { assumeThat(formatVersion).isGreaterThanOrEqualTo(TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); - Long newRows = 30L; + long newRows = 30L; TableMetadata base = baseMetadata(formatVersion); Snapshot addRows = new BaseSnapshot( - 0, 1, null, 0, DataOperations.APPEND, null, 1, "foo", base.lastRowId(), newRows); + 0, 1, null, 0, DataOperations.APPEND, null, 1, "foo", base.nextRowId(), newRows); TableMetadata firstAddition = TableMetadata.buildFrom(base).addSnapshot(addRows).build(); - assertThat(firstAddition.lastRowId()).isEqualTo(newRows); + assertThat(firstAddition.nextRowId()).isEqualTo(newRows); Snapshot addMoreRows = new BaseSnapshot( - 1, 2, 1L, 0, DataOperations.APPEND, null, 1, "foo", firstAddition.lastRowId(), newRows); + 1, 2, 1L, 0, DataOperations.APPEND, null, 1, "foo", firstAddition.nextRowId(), newRows); TableMetadata secondAddition = TableMetadata.buildFrom(firstAddition).addSnapshot(addMoreRows).build(); - assertThat(secondAddition.lastRowId()).isEqualTo(newRows * 2); + assertThat(secondAddition.nextRowId()).isEqualTo(newRows * 2); } @ParameterizedTest @@ -109,7 +109,7 @@ public void testInvalidSnapshotAddition(int formatVersion) { Snapshot invalidLastRow = new BaseSnapshot( - 0, 1, null, 0, DataOperations.APPEND, null, 1, "foo", base.lastRowId() - 3, newRows); + 0, 1, null, 0, DataOperations.APPEND, null, 1, "foo", base.nextRowId() - 3, newRows); assertThatThrownBy(() -> TableMetadata.buildFrom(base).addSnapshot(invalidLastRow)) .isInstanceOf(ValidationException.class) @@ -117,7 +117,7 @@ public void testInvalidSnapshotAddition(int formatVersion) { Snapshot invalidNewRows = new BaseSnapshot( - 0, 1, null, 0, DataOperations.APPEND, null, 1, "foo", base.lastRowId(), null); + 0, 1, null, 0, DataOperations.APPEND, null, 1, "foo", base.nextRowId(), null); assertThatThrownBy(() -> TableMetadata.buildFrom(base).addSnapshot(invalidNewRows)) .isInstanceOf(ValidationException.class) @@ -137,19 +137,19 @@ public void testFastAppend(int formatVersion) { table.ops().commit(base, TableMetadata.buildFrom(base).enableRowLineage().build()); assertThat(table.ops().current().rowLineageEnabled()).isTrue(); - assertThat(table.ops().current().lastRowId()).isEqualTo(0L); + assertThat(table.ops().current().nextRowId()).isEqualTo(0L); table.newFastAppend().appendFile(fileWithRows(30)).commit(); assertThat(table.ops().current().rowLineageEnabled()).isTrue(); assertThat(table.currentSnapshot().firstRowId()).isEqualTo(0); - assertThat(table.ops().current().lastRowId()).isEqualTo(30); + assertThat(table.ops().current().nextRowId()).isEqualTo(30); table.newFastAppend().appendFile(fileWithRows(17)).appendFile(fileWithRows(11)).commit(); assertThat(table.ops().current().rowLineageEnabled()).isTrue(); assertThat(table.currentSnapshot().firstRowId()).isEqualTo(30); - assertThat(table.ops().current().lastRowId()).isEqualTo(30 + 17 + 11); + assertThat(table.ops().current().nextRowId()).isEqualTo(30 + 17 + 11); } @ParameterizedTest @@ -164,19 +164,19 @@ public void testAppend(int formatVersion) { table.ops().commit(base, TableMetadata.buildFrom(base).enableRowLineage().build()); assertThat(table.ops().current().rowLineageEnabled()).isTrue(); - assertThat(table.ops().current().lastRowId()).isEqualTo(0L); + assertThat(table.ops().current().nextRowId()).isEqualTo(0L); table.newAppend().appendFile(fileWithRows(30)).commit(); assertThat(table.ops().current().rowLineageEnabled()).isTrue(); assertThat(table.currentSnapshot().firstRowId()).isEqualTo(0); - assertThat(table.ops().current().lastRowId()).isEqualTo(30); + assertThat(table.ops().current().nextRowId()).isEqualTo(30); table.newAppend().appendFile(fileWithRows(17)).appendFile(fileWithRows(11)).commit(); assertThat(table.ops().current().rowLineageEnabled()).isTrue(); assertThat(table.currentSnapshot().firstRowId()).isEqualTo(30); - assertThat(table.ops().current().lastRowId()).isEqualTo(30 + 17 + 11); + assertThat(table.ops().current().nextRowId()).isEqualTo(30 + 17 + 11); } @ParameterizedTest @@ -196,7 +196,7 @@ public void testAppendBranch(int formatVersion) { table.ops().commit(base, TableMetadata.buildFrom(base).enableRowLineage().build()); assertThat(table.ops().current().rowLineageEnabled()).isTrue(); - assertThat(table.ops().current().lastRowId()).isEqualTo(0L); + assertThat(table.ops().current().nextRowId()).isEqualTo(0L); // Write to Branch table.newAppend().appendFile(fileWithRows(30)).toBranch(branch).commit(); @@ -204,19 +204,19 @@ public void testAppendBranch(int formatVersion) { assertThat(table.ops().current().rowLineageEnabled()).isTrue(); assertThat(table.currentSnapshot()).isNull(); assertThat(table.snapshot(branch).firstRowId()).isEqualTo(0L); - assertThat(table.ops().current().lastRowId()).isEqualTo(30); + assertThat(table.ops().current().nextRowId()).isEqualTo(30); // Write to Main table.newAppend().appendFile(fileWithRows(17)).appendFile(fileWithRows(11)).commit(); assertThat(table.ops().current().rowLineageEnabled()).isTrue(); assertThat(table.currentSnapshot().firstRowId()).isEqualTo(30); - assertThat(table.ops().current().lastRowId()).isEqualTo(30 + 17 + 11); + assertThat(table.ops().current().nextRowId()).isEqualTo(30 + 17 + 11); // Write again to branch table.newAppend().appendFile(fileWithRows(21)).toBranch(branch).commit(); assertThat(table.snapshot(branch).firstRowId()).isEqualTo(30 + 17 + 11); - assertThat(table.ops().current().lastRowId()).isEqualTo(30 + 17 + 11 + 21); + assertThat(table.ops().current().nextRowId()).isEqualTo(30 + 17 + 11 + 21); } @ParameterizedTest @@ -231,7 +231,7 @@ public void testDeletes(int formatVersion) { table.ops().commit(base, TableMetadata.buildFrom(base).enableRowLineage().build()); assertThat(table.ops().current().rowLineageEnabled()).isTrue(); - assertThat(table.ops().current().lastRowId()).isEqualTo(0L); + assertThat(table.ops().current().nextRowId()).isEqualTo(0L); DataFile file = fileWithRows(30); @@ -239,7 +239,7 @@ public void testDeletes(int formatVersion) { assertThat(table.ops().current().rowLineageEnabled()).isTrue(); assertThat(table.currentSnapshot().firstRowId()).isEqualTo(0); - assertThat(table.ops().current().lastRowId()).isEqualTo(30); + assertThat(table.ops().current().nextRowId()).isEqualTo(30); table.newDelete().deleteFile(file).commit(); @@ -249,7 +249,7 @@ public void testDeletes(int formatVersion) { assertThat(table.ops().current().rowLineageEnabled()).isTrue(); assertThat(table.currentSnapshot().firstRowId()).isEqualTo(30); assertThat(table.currentSnapshot().addedRows()).isEqualTo(0); - assertThat(table.ops().current().lastRowId()).isEqualTo(30); + assertThat(table.ops().current().nextRowId()).isEqualTo(30); } @ParameterizedTest @@ -265,7 +265,7 @@ public void testReplace(int formatVersion) { table.ops().commit(base, TableMetadata.buildFrom(base).enableRowLineage().build()); assertThat(table.ops().current().rowLineageEnabled()).isTrue(); - assertThat(table.ops().current().lastRowId()).isEqualTo(0L); + assertThat(table.ops().current().nextRowId()).isEqualTo(0L); DataFile filePart1 = fileWithRows(30); DataFile filePart2 = fileWithRows(30); @@ -275,14 +275,14 @@ public void testReplace(int formatVersion) { assertThat(table.ops().current().rowLineageEnabled()).isTrue(); assertThat(table.currentSnapshot().firstRowId()).isEqualTo(0); - assertThat(table.ops().current().lastRowId()).isEqualTo(60); + assertThat(table.ops().current().nextRowId()).isEqualTo(60); table.newRewrite().deleteFile(filePart1).deleteFile(filePart2).addFile(fileCompacted).commit(); // Rewrites are currently just treated as appends. In the future we could treat these as no-ops assertThat(table.ops().current().rowLineageEnabled()).isTrue(); assertThat(table.currentSnapshot().firstRowId()).isEqualTo(60); - assertThat(table.ops().current().lastRowId()).isEqualTo(120); + assertThat(table.ops().current().nextRowId()).isEqualTo(120); } @ParameterizedTest diff --git a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java index ce3e799c5312..145ce6c83556 100644 --- a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java +++ b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java @@ -232,7 +232,7 @@ public void testJsonConversion() throws Exception { assertThat(metadata.partitionStatisticsFiles()).isEqualTo(partitionStatisticsFiles); assertThat(metadata.refs()).isEqualTo(refs); assertThat(metadata.rowLineageEnabled()).isEqualTo(expected.rowLineageEnabled()); - assertThat(metadata.lastRowId()).isEqualTo(expected.lastRowId()); + assertThat(metadata.nextRowId()).isEqualTo(expected.nextRowId()); } @Test @@ -342,7 +342,7 @@ public void testBackwardCompat() throws Exception { assertThat(metadata.previousFiles()).isEqualTo(expected.previousFiles()); assertThat(metadata.snapshot(previousSnapshotId).schemaId()).isNull(); assertThat(metadata.rowLineageEnabled()).isEqualTo(expected.rowLineageEnabled()); - assertThat(metadata.lastRowId()).isEqualTo(expected.lastRowId()); + assertThat(metadata.nextRowId()).isEqualTo(expected.nextRowId()); } @Test From a929c32afa2558c9b8cac2658d33bf2090f2bc88 Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Tue, 21 Jan 2025 09:05:40 -0600 Subject: [PATCH 14/22] Spotless --- .../org/apache/iceberg/SnapshotProducer.java | 28 +++++++++---------- .../org/apache/iceberg/TableMetadata.java | 8 +++--- .../iceberg/TestMetadataUpdateParser.java | 4 +-- 3 files changed, 20 insertions(+), 20 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java index 0abbd27ae78e..6703a04dc69f 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java @@ -305,20 +305,20 @@ public Snapshot apply() { private Long calculateAddedRows(List manifests) { return manifests.stream() - .filter( - manifest -> - manifest.snapshotId() == null - || Objects.equals(manifest.snapshotId(), this.snapshotId)) - .mapToLong( - manifest -> { - Preconditions.checkArgument( - manifest.addedRowsCount() != null, - "Cannot determine number of added rows in snapshot because" - + " the entry for manifest %s is missing the field `added-rows-count`", - manifest.path()); - return manifest.addedRowsCount(); - }) - .sum(); + .filter( + manifest -> + manifest.snapshotId() == null + || Objects.equals(manifest.snapshotId(), this.snapshotId)) + .mapToLong( + manifest -> { + Preconditions.checkArgument( + manifest.addedRowsCount() != null, + "Cannot determine number of added rows in snapshot because" + + " the entry for manifest %s is missing the field `added-rows-count`", + manifest.path()); + return manifest.addedRowsCount(); + }) + .sum(); } protected abstract Map summary(); diff --git a/core/src/main/java/org/apache/iceberg/TableMetadata.java b/core/src/main/java/org/apache/iceberg/TableMetadata.java index 56caaf54861e..8bd689ec42ef 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadata.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadata.java @@ -1270,13 +1270,13 @@ public Builder addSnapshot(Snapshot snapshot) { snapshot.firstRowId() >= nextRowId, "Cannot add a snapshot whose first-row-id (%s) is less than the metadata `last-used-id` (%s) because this will end up generating duplicate row_ids.", snapshot.firstRowId(), - nextRowId); + nextRowId); ValidationException.check( snapshot.addedRows() != null, "Cannot add a snapshot with a null `added-rows` field when row lineage is enabled"); Preconditions.checkArgument( - snapshot.addedRows() >= 0, - "Cannot decrease last-row-id, last-row-id must increase monotonically. Snapshot reports %s added rows"); + snapshot.addedRows() >= 0, + "Cannot decrease last-row-id, last-row-id must increase monotonically. Snapshot reports %s added rows"); this.nextRowId += snapshot.addedRows(); } @@ -1616,7 +1616,7 @@ public TableMetadata build() { .collect(Collectors.toList()), discardChanges ? ImmutableList.of() : ImmutableList.copyOf(changes), rowLineage, - nextRowId); + nextRowId); } private int addSchemaInternal(Schema schema, int newLastColumnId) { diff --git a/core/src/test/java/org/apache/iceberg/TestMetadataUpdateParser.java b/core/src/test/java/org/apache/iceberg/TestMetadataUpdateParser.java index 27bf5f26eda2..fd20b15e2d3e 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetadataUpdateParser.java +++ b/core/src/test/java/org/apache/iceberg/TestMetadataUpdateParser.java @@ -938,8 +938,8 @@ public void testEnableRowLineage() { MetadataUpdate expected = new MetadataUpdate.EnableRowLineage(); assertEquals(action, expected, MetadataUpdateParser.fromJson(json)); assertThat(MetadataUpdateParser.toJson(expected)) - .as("Enable row lineage should convert to the correct JSON value") - .isEqualTo(json); + .as("Enable row lineage should convert to the correct JSON value") + .isEqualTo(json); } public void assertEquals( From b51efa81c47e8e32be6b4c8f92322c477e4765fc Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Tue, 21 Jan 2025 09:30:09 -0600 Subject: [PATCH 15/22] Use Test Template --- .../iceberg/TestRowLineageMetadata.java | 45 ++++++++++--------- 1 file changed, 23 insertions(+), 22 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java b/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java index 86ce02112bdf..78b75d79285c 100644 --- a/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java +++ b/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java @@ -23,17 +23,27 @@ import static org.assertj.core.api.Assumptions.assumeThat; import java.io.File; +import java.util.List; import java.util.concurrent.atomic.AtomicInteger; import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.relocated.com.google.common.primitives.Ints; import org.apache.iceberg.types.Types; import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.api.io.TempDir; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.FieldSource; +@ExtendWith(ParameterizedTestExtension.class) public class TestRowLineageMetadata { - private static final String TEST_LOCATION = "s3://bucket/test/location"; + + @Parameters(name = "formatVersion = {0}") + private static List formatVersion() { + return Ints.asList(TestHelpers.ALL_VERSIONS); + } + + + private static final String TEST_LOCATION = "s3://bucket/test/location"; private static final Schema TEST_SCHEMA = new Schema( @@ -59,8 +69,7 @@ public void cleanup() { TestTables.clearTables(); } - @ParameterizedTest - @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + @TestTemplate public void testRowLineageSupported(int formatVersion) { if (formatVersion >= TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE) { assertThat(TableMetadata.buildFromEmpty(formatVersion).enableRowLineage()).isNotNull(); @@ -71,8 +80,7 @@ public void testRowLineageSupported(int formatVersion) { } } - @ParameterizedTest - @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + @TestTemplate public void testSnapshotAddition(int formatVersion) { assumeThat(formatVersion).isGreaterThanOrEqualTo(TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); @@ -98,8 +106,7 @@ public void testSnapshotAddition(int formatVersion) { assertThat(secondAddition.nextRowId()).isEqualTo(newRows * 2); } - @ParameterizedTest - @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + @TestTemplate public void testInvalidSnapshotAddition(int formatVersion) { assumeThat(formatVersion).isGreaterThanOrEqualTo(TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); @@ -122,11 +129,10 @@ public void testInvalidSnapshotAddition(int formatVersion) { assertThatThrownBy(() -> TableMetadata.buildFrom(base).addSnapshot(invalidNewRows)) .isInstanceOf(ValidationException.class) .hasMessageContaining( - "Cannot add a snapshot with a null `addedRows` field when row lineage is enabled"); + "Cannot add a snapshot with a null `added-rows` field when row lineage is enabled"); } - @ParameterizedTest - @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + @TestTemplate public void testFastAppend(int formatVersion) { assumeThat(formatVersion).isGreaterThanOrEqualTo(TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); @@ -152,8 +158,7 @@ public void testFastAppend(int formatVersion) { assertThat(table.ops().current().nextRowId()).isEqualTo(30 + 17 + 11); } - @ParameterizedTest - @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + @TestTemplate public void testAppend(int formatVersion) { assumeThat(formatVersion).isGreaterThanOrEqualTo(TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); @@ -179,8 +184,7 @@ public void testAppend(int formatVersion) { assertThat(table.ops().current().nextRowId()).isEqualTo(30 + 17 + 11); } - @ParameterizedTest - @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + @TestTemplate public void testAppendBranch(int formatVersion) { assumeThat(formatVersion).isGreaterThanOrEqualTo(TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); // Appends to a branch should still change last-row-id even if not on main, these changes @@ -219,8 +223,7 @@ public void testAppendBranch(int formatVersion) { assertThat(table.ops().current().nextRowId()).isEqualTo(30 + 17 + 11 + 21); } - @ParameterizedTest - @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + @TestTemplate public void testDeletes(int formatVersion) { assumeThat(formatVersion).isGreaterThanOrEqualTo(TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); @@ -252,8 +255,7 @@ public void testDeletes(int formatVersion) { assertThat(table.ops().current().nextRowId()).isEqualTo(30); } - @ParameterizedTest - @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + @TestTemplate public void testReplace(int formatVersion) { assumeThat(formatVersion).isGreaterThanOrEqualTo(TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); @@ -285,8 +287,7 @@ public void testReplace(int formatVersion) { assertThat(table.ops().current().nextRowId()).isEqualTo(120); } - @ParameterizedTest - @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + @TestTemplate public void testEnableRowLineageViaProperty(int formatVersion) { assumeThat(formatVersion).isGreaterThanOrEqualTo(TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); From c5674213122dfb080e57dc3971ce799204d35bf9 Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Tue, 21 Jan 2025 09:36:01 -0600 Subject: [PATCH 16/22] Spotless --- .../test/java/org/apache/iceberg/TestRowLineageMetadata.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java b/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java index 78b75d79285c..d97ef650cff5 100644 --- a/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java +++ b/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java @@ -36,14 +36,12 @@ @ExtendWith(ParameterizedTestExtension.class) public class TestRowLineageMetadata { - @Parameters(name = "formatVersion = {0}") private static List formatVersion() { return Ints.asList(TestHelpers.ALL_VERSIONS); } - - private static final String TEST_LOCATION = "s3://bucket/test/location"; + private static final String TEST_LOCATION = "s3://bucket/test/location"; private static final Schema TEST_SCHEMA = new Schema( From d2e016f09b265ea4aed56d02c651eb0b861b7098 Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Wed, 22 Jan 2025 17:20:43 -0600 Subject: [PATCH 17/22] Switch Test Parameters to Class Parameter --- .../iceberg/TestRowLineageMetadata.java | 27 ++++++++++--------- 1 file changed, 15 insertions(+), 12 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java b/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java index d97ef650cff5..61e4a29f8681 100644 --- a/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java +++ b/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java @@ -41,6 +41,9 @@ private static List formatVersion() { return Ints.asList(TestHelpers.ALL_VERSIONS); } + @Parameter + private int formatVersion; + private static final String TEST_LOCATION = "s3://bucket/test/location"; private static final Schema TEST_SCHEMA = @@ -50,7 +53,7 @@ private static List formatVersion() { Types.NestedField.required(2, "y", Types.LongType.get(), "comment"), Types.NestedField.required(3, "z", Types.LongType.get())); - private TableMetadata baseMetadata(int formatVersion) { + private TableMetadata baseMetadata() { return TableMetadata.buildFromEmpty(formatVersion) .enableRowLineage() .addSchema(TEST_SCHEMA) @@ -68,7 +71,7 @@ public void cleanup() { } @TestTemplate - public void testRowLineageSupported(int formatVersion) { + public void testRowLineageSupported() { if (formatVersion >= TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE) { assertThat(TableMetadata.buildFromEmpty(formatVersion).enableRowLineage()).isNotNull(); } else { @@ -79,12 +82,12 @@ public void testRowLineageSupported(int formatVersion) { } @TestTemplate - public void testSnapshotAddition(int formatVersion) { + public void testSnapshotAddition() { assumeThat(formatVersion).isGreaterThanOrEqualTo(TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); long newRows = 30L; - TableMetadata base = baseMetadata(formatVersion); + TableMetadata base = baseMetadata(); Snapshot addRows = new BaseSnapshot( @@ -105,12 +108,12 @@ public void testSnapshotAddition(int formatVersion) { } @TestTemplate - public void testInvalidSnapshotAddition(int formatVersion) { + public void testInvalidSnapshotAddition() { assumeThat(formatVersion).isGreaterThanOrEqualTo(TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); Long newRows = 30L; - TableMetadata base = baseMetadata(formatVersion); + TableMetadata base = baseMetadata(); Snapshot invalidLastRow = new BaseSnapshot( @@ -131,7 +134,7 @@ public void testInvalidSnapshotAddition(int formatVersion) { } @TestTemplate - public void testFastAppend(int formatVersion) { + public void testFastAppend() { assumeThat(formatVersion).isGreaterThanOrEqualTo(TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); TestTables.TestTable table = @@ -157,7 +160,7 @@ public void testFastAppend(int formatVersion) { } @TestTemplate - public void testAppend(int formatVersion) { + public void testAppend() { assumeThat(formatVersion).isGreaterThanOrEqualTo(TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); TestTables.TestTable table = @@ -183,7 +186,7 @@ public void testAppend(int formatVersion) { } @TestTemplate - public void testAppendBranch(int formatVersion) { + public void testAppendBranch() { assumeThat(formatVersion).isGreaterThanOrEqualTo(TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); // Appends to a branch should still change last-row-id even if not on main, these changes // should also affect commits to main @@ -222,7 +225,7 @@ public void testAppendBranch(int formatVersion) { } @TestTemplate - public void testDeletes(int formatVersion) { + public void testDeletes() { assumeThat(formatVersion).isGreaterThanOrEqualTo(TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); TestTables.TestTable table = @@ -254,7 +257,7 @@ public void testDeletes(int formatVersion) { } @TestTemplate - public void testReplace(int formatVersion) { + public void testReplace() { assumeThat(formatVersion).isGreaterThanOrEqualTo(TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); TestTables.TestTable table = @@ -286,7 +289,7 @@ public void testReplace(int formatVersion) { } @TestTemplate - public void testEnableRowLineageViaProperty(int formatVersion) { + public void testEnableRowLineageViaProperty() { assumeThat(formatVersion).isGreaterThanOrEqualTo(TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); TestTables.TestTable table = From 9b574ff025c9d60c858743c0db93f9a3f9ca6e31 Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Thu, 23 Jan 2025 09:45:00 -0600 Subject: [PATCH 18/22] Missing Line in last Commit --- .../test/java/org/apache/iceberg/TestRowLineageMetadata.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java b/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java index 61e4a29f8681..b7fcc7647f0e 100644 --- a/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java +++ b/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java @@ -41,8 +41,7 @@ private static List formatVersion() { return Ints.asList(TestHelpers.ALL_VERSIONS); } - @Parameter - private int formatVersion; + @Parameter private int formatVersion; private static final String TEST_LOCATION = "s3://bucket/test/location"; From 8d143c98c7421f5a2d81d1b33a4a16514f475d92 Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Thu, 23 Jan 2025 10:10:21 -0600 Subject: [PATCH 19/22] Reformat error messages --- core/src/main/java/org/apache/iceberg/TableMetadata.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/TableMetadata.java b/core/src/main/java/org/apache/iceberg/TableMetadata.java index 8bd689ec42ef..463febe224c4 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadata.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadata.java @@ -1268,15 +1268,15 @@ public Builder addSnapshot(Snapshot snapshot) { if (rowLineage) { ValidationException.check( snapshot.firstRowId() >= nextRowId, - "Cannot add a snapshot whose first-row-id (%s) is less than the metadata `last-used-id` (%s) because this will end up generating duplicate row_ids.", + "Cannot add a snapshot whose 'first-row-id' (%s) is less than the metadata 'next-row-id' (%s) because this will end up generating duplicate row_ids.", snapshot.firstRowId(), nextRowId); ValidationException.check( snapshot.addedRows() != null, - "Cannot add a snapshot with a null `added-rows` field when row lineage is enabled"); + "Cannot add a snapshot with a null 'added-rows' field when row lineage is enabled"); Preconditions.checkArgument( snapshot.addedRows() >= 0, - "Cannot decrease last-row-id, last-row-id must increase monotonically. Snapshot reports %s added rows"); + "Cannot decrease 'last-row-id'. 'last-row-id' must increase monotonically. Snapshot reports %s added rows"); this.nextRowId += snapshot.addedRows(); } From e789fda196c2afd09c9516b4e1eaa976b20571b0 Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Thu, 23 Jan 2025 10:50:56 -0600 Subject: [PATCH 20/22] Update Test Error Message Checking --- .../test/java/org/apache/iceberg/TestRowLineageMetadata.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java b/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java index b7fcc7647f0e..2a5dccf5e93c 100644 --- a/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java +++ b/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java @@ -120,7 +120,7 @@ public void testInvalidSnapshotAddition() { assertThatThrownBy(() -> TableMetadata.buildFrom(base).addSnapshot(invalidLastRow)) .isInstanceOf(ValidationException.class) - .hasMessageContaining("Cannot add a snapshot whose first-row-id"); + .hasMessageContaining("Cannot add a snapshot whose 'first-row-id'"); Snapshot invalidNewRows = new BaseSnapshot( @@ -129,7 +129,7 @@ public void testInvalidSnapshotAddition() { assertThatThrownBy(() -> TableMetadata.buildFrom(base).addSnapshot(invalidNewRows)) .isInstanceOf(ValidationException.class) .hasMessageContaining( - "Cannot add a snapshot with a null `added-rows` field when row lineage is enabled"); + "Cannot add a snapshot with a null 'added-rows' field when row lineage is enabled"); } @TestTemplate From 6fac1a469d48397c0c350fe09767cc619655395e Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Fri, 24 Jan 2025 11:19:52 -0600 Subject: [PATCH 21/22] Remove overly paranoid asserts --- .../org/apache/iceberg/TestRowLineageMetadata.java | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java b/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java index 2a5dccf5e93c..0fb9ee880f6b 100644 --- a/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java +++ b/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java @@ -147,13 +147,11 @@ public void testFastAppend() { table.newFastAppend().appendFile(fileWithRows(30)).commit(); - assertThat(table.ops().current().rowLineageEnabled()).isTrue(); assertThat(table.currentSnapshot().firstRowId()).isEqualTo(0); assertThat(table.ops().current().nextRowId()).isEqualTo(30); table.newFastAppend().appendFile(fileWithRows(17)).appendFile(fileWithRows(11)).commit(); - assertThat(table.ops().current().rowLineageEnabled()).isTrue(); assertThat(table.currentSnapshot().firstRowId()).isEqualTo(30); assertThat(table.ops().current().nextRowId()).isEqualTo(30 + 17 + 11); } @@ -173,13 +171,11 @@ public void testAppend() { table.newAppend().appendFile(fileWithRows(30)).commit(); - assertThat(table.ops().current().rowLineageEnabled()).isTrue(); assertThat(table.currentSnapshot().firstRowId()).isEqualTo(0); assertThat(table.ops().current().nextRowId()).isEqualTo(30); table.newAppend().appendFile(fileWithRows(17)).appendFile(fileWithRows(11)).commit(); - assertThat(table.ops().current().rowLineageEnabled()).isTrue(); assertThat(table.currentSnapshot().firstRowId()).isEqualTo(30); assertThat(table.ops().current().nextRowId()).isEqualTo(30 + 17 + 11); } @@ -205,7 +201,6 @@ public void testAppendBranch() { // Write to Branch table.newAppend().appendFile(fileWithRows(30)).toBranch(branch).commit(); - assertThat(table.ops().current().rowLineageEnabled()).isTrue(); assertThat(table.currentSnapshot()).isNull(); assertThat(table.snapshot(branch).firstRowId()).isEqualTo(0L); assertThat(table.ops().current().nextRowId()).isEqualTo(30); @@ -213,7 +208,6 @@ public void testAppendBranch() { // Write to Main table.newAppend().appendFile(fileWithRows(17)).appendFile(fileWithRows(11)).commit(); - assertThat(table.ops().current().rowLineageEnabled()).isTrue(); assertThat(table.currentSnapshot().firstRowId()).isEqualTo(30); assertThat(table.ops().current().nextRowId()).isEqualTo(30 + 17 + 11); @@ -240,7 +234,6 @@ public void testDeletes() { table.newAppend().appendFile(file).commit(); - assertThat(table.ops().current().rowLineageEnabled()).isTrue(); assertThat(table.currentSnapshot().firstRowId()).isEqualTo(0); assertThat(table.ops().current().nextRowId()).isEqualTo(30); @@ -249,7 +242,6 @@ public void testDeletes() { // Deleting a file should create a new snapshot which should inherit last-row-id from the // previous metadata and not // change last-row-id for this metadata. - assertThat(table.ops().current().rowLineageEnabled()).isTrue(); assertThat(table.currentSnapshot().firstRowId()).isEqualTo(30); assertThat(table.currentSnapshot().addedRows()).isEqualTo(0); assertThat(table.ops().current().nextRowId()).isEqualTo(30); @@ -275,14 +267,12 @@ public void testReplace() { table.newAppend().appendFile(filePart1).appendFile(filePart2).commit(); - assertThat(table.ops().current().rowLineageEnabled()).isTrue(); assertThat(table.currentSnapshot().firstRowId()).isEqualTo(0); assertThat(table.ops().current().nextRowId()).isEqualTo(60); table.newRewrite().deleteFile(filePart1).deleteFile(filePart2).addFile(fileCompacted).commit(); // Rewrites are currently just treated as appends. In the future we could treat these as no-ops - assertThat(table.ops().current().rowLineageEnabled()).isTrue(); assertThat(table.currentSnapshot().firstRowId()).isEqualTo(60); assertThat(table.ops().current().nextRowId()).isEqualTo(120); } From 61f18c25f9d64cc0eccac091993fa22399b41df7 Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Fri, 24 Jan 2025 11:24:45 -0600 Subject: [PATCH 22/22] Reorder asserts in getBoolOrNull Test --- .../test/java/org/apache/iceberg/util/TestJsonUtil.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/util/TestJsonUtil.java b/core/src/test/java/org/apache/iceberg/util/TestJsonUtil.java index 4bac5677a856..1c76865e5d4a 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestJsonUtil.java +++ b/core/src/test/java/org/apache/iceberg/util/TestJsonUtil.java @@ -197,10 +197,6 @@ public void getBool() throws JsonProcessingException { @Test public void getBoolOrNull() throws JsonProcessingException { - assertThat(JsonUtil.getBoolOrNull("x", JsonUtil.mapper().readTree("{}"))).isNull(); - - assertThat(JsonUtil.getBoolOrNull("x", JsonUtil.mapper().readTree("{\"x\": null}"))).isNull(); - assertThatThrownBy( () -> JsonUtil.getBoolOrNull("x", JsonUtil.mapper().readTree("{\"x\": \"23\"}"))) .isInstanceOf(IllegalArgumentException.class) @@ -211,6 +207,10 @@ public void getBoolOrNull() throws JsonProcessingException { .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse to a boolean value: x: \"true\""); + assertThat(JsonUtil.getBoolOrNull("x", JsonUtil.mapper().readTree("{}"))).isNull(); + + assertThat(JsonUtil.getBoolOrNull("x", JsonUtil.mapper().readTree("{\"x\": null}"))).isNull(); + assertThat(JsonUtil.getBoolOrNull("x", JsonUtil.mapper().readTree("{\"x\": true}"))).isTrue(); assertThat(JsonUtil.getBoolOrNull("x", JsonUtil.mapper().readTree("{\"x\": false}"))).isFalse(); }