diff --git a/api/src/main/java/org/apache/iceberg/Snapshot.java b/api/src/main/java/org/apache/iceberg/Snapshot.java index d10a6c3a49d1..2d9f34e27fe7 100644 --- a/api/src/main/java/org/apache/iceberg/Snapshot.java +++ b/api/src/main/java/org/apache/iceberg/Snapshot.java @@ -135,4 +135,13 @@ public interface Snapshot extends Serializable { default Integer schemaId() { return null; } + + /** + * Return all statistics files associated with the table. + * + * @return all statistics files associated with the table. + */ + default Iterable statisticsFiles() { + return null; + } } diff --git a/api/src/main/java/org/apache/iceberg/StatisticsFile.java b/api/src/main/java/org/apache/iceberg/StatisticsFile.java new file mode 100644 index 000000000000..34b6b15419fe --- /dev/null +++ b/api/src/main/java/org/apache/iceberg/StatisticsFile.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg; + +import java.util.Map; +import java.util.Set; + +public interface StatisticsFile { + String location(); + + long fileSizeInBytes(); + + long fileFooterSizeInBytes(); + + long sequenceNumber(); + + Map>> statistics(); +} diff --git a/core/src/main/java/org/apache/iceberg/BaseSnapshot.java b/core/src/main/java/org/apache/iceberg/BaseSnapshot.java index 7626d4d2b6c7..6a49efeda0ba 100644 --- a/core/src/main/java/org/apache/iceberg/BaseSnapshot.java +++ b/core/src/main/java/org/apache/iceberg/BaseSnapshot.java @@ -44,6 +44,7 @@ class BaseSnapshot implements Snapshot { private final String operation; private final Map summary; private final Integer schemaId; + private final List statisticsFiles; // lazily initialized private transient List allManifests = null; @@ -61,7 +62,7 @@ class BaseSnapshot implements Snapshot { String... manifestFiles) { this(io, snapshotId, null, System.currentTimeMillis(), null, null, schemaId, Lists.transform(Arrays.asList(manifestFiles), - path -> new GenericManifestFile(io.newInputFile(path), 0))); + path -> new GenericManifestFile(io.newInputFile(path), 0)), null); } BaseSnapshot(FileIO io, @@ -72,7 +73,8 @@ class BaseSnapshot implements Snapshot { String operation, Map summary, Integer schemaId, - String manifestList) { + String manifestList, + List statisticsFiles) { this.io = io; this.sequenceNumber = sequenceNumber; this.snapshotId = snapshotId; @@ -82,6 +84,7 @@ class BaseSnapshot implements Snapshot { this.summary = summary; this.schemaId = schemaId; this.manifestListLocation = manifestList; + this.statisticsFiles = statisticsFiles; } BaseSnapshot(FileIO io, @@ -91,8 +94,10 @@ class BaseSnapshot implements Snapshot { String operation, Map summary, Integer schemaId, - List dataManifests) { - this(io, INITIAL_SEQUENCE_NUMBER, snapshotId, parentId, timestampMillis, operation, summary, schemaId, null); + List dataManifests, + List statisticsFiles) { + this(io, INITIAL_SEQUENCE_NUMBER, snapshotId, parentId, timestampMillis, operation, summary, schemaId, null, + statisticsFiles); this.allManifests = dataManifests; } @@ -131,6 +136,11 @@ public Integer schemaId() { return schemaId; } + @Override + public Iterable statisticsFiles() { + return statisticsFiles; + } + private void cacheManifests() { if (io == null) { throw new IllegalStateException("Cannot cache changes: FileIO is null"); diff --git a/core/src/main/java/org/apache/iceberg/GenericStatisticsFile.java b/core/src/main/java/org/apache/iceberg/GenericStatisticsFile.java new file mode 100644 index 000000000000..f97da9500e43 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/GenericStatisticsFile.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg; + +import java.util.Map; +import java.util.Set; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; + +public class GenericStatisticsFile implements StatisticsFile { + private final String location; + private final long fileSizeInBytes; + private final long fileFooterSizeInBytes; + private final long sequenceNumber; + private final Map>> statistics; + + public GenericStatisticsFile( + String location, + long fileSizeInBytes, + long fileFooterSizeInBytes, + long sequenceNumber, + Map>> statistics) { + this.location = Preconditions.checkNotNull(location, "location is null"); + this.fileSizeInBytes = fileSizeInBytes; + this.fileFooterSizeInBytes = fileFooterSizeInBytes; + this.sequenceNumber = sequenceNumber; + Preconditions.checkNotNull(statistics, "statistics is null"); + this.statistics = statistics.entrySet().stream() + .collect(ImmutableMap.toImmutableMap( + Map.Entry::getKey, + entry -> entry.getValue().stream().map(ImmutableSet::copyOf).collect(ImmutableSet.toImmutableSet()))); + } + + @Override + public String location() { + return location; + } + + @Override + public long fileSizeInBytes() { + return fileSizeInBytes; + } + + @Override + public long fileFooterSizeInBytes() { + return fileFooterSizeInBytes; + } + + @Override + public long sequenceNumber() { + return sequenceNumber; + } + + @Override + public Map>> statistics() { + return statistics; + } +} diff --git a/core/src/main/java/org/apache/iceberg/SnapshotParser.java b/core/src/main/java/org/apache/iceberg/SnapshotParser.java index 2675e966056f..8a09c307b924 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotParser.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotParser.java @@ -26,11 +26,15 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Set; import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.io.FileIO; 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.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Streams; import org.apache.iceberg.util.JsonUtil; public class SnapshotParser { @@ -47,6 +51,10 @@ private SnapshotParser() { private static final String MANIFESTS = "manifests"; private static final String MANIFEST_LIST = "manifest-list"; private static final String SCHEMA_ID = "schema-id"; + private static final String STATISTICS = "statistics"; + private static final String LOCATION = "location"; + private static final String FILE_SIZE_IN_BYTES = "file_size_in_bytes"; + private static final String FILE_FOOTER_SIZE_IN_BYTES = "file_footer_size_in_bytes"; static void toJson(Snapshot snapshot, JsonGenerator generator) throws IOException { @@ -94,6 +102,36 @@ static void toJson(Snapshot snapshot, JsonGenerator generator) generator.writeNumberField(SCHEMA_ID, snapshot.schemaId()); } + if (snapshot.statisticsFiles() != null) { + generator.writeArrayFieldStart(STATISTICS); + for (StatisticsFile statisticsFile : snapshot.statisticsFiles()) { + toJson(statisticsFile, generator); + } + generator.writeEndArray(); + } + + generator.writeEndObject(); + } + + private static void toJson(StatisticsFile statisticsFile, JsonGenerator generator) throws IOException { + generator.writeStartObject(); + generator.writeStringField(LOCATION, statisticsFile.location()); + generator.writeNumberField(FILE_SIZE_IN_BYTES, statisticsFile.fileSizeInBytes()); + generator.writeNumberField(FILE_FOOTER_SIZE_IN_BYTES, statisticsFile.fileFooterSizeInBytes()); + generator.writeNumberField(SEQUENCE_NUMBER, statisticsFile.sequenceNumber()); + generator.writeObjectFieldStart(STATISTICS); + for (Map.Entry>> entry : statisticsFile.statistics().entrySet()) { + generator.writeArrayFieldStart(entry.getKey()); + for (Set columnIds : entry.getValue()) { + generator.writeStartArray(); + for (Integer columnId : columnIds) { + generator.writeNumber(columnId); + } + generator.writeEndArray(); + } + generator.writeEndArray(); + } + generator.writeEndObject(); generator.writeEndObject(); } @@ -147,21 +185,69 @@ static Snapshot fromJson(FileIO io, JsonNode node) { Integer schemaId = JsonUtil.getIntOrNull(SCHEMA_ID, node); + List statistics = null; + if (node.has(STATISTICS)) { + JsonNode statisticsNode = node.get(STATISTICS); + Preconditions.checkArgument(statisticsNode != null && !statisticsNode.isNull() && statisticsNode.isArray(), + "Cannot parse statistics from non-array value: %s", statisticsNode); + ImmutableList.Builder builder = ImmutableList.builderWithExpectedSize(statisticsNode.size()); + statisticsNode.elements().forEachRemaining(element -> builder.add(statisticsFileFromJson(element))); + statistics = builder.build(); + } + if (node.has(MANIFEST_LIST)) { // the manifest list is stored in a manifest list file String manifestList = JsonUtil.getString(MANIFEST_LIST, node); return new BaseSnapshot( - io, sequenceNumber, snapshotId, parentId, timestamp, operation, summary, schemaId, manifestList); - + io, sequenceNumber, snapshotId, parentId, timestamp, operation, summary, schemaId, manifestList, statistics); } else { // fall back to an embedded manifest list. pass in the manifest's InputFile so length can be // loaded lazily, if it is needed List manifests = Lists.transform(JsonUtil.getStringList(MANIFESTS, node), location -> new GenericManifestFile(io.newInputFile(location), 0)); - return new BaseSnapshot(io, snapshotId, parentId, timestamp, operation, summary, schemaId, manifests); + return new BaseSnapshot(io, snapshotId, parentId, timestamp, operation, summary, schemaId, manifests, statistics); } } + private static StatisticsFile statisticsFileFromJson(JsonNode json) { + Preconditions.checkArgument(json != null && !json.isNull() && json.isObject(), + "Cannot parse statistics from non-object value: %s", json); + + String location = JsonUtil.getString(LOCATION, json); + long fileSizeInBytes = JsonUtil.getLong(FILE_SIZE_IN_BYTES, json); + long fileFooterSizeInBytes = JsonUtil.getLong(FILE_FOOTER_SIZE_IN_BYTES, json); + long sequenceNumber = JsonUtil.getLong(SEQUENCE_NUMBER, json); + JsonNode statisticsNode = json.get(STATISTICS); + Preconditions.checkArgument(statisticsNode != null && !statisticsNode.isNull() && statisticsNode.isObject(), + "Cannot parse statistics from non-object value: %s", statisticsNode); + ImmutableMap.Builder>> statistics = ImmutableMap.builder(); + statisticsNode.fieldNames().forEachRemaining(fieldName -> { + JsonNode columnSetsNode = statisticsNode.get(fieldName); + Preconditions.checkArgument(columnSetsNode != null && !columnSetsNode.isNull() && columnSetsNode.isArray(), + "Cannot parse column sets from non-array value: %s", columnSetsNode); + Set> columnSets = Streams.stream(columnSetsNode.elements()) + .map(columnIdsNode -> { + Preconditions.checkArgument(columnIdsNode != null && !columnIdsNode.isNull() && columnIdsNode.isArray(), + "Cannot parse column IDs from non-array value: %s", columnIdsNode); + return Streams.stream(columnIdsNode.elements()) + .map(columnId -> { + Preconditions.checkArgument(columnId.isInt(), + "Cannot parse integer from non-int value: %s", columnId); + return columnId.asInt(); + }) + .collect(ImmutableSet.toImmutableSet()); + }) + .collect(ImmutableSet.toImmutableSet()); + statistics.put(fieldName, columnSets); + }); + return new GenericStatisticsFile( + location, + fileSizeInBytes, + fileFooterSizeInBytes, + sequenceNumber, + statistics.buildOrThrow()); + } + public static Snapshot fromJson(FileIO io, String json) { try { return fromJson(io, JsonUtil.mapper().readValue(json, JsonNode.class)); diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java index 08a61b39ff79..6fcd1be745de 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java @@ -37,6 +37,7 @@ import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.io.OutputFile; 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.Sets; @@ -170,6 +171,9 @@ public Snapshot apply() { Long parentSnapshotId = base.currentSnapshot() != null ? base.currentSnapshot().snapshotId() : null; long sequenceNumber = base.nextSequenceNumber(); + List statisticsFiles = + base.currentSnapshot() != null && base.currentSnapshot().statisticsFiles() != null ? + ImmutableList.copyOf(base.currentSnapshot().statisticsFiles()) : null; // run validations from the child operation validate(base); @@ -201,12 +205,12 @@ public Snapshot apply() { return new BaseSnapshot(ops.io(), sequenceNumber, snapshotId(), parentSnapshotId, System.currentTimeMillis(), operation(), summary(base), - base.currentSchemaId(), manifestList.location()); + base.currentSchemaId(), manifestList.location(), statisticsFiles); } else { return new BaseSnapshot(ops.io(), snapshotId(), parentSnapshotId, System.currentTimeMillis(), operation(), summary(base), - base.currentSchemaId(), manifests); + base.currentSchemaId(), manifests, statisticsFiles); } } diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java b/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java index 22f0b0ccdbc6..6ff5f8cdbab3 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java @@ -79,7 +79,7 @@ public void testJsonConversionWithOperation() { Snapshot expected = new BaseSnapshot(ops.io(), id, parentId, System.currentTimeMillis(), DataOperations.REPLACE, ImmutableMap.of("files-added", "4", "files-deleted", "100"), - 3, manifests); + 3, manifests, null); String json = SnapshotParser.toJson(expected); Snapshot snapshot = SnapshotParser.fromJson(ops.io(), json); @@ -122,9 +122,9 @@ public void testJsonConversionWithManifestList() throws IOException { Snapshot expected = new BaseSnapshot( ops.io(), id, 34, parentId, System.currentTimeMillis(), - null, null, 4, localInput(manifestList).location()); + null, null, 4, localInput(manifestList).location(), null); Snapshot inMemory = new BaseSnapshot( - ops.io(), id, parentId, expected.timestampMillis(), null, null, 4, manifests); + ops.io(), id, parentId, expected.timestampMillis(), null, null, 4, manifests, null); Assert.assertEquals("Files should match in memory list", inMemory.allManifests(), expected.allManifests()); diff --git a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java index 51484bdddc98..6181bb366ae7 100644 --- a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java +++ b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java @@ -89,11 +89,11 @@ public void testJsonConversion() throws Exception { long previousSnapshotId = System.currentTimeMillis() - new Random(1234).nextInt(3600); Snapshot previousSnapshot = new BaseSnapshot( ops.io(), previousSnapshotId, null, previousSnapshotId, null, null, null, ImmutableList.of( - new GenericManifestFile(localInput("file:/tmp/manfiest.1.avro"), SPEC_5.specId()))); + new GenericManifestFile(localInput("file:/tmp/manfiest.1.avro"), SPEC_5.specId())), null); long currentSnapshotId = System.currentTimeMillis(); Snapshot currentSnapshot = new BaseSnapshot( ops.io(), currentSnapshotId, previousSnapshotId, currentSnapshotId, null, null, 7, ImmutableList.of( - new GenericManifestFile(localInput("file:/tmp/manfiest.2.avro"), SPEC_5.specId()))); + new GenericManifestFile(localInput("file:/tmp/manfiest.2.avro"), SPEC_5.specId())), null); List snapshotLog = ImmutableList.builder() .add(new SnapshotLogEntry(previousSnapshot.timestampMillis(), previousSnapshot.snapshotId())) @@ -179,11 +179,11 @@ public void testBackwardCompat() throws Exception { long previousSnapshotId = System.currentTimeMillis() - new Random(1234).nextInt(3600); Snapshot previousSnapshot = new BaseSnapshot( ops.io(), previousSnapshotId, null, previousSnapshotId, null, null, null, ImmutableList.of( - new GenericManifestFile(localInput("file:/tmp/manfiest.1.avro"), spec.specId()))); + new GenericManifestFile(localInput("file:/tmp/manfiest.1.avro"), spec.specId())), null); long currentSnapshotId = System.currentTimeMillis(); Snapshot currentSnapshot = new BaseSnapshot( ops.io(), currentSnapshotId, previousSnapshotId, currentSnapshotId, null, null, null, ImmutableList.of( - new GenericManifestFile(localInput("file:/tmp/manfiest.2.avro"), spec.specId()))); + new GenericManifestFile(localInput("file:/tmp/manfiest.2.avro"), spec.specId())), null); TableMetadata expected = new TableMetadata(null, 1, null, TEST_LOCATION, 0, System.currentTimeMillis(), 3, TableMetadata.INITIAL_SCHEMA_ID, @@ -250,11 +250,11 @@ public void testInvalidMainBranch() { long previousSnapshotId = System.currentTimeMillis() - new Random(1234).nextInt(3600); Snapshot previousSnapshot = new BaseSnapshot( ops.io(), previousSnapshotId, null, previousSnapshotId, null, null, null, - ImmutableList.of(new GenericManifestFile(localInput("file:/tmp/manfiest.1.avro"), SPEC_5.specId()))); + ImmutableList.of(new GenericManifestFile(localInput("file:/tmp/manfiest.1.avro"), SPEC_5.specId())), null); long currentSnapshotId = System.currentTimeMillis(); Snapshot currentSnapshot = new BaseSnapshot( ops.io(), currentSnapshotId, previousSnapshotId, currentSnapshotId, null, null, 7, - ImmutableList.of(new GenericManifestFile(localInput("file:/tmp/manfiest.2.avro"), SPEC_5.specId()))); + ImmutableList.of(new GenericManifestFile(localInput("file:/tmp/manfiest.2.avro"), SPEC_5.specId())), null); List snapshotLog = ImmutableList.builder() .add(new SnapshotLogEntry(previousSnapshot.timestampMillis(), previousSnapshot.snapshotId())) @@ -284,7 +284,7 @@ public void testMainWithoutCurrent() { long snapshotId = System.currentTimeMillis() - new Random(1234).nextInt(3600); Snapshot snapshot = new BaseSnapshot( ops.io(), snapshotId, null, snapshotId, null, null, null, - ImmutableList.of(new GenericManifestFile(localInput("file:/tmp/manfiest.1.avro"), SPEC_5.specId()))); + ImmutableList.of(new GenericManifestFile(localInput("file:/tmp/manfiest.1.avro"), SPEC_5.specId())), null); Schema schema = new Schema(6, Types.NestedField.required(10, "x", Types.StringType.get())); @@ -376,11 +376,11 @@ public void testJsonWithPreviousMetadataLog() throws Exception { long previousSnapshotId = System.currentTimeMillis() - new Random(1234).nextInt(3600); Snapshot previousSnapshot = new BaseSnapshot( ops.io(), previousSnapshotId, null, previousSnapshotId, null, null, null, ImmutableList.of( - new GenericManifestFile(localInput("file:/tmp/manfiest.1.avro"), SPEC_5.specId()))); + new GenericManifestFile(localInput("file:/tmp/manfiest.1.avro"), SPEC_5.specId())), null); long currentSnapshotId = System.currentTimeMillis(); Snapshot currentSnapshot = new BaseSnapshot( ops.io(), currentSnapshotId, previousSnapshotId, currentSnapshotId, null, null, null, ImmutableList.of( - new GenericManifestFile(localInput("file:/tmp/manfiest.2.avro"), SPEC_5.specId()))); + new GenericManifestFile(localInput("file:/tmp/manfiest.2.avro"), SPEC_5.specId())), null); List reversedSnapshotLog = Lists.newArrayList(); long currentTimestamp = System.currentTimeMillis(); @@ -406,11 +406,11 @@ public void testAddPreviousMetadataRemoveNone() { long previousSnapshotId = System.currentTimeMillis() - new Random(1234).nextInt(3600); Snapshot previousSnapshot = new BaseSnapshot( ops.io(), previousSnapshotId, null, previousSnapshotId, null, null, null, ImmutableList.of( - new GenericManifestFile(localInput("file:/tmp/manfiest.1.avro"), SPEC_5.specId()))); + new GenericManifestFile(localInput("file:/tmp/manfiest.1.avro"), SPEC_5.specId())), null); long currentSnapshotId = System.currentTimeMillis(); Snapshot currentSnapshot = new BaseSnapshot( ops.io(), currentSnapshotId, previousSnapshotId, currentSnapshotId, null, null, null, ImmutableList.of( - new GenericManifestFile(localInput("file:/tmp/manfiest.2.avro"), SPEC_5.specId()))); + new GenericManifestFile(localInput("file:/tmp/manfiest.2.avro"), SPEC_5.specId())), null); List reversedSnapshotLog = Lists.newArrayList(); reversedSnapshotLog.add(new SnapshotLogEntry(previousSnapshot.timestampMillis(), previousSnapshotId)); @@ -448,11 +448,11 @@ public void testAddPreviousMetadataRemoveOne() { long previousSnapshotId = System.currentTimeMillis() - new Random(1234).nextInt(3600); Snapshot previousSnapshot = new BaseSnapshot( ops.io(), previousSnapshotId, null, previousSnapshotId, null, null, null, ImmutableList.of( - new GenericManifestFile(localInput("file:/tmp/manfiest.1.avro"), SPEC_5.specId()))); + new GenericManifestFile(localInput("file:/tmp/manfiest.1.avro"), SPEC_5.specId())), null); long currentSnapshotId = System.currentTimeMillis(); Snapshot currentSnapshot = new BaseSnapshot( ops.io(), currentSnapshotId, previousSnapshotId, currentSnapshotId, null, null, null, ImmutableList.of( - new GenericManifestFile(localInput("file:/tmp/manfiest.2.avro"), SPEC_5.specId()))); + new GenericManifestFile(localInput("file:/tmp/manfiest.2.avro"), SPEC_5.specId())), null); List reversedSnapshotLog = Lists.newArrayList(); reversedSnapshotLog.add(new SnapshotLogEntry(previousSnapshot.timestampMillis(), previousSnapshotId)); @@ -502,11 +502,11 @@ public void testAddPreviousMetadataRemoveMultiple() { long previousSnapshotId = System.currentTimeMillis() - new Random(1234).nextInt(3600); Snapshot previousSnapshot = new BaseSnapshot( ops.io(), previousSnapshotId, null, previousSnapshotId, null, null, null, ImmutableList.of( - new GenericManifestFile(localInput("file:/tmp/manfiest.1.avro"), SPEC_5.specId()))); + new GenericManifestFile(localInput("file:/tmp/manfiest.1.avro"), SPEC_5.specId())), null); long currentSnapshotId = System.currentTimeMillis(); Snapshot currentSnapshot = new BaseSnapshot( ops.io(), currentSnapshotId, previousSnapshotId, currentSnapshotId, null, null, null, ImmutableList.of( - new GenericManifestFile(localInput("file:/tmp/manfiest.2.avro"), SPEC_5.specId()))); + new GenericManifestFile(localInput("file:/tmp/manfiest.2.avro"), SPEC_5.specId())), null); List reversedSnapshotLog = Lists.newArrayList(); reversedSnapshotLog.add(new SnapshotLogEntry(previousSnapshot.timestampMillis(), previousSnapshotId));