diff --git a/.palantir/revapi.yml b/.palantir/revapi.yml index e523814fe3b4..e24f1272893e 100644 --- a/.palantir/revapi.yml +++ b/.palantir/revapi.yml @@ -781,6 +781,12 @@ acceptedBreaks: - code: "java.class.removed" old: "interface org.apache.iceberg.view.SQLViewRepresentation" justification: "Moving from iceberg-api to iceberg-core" + - code: "java.method.addedToInterface" + new: "method java.lang.String org.apache.iceberg.Snapshot::partitionStatsFileLocation()" + justification: "Track partition stats from Snapshot" + - code: "java.method.addedToInterface" + new: "method java.util.List org.apache.iceberg.Table::partitionStatisticsFiles()" + justification: "Track partition stats from TableMetadata" - code: "java.method.addedToInterface" new: "method org.apache.iceberg.catalog.Namespace org.apache.iceberg.view.ViewVersion::defaultNamespace()" justification: "Acceptable break due to updating View APIs and the View Spec" diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index e160aaec17c0..bb6a41234926 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -72,7 +72,6 @@ __Modules__ `iceberg-core` `iceberg-data` `iceberg-orc` -`iceberg-parquet` Changes to public interfaces and classes in the subprojects listed above require a deprecation cycle of one minor release. These projects contain common and internal code used by other projects and can evolve within a major release. diff --git a/README.md b/README.md index a0b7b00208c0..325e2110d4f9 100644 --- a/README.md +++ b/README.md @@ -65,7 +65,6 @@ Iceberg table support is organized in library modules: * `iceberg-common` contains utility classes used in other modules * `iceberg-api` contains the public Iceberg API * `iceberg-core` contains implementations of the Iceberg API and support for Avro data files, **this is what processing engines should depend on** -* `iceberg-parquet` is an optional module for working with tables backed by Parquet files * `iceberg-arrow` is an optional module for reading Parquet into Arrow memory * `iceberg-orc` is an optional module for working with tables backed by ORC files * `iceberg-hive-metastore` is an implementation of Iceberg tables backed by the Hive metastore Thrift client diff --git a/api/src/main/java/org/apache/iceberg/PartitionStatisticsFile.java b/api/src/main/java/org/apache/iceberg/PartitionStatisticsFile.java new file mode 100644 index 000000000000..e1fa8a9c8f9a --- /dev/null +++ b/api/src/main/java/org/apache/iceberg/PartitionStatisticsFile.java @@ -0,0 +1,44 @@ +/* + * 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.io.Serializable; + +/** + * Represents a partition statistics file in the table default format, that can be used to read + * table data more efficiently. + * + *

Statistics are informational. A reader can choose to ignore statistics information. Statistics + * support is not required to read the table correctly. + */ +public interface PartitionStatisticsFile extends Serializable { + /** ID of the Iceberg table's snapshot the partition statistics file is associated with. */ + long snapshotId(); + + /** + * Returns fully qualified path to the file, suitable for constructing a Hadoop Path. Never null. + */ + String path(); + + /** + * Maximum data sequence number of the Iceberg table's snapshot the partition statistics was + * computed from. + */ + long maxDataSequenceNumber(); +} diff --git a/api/src/main/java/org/apache/iceberg/Snapshot.java b/api/src/main/java/org/apache/iceberg/Snapshot.java index c035259e0e2c..1e588211bbe2 100644 --- a/api/src/main/java/org/apache/iceberg/Snapshot.java +++ b/api/src/main/java/org/apache/iceberg/Snapshot.java @@ -171,4 +171,10 @@ default Iterable removedDeleteFiles(FileIO io) { default Integer schemaId() { return null; } + + /** + * Returns the partition stats file location from snapshot summary, or null if this information is + * not available. + */ + String partitionStatsFileLocation(); } diff --git a/api/src/main/java/org/apache/iceberg/Table.java b/api/src/main/java/org/apache/iceberg/Table.java index 02db808417a4..85d1b7dee822 100644 --- a/api/src/main/java/org/apache/iceberg/Table.java +++ b/api/src/main/java/org/apache/iceberg/Table.java @@ -285,6 +285,17 @@ default UpdateStatistics updateStatistics() { "Updating statistics is not supported by " + getClass().getName()); } + /** + * Create a new {@link UpdatePartitionStatistics update partition statistics API} to add or remove + * partition statistics files in this table. + * + * @return a new {@link UpdatePartitionStatistics} + */ + default UpdatePartitionStatistics updatePartitionStatistics() { + throw new UnsupportedOperationException( + "Updating partition statistics is not supported by " + getClass().getName()); + } + /** * Create a new {@link ExpireSnapshots expire API} to manage snapshots in this table and commit. * @@ -326,6 +337,9 @@ default UpdateStatistics updateStatistics() { */ List statisticsFiles(); + /** Returns the current partition statistics files for the table. */ + List partitionStatisticsFiles(); + /** * Returns the current refs for the table * diff --git a/api/src/main/java/org/apache/iceberg/Transaction.java b/api/src/main/java/org/apache/iceberg/Transaction.java index aeec1f589d06..fd84a974013d 100644 --- a/api/src/main/java/org/apache/iceberg/Transaction.java +++ b/api/src/main/java/org/apache/iceberg/Transaction.java @@ -148,6 +148,17 @@ default UpdateStatistics updateStatistics() { "Updating statistics is not supported by " + getClass().getName()); } + /** + * Create a new {@link UpdatePartitionStatistics update partition statistics API} to add or remove + * partition statistics files in this table. + * + * @return a new {@link UpdatePartitionStatistics} + */ + default UpdatePartitionStatistics updatePartitionStatistics() { + throw new UnsupportedOperationException( + "Updating partition statistics is not supported by " + getClass().getName()); + } + /** * Create a new {@link ExpireSnapshots expire API} to manage snapshots in this table. * diff --git a/api/src/main/java/org/apache/iceberg/UpdatePartitionStatistics.java b/api/src/main/java/org/apache/iceberg/UpdatePartitionStatistics.java new file mode 100644 index 000000000000..a36400036d1c --- /dev/null +++ b/api/src/main/java/org/apache/iceberg/UpdatePartitionStatistics.java @@ -0,0 +1,40 @@ +/* + * 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.List; + +/** API for updating partition statistics files in a table. */ +public interface UpdatePartitionStatistics extends PendingUpdate> { + /** + * Set the table's partition statistics file for given snapshot, replacing the previous partition + * statistics file for the snapshot if any exists. + * + * @return this for method chaining + */ + UpdatePartitionStatistics setPartitionStatistics( + long snapshotId, PartitionStatisticsFile partitionStatisticsFile); + + /** + * Remove the table's partition statistics file for given snapshot. + * + * @return this for method chaining + */ + UpdatePartitionStatistics removePartitionStatistics(long snapshotId); +} diff --git a/api/src/test/java/org/apache/iceberg/TestHelpers.java b/api/src/test/java/org/apache/iceberg/TestHelpers.java index 153e2de7ea9a..38c65a7a0f8e 100644 --- a/api/src/test/java/org/apache/iceberg/TestHelpers.java +++ b/api/src/test/java/org/apache/iceberg/TestHelpers.java @@ -46,6 +46,7 @@ import org.apache.iceberg.expressions.UnboundPredicate; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.util.ByteBuffers; +import org.assertj.core.api.AbstractThrowableAssert; import org.assertj.core.api.Assertions; import org.objenesis.strategy.StdInstantiatorStrategy; @@ -265,6 +266,18 @@ public static void serialize(final Serializable obj, final OutputStream outputSt } } + public static void assertThrows( + String message, + Class expected, + String containedInMessage, + Runnable runnable) { + AbstractThrowableAssert check = + Assertions.assertThatThrownBy(runnable::run).as(message).isInstanceOf(expected); + if (null != containedInMessage) { + check.hasMessageContaining(containedInMessage); + } + } + public static class KryoHelpers { private KryoHelpers() {} diff --git a/build.gradle b/build.gradle index 298d690e6c9d..2bd868c4b698 100644 --- a/build.gradle +++ b/build.gradle @@ -114,7 +114,7 @@ if (file("${rootDir}/iceberg-build.properties").exists()) { } def projectVersion = getProjectVersion() -final REVAPI_PROJECTS = ["iceberg-api", "iceberg-core", "iceberg-parquet", "iceberg-orc", "iceberg-common", "iceberg-data"] +final REVAPI_PROJECTS = ["iceberg-api", "iceberg-core", "iceberg-orc", "iceberg-common", "iceberg-data"] allprojects { group = "org.apache.iceberg" @@ -349,6 +349,13 @@ project(':iceberg-core') { exclude group: 'org.slf4j', module: 'slf4j-log4j12' } + implementation(libs.parquet.avro) { + exclude group: 'org.apache.avro', module: 'avro' + // already shaded by Parquet + exclude group: 'it.unimi.dsi' + exclude group: 'org.codehaus.jackson' + } + testImplementation libs.jetty.servlet testImplementation libs.jetty.server testImplementation libs.mockserver.netty @@ -366,7 +373,6 @@ project(':iceberg-data') { implementation project(path: ':iceberg-bundled-guava', configuration: 'shadow') api project(':iceberg-api') implementation project(':iceberg-core') - compileOnly project(':iceberg-parquet') compileOnly project(':iceberg-orc') compileOnly(libs.hadoop2.common) { exclude group: 'commons-beanutils' @@ -555,7 +561,6 @@ project(':iceberg-delta-lake') { api project(':iceberg-api') implementation project(':iceberg-common') implementation project(':iceberg-core') - implementation project(':iceberg-parquet') implementation platform(libs.jackson.bom) implementation "com.fasterxml.jackson.core:jackson-databind" annotationProcessor libs.immutables.value @@ -761,32 +766,6 @@ project(':iceberg-orc') { } } -project(':iceberg-parquet') { - test { - useJUnitPlatform() - } - dependencies { - implementation project(path: ':iceberg-bundled-guava', configuration: 'shadow') - api project(':iceberg-api') - implementation project(':iceberg-core') - implementation project(':iceberg-common') - - implementation(libs.parquet.avro) { - exclude group: 'org.apache.avro', module: 'avro' - // already shaded by Parquet - exclude group: 'it.unimi.dsi' - exclude group: 'org.codehaus.jackson' - } - - compileOnly libs.avro.avro - compileOnly(libs.hadoop2.client) { - exclude group: 'org.apache.avro', module: 'avro' - } - - testImplementation project(path: ':iceberg-core', configuration: 'testArtifacts') - } -} - project(':iceberg-arrow') { test { useJUnitPlatform() @@ -795,7 +774,6 @@ project(':iceberg-arrow') { implementation project(path: ':iceberg-bundled-guava', configuration: 'shadow') api project(':iceberg-api') implementation project(':iceberg-core') - implementation project(':iceberg-parquet') implementation(libs.arrow.vector) { exclude group: 'io.netty', module: 'netty-buffer' @@ -837,7 +815,6 @@ project(':iceberg-pig') { api project(':iceberg-api') implementation project(':iceberg-common') implementation project(':iceberg-core') - implementation project(':iceberg-parquet') implementation(libs.parquet.avro) { exclude group: 'org.apache.avro', module: 'avro' @@ -905,6 +882,7 @@ project(':iceberg-nessie') { testImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') testImplementation project(path: ':iceberg-core', configuration: 'testArtifacts') + testImplementation libs.hadoop2.mapreduce.client.core // Only there to prevent "warning: unknown enum constant SchemaType.OBJECT" compile messages testCompileOnly libs.microprofile.openapi.api diff --git a/core/src/main/java/org/apache/iceberg/BaseMetadataTable.java b/core/src/main/java/org/apache/iceberg/BaseMetadataTable.java index 5f7c48e95867..ff5f73182ecd 100644 --- a/core/src/main/java/org/apache/iceberg/BaseMetadataTable.java +++ b/core/src/main/java/org/apache/iceberg/BaseMetadataTable.java @@ -194,6 +194,11 @@ public List statisticsFiles() { return ImmutableList.of(); } + @Override + public List partitionStatisticsFiles() { + return ImmutableList.of(); + } + @Override public Map refs() { return table().refs(); diff --git a/core/src/main/java/org/apache/iceberg/BaseReadOnlyTable.java b/core/src/main/java/org/apache/iceberg/BaseReadOnlyTable.java index 4acbf2a16396..243c292ebc25 100644 --- a/core/src/main/java/org/apache/iceberg/BaseReadOnlyTable.java +++ b/core/src/main/java/org/apache/iceberg/BaseReadOnlyTable.java @@ -100,6 +100,12 @@ public UpdateStatistics updateStatistics() { "Cannot update statistics of a " + descriptor + " table"); } + @Override + public UpdatePartitionStatistics updatePartitionStatistics() { + throw new UnsupportedOperationException( + "Cannot update partition statistics of a " + descriptor + " table"); + } + @Override public ExpireSnapshots expireSnapshots() { throw new UnsupportedOperationException( diff --git a/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java b/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java index 54bf3c6e44c4..2f9dd367f8a8 100644 --- a/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java +++ b/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java @@ -24,6 +24,7 @@ import static org.apache.iceberg.TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED_DEFAULT; import java.io.IOException; +import java.io.UncheckedIOException; import java.util.Arrays; import java.util.Collection; import java.util.List; @@ -36,6 +37,7 @@ import java.util.stream.Collectors; import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; @@ -202,6 +204,19 @@ public List apply(TableMetadata base, Snapshot snapshot) { return apply; } + @Override + protected String writeUpdatedPartitionStats(long snapshotCreatedTimeInMillis) { + OutputFile outputFile = newPartitionStatsFile(); + // get entries from base snapshot and use it for current snapshot. + try (CloseableIterable recordIterator = partitionStatsEntriesFromParentSnapshot()) { + writePartitionStatsEntries(recordIterator, outputFile); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + + return outputFile.location(); + } + private boolean requiresRewrite(Set currentManifests) { if (clusterByFunc == null) { // manifests are deleted and added directly so don't perform a rewrite diff --git a/core/src/main/java/org/apache/iceberg/BaseSnapshot.java b/core/src/main/java/org/apache/iceberg/BaseSnapshot.java index ac27191df594..c5f2c991c56c 100644 --- a/core/src/main/java/org/apache/iceberg/BaseSnapshot.java +++ b/core/src/main/java/org/apache/iceberg/BaseSnapshot.java @@ -288,6 +288,11 @@ private void cacheDataFileChanges(FileIO fileIO) { this.removedDataFiles = deletes.build(); } + @Override + public String partitionStatsFileLocation() { + return summary.get(SnapshotSummary.PARTITION_STATS_FILE_LOCATION); + } + @Override public boolean equals(Object o) { if (this == o) { diff --git a/core/src/main/java/org/apache/iceberg/BaseTable.java b/core/src/main/java/org/apache/iceberg/BaseTable.java index b9ed4f8d67ce..7bb9d8f8329d 100644 --- a/core/src/main/java/org/apache/iceberg/BaseTable.java +++ b/core/src/main/java/org/apache/iceberg/BaseTable.java @@ -215,6 +215,11 @@ public UpdateStatistics updateStatistics() { return new SetStatistics(ops); } + @Override + public UpdatePartitionStatistics updatePartitionStatistics() { + return new SetPartitionStatistics(ops); + } + @Override public ExpireSnapshots expireSnapshots() { return new RemoveSnapshots(ops); @@ -250,6 +255,11 @@ public List statisticsFiles() { return ops.current().statisticsFiles(); } + @Override + public List partitionStatisticsFiles() { + return ops.current().partitionStatisticsFiles(); + } + @Override public Map refs() { return ops.current().refs(); diff --git a/core/src/main/java/org/apache/iceberg/BaseTransaction.java b/core/src/main/java/org/apache/iceberg/BaseTransaction.java index 61da776f4c44..d0aae700793c 100644 --- a/core/src/main/java/org/apache/iceberg/BaseTransaction.java +++ b/core/src/main/java/org/apache/iceberg/BaseTransaction.java @@ -245,6 +245,15 @@ public UpdateStatistics updateStatistics() { return updateStatistics; } + @Override + public UpdatePartitionStatistics updatePartitionStatistics() { + checkLastOperationCommitted("UpdatePartitionStatistics"); + UpdatePartitionStatistics updatePartitionStatistics = + new SetPartitionStatistics(transactionOps); + updates.add(updatePartitionStatistics); + return updatePartitionStatistics; + } + @Override public ExpireSnapshots expireSnapshots() { checkLastOperationCommitted("ExpireSnapshots"); @@ -732,6 +741,11 @@ public UpdateStatistics updateStatistics() { return BaseTransaction.this.updateStatistics(); } + @Override + public UpdatePartitionStatistics updatePartitionStatistics() { + return BaseTransaction.this.updatePartitionStatistics(); + } + @Override public ExpireSnapshots expireSnapshots() { return BaseTransaction.this.expireSnapshots(); @@ -768,6 +782,11 @@ public List statisticsFiles() { return current.statisticsFiles(); } + @Override + public List partitionStatisticsFiles() { + return current.partitionStatisticsFiles(); + } + @Override public Map refs() { return current.refs(); diff --git a/core/src/main/java/org/apache/iceberg/CommitCallbackTransaction.java b/core/src/main/java/org/apache/iceberg/CommitCallbackTransaction.java index 9dfcf8c52867..e80f6bcdac95 100644 --- a/core/src/main/java/org/apache/iceberg/CommitCallbackTransaction.java +++ b/core/src/main/java/org/apache/iceberg/CommitCallbackTransaction.java @@ -106,6 +106,11 @@ public UpdateStatistics updateStatistics() { return wrapped.updateStatistics(); } + @Override + public UpdatePartitionStatistics updatePartitionStatistics() { + return wrapped.updatePartitionStatistics(); + } + @Override public ExpireSnapshots expireSnapshots() { return wrapped.expireSnapshots(); diff --git a/core/src/main/java/org/apache/iceberg/FastAppend.java b/core/src/main/java/org/apache/iceberg/FastAppend.java index 3079757392cd..4b79d4a2c701 100644 --- a/core/src/main/java/org/apache/iceberg/FastAppend.java +++ b/core/src/main/java/org/apache/iceberg/FastAppend.java @@ -51,6 +51,7 @@ class FastAppend extends SnapshotProducer implements AppendFiles { private final List rewrittenAppendManifests = Lists.newArrayList(); private List newManifests = null; private boolean hasNewFiles = false; + private final PartitionStatsMap partitionStatsMap; FastAppend(String tableName, TableOperations ops) { super(ops); @@ -61,6 +62,8 @@ class FastAppend extends SnapshotProducer implements AppendFiles { ops.current() .propertyAsBoolean( SNAPSHOT_ID_INHERITANCE_ENABLED, SNAPSHOT_ID_INHERITANCE_ENABLED_DEFAULT); + this.partitionStatsMap = + writePartitionStats() ? new PartitionStatsMap(ops.current().specsById()) : null; } @Override @@ -94,6 +97,10 @@ public FastAppend appendFile(DataFile file) { this.hasNewFiles = true; newFiles.add(file); summaryBuilder.addedFile(spec, file); + if (partitionStatsMap != null) { + partitionStatsMap.put(file); + } + return this; } @@ -118,10 +125,17 @@ public FastAppend appendManifest(ManifestFile manifest) { if (snapshotIdInheritanceEnabled && manifest.snapshotId() == null) { summaryBuilder.addedManifest(manifest); appendManifests.add(manifest); + if (partitionStatsMap != null) { + partitionStatsMap.put( + GenericManifestFile.copyOf(manifest).withSnapshotId(snapshotId()).build(), ops.io()); + } } else { // the manifest must be rewritten with this update's snapshot ID ManifestFile copiedManifest = copyManifest(manifest); rewrittenAppendManifests.add(copiedManifest); + if (partitionStatsMap != null) { + partitionStatsMap.put(copiedManifest, ops.io()); + } } return this; @@ -167,6 +181,16 @@ public List apply(TableMetadata base, Snapshot snapshot) { return manifests; } + @Override + protected String writeUpdatedPartitionStats(long snapshotCreatedTimeInMillis) { + PartitionsTable.PartitionMap partitionMap = partitionStatsMap.getOrCreatePartitionMap(); + updatePartitionStatsMapWithParentEntries(snapshotCreatedTimeInMillis, partitionMap); + + OutputFile outputFile = newPartitionStatsFile(); + writePartitionStatsEntries(partitionMap.all(), outputFile); + return outputFile.location(); + } + @Override public Object updateEvent() { long snapshotId = snapshotId(); diff --git a/core/src/main/java/org/apache/iceberg/FileCleanupStrategy.java b/core/src/main/java/org/apache/iceberg/FileCleanupStrategy.java index c0a113898abe..6457f574b766 100644 --- a/core/src/main/java/org/apache/iceberg/FileCleanupStrategy.java +++ b/core/src/main/java/org/apache/iceberg/FileCleanupStrategy.java @@ -21,7 +21,6 @@ import java.util.Set; import java.util.concurrent.ExecutorService; import java.util.function.Consumer; -import java.util.stream.Collectors; import org.apache.iceberg.avro.Avro; import org.apache.iceberg.exceptions.NotFoundException; import org.apache.iceberg.io.CloseableIterable; @@ -98,10 +97,15 @@ private Set statsFileLocations(TableMetadata tableMetadata) { Set statsFileLocations = Sets.newHashSet(); if (tableMetadata.statisticsFiles() != null) { - statsFileLocations = - tableMetadata.statisticsFiles().stream() - .map(StatisticsFile::path) - .collect(Collectors.toSet()); + tableMetadata.statisticsFiles().stream() + .map(StatisticsFile::path) + .forEach(statsFileLocations::add); + } + + if (tableMetadata.partitionStatisticsFiles() != null) { + tableMetadata.partitionStatisticsFiles().stream() + .map(PartitionStatisticsFile::path) + .forEach(statsFileLocations::add); } return statsFileLocations; diff --git a/core/src/main/java/org/apache/iceberg/GenericPartitionStatisticsFile.java b/core/src/main/java/org/apache/iceberg/GenericPartitionStatisticsFile.java new file mode 100644 index 000000000000..e8f815552927 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/GenericPartitionStatisticsFile.java @@ -0,0 +1,24 @@ +/* + * 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 org.immutables.value.Value; + +@Value.Immutable +public interface GenericPartitionStatisticsFile extends PartitionStatisticsFile {} diff --git a/core/src/main/java/org/apache/iceberg/IncrementalFileCleanup.java b/core/src/main/java/org/apache/iceberg/IncrementalFileCleanup.java index f0a46f1576a6..6b4923830afe 100644 --- a/core/src/main/java/org/apache/iceberg/IncrementalFileCleanup.java +++ b/core/src/main/java/org/apache/iceberg/IncrementalFileCleanup.java @@ -262,7 +262,8 @@ public void cleanFiles(TableMetadata beforeExpiration, TableMetadata afterExpira deleteFiles(manifestsToDelete, "manifest"); deleteFiles(manifestListsToDelete, "manifest list"); - if (!beforeExpiration.statisticsFiles().isEmpty()) { + if (!beforeExpiration.statisticsFiles().isEmpty() + || !beforeExpiration.partitionStatisticsFiles().isEmpty()) { Set expiredStatisticsFilesLocations = expiredStatisticsFilesLocations(beforeExpiration, afterExpiration); deleteFiles(expiredStatisticsFilesLocations, "statistics files"); diff --git a/core/src/main/java/org/apache/iceberg/ManifestFilterManager.java b/core/src/main/java/org/apache/iceberg/ManifestFilterManager.java index 9a50c0cecb13..b0356f2af191 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestFilterManager.java +++ b/core/src/main/java/org/apache/iceberg/ManifestFilterManager.java @@ -78,6 +78,7 @@ public String partition() { private boolean failMissingDeletePaths = false; private int duplicateDeleteCount = 0; private boolean caseSensitive = true; + private final PartitionStatsMap deletePartitionStatsMap; // cache filtered manifests to avoid extra work when commits fail. private final Map filteredManifests = Maps.newConcurrentMap(); @@ -89,11 +90,14 @@ public String partition() { private final Supplier workerPoolSupplier; protected ManifestFilterManager( - Map specsById, Supplier executorSupplier) { + Map specsById, + Supplier executorSupplier, + boolean writePartitionStats) { this.specsById = specsById; this.deleteFilePartitions = PartitionSet.create(specsById); this.dropPartitions = PartitionSet.create(specsById); this.workerPoolSupplier = executorSupplier; + this.deletePartitionStatsMap = writePartitionStats ? new PartitionStatsMap(specsById) : null; } protected abstract void deleteFile(String location); @@ -438,6 +442,9 @@ private ManifestFile filterManifestWithDeletedFiles( if (allRowsMatch) { writer.delete(entry); + if (entry.file() instanceof DataFile && deletePartitionStatsMap != null) { + deletePartitionStatsMap.put(entry.file()); + } CharSequenceWrapper wrapper = CharSequenceWrapper.wrap(entry.file().path()); if (deletedPaths.contains(wrapper)) { @@ -531,4 +538,8 @@ private Pair metricsEvaluator return metricsEvaluators.get(partition); } } + + public PartitionStatsMap partitionStatsMap() { + return deletePartitionStatsMap; + } } diff --git a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java index e7b2ccf69020..d8649caf6fd5 100644 --- a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java @@ -81,6 +81,7 @@ abstract class MergingSnapshotProducer extends SnapshotProducer { private final ManifestMergeManager deleteMergeManager; private final ManifestFilterManager deleteFilterManager; private final boolean snapshotIdInheritanceEnabled; + private final PartitionStatsMap partitionStatsMap; // update data private final List newDataFiles = Lists.newArrayList(); @@ -127,6 +128,8 @@ abstract class MergingSnapshotProducer extends SnapshotProducer { ops.current() .propertyAsBoolean( SNAPSHOT_ID_INHERITANCE_ENABLED, SNAPSHOT_ID_INHERITANCE_ENABLED_DEFAULT); + this.partitionStatsMap = + writePartitionStats() ? new PartitionStatsMap(ops.current().specsById()) : null; } @Override @@ -232,6 +235,9 @@ protected void add(DataFile file) { addedFilesSummary.addedFile(dataSpec(), file); hasNewDataFiles = true; newDataFiles.add(file); + if (partitionStatsMap != null) { + partitionStatsMap.put(file); + } } /** Add a delete file to the new snapshot. */ @@ -254,6 +260,9 @@ private void add(DeleteFileHolder fileHolder) { deleteFiles.add(fileHolder); addedFilesSummary.addedFile(fileSpec, fileHolder.deleteFile()); hasNewDeleteFiles = true; + if (partitionStatsMap != null) { + partitionStatsMap.put(fileHolder.deleteFile()); + } } private void setDataSpec(DataFile file) { @@ -274,10 +283,17 @@ protected void add(ManifestFile manifest) { if (snapshotIdInheritanceEnabled && manifest.snapshotId() == null) { appendedManifestsSummary.addedManifest(manifest); appendManifests.add(manifest); + if (partitionStatsMap != null) { + partitionStatsMap.put( + GenericManifestFile.copyOf(manifest).withSnapshotId(snapshotId()).build(), ops.io()); + } } else { // the manifest must be rewritten with this update's snapshot ID ManifestFile copiedManifest = copyManifest(manifest); rewrittenAppendManifests.add(copiedManifest); + if (partitionStatsMap != null) { + partitionStatsMap.put(copiedManifest, ops.io()); + } } } @@ -1026,9 +1042,42 @@ private List newDeleteFilesAsManifests() { return cachedNewDeleteManifests; } + @Override + protected String writeUpdatedPartitionStats(long snapshotCreatedTimeInMillis) { + PartitionStatsMap deletePartitionStatsMap = filterManager.partitionStatsMap(); + PartitionsTable.PartitionMap deletePartitionMap = + deletePartitionStatsMap.getOrCreatePartitionMap(); + + PartitionsTable.PartitionMap partitionMap = partitionStatsMap.getOrCreatePartitionMap(); + updatePartitionStatsMapWithParentEntries(snapshotCreatedTimeInMillis, partitionMap); + + // remove deleted entries from the current snapshot + deletePartitionMap + .all() + .forEach( + partition -> { + partition.setLastUpdatedSnapshotId(snapshotId()); + partition.setLastUpdatedAt(snapshotCreatedTimeInMillis * 1000L); + partitionMap.get(partition.partitionData()).subtract(partition); + }); + + if (partitionMap.isEmpty()) { + // Few existing testcases uses a DataFile without partition info on a partition table. + return null; + } + + OutputFile outputFile = newPartitionStatsFile(); + writePartitionStatsEntries(partitionMap.all(), outputFile); + + return outputFile.location(); + } + private class DataFileFilterManager extends ManifestFilterManager { private DataFileFilterManager() { - super(ops.current().specsById(), MergingSnapshotProducer.this::workerPool); + super( + ops.current().specsById(), + MergingSnapshotProducer.this::workerPool, + writePartitionStats()); } @Override @@ -1081,7 +1130,10 @@ protected ManifestReader newManifestReader(ManifestFile manifest) { private class DeleteFileFilterManager extends ManifestFilterManager { private DeleteFileFilterManager() { - super(ops.current().specsById(), MergingSnapshotProducer.this::workerPool); + super( + ops.current().specsById(), + MergingSnapshotProducer.this::workerPool, + writePartitionStats()); } @Override diff --git a/core/src/main/java/org/apache/iceberg/MetadataUpdate.java b/core/src/main/java/org/apache/iceberg/MetadataUpdate.java index 2cf16bca6c32..2881e0f37177 100644 --- a/core/src/main/java/org/apache/iceberg/MetadataUpdate.java +++ b/core/src/main/java/org/apache/iceberg/MetadataUpdate.java @@ -238,6 +238,47 @@ public void applyTo(TableMetadata.Builder metadataBuilder) { } } + class SetPartitionStatistics implements MetadataUpdate { + private final long snapshotId; + private final PartitionStatisticsFile partitionStatisticsFile; + + public SetPartitionStatistics( + long snapshotId, PartitionStatisticsFile partitionStatisticsFile) { + this.snapshotId = snapshotId; + this.partitionStatisticsFile = partitionStatisticsFile; + } + + public long snapshotId() { + return snapshotId; + } + + public PartitionStatisticsFile partitionStatisticsFile() { + return partitionStatisticsFile; + } + + @Override + public void applyTo(TableMetadata.Builder metadataBuilder) { + metadataBuilder.setPartitionStatistics(snapshotId, partitionStatisticsFile); + } + } + + class RemovePartitionStatistics implements MetadataUpdate { + private final long snapshotId; + + public RemovePartitionStatistics(long snapshotId) { + this.snapshotId = snapshotId; + } + + public long snapshotId() { + return snapshotId; + } + + @Override + public void applyTo(TableMetadata.Builder metadataBuilder) { + metadataBuilder.removePartitionStatistics(snapshotId); + } + } + class AddSnapshot implements MetadataUpdate { private final Snapshot snapshot; diff --git a/core/src/main/java/org/apache/iceberg/MetadataUpdateParser.java b/core/src/main/java/org/apache/iceberg/MetadataUpdateParser.java index e88f749c80f0..cffe26b098ee 100644 --- a/core/src/main/java/org/apache/iceberg/MetadataUpdateParser.java +++ b/core/src/main/java/org/apache/iceberg/MetadataUpdateParser.java @@ -57,6 +57,8 @@ private MetadataUpdateParser() {} static final String REMOVE_STATISTICS = "remove-statistics"; static final String ADD_VIEW_VERSION = "add-view-version"; static final String SET_CURRENT_VIEW_VERSION = "set-current-view-version"; + static final String SET_PARTITION_STATISTICS = "set-partition-statistics"; + static final String REMOVE_PARTITION_STATISTICS = "remove-partition-statistics"; // AssignUUID private static final String UUID = "uuid"; @@ -86,6 +88,9 @@ private MetadataUpdateParser() {} // SetStatistics private static final String STATISTICS = "statistics"; + // SetPartitionStatistics + private static final String PARTITION_STATISTICS = "partition-statistics"; + // AddSnapshot private static final String SNAPSHOT = "snapshot"; @@ -133,6 +138,8 @@ private MetadataUpdateParser() {} .put(MetadataUpdate.SetDefaultSortOrder.class, SET_DEFAULT_SORT_ORDER) .put(MetadataUpdate.SetStatistics.class, SET_STATISTICS) .put(MetadataUpdate.RemoveStatistics.class, REMOVE_STATISTICS) + .put(MetadataUpdate.SetPartitionStatistics.class, SET_PARTITION_STATISTICS) + .put(MetadataUpdate.RemovePartitionStatistics.class, REMOVE_PARTITION_STATISTICS) .put(MetadataUpdate.AddSnapshot.class, ADD_SNAPSHOT) .put(MetadataUpdate.RemoveSnapshot.class, REMOVE_SNAPSHOTS) .put(MetadataUpdate.RemoveSnapshotRef.class, REMOVE_SNAPSHOT_REF) @@ -198,6 +205,14 @@ public static void toJson(MetadataUpdate metadataUpdate, JsonGenerator generator case REMOVE_STATISTICS: writeRemoveStatistics((MetadataUpdate.RemoveStatistics) metadataUpdate, generator); break; + case SET_PARTITION_STATISTICS: + writeSetPartitionStatistics( + (MetadataUpdate.SetPartitionStatistics) metadataUpdate, generator); + break; + case REMOVE_PARTITION_STATISTICS: + writeRemovePartitionStatistics( + (MetadataUpdate.RemovePartitionStatistics) metadataUpdate, generator); + break; case ADD_SNAPSHOT: writeAddSnapshot((MetadataUpdate.AddSnapshot) metadataUpdate, generator); break; @@ -275,6 +290,10 @@ public static MetadataUpdate fromJson(JsonNode jsonNode) { return readSetStatistics(jsonNode); case REMOVE_STATISTICS: return readRemoveStatistics(jsonNode); + case SET_PARTITION_STATISTICS: + return readSetPartitionStatistics(jsonNode); + case REMOVE_PARTITION_STATISTICS: + return readRemovePartitionStatistics(jsonNode); case ADD_SNAPSHOT: return readAddSnapshot(jsonNode); case REMOVE_SNAPSHOTS: @@ -355,6 +374,18 @@ private static void writeRemoveStatistics( gen.writeNumberField(SNAPSHOT_ID, update.snapshotId()); } + private static void writeSetPartitionStatistics( + MetadataUpdate.SetPartitionStatistics update, JsonGenerator gen) throws IOException { + gen.writeNumberField(SNAPSHOT_ID, update.snapshotId()); + gen.writeFieldName(PARTITION_STATISTICS); + PartitionStatisticsFileParser.toJson(update.partitionStatisticsFile(), gen); + } + + private static void writeRemovePartitionStatistics( + MetadataUpdate.RemovePartitionStatistics update, JsonGenerator gen) throws IOException { + gen.writeNumberField(SNAPSHOT_ID, update.snapshotId()); + } + private static void writeAddSnapshot(MetadataUpdate.AddSnapshot update, JsonGenerator gen) throws IOException { gen.writeFieldName(SNAPSHOT); @@ -478,6 +509,19 @@ private static MetadataUpdate readRemoveStatistics(JsonNode node) { return new MetadataUpdate.RemoveStatistics(snapshotId); } + private static MetadataUpdate readSetPartitionStatistics(JsonNode node) { + long snapshotId = JsonUtil.getLong(SNAPSHOT_ID, node); + JsonNode partitionStatisticsFileNode = JsonUtil.get(PARTITION_STATISTICS, node); + PartitionStatisticsFile partitionStatisticsFile = + PartitionStatisticsFileParser.fromJson(partitionStatisticsFileNode); + return new MetadataUpdate.SetPartitionStatistics(snapshotId, partitionStatisticsFile); + } + + private static MetadataUpdate readRemovePartitionStatistics(JsonNode node) { + long snapshotId = JsonUtil.getLong(SNAPSHOT_ID, node); + return new MetadataUpdate.RemovePartitionStatistics(snapshotId); + } + private static MetadataUpdate readAddSnapshot(JsonNode node) { Snapshot snapshot = SnapshotParser.fromJson(JsonUtil.get(SNAPSHOT, node)); return new MetadataUpdate.AddSnapshot(snapshot); diff --git a/core/src/main/java/org/apache/iceberg/Partition.java b/core/src/main/java/org/apache/iceberg/Partition.java new file mode 100644 index 000000000000..5f4d36370e7f --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/Partition.java @@ -0,0 +1,432 @@ +/* + * 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.Objects; +import org.apache.avro.Schema; +import org.apache.avro.generic.IndexedRecord; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.types.Types; + +public class Partition implements IndexedRecord { + private PartitionData partitionData; + private int specId; + private long dataRecordCount; + private int dataFileCount; + private long dataFileSizeInBytes; + // optional fields are kept as objects instead of primitive. + private Long posDeleteRecordCount; + private Integer posDeleteFileCount; + private Long eqDeleteRecordCount; + private Integer eqDeleteFileCount; + // Commit time of snapshot that last updated this partition + private Long lastUpdatedAt; + // ID of snapshot that last updated this partition + private Long lastUpdatedSnapshotId; + + public enum Column { + PARTITION_DATA, + SPEC_ID, + DATA_RECORD_COUNT, + DATA_FILE_COUNT, + DATA_FILE_SIZE_IN_BYTES, + POSITION_DELETE_RECORD_COUNT, + POSITION_DELETE_FILE_COUNT, + EQUALITY_DELETE_RECORD_COUNT, + EQUALITY_DELETE_FILE_COUNT, + LAST_UPDATED_AT, + LAST_UPDATED_SNAPSHOT_ID + } + + public Partition() {} + + public Partition(StructLike key, Types.StructType keyType) { + this.partitionData = toPartitionData(key, keyType); + this.specId = 0; + this.dataRecordCount = 0L; + this.dataFileCount = 0; + this.dataFileSizeInBytes = 0L; + this.posDeleteRecordCount = 0L; + this.posDeleteFileCount = 0; + this.eqDeleteRecordCount = 0L; + this.eqDeleteFileCount = 0; + } + + public static Builder builder() { + return new Builder(); + } + + public PartitionData partitionData() { + return partitionData; + } + + public int specId() { + return specId; + } + + public long dataRecordCount() { + return dataRecordCount; + } + + public int dataFileCount() { + return dataFileCount; + } + + public long dataFileSizeInBytes() { + return dataFileSizeInBytes; + } + + public Long posDeleteRecordCount() { + return posDeleteRecordCount; + } + + public Integer posDeleteFileCount() { + return posDeleteFileCount; + } + + public Long eqDeleteRecordCount() { + return eqDeleteRecordCount; + } + + public Integer eqDeleteFileCount() { + return eqDeleteFileCount; + } + + public Long lastUpdatedAt() { + return lastUpdatedAt; + } + + public Long lastUpdatedSnapshotId() { + return lastUpdatedSnapshotId; + } + + public void setLastUpdatedAt(Long newLastUpdatedAt) { + this.lastUpdatedAt = newLastUpdatedAt; + } + + public void setLastUpdatedSnapshotId(Long newLastUpdatedSnapshotId) { + this.lastUpdatedSnapshotId = newLastUpdatedSnapshotId; + } + + synchronized void update(ContentFile file, Snapshot snapshot) { + if (snapshot != null) { + long snapshotCommitTime = snapshot.timestampMillis() * 1000; + if (this.lastUpdatedAt == null || snapshotCommitTime > this.lastUpdatedAt) { + this.lastUpdatedAt = snapshotCommitTime; + this.lastUpdatedSnapshotId = snapshot.snapshotId(); + } + } + + switch (file.content()) { + case DATA: + this.dataRecordCount += file.recordCount(); + this.dataFileCount += 1; + this.specId = file.specId(); + this.dataFileSizeInBytes += file.fileSizeInBytes(); + break; + case POSITION_DELETES: + this.posDeleteRecordCount = file.recordCount(); + this.posDeleteFileCount += 1; + this.specId = file.specId(); + break; + case EQUALITY_DELETES: + this.eqDeleteRecordCount = file.recordCount(); + this.eqDeleteFileCount += 1; + this.specId = file.specId(); + break; + default: + throw new UnsupportedOperationException("Unsupported file content type: " + file.content()); + } + } + + synchronized void add(Partition partition) { + this.dataRecordCount += partition.dataRecordCount; + this.dataFileCount += partition.dataFileCount; + this.dataFileSizeInBytes += partition.dataFileSizeInBytes; + this.posDeleteRecordCount += partition.posDeleteRecordCount; + this.posDeleteFileCount += partition.posDeleteFileCount; + this.eqDeleteRecordCount += partition.eqDeleteRecordCount; + this.eqDeleteFileCount += partition.eqDeleteFileCount; + updateRemaining(partition); + } + + synchronized void subtract(Partition partition) { + this.dataRecordCount -= partition.dataRecordCount; + this.dataFileCount -= partition.dataFileCount; + this.dataFileSizeInBytes -= partition.dataFileSizeInBytes; + this.posDeleteRecordCount -= partition.posDeleteRecordCount; + this.posDeleteFileCount -= partition.posDeleteFileCount; + this.eqDeleteRecordCount -= partition.eqDeleteRecordCount; + this.eqDeleteFileCount -= partition.eqDeleteFileCount; + updateRemaining(partition); + } + + private void updateRemaining(Partition partition) { + this.specId = Math.max(this.specId, partition.specId); + if (partition.lastUpdatedAt != null + && partition.lastUpdatedSnapshotId != null + && (this.lastUpdatedAt == null || this.lastUpdatedAt < partition.lastUpdatedAt)) { + this.lastUpdatedAt = partition.lastUpdatedAt; + this.lastUpdatedSnapshotId = partition.lastUpdatedSnapshotId; + } + } + + /** Needed because StructProjection is not serializable */ + private static PartitionData toPartitionData(StructLike key, Types.StructType keyType) { + PartitionData data = new PartitionData(keyType); + for (int i = 0; i < keyType.fields().size(); i++) { + Object val = key.get(i, keyType.fields().get(i).type().typeId().javaClass()); + if (val != null) { + data.set(i, val); + } + } + + return data; + } + + @Override + public void put(int i, Object v) { + switch (i) { + case 0: + this.partitionData = (PartitionData) v; + return; + case 1: + this.specId = (int) v; + return; + case 2: + this.dataRecordCount = (long) v; + return; + case 3: + this.dataFileCount = (int) v; + return; + case 4: + this.dataFileSizeInBytes = (long) v; + return; + case 5: + this.posDeleteRecordCount = (v == null) ? null : (Long) v; + return; + case 6: + this.posDeleteFileCount = (v == null) ? null : (Integer) v; + return; + case 7: + this.eqDeleteRecordCount = (v == null) ? null : (Long) v; + return; + case 8: + this.eqDeleteFileCount = (v == null) ? null : (Integer) v; + return; + case 9: + this.lastUpdatedAt = (v == null) ? null : (Long) v; + return; + case 10: + this.lastUpdatedSnapshotId = (v == null) ? null : (Long) v; + return; + default: + throw new UnsupportedOperationException("Unknown field ordinal: " + i); + } + } + + @Override + public Object get(int i) { + switch (i) { + case 0: + return partitionData; + case 1: + return specId; + case 2: + return dataRecordCount; + case 3: + return dataFileCount; + case 4: + return dataFileSizeInBytes; + case 5: + return posDeleteRecordCount; + case 6: + return posDeleteFileCount; + case 7: + return eqDeleteRecordCount; + case 8: + return eqDeleteFileCount; + case 9: + return lastUpdatedAt; + case 10: + return lastUpdatedSnapshotId; + default: + throw new UnsupportedOperationException("Unknown field ordinal: " + i); + } + } + + @Override + public Schema getSchema() { + return prepareAvroSchema(partitionData.getPartitionType()); + } + + @Override + @SuppressWarnings("checkstyle:CyclomaticComplexity") + public boolean equals(Object o) { + if (this == o) { + return true; + } else if (!(o instanceof Partition)) { + return false; + } + + Partition that = (Partition) o; + if (!(partitionData.equals(that.partitionData) + && specId == that.specId + && dataRecordCount == that.dataRecordCount + && dataFileCount == that.dataFileCount + && dataFileSizeInBytes == that.dataFileSizeInBytes)) { + return false; + } + + return Objects.equals(posDeleteRecordCount, that.posDeleteRecordCount) + && Objects.equals(posDeleteFileCount, that.posDeleteFileCount) + && Objects.equals(eqDeleteRecordCount, that.eqDeleteRecordCount) + && Objects.equals(eqDeleteFileCount, that.eqDeleteFileCount) + && Objects.equals(lastUpdatedAt, that.lastUpdatedAt) + && Objects.equals(lastUpdatedSnapshotId, that.lastUpdatedSnapshotId); + } + + @Override + public int hashCode() { + return Objects.hash( + partitionData, + specId, + dataRecordCount, + dataFileCount, + dataFileSizeInBytes, + posDeleteRecordCount, + posDeleteFileCount, + eqDeleteRecordCount, + eqDeleteFileCount, + lastUpdatedAt, + lastUpdatedSnapshotId); + } + + public static org.apache.iceberg.Schema icebergSchema(Types.StructType partitionType) { + if (partitionType.fields().isEmpty()) { + throw new IllegalArgumentException("getting schema for an unpartitioned table"); + } + + return new org.apache.iceberg.Schema( + Types.NestedField.required(1, Column.PARTITION_DATA.name(), partitionType), + Types.NestedField.required(2, Column.SPEC_ID.name(), Types.IntegerType.get()), + Types.NestedField.required(3, Column.DATA_RECORD_COUNT.name(), Types.LongType.get()), + Types.NestedField.required(4, Column.DATA_FILE_COUNT.name(), Types.IntegerType.get()), + Types.NestedField.required(5, Column.DATA_FILE_SIZE_IN_BYTES.name(), Types.LongType.get()), + Types.NestedField.optional( + 6, Column.POSITION_DELETE_RECORD_COUNT.name(), Types.LongType.get()), + Types.NestedField.optional( + 7, Column.POSITION_DELETE_FILE_COUNT.name(), Types.IntegerType.get()), + Types.NestedField.optional( + 8, Column.EQUALITY_DELETE_RECORD_COUNT.name(), Types.LongType.get()), + Types.NestedField.optional( + 9, Column.EQUALITY_DELETE_FILE_COUNT.name(), Types.IntegerType.get()), + Types.NestedField.optional(10, Column.LAST_UPDATED_AT.name(), Types.LongType.get()), + Types.NestedField.optional( + 11, Column.LAST_UPDATED_SNAPSHOT_ID.name(), Types.LongType.get())); + } + + private static Schema prepareAvroSchema(Types.StructType partitionType) { + return AvroSchemaUtil.convert(icebergSchema(partitionType), "partitionEntry"); + } + + public static class Builder { + private PartitionData partitionData; + private int specId; + private long dataRecordCount; + private int dataFileCount; + private long dataFileSizeInBytes; + private Long posDeleteRecordCount; + private Integer posDeleteFileCount; + private Long eqDeleteRecordCount; + private Integer eqDeleteFileCount; + private Long lastUpdatedAt; + private Long lastUpdatedSnapshotId; + + public Builder withPartitionData(PartitionData newPartitionData) { + this.partitionData = newPartitionData; + return this; + } + + public Builder withSpecId(int newSpecId) { + this.specId = newSpecId; + return this; + } + + public Builder withDataRecordCount(long newDataRecordCount) { + this.dataRecordCount = newDataRecordCount; + return this; + } + + public Builder withDataFileCount(int newDataFileCount) { + this.dataFileCount = newDataFileCount; + return this; + } + + public Builder withDataFileSizeInBytes(long newDataFileSizeInBytes) { + this.dataFileSizeInBytes = newDataFileSizeInBytes; + return this; + } + + public Builder withPosDeleteRecordCount(Long newPosDeleteRecordCount) { + this.posDeleteRecordCount = newPosDeleteRecordCount; + return this; + } + + public Builder withPosDeleteFileCount(Integer newPosDeleteFileCount) { + this.posDeleteFileCount = newPosDeleteFileCount; + return this; + } + + public Builder withEqDeleteRecordCount(Long newEqDeleteRecordCount) { + this.eqDeleteRecordCount = newEqDeleteRecordCount; + return this; + } + + public Builder withEqDeleteFileCount(Integer newEqDeleteFileCount) { + this.eqDeleteFileCount = newEqDeleteFileCount; + return this; + } + + public Builder withLastUpdatedAt(Long newLastUpdatedAt) { + this.lastUpdatedAt = newLastUpdatedAt; + return this; + } + + public Builder withLastUpdatedSnapshotId(Long newLastUpdatedSnapshotId) { + this.lastUpdatedSnapshotId = newLastUpdatedSnapshotId; + return this; + } + + public Partition build() { + Partition partition = new Partition(partitionData, partitionData.getPartitionType()); + partition.specId = specId; + partition.dataRecordCount = dataRecordCount; + partition.dataFileCount = dataFileCount; + partition.dataFileSizeInBytes = dataFileSizeInBytes; + partition.posDeleteRecordCount = posDeleteRecordCount; + partition.posDeleteFileCount = posDeleteFileCount; + partition.eqDeleteRecordCount = eqDeleteRecordCount; + partition.eqDeleteFileCount = eqDeleteFileCount; + partition.lastUpdatedAt = lastUpdatedAt; + partition.lastUpdatedSnapshotId = lastUpdatedSnapshotId; + return partition; + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/PartitionStatisticsFileParser.java b/core/src/main/java/org/apache/iceberg/PartitionStatisticsFileParser.java new file mode 100644 index 000000000000..fccbb3bb156d --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/PartitionStatisticsFileParser.java @@ -0,0 +1,61 @@ +/* + * 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 com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import java.io.IOException; +import org.apache.iceberg.util.JsonUtil; + +public class PartitionStatisticsFileParser { + + private static final String SNAPSHOT_ID = "snapshot-id"; + private static final String STATISTICS_PATH = "statistics-path"; + private static final String MAX_DATA_SEQUENCE_NUMBER = "max-data-sequence-number"; + + private PartitionStatisticsFileParser() {} + + public static String toJson(PartitionStatisticsFile partitionStatisticsFile) { + return toJson(partitionStatisticsFile, false); + } + + public static String toJson(PartitionStatisticsFile partitionStatisticsFile, boolean pretty) { + return JsonUtil.generate(gen -> toJson(partitionStatisticsFile, gen), pretty); + } + + public static void toJson(PartitionStatisticsFile statisticsFile, JsonGenerator generator) + throws IOException { + generator.writeStartObject(); + generator.writeNumberField(SNAPSHOT_ID, statisticsFile.snapshotId()); + generator.writeStringField(STATISTICS_PATH, statisticsFile.path()); + generator.writeNumberField(MAX_DATA_SEQUENCE_NUMBER, statisticsFile.maxDataSequenceNumber()); + generator.writeEndObject(); + } + + static PartitionStatisticsFile fromJson(JsonNode node) { + long snapshotId = JsonUtil.getLong(SNAPSHOT_ID, node); + String path = JsonUtil.getString(STATISTICS_PATH, node); + long maxDataSequenceNumber = JsonUtil.getLong(MAX_DATA_SEQUENCE_NUMBER, node); + return ImmutableGenericPartitionStatisticsFile.builder() + .snapshotId(snapshotId) + .path(path) + .maxDataSequenceNumber(maxDataSequenceNumber) + .build(); + } +} diff --git a/core/src/main/java/org/apache/iceberg/PartitionStatsMap.java b/core/src/main/java/org/apache/iceberg/PartitionStatsMap.java new file mode 100644 index 000000000000..b2606012ba62 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/PartitionStatsMap.java @@ -0,0 +1,91 @@ +/* + * 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.io.IOException; +import java.io.UncheckedIOException; +import java.util.Collections; +import java.util.Map; +import org.apache.iceberg.PartitionsTable.PartitionMap; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.PartitionUtil; + +class PartitionStatsMap { + private final Map specsById; + private PartitionMap updatedPartitionMap = null; + private final Types.StructType partitionType; + + PartitionStatsMap(Map specsById) { + this.specsById = Collections.unmodifiableMap(specsById); + this.partitionType = Partitioning.partitionType(specsById.values()); + } + + void put(ContentFile file) { + updatePartitionMap(file); + } + + void put(ManifestFile manifestFile, FileIO io) { + try (ManifestReader reader = ManifestFiles.read(manifestFile, io, specsById)) { + try (CloseableIterable> manifestEntries = reader.entries()) { + manifestEntries.forEach( + manifestEntry -> { + Preconditions.checkArgument(manifestEntry.status() == ManifestEntry.Status.ADDED); + updatePartitionMap(manifestEntry.file()); + }); + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + PartitionMap getOrCreatePartitionMap() { + if (updatedPartitionMap == null) { + updatedPartitionMap = new PartitionMap(partitionType); + } + + return updatedPartitionMap; + } + + private void updatePartitionMap(ContentFile file) { + int partitionSpecId = file.specId(); + PartitionSpec partitionSpec = specsById.get(partitionSpecId); + if (!partitionSpec.isPartitioned()) { + return; + } + + PartitionData pd = (PartitionData) file.partition(); + Types.StructType specPartitionType = partitionSpec.partitionType(); + if (!specPartitionType.equals(pd.getPartitionType())) { + // file was created with a different partition spec than table's - bail out as this is an + // invalid case + return; + } + + PartitionMap partitionMap = getOrCreatePartitionMap(); + StructLike partition = + PartitionUtil.coercePartition( + partitionType, specsById.get(file.specId()), ((PartitionData) file.partition()).copy()); + // Snapshot info will be dynamically updated during writing stats to file + // from SnapshotProducer#updatePartitionStatsMapWithParentEntries() + partitionMap.get(partition).update(file, null); + } +} diff --git a/core/src/main/java/org/apache/iceberg/Partitioning.java b/core/src/main/java/org/apache/iceberg/Partitioning.java index 7e4fcae333d8..872eb3bb09af 100644 --- a/core/src/main/java/org/apache/iceberg/Partitioning.java +++ b/core/src/main/java/org/apache/iceberg/Partitioning.java @@ -238,7 +238,10 @@ public static StructType groupingKeyType(Schema schema, Collection specs = table.specs().values(); + return partitionType(table.specs().values()); + } + + public static StructType partitionType(Collection specs) { return buildPartitionProjectionType("table partition", specs, allFieldIds(specs)); } diff --git a/core/src/main/java/org/apache/iceberg/PartitionsTable.java b/core/src/main/java/org/apache/iceberg/PartitionsTable.java index d93200c7cfca..1c0f7f1a09ca 100644 --- a/core/src/main/java/org/apache/iceberg/PartitionsTable.java +++ b/core/src/main/java/org/apache/iceberg/PartitionsTable.java @@ -129,15 +129,15 @@ private DataTask task(StaticTableScan scan) { partitions, root -> StaticDataTask.Row.of( - root.dataRecordCount, - root.dataFileCount, - root.dataFileSizeInBytes, - root.posDeleteRecordCount, - root.posDeleteFileCount, - root.eqDeleteRecordCount, - root.eqDeleteFileCount, - root.lastUpdatedAt, - root.lastUpdatedSnapshotId)); + root.dataRecordCount(), + root.dataFileCount(), + root.dataFileSizeInBytes(), + root.posDeleteRecordCount(), + root.posDeleteFileCount(), + root.eqDeleteRecordCount(), + root.eqDeleteFileCount(), + root.lastUpdatedAt(), + root.lastUpdatedSnapshotId())); } else { return StaticDataTask.of( io().newInputFile(table().operations().current().metadataFileLocation()), @@ -150,17 +150,17 @@ private DataTask task(StaticTableScan scan) { private static StaticDataTask.Row convertPartition(Partition partition) { return StaticDataTask.Row.of( - partition.partitionData, - partition.specId, - partition.dataRecordCount, - partition.dataFileCount, - partition.dataFileSizeInBytes, - partition.posDeleteRecordCount, - partition.posDeleteFileCount, - partition.eqDeleteRecordCount, - partition.eqDeleteFileCount, - partition.lastUpdatedAt, - partition.lastUpdatedSnapshotId); + partition.partitionData(), + partition.specId(), + partition.dataRecordCount(), + partition.dataFileCount(), + partition.dataFileSizeInBytes(), + partition.posDeleteRecordCount(), + partition.posDeleteFileCount(), + partition.eqDeleteRecordCount(), + partition.eqDeleteFileCount(), + partition.lastUpdatedAt(), + partition.lastUpdatedSnapshotId()); } private static Iterable partitions(Table table, StaticTableScan scan) { @@ -267,78 +267,12 @@ Partition get(StructLike key) { return partition; } - Iterable all() { - return partitions.values(); - } - } - - static class Partition { - private final PartitionData partitionData; - private int specId; - private long dataRecordCount; - private int dataFileCount; - private long dataFileSizeInBytes; - private long posDeleteRecordCount; - private int posDeleteFileCount; - private long eqDeleteRecordCount; - private int eqDeleteFileCount; - private Long lastUpdatedAt; - private Long lastUpdatedSnapshotId; - - Partition(StructLike key, Types.StructType keyType) { - this.partitionData = toPartitionData(key, keyType); - this.specId = 0; - this.dataRecordCount = 0L; - this.dataFileCount = 0; - this.dataFileSizeInBytes = 0L; - this.posDeleteRecordCount = 0L; - this.posDeleteFileCount = 0; - this.eqDeleteRecordCount = 0L; - this.eqDeleteFileCount = 0; - } - - void update(ContentFile file, Snapshot snapshot) { - if (snapshot != null) { - long snapshotCommitTime = snapshot.timestampMillis() * 1000; - if (this.lastUpdatedAt == null || snapshotCommitTime > this.lastUpdatedAt) { - this.lastUpdatedAt = snapshotCommitTime; - this.lastUpdatedSnapshotId = snapshot.snapshotId(); - } - } - - switch (file.content()) { - case DATA: - this.dataRecordCount += file.recordCount(); - this.dataFileCount += 1; - this.specId = file.specId(); - this.dataFileSizeInBytes += file.fileSizeInBytes(); - break; - case POSITION_DELETES: - this.posDeleteRecordCount = file.recordCount(); - this.posDeleteFileCount += 1; - this.specId = file.specId(); - break; - case EQUALITY_DELETES: - this.eqDeleteRecordCount = file.recordCount(); - this.eqDeleteFileCount += 1; - this.specId = file.specId(); - break; - default: - throw new UnsupportedOperationException( - "Unsupported file content type: " + file.content()); - } + boolean isEmpty() { + return partitions.isEmpty(); } - /** Needed because StructProjection is not serializable */ - private PartitionData toPartitionData(StructLike key, Types.StructType keyType) { - PartitionData data = new PartitionData(keyType); - for (int i = 0; i < keyType.fields().size(); i++) { - Object val = key.get(i, keyType.fields().get(i).type().typeId().javaClass()); - if (val != null) { - data.set(i, val); - } - } - return data; + Iterable all() { + return partitions.values(); } } } diff --git a/core/src/main/java/org/apache/iceberg/ReachableFileCleanup.java b/core/src/main/java/org/apache/iceberg/ReachableFileCleanup.java index 6b15e9147dc0..9db2ef4d90c6 100644 --- a/core/src/main/java/org/apache/iceberg/ReachableFileCleanup.java +++ b/core/src/main/java/org/apache/iceberg/ReachableFileCleanup.java @@ -82,7 +82,8 @@ public void cleanFiles(TableMetadata beforeExpiration, TableMetadata afterExpira deleteFiles(manifestListsToDelete, "manifest list"); - if (!beforeExpiration.statisticsFiles().isEmpty()) { + if (!beforeExpiration.statisticsFiles().isEmpty() + || !beforeExpiration.partitionStatisticsFiles().isEmpty()) { deleteFiles( expiredStatisticsFilesLocations(beforeExpiration, afterExpiration), "statistics files"); } diff --git a/core/src/main/java/org/apache/iceberg/ReachableFileUtil.java b/core/src/main/java/org/apache/iceberg/ReachableFileUtil.java index 60a1f5290ae6..7adaf875e629 100644 --- a/core/src/main/java/org/apache/iceberg/ReachableFileUtil.java +++ b/core/src/main/java/org/apache/iceberg/ReachableFileUtil.java @@ -137,7 +137,9 @@ public static List manifestListLocations(Table table, Set snapshot * * @param table table for which statistics files needs to be listed * @return the location of statistics files + * @deprecated use the {@code allStatisticsFilesLocations(table)} instead. */ + @Deprecated public static List statisticsFilesLocations(Table table) { return statisticsFilesLocations(table, statisticsFile -> true); } @@ -148,7 +150,9 @@ public static List statisticsFilesLocations(Table table) { * @param table table for which statistics files needs to be listed * @param predicate predicate for filtering the statistics files * @return the location of statistics files + * @deprecated use the {@code allStatisticsFilesLocations(table, snapshotIds)} instead. */ + @Deprecated public static List statisticsFilesLocations( Table table, Predicate predicate) { return table.statisticsFiles().stream() @@ -156,4 +160,52 @@ public static List statisticsFilesLocations( .map(StatisticsFile::path) .collect(Collectors.toList()); } + + /** + * Returns locations of statistics files in a table. + * + * @param table table for which statistics files needs to be listed + * @return the location of statistics files + */ + public static List allStatisticsFilesLocations(Table table) { + return allStatisticsFilesLocations(table, null); + } + + /** + * Returns locations of statistics files for a table for given snapshots. + * + * @param table table for which statistics files needs to be listed + * @param snapshotIds snapshot IDs for which statistics files needs to be listed + * @return the location of statistics files + */ + public static List allStatisticsFilesLocations(Table table, Set snapshotIds) { + List statsFileLocations = Lists.newArrayList(); + + Predicate statisticsFilePredicate; + Predicate partitionStatisticsFilePredicate; + if (snapshotIds == null) { + statisticsFilePredicate = statisticsFile -> true; + partitionStatisticsFilePredicate = partitionStatisticsFile -> true; + } else { + statisticsFilePredicate = statisticsFile -> snapshotIds.contains(statisticsFile.snapshotId()); + partitionStatisticsFilePredicate = + partitionStatisticsFile -> snapshotIds.contains(partitionStatisticsFile.snapshotId()); + } + + if (table.statisticsFiles() != null) { + table.statisticsFiles().stream() + .filter(statisticsFilePredicate) + .map(StatisticsFile::path) + .forEach(statsFileLocations::add); + } + + if (table.partitionStatisticsFiles() != null) { + table.partitionStatisticsFiles().stream() + .filter(partitionStatisticsFilePredicate) + .map(PartitionStatisticsFile::path) + .forEach(statsFileLocations::add); + } + + return statsFileLocations; + } } diff --git a/core/src/main/java/org/apache/iceberg/SerializableTable.java b/core/src/main/java/org/apache/iceberg/SerializableTable.java index cf8e1b3fbaa7..8bac814c003b 100644 --- a/core/src/main/java/org/apache/iceberg/SerializableTable.java +++ b/core/src/main/java/org/apache/iceberg/SerializableTable.java @@ -242,6 +242,11 @@ public List statisticsFiles() { return lazyTable().statisticsFiles(); } + @Override + public List partitionStatisticsFiles() { + return lazyTable().partitionStatisticsFiles(); + } + @Override public Map refs() { return refs; @@ -347,6 +352,11 @@ public UpdateStatistics updateStatistics() { throw new UnsupportedOperationException(errorMsg("updateStatistics")); } + @Override + public UpdatePartitionStatistics updatePartitionStatistics() { + throw new UnsupportedOperationException(errorMsg("updatePartitionStatistics")); + } + @Override public ExpireSnapshots expireSnapshots() { throw new UnsupportedOperationException(errorMsg("expireSnapshots")); diff --git a/core/src/main/java/org/apache/iceberg/SetPartitionStatistics.java b/core/src/main/java/org/apache/iceberg/SetPartitionStatistics.java new file mode 100644 index 000000000000..09b41a9244a8 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/SetPartitionStatistics.java @@ -0,0 +1,74 @@ +/* + * 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.List; +import java.util.Map; +import java.util.Optional; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; + +public class SetPartitionStatistics implements UpdatePartitionStatistics { + private final TableOperations ops; + private final Map> partitionStatisticsToSet = + Maps.newHashMap(); + + public SetPartitionStatistics(TableOperations ops) { + this.ops = ops; + } + + @Override + public UpdatePartitionStatistics setPartitionStatistics( + long snapshotId, PartitionStatisticsFile partitionStatisticsFile) { + Preconditions.checkArgument(snapshotId == partitionStatisticsFile.snapshotId()); + partitionStatisticsToSet.put(snapshotId, Optional.of(partitionStatisticsFile)); + return this; + } + + @Override + public UpdatePartitionStatistics removePartitionStatistics(long snapshotId) { + partitionStatisticsToSet.put(snapshotId, Optional.empty()); + return this; + } + + @Override + public List apply() { + return internalApply(ops.current()).partitionStatisticsFiles(); + } + + @Override + public void commit() { + TableMetadata base = ops.current(); + TableMetadata newMetadata = internalApply(base); + ops.commit(base, newMetadata); + } + + private TableMetadata internalApply(TableMetadata base) { + TableMetadata.Builder builder = TableMetadata.buildFrom(base); + partitionStatisticsToSet.forEach( + (snapshotId, statistics) -> { + if (statistics.isPresent()) { + builder.setPartitionStatistics(snapshotId, statistics.get()); + } else { + builder.removePartitionStatistics(snapshotId); + } + }); + return builder.build(); + } +} diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java index 5a6a01ea062d..4bb82d7fc213 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java @@ -28,10 +28,13 @@ import static org.apache.iceberg.TableProperties.COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT; import static org.apache.iceberg.TableProperties.MANIFEST_TARGET_SIZE_BYTES; import static org.apache.iceberg.TableProperties.MANIFEST_TARGET_SIZE_BYTES_DEFAULT; +import static org.apache.iceberg.TableProperties.PARTITION_STATS_ENABLED; +import static org.apache.iceberg.TableProperties.PARTITION_STATS_ENABLED_DEFAULT; import com.github.benmanes.caffeine.cache.Caffeine; import com.github.benmanes.caffeine.cache.LoadingCache; import java.io.IOException; +import java.io.UncheckedIOException; import java.util.Arrays; import java.util.List; import java.util.Map; @@ -46,6 +49,7 @@ import org.apache.iceberg.exceptions.CommitFailedException; import org.apache.iceberg.exceptions.CommitStateUnknownException; import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.metrics.CommitMetrics; import org.apache.iceberg.metrics.CommitMetricsResult; @@ -54,11 +58,14 @@ import org.apache.iceberg.metrics.LoggingMetricsReporter; import org.apache.iceberg.metrics.MetricsReporter; import org.apache.iceberg.metrics.Timer.Timed; +import org.apache.iceberg.partition.stats.PartitionStatsUtil; 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.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.util.Exceptions; +import org.apache.iceberg.util.PropertyUtil; import org.apache.iceberg.util.SnapshotUtil; import org.apache.iceberg.util.Tasks; import org.apache.iceberg.util.ThreadPools; @@ -97,6 +104,7 @@ public void accept(String file) { private ExecutorService workerPool = ThreadPools.getWorkerPool(); private String targetBranch = SnapshotRef.MAIN_BRANCH; private CommitMetrics commitMetrics; + private final boolean writePartitionStats; protected SnapshotProducer(TableOperations ops) { this.ops = ops; @@ -113,6 +121,10 @@ protected SnapshotProducer(TableOperations ops) { this.targetManifestSizeBytes = ops.current() .propertyAsLong(MANIFEST_TARGET_SIZE_BYTES, MANIFEST_TARGET_SIZE_BYTES_DEFAULT); + writePartitionStats = + ops.current().spec().isPartitioned() + && PropertyUtil.propertyAsBoolean( + base.properties(), PARTITION_STATS_ENABLED, PARTITION_STATS_ENABLED_DEFAULT); } protected abstract ThisT self(); @@ -211,6 +223,61 @@ protected void validate(TableMetadata currentMetadata, Snapshot snapshot) {} */ protected abstract List apply(TableMetadata metadataToUpdate, Snapshot snapshot); + /** + * Update stats for each partition along with previous values and write new stats file for the + * current snapshot. + * + * @return the stats file location + */ + protected abstract String writeUpdatedPartitionStats(long snapshotCreatedTimeInMillis); + + protected CloseableIterable partitionStatsEntriesFromParentSnapshot() { + if (base.currentSnapshot() == null) { + return CloseableIterable.empty(); + } + + String oldPartitionStatsFileLocation = base.currentSnapshot().partitionStatsFileLocation(); + if (oldPartitionStatsFileLocation == null) { + return CloseableIterable.empty(); + } + + return PartitionStatsUtil.readPartitionStatsFile( + Partition.icebergSchema(Partitioning.partitionType(base.specsById().values())), + ops.io().newInputFile(oldPartitionStatsFileLocation)); + } + + protected void writePartitionStatsEntries( + Iterable partitionStatsEntries, OutputFile file) { + PartitionStatsUtil.writePartitionStatsFile( + partitionStatsEntries.iterator(), file, ops.current().specsById().values()); + } + + protected void updatePartitionStatsMapWithParentEntries( + long snapshotCreatedTimeInMillis, PartitionsTable.PartitionMap partitionMap) { + // update the snapshot ID and snapshot created time for the current snapshot + // as this information was not available at the place of partition creation. + // see PartitionStatsMap#updatePartitionMap() + partitionMap + .all() + .forEach( + partition -> { + partition.setLastUpdatedAt(snapshotCreatedTimeInMillis * 1000L); + partition.setLastUpdatedSnapshotId(snapshotId()); + }); + + // get entries from base snapshot and add to current snapshot's entries + try (CloseableIterable recordIterator = partitionStatsEntriesFromParentSnapshot()) { + recordIterator.forEach( + partition -> partitionMap.get(partition.partitionData()).add(partition)); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + public boolean writePartitionStats() { + return writePartitionStats; + } + @Override public Snapshot apply() { refresh(); @@ -249,13 +316,25 @@ public Snapshot apply() { throw new RuntimeIOException(e, "Failed to write manifest list file"); } + long snapshotCreatedTimeInMillis = System.currentTimeMillis(); + + Map additionalSummary = Maps.newHashMap(); + // Update Partition Stats for new Snapshot + if (writePartitionStats) { + String partitionStatsLocation = writeUpdatedPartitionStats(snapshotCreatedTimeInMillis); + if (partitionStatsLocation != null) { + additionalSummary.put( + SnapshotSummary.PARTITION_STATS_FILE_LOCATION, partitionStatsLocation); + } + } + return new BaseSnapshot( sequenceNumber, snapshotId(), parentSnapshotId, - System.currentTimeMillis(), + snapshotCreatedTimeInMillis, operation(), - summary(base), + summary(base, additionalSummary), base.currentSchemaId(), manifestList.location()); } @@ -263,7 +342,8 @@ public Snapshot apply() { protected abstract Map summary(); /** Returns the snapshot summary from the implementation and updates totals. */ - private Map summary(TableMetadata previous) { + private Map summary( + TableMetadata previous, Map additionalSummary) { Map summary = summary(); if (summary == null) { @@ -340,6 +420,10 @@ private Map summary(TableMetadata previous) { SnapshotSummary.ADDED_EQ_DELETES_PROP, SnapshotSummary.REMOVED_EQ_DELETES_PROP); + if (!additionalSummary.isEmpty()) { + builder.putAll(additionalSummary); + } + return builder.build(); } @@ -381,6 +465,16 @@ public void commit() { update.setBranchSnapshot(newSnapshot, targetBranch); } + if (newSnapshot.partitionStatsFileLocation() != null) { + PartitionStatisticsFile partitionStatisticsFile = + ImmutableGenericPartitionStatisticsFile.builder() + .snapshotId(newSnapshotId.get()) + .path(newSnapshot.partitionStatsFileLocation()) + .maxDataSequenceNumber(newSnapshot.sequenceNumber()) + .build(); + update.setPartitionStatistics(newSnapshotId.get(), partitionStatisticsFile); + } + TableMetadata updated = update.build(); if (updated.changes().isEmpty()) { // do not commit if the metadata has not changed. for example, this may happen @@ -490,6 +584,14 @@ protected OutputFile newManifestOutput() { FileFormat.AVRO.addExtension(commitUUID + "-m" + manifestCount.getAndIncrement()))); } + protected OutputFile newPartitionStatsFile() { + return ops.io() + .newOutputFile( + ops.metadataFileLocation( + FileFormat.PARQUET.addExtension( + String.format("partition-stats-%d", snapshotId())))); + } + protected ManifestWriter newManifestWriter(PartitionSpec spec) { return ManifestFiles.write( ops.current().formatVersion(), spec, newManifestOutput(), snapshotId()); diff --git a/core/src/main/java/org/apache/iceberg/SnapshotSummary.java b/core/src/main/java/org/apache/iceberg/SnapshotSummary.java index e26365b56242..4b28cbbf707e 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotSummary.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotSummary.java @@ -58,6 +58,7 @@ public class SnapshotSummary { public static final String SOURCE_SNAPSHOT_ID_PROP = "source-snapshot-id"; public static final String REPLACE_PARTITIONS_PROP = "replace-partitions"; public static final String EXTRA_METADATA_PREFIX = "snapshot-property."; + public static final String PARTITION_STATS_FILE_LOCATION = "partition-stats-file-location"; public static final MapJoiner MAP_JOINER = Joiner.on(",").withKeyValueSeparator("="); diff --git a/core/src/main/java/org/apache/iceberg/TableMetadata.java b/core/src/main/java/org/apache/iceberg/TableMetadata.java index a6f1d428f41a..d6caa117b44f 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadata.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadata.java @@ -242,6 +242,7 @@ public String toString() { private final List snapshotLog; private final List previousFiles; private final List statisticsFiles; + private final List partitionStatisticsFiles; private final List changes; private SerializableSupplier> snapshotsSupplier; private volatile List snapshots; @@ -273,6 +274,7 @@ public String toString() { List previousFiles, Map refs, List statisticsFiles, + List partitionStatisticsFiles, List changes) { Preconditions.checkArgument( specs != null && !specs.isEmpty(), "Partition specs cannot be null or empty"); @@ -323,6 +325,7 @@ public String toString() { this.sortOrdersById = indexSortOrders(sortOrders); this.refs = validateRefs(currentSnapshotId, refs, snapshotsById); this.statisticsFiles = ImmutableList.copyOf(statisticsFiles); + this.partitionStatisticsFiles = ImmutableList.copyOf(partitionStatisticsFiles); HistoryEntry last = null; for (HistoryEntry logEntry : snapshotLog) { @@ -522,6 +525,10 @@ public List statisticsFiles() { return statisticsFiles; } + public List partitionStatisticsFiles() { + return partitionStatisticsFiles; + } + public List snapshotLog() { return snapshotLog; } @@ -863,6 +870,7 @@ public static class Builder { private SerializableSupplier> snapshotsSupplier; private final Map refs; private final Map> statisticsFiles; + private final Map> partitionStatisticsFiles; // change tracking private final List changes; @@ -900,6 +908,7 @@ private Builder() { this.previousFiles = Lists.newArrayList(); this.refs = Maps.newHashMap(); this.statisticsFiles = Maps.newHashMap(); + this.partitionStatisticsFiles = Maps.newHashMap(); this.snapshotsById = Maps.newHashMap(); this.schemasById = Maps.newHashMap(); this.specsById = Maps.newHashMap(); @@ -933,6 +942,9 @@ private Builder(TableMetadata base) { this.refs = Maps.newHashMap(base.refs); this.statisticsFiles = base.statisticsFiles.stream().collect(Collectors.groupingBy(StatisticsFile::snapshotId)); + this.partitionStatisticsFiles = + base.partitionStatisticsFiles.stream() + .collect(Collectors.groupingBy(PartitionStatisticsFile::snapshotId)); this.snapshotsById = Maps.newHashMap(base.snapshotsById); this.schemasById = Maps.newHashMap(base.schemasById); @@ -1273,6 +1285,29 @@ public Builder suppressHistoricalSnapshots() { return this; } + public Builder setPartitionStatistics( + long snapshotId, PartitionStatisticsFile partitionStatisticsFile) { + Preconditions.checkNotNull(partitionStatisticsFile, "partition statistics file is null"); + Preconditions.checkArgument( + snapshotId == partitionStatisticsFile.snapshotId(), + "snapshotId does not match: %s vs %s", + snapshotId, + partitionStatisticsFile.snapshotId()); + partitionStatisticsFiles.put(snapshotId, ImmutableList.of(partitionStatisticsFile)); + changes.add(new MetadataUpdate.SetPartitionStatistics(snapshotId, partitionStatisticsFile)); + return this; + } + + public Builder removePartitionStatistics(long snapshotId) { + Preconditions.checkNotNull(snapshotId, "snapshotId is null"); + if (partitionStatisticsFiles.remove(snapshotId) == null) { + return this; + } + + changes.add(new MetadataUpdate.RemovePartitionStatistics(snapshotId)); + return this; + } + public Builder removeSnapshots(List snapshotsToRemove) { Set idsToRemove = snapshotsToRemove.stream().map(Snapshot::snapshotId).collect(Collectors.toSet()); @@ -1303,6 +1338,7 @@ private Builder rewriteSnapshotsInternal(Collection idsToRemove, boolean s changes.add(new MetadataUpdate.RemoveSnapshot(snapshotId)); } removeStatistics(snapshotId); + removePartitionStatistics(snapshotId); } else { retainedSnapshots.add(snapshot); } @@ -1428,6 +1464,9 @@ public TableMetadata build() { ImmutableList.copyOf(metadataHistory), ImmutableMap.copyOf(refs), statisticsFiles.values().stream().flatMap(List::stream).collect(Collectors.toList()), + partitionStatisticsFiles.values().stream() + .flatMap(List::stream) + .collect(Collectors.toList()), discardChanges ? ImmutableList.of() : ImmutableList.copyOf(changes)); } diff --git a/core/src/main/java/org/apache/iceberg/TableMetadataParser.java b/core/src/main/java/org/apache/iceberg/TableMetadataParser.java index 473f8497cbb4..0212738dea9c 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadataParser.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadataParser.java @@ -109,6 +109,7 @@ private TableMetadataParser() {} static final String METADATA_FILE = "metadata-file"; static final String METADATA_LOG = "metadata-log"; static final String STATISTICS = "statistics"; + static final String PARTITION_STATISTICS = "partition-statistics"; public static void overwrite(TableMetadata metadata, OutputFile outputFile) { internalWrite(metadata, outputFile, true); @@ -231,6 +232,12 @@ public static void toJson(TableMetadata metadata, JsonGenerator generator) throw } generator.writeEndArray(); + generator.writeArrayFieldStart(PARTITION_STATISTICS); + for (PartitionStatisticsFile partitionStatisticsFile : metadata.partitionStatisticsFiles()) { + PartitionStatisticsFileParser.toJson(partitionStatisticsFile, generator); + } + generator.writeEndArray(); + generator.writeArrayFieldStart(SNAPSHOT_LOG); for (HistoryEntry logEntry : metadata.snapshotLog()) { generator.writeStartObject(); @@ -481,6 +488,13 @@ public static TableMetadata fromJson(String metadataLocation, JsonNode node) { statisticsFiles = ImmutableList.of(); } + List partitionStatisticsFiles; + if (node.has(PARTITION_STATISTICS)) { + partitionStatisticsFiles = partitionStatisticsFilesFromJson(node.get(PARTITION_STATISTICS)); + } else { + partitionStatisticsFiles = ImmutableList.of(); + } + ImmutableList.Builder entries = ImmutableList.builder(); if (node.has(SNAPSHOT_LOG)) { Iterator logIterator = node.get(SNAPSHOT_LOG).elements(); @@ -528,6 +542,7 @@ public static TableMetadata fromJson(String metadataLocation, JsonNode node) { metadataEntries.build(), refs, statisticsFiles, + partitionStatisticsFiles, ImmutableList.of() /* no changes from the file */); } @@ -561,4 +576,21 @@ private static List statisticsFilesFromJson(JsonNode statisticsF return statisticsFilesBuilder.build(); } + + private static List partitionStatisticsFilesFromJson( + JsonNode partitionStatisticsFilesList) { + Preconditions.checkArgument( + partitionStatisticsFilesList.isArray(), + "Cannot parse partition statistics files from non-array: %s", + partitionStatisticsFilesList); + + ImmutableList.Builder partitionStatisticsFileBuilder = + ImmutableList.builder(); + for (JsonNode partitionStatisticsFile : partitionStatisticsFilesList) { + partitionStatisticsFileBuilder.add( + PartitionStatisticsFileParser.fromJson(partitionStatisticsFile)); + } + + return partitionStatisticsFileBuilder.build(); + } } diff --git a/core/src/main/java/org/apache/iceberg/TableProperties.java b/core/src/main/java/org/apache/iceberg/TableProperties.java index a9116bc57f83..9a9f63ebfe5b 100644 --- a/core/src/main/java/org/apache/iceberg/TableProperties.java +++ b/core/src/main/java/org/apache/iceberg/TableProperties.java @@ -365,4 +365,7 @@ private TableProperties() {} public static final String UPSERT_ENABLED = "write.upsert.enabled"; public static final boolean UPSERT_ENABLED_DEFAULT = false; + + public static final String PARTITION_STATS_ENABLED = "write.partition.statistics"; + public static final boolean PARTITION_STATS_ENABLED_DEFAULT = true; } diff --git a/parquet/src/main/java/org/apache/iceberg/data/parquet/BaseParquetReaders.java b/core/src/main/java/org/apache/iceberg/data/parquet/BaseParquetReaders.java similarity index 100% rename from parquet/src/main/java/org/apache/iceberg/data/parquet/BaseParquetReaders.java rename to core/src/main/java/org/apache/iceberg/data/parquet/BaseParquetReaders.java diff --git a/parquet/src/main/java/org/apache/iceberg/data/parquet/BaseParquetWriter.java b/core/src/main/java/org/apache/iceberg/data/parquet/BaseParquetWriter.java similarity index 100% rename from parquet/src/main/java/org/apache/iceberg/data/parquet/BaseParquetWriter.java rename to core/src/main/java/org/apache/iceberg/data/parquet/BaseParquetWriter.java diff --git a/parquet/src/main/java/org/apache/iceberg/data/parquet/GenericParquetReaders.java b/core/src/main/java/org/apache/iceberg/data/parquet/GenericParquetReaders.java similarity index 100% rename from parquet/src/main/java/org/apache/iceberg/data/parquet/GenericParquetReaders.java rename to core/src/main/java/org/apache/iceberg/data/parquet/GenericParquetReaders.java diff --git a/parquet/src/main/java/org/apache/iceberg/data/parquet/GenericParquetWriter.java b/core/src/main/java/org/apache/iceberg/data/parquet/GenericParquetWriter.java similarity index 100% rename from parquet/src/main/java/org/apache/iceberg/data/parquet/GenericParquetWriter.java rename to core/src/main/java/org/apache/iceberg/data/parquet/GenericParquetWriter.java diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ApplyNameMapping.java b/core/src/main/java/org/apache/iceberg/parquet/ApplyNameMapping.java similarity index 100% rename from parquet/src/main/java/org/apache/iceberg/parquet/ApplyNameMapping.java rename to core/src/main/java/org/apache/iceberg/parquet/ApplyNameMapping.java diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/BaseColumnIterator.java b/core/src/main/java/org/apache/iceberg/parquet/BaseColumnIterator.java similarity index 100% rename from parquet/src/main/java/org/apache/iceberg/parquet/BaseColumnIterator.java rename to core/src/main/java/org/apache/iceberg/parquet/BaseColumnIterator.java diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/BasePageIterator.java b/core/src/main/java/org/apache/iceberg/parquet/BasePageIterator.java similarity index 100% rename from parquet/src/main/java/org/apache/iceberg/parquet/BasePageIterator.java rename to core/src/main/java/org/apache/iceberg/parquet/BasePageIterator.java diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ColumnIterator.java b/core/src/main/java/org/apache/iceberg/parquet/ColumnIterator.java similarity index 100% rename from parquet/src/main/java/org/apache/iceberg/parquet/ColumnIterator.java rename to core/src/main/java/org/apache/iceberg/parquet/ColumnIterator.java diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ColumnWriter.java b/core/src/main/java/org/apache/iceberg/parquet/ColumnWriter.java similarity index 100% rename from parquet/src/main/java/org/apache/iceberg/parquet/ColumnWriter.java rename to core/src/main/java/org/apache/iceberg/parquet/ColumnWriter.java diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/MessageTypeToType.java b/core/src/main/java/org/apache/iceberg/parquet/MessageTypeToType.java similarity index 100% rename from parquet/src/main/java/org/apache/iceberg/parquet/MessageTypeToType.java rename to core/src/main/java/org/apache/iceberg/parquet/MessageTypeToType.java diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/PageIterator.java b/core/src/main/java/org/apache/iceberg/parquet/PageIterator.java similarity index 100% rename from parquet/src/main/java/org/apache/iceberg/parquet/PageIterator.java rename to core/src/main/java/org/apache/iceberg/parquet/PageIterator.java diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java b/core/src/main/java/org/apache/iceberg/parquet/Parquet.java similarity index 100% rename from parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java rename to core/src/main/java/org/apache/iceberg/parquet/Parquet.java diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetAvro.java b/core/src/main/java/org/apache/iceberg/parquet/ParquetAvro.java similarity index 100% rename from parquet/src/main/java/org/apache/iceberg/parquet/ParquetAvro.java rename to core/src/main/java/org/apache/iceberg/parquet/ParquetAvro.java diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetAvroReader.java b/core/src/main/java/org/apache/iceberg/parquet/ParquetAvroReader.java similarity index 100% rename from parquet/src/main/java/org/apache/iceberg/parquet/ParquetAvroReader.java rename to core/src/main/java/org/apache/iceberg/parquet/ParquetAvroReader.java diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetAvroValueReaders.java b/core/src/main/java/org/apache/iceberg/parquet/ParquetAvroValueReaders.java similarity index 100% rename from parquet/src/main/java/org/apache/iceberg/parquet/ParquetAvroValueReaders.java rename to core/src/main/java/org/apache/iceberg/parquet/ParquetAvroValueReaders.java diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetAvroWriter.java b/core/src/main/java/org/apache/iceberg/parquet/ParquetAvroWriter.java similarity index 100% rename from parquet/src/main/java/org/apache/iceberg/parquet/ParquetAvroWriter.java rename to core/src/main/java/org/apache/iceberg/parquet/ParquetAvroWriter.java diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetBloomRowGroupFilter.java b/core/src/main/java/org/apache/iceberg/parquet/ParquetBloomRowGroupFilter.java similarity index 100% rename from parquet/src/main/java/org/apache/iceberg/parquet/ParquetBloomRowGroupFilter.java rename to core/src/main/java/org/apache/iceberg/parquet/ParquetBloomRowGroupFilter.java diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetCodecFactory.java b/core/src/main/java/org/apache/iceberg/parquet/ParquetCodecFactory.java similarity index 100% rename from parquet/src/main/java/org/apache/iceberg/parquet/ParquetCodecFactory.java rename to core/src/main/java/org/apache/iceberg/parquet/ParquetCodecFactory.java diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetConversions.java b/core/src/main/java/org/apache/iceberg/parquet/ParquetConversions.java similarity index 100% rename from parquet/src/main/java/org/apache/iceberg/parquet/ParquetConversions.java rename to core/src/main/java/org/apache/iceberg/parquet/ParquetConversions.java diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetDictionaryRowGroupFilter.java b/core/src/main/java/org/apache/iceberg/parquet/ParquetDictionaryRowGroupFilter.java similarity index 100% rename from parquet/src/main/java/org/apache/iceberg/parquet/ParquetDictionaryRowGroupFilter.java rename to core/src/main/java/org/apache/iceberg/parquet/ParquetDictionaryRowGroupFilter.java diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetFilters.java b/core/src/main/java/org/apache/iceberg/parquet/ParquetFilters.java similarity index 100% rename from parquet/src/main/java/org/apache/iceberg/parquet/ParquetFilters.java rename to core/src/main/java/org/apache/iceberg/parquet/ParquetFilters.java diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetIO.java b/core/src/main/java/org/apache/iceberg/parquet/ParquetIO.java similarity index 100% rename from parquet/src/main/java/org/apache/iceberg/parquet/ParquetIO.java rename to core/src/main/java/org/apache/iceberg/parquet/ParquetIO.java diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetIterable.java b/core/src/main/java/org/apache/iceberg/parquet/ParquetIterable.java similarity index 100% rename from parquet/src/main/java/org/apache/iceberg/parquet/ParquetIterable.java rename to core/src/main/java/org/apache/iceberg/parquet/ParquetIterable.java diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetMetricsRowGroupFilter.java b/core/src/main/java/org/apache/iceberg/parquet/ParquetMetricsRowGroupFilter.java similarity index 100% rename from parquet/src/main/java/org/apache/iceberg/parquet/ParquetMetricsRowGroupFilter.java rename to core/src/main/java/org/apache/iceberg/parquet/ParquetMetricsRowGroupFilter.java diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReadSupport.java b/core/src/main/java/org/apache/iceberg/parquet/ParquetReadSupport.java similarity index 100% rename from parquet/src/main/java/org/apache/iceberg/parquet/ParquetReadSupport.java rename to core/src/main/java/org/apache/iceberg/parquet/ParquetReadSupport.java diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReader.java b/core/src/main/java/org/apache/iceberg/parquet/ParquetReader.java similarity index 100% rename from parquet/src/main/java/org/apache/iceberg/parquet/ParquetReader.java rename to core/src/main/java/org/apache/iceberg/parquet/ParquetReader.java diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java b/core/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java similarity index 100% rename from parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java rename to core/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetTypeVisitor.java b/core/src/main/java/org/apache/iceberg/parquet/ParquetTypeVisitor.java similarity index 100% rename from parquet/src/main/java/org/apache/iceberg/parquet/ParquetTypeVisitor.java rename to core/src/main/java/org/apache/iceberg/parquet/ParquetTypeVisitor.java diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetUtil.java b/core/src/main/java/org/apache/iceberg/parquet/ParquetUtil.java similarity index 100% rename from parquet/src/main/java/org/apache/iceberg/parquet/ParquetUtil.java rename to core/src/main/java/org/apache/iceberg/parquet/ParquetUtil.java diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueReader.java b/core/src/main/java/org/apache/iceberg/parquet/ParquetValueReader.java similarity index 100% rename from parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueReader.java rename to core/src/main/java/org/apache/iceberg/parquet/ParquetValueReader.java diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueReaders.java b/core/src/main/java/org/apache/iceberg/parquet/ParquetValueReaders.java similarity index 100% rename from parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueReaders.java rename to core/src/main/java/org/apache/iceberg/parquet/ParquetValueReaders.java diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueWriter.java b/core/src/main/java/org/apache/iceberg/parquet/ParquetValueWriter.java similarity index 100% rename from parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueWriter.java rename to core/src/main/java/org/apache/iceberg/parquet/ParquetValueWriter.java diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueWriters.java b/core/src/main/java/org/apache/iceberg/parquet/ParquetValueWriters.java similarity index 100% rename from parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueWriters.java rename to core/src/main/java/org/apache/iceberg/parquet/ParquetValueWriters.java diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetWriteAdapter.java b/core/src/main/java/org/apache/iceberg/parquet/ParquetWriteAdapter.java similarity index 100% rename from parquet/src/main/java/org/apache/iceberg/parquet/ParquetWriteAdapter.java rename to core/src/main/java/org/apache/iceberg/parquet/ParquetWriteAdapter.java diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetWriteSupport.java b/core/src/main/java/org/apache/iceberg/parquet/ParquetWriteSupport.java similarity index 100% rename from parquet/src/main/java/org/apache/iceberg/parquet/ParquetWriteSupport.java rename to core/src/main/java/org/apache/iceberg/parquet/ParquetWriteSupport.java diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetWriter.java b/core/src/main/java/org/apache/iceberg/parquet/ParquetWriter.java similarity index 100% rename from parquet/src/main/java/org/apache/iceberg/parquet/ParquetWriter.java rename to core/src/main/java/org/apache/iceberg/parquet/ParquetWriter.java diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/PruneColumns.java b/core/src/main/java/org/apache/iceberg/parquet/PruneColumns.java similarity index 100% rename from parquet/src/main/java/org/apache/iceberg/parquet/PruneColumns.java rename to core/src/main/java/org/apache/iceberg/parquet/PruneColumns.java diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java b/core/src/main/java/org/apache/iceberg/parquet/ReadConf.java similarity index 100% rename from parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java rename to core/src/main/java/org/apache/iceberg/parquet/ReadConf.java diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/RemoveIds.java b/core/src/main/java/org/apache/iceberg/parquet/RemoveIds.java similarity index 100% rename from parquet/src/main/java/org/apache/iceberg/parquet/RemoveIds.java rename to core/src/main/java/org/apache/iceberg/parquet/RemoveIds.java diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/TripleIterator.java b/core/src/main/java/org/apache/iceberg/parquet/TripleIterator.java similarity index 100% rename from parquet/src/main/java/org/apache/iceberg/parquet/TripleIterator.java rename to core/src/main/java/org/apache/iceberg/parquet/TripleIterator.java diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/TripleWriter.java b/core/src/main/java/org/apache/iceberg/parquet/TripleWriter.java similarity index 100% rename from parquet/src/main/java/org/apache/iceberg/parquet/TripleWriter.java rename to core/src/main/java/org/apache/iceberg/parquet/TripleWriter.java diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/TypeToMessageType.java b/core/src/main/java/org/apache/iceberg/parquet/TypeToMessageType.java similarity index 100% rename from parquet/src/main/java/org/apache/iceberg/parquet/TypeToMessageType.java rename to core/src/main/java/org/apache/iceberg/parquet/TypeToMessageType.java diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/TypeWithSchemaVisitor.java b/core/src/main/java/org/apache/iceberg/parquet/TypeWithSchemaVisitor.java similarity index 100% rename from parquet/src/main/java/org/apache/iceberg/parquet/TypeWithSchemaVisitor.java rename to core/src/main/java/org/apache/iceberg/parquet/TypeWithSchemaVisitor.java diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ValuesAsBytesReader.java b/core/src/main/java/org/apache/iceberg/parquet/ValuesAsBytesReader.java similarity index 100% rename from parquet/src/main/java/org/apache/iceberg/parquet/ValuesAsBytesReader.java rename to core/src/main/java/org/apache/iceberg/parquet/ValuesAsBytesReader.java diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReader.java b/core/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReader.java similarity index 100% rename from parquet/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReader.java rename to core/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReader.java diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedReader.java b/core/src/main/java/org/apache/iceberg/parquet/VectorizedReader.java similarity index 100% rename from parquet/src/main/java/org/apache/iceberg/parquet/VectorizedReader.java rename to core/src/main/java/org/apache/iceberg/parquet/VectorizedReader.java diff --git a/core/src/main/java/org/apache/iceberg/partition/stats/PartitionStatsUtil.java b/core/src/main/java/org/apache/iceberg/partition/stats/PartitionStatsUtil.java new file mode 100644 index 000000000000..9a37c276ea07 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/partition/stats/PartitionStatsUtil.java @@ -0,0 +1,132 @@ +/* + * 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.partition.stats; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Collection; +import java.util.Iterator; +import org.apache.avro.generic.GenericData; +import org.apache.iceberg.Partition; +import org.apache.iceberg.PartitionData; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Partitioning; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.DataWriter; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.parquet.ParquetAvroValueReaders; +import org.apache.iceberg.parquet.ParquetAvroWriter; +import org.apache.iceberg.types.Types; + +public final class PartitionStatsUtil { + + private PartitionStatsUtil() {} + + private static final String PARQUET_SUFFIX = ".parquet"; + + public static void writePartitionStatsFile( + Iterator partitions, OutputFile outputFile, Collection specs) { + validateFormat(outputFile.location()); + writeAsParquetFile( + Partition.icebergSchema(Partitioning.partitionType(specs)), partitions, outputFile); + } + + private static void validateFormat(String filePath) { + if (!filePath.toLowerCase().endsWith(PARQUET_SUFFIX)) { + throw new UnsupportedOperationException("Unsupported format : " + filePath); + } + } + + public static CloseableIterable readPartitionStatsFile( + Schema schema, InputFile inputFile) { + validateFormat(inputFile.location()); + // schema of partition column during read could be different from + // what is used for writing due to partition evolution. + // While reading, ParquetAvroValueReaders fills the data as per latest schema. + CloseableIterable records = + Parquet.read(inputFile) + .project(schema) + .createReaderFunc(fileSchema -> ParquetAvroValueReaders.buildReader(schema, fileSchema)) + .build(); + return CloseableIterable.transform(records, record -> toPartition(schema, record)); + } + + private static Partition toPartition(Schema schema, GenericData.Record record) { + Partition partition = new Partition(); + partition.put( + Partition.Column.PARTITION_DATA.ordinal(), extractPartitionDataFromRecord(schema, record)); + + int recordCount = record.getSchema().getFields().size(); + for (int columnIndex = 1; columnIndex < recordCount; columnIndex++) { + partition.put(columnIndex, record.get(columnIndex)); + } + + return partition; + } + + private static PartitionData extractPartitionDataFromRecord( + Schema schema, GenericData.Record record) { + int partitionDataCount = + record + .getSchema() + .getField(Partition.Column.PARTITION_DATA.name()) + .schema() + .getFields() + .size(); + PartitionData partitionData = + new PartitionData( + (Types.StructType) schema.findField(Partition.Column.PARTITION_DATA.name()).type()); + for (int partitionColIndex = 0; partitionColIndex < partitionDataCount; partitionColIndex++) { + partitionData.set( + partitionColIndex, + ((GenericData.Record) record.get(Partition.Column.PARTITION_DATA.ordinal())) + .get(partitionColIndex)); + } + + return partitionData; + } + + private static void writeAsParquetFile( + Schema schema, Iterator records, OutputFile outputFile) { + String firstPartitionCol = + ((Types.StructType) schema.asStruct().fields().get(0).type()).fields().get(0).name(); + // sorting based on the first partition column + SortOrder sortOrder = + SortOrder.builderFor(schema) + .asc(Partition.Column.PARTITION_DATA.name() + "." + firstPartitionCol) + .build(); + + try (DataWriter dataWriter = + Parquet.writeData(outputFile) + .schema(schema) + .createWriterFunc(ParquetAvroWriter::buildWriter) + .overwrite() + .withSpec(PartitionSpec.unpartitioned()) + .withSortOrder(sortOrder) + .build()) { + records.forEachRemaining(dataWriter::write); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/util/PartitionUtil.java b/core/src/main/java/org/apache/iceberg/util/PartitionUtil.java index b3a5e85d3383..e9e2d9f26642 100644 --- a/core/src/main/java/org/apache/iceberg/util/PartitionUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/PartitionUtil.java @@ -93,6 +93,10 @@ private PartitionUtil() {} // adapts the provided partition data to match the table partition type public static StructLike coercePartition( Types.StructType partitionType, PartitionSpec spec, StructLike partition) { + if (spec.partitionType().equals(partitionType)) { + return partition; + } + StructProjection projection = StructProjection.createAllowMissing(spec.partitionType(), partitionType); projection.wrap(partition); diff --git a/core/src/test/java/org/apache/iceberg/TableTestBase.java b/core/src/test/java/org/apache/iceberg/TableTestBase.java index a800214bc9a7..134bdb391704 100644 --- a/core/src/test/java/org/apache/iceberg/TableTestBase.java +++ b/core/src/test/java/org/apache/iceberg/TableTestBase.java @@ -206,12 +206,13 @@ List listManifestFiles(File tableDirToList) { .listFiles( (dir, name) -> !name.startsWith("snap") + && !name.startsWith("partition-stats") && Files.getFileExtension(name).equalsIgnoreCase("avro"))); } public static long countAllMetadataFiles(File tableDir) { return Arrays.stream(new File(tableDir, "metadata").listFiles()) - .filter(f -> f.isFile()) + .filter(f -> f.isFile() && !f.getName().startsWith("partition-stats")) .count(); } diff --git a/core/src/test/java/org/apache/iceberg/TestMergeAppend.java b/core/src/test/java/org/apache/iceberg/TestMergeAppend.java index 892d92634c09..4c7daac8a5d6 100644 --- a/core/src/test/java/org/apache/iceberg/TestMergeAppend.java +++ b/core/src/test/java/org/apache/iceberg/TestMergeAppend.java @@ -1289,6 +1289,11 @@ public void testManifestEntryFieldIdsForChangedPartitionSpecForV1Table() { V1Assert.assertEquals( "Table should end with last-sequence-number 0", 0, readMetadata().lastSequenceNumber()); + // the new spec will again have a field ID 1000 (which is same as old spec) and it is ambiguous + // for partition stats writing. This is not ideal scenario. Hence, disable the partition stats + // writing. + table.updateProperties().set(TableProperties.PARTITION_STATS_ENABLED, "false").commit(); + // create a new with the table's current spec DataFile newFile = DataFiles.builder(table.spec()) @@ -1300,6 +1305,13 @@ public void testManifestEntryFieldIdsForChangedPartitionSpecForV1Table() { Snapshot committedSnapshot = commit(table, table.newAppend().appendFile(newFile), branch); + table + .updateProperties() + .set( + TableProperties.PARTITION_STATS_ENABLED, + String.valueOf(TableProperties.PARTITION_STATS_ENABLED_DEFAULT)) + .commit(); + V2Assert.assertEquals( "Snapshot sequence number should be 2", 2, committedSnapshot.sequenceNumber()); V2Assert.assertEquals( diff --git a/core/src/test/java/org/apache/iceberg/TestMetadataUpdateParser.java b/core/src/test/java/org/apache/iceberg/TestMetadataUpdateParser.java index fe36c5daa26d..18c6641e76d1 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetadataUpdateParser.java +++ b/core/src/test/java/org/apache/iceberg/TestMetadataUpdateParser.java @@ -884,6 +884,48 @@ public void testSetCurrentViewVersionToJson() { Assertions.assertThat(MetadataUpdateParser.toJson(update)).isEqualTo(expected); } + @Test + public void testSetPartitionStatistics() { + String json = + "{\"action\":\"set-partition-statistics\",\"snapshot-id\":1940541653261589030," + + "\"partition-statistics\":{\"snapshot-id\":1940541653261589030," + + "\"statistics-path\":\"s3://bucket/warehouse/stats1.parquet\"," + + "\"max-data-sequence-number\":43}}"; + + long snapshotId = 1940541653261589030L; + MetadataUpdate expected = + new MetadataUpdate.SetPartitionStatistics( + snapshotId, + ImmutableGenericPartitionStatisticsFile.builder() + .snapshotId(snapshotId) + .path("s3://bucket/warehouse/stats1" + ".parquet") + .maxDataSequenceNumber(43L) + .build()); + assertEquals( + MetadataUpdateParser.SET_PARTITION_STATISTICS, + expected, + MetadataUpdateParser.fromJson(json)); + Assert.assertEquals( + "Set partition statistics should convert to the correct JSON value", + json, + MetadataUpdateParser.toJson(expected)); + } + + @Test + public void testRemovePartitionStatistics() { + String json = + "{\"action\":\"remove-partition-statistics\",\"snapshot-id\":1940541653261589030}"; + MetadataUpdate expected = new MetadataUpdate.RemovePartitionStatistics(1940541653261589030L); + assertEquals( + MetadataUpdateParser.REMOVE_PARTITION_STATISTICS, + expected, + MetadataUpdateParser.fromJson(json)); + Assert.assertEquals( + "Remove partition statistics should convert to the correct JSON value", + json, + MetadataUpdateParser.toJson(expected)); + } + public void assertEquals( String action, MetadataUpdate expectedUpdate, MetadataUpdate actualUpdate) { switch (action) { @@ -935,6 +977,16 @@ public void assertEquals( (MetadataUpdate.RemoveStatistics) expectedUpdate, (MetadataUpdate.RemoveStatistics) actualUpdate); break; + case MetadataUpdateParser.SET_PARTITION_STATISTICS: + assertEqualsSetPartitionStatistics( + (MetadataUpdate.SetPartitionStatistics) expectedUpdate, + (MetadataUpdate.SetPartitionStatistics) actualUpdate); + break; + case MetadataUpdateParser.REMOVE_PARTITION_STATISTICS: + assertEqualsRemovePartitionStatistics( + (MetadataUpdate.RemovePartitionStatistics) expectedUpdate, + (MetadataUpdate.RemovePartitionStatistics) actualUpdate); + break; case MetadataUpdateParser.ADD_SNAPSHOT: assertEqualsAddSnapshot( (MetadataUpdate.AddSnapshot) expectedUpdate, (MetadataUpdate.AddSnapshot) actualUpdate); @@ -1134,6 +1186,31 @@ private static void assertEqualsRemoveStatistics( "Snapshots to remove should be the same", expected.snapshotId(), actual.snapshotId()); } + private static void assertEqualsSetPartitionStatistics( + MetadataUpdate.SetPartitionStatistics expected, + MetadataUpdate.SetPartitionStatistics actual) { + Assert.assertEquals("Snapshot IDs should be equal", expected.snapshotId(), actual.snapshotId()); + Assert.assertEquals( + "Partition Statistics files snapshot IDs should be equal", + expected.partitionStatisticsFile().snapshotId(), + actual.partitionStatisticsFile().snapshotId()); + Assert.assertEquals( + "Partition statistics files paths should be equal", + expected.partitionStatisticsFile().path(), + actual.partitionStatisticsFile().path()); + Assert.assertEquals( + "Partition statistics max data sequence number should be equal", + expected.partitionStatisticsFile().maxDataSequenceNumber(), + actual.partitionStatisticsFile().maxDataSequenceNumber()); + } + + private static void assertEqualsRemovePartitionStatistics( + MetadataUpdate.RemovePartitionStatistics expected, + MetadataUpdate.RemovePartitionStatistics actual) { + Assert.assertEquals( + "Snapshots to remove should be the same", expected.snapshotId(), actual.snapshotId()); + } + private static void assertEqualsAddSnapshot( MetadataUpdate.AddSnapshot expected, MetadataUpdate.AddSnapshot actual) { Assert.assertEquals( diff --git a/core/src/test/java/org/apache/iceberg/TestPartition.java b/core/src/test/java/org/apache/iceberg/TestPartition.java new file mode 100644 index 000000000000..4043ddd85e95 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/TestPartition.java @@ -0,0 +1,74 @@ +/* + * 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 org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +public class TestPartition { + + @Test + public void testPartitionBuilder() { + Types.StructType partitionType = + Types.StructType.of( + Types.NestedField.required(1000, "field1", Types.StringType.get()), + Types.NestedField.required(1001, "field2", Types.IntegerType.get())); + PartitionData partitionData = new PartitionData(partitionType); + partitionData.set(0, "value1"); + partitionData.set(1, 42); + + Partition partition = + Partition.builder() + .withPartitionData(partitionData) + .withSpecId(123) + .withDataRecordCount(1000L) + .withDataFileCount(5) + .withDataFileSizeInBytes(1024L * 1024L) + .withPosDeleteRecordCount(50L) + .withPosDeleteFileCount(2) + .withEqDeleteRecordCount(20L) + .withEqDeleteFileCount(1) + .withLastUpdatedAt(1627900200L) + .withLastUpdatedSnapshotId(456789L) + .build(); + + // Verify the get method + Assertions.assertEquals(partitionData, partition.get(0)); + Assertions.assertEquals(123, partition.get(1)); + Assertions.assertEquals(1000L, partition.get(2)); + Assertions.assertEquals(5, partition.get(3)); + Assertions.assertEquals(1024L * 1024L, partition.get(4)); + Assertions.assertEquals(50L, partition.get(5)); + Assertions.assertEquals(2, partition.get(6)); + Assertions.assertEquals(20L, partition.get(7)); + Assertions.assertEquals(1, partition.get(8)); + Assertions.assertEquals(1627900200L, partition.get(9)); + Assertions.assertEquals(456789L, partition.get(10)); + + // Verify the put method + Partition newPartition = new Partition(); + int size = partition.getSchema().getFields().size(); + for (int i = 0; i < size; i++) { + newPartition.put(i, partition.get(i)); + } + + Assertions.assertEquals(newPartition, partition); + } +} diff --git a/core/src/test/java/org/apache/iceberg/TestPartitionStats.java b/core/src/test/java/org/apache/iceberg/TestPartitionStats.java new file mode 100644 index 000000000000..fd940fbd06e3 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/TestPartitionStats.java @@ -0,0 +1,611 @@ +/* + * 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.ManifestEntry.Status.ADDED; +import static org.junit.Assert.fail; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.partition.stats.PartitionStatsUtil; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.assertj.core.api.Assertions; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TestPartitionStats extends TableTestBase { + public TestPartitionStats(int formatVersion) { + super(formatVersion); + } + + @Parameterized.Parameters(name = "formatVersion = {0}") + public static Object[] parameters() { + return new Object[] {1, 2}; + } + + @Test + public void testNoPartitionStatsForUnPartitionedTable() throws Exception { + TestTables.TestTable table = + TestTables.create( + temp.newFolder(), + "unpartitioned", + SCHEMA, + PartitionSpec.unpartitioned(), + formatVersion); + table + .newAppend() + .appendFile(newUnpartitionedDataFile("data-1.parquet", PartitionSpec.unpartitioned())) + .commit(); + + Snapshot snapshot = table.currentSnapshot(); + Assertions.assertThat(snapshot.partitionStatsFileLocation()).isNull(); + Assertions.assertThat(table.partitionStatisticsFiles()).isEmpty(); + } + + @Test + public void testNoPartitionStatsForUnPartitionedDataFiles() { + // partitioned table but un-partitioned data file. + table + .newAppend() + .appendFile(newUnpartitionedDataFile("data-1.parquet", PartitionSpec.unpartitioned())) + .commit(); + + Snapshot snapshot = table.currentSnapshot(); + Assertions.assertThat(snapshot.partitionStatsFileLocation()).isNull(); + Assertions.assertThat(table.partitionStatisticsFiles()).isEmpty(); + } + + @Test + public void testNoPartitionStatsForNoDatafiles() { + table.newAppend().commit(); + Snapshot snapshot = table.currentSnapshot(); + Assertions.assertThat(snapshot.partitionStatsFileLocation()).isNull(); + Assertions.assertThat(table.partitionStatisticsFiles()).isEmpty(); + } + + @Test + public void testNoSideEffectsFromUncommittedAppend() { + // Two uncommitted files + table + .newAppend() + .appendFile(newDataFile("data_bucket=0")) + .appendFile(newDataFile("data_bucket=0")); + + // One committed file + table.newAppend().appendFile(newDataFile("data_bucket=0")).commit(); + + validatePartitionStaticsFile(table.currentSnapshot()); + validatePartition(1, 1, 10); + } + + @Test + public void testPartitionStatsAfterAppendFilesInSinglePartition() { + table + .newAppend() + .appendFile(newDataFile("data_bucket=0")) + .appendFile(newDataFile("data_bucket=0")) + .commit(); + + validatePartitionStaticsFile(table.currentSnapshot()); + validatePartition(2, 2, 20); + } + + @Test + public void testPartitionStatsFileOneTxn() { + Transaction transaction = table.newTransaction(); + transaction + .newAppend() + .appendFile(newDataFile("data_bucket=0")) + .appendFile(newDataFile("data_bucket=0")) + .commit(); + transaction.commitTransaction(); + + validatePartitionStaticsFile(table.currentSnapshot()); + } + + @Test + public void testPartitionStatsFileOnTransactionRetry() { + int retryCount = 1; + table + .updateProperties() + .set(TableProperties.COMMIT_NUM_RETRIES, String.valueOf(retryCount)) + .commit(); + + Transaction transaction = table.newTransaction(); + transaction + .newAppend() + .appendFile(newDataFile("data_bucket=0")) + .appendFile(newDataFile("data_bucket=0")) + .commit(); + + table.ops().failCommits(retryCount); + transaction.commitTransaction(); + + validatePartitionStaticsFile(table.currentSnapshot()); + } + + @Test + public void testNoPartitionStatsFileOnTransactionFailure() { + int retryCount = 1; + table.newAppend().appendFile(newDataFile("data_bucket=0")).commit(); + + Snapshot snapshot1 = table.currentSnapshot(); + validatePartitionStaticsFile(snapshot1); + PartitionStatisticsFile partitionStatisticsFile = table.partitionStatisticsFiles().get(0); + + table + .updateProperties() + .set(TableProperties.COMMIT_NUM_RETRIES, String.valueOf(retryCount)) + .commit(); + + Transaction transaction = table.newTransaction(); + transaction + .newAppend() + .appendFile(newDataFile("data_bucket=0")) + .appendFile(newDataFile("data_bucket=0")) + .commit(); + + table.ops().failCommits(retryCount + 1); + try { + transaction.commitTransaction(); + fail("Expected CommitFailedException"); + } catch (Exception exception) { + Assertions.assertThat(exception instanceof CommitFailedException).isTrue(); + } + + Assertions.assertThat(table.currentSnapshot()).isEqualTo(snapshot1); + Assertions.assertThat(table.partitionStatisticsFiles()) + .containsExactly(partitionStatisticsFile); + } + + @Test + public void testPartitionStatsAfterFastAppendFilesInSinglePartition() { + table + .newFastAppend() + .appendFile(newDataFile("data_bucket=0")) + .appendFile(newDataFile("data_bucket=0")) + .commit(); + + validatePartitionStaticsFile(table.currentSnapshot()); + } + + @Test + public void testPartitionStatsAfterAppendManifestInSinglePartition() throws Exception { + table + .newAppend() + .appendManifest( + writeManifest( + "new-manifest0.avro", + manifestEntry(ADDED, null, newDataFile("data_bucket=0")), + manifestEntry(ADDED, null, newDataFile("data_bucket=0")))) + .appendManifest( + writeManifest( + "new-manifest1.avro", + manifestEntry(ADDED, null, newDataFile("data_bucket=0")), + manifestEntry(ADDED, null, newDataFile("data_bucket=0")))) + .commit(); + + validatePartitionStaticsFile(table.currentSnapshot()); + validatePartition(4, 4, 40); + } + + @Test + public void testPartitionStatsAfterAppendManifestInSinglePartitionWhenSnapshotInheritanceEnabled() + throws Exception { + table.updateProperties().set(TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED, "true").commit(); + + table + .newAppend() + .appendManifest( + writeManifest( + "new-manifest0.avro", + manifestEntry(ADDED, null, newDataFile("data_bucket=0")), + manifestEntry(ADDED, null, newDataFile("data_bucket=0")))) + .appendManifest( + writeManifest( + "new-manifest1.avro", + manifestEntry(ADDED, null, newDataFile("data_bucket=0")), + manifestEntry(ADDED, null, newDataFile("data_bucket=0")))) + .commit(); + + validatePartitionStaticsFile(table.currentSnapshot()); + validatePartition(4, 4, 40); + + table + .updateProperties() + .set( + TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED, + String.valueOf(TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED_DEFAULT)) + .commit(); + } + + @Test + public void + testPartitionStatsAfterFastAppendManifestInSinglePartitionWhenSnapshotInheritanceIsOn() + throws Exception { + table.updateProperties().set(TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED, "true").commit(); + + table + .newFastAppend() + .appendManifest( + writeManifest( + "new-manifest0.avro", + manifestEntry(ADDED, null, newDataFile("data_bucket=0")), + manifestEntry(ADDED, null, newDataFile("data_bucket=0")))) + .appendManifest( + writeManifest( + "new-manifest1.avro", + manifestEntry(ADDED, null, newDataFile("data_bucket=0")), + manifestEntry(ADDED, null, newDataFile("data_bucket=0")))) + .commit(); + + validatePartitionStaticsFile(table.currentSnapshot()); + validatePartition(4, 4, 40); + + table + .updateProperties() + .set( + TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED, + String.valueOf(TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED_DEFAULT)) + .commit(); + } + + @Test + public void testPartitionStatsAfterAppendManifestAcrossPartitions() throws Exception { + table + .newAppend() + .appendManifest( + writeManifest( + "new-manifest0.avro", + manifestEntry(ADDED, null, newDataFile("data_bucket=0")), + manifestEntry(ADDED, null, newDataFile("data_bucket=1")))) + .appendManifest( + writeManifest( + "new-manifest1.avro", + manifestEntry(ADDED, null, newDataFile("data_bucket=0")), + manifestEntry(ADDED, null, newDataFile("data_bucket=1")))) + .commit(); + + validatePartitionStaticsFile(table.currentSnapshot()); + PartitionData partitionData1 = new PartitionData(Partitioning.partitionType(table)); + partitionData1.set(0, 0); + Partition partition1 = + buildPartition(partitionData1, 2, 2, 20, table.currentSnapshot(), table.spec().specId()); + + PartitionData partitionData2 = new PartitionData(Partitioning.partitionType(table)); + partitionData2.set(0, 1); + Partition partition2 = + buildPartition(partitionData2, 2, 2, 20, table.currentSnapshot(), table.spec().specId()); + validatePartitions(table, Lists.newArrayList(partition1, partition2)); + } + + @Test + public void testPartitionStatsAfterAppendDeleteDataFilesAndManifestFilesInOneTxn() + throws Exception { + table + .newAppend() + .appendFile(newDataFile("data_bucket=0")) + .appendFile(newDataFile("data_bucket=1")) + .commit(); + + Transaction transaction = table.newTransaction(); + transaction + .newAppend() + .appendFile(newDataFile("data_bucket=0")) + .appendFile(newDataFile("data_bucket=1")) + .appendManifest(writeManifest(newDataFile("data_bucket=0"), newDataFile("data_bucket=1"))) + .commit(); + transaction + .newDelete() + .deleteFile(newDataFile("data_bucket=0")) + .deleteFile(newDataFile("data_bucket=1").path()) + .commit(); + transaction.commitTransaction(); + + // Delete is unaffected due to the logic in ManifestFilterManager.manifestHasDeletedFiles() + // returning false. TODO: Need to analyze. + validatePartitionStaticsFile(table.currentSnapshot()); + PartitionData partitionData1 = new PartitionData(Partitioning.partitionType(table)); + partitionData1.set(0, 0); + Partition partition1 = + buildPartition( + partitionData1, + 3, + 3, + 30, + table.snapshot(table.currentSnapshot().parentId()), + table.spec().specId()); + + PartitionData partitionData2 = new PartitionData(Partitioning.partitionType(table)); + partitionData2.set(0, 1); + Partition partition2 = + buildPartition( + partitionData2, + 3, + 3, + 30, + table.snapshot(table.currentSnapshot().parentId()), + table.spec().specId()); + validatePartitions(table, Lists.newArrayList(partition1, partition2)); + } + + @Test + public void testPartitionStatsAfterFastAppendDeleteDataFilesAndManifestFilesInOneTxn() + throws Exception { + table + .newFastAppend() + .appendFile(newDataFile("data_bucket=0")) + .appendFile(newDataFile("data_bucket=1")) + .commit(); + + Transaction transaction = table.newTransaction(); + transaction + .newFastAppend() + .appendFile(newDataFile("data_bucket=0")) + .appendFile(newDataFile("data_bucket=1")) + .appendManifest(writeManifest(newDataFile("data_bucket=0"), newDataFile("data_bucket=1"))) + .commit(); + transaction + .newDelete() + .deleteFile(newDataFile("data_bucket=0")) + .deleteFile(newDataFile("data_bucket=1").path()) + .commit(); + transaction.commitTransaction(); + + validatePartitionStaticsFile(table.currentSnapshot()); + // Delete is unaffected due to the logic in ManifestFilterManager.manifestHasDeletedFiles() + // returning false. TODO: Need to analyze. + validatePartitionStaticsFile(table.currentSnapshot()); + PartitionData partitionData1 = new PartitionData(Partitioning.partitionType(table)); + partitionData1.set(0, 0); + Partition partition1 = + buildPartition( + partitionData1, + 3, + 3, + 30, + table.snapshot(table.currentSnapshot().parentId()), + table.spec().specId()); + + PartitionData partitionData2 = new PartitionData(Partitioning.partitionType(table)); + partitionData2.set(0, 1); + Partition partition2 = + buildPartition( + partitionData2, + 3, + 3, + 30, + table.snapshot(table.currentSnapshot().parentId()), + table.spec().specId()); + validatePartitions(table, Lists.newArrayList(partition1, partition2)); + } + + @Test + public void testPartitionStatsAfterAppendFilesUpdatedPartitionSpec() { + table + .newAppend() + .appendFile(newDataFile("data_bucket=0")) + .appendFile(newDataFile("data_bucket=0")) + .commit(); + + validatePartitionStaticsFile(table.currentSnapshot()); + int specId = table.spec().specId(); + PartitionData partitionData1 = new PartitionData(Partitioning.partitionType(table)); + partitionData1.set(0, 0); + Partition partition1 = + buildPartition(partitionData1, 2, 2, 20, table.currentSnapshot(), specId); + validatePartitions(table, Collections.singletonList(partition1)); + + table.updateSpec().addField("id").commit(); + + table + .newAppend() + .appendFile(newDataFile("data_bucket=1/id=2")) + .appendFile(newDataFile("data_bucket=1/id=2")) + .commit(); + + validatePartitionStaticsFile(table.currentSnapshot()); + + PartitionData partitionData1Coerced = new PartitionData(Partitioning.partitionType(table)); + partitionData1Coerced.set(0, 0); + Partition partition1Coerced = + buildPartition( + partitionData1Coerced, + 2, + 2, + 20, + table.snapshot(table.currentSnapshot().parentId()), + specId); + + PartitionData partitionData2 = new PartitionData(Partitioning.partitionType(table)); + partitionData2.set(0, 1); + partitionData2.set(1, 2); + Partition partition2 = + buildPartition(partitionData2, 2, 2, 20, table.currentSnapshot(), table.spec().specId()); + validatePartitions(table, Lists.newArrayList(partition1Coerced, partition2)); + } + + @Test + public void testNoPartitionStatsAfterUpdateSpecRemovesField() { + table.updateSpec().removeField("data_bucket").commit(); + + table.newAppend().appendFile(newUnpartitionedDataFile("data-1.parquet", table.spec())).commit(); + + Assertions.assertThat(table.currentSnapshot().partitionStatsFileLocation()).isNull(); + Assertions.assertThat(table.partitionStatisticsFiles()).isEmpty(); + } + + @Test + public void testPartitionStatsAfterDeleteFiles() { + table + .newAppend() + .appendFile(newDataFile("data_bucket=0")) + .appendFile(newDataFile("data_bucket=0")) + .appendFile(newDataFile("data_bucket=1")) + .appendFile(newDataFile("data_bucket=1")) + .commit(); + validatePartitionStaticsFile(table.currentSnapshot()); + + table + .newDelete() + .deleteFile(newDataFile("data_bucket=0")) + .deleteFile(newDataFile("data_bucket=1").path()) + .commit(); + validatePartitionStaticsFile(table.currentSnapshot()); + } + + @Test + public void testPartitionStatsAfterUpdateSpecAndDeleteAllFiles() { + table + .newAppend() + .appendFile(newDataFile("data_bucket=0")) + .appendFile(newDataFile("data_bucket=0")) + .commit(); + + table.updateSpec().addField("id").commit(); + + table + .newAppend() + .appendFile(newDataFile("data_bucket=1/id=2")) + .appendFile(newDataFile("data_bucket=1/id=2")) + .commit(); + validatePartitionStaticsFile(table.currentSnapshot()); + + table.newDelete().deleteFromRowFilter(Expressions.alwaysTrue()).commit(); + validatePartitionStaticsFile(table.currentSnapshot()); + } + + @Test + public void testPartitionStatsAfterRewriteDataFiles() { + DataFile file1 = newDataFile("data_bucket=0"); + DataFile file2 = newDataFile("data_bucket=0"); + DataFile file3 = newDataFile("data_bucket=0"); + + table.newAppend().appendFile(file1).appendFile(file2).commit(); + table.newRewrite().rewriteFiles(Sets.newHashSet(file1, file2), Sets.newHashSet(file3)).commit(); + + validatePartitionStaticsFile(table.currentSnapshot()); + } + + @Test + public void testPartitionStatsAfterRewriteManifestFiles() { + table + .newAppend() + .appendFile(newDataFile("data_bucket=0")) + .appendFile(newDataFile("data_bucket=0")) + .appendFile(newDataFile("data_bucket=0")) + .appendFile(newDataFile("data_bucket=0")) + .commit(); + validatePartitionStaticsFile(table.currentSnapshot()); + + table.rewriteManifests().clusterBy(ContentFile::format).commit(); + + validatePartitionStaticsFile(table.currentSnapshot()); + } + + private DataFile newUnpartitionedDataFile(String newFilePath, PartitionSpec spec) { + return DataFiles.builder(spec) + .withPath(newFilePath) + .withFileSizeInBytes(10) + .withRecordCount(1) + .build(); + } + + private void validatePartition(long dataRecordCount, int dataFileCount, long dataFileSize) { + PartitionData partitionData = new PartitionData(Partitioning.partitionType(table)); + partitionData.set(0, 0); + validatePartition(table, partitionData, dataRecordCount, dataFileCount, dataFileSize); + } + + private void validatePartition( + Table table, + PartitionData partitionData, + long dataRecordCount, + int dataFileCount, + long dataFileSize) { + Partition partition = + buildPartition( + partitionData, + dataRecordCount, + dataFileCount, + dataFileSize, + table.currentSnapshot(), + table.spec().specId()); + validatePartitions(table, Collections.singletonList(partition)); + } + + private void validatePartitions(Table table, List partitions) { + Assertions.assertThat(readPartitionStats(table, table.currentSnapshot())) + .containsExactlyInAnyOrderElementsOf(partitions); + } + + private static Partition buildPartition( + PartitionData partitionData, + long dataRecordCount, + int dataFileCount, + long dataFileSize, + Snapshot snapshot, + int specId) { + return Partition.builder() + .withPartitionData(partitionData) + .withSpecId(specId) + .withDataRecordCount(dataRecordCount) + .withDataFileCount(dataFileCount) + .withDataFileSizeInBytes(dataFileSize) + .withPosDeleteRecordCount(0L) + .withPosDeleteFileCount(0) + .withEqDeleteRecordCount(0L) + .withEqDeleteFileCount(0) + .withLastUpdatedAt(snapshot.timestampMillis() * 1000L) + .withLastUpdatedSnapshotId(snapshot.snapshotId()) + .build(); + } + + private List readPartitionStats(Table table, Snapshot snapshot) { + Schema schema = Partition.icebergSchema(Partitioning.partitionType(table)); + try (CloseableIterable recordIterator = + PartitionStatsUtil.readPartitionStatsFile( + schema, Files.localInput(snapshot.partitionStatsFileLocation()))) { + return Lists.newArrayList(recordIterator); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + private void validatePartitionStaticsFile(Snapshot snapshot) { + validatePartitionStaticsFile(snapshot, table); + } + + private void validatePartitionStaticsFile(Snapshot snapshot, Table table) { + PartitionStatisticsFile partitionStatisticsFile = + ImmutableGenericPartitionStatisticsFile.builder() + .snapshotId(snapshot.snapshotId()) + .path(snapshot.partitionStatsFileLocation()) + .maxDataSequenceNumber(snapshot.sequenceNumber()) + .build(); + Assertions.assertThat(table.partitionStatisticsFiles()).contains(partitionStatisticsFile); + } +} diff --git a/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java b/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java index 71455c571282..548273443c71 100644 --- a/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java +++ b/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java @@ -20,8 +20,10 @@ import java.io.File; import java.io.IOException; +import java.io.UncheckedIOException; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; +import java.util.Collections; import java.util.List; import java.util.Set; import java.util.UUID; @@ -32,6 +34,7 @@ import org.apache.iceberg.ManifestEntry.Status; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.PositionOutputStream; import org.apache.iceberg.puffin.Blob; import org.apache.iceberg.puffin.Puffin; import org.apache.iceberg.puffin.PuffinWriter; @@ -40,6 +43,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.util.PropertyUtil; import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Assume; @@ -98,10 +102,9 @@ public void testExpireOlderThan() { table.currentSnapshot().snapshotId()); Assert.assertNull( "Expire should remove the oldest snapshot", table.snapshot(firstSnapshot.snapshotId())); - Assert.assertEquals( - "Should remove only the expired manifest list location", - Sets.newHashSet(firstSnapshot.manifestListLocation()), - deletedFiles); + Set expected = Sets.newHashSet(firstSnapshot.manifestListLocation()); + checkAndAddPartitionStatsFiles(Collections.singleton(firstSnapshot), expected); + Assert.assertEquals("Should remove only the expected files", expected, deletedFiles); } @Test @@ -144,8 +147,7 @@ public void testExpireOlderThanWithDelete() { "Expire should remove the second oldest snapshot", table.snapshot(secondSnapshot.snapshotId())); - Assert.assertEquals( - "Should remove expired manifest lists and deleted data file", + Set expected = Sets.newHashSet( firstSnapshot.manifestListLocation(), // snapshot expired firstSnapshot @@ -157,7 +159,12 @@ public void testExpireOlderThanWithDelete() { .allManifests(table.io()) .get(0) .path(), // manifest contained only deletes, was dropped - FILE_A.path()), // deleted + FILE_A.path()); + checkAndAddPartitionStatsFiles(Sets.newHashSet(firstSnapshot, secondSnapshot), expected); + + Assert.assertEquals( + "Should remove expected files", + expected, // deleted deletedFiles); } @@ -210,8 +217,7 @@ public void testExpireOlderThanWithDeleteInMergedManifests() { "Expire should remove the second oldest snapshot", table.snapshot(secondSnapshot.snapshotId())); - Assert.assertEquals( - "Should remove expired manifest lists and deleted data file", + Set expected = Sets.newHashSet( firstSnapshot.manifestListLocation(), // snapshot expired firstSnapshot @@ -219,7 +225,12 @@ public void testExpireOlderThanWithDeleteInMergedManifests() { .get(0) .path(), // manifest was rewritten for delete secondSnapshot.manifestListLocation(), // snapshot expired - FILE_A.path()), // deleted + FILE_A.path()); + checkAndAddPartitionStatsFiles(Sets.newHashSet(firstSnapshot, secondSnapshot), expected); + + Assert.assertEquals( + "Should remove expected files", + expected, // deleted deletedFiles); } @@ -265,12 +276,15 @@ public void testExpireOlderThanWithRollback() { Assert.assertNull( "Expire should remove the orphaned snapshot", table.snapshot(secondSnapshot.snapshotId())); - Assert.assertEquals( - "Should remove expired manifest lists and reverted appended data file", + Set expected = Sets.newHashSet( secondSnapshot.manifestListLocation(), // snapshot expired - Iterables.getOnlyElement(secondSnapshotManifests) - .path()), // manifest is no longer referenced + Iterables.getOnlyElement(secondSnapshotManifests).path()); + checkAndAddPartitionStatsFiles(Collections.singleton(secondSnapshot), expected); + + Assert.assertEquals( + "Should remove expected files", + expected, // manifest is no longer referenced deletedFiles); } @@ -313,16 +327,29 @@ public void testExpireOlderThanWithRollbackAndMergedManifests() { Assert.assertNull( "Expire should remove the orphaned snapshot", table.snapshot(secondSnapshot.snapshotId())); - Assert.assertEquals( - "Should remove expired manifest lists and reverted appended data file", + Set expected = Sets.newHashSet( secondSnapshot.manifestListLocation(), // snapshot expired Iterables.getOnlyElement(secondSnapshotManifests) .path(), // manifest is no longer referenced - FILE_B.path()), // added, but rolled back + FILE_B.path()); + checkAndAddPartitionStatsFiles(Collections.singleton(secondSnapshot), expected); + Assert.assertEquals( + "Should remove expected files", + expected, // added, but rolled back deletedFiles); } + private void checkAndAddPartitionStatsFiles( + Set expiredSnapshots, Set expected) { + if (PropertyUtil.propertyAsBoolean( + table.properties(), + TableProperties.PARTITION_STATS_ENABLED, + TableProperties.PARTITION_STATS_ENABLED_DEFAULT)) { + expiredSnapshots.forEach(snapshot -> expected.add(snapshot.partitionStatsFileLocation())); + } + } + @Test public void testRetainLastWithExpireOlderThan() { long t0 = System.currentTimeMillis(); @@ -889,7 +916,7 @@ public void testWithExpiringDanglingStageCommit() { .expireOlderThan(snapshotB.timestampMillis() + 1) .commit(); - Set expectedDeletes = Sets.newHashSet(); + Set expectedDeletes = Sets.newHashSet(); expectedDeletes.add(snapshotA.manifestListLocation()); // Files should be deleted of dangling staged snapshot @@ -911,6 +938,7 @@ public void testWithExpiringDanglingStageCommit() { expectedDeletes.add(file.path()); } }); + checkAndAddPartitionStatsFiles(Sets.newHashSet(snapshotA, snapshotB), expectedDeletes); Assert.assertSame( "Files deleted count should be expected", expectedDeletes.size(), deletedFiles.size()); // Take the diff @@ -1098,11 +1126,11 @@ public void testExpireWithDefaultSnapshotAge() { Assert.assertEquals( "Should not change current snapshot", thirdSnapshot, table.currentSnapshot()); Assert.assertEquals("Should keep 1 snapshot", 1, Iterables.size(table.snapshots())); - Assert.assertEquals( - "Should remove expired manifest lists", + Set expected = Sets.newHashSet( - firstSnapshot.manifestListLocation(), secondSnapshot.manifestListLocation()), - deletedFiles); + firstSnapshot.manifestListLocation(), secondSnapshot.manifestListLocation()); + checkAndAddPartitionStatsFiles(Sets.newHashSet(firstSnapshot, secondSnapshot), expected); + Assert.assertEquals("Should remove expected files", expected, deletedFiles); } @Test @@ -1148,21 +1176,20 @@ public void testExpireWithDeleteFiles() { Set deletedFiles = Sets.newHashSet(); removeSnapshots(table).expireOlderThan(fourthSnapshotTs).deleteWith(deletedFiles::add).commit(); - Assert.assertEquals( - "Should remove old delete files and delete file manifests", - ImmutableSet.builder() - .add(FILE_A.path()) - .add(FILE_A_DELETES.path()) - .add(firstSnapshot.manifestListLocation()) - .add(secondSnapshot.manifestListLocation()) - .add(thirdSnapshot.manifestListLocation()) - .addAll(manifestPaths(secondSnapshot, table.io())) - .addAll( - manifestOfDeletedFiles.stream() - .map(ManifestFile::path) - .collect(Collectors.toList())) - .build(), - deletedFiles); + Set expected = + Sets.newHashSet( + FILE_A.path(), + FILE_A_DELETES.path(), + firstSnapshot.manifestListLocation(), + secondSnapshot.manifestListLocation(), + thirdSnapshot.manifestListLocation()); + expected.addAll(manifestPaths(secondSnapshot, table.io())); + expected.addAll( + manifestOfDeletedFiles.stream().map(ManifestFile::path).collect(Collectors.toList())); + checkAndAddPartitionStatsFiles( + Sets.newHashSet(firstSnapshot, secondSnapshot, thirdSnapshot), expected); + + Assert.assertEquals("Should remove expected files", expected, deletedFiles); } @Test @@ -1272,8 +1299,8 @@ public void testExpireWithStatisticsFiles() throws IOException { .as("Should contain only the statistics file of snapshot2") .isEqualTo(Lists.newArrayList(statisticsFile2.snapshotId())); - Assertions.assertThat(new File(statsFileLocation1).exists()).isFalse(); - Assertions.assertThat(new File(statsFileLocation2).exists()).isTrue(); + Assertions.assertThat(new File(statsFileLocation1)).doesNotExist(); + Assertions.assertThat(new File(statsFileLocation2)).exists(); } @Test @@ -1310,7 +1337,84 @@ public void testExpireWithStatisticsFilesWithReuse() throws IOException { .as("Should contain only the statistics file of snapshot2") .isEqualTo(Lists.newArrayList(statisticsFile2.snapshotId())); // the reused stats file should exist. - Assertions.assertThat(new File(statsFileLocation1).exists()).isTrue(); + Assertions.assertThat(new File(statsFileLocation1)).exists(); + } + + @Test + public void testExpireWithPartitionStatisticsFiles() throws IOException { + table.newAppend().appendFile(FILE_A).commit(); + String statsFileLocation1 = statsFileLocation(table.location()); + PartitionStatisticsFile statisticsFile1 = + writePartitionStatsFile( + table.currentSnapshot().snapshotId(), + table.currentSnapshot().sequenceNumber(), + statsFileLocation1, + table.io()); + commitPartitionStats(table, statisticsFile1); + + table.newAppend().appendFile(FILE_B).commit(); + String statsFileLocation2 = statsFileLocation(table.location()); + PartitionStatisticsFile statisticsFile2 = + writePartitionStatsFile( + table.currentSnapshot().snapshotId(), + table.currentSnapshot().sequenceNumber(), + statsFileLocation2, + table.io()); + commitPartitionStats(table, statisticsFile2); + Assert.assertEquals( + "Should have 2 partition statistics file", 2, table.partitionStatisticsFiles().size()); + + long tAfterCommits = waitUntilAfter(table.currentSnapshot().timestampMillis()); + removeSnapshots(table).expireOlderThan(tAfterCommits).commit(); + + // only the current snapshot and its stats file should be retained + Assert.assertEquals("Should keep 1 snapshot", 1, Iterables.size(table.snapshots())); + Assertions.assertThat(table.partitionStatisticsFiles()) + .hasSize(1) + .extracting(PartitionStatisticsFile::snapshotId) + .as("Should contain only the statistics file of snapshot2") + .isEqualTo(Lists.newArrayList(statisticsFile2.snapshotId())); + + Assertions.assertThat(new File(statsFileLocation1)).doesNotExist(); + Assertions.assertThat(new File(statsFileLocation2)).exists(); + } + + @Test + public void testExpireWithPartitionStatisticsFilesWithReuse() throws IOException { + table.newAppend().appendFile(FILE_A).commit(); + String statsFileLocation1 = statsFileLocation(table.location()); + PartitionStatisticsFile statisticsFile1 = + writePartitionStatsFile( + table.currentSnapshot().snapshotId(), + table.currentSnapshot().sequenceNumber(), + statsFileLocation1, + table.io()); + commitPartitionStats(table, statisticsFile1); + + table.newAppend().appendFile(FILE_B).commit(); + // If an expired snapshot's stats file is reused for some reason by the live snapshots, + // that stats file should not get deleted from the file system as the live snapshots still + // reference it. + PartitionStatisticsFile statisticsFile2 = + reusePartitionStatsFile(table.currentSnapshot().snapshotId(), statisticsFile1); + commitPartitionStats(table, statisticsFile2); + + Assert.assertEquals( + "Should have 2 partition statistics file", 2, table.partitionStatisticsFiles().size()); + + long tAfterCommits = waitUntilAfter(table.currentSnapshot().timestampMillis()); + removeSnapshots(table).expireOlderThan(tAfterCommits).commit(); + + // only the current snapshot and its stats file (reused from previous snapshot) should be + // retained + Assert.assertEquals("Should keep 1 snapshot", 1, Iterables.size(table.snapshots())); + Assertions.assertThat(table.partitionStatisticsFiles()) + .hasSize(1) + .extracting(PartitionStatisticsFile::snapshotId) + .as("Should contain only the statistics file of snapshot2") + .isEqualTo(Lists.newArrayList(statisticsFile2.snapshotId())); + // the reused stats file should exist. + Assertions.assertThat(new File(statsFileLocation1)).exists(); } @Test @@ -1546,13 +1650,14 @@ public void testRetainFilesOnRetainedBranches() { long tAfterCommits = waitUntilAfter(table.currentSnapshot().timestampMillis()); Set deletedFiles = Sets.newHashSet(); - Set expectedDeletes = Sets.newHashSet(); + Set expectedDeletes = Sets.newHashSet(); // Only deletionA's manifest list and manifests should be removed expectedDeletes.add(deletionA.manifestListLocation()); expectedDeletes.addAll(manifestPaths(deletionA, table.io())); table.expireSnapshots().expireOlderThan(tAfterCommits).deleteWith(deletedFiles::add).commit(); + checkAndAddPartitionStatsFiles(Collections.singleton(deletionA), expectedDeletes); Assert.assertEquals(2, Iterables.size(table.snapshots())); Assert.assertEquals(expectedDeletes, deletedFiles); @@ -1579,6 +1684,7 @@ public void testRetainFilesOnRetainedBranches() { expectedDeletes.add(branchDelete.manifestListLocation()); expectedDeletes.addAll(manifestPaths(branchDelete, table.io())); expectedDeletes.add(FILE_A.path().toString()); + checkAndAddPartitionStatsFiles(Sets.newHashSet(appendA, branchDelete), expectedDeletes); Assert.assertEquals(2, Iterables.size(table.snapshots())); Assert.assertEquals(expectedDeletes, deletedFiles); @@ -1634,4 +1740,37 @@ private String statsFileLocation(String tableLocation) { String statsFileName = "stats-file-" + UUID.randomUUID(); return tableLocation + "/metadata/" + statsFileName; } + + private static PartitionStatisticsFile writePartitionStatsFile( + long snapshotId, long snapshotSequenceNumber, String statsLocation, FileIO fileIO) { + PositionOutputStream positionOutputStream; + try { + positionOutputStream = fileIO.newOutputFile(statsLocation).create(); + positionOutputStream.close(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + + return ImmutableGenericPartitionStatisticsFile.builder() + .snapshotId(snapshotId) + .maxDataSequenceNumber(snapshotSequenceNumber) + .path(statsLocation) + .build(); + } + + private static PartitionStatisticsFile reusePartitionStatsFile( + long snapshotId, PartitionStatisticsFile statisticsFile) { + return ImmutableGenericPartitionStatisticsFile.builder() + .path(statisticsFile.path()) + .maxDataSequenceNumber(statisticsFile.maxDataSequenceNumber()) + .snapshotId(snapshotId) + .build(); + } + + private static void commitPartitionStats(Table table, PartitionStatisticsFile statisticsFile) { + table + .updatePartitionStatistics() + .setPartitionStatistics(statisticsFile.snapshotId(), statisticsFile) + .commit(); + } } diff --git a/core/src/test/java/org/apache/iceberg/TestSetPartitionStatistics.java b/core/src/test/java/org/apache/iceberg/TestSetPartitionStatistics.java new file mode 100644 index 000000000000..9db9e9e22086 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/TestSetPartitionStatistics.java @@ -0,0 +1,137 @@ +/* + * 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 org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TestSetPartitionStatistics extends TableTestBase { + @Parameterized.Parameters(name = "formatVersion = {0}") + public static Object[] parameters() { + return new Object[] {1, 2}; + } + + public TestSetPartitionStatistics(int formatVersion) { + super(formatVersion); + } + + @Test + public void testEmptyUpdateStatistics() { + assertTableMetadataVersion(0); + TableMetadata base = readMetadata(); + + table.updatePartitionStatistics().commit(); + + Assert.assertSame( + "Base metadata should not change when commit is created", base, table.ops().current()); + assertTableMetadataVersion(1); + } + + @Test + public void testEmptyTransactionalUpdateStatistics() { + assertTableMetadataVersion(0); + TableMetadata base = readMetadata(); + + Transaction transaction = table.newTransaction(); + transaction.updatePartitionStatistics().commit(); + transaction.commitTransaction(); + + Assert.assertSame( + "Base metadata should not change when commit is created", base, table.ops().current()); + assertTableMetadataVersion(0); + } + + @Test + public void testUpdateStatistics() { + // Create a snapshot + table.newFastAppend().commit(); + assertTableMetadataVersion(1); + + TableMetadata base = readMetadata(); + long snapshotId = base.currentSnapshot().snapshotId(); + PartitionStatisticsFile partitionStatisticsFile = + ImmutableGenericPartitionStatisticsFile.builder() + .snapshotId(snapshotId) + .path("/some/partition/statistics/file.parquet") + .maxDataSequenceNumber(42L) + .build(); + + table + .updatePartitionStatistics() + .setPartitionStatistics(snapshotId, partitionStatisticsFile) + .commit(); + + TableMetadata metadata = readMetadata(); + assertTableMetadataVersion(2); + Assert.assertEquals( + "Table snapshot should be the same after setting partition statistics file", + snapshotId, + metadata.currentSnapshot().snapshotId()); + Assert.assertEquals( + "Table metadata should have partition statistics files", + ImmutableList.of(partitionStatisticsFile), + metadata.partitionStatisticsFiles()); + } + + @Test + public void testRemoveStatistics() { + // Create a snapshot + table.newFastAppend().commit(); + assertTableMetadataVersion(1); + + TableMetadata base = readMetadata(); + long snapshotId = base.currentSnapshot().snapshotId(); + PartitionStatisticsFile partitionStatisticsFile = + ImmutableGenericPartitionStatisticsFile.builder() + .snapshotId(snapshotId) + .path("/some/partition/statistics/file.parquet") + .maxDataSequenceNumber(42L) + .build(); + + table + .updatePartitionStatistics() + .setPartitionStatistics(snapshotId, partitionStatisticsFile) + .commit(); + + TableMetadata metadata = readMetadata(); + assertTableMetadataVersion(2); + Assert.assertEquals( + "Table metadata should have partition statistics files", + ImmutableList.of(partitionStatisticsFile), + metadata.partitionStatisticsFiles()); + + table.updatePartitionStatistics().removePartitionStatistics(snapshotId).commit(); + + metadata = readMetadata(); + assertTableMetadataVersion(3); + Assert.assertEquals( + "Table metadata should have no partition statistics files", + ImmutableList.of(), + metadata.partitionStatisticsFiles()); + } + + private void assertTableMetadataVersion(int expected) { + Assert.assertEquals( + String.format("Table should be on version %s", expected), expected, (int) version()); + } +} diff --git a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java index 9a75beb59d2a..bbedec42383a 100644 --- a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java +++ b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java @@ -91,6 +91,7 @@ public class TestTableMetadata { public TableOperations ops = new LocalTableOperations(temp); @Test + @SuppressWarnings("MethodLength") public void testJsonConversion() throws Exception { long previousSnapshotId = System.currentTimeMillis() - new Random(1234).nextInt(3600); @@ -144,6 +145,14 @@ public void testJsonConversion() throws Exception { new GenericBlobMetadata( "some-stats", 11L, 2, ImmutableList.of(4), ImmutableMap.of())))); + List partitionStatisticsFiles = + ImmutableList.of( + ImmutableGenericPartitionStatisticsFile.builder() + .snapshotId(11L) + .path("/some/partition/stats/file.parquet") + .maxDataSequenceNumber(42L) + .build()); + TableMetadata expected = new TableMetadata( null, @@ -168,6 +177,7 @@ public void testJsonConversion() throws Exception { ImmutableList.of(), refs, statisticsFiles, + partitionStatisticsFiles, ImmutableList.of()); String asJson = TableMetadataParser.toJson(expected); @@ -234,6 +244,10 @@ public void testJsonConversion() throws Exception { metadata.snapshot(previousSnapshotId).schemaId()); Assert.assertEquals( "Statistics files should match", statisticsFiles, metadata.statisticsFiles()); + Assert.assertEquals( + "Partition statistics files should match", + partitionStatisticsFiles, + metadata.partitionStatisticsFiles()); Assert.assertEquals("Refs map should match", refs, metadata.refs()); } @@ -290,6 +304,7 @@ public void testBackwardCompat() throws Exception { ImmutableList.of(), ImmutableMap.of(), ImmutableList.of(), + ImmutableList.of(), ImmutableList.of()); String asJson = toJsonWithoutSpecAndSchemaList(expected); @@ -431,6 +446,7 @@ public void testInvalidMainBranch() throws IOException { ImmutableList.of(), refs, ImmutableList.of(), + ImmutableList.of(), ImmutableList.of())) .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith("Current snapshot ID does not match main branch"); @@ -475,6 +491,7 @@ public void testMainWithoutCurrent() throws IOException { ImmutableList.of(), refs, ImmutableList.of(), + ImmutableList.of(), ImmutableList.of())) .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith("Current snapshot is not set, but main branch exists"); @@ -514,6 +531,7 @@ public void testBranchSnapshotMissing() { ImmutableList.of(), refs, ImmutableList.of(), + ImmutableList.of(), ImmutableList.of())) .isInstanceOf(IllegalArgumentException.class) .hasMessageEndingWith("does not exist in the existing snapshots list"); @@ -617,6 +635,7 @@ public void testJsonWithPreviousMetadataLog() throws Exception { ImmutableList.copyOf(previousMetadataLog), ImmutableMap.of(), ImmutableList.of(), + ImmutableList.of(), ImmutableList.of()); String asJson = TableMetadataParser.toJson(base); @@ -696,6 +715,7 @@ public void testAddPreviousMetadataRemoveNone() throws IOException { ImmutableList.copyOf(previousMetadataLog), ImmutableMap.of(), ImmutableList.of(), + ImmutableList.of(), ImmutableList.of()); previousMetadataLog.add(latestPreviousMetadata); @@ -793,6 +813,7 @@ public void testAddPreviousMetadataRemoveOne() throws IOException { ImmutableList.copyOf(previousMetadataLog), ImmutableMap.of(), ImmutableList.of(), + ImmutableList.of(), ImmutableList.of()); previousMetadataLog.add(latestPreviousMetadata); @@ -896,6 +917,7 @@ public void testAddPreviousMetadataRemoveMultiple() throws IOException { ImmutableList.copyOf(previousMetadataLog), ImmutableMap.of(), ImmutableList.of(), + ImmutableList.of(), ImmutableList.of()); previousMetadataLog.add(latestPreviousMetadata); @@ -944,6 +966,7 @@ public void testV2UUIDValidation() { ImmutableList.of(), ImmutableMap.of(), ImmutableList.of(), + ImmutableList.of(), ImmutableList.of())) .isInstanceOf(IllegalArgumentException.class) .hasMessage("UUID is required in format v2"); @@ -977,6 +1000,7 @@ public void testVersionValidation() { ImmutableList.of(), ImmutableMap.of(), ImmutableList.of(), + ImmutableList.of(), ImmutableList.of())) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Unsupported format version: v" + unsupportedVersion); @@ -1321,6 +1345,128 @@ public void testRemoveStatistics() { Assert.assertEquals("Statistics file path", "/some/path/to/stats/file2", statisticsFile.path()); } + @Test + public void testPartitionStatistics() { + Schema schema = new Schema(Types.NestedField.required(10, "x", Types.StringType.get())); + + TableMetadata meta = + TableMetadata.newTableMetadata( + schema, PartitionSpec.unpartitioned(), null, ImmutableMap.of()); + Assert.assertEquals( + "Should default to no partition statistics files", + ImmutableList.of(), + meta.partitionStatisticsFiles()); + } + + @Test + public void testSetPartitionStatistics() { + Schema schema = new Schema(Types.NestedField.required(10, "x", Types.StringType.get())); + + TableMetadata meta = + TableMetadata.newTableMetadata( + schema, PartitionSpec.unpartitioned(), null, ImmutableMap.of()); + + TableMetadata withPartitionStatistics = + TableMetadata.buildFrom(meta) + .setPartitionStatistics( + 43, + ImmutableGenericPartitionStatisticsFile.builder() + .snapshotId(43) + .path("/some/path/to/partition/stats/file" + ".parquet") + .maxDataSequenceNumber(42L) + .build()) + .build(); + + Assertions.assertThat(withPartitionStatistics.partitionStatisticsFiles()) + .as("There should be one partition statistics file registered") + .hasSize(1); + PartitionStatisticsFile partitionStatisticsFile = + Iterables.getOnlyElement(withPartitionStatistics.partitionStatisticsFiles()); + Assert.assertEquals("Statistics file snapshot", 43L, partitionStatisticsFile.snapshotId()); + Assert.assertEquals( + "Statistics file path", + "/some/path/to/partition/stats/file.parquet", + partitionStatisticsFile.path()); + Assert.assertEquals( + "Statistics file max data sequence number", + 42L, + partitionStatisticsFile.maxDataSequenceNumber()); + + TableMetadata withStatisticsReplaced = + TableMetadata.buildFrom(withPartitionStatistics) + .setPartitionStatistics( + 43, + ImmutableGenericPartitionStatisticsFile.builder() + .snapshotId(43) + .path("/some/path/to/partition/stats/file2" + ".parquet") + .maxDataSequenceNumber(48L) + .build()) + .build(); + + Assertions.assertThat(withStatisticsReplaced.partitionStatisticsFiles()) + .as("There should be one statistics file registered") + .hasSize(1); + partitionStatisticsFile = + Iterables.getOnlyElement(withStatisticsReplaced.partitionStatisticsFiles()); + Assert.assertEquals("Statistics file snapshot", 43L, partitionStatisticsFile.snapshotId()); + Assert.assertEquals( + "Statistics file path", + "/some/path/to/partition/stats/file2.parquet", + partitionStatisticsFile.path()); + Assert.assertEquals( + "Statistics file max data sequence number", + 48L, + partitionStatisticsFile.maxDataSequenceNumber()); + } + + @Test + public void testRemovePartitionStatistics() { + Schema schema = new Schema(Types.NestedField.required(10, "x", Types.StringType.get())); + + TableMetadata meta = + TableMetadata.buildFrom( + TableMetadata.newTableMetadata( + schema, PartitionSpec.unpartitioned(), null, ImmutableMap.of())) + .setPartitionStatistics( + 43, + ImmutableGenericPartitionStatisticsFile.builder() + .snapshotId(43) + .path("/some/path/to/partition/stats/file1" + ".parquet") + .maxDataSequenceNumber(48L) + .build()) + .setPartitionStatistics( + 44, + ImmutableGenericPartitionStatisticsFile.builder() + .snapshotId(44) + .path("/some/path/to/partition/stats/file2" + ".parquet") + .maxDataSequenceNumber(49L) + .build()) + .build(); + + Assert.assertSame( + "Should detect no partition statistics to remove", + meta, + TableMetadata.buildFrom(meta).removePartitionStatistics(42L).build()); + + TableMetadata withOneRemoved = + TableMetadata.buildFrom(meta).removePartitionStatistics(43).build(); + + Assertions.assertThat(withOneRemoved.partitionStatisticsFiles()) + .as("There should be one partition statistics file retained") + .hasSize(1); + PartitionStatisticsFile partitionStatisticsFile = + Iterables.getOnlyElement(withOneRemoved.partitionStatisticsFiles()); + Assert.assertEquals("Statistics file snapshot", 44L, partitionStatisticsFile.snapshotId()); + Assert.assertEquals( + "Statistics file path", + "/some/path/to/partition/stats/file2.parquet", + partitionStatisticsFile.path()); + Assert.assertEquals( + "Statistics file max data sequence number", + 49L, + partitionStatisticsFile.maxDataSequenceNumber()); + } + @Test public void testParseSchemaIdentifierFields() throws Exception { String data = readTableMetadataInputFile("TableMetadataV2Valid.json"); @@ -1539,6 +1685,23 @@ public void testParseStatisticsFiles() throws Exception { Iterables.getOnlyElement(parsed.statisticsFiles())); } + @Test + public void testParsePartitionStatisticsFiles() throws Exception { + String data = readTableMetadataInputFile("TableMetadataPartitionStatisticsFiles.json"); + TableMetadata parsed = TableMetadataParser.fromJson(data); + Assertions.assertThat(parsed.partitionStatisticsFiles()) + .as("parsed partition statistics files") + .hasSize(1); + Assert.assertEquals( + "parsed partition statistics file", + ImmutableGenericPartitionStatisticsFile.builder() + .snapshotId(3055729675574597004L) + .path("s3://a/b/partition-stats.parquet") + .maxDataSequenceNumber(43L) + .build(), + Iterables.getOnlyElement(parsed.partitionStatisticsFiles())); + } + @Test public void testNoReservedPropertyForTableMetadataCreation() { Schema schema = new Schema(Types.NestedField.required(10, "x", Types.StringType.get())); diff --git a/parquet/src/test/java/org/apache/iceberg/avro/TestParquetReadProjection.java b/core/src/test/java/org/apache/iceberg/avro/TestParquetReadProjection.java similarity index 77% rename from parquet/src/test/java/org/apache/iceberg/avro/TestParquetReadProjection.java rename to core/src/test/java/org/apache/iceberg/avro/TestParquetReadProjection.java index 2df806b1fb9a..1b20e2610ae6 100644 --- a/parquet/src/test/java/org/apache/iceberg/avro/TestParquetReadProjection.java +++ b/core/src/test/java/org/apache/iceberg/avro/TestParquetReadProjection.java @@ -32,7 +32,7 @@ public class TestParquetReadProjection extends TestReadProjection { protected GenericData.Record writeAndRead( String desc, Schema writeSchema, Schema readSchema, GenericData.Record record) throws IOException { - File file = temp.resolve(desc + ".parquet").toFile(); + File file = temp.newFile(desc + ".parquet"); file.delete(); try (FileAppender appender = @@ -45,4 +45,20 @@ protected GenericData.Record writeAndRead( return Iterables.getOnlyElement(records); } + + // Empty struct read is not supported for Parquet + @Override + public void testEmptyStructProjection() throws Exception {} + + @Override + public void testEmptyStructRequiredProjection() throws Exception {} + + @Override + public void testRequiredEmptyStructInRequiredStruct() throws Exception {} + + @Override + public void testEmptyNestedStructProjection() throws Exception {} + + @Override + public void testEmptyNestedStructRequiredProjection() throws Exception {} } diff --git a/core/src/test/java/org/apache/iceberg/hadoop/HadoopTableTestBase.java b/core/src/test/java/org/apache/iceberg/hadoop/HadoopTableTestBase.java index b8f7dcb80ebc..d4b8d17324f9 100644 --- a/core/src/test/java/org/apache/iceberg/hadoop/HadoopTableTestBase.java +++ b/core/src/test/java/org/apache/iceberg/hadoop/HadoopTableTestBase.java @@ -132,7 +132,9 @@ List listManifestFiles() { return Lists.newArrayList( metadataDir.listFiles( (dir, name) -> - !name.startsWith("snap") && Files.getFileExtension(name).equalsIgnoreCase("avro"))); + !name.startsWith("snap") + && !name.startsWith("partition-stats") + && Files.getFileExtension(name).equalsIgnoreCase("avro"))); } List listMetadataJsonFiles() { diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/ParquetWritingTestUtils.java b/core/src/test/java/org/apache/iceberg/parquet/ParquetWritingTestUtils.java similarity index 100% rename from parquet/src/test/java/org/apache/iceberg/parquet/ParquetWritingTestUtils.java rename to core/src/test/java/org/apache/iceberg/parquet/ParquetWritingTestUtils.java diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/TestBloomRowGroupFilter.java b/core/src/test/java/org/apache/iceberg/parquet/TestBloomRowGroupFilter.java similarity index 100% rename from parquet/src/test/java/org/apache/iceberg/parquet/TestBloomRowGroupFilter.java rename to core/src/test/java/org/apache/iceberg/parquet/TestBloomRowGroupFilter.java diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/TestCDHParquetStatistics.java b/core/src/test/java/org/apache/iceberg/parquet/TestCDHParquetStatistics.java similarity index 100% rename from parquet/src/test/java/org/apache/iceberg/parquet/TestCDHParquetStatistics.java rename to core/src/test/java/org/apache/iceberg/parquet/TestCDHParquetStatistics.java diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/TestDictionaryRowGroupFilter.java b/core/src/test/java/org/apache/iceberg/parquet/TestDictionaryRowGroupFilter.java similarity index 100% rename from parquet/src/test/java/org/apache/iceberg/parquet/TestDictionaryRowGroupFilter.java rename to core/src/test/java/org/apache/iceberg/parquet/TestDictionaryRowGroupFilter.java diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquet.java b/core/src/test/java/org/apache/iceberg/parquet/TestParquet.java similarity index 100% rename from parquet/src/test/java/org/apache/iceberg/parquet/TestParquet.java rename to core/src/test/java/org/apache/iceberg/parquet/TestParquet.java diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetDataWriter.java b/core/src/test/java/org/apache/iceberg/parquet/TestParquetDataWriter.java similarity index 100% rename from parquet/src/test/java/org/apache/iceberg/parquet/TestParquetDataWriter.java rename to core/src/test/java/org/apache/iceberg/parquet/TestParquetDataWriter.java diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetDeleteWriters.java b/core/src/test/java/org/apache/iceberg/parquet/TestParquetDeleteWriters.java similarity index 100% rename from parquet/src/test/java/org/apache/iceberg/parquet/TestParquetDeleteWriters.java rename to core/src/test/java/org/apache/iceberg/parquet/TestParquetDeleteWriters.java diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetEncryption.java b/core/src/test/java/org/apache/iceberg/parquet/TestParquetEncryption.java similarity index 100% rename from parquet/src/test/java/org/apache/iceberg/parquet/TestParquetEncryption.java rename to core/src/test/java/org/apache/iceberg/parquet/TestParquetEncryption.java diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetSchemaUtil.java b/core/src/test/java/org/apache/iceberg/parquet/TestParquetSchemaUtil.java similarity index 100% rename from parquet/src/test/java/org/apache/iceberg/parquet/TestParquetSchemaUtil.java rename to core/src/test/java/org/apache/iceberg/parquet/TestParquetSchemaUtil.java diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/TestPruneColumns.java b/core/src/test/java/org/apache/iceberg/parquet/TestPruneColumns.java similarity index 100% rename from parquet/src/test/java/org/apache/iceberg/parquet/TestPruneColumns.java rename to core/src/test/java/org/apache/iceberg/parquet/TestPruneColumns.java diff --git a/core/src/test/java/org/apache/iceberg/partition/stats/TestPartitionStatsUtil.java b/core/src/test/java/org/apache/iceberg/partition/stats/TestPartitionStatsUtil.java new file mode 100644 index 000000000000..83695ae36bf7 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/partition/stats/TestPartitionStatsUtil.java @@ -0,0 +1,129 @@ +/* + * 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.partition.stats; + +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.List; +import java.util.Random; +import java.util.UUID; +import org.apache.iceberg.Files; +import org.apache.iceberg.Partition; +import org.apache.iceberg.PartitionData; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Partitioning; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TestTables; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TestPartitionStatsUtil { + private static final Logger LOG = LoggerFactory.getLogger(TestPartitionStatsUtil.class); + + private static final Schema SCHEMA = + new Schema( + Types.NestedField.required(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "data", Types.StringType.get()), + Types.NestedField.optional(3, "binary", Types.BinaryType.get())); + + @TempDir protected Path temp; + + @Test + public void testPartitionStats() throws Exception { + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("id").build(); + Table testTable = + TestTables.create( + temp.resolve("test_partition_stats").toFile(), + "test_partition_stats", + SCHEMA, + spec, + SortOrder.unsorted(), + 2); + + Schema schema = Partition.icebergSchema(Partitioning.partitionType(testTable.specs().values())); + + ImmutableList.Builder partitionListBuilder = ImmutableList.builder(); + + long seed = System.currentTimeMillis(); + LOG.info("Seed used for random generator is {}", seed); + Random random = new Random(seed); + + for (int i = 0; i < 42; i++) { + PartitionData partitionData = + new PartitionData( + schema.findField(Partition.Column.PARTITION_DATA.name()).type().asStructType()); + partitionData.set(0, random.nextLong()); + + Partition partition = + Partition.builder() + .withPartitionData(partitionData) + .withSpecId(random.nextInt(10)) + .withDataRecordCount(random.nextLong()) + .withDataFileCount(random.nextInt()) + .withDataFileSizeInBytes(1024L * random.nextInt(20)) + .withPosDeleteRecordCount(random.nextLong()) + .withPosDeleteFileCount(random.nextInt()) + .withEqDeleteRecordCount(random.nextLong()) + .withEqDeleteFileCount(random.nextInt()) + .withLastUpdatedAt(random.nextLong()) + .withLastUpdatedSnapshotId(random.nextLong()) + .build(); + + partitionListBuilder.add(partition); + } + List records = partitionListBuilder.build(); + + String dataFileFormatName = + testTable + .properties() + .getOrDefault( + TableProperties.DEFAULT_FILE_FORMAT, TableProperties.DEFAULT_FILE_FORMAT_DEFAULT); + + OutputFile outputFile = + Files.localOutput( + testTable.location() + "/metadata/" + UUID.randomUUID() + "." + dataFileFormatName); + PartitionStatsUtil.writePartitionStatsFile( + records.iterator(), outputFile, testTable.specs().values()); + + Assertions.assertThat(Paths.get(outputFile.location())).exists(); + + List rows; + try (CloseableIterable recordIterator = + PartitionStatsUtil.readPartitionStatsFile( + schema, Files.localInput(outputFile.location()))) { + rows = Lists.newArrayList(recordIterator); + } + + Assertions.assertThat(rows).hasSize(records.size()); + for (int i = 0; i < records.size(); i++) { + Assertions.assertThat(rows.get(i)).isEqualTo(records.get(i)); + } + } +} diff --git a/core/src/test/resources/TableMetadataPartitionStatisticsFiles.json b/core/src/test/resources/TableMetadataPartitionStatisticsFiles.json new file mode 100644 index 000000000000..b4da92619f6e --- /dev/null +++ b/core/src/test/resources/TableMetadataPartitionStatisticsFiles.json @@ -0,0 +1,61 @@ +{ + "format-version": 2, + "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", + "max-data-sequence-number": 43 + } + ], + "snapshot-log": [], + "metadata-log": [] +} \ No newline at end of file diff --git a/docs/java-api.md b/docs/java-api.md index 62b51e096fb2..48b02a6b5515 100644 --- a/docs/java-api.md +++ b/docs/java-api.md @@ -247,7 +247,6 @@ Iceberg table support is organized in library modules: * `iceberg-arrow` is an implementation of the Iceberg type system for reading and writing data stored in Iceberg tables using Apache Arrow as the in-memory data format * `iceberg-aws` contains implementations of the Iceberg API to be used with tables stored on AWS S3 and/or for tables defined using the AWS Glue data catalog * `iceberg-core` contains implementations of the Iceberg API and support for Avro data files, **this is what processing engines should depend on** -* `iceberg-parquet` is an optional module for working with tables backed by Parquet files * `iceberg-orc` is an optional module for working with tables backed by ORC files (*experimental*) * `iceberg-hive-metastore` is an implementation of Iceberg tables backed by the Hive metastore Thrift client diff --git a/flink/v1.15/build.gradle b/flink/v1.15/build.gradle index ff8a8e8fe34a..f8daaa9fc464 100644 --- a/flink/v1.15/build.gradle +++ b/flink/v1.15/build.gradle @@ -29,7 +29,6 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { implementation project(':iceberg-core') api project(':iceberg-data') implementation project(':iceberg-orc') - implementation project(':iceberg-parquet') implementation project(':iceberg-hive-metastore') compileOnly libs.flink115.avro diff --git a/flink/v1.16/build.gradle b/flink/v1.16/build.gradle index 260e7c5bd12b..7f98300dcda8 100644 --- a/flink/v1.16/build.gradle +++ b/flink/v1.16/build.gradle @@ -29,7 +29,6 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { implementation project(':iceberg-core') api project(':iceberg-data') implementation project(':iceberg-orc') - implementation project(':iceberg-parquet') implementation project(':iceberg-hive-metastore') compileOnly libs.flink116.avro diff --git a/flink/v1.17/build.gradle b/flink/v1.17/build.gradle index b597a1bcb0b9..80c412ca830a 100644 --- a/flink/v1.17/build.gradle +++ b/flink/v1.17/build.gradle @@ -29,7 +29,6 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { implementation project(':iceberg-core') api project(':iceberg-data') implementation project(':iceberg-orc') - implementation project(':iceberg-parquet') implementation project(':iceberg-hive-metastore') compileOnly libs.flink117.avro diff --git a/format/spec.md b/format/spec.md index 60c0f99c3f90..ffc66510b308 100644 --- a/format/spec.md +++ b/format/spec.md @@ -671,6 +671,7 @@ Table metadata consists of the following fields: | _optional_ | _required_ | **`default-sort-order-id`**| Default sort order id of the table. Note that this could be used by writers, but is not used when reading because reads use the specs stored in manifest files. | | | _optional_ | **`refs`** | A map of snapshot references. The map keys are the unique snapshot reference names in the table, and the map values are snapshot reference objects. There is always a `main` branch reference pointing to the `current-snapshot-id` even if the `refs` map is null. | | _optional_ | _optional_ | **`statistics`** | A list (optional) of [table statistics](#table-statistics). | +| _optional_ | _optional_ | **`partition-statistics`** | A list (optional) of [partition statistics](#partition-statistics). | For serialization details, see Appendix C. @@ -702,6 +703,45 @@ Blob metadata is a struct with the following fields: | _optional_ | _optional_ | **`properties`** | `map` | Additional properties associated with the statistic. Subset of Blob properties in the Puffin file. | +#### Partition statistics + +Partition statistics files are based on [Partition Statistics file spec](#partition-statistics-file). Partition statistics are informational. A reader can choose to +ignore partition statistics information. Partition statistics support is not required to read the table correctly. A table can contain +many partition statistics files associated with different table snapshots. +A writer can optionally write the partition statistics file during each write operation. If the statistics file is written for the specific snapshot, +it must be accurate and must be registered in the table metadata file to be considered as a valid statistics file for the reader. + +Partition statistics files metadata within `partition-statistics` table metadata field is a struct with the following fields: + +| v1 | v2 | Field name | Type | Description | +|----|----|------------|------|-------------| +| _required_ | _required_ | **`snapshot-id`** | `long` | ID of the Iceberg table's snapshot the partition statistics file is associated with. | +| _required_ | _required_ | **`statistics-file-path`** | `string` | Path of the partition statistics file. See [Partition Statistics file](#partition-statistics-file). | +| _required_ | _required_ | **`max-data-sequence-number`** | `long` | Maximum data sequence number of the Iceberg table's snapshot the partition statistics was computed from. | + +#### Partition Statistics file + +Statistics information for every partition tuple is stored as a row in the **table default format**. +These rows are sorted (in ascending manner with NULL FIRST) based on the first partition column from `partition` +to optimize filtering rows while scanning. +Each unique partition tuple must have exactly one corresponding row, ensuring all partition tuples are present. + +Partition statistics file store the statistics as a struct with the following fields: + +| v1 | v2 | Field id, name | Type | Description | +|----|----|----------------|------|-------------| +| _required_ | _required_ | **`1 partition`** | `struct<..>` | Partition data tuple, schema based on the partition spec output using partition field ids for the struct field ids | +| _required_ | _required_ | **`2 spec_id`** | `int` | Partition spec id | +| _required_ | _required_ | **`3 data_record_count`** | `long` | Count of records in data files | +| _required_ | _required_ | **`4 data_file_count`** | `int` | Count of data files | +| _required_ | _required_ | **`5 total_data_file_size_in_bytes`** | `long` | Total size of data files in bytes | +| _optional_ | _optional_ | **`6 position_delete_record_count`** | `long` | Count of records in position delete files | +| _optional_ | _optional_ | **`7 position_delete_file_count`** | `int` | Count of position delete files | +| _optional_ | _optional_ | **`8 equality_delete_record_count`** | `long` | Count of records in equality delete files | +| _optional_ | _optional_ | **`9 equality_delete_file_count`** | `int` | Count of equality delete files | +| _optional_ | _optional_ | **`10 last_updated_at`** | `timestamptz` | Commit time of snapshot that last updated this partition | +| _optional_ | _optional_ | **`11 last_updated_snapshot_id`** | `long` | ID of snapshot that last updated this partition | + #### Commit Conflict Resolution and Retry When two commits happen at the same time and are based on the same version, only one commit will succeed. In most cases, the failed commit can be applied to the new current version of table metadata and retried. Updates verify the conditions under which they can be applied to a new version and retry if those conditions are met. diff --git a/hive3/build.gradle b/hive3/build.gradle index 67fe1090c5b0..55910749baa9 100644 --- a/hive3/build.gradle +++ b/hive3/build.gradle @@ -40,7 +40,6 @@ project(':iceberg-hive3') { compileOnly project(':iceberg-core') compileOnly project(':iceberg-common') compileOnly project(':iceberg-hive-metastore') - compileOnly project(':iceberg-parquet') compileOnly project(':iceberg-hive3-orc-bundle') compileOnly project(':iceberg-mr') compileOnly project(':iceberg-data') diff --git a/mr/build.gradle b/mr/build.gradle index a7ff1b73f869..7bd906453a91 100644 --- a/mr/build.gradle +++ b/mr/build.gradle @@ -32,7 +32,6 @@ project(':iceberg-mr') { api project(':iceberg-data') implementation project(':iceberg-hive-metastore') implementation project(':iceberg-orc') - implementation project(':iceberg-parquet') compileOnly(libs.hadoop2.client) { exclude group: 'org.apache.avro', module: 'avro' diff --git a/parquet/src/test/java/org/apache/iceberg/TestHelpers.java b/parquet/src/test/java/org/apache/iceberg/TestHelpers.java deleted file mode 100644 index 0e7627cab1f5..000000000000 --- a/parquet/src/test/java/org/apache/iceberg/TestHelpers.java +++ /dev/null @@ -1,85 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg; - -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -import java.util.concurrent.Callable; -import org.apache.avro.AvroRuntimeException; -import org.apache.avro.generic.GenericRecord; -import org.assertj.core.api.AbstractThrowableAssert; - -public class TestHelpers { - - private TestHelpers() {} - - /** - * A convenience method to avoid a large number of @Test(expected=...) tests - * - * @param message A String message to describe this assertion - * @param expected An Exception class that the Runnable should throw - * @param containedInMessage A String that should be contained by the thrown exception's message - * @param callable A Callable that is expected to throw the exception - */ - public static void assertThrows( - String message, - Class expected, - String containedInMessage, - Callable callable) { - AbstractThrowableAssert check = - assertThatThrownBy(callable::call).as(message).isInstanceOf(expected); - if (null != containedInMessage) { - check.hasMessageContaining(containedInMessage); - } - } - - /** - * A convenience method to avoid a large number of @Test(expected=...) tests - * - * @param message A String message to describe this assertion - * @param expected An Exception class that the Runnable should throw - * @param containedInMessage A String that should be contained by the thrown exception's message - * @param runnable A Runnable that is expected to throw the runtime exception - */ - public static void assertThrows( - String message, - Class expected, - String containedInMessage, - Runnable runnable) { - AbstractThrowableAssert check = - assertThatThrownBy(runnable::run).as(message).isInstanceOf(expected); - if (null != containedInMessage) { - check.hasMessageContaining(containedInMessage); - } - } - - /** - * A convenience method to assert if an Avro field is empty - * - * @param record The record to read from - * @param field The name of the field - */ - public static void assertEmptyAvroField(GenericRecord record, String field) { - TestHelpers.assertThrows( - "Not a valid schema field: " + field, - AvroRuntimeException.class, - "Not a valid schema field: " + field, - () -> record.get(field)); - } -} diff --git a/parquet/src/test/java/org/apache/iceberg/avro/TestReadProjection.java b/parquet/src/test/java/org/apache/iceberg/avro/TestReadProjection.java deleted file mode 100644 index 93c6ad05379e..000000000000 --- a/parquet/src/test/java/org/apache/iceberg/avro/TestReadProjection.java +++ /dev/null @@ -1,549 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.avro; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.assertj.core.api.Assertions.within; - -import java.io.IOException; -import java.nio.file.Path; -import java.util.List; -import java.util.Map; -import org.apache.avro.generic.GenericData.Record; -import org.apache.iceberg.Schema; -import org.apache.iceberg.TestHelpers; -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.Maps; -import org.apache.iceberg.types.Types; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.io.TempDir; - -public abstract class TestReadProjection { - protected abstract Record writeAndRead( - String desc, Schema writeSchema, Schema readSchema, Record record) throws IOException; - - @TempDir protected Path temp; - - @Test - public void testFullProjection() throws Exception { - Schema schema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get())); - - Record record = new Record(AvroSchemaUtil.convert(schema, "table")); - record.put("id", 34L); - record.put("data", "test"); - - Record projected = writeAndRead("full_projection", schema, schema, record); - - assertThat((long) projected.get("id")).as("Should contain the correct id value").isEqualTo(34L); - - assertThat(projected.get("data").toString()) - .as("Should contain the correct data value") - .isEqualTo("test"); - } - - @Test - public void testReorderedFullProjection() throws Exception { - Schema schema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get())); - - Record record = new Record(AvroSchemaUtil.convert(schema, "table")); - record.put("id", 34L); - record.put("data", "test"); - - Schema reordered = - new Schema( - Types.NestedField.optional(1, "data", Types.StringType.get()), - Types.NestedField.required(0, "id", Types.LongType.get())); - - Record projected = writeAndRead("full_projection", schema, reordered, record); - - assertThat(projected.get(0).toString()) - .as("Should contain the correct 0 value") - .isEqualTo("test"); - assertThat(projected.get(1)).as("Should contain the correct 1 value").isEqualTo(34L); - } - - @Test - public void testReorderedProjection() throws Exception { - Schema schema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get())); - - Record record = new Record(AvroSchemaUtil.convert(schema, "table")); - record.put("id", 34L); - record.put("data", "test"); - - Schema reordered = - new Schema( - Types.NestedField.optional(2, "missing_1", Types.StringType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get()), - Types.NestedField.optional(3, "missing_2", Types.LongType.get())); - - Record projected = writeAndRead("full_projection", schema, reordered, record); - - assertThat(projected.get(0)).as("Should contain the correct 0 value").isNull(); - assertThat(projected.get(1).toString()) - .as("Should contain the correct 1 value") - .isEqualTo("test"); - assertThat(projected.get(2)).as("Should contain the correct 2 value").isNull(); - } - - @Test - public void testEmptyProjection() throws Exception { - Schema schema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get())); - - Record record = new Record(AvroSchemaUtil.convert(schema, "table")); - record.put("id", 34L); - record.put("data", "test"); - - Record projected = writeAndRead("empty_projection", schema, schema.select(), record); - - assertThat(projected).as("Should read a non-null record").isNotNull(); - // this is expected because there are no values - assertThatThrownBy(() -> projected.get(0)).isInstanceOf(ArrayIndexOutOfBoundsException.class); - } - - @Test - public void testBasicProjection() throws Exception { - Schema writeSchema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get())); - - Record record = new Record(AvroSchemaUtil.convert(writeSchema, "table")); - record.put("id", 34L); - record.put("data", "test"); - - Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); - - Record projected = writeAndRead("basic_projection_id", writeSchema, idOnly, record); - TestHelpers.assertEmptyAvroField(projected, "data"); - assertThat((long) projected.get("id")).as("Should contain the correct id value").isEqualTo(34L); - - Schema dataOnly = new Schema(Types.NestedField.optional(1, "data", Types.StringType.get())); - - projected = writeAndRead("basic_projection_data", writeSchema, dataOnly, record); - - TestHelpers.assertEmptyAvroField(projected, "id"); - assertThat(projected.get("data").toString()) - .as("Should contain the correct data value") - .isEqualTo("test"); - } - - @Test - public void testRename() throws Exception { - Schema writeSchema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get())); - - Record record = new Record(AvroSchemaUtil.convert(writeSchema, "table")); - record.put("id", 34L); - record.put("data", "test"); - - Schema readSchema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "renamed", Types.StringType.get())); - - Record projected = writeAndRead("project_and_rename", writeSchema, readSchema, record); - - assertThat((long) projected.get("id")).as("Should contain the correct id value").isEqualTo(34L); - - assertThat(projected.get("renamed").toString()) - .as("Should contain the correct data/renamed value") - .isEqualTo("test"); - } - - @Test - public void testNestedStructProjection() throws Exception { - Schema writeSchema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional( - 3, - "location", - Types.StructType.of( - Types.NestedField.required(1, "lat", Types.FloatType.get()), - Types.NestedField.required(2, "long", Types.FloatType.get())))); - - Record record = new Record(AvroSchemaUtil.convert(writeSchema, "table")); - record.put("id", 34L); - Record location = - new Record(AvroSchemaUtil.fromOption(record.getSchema().getField("location").schema())); - location.put("lat", 52.995143f); - location.put("long", -1.539054f); - record.put("location", location); - - Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); - - Record projected = writeAndRead("id_only", writeSchema, idOnly, record); - TestHelpers.assertEmptyAvroField(projected, "location"); - assertThat((long) projected.get("id")).as("Should contain the correct id value").isEqualTo(34L); - - Schema latOnly = - new Schema( - Types.NestedField.optional( - 3, - "location", - Types.StructType.of(Types.NestedField.required(1, "lat", Types.FloatType.get())))); - - projected = writeAndRead("latitude_only", writeSchema, latOnly, record); - Record projectedLocation = (Record) projected.get("location"); - TestHelpers.assertEmptyAvroField(projected, "id"); - assertThat(projected.get("location")).as("Should project location").isNotNull(); - TestHelpers.assertEmptyAvroField(projectedLocation, "long"); - assertThat((float) projectedLocation.get("lat")) - .as("Should project latitude") - .isCloseTo(52.995143f, within(0.000001f)); - - Schema longOnly = - new Schema( - Types.NestedField.optional( - 3, - "location", - Types.StructType.of(Types.NestedField.required(2, "long", Types.FloatType.get())))); - - projected = writeAndRead("longitude_only", writeSchema, longOnly, record); - projectedLocation = (Record) projected.get("location"); - TestHelpers.assertEmptyAvroField(projected, "id"); - assertThat(projected.get("location")).as("Should project location").isNotNull(); - TestHelpers.assertEmptyAvroField(projectedLocation, "lat"); - assertThat((float) projectedLocation.get("long")) - .as("Should project longitude") - .isCloseTo(-1.539054f, within(0.000001f)); - - Schema locationOnly = writeSchema.select("location"); - projected = writeAndRead("location_only", writeSchema, locationOnly, record); - projectedLocation = (Record) projected.get("location"); - TestHelpers.assertEmptyAvroField(projected, "id"); - assertThat(projected.get("location")).as("Should project location").isNotNull(); - assertThat((float) projectedLocation.get("lat")) - .as("Should project latitude") - .isCloseTo(52.995143f, within(0.000001f)); - - assertThat((float) projectedLocation.get("long")) - .as("Should project longitude") - .isCloseTo(-1.539054f, within(0.000001f)); - } - - @Test - public void testMapProjection() throws IOException { - Schema writeSchema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional( - 5, - "properties", - Types.MapType.ofOptional(6, 7, Types.StringType.get(), Types.StringType.get()))); - - Map properties = ImmutableMap.of("a", "A", "b", "B"); - - Record record = new Record(AvroSchemaUtil.convert(writeSchema, "table")); - record.put("id", 34L); - record.put("properties", properties); - - Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); - - Record projected = writeAndRead("id_only", writeSchema, idOnly, record); - assertThat((long) projected.get("id")).as("Should contain the correct id value").isEqualTo(34L); - TestHelpers.assertEmptyAvroField(projected, "properties"); - - Schema keyOnly = writeSchema.select("properties.key"); - projected = writeAndRead("key_only", writeSchema, keyOnly, record); - TestHelpers.assertEmptyAvroField(projected, "id"); - assertThat(toStringMap((Map) projected.get("properties"))) - .as("Should project entire map") - .isEqualTo(properties); - - Schema valueOnly = writeSchema.select("properties.value"); - projected = writeAndRead("value_only", writeSchema, valueOnly, record); - TestHelpers.assertEmptyAvroField(projected, "id"); - assertThat(toStringMap((Map) projected.get("properties"))) - .as("Should project entire map") - .isEqualTo(properties); - - Schema mapOnly = writeSchema.select("properties"); - projected = writeAndRead("map_only", writeSchema, mapOnly, record); - TestHelpers.assertEmptyAvroField(projected, "id"); - assertThat(toStringMap((Map) projected.get("properties"))) - .as("Should project entire map") - .isEqualTo(properties); - } - - private Map toStringMap(Map map) { - Map stringMap = Maps.newHashMap(); - for (Map.Entry entry : map.entrySet()) { - if (entry.getValue() instanceof CharSequence) { - stringMap.put(entry.getKey().toString(), entry.getValue().toString()); - } else { - stringMap.put(entry.getKey().toString(), entry.getValue()); - } - } - return stringMap; - } - - @Test - public void testMapOfStructsProjection() throws IOException { - Schema writeSchema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional( - 5, - "locations", - Types.MapType.ofOptional( - 6, - 7, - Types.StringType.get(), - Types.StructType.of( - Types.NestedField.required(1, "lat", Types.FloatType.get()), - Types.NestedField.required(2, "long", Types.FloatType.get()))))); - - Record record = new Record(AvroSchemaUtil.convert(writeSchema, "table")); - record.put("id", 34L); - Record l1 = - new Record( - AvroSchemaUtil.fromOption( - AvroSchemaUtil.fromOption(record.getSchema().getField("locations").schema()) - .getValueType())); - l1.put("lat", 53.992811f); - l1.put("long", -1.542616f); - Record l2 = new Record(l1.getSchema()); - l2.put("lat", 52.995143f); - l2.put("long", -1.539054f); - record.put("locations", ImmutableMap.of("L1", l1, "L2", l2)); - - Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); - - Record projected = writeAndRead("id_only", writeSchema, idOnly, record); - assertThat((long) projected.get("id")).as("Should contain the correct id value").isEqualTo(34L); - TestHelpers.assertEmptyAvroField(projected, "locations"); - - projected = writeAndRead("all_locations", writeSchema, writeSchema.select("locations"), record); - TestHelpers.assertEmptyAvroField(projected, "id"); - assertThat(toStringMap((Map) projected.get("locations"))) - .as("Should project locations map") - .isEqualTo(record.get("locations")); - - projected = writeAndRead("lat_only", writeSchema, writeSchema.select("locations.lat"), record); - TestHelpers.assertEmptyAvroField(projected, "id"); - Map locations = toStringMap((Map) projected.get("locations")); - assertThat(locations).as("Should project locations map").isNotNull(); - assertThat(locations.keySet()).as("Should contain L1 and L2").containsExactly("L1", "L2"); - Record projectedL1 = (Record) locations.get("L1"); - assertThat(projectedL1).as("L1 should not be null").isNotNull(); - assertThat((float) projectedL1.get("lat")) - .as("L1 should contain lat") - .isCloseTo(53.992811f, within(0.000001f)); - - TestHelpers.assertEmptyAvroField(projectedL1, "long"); - Record projectedL2 = (Record) locations.get("L2"); - assertThat(projectedL2).as("L2 should not be null").isNotNull(); - assertThat((float) projectedL2.get("lat")) - .as("L2 should contain lat") - .isCloseTo(52.995143f, within(0.000001f)); - - TestHelpers.assertEmptyAvroField(projectedL2, "long"); - - projected = - writeAndRead("long_only", writeSchema, writeSchema.select("locations.long"), record); - TestHelpers.assertEmptyAvroField(projected, "id"); - locations = toStringMap((Map) projected.get("locations")); - assertThat(locations).as("Should project locations map").isNotNull(); - assertThat(locations.keySet()).as("Should contain L1 and L2").containsExactly("L1", "L2"); - projectedL1 = (Record) locations.get("L1"); - assertThat(projectedL1).as("L1 should not be null").isNotNull(); - TestHelpers.assertEmptyAvroField(projectedL1, "lat"); - assertThat((float) projectedL1.get("long")) - .as("L1 should contain long") - .isCloseTo(-1.542616f, within(0.000001f)); - - projectedL2 = (Record) locations.get("L2"); - assertThat(projectedL2).as("L2 should not be null").isNotNull(); - TestHelpers.assertEmptyAvroField(projectedL2, "lat"); - assertThat((float) projectedL2.get("long")) - .as("L2 should contain long") - .isCloseTo(-1.539054f, within(0.000001f)); - - Schema latitiudeRenamed = - new Schema( - Types.NestedField.optional( - 5, - "locations", - Types.MapType.ofOptional( - 6, - 7, - Types.StringType.get(), - Types.StructType.of( - Types.NestedField.required(1, "latitude", Types.FloatType.get()))))); - - projected = writeAndRead("latitude_renamed", writeSchema, latitiudeRenamed, record); - TestHelpers.assertEmptyAvroField(projected, "id"); - locations = toStringMap((Map) projected.get("locations")); - assertThat(locations).as("Should project locations map").isNotNull(); - assertThat(locations.keySet()).as("Should contain L1 and L2").containsExactly("L1", "L2"); - projectedL1 = (Record) locations.get("L1"); - assertThat(projectedL1).as("L1 should not be null").isNotNull(); - assertThat((float) projectedL1.get("latitude")) - .as("L1 should contain latitude") - .isCloseTo(53.992811f, within(0.000001f)); - TestHelpers.assertEmptyAvroField(projectedL1, "lat"); - TestHelpers.assertEmptyAvroField(projectedL1, "long"); - projectedL2 = (Record) locations.get("L2"); - assertThat(projectedL2).as("L2 should not be null").isNotNull(); - assertThat((float) projectedL2.get("latitude")) - .as("L2 should contain latitude") - .isCloseTo(52.995143f, within(0.000001f)); - - TestHelpers.assertEmptyAvroField(projectedL2, "lat"); - TestHelpers.assertEmptyAvroField(projectedL2, "long"); - } - - @Test - public void testListProjection() throws IOException { - Schema writeSchema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional( - 10, "values", Types.ListType.ofOptional(11, Types.LongType.get()))); - - List values = ImmutableList.of(56L, 57L, 58L); - - Record record = new Record(AvroSchemaUtil.convert(writeSchema, "table")); - record.put("id", 34L); - record.put("values", values); - - Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); - - Record projected = writeAndRead("id_only", writeSchema, idOnly, record); - assertThat((long) projected.get("id")).as("Should contain the correct id value").isEqualTo(34L); - TestHelpers.assertEmptyAvroField(projected, "values"); - - Schema elementOnly = writeSchema.select("values.element"); - projected = writeAndRead("element_only", writeSchema, elementOnly, record); - TestHelpers.assertEmptyAvroField(projected, "id"); - assertThat(projected.get("values")).as("Should project entire list").isEqualTo(values); - - Schema listOnly = writeSchema.select("values"); - projected = writeAndRead("list_only", writeSchema, listOnly, record); - TestHelpers.assertEmptyAvroField(projected, "id"); - assertThat(projected.get("values")).as("Should project entire list").isEqualTo(values); - } - - @Test - @SuppressWarnings("unchecked") - public void testListOfStructsProjection() throws IOException { - Schema writeSchema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional( - 22, - "points", - Types.ListType.ofOptional( - 21, - Types.StructType.of( - Types.NestedField.required(19, "x", Types.IntegerType.get()), - Types.NestedField.optional(18, "y", Types.IntegerType.get()))))); - - Record record = new Record(AvroSchemaUtil.convert(writeSchema, "table")); - record.put("id", 34L); - Record p1 = - new Record( - AvroSchemaUtil.fromOption( - AvroSchemaUtil.fromOption(record.getSchema().getField("points").schema()) - .getElementType())); - p1.put("x", 1); - p1.put("y", 2); - Record p2 = new Record(p1.getSchema()); - p2.put("x", 3); - p2.put("y", null); - record.put("points", ImmutableList.of(p1, p2)); - - Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); - - Record projected = writeAndRead("id_only", writeSchema, idOnly, record); - assertThat((long) projected.get("id")).as("Should contain the correct id value").isEqualTo(34L); - TestHelpers.assertEmptyAvroField(projected, "points"); - - projected = writeAndRead("all_points", writeSchema, writeSchema.select("points"), record); - TestHelpers.assertEmptyAvroField(projected, "id"); - assertThat(projected.get("points")) - .as("Should project points list") - .isEqualTo(record.get("points")); - - projected = writeAndRead("x_only", writeSchema, writeSchema.select("points.x"), record); - TestHelpers.assertEmptyAvroField(projected, "id"); - assertThat(projected.get("points")).as("Should project points list").isNotNull(); - List points = (List) projected.get("points"); - assertThat(points).as("Should read 2 points").hasSize(2); - Record projectedP1 = points.get(0); - assertThat((int) projectedP1.get("x")).as("Should project x").isEqualTo(1); - TestHelpers.assertEmptyAvroField(projectedP1, "y"); - Record projectedP2 = points.get(1); - assertThat((int) projectedP2.get("x")).as("Should project x").isEqualTo(3); - TestHelpers.assertEmptyAvroField(projectedP2, "y"); - - projected = writeAndRead("y_only", writeSchema, writeSchema.select("points.y"), record); - TestHelpers.assertEmptyAvroField(projected, "id"); - assertThat(projected.get("points")).as("Should project points list").isNotNull(); - points = (List) projected.get("points"); - assertThat(points).as("Should read 2 points").hasSize(2); - projectedP1 = points.get(0); - TestHelpers.assertEmptyAvroField(projectedP1, "x"); - assertThat((int) projectedP1.get("y")).as("Should project y").isEqualTo(2); - projectedP2 = points.get(1); - TestHelpers.assertEmptyAvroField(projectedP2, "x"); - assertThat(projectedP2.get("y")).as("Should project null y").isNull(); - - Schema yRenamed = - new Schema( - Types.NestedField.optional( - 22, - "points", - Types.ListType.ofOptional( - 21, - Types.StructType.of( - Types.NestedField.optional(18, "z", Types.IntegerType.get()))))); - - projected = writeAndRead("y_renamed", writeSchema, yRenamed, record); - TestHelpers.assertEmptyAvroField(projected, "id"); - assertThat(projected.get("points")).as("Should project points list").isNotNull(); - points = (List) projected.get("points"); - assertThat(points).as("Should read 2 points").hasSize(2); - projectedP1 = points.get(0); - TestHelpers.assertEmptyAvroField(projectedP1, "x"); - TestHelpers.assertEmptyAvroField(projectedP1, "y"); - assertThat((int) projectedP1.get("z")).as("Should project z").isEqualTo(2); - projectedP2 = points.get(1); - TestHelpers.assertEmptyAvroField(projectedP2, "x"); - TestHelpers.assertEmptyAvroField(projectedP2, "y"); - assertThat(projectedP2.get("z")).as("Should project null z").isNull(); - } -} diff --git a/settings.gradle b/settings.gradle index f557712cf72b..46141732783e 100644 --- a/settings.gradle +++ b/settings.gradle @@ -29,7 +29,6 @@ include 'azure' include 'azure-bundle' include 'orc' include 'arrow' -include 'parquet' include 'bundled-guava' include 'spark' include 'pig' @@ -52,7 +51,6 @@ project(':azure').name = 'iceberg-azure' project(':azure-bundle').name = 'iceberg-azure-bundle' project(':orc').name = 'iceberg-orc' project(':arrow').name = 'iceberg-arrow' -project(':parquet').name = 'iceberg-parquet' project(':bundled-guava').name = 'iceberg-bundled-guava' project(':spark').name = 'iceberg-spark' project(':pig').name = 'iceberg-pig' diff --git a/spark/v3.1/build.gradle b/spark/v3.1/build.gradle index 23ba2d02df74..4d5e32a2b1c0 100644 --- a/spark/v3.1/build.gradle +++ b/spark/v3.1/build.gradle @@ -57,7 +57,6 @@ project(':iceberg-spark:iceberg-spark-3.1_2.12') { implementation project(':iceberg-core') implementation project(':iceberg-data') implementation project(':iceberg-orc') - implementation project(':iceberg-parquet') implementation project(':iceberg-arrow') compileOnly libs.errorprone.annotations diff --git a/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/actions/BaseDeleteReachableFilesSparkAction.java b/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/actions/BaseDeleteReachableFilesSparkAction.java index bba3a6c5cfd9..22eb50e6a17c 100644 --- a/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/actions/BaseDeleteReachableFilesSparkAction.java +++ b/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/actions/BaseDeleteReachableFilesSparkAction.java @@ -151,7 +151,7 @@ protected Dataset buildOtherMetadataFileDF(Table table) { List otherMetadataFiles = Lists.newArrayList(); otherMetadataFiles.addAll(ReachableFileUtil.metadataFileLocations(table, true)); otherMetadataFiles.add(ReachableFileUtil.versionHintLocation(table)); - otherMetadataFiles.addAll(ReachableFileUtil.statisticsFilesLocations(table)); + otherMetadataFiles.addAll(ReachableFileUtil.allStatisticsFilesLocations(table)); return spark().createDataset(otherMetadataFiles, Encoders.STRING()).toDF("file_path"); } diff --git a/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java b/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java index f47d113e1364..ca34ef652126 100644 --- a/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java +++ b/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java @@ -146,7 +146,7 @@ protected Dataset buildOtherMetadataFileDF(Table table) { List otherMetadataFiles = Lists.newArrayList(); otherMetadataFiles.addAll(ReachableFileUtil.metadataFileLocations(table, false)); otherMetadataFiles.add(ReachableFileUtil.versionHintLocation(table)); - otherMetadataFiles.addAll(ReachableFileUtil.statisticsFilesLocations(table)); + otherMetadataFiles.addAll(ReachableFileUtil.allStatisticsFilesLocations(table)); return spark.createDataset(otherMetadataFiles, Encoders.STRING()).toDF("file_path"); } diff --git a/spark/v3.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java b/spark/v3.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java index 7124c51ddd3d..bbb055be3628 100644 --- a/spark/v3.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java +++ b/spark/v3.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java @@ -48,6 +48,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.SparkTestBase; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.PropertyUtil; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -124,10 +125,16 @@ private void checkRemoveFilesResults( "Incorrect number of manifest lists deleted", expectedManifestListsDeleted, results.deletedManifestListsCount()); + long expected = expectedOtherFilesDeleted; + if (PropertyUtil.propertyAsBoolean( + table.properties(), + TableProperties.PARTITION_STATS_ENABLED, + TableProperties.PARTITION_STATS_ENABLED_DEFAULT)) { + // one partition stats file per snapshot (or manifestList) + expected += expectedManifestListsDeleted; + } Assert.assertEquals( - "Incorrect number of other lists deleted", - expectedOtherFilesDeleted, - results.deletedOtherFilesCount()); + "Incorrect number of other lists deleted", expected, results.deletedOtherFilesCount()); } @Test diff --git a/spark/v3.1/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java b/spark/v3.1/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java index 9ca073ecf338..8742ac64d8a9 100644 --- a/spark/v3.1/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java +++ b/spark/v3.1/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java @@ -202,7 +202,14 @@ public void testSparkCanReadUnknownTransform() throws IOException { manifestWriter.close(); } + table.updateProperties().set(TableProperties.PARTITION_STATS_ENABLED, "false").commit(); table.newFastAppend().appendManifest(manifestWriter.toManifestFile()).commit(); + table + .updateProperties() + .set( + TableProperties.PARTITION_STATS_ENABLED, + String.valueOf(TableProperties.PARTITION_STATS_ENABLED_DEFAULT)) + .commit(); Dataset df = spark.read().format("iceberg").load(location.toString()); diff --git a/spark/v3.2/build.gradle b/spark/v3.2/build.gradle index c4df5015458c..fa6d7ed1b1cd 100644 --- a/spark/v3.2/build.gradle +++ b/spark/v3.2/build.gradle @@ -57,7 +57,6 @@ project(":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}") { implementation project(':iceberg-core') implementation project(':iceberg-data') implementation project(':iceberg-orc') - implementation project(':iceberg-parquet') implementation project(':iceberg-arrow') implementation "org.scala-lang.modules:scala-collection-compat_${scalaVersion}:${libs.versions.scala.collection.compat.get()}" @@ -152,7 +151,6 @@ project(":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVer } testImplementation project(path: ':iceberg-data') - testImplementation project(path: ':iceberg-parquet') testImplementation project(path: ':iceberg-hive-metastore') testImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') testImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts') diff --git a/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java b/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java index efb3d43668f1..79630db6a444 100644 --- a/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java +++ b/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java @@ -22,6 +22,7 @@ import java.io.File; import java.io.IOException; +import java.io.UncheckedIOException; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.sql.Timestamp; @@ -36,11 +37,14 @@ import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.GenericBlobMetadata; import org.apache.iceberg.GenericStatisticsFile; +import org.apache.iceberg.ImmutableGenericPartitionStatisticsFile; +import org.apache.iceberg.PartitionStatisticsFile; import org.apache.iceberg.Snapshot; import org.apache.iceberg.StatisticsFile; import org.apache.iceberg.Table; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.PositionOutputStream; import org.apache.iceberg.puffin.Blob; import org.apache.iceberg.puffin.Puffin; import org.apache.iceberg.puffin.PuffinWriter; @@ -467,6 +471,18 @@ public void testExpireSnapshotsWithStatisticFiles() throws Exception { table.io()); table.updateStatistics().setStatistics(statisticsFile1.snapshotId(), statisticsFile1).commit(); + String partitionStatsFileLocation1 = statsFileLocation(table.location()); + PartitionStatisticsFile partitionStatisticsFile1 = + writePartitionStatsFile( + table.currentSnapshot().snapshotId(), + table.currentSnapshot().sequenceNumber(), + partitionStatsFileLocation1, + table.io()); + table + .updatePartitionStatistics() + .setPartitionStatistics(partitionStatisticsFile1.snapshotId(), partitionStatisticsFile1) + .commit(); + sql("INSERT INTO %s SELECT 20, 'def'", tableName); table.refresh(); String statsFileLocation2 = statsFileLocation(table.location()); @@ -478,6 +494,18 @@ public void testExpireSnapshotsWithStatisticFiles() throws Exception { table.io()); table.updateStatistics().setStatistics(statisticsFile2.snapshotId(), statisticsFile2).commit(); + String partitionStatsFileLocation2 = statsFileLocation(table.location()); + PartitionStatisticsFile partitionStatisticsFile2 = + writePartitionStatsFile( + table.currentSnapshot().snapshotId(), + table.currentSnapshot().sequenceNumber(), + partitionStatsFileLocation2, + table.io()); + table + .updatePartitionStatistics() + .setPartitionStatistics(partitionStatisticsFile2.snapshotId(), partitionStatisticsFile2) + .commit(); + waitUntilAfter(table.currentSnapshot().timestampMillis()); Timestamp currentTimestamp = Timestamp.from(Instant.ofEpochMilli(System.currentTimeMillis())); @@ -485,7 +513,9 @@ public void testExpireSnapshotsWithStatisticFiles() throws Exception { sql( "CALL %s.system.expire_snapshots(older_than => TIMESTAMP '%s',table => '%s')", catalogName, currentTimestamp, tableIdent); - Assertions.assertThat(output.get(0)[5]).as("should be 1 deleted statistics file").isEqualTo(1L); + Assertions.assertThat(output.get(0)[5]) + .as("should be 1 deleted statistics file and 1 partition statistics file") + .isEqualTo(2L); table.refresh(); List statsWithSnapshotId1 = @@ -511,6 +541,36 @@ public void testExpireSnapshotsWithStatisticFiles() throws Exception { Assertions.assertThat(new File(statsFileLocation2)) .as("Statistics file should exist for snapshot %s", statisticsFile2.snapshotId()) .exists(); + + List partitionStatsWithSnapshotId1 = + table.partitionStatisticsFiles().stream() + .filter( + statisticsFile -> + statisticsFile.snapshotId() == partitionStatisticsFile1.snapshotId()) + .collect(Collectors.toList()); + Assertions.assertThat(partitionStatsWithSnapshotId1) + .as( + "Partition statistics file entry in TableMetadata should be deleted for the snapshot %s", + partitionStatisticsFile1.snapshotId()) + .isEmpty(); + Assertions.assertThat(table.partitionStatisticsFiles()) + .as( + "Partition statistics file entry in TableMetadata should be present for the snapshot %s", + partitionStatisticsFile2.snapshotId()) + .extracting(PartitionStatisticsFile::snapshotId) + .containsExactly(partitionStatisticsFile2.snapshotId()); + + Assertions.assertThat(new File(partitionStatsFileLocation1)) + .as( + "Partition statistics file should not exist for snapshot %s", + partitionStatisticsFile1.snapshotId()) + .doesNotExist(); + + Assertions.assertThat(new File(partitionStatsFileLocation2)) + .as( + "Partition statistics file should exist for snapshot %s", + partitionStatisticsFile2.snapshotId()) + .exists(); } private StatisticsFile writeStatsFile( @@ -541,4 +601,20 @@ private String statsFileLocation(String tableLocation) { String statsFileName = "stats-file-" + UUID.randomUUID(); return tableLocation.replaceFirst("file:", "") + "/metadata/" + statsFileName; } + + private PartitionStatisticsFile writePartitionStatsFile( + long snapshotId, long snapshotSequenceNumber, String statsLocation, FileIO fileIO) { + PositionOutputStream positionOutputStream; + try { + positionOutputStream = fileIO.newOutputFile(statsLocation).create(); + positionOutputStream.close(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + return ImmutableGenericPartitionStatisticsFile.builder() + .snapshotId(snapshotId) + .maxDataSequenceNumber(snapshotSequenceNumber) + .path(statsLocation) + .build(); + } } diff --git a/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java b/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java index 3c007c6214c2..8ead8f33387f 100644 --- a/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java +++ b/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java @@ -31,7 +31,6 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Consumer; -import java.util.function.Predicate; import java.util.function.Supplier; import org.apache.iceberg.AllManifestsTable; import org.apache.iceberg.BaseTable; @@ -44,7 +43,6 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.ReachableFileUtil; import org.apache.iceberg.StaticTableOperations; -import org.apache.iceberg.StatisticsFile; import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.exceptions.NotFoundException; @@ -204,14 +202,8 @@ protected Dataset manifestListDS(Table table, Set snapshotIds) { } protected Dataset statisticsFileDS(Table table, Set snapshotIds) { - Predicate predicate; - if (snapshotIds == null) { - predicate = statisticsFile -> true; - } else { - predicate = statisticsFile -> snapshotIds.contains(statisticsFile.snapshotId()); - } - - List statisticsFiles = ReachableFileUtil.statisticsFilesLocations(table, predicate); + List statisticsFiles = + ReachableFileUtil.allStatisticsFilesLocations(table, snapshotIds); return toFileInfoDS(statisticsFiles, STATISTICS_FILES); } @@ -227,7 +219,7 @@ private Dataset otherMetadataFileDS(Table table, boolean recursive) { List otherMetadataFiles = Lists.newArrayList(); otherMetadataFiles.addAll(ReachableFileUtil.metadataFileLocations(table, recursive)); otherMetadataFiles.add(ReachableFileUtil.versionHintLocation(table)); - otherMetadataFiles.addAll(ReachableFileUtil.statisticsFilesLocations(table)); + otherMetadataFiles.addAll(ReachableFileUtil.allStatisticsFilesLocations(table)); return toFileInfoDS(otherMetadataFiles, OTHERS); } diff --git a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java index 0e333f216c26..88547052441b 100644 --- a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java +++ b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java @@ -51,6 +51,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.SparkTestBase; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.PropertyUtil; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -153,10 +154,16 @@ private void checkRemoveFilesResults( "Incorrect number of manifest lists deleted", expectedManifestListsDeleted, results.deletedManifestListsCount()); + long expected = expectedOtherFilesDeleted; + if (PropertyUtil.propertyAsBoolean( + table.properties(), + TableProperties.PARTITION_STATS_ENABLED, + TableProperties.PARTITION_STATS_ENABLED_DEFAULT)) { + // one partition stats file per snapshot (or manifestList) + expected += expectedManifestListsDeleted; + } Assert.assertEquals( - "Incorrect number of other lists deleted", - expectedOtherFilesDeleted, - results.deletedOtherFilesCount()); + "Incorrect number of other lists deleted", expected, results.deletedOtherFilesCount()); } @Test diff --git a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java index ae5cdc3468eb..951d96cd49f1 100644 --- a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java +++ b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java @@ -22,6 +22,7 @@ import java.io.File; import java.io.IOException; +import java.util.Collections; import java.util.List; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; @@ -56,6 +57,7 @@ import org.apache.iceberg.spark.SparkTestBase; import org.apache.iceberg.spark.data.TestHelpers; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.PropertyUtil; import org.apache.spark.sql.Dataset; import org.junit.Assert; import org.junit.Before; @@ -619,7 +621,7 @@ public void testWithExpiringDanglingStageCommit() { checkExpirationResults(1L, 0L, 0L, 1L, 2L, result); - Set expectedDeletes = Sets.newHashSet(); + Set expectedDeletes = Sets.newHashSet(); expectedDeletes.add(snapshotA.manifestListLocation()); // Files should be deleted of dangling staged snapshot @@ -641,6 +643,7 @@ public void testWithExpiringDanglingStageCommit() { expectedDeletes.add(file.path()); } }); + checkAndAddPartitionStatsFiles(Sets.newHashSet(snapshotA, snapshotB), expectedDeletes); Assert.assertSame( "Files deleted count should be expected", expectedDeletes.size(), deletedFiles.size()); // Take the diff @@ -800,10 +803,9 @@ public void testExpireOlderThan() { table.currentSnapshot().snapshotId()); Assert.assertNull( "Expire should remove the oldest snapshot", table.snapshot(firstSnapshot.snapshotId())); - Assert.assertEquals( - "Should remove only the expired manifest list location", - Sets.newHashSet(firstSnapshot.manifestListLocation()), - deletedFiles); + Set expected = Sets.newHashSet(firstSnapshot.manifestListLocation()); + checkAndAddPartitionStatsFiles(Collections.singleton(firstSnapshot), expected); + Assert.assertEquals("Should remove expected files", expected, deletedFiles); checkExpirationResults(0, 0, 0, 0, 1, result); } @@ -853,8 +855,7 @@ public void testExpireOlderThanWithDelete() { "Expire should remove the second oldest snapshot", table.snapshot(secondSnapshot.snapshotId())); - Assert.assertEquals( - "Should remove expired manifest lists and deleted data file", + Set expected = Sets.newHashSet( firstSnapshot.manifestListLocation(), // snapshot expired firstSnapshot @@ -866,7 +867,11 @@ public void testExpireOlderThanWithDelete() { .allManifests(table.io()) .get(0) .path(), // manifest contained only deletes, was dropped - FILE_A.path()), // deleted + FILE_A.path()); + checkAndAddPartitionStatsFiles(Sets.newHashSet(firstSnapshot, secondSnapshot), expected); + Assert.assertEquals( + "Should remove expected files", + expected, // deleted deletedFiles); checkExpirationResults(1, 0, 0, 2, 2, result); @@ -926,8 +931,7 @@ public void testExpireOlderThanWithDeleteInMergedManifests() { "Expire should remove the second oldest snapshot", table.snapshot(secondSnapshot.snapshotId())); - Assert.assertEquals( - "Should remove expired manifest lists and deleted data file", + Set expected = Sets.newHashSet( firstSnapshot.manifestListLocation(), // snapshot expired firstSnapshot @@ -935,7 +939,11 @@ public void testExpireOlderThanWithDeleteInMergedManifests() { .get(0) .path(), // manifest was rewritten for delete secondSnapshot.manifestListLocation(), // snapshot expired - FILE_A.path()), // deleted + FILE_A.path()); + checkAndAddPartitionStatsFiles(Sets.newHashSet(firstSnapshot, secondSnapshot), expected); + Assert.assertEquals( + "Should remove expected files", + expected, // deleted deletedFiles); checkExpirationResults(1, 0, 0, 1, 2, result); @@ -988,12 +996,14 @@ public void testExpireOlderThanWithRollback() { Assert.assertNull( "Expire should remove the orphaned snapshot", table.snapshot(secondSnapshot.snapshotId())); - Assert.assertEquals( - "Should remove expired manifest lists and reverted appended data file", + Set expected = Sets.newHashSet( secondSnapshot.manifestListLocation(), // snapshot expired - Iterables.getOnlyElement(secondSnapshotManifests) - .path()), // manifest is no longer referenced + Iterables.getOnlyElement(secondSnapshotManifests).path()); + checkAndAddPartitionStatsFiles(Collections.singleton(secondSnapshot), expected); + Assert.assertEquals( + "Should remove expected files", + expected, // manifest is no longer referenced deletedFiles); checkExpirationResults(0, 0, 0, 1, 1, result); @@ -1043,13 +1053,16 @@ public void testExpireOlderThanWithRollbackAndMergedManifests() { Assert.assertNull( "Expire should remove the orphaned snapshot", table.snapshot(secondSnapshot.snapshotId())); - Assert.assertEquals( - "Should remove expired manifest lists and reverted appended data file", + Set expected = Sets.newHashSet( secondSnapshot.manifestListLocation(), // snapshot expired Iterables.getOnlyElement(secondSnapshotManifests) .path(), // manifest is no longer referenced - FILE_B.path()), // added, but rolled back + FILE_B.path()); + checkAndAddPartitionStatsFiles(Collections.singleton(secondSnapshot), expected); + Assert.assertEquals( + "Should remove expected files", + expected, // added, but rolled back deletedFiles); checkExpirationResults(1, 0, 0, 1, 1, result); @@ -1092,7 +1105,7 @@ public void testExpireOlderThanWithDeleteFile() { .deleteWith(deletedFiles::add) .execute(); - Set expectedDeletes = + Set expectedDeletes = Sets.newHashSet( firstSnapshot.manifestListLocation(), secondSnapshot.manifestListLocation(), @@ -1113,6 +1126,9 @@ public void testExpireOlderThanWithDeleteFile() { .map(ManifestFile::path) .map(CharSequence::toString) .collect(Collectors.toSet())); + checkAndAddPartitionStatsFiles( + Sets.newHashSet(firstSnapshot, secondSnapshot, thirdSnapshot, fourthSnapshot), + expectedDeletes); Assert.assertEquals( "Should remove expired manifest lists and deleted data file", @@ -1167,13 +1183,12 @@ public void testExpireAction() { Assert.assertNull( "Should remove the oldest snapshot", table.snapshot(firstSnapshot.snapshotId())); - Assert.assertEquals("Pending deletes should contain one row", 1, pending.size()); - Assert.assertEquals( - "Pending delete should be the expired manifest list location", - firstSnapshot.manifestListLocation(), - pending.get(0).getPath()); + Set expected = Sets.newHashSet(firstSnapshot.manifestListLocation()); + checkAndAddPartitionStatsFiles(Collections.singleton(firstSnapshot), expected); Assert.assertEquals( - "Pending delete should be a manifest list", "Manifest List", pending.get(0).getType()); + "Pending deletes should match", + expected, + pending.stream().map(FileInfo::getPath).collect(Collectors.toSet())); Assert.assertEquals("Should not delete any files", 0, deletedFiles.size()); @@ -1246,10 +1261,17 @@ public void testExpireAfterExecute() { checkExpirationResults(0L, 0L, 0L, 0L, 1L, result); List typedExpiredFiles = action.expireFiles().collectAsList(); - Assert.assertEquals("Expired results must match", 1, typedExpiredFiles.size()); + int expected = 1; + if (PropertyUtil.propertyAsBoolean( + table.properties(), + TableProperties.PARTITION_STATS_ENABLED, + TableProperties.PARTITION_STATS_ENABLED_DEFAULT)) { + expected = 2; + } + Assert.assertEquals("Expired results must match", expected, typedExpiredFiles.size()); List untypedExpiredFiles = action.expireFiles().collectAsList(); - Assert.assertEquals("Expired results must match", 1, untypedExpiredFiles.size()); + Assert.assertEquals("Expired results must match", expected, untypedExpiredFiles.size()); } @Test @@ -1298,7 +1320,7 @@ public void testExpireFilesAreDeleted(int dataFilesExpired, int dataFilesRetaine long end = rightAfterSnapshot(); - Set expectedDeletes = Sets.newHashSet(); + Set expectedDeletes = Sets.newHashSet(); expectedDeletes.addAll(ReachableFileUtil.manifestListLocations(table)); // all snapshot manifest lists except current will be deleted expectedDeletes.remove(table.currentSnapshot().manifestListLocation()); @@ -1306,6 +1328,9 @@ public void testExpireFilesAreDeleted(int dataFilesExpired, int dataFilesRetaine manifestsBefore); // new manifests are reachable from current snapshot and not deleted expectedDeletes.addAll( dataFiles); // new data files are reachable from current snapshot and not deleted + Set expiredSnapshots = Sets.newHashSet(table.snapshots()); + expiredSnapshots.remove(table.currentSnapshot()); + checkAndAddPartitionStatsFiles(expiredSnapshots, expectedDeletes); Set deletedFiles = Sets.newHashSet(); SparkActions.get() @@ -1351,4 +1376,14 @@ public void testExpireSomeCheckFilesDeleted() { Assert.assertFalse(deletedFiles.contains(FILE_C.path().toString())); Assert.assertFalse(deletedFiles.contains(FILE_D.path().toString())); } + + private void checkAndAddPartitionStatsFiles( + Set expiredSnapshots, Set expected) { + if (PropertyUtil.propertyAsBoolean( + table.properties(), + TableProperties.PARTITION_STATS_ENABLED, + TableProperties.PARTITION_STATS_ENABLED_DEFAULT)) { + expiredSnapshots.forEach(snapshot -> expected.add(snapshot.partitionStatsFileLocation())); + } + } } diff --git a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java index fe440235901c..cb92aaf176b8 100644 --- a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java +++ b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java @@ -202,7 +202,14 @@ public void testSparkCanReadUnknownTransform() throws IOException { manifestWriter.close(); } + table.updateProperties().set(TableProperties.PARTITION_STATS_ENABLED, "false").commit(); table.newFastAppend().appendManifest(manifestWriter.toManifestFile()).commit(); + table + .updateProperties() + .set( + TableProperties.PARTITION_STATS_ENABLED, + String.valueOf(TableProperties.PARTITION_STATS_ENABLED_DEFAULT)) + .commit(); Dataset df = spark.read().format("iceberg").load(location.toString()); diff --git a/spark/v3.3/build.gradle b/spark/v3.3/build.gradle index abd614923325..e9d86e5e35b9 100644 --- a/spark/v3.3/build.gradle +++ b/spark/v3.3/build.gradle @@ -56,7 +56,6 @@ project(":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}") { implementation project(':iceberg-core') implementation project(':iceberg-data') implementation project(':iceberg-orc') - implementation project(':iceberg-parquet') implementation project(':iceberg-arrow') implementation("org.scala-lang.modules:scala-collection-compat_${scalaVersion}:${libs.versions.scala.collection.compat.get()}") @@ -158,7 +157,6 @@ project(":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVer compileOnly libs.errorprone.annotations testImplementation project(path: ':iceberg-data') - testImplementation project(path: ':iceberg-parquet') testImplementation project(path: ':iceberg-hive-metastore') testImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') testImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts') diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java index efb3d43668f1..79630db6a444 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java @@ -22,6 +22,7 @@ import java.io.File; import java.io.IOException; +import java.io.UncheckedIOException; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.sql.Timestamp; @@ -36,11 +37,14 @@ import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.GenericBlobMetadata; import org.apache.iceberg.GenericStatisticsFile; +import org.apache.iceberg.ImmutableGenericPartitionStatisticsFile; +import org.apache.iceberg.PartitionStatisticsFile; import org.apache.iceberg.Snapshot; import org.apache.iceberg.StatisticsFile; import org.apache.iceberg.Table; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.PositionOutputStream; import org.apache.iceberg.puffin.Blob; import org.apache.iceberg.puffin.Puffin; import org.apache.iceberg.puffin.PuffinWriter; @@ -467,6 +471,18 @@ public void testExpireSnapshotsWithStatisticFiles() throws Exception { table.io()); table.updateStatistics().setStatistics(statisticsFile1.snapshotId(), statisticsFile1).commit(); + String partitionStatsFileLocation1 = statsFileLocation(table.location()); + PartitionStatisticsFile partitionStatisticsFile1 = + writePartitionStatsFile( + table.currentSnapshot().snapshotId(), + table.currentSnapshot().sequenceNumber(), + partitionStatsFileLocation1, + table.io()); + table + .updatePartitionStatistics() + .setPartitionStatistics(partitionStatisticsFile1.snapshotId(), partitionStatisticsFile1) + .commit(); + sql("INSERT INTO %s SELECT 20, 'def'", tableName); table.refresh(); String statsFileLocation2 = statsFileLocation(table.location()); @@ -478,6 +494,18 @@ public void testExpireSnapshotsWithStatisticFiles() throws Exception { table.io()); table.updateStatistics().setStatistics(statisticsFile2.snapshotId(), statisticsFile2).commit(); + String partitionStatsFileLocation2 = statsFileLocation(table.location()); + PartitionStatisticsFile partitionStatisticsFile2 = + writePartitionStatsFile( + table.currentSnapshot().snapshotId(), + table.currentSnapshot().sequenceNumber(), + partitionStatsFileLocation2, + table.io()); + table + .updatePartitionStatistics() + .setPartitionStatistics(partitionStatisticsFile2.snapshotId(), partitionStatisticsFile2) + .commit(); + waitUntilAfter(table.currentSnapshot().timestampMillis()); Timestamp currentTimestamp = Timestamp.from(Instant.ofEpochMilli(System.currentTimeMillis())); @@ -485,7 +513,9 @@ public void testExpireSnapshotsWithStatisticFiles() throws Exception { sql( "CALL %s.system.expire_snapshots(older_than => TIMESTAMP '%s',table => '%s')", catalogName, currentTimestamp, tableIdent); - Assertions.assertThat(output.get(0)[5]).as("should be 1 deleted statistics file").isEqualTo(1L); + Assertions.assertThat(output.get(0)[5]) + .as("should be 1 deleted statistics file and 1 partition statistics file") + .isEqualTo(2L); table.refresh(); List statsWithSnapshotId1 = @@ -511,6 +541,36 @@ public void testExpireSnapshotsWithStatisticFiles() throws Exception { Assertions.assertThat(new File(statsFileLocation2)) .as("Statistics file should exist for snapshot %s", statisticsFile2.snapshotId()) .exists(); + + List partitionStatsWithSnapshotId1 = + table.partitionStatisticsFiles().stream() + .filter( + statisticsFile -> + statisticsFile.snapshotId() == partitionStatisticsFile1.snapshotId()) + .collect(Collectors.toList()); + Assertions.assertThat(partitionStatsWithSnapshotId1) + .as( + "Partition statistics file entry in TableMetadata should be deleted for the snapshot %s", + partitionStatisticsFile1.snapshotId()) + .isEmpty(); + Assertions.assertThat(table.partitionStatisticsFiles()) + .as( + "Partition statistics file entry in TableMetadata should be present for the snapshot %s", + partitionStatisticsFile2.snapshotId()) + .extracting(PartitionStatisticsFile::snapshotId) + .containsExactly(partitionStatisticsFile2.snapshotId()); + + Assertions.assertThat(new File(partitionStatsFileLocation1)) + .as( + "Partition statistics file should not exist for snapshot %s", + partitionStatisticsFile1.snapshotId()) + .doesNotExist(); + + Assertions.assertThat(new File(partitionStatsFileLocation2)) + .as( + "Partition statistics file should exist for snapshot %s", + partitionStatisticsFile2.snapshotId()) + .exists(); } private StatisticsFile writeStatsFile( @@ -541,4 +601,20 @@ private String statsFileLocation(String tableLocation) { String statsFileName = "stats-file-" + UUID.randomUUID(); return tableLocation.replaceFirst("file:", "") + "/metadata/" + statsFileName; } + + private PartitionStatisticsFile writePartitionStatsFile( + long snapshotId, long snapshotSequenceNumber, String statsLocation, FileIO fileIO) { + PositionOutputStream positionOutputStream; + try { + positionOutputStream = fileIO.newOutputFile(statsLocation).create(); + positionOutputStream.close(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + return ImmutableGenericPartitionStatisticsFile.builder() + .snapshotId(snapshotId) + .maxDataSequenceNumber(snapshotSequenceNumber) + .path(statsLocation) + .build(); + } } diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java index 3c007c6214c2..8ead8f33387f 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java @@ -31,7 +31,6 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Consumer; -import java.util.function.Predicate; import java.util.function.Supplier; import org.apache.iceberg.AllManifestsTable; import org.apache.iceberg.BaseTable; @@ -44,7 +43,6 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.ReachableFileUtil; import org.apache.iceberg.StaticTableOperations; -import org.apache.iceberg.StatisticsFile; import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.exceptions.NotFoundException; @@ -204,14 +202,8 @@ protected Dataset manifestListDS(Table table, Set snapshotIds) { } protected Dataset statisticsFileDS(Table table, Set snapshotIds) { - Predicate predicate; - if (snapshotIds == null) { - predicate = statisticsFile -> true; - } else { - predicate = statisticsFile -> snapshotIds.contains(statisticsFile.snapshotId()); - } - - List statisticsFiles = ReachableFileUtil.statisticsFilesLocations(table, predicate); + List statisticsFiles = + ReachableFileUtil.allStatisticsFilesLocations(table, snapshotIds); return toFileInfoDS(statisticsFiles, STATISTICS_FILES); } @@ -227,7 +219,7 @@ private Dataset otherMetadataFileDS(Table table, boolean recursive) { List otherMetadataFiles = Lists.newArrayList(); otherMetadataFiles.addAll(ReachableFileUtil.metadataFileLocations(table, recursive)); otherMetadataFiles.add(ReachableFileUtil.versionHintLocation(table)); - otherMetadataFiles.addAll(ReachableFileUtil.statisticsFilesLocations(table)); + otherMetadataFiles.addAll(ReachableFileUtil.allStatisticsFilesLocations(table)); return toFileInfoDS(otherMetadataFiles, OTHERS); } diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java index 0e333f216c26..88547052441b 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java @@ -51,6 +51,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.SparkTestBase; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.PropertyUtil; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -153,10 +154,16 @@ private void checkRemoveFilesResults( "Incorrect number of manifest lists deleted", expectedManifestListsDeleted, results.deletedManifestListsCount()); + long expected = expectedOtherFilesDeleted; + if (PropertyUtil.propertyAsBoolean( + table.properties(), + TableProperties.PARTITION_STATS_ENABLED, + TableProperties.PARTITION_STATS_ENABLED_DEFAULT)) { + // one partition stats file per snapshot (or manifestList) + expected += expectedManifestListsDeleted; + } Assert.assertEquals( - "Incorrect number of other lists deleted", - expectedOtherFilesDeleted, - results.deletedOtherFilesCount()); + "Incorrect number of other lists deleted", expected, results.deletedOtherFilesCount()); } @Test diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java index 25e6f6486aae..2d76ac90ba4b 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java @@ -22,6 +22,7 @@ import java.io.File; import java.io.IOException; +import java.util.Collections; import java.util.List; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; @@ -56,6 +57,7 @@ import org.apache.iceberg.spark.SparkTestBase; import org.apache.iceberg.spark.data.TestHelpers; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.PropertyUtil; import org.apache.spark.sql.Dataset; import org.junit.Assert; import org.junit.Before; @@ -619,7 +621,7 @@ public void testWithExpiringDanglingStageCommit() { checkExpirationResults(1L, 0L, 0L, 1L, 2L, result); - Set expectedDeletes = Sets.newHashSet(); + Set expectedDeletes = Sets.newHashSet(); expectedDeletes.add(snapshotA.manifestListLocation()); // Files should be deleted of dangling staged snapshot @@ -641,6 +643,7 @@ public void testWithExpiringDanglingStageCommit() { expectedDeletes.add(file.path()); } }); + checkAndAddPartitionStatsFiles(Sets.newHashSet(snapshotA, snapshotB), expectedDeletes); Assert.assertSame( "Files deleted count should be expected", expectedDeletes.size(), deletedFiles.size()); // Take the diff @@ -800,10 +803,9 @@ public void testExpireOlderThan() { table.currentSnapshot().snapshotId()); Assert.assertNull( "Expire should remove the oldest snapshot", table.snapshot(firstSnapshot.snapshotId())); - Assert.assertEquals( - "Should remove only the expired manifest list location", - Sets.newHashSet(firstSnapshot.manifestListLocation()), - deletedFiles); + Set expected = Sets.newHashSet(firstSnapshot.manifestListLocation()); + checkAndAddPartitionStatsFiles(Collections.singleton(firstSnapshot), expected); + Assert.assertEquals("Should remove expected files", expected, deletedFiles); checkExpirationResults(0, 0, 0, 0, 1, result); } @@ -853,8 +855,7 @@ public void testExpireOlderThanWithDelete() { "Expire should remove the second oldest snapshot", table.snapshot(secondSnapshot.snapshotId())); - Assert.assertEquals( - "Should remove expired manifest lists and deleted data file", + Set expected = Sets.newHashSet( firstSnapshot.manifestListLocation(), // snapshot expired firstSnapshot @@ -866,7 +867,11 @@ public void testExpireOlderThanWithDelete() { .allManifests(table.io()) .get(0) .path(), // manifest contained only deletes, was dropped - FILE_A.path()), // deleted + FILE_A.path()); + checkAndAddPartitionStatsFiles(Sets.newHashSet(firstSnapshot, secondSnapshot), expected); + Assert.assertEquals( + "Should remove expected files", + expected, // deleted deletedFiles); checkExpirationResults(1, 0, 0, 2, 2, result); @@ -926,8 +931,7 @@ public void testExpireOlderThanWithDeleteInMergedManifests() { "Expire should remove the second oldest snapshot", table.snapshot(secondSnapshot.snapshotId())); - Assert.assertEquals( - "Should remove expired manifest lists and deleted data file", + Set expected = Sets.newHashSet( firstSnapshot.manifestListLocation(), // snapshot expired firstSnapshot @@ -935,7 +939,11 @@ public void testExpireOlderThanWithDeleteInMergedManifests() { .get(0) .path(), // manifest was rewritten for delete secondSnapshot.manifestListLocation(), // snapshot expired - FILE_A.path()), // deleted + FILE_A.path()); + checkAndAddPartitionStatsFiles(Sets.newHashSet(firstSnapshot, secondSnapshot), expected); + Assert.assertEquals( + "Should remove expected files", + expected, // deleted deletedFiles); checkExpirationResults(1, 0, 0, 1, 2, result); @@ -988,12 +996,14 @@ public void testExpireOlderThanWithRollback() { Assert.assertNull( "Expire should remove the orphaned snapshot", table.snapshot(secondSnapshot.snapshotId())); - Assert.assertEquals( - "Should remove expired manifest lists and reverted appended data file", + Set expected = Sets.newHashSet( secondSnapshot.manifestListLocation(), // snapshot expired - Iterables.getOnlyElement(secondSnapshotManifests) - .path()), // manifest is no longer referenced + Iterables.getOnlyElement(secondSnapshotManifests).path()); + checkAndAddPartitionStatsFiles(Collections.singleton(secondSnapshot), expected); + Assert.assertEquals( + "Should remove expected files", + expected, // manifest is no longer referenced deletedFiles); checkExpirationResults(0, 0, 0, 1, 1, result); @@ -1043,13 +1053,16 @@ public void testExpireOlderThanWithRollbackAndMergedManifests() { Assert.assertNull( "Expire should remove the orphaned snapshot", table.snapshot(secondSnapshot.snapshotId())); - Assert.assertEquals( - "Should remove expired manifest lists and reverted appended data file", + Set expected = Sets.newHashSet( secondSnapshot.manifestListLocation(), // snapshot expired Iterables.getOnlyElement(secondSnapshotManifests) .path(), // manifest is no longer referenced - FILE_B.path()), // added, but rolled back + FILE_B.path()); + checkAndAddPartitionStatsFiles(Collections.singleton(secondSnapshot), expected); + Assert.assertEquals( + "Should remove expected files", + expected, // added, but rolled back deletedFiles); checkExpirationResults(1, 0, 0, 1, 1, result); @@ -1092,7 +1105,7 @@ public void testExpireOlderThanWithDeleteFile() { .deleteWith(deletedFiles::add) .execute(); - Set expectedDeletes = + Set expectedDeletes = Sets.newHashSet( firstSnapshot.manifestListLocation(), secondSnapshot.manifestListLocation(), @@ -1113,6 +1126,9 @@ public void testExpireOlderThanWithDeleteFile() { .map(ManifestFile::path) .map(CharSequence::toString) .collect(Collectors.toSet())); + checkAndAddPartitionStatsFiles( + Sets.newHashSet(firstSnapshot, secondSnapshot, thirdSnapshot, fourthSnapshot), + expectedDeletes); Assert.assertEquals( "Should remove expired manifest lists and deleted data file", @@ -1167,13 +1183,12 @@ public void testExpireAction() { Assert.assertNull( "Should remove the oldest snapshot", table.snapshot(firstSnapshot.snapshotId())); - Assert.assertEquals("Pending deletes should contain one row", 1, pending.size()); - Assert.assertEquals( - "Pending delete should be the expired manifest list location", - firstSnapshot.manifestListLocation(), - pending.get(0).getPath()); + Set expected = Sets.newHashSet(firstSnapshot.manifestListLocation()); + checkAndAddPartitionStatsFiles(Collections.singleton(firstSnapshot), expected); Assert.assertEquals( - "Pending delete should be a manifest list", "Manifest List", pending.get(0).getType()); + "Pending deletes should match", + expected, + pending.stream().map(FileInfo::getPath).collect(Collectors.toSet())); Assert.assertEquals("Should not delete any files", 0, deletedFiles.size()); @@ -1246,10 +1261,17 @@ public void testExpireAfterExecute() { checkExpirationResults(0L, 0L, 0L, 0L, 1L, result); List typedExpiredFiles = action.expireFiles().collectAsList(); - Assert.assertEquals("Expired results must match", 1, typedExpiredFiles.size()); + int expected = 1; + if (PropertyUtil.propertyAsBoolean( + table.properties(), + TableProperties.PARTITION_STATS_ENABLED, + TableProperties.PARTITION_STATS_ENABLED_DEFAULT)) { + expected = 2; + } + Assert.assertEquals("Expired results must match", expected, typedExpiredFiles.size()); List untypedExpiredFiles = action.expireFiles().collectAsList(); - Assert.assertEquals("Expired results must match", 1, untypedExpiredFiles.size()); + Assert.assertEquals("Expired results must match", expected, untypedExpiredFiles.size()); } @Test @@ -1298,7 +1320,7 @@ public void textExpireAllCheckFilesDeleted(int dataFilesExpired, int dataFilesRe long end = rightAfterSnapshot(); - Set expectedDeletes = Sets.newHashSet(); + Set expectedDeletes = Sets.newHashSet(); expectedDeletes.addAll(ReachableFileUtil.manifestListLocations(table)); // all snapshot manifest lists except current will be deleted expectedDeletes.remove(table.currentSnapshot().manifestListLocation()); @@ -1306,6 +1328,9 @@ public void textExpireAllCheckFilesDeleted(int dataFilesExpired, int dataFilesRe manifestsBefore); // new manifests are reachable from current snapshot and not deleted expectedDeletes.addAll( dataFiles); // new data files are reachable from current snapshot and not deleted + Set expiredSnapshots = Sets.newHashSet(table.snapshots()); + expiredSnapshots.remove(table.currentSnapshot()); + checkAndAddPartitionStatsFiles(expiredSnapshots, expectedDeletes); Set deletedFiles = Sets.newHashSet(); SparkActions.get() @@ -1351,4 +1376,14 @@ public void testExpireSomeCheckFilesDeleted() { Assert.assertFalse(deletedFiles.contains(FILE_C.path().toString())); Assert.assertFalse(deletedFiles.contains(FILE_D.path().toString())); } + + private void checkAndAddPartitionStatsFiles( + Set expiredSnapshots, Set expected) { + if (PropertyUtil.propertyAsBoolean( + table.properties(), + TableProperties.PARTITION_STATS_ENABLED, + TableProperties.PARTITION_STATS_ENABLED_DEFAULT)) { + expiredSnapshots.forEach(snapshot -> expected.add(snapshot.partitionStatsFileLocation())); + } + } } diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java index fe440235901c..cb92aaf176b8 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java @@ -202,7 +202,14 @@ public void testSparkCanReadUnknownTransform() throws IOException { manifestWriter.close(); } + table.updateProperties().set(TableProperties.PARTITION_STATS_ENABLED, "false").commit(); table.newFastAppend().appendManifest(manifestWriter.toManifestFile()).commit(); + table + .updateProperties() + .set( + TableProperties.PARTITION_STATS_ENABLED, + String.valueOf(TableProperties.PARTITION_STATS_ENABLED_DEFAULT)) + .commit(); Dataset df = spark.read().format("iceberg").load(location.toString()); diff --git a/spark/v3.4/build.gradle b/spark/v3.4/build.gradle index 3ec10c3b6b31..7612728d3cf0 100644 --- a/spark/v3.4/build.gradle +++ b/spark/v3.4/build.gradle @@ -56,7 +56,6 @@ project(":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}") { implementation project(':iceberg-core') implementation project(':iceberg-data') implementation project(':iceberg-orc') - implementation project(':iceberg-parquet') implementation project(':iceberg-arrow') implementation("org.scala-lang.modules:scala-collection-compat_${scalaVersion}:${libs.versions.scala.collection.compat.get()}") @@ -157,7 +156,6 @@ project(":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVer compileOnly libs.errorprone.annotations testImplementation project(path: ':iceberg-data') - testImplementation project(path: ':iceberg-parquet') testImplementation project(path: ':iceberg-hive-metastore') testImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') testImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts') diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java index c4b93c6d6a44..7b5547372469 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java @@ -22,6 +22,7 @@ import java.io.File; import java.io.IOException; +import java.io.UncheckedIOException; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.sql.Timestamp; @@ -35,11 +36,14 @@ import org.apache.hadoop.fs.Path; import org.apache.iceberg.GenericBlobMetadata; import org.apache.iceberg.GenericStatisticsFile; +import org.apache.iceberg.ImmutableGenericPartitionStatisticsFile; +import org.apache.iceberg.PartitionStatisticsFile; import org.apache.iceberg.Snapshot; import org.apache.iceberg.StatisticsFile; import org.apache.iceberg.Table; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.PositionOutputStream; import org.apache.iceberg.puffin.Blob; import org.apache.iceberg.puffin.Puffin; import org.apache.iceberg.puffin.PuffinWriter; @@ -454,6 +458,18 @@ public void testExpireSnapshotsWithStatisticFiles() throws Exception { table.io()); table.updateStatistics().setStatistics(statisticsFile1.snapshotId(), statisticsFile1).commit(); + String partitionStatsFileLocation1 = statsFileLocation(table.location()); + PartitionStatisticsFile partitionStatisticsFile1 = + writePartitionStatsFile( + table.currentSnapshot().snapshotId(), + table.currentSnapshot().sequenceNumber(), + partitionStatsFileLocation1, + table.io()); + table + .updatePartitionStatistics() + .setPartitionStatistics(partitionStatisticsFile1.snapshotId(), partitionStatisticsFile1) + .commit(); + sql("INSERT INTO %s SELECT 20, 'def'", tableName); table.refresh(); String statsFileLocation2 = statsFileLocation(table.location()); @@ -465,6 +481,18 @@ public void testExpireSnapshotsWithStatisticFiles() throws Exception { table.io()); table.updateStatistics().setStatistics(statisticsFile2.snapshotId(), statisticsFile2).commit(); + String partitionStatsFileLocation2 = statsFileLocation(table.location()); + PartitionStatisticsFile partitionStatisticsFile2 = + writePartitionStatsFile( + table.currentSnapshot().snapshotId(), + table.currentSnapshot().sequenceNumber(), + partitionStatsFileLocation2, + table.io()); + table + .updatePartitionStatistics() + .setPartitionStatistics(partitionStatisticsFile2.snapshotId(), partitionStatisticsFile2) + .commit(); + waitUntilAfter(table.currentSnapshot().timestampMillis()); Timestamp currentTimestamp = Timestamp.from(Instant.ofEpochMilli(System.currentTimeMillis())); @@ -472,7 +500,9 @@ public void testExpireSnapshotsWithStatisticFiles() throws Exception { sql( "CALL %s.system.expire_snapshots(older_than => TIMESTAMP '%s',table => '%s')", catalogName, currentTimestamp, tableIdent); - Assertions.assertThat(output.get(0)[5]).as("should be 1 deleted statistics file").isEqualTo(1L); + Assertions.assertThat(output.get(0)[5]) + .as("should be 1 deleted statistics file and 1 partition statistics file") + .isEqualTo(2L); table.refresh(); List statsWithSnapshotId1 = @@ -498,6 +528,36 @@ public void testExpireSnapshotsWithStatisticFiles() throws Exception { Assertions.assertThat(new File(statsFileLocation2)) .as("Statistics file should exist for snapshot %s", statisticsFile2.snapshotId()) .exists(); + + List partitionStatsWithSnapshotId1 = + table.partitionStatisticsFiles().stream() + .filter( + statisticsFile -> + statisticsFile.snapshotId() == partitionStatisticsFile1.snapshotId()) + .collect(Collectors.toList()); + Assertions.assertThat(partitionStatsWithSnapshotId1) + .as( + "Partition statistics file entry in TableMetadata should be deleted for the snapshot %s", + partitionStatisticsFile1.snapshotId()) + .isEmpty(); + Assertions.assertThat(table.partitionStatisticsFiles()) + .as( + "Partition statistics file entry in TableMetadata should be present for the snapshot %s", + partitionStatisticsFile2.snapshotId()) + .extracting(PartitionStatisticsFile::snapshotId) + .containsExactly(partitionStatisticsFile2.snapshotId()); + + Assertions.assertThat(new File(partitionStatsFileLocation1)) + .as( + "Partition statistics file should not exist for snapshot %s", + partitionStatisticsFile1.snapshotId()) + .doesNotExist(); + + Assertions.assertThat(new File(partitionStatsFileLocation2)) + .as( + "Partition statistics file should exist for snapshot %s", + partitionStatisticsFile2.snapshotId()) + .exists(); } private StatisticsFile writeStatsFile( @@ -528,4 +588,20 @@ private String statsFileLocation(String tableLocation) { String statsFileName = "stats-file-" + UUID.randomUUID(); return tableLocation.replaceFirst("file:", "") + "/metadata/" + statsFileName; } + + private PartitionStatisticsFile writePartitionStatsFile( + long snapshotId, long snapshotSequenceNumber, String statsLocation, FileIO fileIO) { + PositionOutputStream positionOutputStream; + try { + positionOutputStream = fileIO.newOutputFile(statsLocation).create(); + positionOutputStream.close(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + return ImmutableGenericPartitionStatisticsFile.builder() + .snapshotId(snapshotId) + .maxDataSequenceNumber(snapshotSequenceNumber) + .path(statsLocation) + .build(); + } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java index 613e53767efa..6d5a8a664505 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java @@ -31,7 +31,6 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Consumer; -import java.util.function.Predicate; import java.util.function.Supplier; import org.apache.iceberg.AllManifestsTable; import org.apache.iceberg.BaseTable; @@ -44,7 +43,6 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.ReachableFileUtil; import org.apache.iceberg.StaticTableOperations; -import org.apache.iceberg.StatisticsFile; import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.exceptions.NotFoundException; @@ -196,14 +194,8 @@ protected Dataset manifestListDS(Table table, Set snapshotIds) { } protected Dataset statisticsFileDS(Table table, Set snapshotIds) { - Predicate predicate; - if (snapshotIds == null) { - predicate = statisticsFile -> true; - } else { - predicate = statisticsFile -> snapshotIds.contains(statisticsFile.snapshotId()); - } - - List statisticsFiles = ReachableFileUtil.statisticsFilesLocations(table, predicate); + List statisticsFiles = + ReachableFileUtil.allStatisticsFilesLocations(table, snapshotIds); return toFileInfoDS(statisticsFiles, STATISTICS_FILES); } @@ -219,7 +211,7 @@ private Dataset otherMetadataFileDS(Table table, boolean recursive) { List otherMetadataFiles = Lists.newArrayList(); otherMetadataFiles.addAll(ReachableFileUtil.metadataFileLocations(table, recursive)); otherMetadataFiles.add(ReachableFileUtil.versionHintLocation(table)); - otherMetadataFiles.addAll(ReachableFileUtil.statisticsFilesLocations(table)); + otherMetadataFiles.addAll(ReachableFileUtil.allStatisticsFilesLocations(table)); return toFileInfoDS(otherMetadataFiles, OTHERS); } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java index 384405265500..4c326dc1eb4c 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java @@ -50,6 +50,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.SparkTestBase; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.PropertyUtil; import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; @@ -153,10 +154,16 @@ private void checkRemoveFilesResults( "Incorrect number of manifest lists deleted", expectedManifestListsDeleted, results.deletedManifestListsCount()); + long expected = expectedOtherFilesDeleted; + if (PropertyUtil.propertyAsBoolean( + table.properties(), + TableProperties.PARTITION_STATS_ENABLED, + TableProperties.PARTITION_STATS_ENABLED_DEFAULT)) { + // one partition stats file per snapshot (or manifestList) + expected += expectedManifestListsDeleted; + } Assert.assertEquals( - "Incorrect number of other lists deleted", - expectedOtherFilesDeleted, - results.deletedOtherFilesCount()); + "Incorrect number of other lists deleted", expected, results.deletedOtherFilesCount()); } @Test diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java index e90626cbda6d..58b2c5856b1c 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java @@ -22,6 +22,7 @@ import java.io.File; import java.io.IOException; +import java.util.Collections; import java.util.List; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; @@ -55,6 +56,7 @@ import org.apache.iceberg.spark.SparkTestBase; import org.apache.iceberg.spark.data.TestHelpers; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.PropertyUtil; import org.apache.spark.sql.Dataset; import org.assertj.core.api.Assertions; import org.junit.Assert; @@ -617,7 +619,7 @@ public void testWithExpiringDanglingStageCommit() { checkExpirationResults(1L, 0L, 0L, 1L, 2L, result); - Set expectedDeletes = Sets.newHashSet(); + Set expectedDeletes = Sets.newHashSet(); expectedDeletes.add(snapshotA.manifestListLocation()); // Files should be deleted of dangling staged snapshot @@ -639,6 +641,7 @@ public void testWithExpiringDanglingStageCommit() { expectedDeletes.add(file.path()); } }); + checkAndAddPartitionStatsFiles(Sets.newHashSet(snapshotA, snapshotB), expectedDeletes); Assert.assertSame( "Files deleted count should be expected", expectedDeletes.size(), deletedFiles.size()); // Take the diff @@ -798,10 +801,9 @@ public void testExpireOlderThan() { table.currentSnapshot().snapshotId()); Assert.assertNull( "Expire should remove the oldest snapshot", table.snapshot(firstSnapshot.snapshotId())); - Assert.assertEquals( - "Should remove only the expired manifest list location", - Sets.newHashSet(firstSnapshot.manifestListLocation()), - deletedFiles); + Set expected = Sets.newHashSet(firstSnapshot.manifestListLocation()); + checkAndAddPartitionStatsFiles(Collections.singleton(firstSnapshot), expected); + Assert.assertEquals("Should remove expected files", expected, deletedFiles); checkExpirationResults(0, 0, 0, 0, 1, result); } @@ -851,8 +853,7 @@ public void testExpireOlderThanWithDelete() { "Expire should remove the second oldest snapshot", table.snapshot(secondSnapshot.snapshotId())); - Assert.assertEquals( - "Should remove expired manifest lists and deleted data file", + Set expected = Sets.newHashSet( firstSnapshot.manifestListLocation(), // snapshot expired firstSnapshot @@ -864,7 +865,11 @@ public void testExpireOlderThanWithDelete() { .allManifests(table.io()) .get(0) .path(), // manifest contained only deletes, was dropped - FILE_A.path()), // deleted + FILE_A.path()); + checkAndAddPartitionStatsFiles(Sets.newHashSet(firstSnapshot, secondSnapshot), expected); + Assert.assertEquals( + "Should remove expected files", + expected, // deleted deletedFiles); checkExpirationResults(1, 0, 0, 2, 2, result); @@ -924,8 +929,7 @@ public void testExpireOlderThanWithDeleteInMergedManifests() { "Expire should remove the second oldest snapshot", table.snapshot(secondSnapshot.snapshotId())); - Assert.assertEquals( - "Should remove expired manifest lists and deleted data file", + Set expected = Sets.newHashSet( firstSnapshot.manifestListLocation(), // snapshot expired firstSnapshot @@ -933,7 +937,11 @@ public void testExpireOlderThanWithDeleteInMergedManifests() { .get(0) .path(), // manifest was rewritten for delete secondSnapshot.manifestListLocation(), // snapshot expired - FILE_A.path()), // deleted + FILE_A.path()); + checkAndAddPartitionStatsFiles(Sets.newHashSet(firstSnapshot, secondSnapshot), expected); + Assert.assertEquals( + "Should remove expected files", + expected, // deleted deletedFiles); checkExpirationResults(1, 0, 0, 1, 2, result); @@ -986,12 +994,14 @@ public void testExpireOlderThanWithRollback() { Assert.assertNull( "Expire should remove the orphaned snapshot", table.snapshot(secondSnapshot.snapshotId())); - Assert.assertEquals( - "Should remove expired manifest lists and reverted appended data file", + Set expected = Sets.newHashSet( secondSnapshot.manifestListLocation(), // snapshot expired - Iterables.getOnlyElement(secondSnapshotManifests) - .path()), // manifest is no longer referenced + Iterables.getOnlyElement(secondSnapshotManifests).path()); + checkAndAddPartitionStatsFiles(Collections.singleton(secondSnapshot), expected); + Assert.assertEquals( + "Should remove expected files", + expected, // manifest is no longer referenced deletedFiles); checkExpirationResults(0, 0, 0, 1, 1, result); @@ -1041,13 +1051,16 @@ public void testExpireOlderThanWithRollbackAndMergedManifests() { Assert.assertNull( "Expire should remove the orphaned snapshot", table.snapshot(secondSnapshot.snapshotId())); - Assert.assertEquals( - "Should remove expired manifest lists and reverted appended data file", + Set expected = Sets.newHashSet( secondSnapshot.manifestListLocation(), // snapshot expired Iterables.getOnlyElement(secondSnapshotManifests) .path(), // manifest is no longer referenced - FILE_B.path()), // added, but rolled back + FILE_B.path()); + checkAndAddPartitionStatsFiles(Collections.singleton(secondSnapshot), expected); + Assert.assertEquals( + "Should remove expected files", + expected, // added, but rolled back deletedFiles); checkExpirationResults(1, 0, 0, 1, 1, result); @@ -1090,7 +1103,7 @@ public void testExpireOlderThanWithDeleteFile() { .deleteWith(deletedFiles::add) .execute(); - Set expectedDeletes = + Set expectedDeletes = Sets.newHashSet( firstSnapshot.manifestListLocation(), secondSnapshot.manifestListLocation(), @@ -1111,6 +1124,9 @@ public void testExpireOlderThanWithDeleteFile() { .map(ManifestFile::path) .map(CharSequence::toString) .collect(Collectors.toSet())); + checkAndAddPartitionStatsFiles( + Sets.newHashSet(firstSnapshot, secondSnapshot, thirdSnapshot, fourthSnapshot), + expectedDeletes); Assert.assertEquals( "Should remove expired manifest lists and deleted data file", @@ -1165,13 +1181,12 @@ public void testExpireAction() { Assert.assertNull( "Should remove the oldest snapshot", table.snapshot(firstSnapshot.snapshotId())); - Assert.assertEquals("Pending deletes should contain one row", 1, pending.size()); - Assert.assertEquals( - "Pending delete should be the expired manifest list location", - firstSnapshot.manifestListLocation(), - pending.get(0).getPath()); + Set expected = Sets.newHashSet(firstSnapshot.manifestListLocation()); + checkAndAddPartitionStatsFiles(Collections.singleton(firstSnapshot), expected); Assert.assertEquals( - "Pending delete should be a manifest list", "Manifest List", pending.get(0).getType()); + "Pending deletes should match", + expected, + pending.stream().map(FileInfo::getPath).collect(Collectors.toSet())); Assert.assertEquals("Should not delete any files", 0, deletedFiles.size()); @@ -1244,10 +1259,17 @@ public void testExpireAfterExecute() { checkExpirationResults(0L, 0L, 0L, 0L, 1L, result); List typedExpiredFiles = action.expireFiles().collectAsList(); - Assert.assertEquals("Expired results must match", 1, typedExpiredFiles.size()); + int expected = 1; + if (PropertyUtil.propertyAsBoolean( + table.properties(), + TableProperties.PARTITION_STATS_ENABLED, + TableProperties.PARTITION_STATS_ENABLED_DEFAULT)) { + expected = 2; + } + Assert.assertEquals("Expired results must match", expected, typedExpiredFiles.size()); List untypedExpiredFiles = action.expireFiles().collectAsList(); - Assert.assertEquals("Expired results must match", 1, untypedExpiredFiles.size()); + Assert.assertEquals("Expired results must match", expected, untypedExpiredFiles.size()); } @Test @@ -1296,7 +1318,7 @@ public void textExpireAllCheckFilesDeleted(int dataFilesExpired, int dataFilesRe long end = rightAfterSnapshot(); - Set expectedDeletes = Sets.newHashSet(); + Set expectedDeletes = Sets.newHashSet(); expectedDeletes.addAll(ReachableFileUtil.manifestListLocations(table)); // all snapshot manifest lists except current will be deleted expectedDeletes.remove(table.currentSnapshot().manifestListLocation()); @@ -1304,6 +1326,9 @@ public void textExpireAllCheckFilesDeleted(int dataFilesExpired, int dataFilesRe manifestsBefore); // new manifests are reachable from current snapshot and not deleted expectedDeletes.addAll( dataFiles); // new data files are reachable from current snapshot and not deleted + Set expiredSnapshots = Sets.newHashSet(table.snapshots()); + expiredSnapshots.remove(table.currentSnapshot()); + checkAndAddPartitionStatsFiles(expiredSnapshots, expectedDeletes); Set deletedFiles = Sets.newHashSet(); SparkActions.get() @@ -1349,4 +1374,14 @@ public void testExpireSomeCheckFilesDeleted() { Assert.assertFalse(deletedFiles.contains(FILE_C.path().toString())); Assert.assertFalse(deletedFiles.contains(FILE_D.path().toString())); } + + private void checkAndAddPartitionStatsFiles( + Set expiredSnapshots, Set expected) { + if (PropertyUtil.propertyAsBoolean( + table.properties(), + TableProperties.PARTITION_STATS_ENABLED, + TableProperties.PARTITION_STATS_ENABLED_DEFAULT)) { + expiredSnapshots.forEach(snapshot -> expected.add(snapshot.partitionStatsFileLocation())); + } + } } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java index 73e572ecaecd..7b505d76cacc 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java @@ -199,7 +199,14 @@ public void testSparkCanReadUnknownTransform() throws IOException { manifestWriter.close(); } + table.updateProperties().set(TableProperties.PARTITION_STATS_ENABLED, "false").commit(); table.newFastAppend().appendManifest(manifestWriter.toManifestFile()).commit(); + table + .updateProperties() + .set( + TableProperties.PARTITION_STATS_ENABLED, + String.valueOf(TableProperties.PARTITION_STATS_ENABLED_DEFAULT)) + .commit(); Dataset df = spark.read().format("iceberg").load(location.toString());