diff --git a/core/src/main/java/org/apache/iceberg/TableMetadata.java b/core/src/main/java/org/apache/iceberg/TableMetadata.java index 4ba3bdf8d737..fd623ea4436c 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadata.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadata.java @@ -56,6 +56,8 @@ 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 BRANCHING_MIN_SUPPORT_VERSION = 2; + static final int UUID_REQUIRED_MIN_VERSION = 2; private static final long ONE_MINUTE = TimeUnit.MINUTES.toMillis(1); diff --git a/core/src/main/java/org/apache/iceberg/TableMetadataParser.java b/core/src/main/java/org/apache/iceberg/TableMetadataParser.java index d7f2b29be75a..e1b99bd3d4e2 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadataParser.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadataParser.java @@ -352,6 +352,7 @@ public static TableMetadata fromJson(String metadataLocation, JsonNode node) { ImmutableList.Builder builder = ImmutableList.builder(); for (JsonNode schemaNode : schemaArray) { Schema current = SchemaParser.fromJson(schemaNode); + Schema.checkCompatibility(current, formatVersion); if (current.schemaId() == currentSchemaId) { schema = current; } @@ -372,6 +373,7 @@ public static TableMetadata fromJson(String metadataLocation, JsonNode node) { formatVersion == 1, "%s must exist in format v%s", SCHEMAS, formatVersion); schema = SchemaParser.fromJson(JsonUtil.get(SCHEMA, node)); + Schema.checkCompatibility(schema, formatVersion); currentSchemaId = schema.schemaId(); schemas = ImmutableList.of(schema); } diff --git a/core/src/test/java/org/apache/iceberg/MetadataTestUtils.java b/core/src/test/java/org/apache/iceberg/MetadataTestUtils.java new file mode 100644 index 000000000000..506eb5e27e65 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/MetadataTestUtils.java @@ -0,0 +1,428 @@ +/* + * 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.apache.iceberg.Files.localInput; +import static org.apache.iceberg.TableMetadata.INITIAL_SEQUENCE_NUMBER; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Path; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.stream.Collectors; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.SerializableSupplier; + +public class MetadataTestUtils { + + private static final long EXAMPLE_SNAPSHOT_ID = 3; + private static final long EXAMPLE_PARENT_ID = 1; + private static final Integer EXAMPLE_SCHEMA_ID = 2; + private static final long EXAMPLE_SEQUENCE_NUMBER = 0; + private static final int EXAMPLE_SPEC_ID = 5; + + 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 PartitionSpec SPEC_5 = + PartitionSpec.builderFor(TEST_SCHEMA).withSpecId(EXAMPLE_SPEC_ID).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(); + + public static final String EXAMPLE_MANIFEST_PATH_1 = "file:/tmp/manifest1.avro"; + public static final String EXAMPLE_MANIFEST_PATH_2 = "file:/tmp/manifest2.avro"; + + private MetadataTestUtils() {} + + public static TableMetadataBuilder buildTestTableMetadata(int formatVersion) { + return new TableMetadataBuilder(formatVersion); + } + + public static TableMetadataBuilder buildTestTableMetadataWithExampleValues(int formatVersion) { + return new TableMetadataBuilder(formatVersion) + .setLocation(TEST_LOCATION) + .setLastSequenceNumber(formatVersion >= 2 ? SEQ_NO : 0L) + .setLastColumnId(TEST_SCHEMA.highestFieldId()) + .setCurrentSchemaId(TEST_SCHEMA.schemaId()) + .setSchemas(ImmutableList.of(TEST_SCHEMA)) + .setDefaultSpecId(SPEC_5.specId()) + .setSpecs(ImmutableList.of(SPEC_5)) + .setLastAssignedPartitionId(SPEC_5.lastAssignedFieldId()) + .setDefaultSortOrderId(SORT_ORDER_3.orderId()) + .setSortOrders(ImmutableList.of(SORT_ORDER_3)) + .setProperties(ImmutableMap.of("property", "value")); + } + + public static class TableMetadataBuilder { + private String metadataLocation; + private int formatVersion; + private String uuid; + private String location; + private long lastSequenceNumber; + private Long lastUpdatedMillis; + private int lastColumnId; + private int currentSchemaId; + private List schemas; + private int defaultSpecId; + private List specs; + private int lastAssignedPartitionId; + private int defaultSortOrderId; + private List sortOrders; + private Map properties; + private long currentSnapshotId; + private List snapshotLog; + private List previousFiles; + private List statisticsFiles; + private List partitionStatisticsFiles; + private List changes; + private SerializableSupplier> snapshotsSupplier; + private List snapshots; + private Map refs; + + private TableMetadataBuilder(int formatVersion) { + this.formatVersion = formatVersion; + this.uuid = UUID.randomUUID().toString(); + this.lastSequenceNumber = INITIAL_SEQUENCE_NUMBER; + this.lastUpdatedMillis = System.currentTimeMillis(); + this.lastColumnId = -1; + this.currentSchemaId = -1; + this.schemas = Lists.newArrayList(); + this.defaultSpecId = -1; + this.specs = Lists.newArrayList(); + this.lastAssignedPartitionId = 999; + this.defaultSortOrderId = -1; + this.sortOrders = Lists.newArrayList(); + this.properties = Maps.newHashMap(); + this.snapshots = Lists.newArrayList(); + this.currentSnapshotId = -1; + this.changes = Lists.newArrayList(); + this.snapshotLog = Lists.newArrayList(); + this.previousFiles = Lists.newArrayList(); + this.refs = Maps.newHashMap(); + this.statisticsFiles = Lists.newArrayList(); + this.partitionStatisticsFiles = Lists.newArrayList(); + } + + public TableMetadataBuilder setMetadataLocation(String metadataFileLocation) { + this.metadataLocation = metadataFileLocation; + return this; + } + + public TableMetadataBuilder setFormatVersion(int formatVersion) { + this.formatVersion = formatVersion; + return this; + } + + public TableMetadataBuilder setUuid(String uuid) { + this.uuid = uuid; + return this; + } + + public TableMetadataBuilder setLocation(String location) { + this.location = location; + return this; + } + + public TableMetadataBuilder setLastSequenceNumber(long lastSequenceNumber) { + this.lastSequenceNumber = lastSequenceNumber; + return this; + } + + public TableMetadataBuilder setLastUpdatedMillis(long lastUpdatedMillis) { + this.lastUpdatedMillis = lastUpdatedMillis; + return this; + } + + public TableMetadataBuilder setLastColumnId(int lastColumnId) { + this.lastColumnId = lastColumnId; + return this; + } + + public TableMetadataBuilder setCurrentSchemaId(int currentSchemaId) { + this.currentSchemaId = currentSchemaId; + return this; + } + + public TableMetadataBuilder setSchemas(List schemas) { + this.schemas = schemas; + return this; + } + + public TableMetadataBuilder setDefaultSpecId(int defaultSpecId) { + this.defaultSpecId = defaultSpecId; + return this; + } + + public TableMetadataBuilder setSpecs(List specs) { + this.specs = specs; + return this; + } + + public TableMetadataBuilder setLastAssignedPartitionId(int lastAssignedPartitionId) { + this.lastAssignedPartitionId = lastAssignedPartitionId; + return this; + } + + public TableMetadataBuilder setDefaultSortOrderId(int defaultSortOrderId) { + this.defaultSortOrderId = defaultSortOrderId; + return this; + } + + public TableMetadataBuilder setSortOrders(List sortOrders) { + this.sortOrders = sortOrders; + return this; + } + + public TableMetadataBuilder setProperties(Map properties) { + this.properties = properties; + return this; + } + + public TableMetadataBuilder setCurrentSnapshotId(long snapshotId) { + this.currentSnapshotId = snapshotId; + return this; + } + + public TableMetadataBuilder setSnapshotsSupplier( + SerializableSupplier> snapshotsSupplier) { + this.snapshotsSupplier = snapshotsSupplier; + return this; + } + + public TableMetadataBuilder setSnapshots(List snapshots) { + this.snapshots = snapshots; + return this; + } + + public TableMetadataBuilder setSnapshotLog(List snapshotLog) { + this.snapshotLog = snapshotLog; + return this; + } + + public TableMetadataBuilder setMetadataHistory( + List metadataHistory) { + this.previousFiles = metadataHistory; + return this; + } + + public TableMetadataBuilder setRefs(Map refs) { + this.refs = refs; + return this; + } + + public TableMetadataBuilder setChanges(List changes) { + this.changes = changes; + return this; + } + + public TableMetadataBuilder setStatisticsFiles(List statisticsFiles) { + this.statisticsFiles = statisticsFiles; + return this; + } + + public TableMetadataBuilder setPartitionStatisticsFiles( + List partitionStatisticsFiles) { + this.partitionStatisticsFiles = partitionStatisticsFiles; + return this; + } + + public TableMetadata build() { + return new TableMetadata( + metadataLocation, + formatVersion, + uuid, + location, + lastSequenceNumber, + lastUpdatedMillis, + lastColumnId, + currentSchemaId, + ImmutableList.copyOf(schemas), + defaultSpecId, + ImmutableList.copyOf(specs), + lastAssignedPartitionId, + defaultSortOrderId, + ImmutableList.copyOf(sortOrders), + ImmutableMap.copyOf(properties), + currentSnapshotId, + ImmutableList.copyOf(snapshots), + snapshotsSupplier, + ImmutableList.copyOf(snapshotLog), + ImmutableList.copyOf(previousFiles), + ImmutableMap.copyOf(refs), + ImmutableList.copyOf(statisticsFiles), + ImmutableList.copyOf(partitionStatisticsFiles), + ImmutableList.copyOf(changes)); + } + } + + public static BaseSnapshotBuilder buildTestSnapshot() { + return new BaseSnapshotBuilder(); + } + + public static BaseSnapshotBuilder buildTestSnapshotWithExampleValues() { + return new BaseSnapshotBuilder() + .setSequenceNumber(EXAMPLE_SEQUENCE_NUMBER) + .setSnapshotId(EXAMPLE_SNAPSHOT_ID) + .setParentId(EXAMPLE_PARENT_ID) + .setOperation(DataOperations.REPLACE) + .setSummary(ImmutableMap.of("files-added", "4", "files-deleted", "100")) + .setSchemaId(EXAMPLE_SCHEMA_ID) + .setTimestampMillis(System.currentTimeMillis()); + } + + public static class BaseSnapshotBuilder { + private long snapshotId; + private Long parentId; + private long sequenceNumber; + private long timestampMillis; + private String manifestListLocation; + private String operation; + private Map summary; + private Integer schemaId; + private String[] v1ManifestLocations; + + private BaseSnapshotBuilder() { + this.snapshotId = -1L; + this.sequenceNumber = -1L; + this.timestampMillis = System.currentTimeMillis(); + this.summary = ImmutableMap.of(); + } + + public BaseSnapshotBuilder setSnapshotId(long snapshotId) { + this.snapshotId = snapshotId; + return this; + } + + public BaseSnapshotBuilder setParentId(Long parentId) { + this.parentId = parentId; + return this; + } + + public BaseSnapshotBuilder setSequenceNumber(long sequenceNumber) { + this.sequenceNumber = sequenceNumber; + return this; + } + + public BaseSnapshotBuilder setTimestampMillis(long timestampMillis) { + this.timestampMillis = timestampMillis; + return this; + } + + public BaseSnapshotBuilder setManifestListLocation(String manifestListLocation) { + this.manifestListLocation = manifestListLocation; + return this; + } + + public BaseSnapshotBuilder setOperation(String operation) { + this.operation = operation; + return this; + } + + public BaseSnapshotBuilder setSummary(Map summary) { + this.summary = summary; + return this; + } + + public BaseSnapshotBuilder setSchemaId(Integer schemaId) { + this.schemaId = schemaId; + return this; + } + + public BaseSnapshotBuilder setV1ManifestLocations(String[] v1ManifestLocations) { + this.v1ManifestLocations = v1ManifestLocations; + return this; + } + + public Snapshot build() { + Preconditions.checkArgument( + manifestListLocation != null || v1ManifestLocations != null, + "Cannot set both ManifestListLocation and V1ManifestLocations"); + if (v1ManifestLocations != null) { + return new BaseSnapshot( + sequenceNumber, + snapshotId, + parentId, + timestampMillis, + operation, + summary, + schemaId, + v1ManifestLocations); + } + return new BaseSnapshot( + sequenceNumber, + snapshotId, + parentId, + timestampMillis, + operation, + summary, + schemaId, + manifestListLocation); + } + + public Snapshot buildWithExampleManifestList(Path temp, List manifestFiles) + throws IOException { + Preconditions.checkArgument( + manifestListLocation == null && v1ManifestLocations == null, + "An example manifest list with manifest files will be created"); + + this.manifestListLocation = + createManifestListWithManifestFiles(snapshotId, parentId, temp, manifestFiles); + return build(); + } + } + + private static String createManifestListWithManifestFiles( + long snapshotId, Long parentSnapshotId, Path temp, List manifestFiles) + throws IOException { + File manifestList = File.createTempFile("manifests", null, temp.toFile()); + manifestList.deleteOnExit(); + + List manifests = + manifestFiles.stream() + .map(name -> new GenericManifestFile(localInput(name), EXAMPLE_SPEC_ID, snapshotId)) + .collect(Collectors.toList()); + + try (ManifestListWriter writer = + ManifestLists.write(1, Files.localOutput(manifestList), snapshotId, parentSnapshotId, 0)) { + writer.addAll(manifests); + } + + return localInput(manifestList).location(); + } +} diff --git a/core/src/test/java/org/apache/iceberg/TestDataTaskParser.java b/core/src/test/java/org/apache/iceberg/TestDataTaskParser.java index 5a3d119046f5..ae1b126524f4 100644 --- a/core/src/test/java/org/apache/iceberg/TestDataTaskParser.java +++ b/core/src/test/java/org/apache/iceberg/TestDataTaskParser.java @@ -193,10 +193,26 @@ private DataTask createDataTask() { List snapshots = Arrays.asList( - new BaseSnapshot( - 1L, 1L, null, 1234567890000L, "append", summary1, 1, "file:/tmp/manifest1.avro"), - new BaseSnapshot( - 2L, 2L, 1L, 9876543210000L, "append", summary2, 1, "file:/tmp/manifest2.avro")); + MetadataTestUtils.buildTestSnapshot() + .setSequenceNumber(1L) + .setSnapshotId(1L) + .setParentId(null) + .setTimestampMillis(1234567890000L) + .setOperation("append") + .setSummary(summary1) + .setSchemaId(1) + .setManifestListLocation("file:/tmp/manifest1.avro") + .build(), + MetadataTestUtils.buildTestSnapshot() + .setSequenceNumber(2L) + .setSnapshotId(2L) + .setParentId(1L) + .setTimestampMillis(9876543210000L) + .setOperation("append") + .setSummary(summary2) + .setSchemaId(1) + .setManifestListLocation("file:/tmp/manifest2.avro") + .build()); 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..57de6ce2a47a 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetadataUpdateParser.java +++ b/core/src/test/java/org/apache/iceberg/TestMetadataUpdateParser.java @@ -18,12 +18,10 @@ */ package org.apache.iceberg; -import static org.apache.iceberg.Files.localInput; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.assertj.core.api.Assertions.fail; -import java.io.File; import java.io.IOException; import java.nio.file.Path; import java.util.List; @@ -357,22 +355,14 @@ public void testSetDefaultSortOrderFromJson() { @Test public void testAddSnapshotToJson() throws IOException { String action = MetadataUpdateParser.ADD_SNAPSHOT; - long parentId = 1; - long snapshotId = 2; - int schemaId = 3; - - String manifestList = createManifestListWithManifestFiles(snapshotId, parentId); Snapshot snapshot = - new BaseSnapshot( - 0, - snapshotId, - parentId, - System.currentTimeMillis(), - DataOperations.REPLACE, - ImmutableMap.of("files-added", "4", "files-deleted", "100"), - schemaId, - manifestList); + MetadataTestUtils.buildTestSnapshotWithExampleValues() + .buildWithExampleManifestList( + temp, + ImmutableList.of( + MetadataTestUtils.EXAMPLE_MANIFEST_PATH_1, + MetadataTestUtils.EXAMPLE_MANIFEST_PATH_2)); String snapshotJson = SnapshotParser.toJson(snapshot, /* pretty */ false); String expected = String.format("{\"action\":\"%s\",\"snapshot\":%s}", action, snapshotJson); MetadataUpdate update = new MetadataUpdate.AddSnapshot(snapshot); @@ -385,22 +375,13 @@ public void testAddSnapshotToJson() throws IOException { @Test public void testAddSnapshotFromJson() throws IOException { String action = MetadataUpdateParser.ADD_SNAPSHOT; - long parentId = 1; - long snapshotId = 2; - int schemaId = 3; - Map summary = ImmutableMap.of("files-added", "4", "files-deleted", "100"); - - String manifestList = createManifestListWithManifestFiles(snapshotId, parentId); Snapshot snapshot = - new BaseSnapshot( - 0, - snapshotId, - parentId, - System.currentTimeMillis(), - DataOperations.REPLACE, - summary, - schemaId, - manifestList); + MetadataTestUtils.buildTestSnapshotWithExampleValues() + .buildWithExampleManifestList( + temp, + ImmutableList.of( + MetadataTestUtils.EXAMPLE_MANIFEST_PATH_1, + MetadataTestUtils.EXAMPLE_MANIFEST_PATH_2)); String snapshotJson = SnapshotParser.toJson(snapshot, /* pretty */ false); String json = String.format("{\"action\":\"%s\",\"snapshot\":%s}", action, snapshotJson); MetadataUpdate expected = new MetadataUpdate.AddSnapshot(snapshot); @@ -1257,22 +1238,4 @@ private static void assertEqualsRemovePartitionSpecs( MetadataUpdate.RemovePartitionSpecs expected, MetadataUpdate.RemovePartitionSpecs actual) { assertThat(actual.specIds()).containsExactlyInAnyOrderElementsOf(expected.specIds()); } - - private String createManifestListWithManifestFiles(long snapshotId, Long parentSnapshotId) - throws IOException { - File manifestList = File.createTempFile("manifests", null, temp.toFile()); - manifestList.deleteOnExit(); - - List manifests = - ImmutableList.of( - new GenericManifestFile(localInput("file:/tmp/manifest1.avro"), 0, snapshotId), - new GenericManifestFile(localInput("file:/tmp/manifest2.avro"), 0, snapshotId)); - - try (ManifestListWriter writer = - ManifestLists.write(1, Files.localOutput(manifestList), snapshotId, parentSnapshotId, 0)) { - writer.addAll(manifests); - } - - return localInput(manifestList).location(); - } } diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java b/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java index e4c2ba5ec2df..3daffa5378c0 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java @@ -18,13 +18,10 @@ */ package org.apache.iceberg; -import static org.apache.iceberg.Files.localInput; import static org.assertj.core.api.Assertions.assertThat; -import java.io.File; import java.io.IOException; import java.nio.file.Path; -import java.util.List; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.junit.jupiter.api.Test; @@ -37,13 +34,16 @@ public class TestSnapshotJson { @Test public void testJsonConversion() throws IOException { - int snapshotId = 23; - Long parentId = null; - String manifestList = createManifestListWithManifestFiles(snapshotId, parentId); - Snapshot expected = - new BaseSnapshot( - 0, snapshotId, parentId, System.currentTimeMillis(), null, null, 1, manifestList); + MetadataTestUtils.buildTestSnapshotWithExampleValues() + .setOperation(null) + .setSummary(null) + .setSchemaId(1) + .buildWithExampleManifestList( + temp, + ImmutableList.of( + MetadataTestUtils.EXAMPLE_MANIFEST_PATH_1, + MetadataTestUtils.EXAMPLE_MANIFEST_PATH_2)); String json = SnapshotParser.toJson(expected); Snapshot snapshot = SnapshotParser.fromJson(json); @@ -56,13 +56,16 @@ public void testJsonConversion() throws IOException { @Test public void testJsonConversionWithoutSchemaId() throws IOException { - int snapshotId = 23; - Long parentId = null; - String manifestList = createManifestListWithManifestFiles(snapshotId, parentId); - Snapshot expected = - new BaseSnapshot( - 0, snapshotId, parentId, System.currentTimeMillis(), null, null, null, manifestList); + MetadataTestUtils.buildTestSnapshotWithExampleValues() + .setOperation(null) + .setSummary(null) + .setSchemaId(null) + .buildWithExampleManifestList( + temp, + ImmutableList.of( + MetadataTestUtils.EXAMPLE_MANIFEST_PATH_1, + MetadataTestUtils.EXAMPLE_MANIFEST_PATH_2)); String json = SnapshotParser.toJson(expected); Snapshot snapshot = SnapshotParser.fromJson(json); @@ -75,21 +78,15 @@ public void testJsonConversionWithoutSchemaId() throws IOException { @Test public void testJsonConversionWithOperation() throws IOException { - long parentId = 1; - long id = 2; - - String manifestList = createManifestListWithManifestFiles(id, parentId); - Snapshot expected = - new BaseSnapshot( - 0, - id, - parentId, - System.currentTimeMillis(), - DataOperations.REPLACE, - ImmutableMap.of("files-added", "4", "files-deleted", "100"), - 3, - manifestList); + MetadataTestUtils.buildTestSnapshotWithExampleValues() + .setSequenceNumber(0L) + .setOperation(DataOperations.REPLACE) + .buildWithExampleManifestList( + temp, + ImmutableList.of( + MetadataTestUtils.EXAMPLE_MANIFEST_PATH_1, + MetadataTestUtils.EXAMPLE_MANIFEST_PATH_2)); String json = SnapshotParser.toJson(expected); Snapshot snapshot = SnapshotParser.fromJson(json); @@ -115,15 +112,16 @@ public void testJsonConversionWithV1Manifests() { // this creates a V1 snapshot with manifests long timestampMillis = System.currentTimeMillis(); Snapshot expected = - new BaseSnapshot( - 0, - id, - parentId, - timestampMillis, - DataOperations.REPLACE, - ImmutableMap.of("files-added", "4", "files-deleted", "100"), - 3, - new String[] {"/tmp/manifest1.avro", "/tmp/manifest2.avro"}); + MetadataTestUtils.buildTestSnapshot() + .setSequenceNumber(0) + .setSnapshotId(id) + .setParentId(parentId) + .setTimestampMillis(timestampMillis) + .setOperation(DataOperations.REPLACE) + .setSummary(ImmutableMap.of("files-added", "4", "files-deleted", "100")) + .setSchemaId(3) + .setV1ManifestLocations(new String[] {"/tmp/manifest1.avro", "/tmp/manifest2.avro"}) + .build(); String expectedJson = String.format( @@ -159,24 +157,6 @@ public void testJsonConversionWithV1Manifests() { assertThat(snapshot.schemaId()).isEqualTo(expected.schemaId()); } - private String createManifestListWithManifestFiles(long snapshotId, Long parentSnapshotId) - throws IOException { - File manifestList = File.createTempFile("manifests", null, temp.toFile()); - manifestList.deleteOnExit(); - - List manifests = - ImmutableList.of( - new GenericManifestFile(localInput("file:/tmp/manifest1.avro"), 0, snapshotId), - new GenericManifestFile(localInput("file:/tmp/manifest2.avro"), 0, snapshotId)); - - try (ManifestListWriter writer = - ManifestLists.write(1, Files.localOutput(manifestList), snapshotId, parentSnapshotId, 0)) { - writer.addAll(manifests); - } - - return localInput(manifestList).location(); - } - @Test public void testJsonConversionSummaryWithoutOperation() { // This behavior is out of spec, but we don't want to fail on it. diff --git a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java index 45aa211e5187..40ebc6e3ae11 100644 --- a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java +++ b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java @@ -18,7 +18,8 @@ */ package org.apache.iceberg; -import static org.apache.iceberg.Files.localInput; +import static org.apache.iceberg.TableMetadata.BRANCHING_MIN_SUPPORT_VERSION; +import static org.apache.iceberg.TableMetadata.UUID_REQUIRED_MIN_VERSION; import static org.apache.iceberg.TableMetadataParser.CURRENT_SNAPSHOT_ID; import static org.apache.iceberg.TableMetadataParser.FORMAT_VERSION; import static org.apache.iceberg.TableMetadataParser.LAST_COLUMN_ID; @@ -32,10 +33,10 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.assertj.core.api.Assertions.entry; +import static org.assertj.core.api.Assumptions.assumeThat; import static org.junit.jupiter.params.provider.Arguments.arguments; import com.fasterxml.jackson.core.JsonGenerator; -import java.io.File; import java.io.IOException; import java.io.StringWriter; import java.io.UncheckedIOException; @@ -68,6 +69,7 @@ import org.junit.jupiter.api.io.TempDir; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.FieldSource; import org.junit.jupiter.params.provider.MethodSource; public class TestTableMetadata { @@ -80,9 +82,6 @@ public class TestTableMetadata { 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 = @@ -96,31 +95,27 @@ public class TestTableMetadata { public TableOperations ops = new LocalTableOperations(temp); - @Test + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") @SuppressWarnings("MethodLength") - public void testJsonConversion() throws Exception { + public void testJsonConversion(int formatVersion) throws Exception { long previousSnapshotId = System.currentTimeMillis() - new Random(1234).nextInt(3600); - String manifestList = - createManifestListWithManifestFile(previousSnapshotId, null, "file:/tmp/manifest1.avro"); Snapshot previousSnapshot = - new BaseSnapshot( - 0, previousSnapshotId, null, previousSnapshotId, null, null, null, manifestList); + MetadataTestUtils.buildTestSnapshotWithExampleValues() + .setSnapshotId(previousSnapshotId) + .setParentId(null) + .setSchemaId(null) + .buildWithExampleManifestList(temp, ImmutableList.of("file:/tmp/manifest1.avro")); long currentSnapshotId = System.currentTimeMillis(); - manifestList = - createManifestListWithManifestFile( - currentSnapshotId, previousSnapshotId, "file:/tmp/manifest2.avro"); Snapshot currentSnapshot = - new BaseSnapshot( - 0, - currentSnapshotId, - previousSnapshotId, - currentSnapshotId, - null, - null, - 7, - manifestList); + MetadataTestUtils.buildTestSnapshot() + .setSequenceNumber(0L) + .setSnapshotId(currentSnapshotId) + .setParentId(previousSnapshotId) + .setSchemaId(7) + .buildWithExampleManifestList(temp, ImmutableList.of("file:/tmp/manifest2.avro")); List snapshotLog = ImmutableList.builder() @@ -160,31 +155,16 @@ public void testJsonConversion() throws Exception { .build()); TableMetadata expected = - new TableMetadata( - null, - 2, - UUID.randomUUID().toString(), - TEST_LOCATION, - SEQ_NO, - System.currentTimeMillis(), - 3, - 7, - ImmutableList.of(TEST_SCHEMA, schema), - 5, - ImmutableList.of(SPEC_5), - SPEC_5.lastAssignedFieldId(), - 3, - ImmutableList.of(SORT_ORDER_3), - ImmutableMap.of("property", "value"), - currentSnapshotId, - Arrays.asList(previousSnapshot, currentSnapshot), - null, - snapshotLog, - ImmutableList.of(), - refs, - statisticsFiles, - partitionStatisticsFiles, - ImmutableList.of()); + MetadataTestUtils.buildTestTableMetadataWithExampleValues(formatVersion) + .setCurrentSchemaId(7) + .setSchemas(ImmutableList.of(TEST_SCHEMA, schema)) + .setCurrentSnapshotId(currentSnapshotId) + .setSnapshots(Arrays.asList(previousSnapshot, currentSnapshot)) + .setSnapshotLog(snapshotLog) + .setRefs(refs) + .setStatisticsFiles(statisticsFiles) + .setPartitionStatisticsFiles(partitionStatisticsFiles) + .build(); String asJson = TableMetadataParser.toJson(expected); TableMetadata metadata = TableMetadataParser.fromJson(asJson); @@ -227,53 +207,35 @@ public void testBackwardCompat() throws Exception { long previousSnapshotId = System.currentTimeMillis() - new Random(1234).nextInt(3600); - String manifestList = - createManifestListWithManifestFile(previousSnapshotId, null, "file:/tmp/manifest1.avro"); Snapshot previousSnapshot = - new BaseSnapshot( - 0, previousSnapshotId, null, previousSnapshotId, null, null, null, manifestList); + MetadataTestUtils.buildTestSnapshotWithExampleValues() + .setSnapshotId(previousSnapshotId) + .setParentId(null) + .setSchemaId(null) + .buildWithExampleManifestList(temp, ImmutableList.of("file:/tmp/manifest1.avro")); long currentSnapshotId = System.currentTimeMillis(); - manifestList = - createManifestListWithManifestFile( - currentSnapshotId, previousSnapshotId, "file:/tmp/manifest2.avro"); Snapshot currentSnapshot = - new BaseSnapshot( - 0, - currentSnapshotId, - previousSnapshotId, - currentSnapshotId, - null, - null, - null, - manifestList); + MetadataTestUtils.buildTestSnapshot() + .setSequenceNumber(0L) + .setSnapshotId(currentSnapshotId) + .setParentId(previousSnapshotId) + .buildWithExampleManifestList(temp, ImmutableList.of("file:/tmp/manifest1.avro")); TableMetadata expected = - new TableMetadata( - null, - 1, - null, - TEST_LOCATION, - 0, - System.currentTimeMillis(), - 3, - TableMetadata.INITIAL_SCHEMA_ID, - ImmutableList.of(schema), - 6, - ImmutableList.of(spec), - spec.lastAssignedFieldId(), - TableMetadata.INITIAL_SORT_ORDER_ID, - ImmutableList.of(sortOrder), - ImmutableMap.of("property", "value"), - currentSnapshotId, - Arrays.asList(previousSnapshot, currentSnapshot), - null, - ImmutableList.of(), - ImmutableList.of(), - ImmutableMap.of(), - ImmutableList.of(), - ImmutableList.of(), - ImmutableList.of()); + MetadataTestUtils.buildTestTableMetadataWithExampleValues(1) + .setFormatVersion(1) + .setUuid(null) + .setCurrentSchemaId(TableMetadata.INITIAL_SCHEMA_ID) + .setSchemas(ImmutableList.of(schema)) + .setDefaultSpecId(6) + .setSpecs(ImmutableList.of(spec)) + .setLastAssignedPartitionId(spec.lastAssignedFieldId()) + .setDefaultSortOrderId(TableMetadata.INITIAL_SORT_ORDER_ID) + .setSortOrders(ImmutableList.of(sortOrder)) + .setCurrentSnapshotId(currentSnapshotId) + .setSnapshots(Arrays.asList(previousSnapshot, currentSnapshot)) + .build(); String asJson = toJsonWithoutSpecAndSchemaList(expected); TableMetadata metadata = TableMetadataParser.fromJson(asJson); @@ -314,31 +276,25 @@ public void testBackwardCompat() throws Exception { assertThat(metadata.snapshot(previousSnapshotId).schemaId()).isNull(); } - @Test - public void testInvalidMainBranch() throws IOException { + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + public void testInvalidMainBranch(int formatVersion) throws IOException { + assumeThat(formatVersion).isGreaterThanOrEqualTo(BRANCHING_MIN_SUPPORT_VERSION); + long previousSnapshotId = System.currentTimeMillis() - new Random(1234).nextInt(3600); - String manifestList = - createManifestListWithManifestFile(previousSnapshotId, null, "file:/tmp/manifest1.avro"); Snapshot previousSnapshot = - new BaseSnapshot( - 0, previousSnapshotId, null, previousSnapshotId, null, null, null, manifestList); + MetadataTestUtils.buildTestSnapshotWithExampleValues() + .setSnapshotId(previousSnapshotId) + .buildWithExampleManifestList(temp, ImmutableList.of("file:/tmp/manifest1.avro")); long currentSnapshotId = System.currentTimeMillis(); - manifestList = - createManifestListWithManifestFile( - currentSnapshotId, previousSnapshotId, "file:/tmp/manifest2.avro"); Snapshot currentSnapshot = - new BaseSnapshot( - 0, - currentSnapshotId, - previousSnapshotId, - currentSnapshotId, - null, - null, - 7, - manifestList); + MetadataTestUtils.buildTestSnapshot() + .setSnapshotId(currentSnapshotId) + .setParentId(previousSnapshotId) + .buildWithExampleManifestList(temp, ImmutableList.of("file:/tmp/manifest2.avro")); List snapshotLog = ImmutableList.builder() @@ -350,123 +306,62 @@ public void testInvalidMainBranch() throws IOException { currentSnapshot.timestampMillis(), currentSnapshot.snapshotId())) .build(); - Schema schema = new Schema(6, Types.NestedField.required(10, "x", Types.StringType.get())); - Map refs = ImmutableMap.of("main", SnapshotRef.branchBuilder(previousSnapshotId).build()); assertThatThrownBy( () -> - new TableMetadata( - null, - 2, - UUID.randomUUID().toString(), - TEST_LOCATION, - SEQ_NO, - System.currentTimeMillis(), - 3, - 7, - ImmutableList.of(TEST_SCHEMA, schema), - 5, - ImmutableList.of(SPEC_5), - SPEC_5.lastAssignedFieldId(), - 3, - ImmutableList.of(SORT_ORDER_3), - ImmutableMap.of("property", "value"), - currentSnapshotId, - Arrays.asList(previousSnapshot, currentSnapshot), - null, - snapshotLog, - ImmutableList.of(), - refs, - ImmutableList.of(), - ImmutableList.of(), - ImmutableList.of())) + MetadataTestUtils.buildTestTableMetadataWithExampleValues(formatVersion) + .setCurrentSnapshotId(currentSnapshotId) + .setSnapshots(Arrays.asList(previousSnapshot, currentSnapshot)) + .setSnapshotLog(snapshotLog) + .setRefs(refs) + .build()) .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith("Current snapshot ID does not match main branch"); } - @Test - public void testMainWithoutCurrent() throws IOException { + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + public void testMainWithoutCurrent(int formatVersion) throws IOException { + assumeThat(formatVersion).isGreaterThanOrEqualTo(BRANCHING_MIN_SUPPORT_VERSION); + long snapshotId = System.currentTimeMillis() - new Random(1234).nextInt(3600); - String manifestList = - createManifestListWithManifestFile(snapshotId, null, "file:/tmp/manifest1.avro"); Snapshot snapshot = - new BaseSnapshot(0, snapshotId, null, snapshotId, null, null, null, manifestList); - - Schema schema = new Schema(6, Types.NestedField.required(10, "x", Types.StringType.get())); + MetadataTestUtils.buildTestSnapshot() + .setSnapshotId(snapshotId) + .buildWithExampleManifestList(temp, ImmutableList.of("file:/tmp/manifest1.avro")); Map refs = ImmutableMap.of("main", SnapshotRef.branchBuilder(snapshotId).build()); assertThatThrownBy( () -> - new TableMetadata( - null, - 2, - UUID.randomUUID().toString(), - TEST_LOCATION, - SEQ_NO, - System.currentTimeMillis(), - 3, - 7, - ImmutableList.of(TEST_SCHEMA, schema), - 5, - ImmutableList.of(SPEC_5), - SPEC_5.lastAssignedFieldId(), - 3, - ImmutableList.of(SORT_ORDER_3), - ImmutableMap.of("property", "value"), - -1, - ImmutableList.of(snapshot), - null, - ImmutableList.of(), - ImmutableList.of(), - refs, - ImmutableList.of(), - ImmutableList.of(), - ImmutableList.of())) + MetadataTestUtils.buildTestTableMetadataWithExampleValues(formatVersion) + .setSnapshots(ImmutableList.of(snapshot)) + .setCurrentSnapshotId(-1L) + .setSnapshotsSupplier(null) + .setRefs(refs) + .build()) .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith("Current snapshot is not set, but main branch exists"); } - @Test - public void testBranchSnapshotMissing() { - long snapshotId = System.currentTimeMillis() - new Random(1234).nextInt(3600); - - Schema schema = new Schema(6, Types.NestedField.required(10, "x", Types.StringType.get())); + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + public void testBranchSnapshotMissing(int formatVersion) { + assumeThat(formatVersion).isGreaterThanOrEqualTo(2); + long snapshotId = System.currentTimeMillis() - new Random(1234).nextInt(3600); Map refs = ImmutableMap.of("main", SnapshotRef.branchBuilder(snapshotId).build()); assertThatThrownBy( () -> - new TableMetadata( - null, - 2, - UUID.randomUUID().toString(), - TEST_LOCATION, - SEQ_NO, - System.currentTimeMillis(), - 3, - 7, - ImmutableList.of(TEST_SCHEMA, schema), - 5, - ImmutableList.of(SPEC_5), - SPEC_5.lastAssignedFieldId(), - 3, - ImmutableList.of(SORT_ORDER_3), - ImmutableMap.of("property", "value"), - -1, - ImmutableList.of(), - null, - ImmutableList.of(), - ImmutableList.of(), - refs, - ImmutableList.of(), - ImmutableList.of(), - ImmutableList.of())) + MetadataTestUtils.buildTestTableMetadataWithExampleValues(formatVersion) + .setRefs(refs) + .build()) .isInstanceOf(IllegalArgumentException.class) .hasMessageEndingWith("does not exist in the existing snapshots list"); } @@ -513,32 +408,23 @@ private static String toJsonWithoutSpecAndSchemaList(TableMetadata metadata) { return writer.toString(); } - @Test - public void testJsonWithPreviousMetadataLog() throws Exception { + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + public void testJsonWithPreviousMetadataLog(int formatVersion) throws Exception { long previousSnapshotId = System.currentTimeMillis() - new Random(1234).nextInt(3600); - String manifestList = - createManifestListWithManifestFile(previousSnapshotId, null, "file:/tmp/manifest1.avro"); Snapshot previousSnapshot = - new BaseSnapshot( - 0, previousSnapshotId, null, previousSnapshotId, null, null, null, manifestList); + MetadataTestUtils.buildTestSnapshot() + .setSnapshotId(previousSnapshotId) + .buildWithExampleManifestList(temp, ImmutableList.of("file:/tmp/manifest1.avro")); long currentSnapshotId = System.currentTimeMillis(); - manifestList = - createManifestListWithManifestFile( - currentSnapshotId, previousSnapshotId, "file:/tmp/manifest2.avro"); Snapshot currentSnapshot = - new BaseSnapshot( - 0, - currentSnapshotId, - previousSnapshotId, - currentSnapshotId, - null, - null, - null, - manifestList); + MetadataTestUtils.buildTestSnapshot() + .setSnapshotId(currentSnapshotId) + .setParentId(previousSnapshotId) + .buildWithExampleManifestList(temp, ImmutableList.of("file:/tmp/manifest2.avro")); - List reversedSnapshotLog = Lists.newArrayList(); long currentTimestamp = System.currentTimeMillis(); List previousMetadataLog = Lists.newArrayList(); previousMetadataLog.add( @@ -546,31 +432,11 @@ public void testJsonWithPreviousMetadataLog() throws Exception { currentTimestamp, "/tmp/000001-" + UUID.randomUUID() + ".metadata.json")); TableMetadata base = - new TableMetadata( - null, - 1, - UUID.randomUUID().toString(), - TEST_LOCATION, - 0, - System.currentTimeMillis(), - 3, - 7, - ImmutableList.of(TEST_SCHEMA), - 5, - ImmutableList.of(SPEC_5), - SPEC_5.lastAssignedFieldId(), - 3, - ImmutableList.of(SORT_ORDER_3), - ImmutableMap.of("property", "value"), - currentSnapshotId, - Arrays.asList(previousSnapshot, currentSnapshot), - null, - reversedSnapshotLog, - ImmutableList.copyOf(previousMetadataLog), - ImmutableMap.of(), - ImmutableList.of(), - ImmutableList.of(), - ImmutableList.of()); + MetadataTestUtils.buildTestTableMetadataWithExampleValues(formatVersion) + .setCurrentSnapshotId(currentSnapshotId) + .setSnapshots(Arrays.asList(previousSnapshot, currentSnapshot)) + .setMetadataHistory(ImmutableList.copyOf(previousMetadataLog)) + .build(); String asJson = TableMetadataParser.toJson(base); TableMetadata metadataFromJson = TableMetadataParser.fromJson(asJson); @@ -578,30 +444,23 @@ public void testJsonWithPreviousMetadataLog() throws Exception { assertThat(metadataFromJson.previousFiles()).isEqualTo(previousMetadataLog); } - @Test - public void testAddPreviousMetadataRemoveNone() throws IOException { + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + public void testAddPreviousMetadataRemoveNone(int formatVersion) throws IOException { long previousSnapshotId = System.currentTimeMillis() - new Random(1234).nextInt(3600); - String manifestList = - createManifestListWithManifestFile(previousSnapshotId, null, "file:/tmp/manifest1.avro"); Snapshot previousSnapshot = - new BaseSnapshot( - 0, previousSnapshotId, null, previousSnapshotId, null, null, null, manifestList); + MetadataTestUtils.buildTestSnapshot() + .setSnapshotId(previousSnapshotId) + .buildWithExampleManifestList(temp, ImmutableList.of("file:/tmp/manifest1.avro")); + long currentSnapshotId = System.currentTimeMillis(); - manifestList = - createManifestListWithManifestFile( - currentSnapshotId, previousSnapshotId, "file:/tmp/manifest2.avro"); Snapshot currentSnapshot = - new BaseSnapshot( - 0, - currentSnapshotId, - previousSnapshotId, - currentSnapshotId, - null, - null, - null, - manifestList); + MetadataTestUtils.buildTestSnapshot() + .setSnapshotId(currentSnapshotId) + .setParentId(previousSnapshotId) + .buildWithExampleManifestList(temp, ImmutableList.of("file:/tmp/manifest2.avro")); List reversedSnapshotLog = Lists.newArrayList(); reversedSnapshotLog.add( @@ -622,31 +481,14 @@ public void testAddPreviousMetadataRemoveNone() throws IOException { currentTimestamp - 80, "/tmp/000003-" + UUID.randomUUID() + ".metadata.json"); TableMetadata base = - new TableMetadata( - latestPreviousMetadata.file(), - 1, - UUID.randomUUID().toString(), - TEST_LOCATION, - 0, - currentTimestamp - 80, - 3, - 7, - ImmutableList.of(TEST_SCHEMA), - 5, - ImmutableList.of(SPEC_5), - SPEC_5.lastAssignedFieldId(), - 3, - ImmutableList.of(SORT_ORDER_3), - ImmutableMap.of("property", "value"), - currentSnapshotId, - Arrays.asList(previousSnapshot, currentSnapshot), - null, - reversedSnapshotLog, - ImmutableList.copyOf(previousMetadataLog), - ImmutableMap.of(), - ImmutableList.of(), - ImmutableList.of(), - ImmutableList.of()); + MetadataTestUtils.buildTestTableMetadataWithExampleValues(formatVersion) + .setMetadataLocation(latestPreviousMetadata.file()) + .setCurrentSnapshotId(currentSnapshotId) + .setSnapshots(Arrays.asList(previousSnapshot, currentSnapshot)) + .setSnapshotLog(reversedSnapshotLog) + .setLastUpdatedMillis(currentTimestamp - 80) + .setMetadataHistory(ImmutableList.copyOf(previousMetadataLog)) + .build(); previousMetadataLog.add(latestPreviousMetadata); @@ -660,30 +502,22 @@ public void testAddPreviousMetadataRemoveNone() throws IOException { assertThat(removedPreviousMetadata).isEmpty(); } - @Test - public void testAddPreviousMetadataRemoveOne() throws IOException { + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + public void testAddPreviousMetadataRemoveOne(int formatVersion) throws IOException { long previousSnapshotId = System.currentTimeMillis() - new Random(1234).nextInt(3600); - String manifestList = - createManifestListWithManifestFile(previousSnapshotId, null, "file:/tmp/manifest1.avro"); Snapshot previousSnapshot = - new BaseSnapshot( - 0, previousSnapshotId, null, previousSnapshotId, null, null, null, manifestList); + MetadataTestUtils.buildTestSnapshot() + .setSnapshotId(previousSnapshotId) + .buildWithExampleManifestList(temp, ImmutableList.of("file:/tmp/manifest1.avro")); long currentSnapshotId = System.currentTimeMillis(); - manifestList = - createManifestListWithManifestFile( - currentSnapshotId, previousSnapshotId, "file:/tmp/manifest2.avro"); Snapshot currentSnapshot = - new BaseSnapshot( - 0, - currentSnapshotId, - previousSnapshotId, - currentSnapshotId, - null, - null, - null, - manifestList); + MetadataTestUtils.buildTestSnapshot() + .setSnapshotId(currentSnapshotId) + .setParentId(previousSnapshotId) + .buildWithExampleManifestList(temp, ImmutableList.of("file:/tmp/manifest2.avro")); List reversedSnapshotLog = Lists.newArrayList(); reversedSnapshotLog.add( @@ -713,31 +547,15 @@ public void testAddPreviousMetadataRemoveOne() throws IOException { currentTimestamp - 50, "/tmp/000006-" + UUID.randomUUID() + ".metadata.json"); TableMetadata base = - new TableMetadata( - latestPreviousMetadata.file(), - 1, - UUID.randomUUID().toString(), - TEST_LOCATION, - 0, - currentTimestamp - 50, - 3, - 7, - ImmutableList.of(TEST_SCHEMA), - 5, - ImmutableList.of(SPEC_5), - SPEC_5.lastAssignedFieldId(), - 3, - ImmutableList.of(SORT_ORDER_3), - ImmutableMap.of("property", "value"), - currentSnapshotId, - Arrays.asList(previousSnapshot, currentSnapshot), - null, - reversedSnapshotLog, - ImmutableList.copyOf(previousMetadataLog), - ImmutableMap.of(), - ImmutableList.of(), - ImmutableList.of(), - ImmutableList.of()); + MetadataTestUtils.buildTestTableMetadataWithExampleValues(formatVersion) + .setMetadataLocation(latestPreviousMetadata.file()) + .setLastUpdatedMillis(currentTimestamp - 50) + .setLastColumnId(3) + .setCurrentSnapshotId(currentSnapshotId) + .setSnapshots(Arrays.asList(previousSnapshot, currentSnapshot)) + .setSnapshotLog(reversedSnapshotLog) + .setMetadataHistory(ImmutableList.copyOf(previousMetadataLog)) + .build(); previousMetadataLog.add(latestPreviousMetadata); @@ -755,30 +573,22 @@ public void testAddPreviousMetadataRemoveOne() throws IOException { .isEqualTo(previousMetadataLog.subList(0, 1)); } - @Test - public void testAddPreviousMetadataRemoveMultiple() throws IOException { + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + public void testAddPreviousMetadataRemoveMultiple(int formatVersion) throws IOException { long previousSnapshotId = System.currentTimeMillis() - new Random(1234).nextInt(3600); - String manifestList = - createManifestListWithManifestFile(previousSnapshotId, null, "file:/tmp/manifest1.avro"); Snapshot previousSnapshot = - new BaseSnapshot( - 0, previousSnapshotId, null, previousSnapshotId, null, null, null, manifestList); + MetadataTestUtils.buildTestSnapshot() + .setSnapshotId(previousSnapshotId) + .buildWithExampleManifestList(temp, ImmutableList.of("file:/tmp/manifest1.avro")); long currentSnapshotId = System.currentTimeMillis(); - manifestList = - createManifestListWithManifestFile( - currentSnapshotId, previousSnapshotId, "file:/tmp/manifest2.avro"); Snapshot currentSnapshot = - new BaseSnapshot( - 0, - currentSnapshotId, - previousSnapshotId, - currentSnapshotId, - null, - null, - null, - manifestList); + MetadataTestUtils.buildTestSnapshot() + .setSnapshotId(currentSnapshotId) + .setParentId(previousSnapshotId) + .buildWithExampleManifestList(temp, ImmutableList.of("file:/tmp/manifest2.avro")); List reversedSnapshotLog = Lists.newArrayList(); reversedSnapshotLog.add( @@ -808,31 +618,14 @@ public void testAddPreviousMetadataRemoveMultiple() throws IOException { currentTimestamp - 50, "/tmp/000006-" + UUID.randomUUID() + ".metadata.json"); TableMetadata base = - new TableMetadata( - latestPreviousMetadata.file(), - 1, - UUID.randomUUID().toString(), - TEST_LOCATION, - 0, - currentTimestamp - 50, - 3, - 7, - ImmutableList.of(TEST_SCHEMA), - SPEC_5.specId(), - ImmutableList.of(SPEC_5), - SPEC_5.lastAssignedFieldId(), - SortOrder.unsorted().orderId(), - ImmutableList.of(SortOrder.unsorted()), - ImmutableMap.of("property", "value"), - currentSnapshotId, - Arrays.asList(previousSnapshot, currentSnapshot), - null, - reversedSnapshotLog, - ImmutableList.copyOf(previousMetadataLog), - ImmutableMap.of(), - ImmutableList.of(), - ImmutableList.of(), - ImmutableList.of()); + MetadataTestUtils.buildTestTableMetadataWithExampleValues(formatVersion) + .setMetadataLocation(latestPreviousMetadata.file()) + .setLastUpdatedMillis(currentTimestamp - 50) + .setCurrentSnapshotId(currentSnapshotId) + .setSnapshots(Arrays.asList(previousSnapshot, currentSnapshot)) + .setSnapshotLog(reversedSnapshotLog) + .setMetadataHistory(ImmutableList.copyOf(previousMetadataLog)) + .build(); previousMetadataLog.add(latestPreviousMetadata); @@ -850,37 +643,18 @@ public void testAddPreviousMetadataRemoveMultiple() throws IOException { .isEqualTo(previousMetadataLog.subList(0, 4)); } - @Test - public void testV2UUIDValidation() { + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + public void testUUIDValidation(int formatVersion) { + assumeThat(formatVersion).isGreaterThanOrEqualTo(UUID_REQUIRED_MIN_VERSION); + assertThatThrownBy( () -> - new TableMetadata( - null, - 2, - null, - TEST_LOCATION, - SEQ_NO, - System.currentTimeMillis(), - LAST_ASSIGNED_COLUMN_ID, - 7, - ImmutableList.of(TEST_SCHEMA), - SPEC_5.specId(), - ImmutableList.of(SPEC_5), - SPEC_5.lastAssignedFieldId(), - 3, - ImmutableList.of(SORT_ORDER_3), - ImmutableMap.of(), - -1L, - ImmutableList.of(), - null, - ImmutableList.of(), - ImmutableList.of(), - ImmutableMap.of(), - ImmutableList.of(), - ImmutableList.of(), - ImmutableList.of())) + MetadataTestUtils.buildTestTableMetadataWithExampleValues(formatVersion) + .setUuid(null) + .build()) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("UUID is required in format v2"); + .hasMessage(String.format("UUID is required in format v%s", formatVersion)); } @Test @@ -889,31 +663,8 @@ public void testVersionValidation() { int unsupportedVersion = supportedVersion + 1; assertThatThrownBy( () -> - new TableMetadata( - null, - unsupportedVersion, - null, - TEST_LOCATION, - SEQ_NO, - System.currentTimeMillis(), - LAST_ASSIGNED_COLUMN_ID, - 7, - ImmutableList.of(TEST_SCHEMA), - SPEC_5.specId(), - ImmutableList.of(SPEC_5), - SPEC_5.lastAssignedFieldId(), - 3, - ImmutableList.of(SORT_ORDER_3), - ImmutableMap.of(), - -1L, - ImmutableList.of(), - null, - ImmutableList.of(), - ImmutableList.of(), - ImmutableMap.of(), - ImmutableList.of(), - ImmutableList.of(), - ImmutableList.of())) + MetadataTestUtils.buildTestTableMetadataWithExampleValues(unsupportedVersion) + .build()) .isInstanceOf(IllegalArgumentException.class) .hasMessage( "Unsupported format version: v%s (supported: v%s)", @@ -934,32 +685,7 @@ public void testVersionValidation() { unsupportedVersion, supportedVersion); // should be allowed in the supported version - assertThat( - new TableMetadata( - null, - supportedVersion, - UUID.randomUUID().toString(), - TEST_LOCATION, - SEQ_NO, - System.currentTimeMillis(), - LAST_ASSIGNED_COLUMN_ID, - 7, - ImmutableList.of(TEST_SCHEMA), - SPEC_5.specId(), - ImmutableList.of(SPEC_5), - SPEC_5.lastAssignedFieldId(), - 3, - ImmutableList.of(SORT_ORDER_3), - ImmutableMap.of(), - -1L, - ImmutableList.of(), - null, - ImmutableList.of(), - ImmutableList.of(), - ImmutableMap.of(), - ImmutableList.of(), - ImmutableList.of(), - ImmutableList.of())) + assertThat(MetadataTestUtils.buildTestTableMetadataWithExampleValues(supportedVersion).build()) .isNotNull(); assertThat( @@ -973,62 +699,86 @@ public void testVersionValidation() { .isNotNull(); } - @Test - public void testParserVersionValidation() throws Exception { - String supportedVersion1 = readTableMetadataInputFile("TableMetadataV1Valid.json"); - TableMetadata parsed1 = TableMetadataParser.fromJson(supportedVersion1); - assertThat(parsed1).as("Should successfully read supported metadata version").isNotNull(); - - String supportedVersion2 = readTableMetadataInputFile("TableMetadataV2Valid.json"); - TableMetadata parsed2 = TableMetadataParser.fromJson(supportedVersion2); - assertThat(parsed2).as("Should successfully read supported metadata version").isNotNull(); + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + public void testParserVersionValidation(int formatVersion) throws Exception { + String supportedVersion = + readTableMetadataInputFile(String.format("TableMetadataV%sValid.json", formatVersion)); + TableMetadata parsed = TableMetadataParser.fromJson(supportedVersion); + assertThat(parsed).as("Should successfully read supported metadata version").isNotNull(); + } + @Test + public void testParserUnsupportedVersion() throws Exception { String unsupportedVersion = readTableMetadataInputFile("TableMetadataUnsupportedVersion.json"); assertThatThrownBy(() -> TableMetadataParser.fromJson(unsupportedVersion)) .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith("Cannot read unsupported version"); } - @Test - public void testParserV2PartitionSpecsValidation() throws Exception { + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + public void testParserV2PartitionSpecsValidation(int formatVersion) throws Exception { + assumeThat(formatVersion).isGreaterThanOrEqualTo(2); + String unsupportedVersion = - readTableMetadataInputFile("TableMetadataV2MissingPartitionSpecs.json"); + readTableMetadataInputFile( + String.format("TableMetadataV%sMissingPartitionSpecs.json", formatVersion)); assertThatThrownBy(() -> TableMetadataParser.fromJson(unsupportedVersion)) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("partition-specs must exist in format v2"); + .hasMessage(String.format("partition-specs must exist in format v%s", formatVersion)); } - @Test - public void testParserV2LastAssignedFieldIdValidation() throws Exception { + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + public void testParserLastAssignedFieldIdValidation(int formatVersion) throws Exception { + assumeThat(formatVersion).isGreaterThanOrEqualTo(2); + String unsupportedVersion = - readTableMetadataInputFile("TableMetadataV2MissingLastPartitionId.json"); + readTableMetadataInputFile( + String.format("TableMetadataV%sMissingLastPartitionId.json", formatVersion)); assertThatThrownBy(() -> TableMetadataParser.fromJson(unsupportedVersion)) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("last-partition-id must exist in format v2"); + .hasMessage(String.format("last-partition-id must exist in format v%s", formatVersion)); } - @Test - public void testParserV2SortOrderValidation() throws Exception { - String unsupportedVersion = readTableMetadataInputFile("TableMetadataV2MissingSortOrder.json"); + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + public void testParserSortOrderValidation(int formatVersion) throws Exception { + assumeThat(formatVersion).isGreaterThanOrEqualTo(2); + + String unsupportedVersion = + readTableMetadataInputFile( + String.format("TableMetadataV%sMissingSortOrder.json", formatVersion)); assertThatThrownBy(() -> TableMetadataParser.fromJson(unsupportedVersion)) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("sort-orders must exist in format v2"); + .hasMessage(String.format("sort-orders must exist in format v%s", formatVersion)); } - @Test - public void testParserV2CurrentSchemaIdValidation() throws Exception { - String unsupported = readTableMetadataInputFile("TableMetadataV2CurrentSchemaNotFound.json"); + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + public void testParserCurrentSchemaIdValidation(int formatVersion) throws Exception { + assumeThat(formatVersion).isGreaterThanOrEqualTo(2); + + String unsupported = + readTableMetadataInputFile( + String.format("TableMetadataV%sCurrentSchemaNotFound.json", formatVersion)); assertThatThrownBy(() -> TableMetadataParser.fromJson(unsupported)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot find schema with current-schema-id=2 from schemas"); } - @Test - public void testParserV2SchemasValidation() throws Exception { - String unsupported = readTableMetadataInputFile("TableMetadataV2MissingSchemas.json"); + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + public void testParserV2SchemasValidation(int formatVersion) throws Exception { + assumeThat(formatVersion).isGreaterThanOrEqualTo(2); + + String unsupported = + readTableMetadataInputFile( + String.format("TableMetadataV%sMissingSchemas.json", formatVersion)); assertThatThrownBy(() -> TableMetadataParser.fromJson(unsupported)) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("schemas must exist in format v2"); + .hasMessage(String.format("schemas must exist in format v%s", formatVersion)); } private String readTableMetadataInputFile(String fileName) throws Exception { @@ -1036,8 +786,9 @@ private String readTableMetadataInputFile(String fileName) throws Exception { return String.join("", java.nio.file.Files.readAllLines(path)); } - @Test - public void testNewTableMetadataReassignmentAllIds() throws Exception { + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + public void testNewTableMetadataReassignmentAllIds(int formatVersion) throws Exception { Schema schema = new Schema( Types.NestedField.required(3, "x", Types.LongType.get()), @@ -1052,7 +803,8 @@ public void testNewTableMetadataReassignmentAllIds() throws Exception { .build(); String location = "file://tmp/db/table"; TableMetadata metadata = - TableMetadata.newTableMetadata(schema, spec, location, ImmutableMap.of()); + TableMetadata.newTableMetadata( + schema, spec, SortOrder.unsorted(), location, ImmutableMap.of(), formatVersion); // newTableMetadata should reassign column ids and partition field ids. PartitionSpec expected = @@ -1126,8 +878,11 @@ public void testBuildReplacementForV1Table() { .isEqualTo(expected); } - @Test - public void testBuildReplacementForV2Table() { + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + public void testBuildReplacementForV2AndV3Table(int formatVersion) { + assumeThat(formatVersion).isGreaterThanOrEqualTo(2); + Schema schema = new Schema( Types.NestedField.required(1, "x", Types.LongType.get()), @@ -1137,7 +892,7 @@ public void testBuildReplacementForV2Table() { String location = "file://tmp/db/table"; TableMetadata metadata = TableMetadata.newTableMetadata( - schema, spec, SortOrder.unsorted(), location, ImmutableMap.of(), 2); + schema, spec, SortOrder.unsorted(), location, ImmutableMap.of(), formatVersion); assertThat(metadata.spec()).isEqualTo(spec); Schema updatedSchema = @@ -1161,28 +916,39 @@ public void testBuildReplacementForV2Table() { .isEqualTo(expected); } - @Test - public void testSortOrder() { + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + public void testSortOrder(int formatVersion) { Schema schema = new Schema(Types.NestedField.required(10, "x", Types.StringType.get())); TableMetadata meta = TableMetadata.newTableMetadata( - schema, PartitionSpec.unpartitioned(), null, ImmutableMap.of()); + schema, + PartitionSpec.unpartitioned(), + null, + ImmutableMap.of("format-version", String.valueOf(formatVersion))); + assertThat(meta.formatVersion()).isEqualTo(formatVersion); assertThat(meta.sortOrder().isUnsorted()).isTrue(); assertThat(meta.replaceSortOrder(SortOrder.unsorted())) .as("Should detect identical unsorted order") .isSameAs(meta); } - @Test - public void testUpdateSortOrder() { + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + public void testUpdateSortOrder(int formatVersion) { Schema schema = new Schema(Types.NestedField.required(10, "x", Types.StringType.get())); SortOrder order = SortOrder.builderFor(schema).asc("x").build(); TableMetadata sortedByX = TableMetadata.newTableMetadata( - schema, PartitionSpec.unpartitioned(), order, null, ImmutableMap.of()); + schema, + PartitionSpec.unpartitioned(), + order, + null, + ImmutableMap.of("format-version", String.valueOf(formatVersion))); + assertThat(sortedByX.formatVersion()).isEqualTo(formatVersion); assertThat(sortedByX.sortOrders()).hasSize(1); assertThat(sortedByX.sortOrder().orderId()).isEqualTo(1); assertThat(sortedByX.sortOrder().fields()).hasSize(1); @@ -1213,23 +979,35 @@ public void testUpdateSortOrder() { assertThat(sortedByX.sortOrder().fields().get(0).nullOrder()).isEqualTo(NullOrder.NULLS_FIRST); } - @Test - public void testStatistics() { + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + public void testStatistics(int formatVersion) { Schema schema = new Schema(Types.NestedField.required(10, "x", Types.StringType.get())); TableMetadata meta = TableMetadata.newTableMetadata( - schema, PartitionSpec.unpartitioned(), null, ImmutableMap.of()); + schema, + PartitionSpec.unpartitioned(), + SortOrder.unsorted(), + null, + ImmutableMap.of(), + formatVersion); assertThat(meta.statisticsFiles()).as("Should default to no statistics files").isEmpty(); } - @Test - public void testSetStatistics() { + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + public void testSetStatistics(int formatVersion) { Schema schema = new Schema(Types.NestedField.required(10, "x", Types.StringType.get())); TableMetadata meta = TableMetadata.newTableMetadata( - schema, PartitionSpec.unpartitioned(), null, ImmutableMap.of()); + schema, + PartitionSpec.unpartitioned(), + SortOrder.unsorted(), + null, + ImmutableMap.of(), + formatVersion); TableMetadata withStatistics = TableMetadata.buildFrom(meta) @@ -1262,14 +1040,20 @@ public void testSetStatistics() { assertThat(statisticsFile.path()).isEqualTo("/some/path/to/stats/file2"); } - @Test - public void testRemoveStatistics() { + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + public void testRemoveStatistics(int formatVersion) { Schema schema = new Schema(Types.NestedField.required(10, "x", Types.StringType.get())); TableMetadata meta = TableMetadata.buildFrom( TableMetadata.newTableMetadata( - schema, PartitionSpec.unpartitioned(), null, ImmutableMap.of())) + schema, + PartitionSpec.unpartitioned(), + SortOrder.unsorted(), + null, + ImmutableMap.of(), + formatVersion)) .setStatistics( 43, new GenericStatisticsFile( @@ -1294,25 +1078,37 @@ public void testRemoveStatistics() { assertThat(statisticsFile.path()).isEqualTo("/some/path/to/stats/file2"); } - @Test - public void testPartitionStatistics() { + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + public void testPartitionStatistics(int formatVersion) { Schema schema = new Schema(Types.NestedField.required(10, "x", Types.StringType.get())); TableMetadata meta = TableMetadata.newTableMetadata( - schema, PartitionSpec.unpartitioned(), null, ImmutableMap.of()); + schema, + PartitionSpec.unpartitioned(), + SortOrder.unsorted(), + null, + ImmutableMap.of(), + formatVersion); assertThat(meta.partitionStatisticsFiles()) .as("Should default to no partition statistics files") .isEmpty(); } - @Test - public void testSetPartitionStatistics() { + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + public void testSetPartitionStatistics(int formatVersion) { Schema schema = new Schema(Types.NestedField.required(10, "x", Types.StringType.get())); TableMetadata meta = TableMetadata.newTableMetadata( - schema, PartitionSpec.unpartitioned(), null, ImmutableMap.of()); + schema, + PartitionSpec.unpartitioned(), + SortOrder.unsorted(), + null, + ImmutableMap.of(), + formatVersion); TableMetadata withPartitionStatistics = TableMetadata.buildFrom(meta) @@ -1355,14 +1151,20 @@ public void testSetPartitionStatistics() { assertThat(partitionStatisticsFile.fileSizeInBytes()).isEqualTo(48L); } - @Test - public void testRemovePartitionStatistics() { + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + public void testRemovePartitionStatistics(int formatVersion) { Schema schema = new Schema(Types.NestedField.required(10, "x", Types.StringType.get())); TableMetadata meta = TableMetadata.buildFrom( TableMetadata.newTableMetadata( - schema, PartitionSpec.unpartitioned(), null, ImmutableMap.of())) + schema, + PartitionSpec.unpartitioned(), + SortOrder.unsorted(), + null, + ImmutableMap.of(), + formatVersion)) .setPartitionStatistics( ImmutableGenericPartitionStatisticsFile.builder() .snapshotId(43) @@ -1395,17 +1197,26 @@ public void testRemovePartitionStatistics() { assertThat(partitionStatisticsFile.fileSizeInBytes()).isEqualTo(49L); } - @Test - public void testParseSchemaIdentifierFields() throws Exception { - String data = readTableMetadataInputFile("TableMetadataV2Valid.json"); + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + public void testParseSchemaIdentifierFields(int formatVersion) throws Exception { + assumeThat(formatVersion).isGreaterThanOrEqualTo(2); + + String data = + readTableMetadataInputFile(String.format("TableMetadataV%sValid.json", formatVersion)); TableMetadata parsed = TableMetadataParser.fromJson(data); assertThat(parsed.schemasById().get(0).identifierFieldIds()).isEmpty(); assertThat(parsed.schemasById().get(1).identifierFieldIds()).containsExactly(1, 2); } - @Test - public void testParseMinimal() throws Exception { - String data = readTableMetadataInputFile("TableMetadataV2ValidMinimal.json"); + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + public void testParseMinimal(int formatVersion) throws Exception { + assumeThat(formatVersion).isGreaterThanOrEqualTo(2); + + String data = + readTableMetadataInputFile( + String.format("TableMetadataV%sValidMinimal.json", formatVersion)); TableMetadata parsed = TableMetadataParser.fromJson(data); assertThat(parsed.snapshots()).isEmpty(); assertThat(parsed.snapshotLog()).isEmpty(); @@ -1413,13 +1224,19 @@ public void testParseMinimal() throws Exception { assertThat(parsed.previousFiles()).isEmpty(); } - @Test - public void testUpdateSchemaIdentifierFields() { + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + public void testUpdateSchemaIdentifierFields(int formatVersion) { Schema schema = new Schema(Types.NestedField.required(10, "x", Types.StringType.get())); TableMetadata meta = TableMetadata.newTableMetadata( - schema, PartitionSpec.unpartitioned(), null, ImmutableMap.of()); + schema, + PartitionSpec.unpartitioned(), + SortOrder.unsorted(), + null, + ImmutableMap.of(), + formatVersion); Schema newSchema = new Schema( @@ -1430,13 +1247,19 @@ public void testUpdateSchemaIdentifierFields() { assertThat(newMeta.schema().identifierFieldIds()).containsExactly(1); } - @Test - public void testUpdateSchema() { + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + public void testUpdateSchema(int formatVersion) { Schema schema = new Schema(0, Types.NestedField.required(1, "y", Types.LongType.get(), "comment")); TableMetadata freshTable = TableMetadata.newTableMetadata( - schema, PartitionSpec.unpartitioned(), null, ImmutableMap.of()); + schema, + PartitionSpec.unpartitioned(), + SortOrder.unsorted(), + null, + ImmutableMap.of(), + formatVersion); assertThat(freshTable.currentSchemaId()).isEqualTo(TableMetadata.INITIAL_SCHEMA_ID); assertSameSchemaList(ImmutableList.of(schema), freshTable.schemas()); assertThat(freshTable.schema().asStruct()).isEqualTo(schema.asStruct()); @@ -1496,8 +1319,9 @@ schema, new Schema(1, schema2.columns()), new Schema(2, schema3.columns())), assertThat(threeSchemaTable.lastColumnId()).isEqualTo(6); } - @Test - public void testCreateV2MetadataThroughTableProperty() { + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + public void testCreateMetadataThroughTableProperty(int formatVersion) { Schema schema = new Schema(Types.NestedField.required(10, "x", Types.StringType.get())); TableMetadata meta = @@ -1505,9 +1329,10 @@ public void testCreateV2MetadataThroughTableProperty() { schema, PartitionSpec.unpartitioned(), null, - ImmutableMap.of(TableProperties.FORMAT_VERSION, "2", "key", "val")); + ImmutableMap.of( + TableProperties.FORMAT_VERSION, String.valueOf(formatVersion), "key", "val")); - assertThat(meta.formatVersion()).isEqualTo(2); + assertThat(meta.formatVersion()).isEqualTo(formatVersion); assertThat(meta.properties()) .containsEntry("key", "val") .doesNotContainKey(TableProperties.FORMAT_VERSION); @@ -1579,9 +1404,14 @@ public void testUpgradeMetadataThroughTableProperty(int baseFormatVersion, int n .containsExactly(entry("key2", "val2")); } - @Test - public void testParseStatisticsFiles() throws Exception { - String data = readTableMetadataInputFile("TableMetadataStatisticsFiles.json"); + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + public void testParseStatisticsFiles(int formatVersion) throws Exception { + assumeThat(formatVersion).isGreaterThanOrEqualTo(2); + + String data = + readTableMetadataInputFile( + String.format("TableMetadataV%sStatisticsFiles.json", formatVersion)); TableMetadata parsed = TableMetadataParser.fromJson(data); assertThat(parsed.statisticsFiles()).hasSize(1); assertThat(parsed.statisticsFiles()) @@ -1598,9 +1428,14 @@ public void testParseStatisticsFiles() throws Exception { "ndv", 3055729675574597004L, 1, ImmutableList.of(1), ImmutableMap.of())))); } - @Test - public void testParsePartitionStatisticsFiles() throws Exception { - String data = readTableMetadataInputFile("TableMetadataPartitionStatisticsFiles.json"); + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + public void testParsePartitionStatisticsFiles(int formatVersion) throws Exception { + assumeThat(formatVersion).isGreaterThanOrEqualTo(2); + + String data = + readTableMetadataInputFile( + String.format("TableMetadataV%sPartitionStatisticsFiles.json", formatVersion)); TableMetadata parsed = TableMetadataParser.fromJson(data); assertThat(parsed.partitionStatisticsFiles()) .hasSize(1) @@ -1613,8 +1448,9 @@ public void testParsePartitionStatisticsFiles() throws Exception { .build()); } - @Test - public void testNoReservedPropertyForTableMetadataCreation() { + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + public void testNoReservedPropertyForTableMetadataCreation(int formatVersion) { Schema schema = new Schema(Types.NestedField.required(10, "x", Types.StringType.get())); assertThatThrownBy( @@ -1624,11 +1460,13 @@ public void testNoReservedPropertyForTableMetadataCreation() { PartitionSpec.unpartitioned(), null, "/tmp", - ImmutableMap.of(TableProperties.FORMAT_VERSION, "1"), - 1)) + ImmutableMap.of(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)), + formatVersion)) .isInstanceOf(IllegalArgumentException.class) .hasMessage( - "Table properties should not contain reserved properties, but got {format-version=1}"); + String.format( + "Table properties should not contain reserved properties, but got {format-version=%s}", + formatVersion)); assertThatThrownBy( () -> @@ -1638,42 +1476,37 @@ public void testNoReservedPropertyForTableMetadataCreation() { null, "/tmp", ImmutableMap.of(TableProperties.UUID, "uuid"), - 1)) + formatVersion)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Table properties should not contain reserved properties, but got {uuid=uuid}"); } - @Test - public void testNoTrailingLocationSlash() { + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + public void testNoTrailingLocationSlash(int formatVersion) { String locationWithSlash = "/with_trailing_slash/"; String locationWithoutSlash = "/with_trailing_slash"; TableMetadata meta = TableMetadata.newTableMetadata( - TEST_SCHEMA, SPEC_5, SORT_ORDER_3, locationWithSlash, Collections.emptyMap()); + TEST_SCHEMA, + SPEC_5, + SORT_ORDER_3, + locationWithSlash, + Collections.emptyMap(), + formatVersion); assertThat(meta.location()) .as("Metadata should never return a location ending in a slash") .isEqualTo(locationWithoutSlash); } - private String createManifestListWithManifestFile( - long snapshotId, Long parentSnapshotId, String manifestFile) throws IOException { - File manifestList = File.createTempFile("manifests", null, temp.toFile()); - manifestList.deleteOnExit(); - - try (ManifestListWriter writer = - ManifestLists.write(1, Files.localOutput(manifestList), snapshotId, parentSnapshotId, 0)) { - writer.addAll( - ImmutableList.of( - new GenericManifestFile(localInput(manifestFile), SPEC_5.specId(), snapshotId))); - } - - return localInput(manifestList).location(); - } + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + public void buildReplacementKeepsSnapshotLog(int formatVersion) throws Exception { + assumeThat(formatVersion).isGreaterThanOrEqualTo(2); - @Test - public void buildReplacementKeepsSnapshotLog() throws Exception { TableMetadata metadata = - TableMetadataParser.fromJson(readTableMetadataInputFile("TableMetadataV2Valid.json")); + TableMetadataParser.fromJson( + readTableMetadataInputFile(String.format("TableMetadataV%sValid.json", formatVersion))); assertThat(metadata.currentSnapshot()).isNotNull(); assertThat(metadata.snapshots()).hasSize(2); assertThat(metadata.snapshotLog()).hasSize(2); @@ -1711,22 +1544,12 @@ public void removeRefKeepsSnapshotLog() throws Exception { .containsExactlyElementsOf(metadata.snapshotLog()); } - @Test - public void testConstructV3Metadata() { - TableMetadata.newTableMetadata( - TEST_SCHEMA, - PartitionSpec.unpartitioned(), - SortOrder.unsorted(), - TEST_LOCATION, - ImmutableMap.of(), - 3); - } - - @Test - public void onlyMetadataLocationIsUpdatedWithoutTimestampAndMetadataLogEntry() { + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + public void onlyMetadataLocationIsUpdatedWithoutTimestampAndMetadataLogEntry(int formatVersion) { String uuid = "386b9f01-002b-4d8c-b77f-42c3fd3b7c9b"; TableMetadata metadata = - TableMetadata.buildFromEmpty() + TableMetadata.buildFromEmpty(formatVersion) .assignUUID(uuid) .setLocation("location") .setCurrentSchema(TEST_SCHEMA, 3) diff --git a/core/src/test/resources/TableMetadataPartitionStatisticsFiles.json b/core/src/test/resources/TableMetadataV2PartitionStatisticsFiles.json similarity index 100% rename from core/src/test/resources/TableMetadataPartitionStatisticsFiles.json rename to core/src/test/resources/TableMetadataV2PartitionStatisticsFiles.json diff --git a/core/src/test/resources/TableMetadataStatisticsFiles.json b/core/src/test/resources/TableMetadataV2StatisticsFiles.json similarity index 100% rename from core/src/test/resources/TableMetadataStatisticsFiles.json rename to core/src/test/resources/TableMetadataV2StatisticsFiles.json diff --git a/core/src/test/resources/TableMetadataV3CurrentSchemaNotFound.json b/core/src/test/resources/TableMetadataV3CurrentSchemaNotFound.json new file mode 100644 index 000000000000..6f1a3d907d7d --- /dev/null +++ b/core/src/test/resources/TableMetadataV3CurrentSchemaNotFound.json @@ -0,0 +1,146 @@ +{ + "format-version": 3, + "table-uuid": "9faafb13-1ce7-4603-93f8-197afc7394a9", + "location": "s3://bucket/test/location", + "last-sequence-number": 34, + "last-updated-ms": 1733710954170, + "last-column-id": 3, + "current-schema-id": 2, + "schemas": [ + { + "type": "struct", + "schema-id": 7, + "fields": [ + { + "id": 1, + "name": "x", + "required": true, + "type": "long" + }, + { + "id": 2, + "name": "y", + "required": true, + "type": "long", + "doc": "comment" + }, + { + "id": 3, + "name": "z", + "required": true, + "type": "long" + } + ] + }, + { + "type": "struct", + "schema-id": 6, + "fields": [ + { + "id": 10, + "name": "x", + "required": true, + "type": "string" + } + ] + } + ], + "default-spec-id": 5, + "partition-specs": [ + { + "spec-id": 5, + "fields": [] + } + ], + "last-partition-id": 999, + "default-sort-order-id": 3, + "sort-orders": [ + { + "order-id": 3, + "fields": [ + { + "transform": "identity", + "source-id": 2, + "direction": "asc", + "null-order": "nulls-first" + }, + { + "transform": "bucket[4]", + "source-id": 3, + "direction": "desc", + "null-order": "nulls-last" + } + ] + } + ], + "properties": { + "property": "value" + }, + "current-snapshot-id": 1733710954168, + "refs": { + "main": { + "snapshot-id": 1733710954168, + "type": "branch" + }, + "previous": { + "snapshot-id": 1733710953138, + "type": "tag" + }, + "test": { + "snapshot-id": 1733710953138, + "type": "branch" + } + }, + "snapshots": [ + { + "snapshot-id": 1733710953138, + "timestamp-ms": 1733710953138, + "manifest-list": "/var/folders/z8/cv70q7pj1l13r_j628cyyks80000gn/T/junit-9477945946492951907/manifests17078459248221091859.tmp" + }, + { + "snapshot-id": 1733710954168, + "parent-snapshot-id": 1733710953138, + "timestamp-ms": 1733710954168, + "manifest-list": "/var/folders/z8/cv70q7pj1l13r_j628cyyks80000gn/T/junit-9477945946492951907/manifests17490643917806818981.tmp", + "schema-id": 7 + } + ], + "statistics": [ + { + "snapshot-id": 11, + "statistics-path": "/some/stats/file.puffin", + "file-size-in-bytes": 100, + "file-footer-size-in-bytes": 42, + "blob-metadata": [ + { + "type": "some-stats", + "snapshot-id": 11, + "sequence-number": 2, + "fields": [ + 4 + ] + } + ] + } + ], + "partition-statistics": [ + { + "snapshot-id": 11, + "statistics-path": "/some/partition/stats/file.parquet", + "file-size-in-bytes": 42 + } + ], + "row-lineage": false, + "next-row-id": 0, + "snapshot-log": [ + { + "timestamp-ms": 1733710953138, + "snapshot-id": 1733710953138 + }, + { + "timestamp-ms": 1733710954168, + "snapshot-id": 1733710954168 + } + ], + "metadata-log": [] +} \ No newline at end of file diff --git a/core/src/test/resources/TableMetadataV3MissingLastPartitionId.json b/core/src/test/resources/TableMetadataV3MissingLastPartitionId.json new file mode 100644 index 000000000000..afd819feb8b5 --- /dev/null +++ b/core/src/test/resources/TableMetadataV3MissingLastPartitionId.json @@ -0,0 +1,145 @@ +{ + "format-version": 3, + "table-uuid": "9faafb13-1ce7-4603-93f8-197afc7394a9", + "location": "s3://bucket/test/location", + "last-sequence-number": 34, + "last-updated-ms": 1733710954170, + "last-column-id": 3, + "current-schema-id": 7, + "schemas": [ + { + "type": "struct", + "schema-id": 7, + "fields": [ + { + "id": 1, + "name": "x", + "required": true, + "type": "long" + }, + { + "id": 2, + "name": "y", + "required": true, + "type": "long", + "doc": "comment" + }, + { + "id": 3, + "name": "z", + "required": true, + "type": "long" + } + ] + }, + { + "type": "struct", + "schema-id": 6, + "fields": [ + { + "id": 10, + "name": "x", + "required": true, + "type": "string" + } + ] + } + ], + "default-spec-id": 5, + "partition-specs": [ + { + "spec-id": 5, + "fields": [] + } + ], + "default-sort-order-id": 3, + "sort-orders": [ + { + "order-id": 3, + "fields": [ + { + "transform": "identity", + "source-id": 2, + "direction": "asc", + "null-order": "nulls-first" + }, + { + "transform": "bucket[4]", + "source-id": 3, + "direction": "desc", + "null-order": "nulls-last" + } + ] + } + ], + "properties": { + "property": "value" + }, + "current-snapshot-id": 1733710954168, + "refs": { + "main": { + "snapshot-id": 1733710954168, + "type": "branch" + }, + "previous": { + "snapshot-id": 1733710953138, + "type": "tag" + }, + "test": { + "snapshot-id": 1733710953138, + "type": "branch" + } + }, + "snapshots": [ + { + "snapshot-id": 1733710953138, + "timestamp-ms": 1733710953138, + "manifest-list": "/var/folders/z8/cv70q7pj1l13r_j628cyyks80000gn/T/junit-9477945946492951907/manifests17078459248221091859.tmp" + }, + { + "snapshot-id": 1733710954168, + "parent-snapshot-id": 1733710953138, + "timestamp-ms": 1733710954168, + "manifest-list": "/var/folders/z8/cv70q7pj1l13r_j628cyyks80000gn/T/junit-9477945946492951907/manifests17490643917806818981.tmp", + "schema-id": 7 + } + ], + "statistics": [ + { + "snapshot-id": 11, + "statistics-path": "/some/stats/file.puffin", + "file-size-in-bytes": 100, + "file-footer-size-in-bytes": 42, + "blob-metadata": [ + { + "type": "some-stats", + "snapshot-id": 11, + "sequence-number": 2, + "fields": [ + 4 + ] + } + ] + } + ], + "partition-statistics": [ + { + "snapshot-id": 11, + "statistics-path": "/some/partition/stats/file.parquet", + "file-size-in-bytes": 42 + } + ], + "row-lineage": false, + "next-row-id": 0, + "snapshot-log": [ + { + "timestamp-ms": 1733710953138, + "snapshot-id": 1733710953138 + }, + { + "timestamp-ms": 1733710954168, + "snapshot-id": 1733710954168 + } + ], + "metadata-log": [] +} \ No newline at end of file diff --git a/core/src/test/resources/TableMetadataV3MissingPartitionSpecs.json b/core/src/test/resources/TableMetadataV3MissingPartitionSpecs.json new file mode 100644 index 000000000000..ea7e6002856b --- /dev/null +++ b/core/src/test/resources/TableMetadataV3MissingPartitionSpecs.json @@ -0,0 +1,146 @@ +{ + "format-version": 3, + "table-uuid": "9faafb13-1ce7-4603-93f8-197afc7394a9", + "location": "s3://bucket/test/location", + "last-sequence-number": 34, + "last-updated-ms": 1733710954170, + "last-column-id": 3, + "current-schema-id": 7, + "schemas": [ + { + "type": "struct", + "schema-id": 7, + "fields": [ + { + "id": 1, + "name": "x", + "required": true, + "type": "long" + }, + { + "id": 2, + "name": "y", + "required": true, + "type": "long", + "doc": "comment" + }, + { + "id": 3, + "name": "z", + "required": true, + "type": "long" + } + ] + }, + { + "type": "struct", + "schema-id": 6, + "fields": [ + { + "id": 10, + "name": "x", + "required": true, + "type": "string" + } + ] + } + ], + "partition-spec": [ + { + "name": "x", + "transform": "identity", + "source-id": 1, + "field-id": 1000 + } + ], + "default-sort-order-id": 3, + "sort-orders": [ + { + "order-id": 3, + "fields": [ + { + "transform": "identity", + "source-id": 2, + "direction": "asc", + "null-order": "nulls-first" + }, + { + "transform": "bucket[4]", + "source-id": 3, + "direction": "desc", + "null-order": "nulls-last" + } + ] + } + ], + "properties": { + "property": "value" + }, + "current-snapshot-id": 1733710954168, + "refs": { + "main": { + "snapshot-id": 1733710954168, + "type": "branch" + }, + "previous": { + "snapshot-id": 1733710953138, + "type": "tag" + }, + "test": { + "snapshot-id": 1733710953138, + "type": "branch" + } + }, + "snapshots": [ + { + "snapshot-id": 1733710953138, + "timestamp-ms": 1733710953138, + "manifest-list": "/var/folders/z8/cv70q7pj1l13r_j628cyyks80000gn/T/junit-9477945946492951907/manifests17078459248221091859.tmp" + }, + { + "snapshot-id": 1733710954168, + "parent-snapshot-id": 1733710953138, + "timestamp-ms": 1733710954168, + "manifest-list": "/var/folders/z8/cv70q7pj1l13r_j628cyyks80000gn/T/junit-9477945946492951907/manifests17490643917806818981.tmp", + "schema-id": 7 + } + ], + "statistics": [ + { + "snapshot-id": 11, + "statistics-path": "/some/stats/file.puffin", + "file-size-in-bytes": 100, + "file-footer-size-in-bytes": 42, + "blob-metadata": [ + { + "type": "some-stats", + "snapshot-id": 11, + "sequence-number": 2, + "fields": [ + 4 + ] + } + ] + } + ], + "partition-statistics": [ + { + "snapshot-id": 11, + "statistics-path": "/some/partition/stats/file.parquet", + "file-size-in-bytes": 42 + } + ], + "row-lineage": false, + "next-row-id": 0, + "snapshot-log": [ + { + "timestamp-ms": 1733710953138, + "snapshot-id": 1733710953138 + }, + { + "timestamp-ms": 1733710954168, + "snapshot-id": 1733710954168 + } + ], + "metadata-log": [] +} \ No newline at end of file diff --git a/core/src/test/resources/TableMetadataV3MissingSchemas.json b/core/src/test/resources/TableMetadataV3MissingSchemas.json new file mode 100644 index 000000000000..a4da9157cf38 --- /dev/null +++ b/core/src/test/resources/TableMetadataV3MissingSchemas.json @@ -0,0 +1,130 @@ +{ + "format-version": 3, + "table-uuid": "9faafb13-1ce7-4603-93f8-197afc7394a9", + "location": "s3://bucket/test/location", + "last-sequence-number": 34, + "last-updated-ms": 1733710954170, + "last-column-id": 3, + "schema": { + "type": "struct", + "fields": [ + { + "id": 1, + "name": "x", + "required": true, + "type": "long" + }, + { + "id": 2, + "name": "y", + "required": true, + "type": "long", + "doc": "comment" + }, + { + "id": 3, + "name": "z", + "required": true, + "type": "long" + } + ] + }, + "default-spec-id": 5, + "partition-specs": [ + { + "spec-id": 5, + "fields": [] + } + ], + "last-partition-id": 999, + "default-sort-order-id": 3, + "sort-orders": [ + { + "order-id": 3, + "fields": [ + { + "transform": "identity", + "source-id": 2, + "direction": "asc", + "null-order": "nulls-first" + }, + { + "transform": "bucket[4]", + "source-id": 3, + "direction": "desc", + "null-order": "nulls-last" + } + ] + } + ], + "properties": { + "property": "value" + }, + "current-snapshot-id": 1733710954168, + "refs": { + "main": { + "snapshot-id": 1733710954168, + "type": "branch" + }, + "previous": { + "snapshot-id": 1733710953138, + "type": "tag" + }, + "test": { + "snapshot-id": 1733710953138, + "type": "branch" + } + }, + "snapshots": [ + { + "snapshot-id": 1733710953138, + "timestamp-ms": 1733710953138, + "manifest-list": "/var/folders/z8/cv70q7pj1l13r_j628cyyks80000gn/T/junit-9477945946492951907/manifests17078459248221091859.tmp" + }, + { + "snapshot-id": 1733710954168, + "parent-snapshot-id": 1733710953138, + "timestamp-ms": 1733710954168, + "manifest-list": "/var/folders/z8/cv70q7pj1l13r_j628cyyks80000gn/T/junit-9477945946492951907/manifests17490643917806818981.tmp", + "schema-id": 7 + } + ], + "statistics": [ + { + "snapshot-id": 11, + "statistics-path": "/some/stats/file.puffin", + "file-size-in-bytes": 100, + "file-footer-size-in-bytes": 42, + "blob-metadata": [ + { + "type": "some-stats", + "snapshot-id": 11, + "sequence-number": 2, + "fields": [ + 4 + ] + } + ] + } + ], + "partition-statistics": [ + { + "snapshot-id": 11, + "statistics-path": "/some/partition/stats/file.parquet", + "file-size-in-bytes": 42 + } + ], + "row-lineage": false, + "next-row-id": 0, + "snapshot-log": [ + { + "timestamp-ms": 1733710953138, + "snapshot-id": 1733710953138 + }, + { + "timestamp-ms": 1733710954168, + "snapshot-id": 1733710954168 + } + ], + "metadata-log": [] +} \ No newline at end of file diff --git a/core/src/test/resources/TableMetadataV3MissingSortOrder.json b/core/src/test/resources/TableMetadataV3MissingSortOrder.json new file mode 100644 index 000000000000..80a3dd11d5f2 --- /dev/null +++ b/core/src/test/resources/TableMetadataV3MissingSortOrder.json @@ -0,0 +1,127 @@ +{ + "format-version": 3, + "table-uuid": "9faafb13-1ce7-4603-93f8-197afc7394a9", + "location": "s3://bucket/test/location", + "last-sequence-number": 34, + "last-updated-ms": 1733710954170, + "last-column-id": 3, + "current-schema-id": 7, + "schemas": [ + { + "type": "struct", + "schema-id": 7, + "fields": [ + { + "id": 1, + "name": "x", + "required": true, + "type": "long" + }, + { + "id": 2, + "name": "y", + "required": true, + "type": "long", + "doc": "comment" + }, + { + "id": 3, + "name": "z", + "required": true, + "type": "long" + } + ] + }, + { + "type": "struct", + "schema-id": 6, + "fields": [ + { + "id": 10, + "name": "x", + "required": true, + "type": "string" + } + ] + } + ], + "default-spec-id": 5, + "partition-specs": [ + { + "spec-id": 5, + "fields": [] + } + ], + "last-partition-id": 999, + "default-sort-order-id": 3, + "properties": { + "property": "value" + }, + "current-snapshot-id": 1733710954168, + "refs": { + "main": { + "snapshot-id": 1733710954168, + "type": "branch" + }, + "previous": { + "snapshot-id": 1733710953138, + "type": "tag" + }, + "test": { + "snapshot-id": 1733710953138, + "type": "branch" + } + }, + "snapshots": [ + { + "snapshot-id": 1733710953138, + "timestamp-ms": 1733710953138, + "manifest-list": "/var/folders/z8/cv70q7pj1l13r_j628cyyks80000gn/T/junit-9477945946492951907/manifests17078459248221091859.tmp" + }, + { + "snapshot-id": 1733710954168, + "parent-snapshot-id": 1733710953138, + "timestamp-ms": 1733710954168, + "manifest-list": "/var/folders/z8/cv70q7pj1l13r_j628cyyks80000gn/T/junit-9477945946492951907/manifests17490643917806818981.tmp", + "schema-id": 7 + } + ], + "statistics": [ + { + "snapshot-id": 11, + "statistics-path": "/some/stats/file.puffin", + "file-size-in-bytes": 100, + "file-footer-size-in-bytes": 42, + "blob-metadata": [ + { + "type": "some-stats", + "snapshot-id": 11, + "sequence-number": 2, + "fields": [ + 4 + ] + } + ] + } + ], + "partition-statistics": [ + { + "snapshot-id": 11, + "statistics-path": "/some/partition/stats/file.parquet", + "file-size-in-bytes": 42 + } + ], + "row-lineage": false, + "next-row-id": 0, + "snapshot-log": [ + { + "timestamp-ms": 1733710953138, + "snapshot-id": 1733710953138 + }, + { + "timestamp-ms": 1733710954168, + "snapshot-id": 1733710954168 + } + ], + "metadata-log": [] +} \ No newline at end of file diff --git a/core/src/test/resources/TableMetadataV3PartitionStatisticsFiles.json b/core/src/test/resources/TableMetadataV3PartitionStatisticsFiles.json new file mode 100644 index 000000000000..796f5bc8b13b --- /dev/null +++ b/core/src/test/resources/TableMetadataV3PartitionStatisticsFiles.json @@ -0,0 +1,61 @@ +{ + "format-version": 3, + "table-uuid": "9c12d441-03fe-4693-9a96-a0705ddf69c1", + "location": "s3://bucket/test/location", + "last-sequence-number": 34, + "last-updated-ms": 1602638573590, + "last-column-id": 3, + "current-schema-id": 0, + "schemas": [ + { + "type": "struct", + "schema-id": 0, + "fields": [ + { + "id": 1, + "name": "x", + "required": true, + "type": "long" + } + ] + } + ], + "default-spec-id": 0, + "partition-specs": [ + { + "spec-id": 0, + "fields": [] + } + ], + "last-partition-id": 1000, + "default-sort-order-id": 0, + "sort-orders": [ + { + "order-id": 0, + "fields": [] + } + ], + "properties": {}, + "current-snapshot-id": 3055729675574597004, + "snapshots": [ + { + "snapshot-id": 3055729675574597004, + "timestamp-ms": 1555100955770, + "sequence-number": 1, + "summary": { + "operation": "append" + }, + "manifest-list": "s3://a/b/2.avro", + "schema-id": 0 + } + ], + "partition-statistics": [ + { + "snapshot-id": 3055729675574597004, + "statistics-path": "s3://a/b/partition-stats.parquet", + "file-size-in-bytes": 43 + } + ], + "snapshot-log": [], + "metadata-log": [] +} diff --git a/core/src/test/resources/TableMetadataV3StatisticsFiles.json b/core/src/test/resources/TableMetadataV3StatisticsFiles.json new file mode 100644 index 000000000000..6768f83f7a34 --- /dev/null +++ b/core/src/test/resources/TableMetadataV3StatisticsFiles.json @@ -0,0 +1,70 @@ +{ + "format-version": 3, + "table-uuid": "9c12d441-03fe-4693-9a96-a0705ddf69c1", + "location": "s3://bucket/test/location", + "last-sequence-number": 34, + "last-updated-ms": 1602638573590, + "last-column-id": 3, + "current-schema-id": 0, + "schemas": [ + { + "type": "struct", + "schema-id": 0, + "fields": [ + { + "id": 1, + "name": "x", + "required": true, + "type": "long" + } + ] + } + ], + "default-spec-id": 0, + "partition-specs": [ + { + "spec-id": 0, + "fields": [] + } + ], + "last-partition-id": 1000, + "default-sort-order-id": 0, + "sort-orders": [ + { + "order-id": 0, + "fields": [] + } + ], + "properties": {}, + "current-snapshot-id": 3055729675574597004, + "snapshots": [ + { + "snapshot-id": 3055729675574597004, + "timestamp-ms": 1555100955770, + "sequence-number": 1, + "summary": { + "operation": "append" + }, + "manifest-list": "s3://a/b/2.avro", + "schema-id": 0 + } + ], + "statistics": [ + { + "snapshot-id": 3055729675574597004, + "statistics-path": "s3://a/b/stats.puffin", + "file-size-in-bytes": 413, + "file-footer-size-in-bytes": 42, + "blob-metadata": [ + { + "type": "ndv", + "snapshot-id": 3055729675574597004, + "sequence-number": 1, + "fields": [1] + } + ] + } + ], + "snapshot-log": [], + "metadata-log": [] +} \ No newline at end of file diff --git a/core/src/test/resources/TableMetadataV3Valid.json b/core/src/test/resources/TableMetadataV3Valid.json new file mode 100644 index 000000000000..a65b77b4c0d2 --- /dev/null +++ b/core/src/test/resources/TableMetadataV3Valid.json @@ -0,0 +1,162 @@ +{ + "format-version": 3, + "table-uuid": "9faafb13-1ce7-4603-93f8-197afc7394a9", + "location": "s3://bucket/test/location", + "last-sequence-number": 34, + "last-updated-ms": 1733710954170, + "last-column-id": 3, + "current-schema-id": 1, + "schemas": [ + { + "type": "struct", + "schema-id": 0, + "fields": [ + { + "id": 1, + "name": "x", + "required": true, + "type": "long" + } + ] + }, + { + "type": "struct", + "schema-id": 1, + "identifier-field-ids": [ + 1, + 2 + ], + "fields": [ + { + "id": 1, + "name": "x", + "required": true, + "type": "long" + }, + { + "id": 2, + "name": "y", + "required": true, + "type": "long", + "doc": "comment" + }, + { + "id": 3, + "name": "z", + "required": true, + "type": "long" + } + ] + }, + { + "type": "struct", + "schema-id": 6, + "fields": [ + { + "id": 10, + "name": "x", + "required": true, + "type": "string" + } + ] + } + ], + "default-spec-id": 5, + "partition-specs": [ + { + "spec-id": 5, + "fields": [] + } + ], + "last-partition-id": 999, + "default-sort-order-id": 3, + "sort-orders": [ + { + "order-id": 3, + "fields": [ + { + "transform": "identity", + "source-id": 2, + "direction": "asc", + "null-order": "nulls-first" + }, + { + "transform": "bucket[4]", + "source-id": 3, + "direction": "desc", + "null-order": "nulls-last" + } + ] + } + ], + "properties": { + "property": "value" + }, + "current-snapshot-id": 1733710954168, + "refs": { + "main": { + "snapshot-id": 1733710954168, + "type": "branch" + }, + "previous": { + "snapshot-id": 1733710953138, + "type": "tag" + }, + "test": { + "snapshot-id": 1733710953138, + "type": "branch" + } + }, + "snapshots": [ + { + "snapshot-id": 1733710953138, + "timestamp-ms": 1733710953138, + "manifest-list": "/var/folders/z8/cv70q7pj1l13r_j628cyyks80000gn/T/junit-9477945946492951907/manifests17078459248221091859.tmp" + }, + { + "snapshot-id": 1733710954168, + "parent-snapshot-id": 1733710953138, + "timestamp-ms": 1733710954168, + "manifest-list": "/var/folders/z8/cv70q7pj1l13r_j628cyyks80000gn/T/junit-9477945946492951907/manifests17490643917806818981.tmp", + "schema-id": 7 + } + ], + "statistics": [ + { + "snapshot-id": 11, + "statistics-path": "/some/stats/file.puffin", + "file-size-in-bytes": 100, + "file-footer-size-in-bytes": 42, + "blob-metadata": [ + { + "type": "some-stats", + "snapshot-id": 11, + "sequence-number": 2, + "fields": [ + 4 + ] + } + ] + } + ], + "partition-statistics": [ + { + "snapshot-id": 11, + "statistics-path": "/some/partition/stats/file.parquet", + "file-size-in-bytes": 42 + } + ], + "row-lineage": false, + "next-row-id": 0, + "snapshot-log": [ + { + "timestamp-ms": 1733710953138, + "snapshot-id": 1733710953138 + }, + { + "timestamp-ms": 1733710954168, + "snapshot-id": 1733710954168 + } + ], + "metadata-log": [] +} \ No newline at end of file diff --git a/core/src/test/resources/TableMetadataV3ValidMinimal.json b/core/src/test/resources/TableMetadataV3ValidMinimal.json new file mode 100644 index 000000000000..a6bfb5c7f10f --- /dev/null +++ b/core/src/test/resources/TableMetadataV3ValidMinimal.json @@ -0,0 +1,76 @@ +{ + "format-version": 3, + "table-uuid": "9faafb13-1ce7-4603-93f8-197afc7394a9", + "location": "s3://bucket/test/location", + "last-sequence-number": 34, + "last-updated-ms": 1733710954170, + "last-column-id": 3, + "current-schema-id": 7, + "schemas": [ + { + "type": "struct", + "schema-id": 7, + "fields": [ + { + "id": 1, + "name": "x", + "required": true, + "type": "long" + }, + { + "id": 2, + "name": "y", + "required": true, + "type": "long", + "doc": "comment" + }, + { + "id": 3, + "name": "z", + "required": true, + "type": "long" + } + ] + }, + { + "type": "struct", + "schema-id": 6, + "fields": [ + { + "id": 10, + "name": "x", + "required": true, + "type": "string" + } + ] + } + ], + "default-spec-id": 5, + "partition-specs": [ + { + "spec-id": 5, + "fields": [] + } + ], + "last-partition-id": 999, + "default-sort-order-id": 3, + "sort-orders": [ + { + "order-id": 3, + "fields": [ + { + "transform": "identity", + "source-id": 2, + "direction": "asc", + "null-order": "nulls-first" + }, + { + "transform": "bucket[4]", + "source-id": 3, + "direction": "desc", + "null-order": "nulls-last" + } + ] + } + ] +} \ No newline at end of file