diff --git a/be/src/catalog/catalog.cc b/be/src/catalog/catalog.cc
index 310424270..cfeafd93e 100644
--- a/be/src/catalog/catalog.cc
+++ b/be/src/catalog/catalog.cc
@@ -46,6 +46,8 @@ DEFINE_int32(max_nonhdfs_partitions_parallel_load, 20,
DEFINE_int32(initial_hms_cnxn_timeout_s, 120,
"Number of seconds catalogd will wait to establish an initial connection to the HMS "
"before exiting.");
+DEFINE_bool(disable_reading_puffin_stats, true, "If this flag is set to true, "
+ "Impala will not read Iceberg Puffin stats files.");
Catalog::Catalog() {
JniMethodDescriptor methods[] = {
diff --git a/be/src/util/backend-gflag-util.cc b/be/src/util/backend-gflag-util.cc
index f58d57223..1c0e2aef5 100644
--- a/be/src/util/backend-gflag-util.cc
+++ b/be/src/util/backend-gflag-util.cc
@@ -113,6 +113,7 @@ DECLARE_bool(enable_skipping_older_events);
DECLARE_bool(enable_json_scanner);
DECLARE_bool(iceberg_allow_datafiles_in_table_location_only);
DECLARE_bool(iceberg_always_allow_merge_on_read_operations);
+DECLARE_bool(disable_reading_puffin_stats);
DECLARE_int32(catalog_operation_log_size);
DECLARE_string(hostname);
DECLARE_bool(allow_catalog_cache_op_from_masked_users);
@@ -472,6 +473,8 @@ Status PopulateThriftBackendGflags(TBackendGflags& cfg) {
FLAGS_iceberg_allow_datafiles_in_table_location_only);
cfg.__set_iceberg_always_allow_merge_on_read_operations(
FLAGS_iceberg_always_allow_merge_on_read_operations);
+ cfg.__set_disable_reading_puffin_stats(
+ FLAGS_disable_reading_puffin_stats);
cfg.__set_max_filter_error_rate_from_full_scan(
FLAGS_max_filter_error_rate_from_full_scan);
cfg.__set_catalog_operation_log_size(FLAGS_catalog_operation_log_size);
diff --git a/bin/impala-config.sh b/bin/impala-config.sh
index e36ad542a..030a302ac 100755
--- a/bin/impala-config.sh
+++ b/bin/impala-config.sh
@@ -282,6 +282,7 @@ export IMPALA_OBS_VERSION=3.1.1-hw-42
export IMPALA_DBCP2_VERSION=2.9.0
export IMPALA_DROPWIZARD_METRICS_VERSION=4.2.26
export IMPALA_AIRCOMPRESSOR_VERSION=0.27
+export IMPALA_DATASKETCHES_VERSION=6.0.0
# When Impala is building docker images on Redhat-based distributions,
# it is useful to be able to customize the base image. Some users will
diff --git a/common/thrift/BackendGflags.thrift b/common/thrift/BackendGflags.thrift
index 3a4d52ac6..daf11bc77 100644
--- a/common/thrift/BackendGflags.thrift
+++ b/common/thrift/BackendGflags.thrift
@@ -312,4 +312,6 @@ struct TBackendGflags {
140: required bool iceberg_always_allow_merge_on_read_operations
141: required i64 data_stream_sender_buffer_size_used_by_planner
+
+ 142: required bool disable_reading_puffin_stats
}
diff --git a/fe/pom.xml b/fe/pom.xml
index 529dafa7b..f75189f2e 100644
--- a/fe/pom.xml
+++ b/fe/pom.xml
@@ -435,6 +435,13 @@ under the License.
${iceberg.version}
+
+
+ org.apache.datasketches
+ datasketches-java
+ ${datasketches.version}
+
+
org.apache.hive
diff --git a/fe/src/main/java/org/apache/impala/catalog/ColumnStats.java b/fe/src/main/java/org/apache/impala/catalog/ColumnStats.java
index d60d2ce8e..de41af094 100644
--- a/fe/src/main/java/org/apache/impala/catalog/ColumnStats.java
+++ b/fe/src/main/java/org/apache/impala/catalog/ColumnStats.java
@@ -1018,6 +1018,13 @@ public class ColumnStats {
return SUPPORTED_COL_TYPES.contains(scalarType.getPrimitiveType());
}
+ /**
+ * Returns whether the given type supports NDV stats.
+ */
+ public static boolean supportsNdv(Type colType) {
+ return isSupportedColType(colType) && !colType.isBoolean() && !colType.isBinary();
+ }
+
public void update(Type colType, TColumnStats stats) {
initColStats(colType);
if (!colType.isFixedLengthType() && stats.getAvg_size() >= 0) {
diff --git a/fe/src/main/java/org/apache/impala/catalog/IcebergTable.java b/fe/src/main/java/org/apache/impala/catalog/IcebergTable.java
index 1f245a665..1bdc132e2 100644
--- a/fe/src/main/java/org/apache/impala/catalog/IcebergTable.java
+++ b/fe/src/main/java/org/apache/impala/catalog/IcebergTable.java
@@ -36,6 +36,7 @@ import org.apache.impala.analysis.IcebergPartitionSpec;
import org.apache.impala.analysis.IcebergPartitionTransform;
import org.apache.impala.catalog.iceberg.GroupedContentFiles;
import org.apache.impala.common.ImpalaRuntimeException;
+import org.apache.impala.service.BackendConfig;
import org.apache.impala.thrift.TCatalogObjectType;
import org.apache.impala.thrift.TCompressionCodec;
import org.apache.impala.thrift.TGetPartialCatalogObjectRequest;
@@ -59,9 +60,20 @@ import org.apache.thrift.TException;
/**
* Representation of an Iceberg table in the catalog cache.
+ *
+ * For an Iceberg table, stats can come from 3 places:
+ * 1. numRows: written by Iceberg
+ * 2. HMS column stats: used even if stale
+ * 3. NDV from Puffin: only from the current snapshot, overrides HMS NDV stats if present.
+ *
+ * As Puffin only contains NDV stats, it is possible that at a given point the NDV is from
+ * Puffin but other column stats, e.g. num nulls, come from the HMS and are based on a
+ * much older state of the table.
+ * Note that reading Puffin stats may be disabled by setting the
+ * 'disable_reading_puffin_stats' startup flag or the table property
+ * 'impala.iceberg_disable_reading_puffin_stats' to true.
*/
public class IcebergTable extends Table implements FeIcebergTable {
-
// Alias to the string key that identifies the storage handler for Iceberg tables.
public static final String KEY_STORAGE_HANDLER =
hive_metastoreConstants.META_TABLE_STORAGE;
@@ -90,6 +102,9 @@ public class IcebergTable extends Table implements FeIcebergTable {
// We use database.table instead if this property not been set in SQL
public static final String ICEBERG_TABLE_IDENTIFIER = "iceberg.table_identifier";
+ public static final String ICEBERG_DISABLE_READING_PUFFIN_STATS =
+ "impala.iceberg_disable_reading_puffin_stats";
+
// Internal Iceberg table property that specifies the absolute path of the current
// table metadata. This property is only valid for tables in 'hive.catalog'.
public static final String METADATA_LOCATION = "metadata_location";
@@ -208,6 +223,7 @@ public class IcebergTable extends Table implements FeIcebergTable {
// The snapshot id cached in the CatalogD, necessary to syncronize the caches.
private long catalogSnapshotId_ = -1;
+ private Map icebergFieldIdToCol_;
private Map partitionStats_;
protected IcebergTable(org.apache.hadoop.hive.metastore.api.Table msTable,
@@ -221,6 +237,7 @@ public class IcebergTable extends Table implements FeIcebergTable {
icebergParquetPlainPageSize_ = Utils.getIcebergParquetPlainPageSize(msTable);
icebergParquetDictPageSize_ = Utils.getIcebergParquetDictPageSize(msTable);
hdfsTable_ = new HdfsTable(msTable, db, name, owner);
+ icebergFieldIdToCol_ = new HashMap<>();
}
/**
@@ -355,6 +372,10 @@ public class IcebergTable extends Table implements FeIcebergTable {
return partitionStats_;
}
+ public IcebergColumn getColumnByIcebergFieldId(int fieldId) {
+ return icebergFieldIdToCol_.get(fieldId);
+ }
+
@Override
public TTable toThrift() {
TTable table = super.toThrift();
@@ -422,6 +443,7 @@ public class IcebergTable extends Table implements FeIcebergTable {
partitionStats_ = Utils.loadPartitionStats(this, icebergFiles);
setIcebergTableStats();
loadAllColumnStats(msClient, catalogTimeline);
+ applyPuffinNdvStats(catalogTimeline);
setAvroSchema(msClient, msTbl, fileStore_, catalogTimeline);
// We no longer need to keep Iceberg's content files in memory.
@@ -455,6 +477,56 @@ public class IcebergTable extends Table implements FeIcebergTable {
}
}
+ // Reads NDV stats from Puffin files belonging to the table (if any). Only considers
+ // statistics written for the current snapshot. Overrides NDV stats coming from the HMS
+ // - this is safe because we only consider Puffin stats for the current snapshot, so the
+ // NDV from Puffin is at least as current as the one in the HMS. Note that even if a
+ // value from HMS is overridden here, the new value will not be written back to HMS.
+ // Other stats, e.g. number of nulls, are not modified as Puffin stats only contain NDV
+ // values.
+ private void applyPuffinNdvStats(EventSequence catalogTimeline) {
+ if (BackendConfig.INSTANCE.disableReadingPuffinStats()) return;
+ if (isPuffinStatsReadingDisabledForTable()) return;
+
+ Map puffinNdvs =
+ PuffinStatsLoader.loadPuffinStats(icebergApiTable_, getFullName());
+ for (Map.Entry entry
+ : puffinNdvs.entrySet()) {
+ int fieldId = entry.getKey();
+ long ndv = entry.getValue().ndv;
+
+ // Don't override a possibly existing HMS stat with an explicitly invalid value.
+ if (ndv >= 0) {
+ IcebergColumn col = getColumnByIcebergFieldId(fieldId);
+ Preconditions.checkNotNull(col);
+ Type colType = col.getType();
+ if (ColumnStats.supportsNdv(colType)) {
+ // For some types, e.g. BOOLEAN, HMS does not support NDV stats. We could still
+ // set them here, but it would cause differences between legacy and local
+ // catalog mode: in local catalog mode, the catalog sends the stats in HMS
+ // objects, so NDVs for unsupported types would be lost.
+ col.getStats().setNumDistinctValues(ndv);
+
+ // In local catalog mode, the stats sent from the catalog are those of
+ // 'hdfsTable_', not those of this class.
+ Column hdfsTableCol = hdfsTable_.getColumn(col.getName());
+ Preconditions.checkNotNull(hdfsTableCol);
+ hdfsTableCol.getStats().setNumDistinctValues(ndv);
+ }
+ }
+ }
+
+ if (!puffinNdvs.isEmpty()) {
+ catalogTimeline.markEvent("Loaded Puffin stats");
+ }
+ }
+
+ private boolean isPuffinStatsReadingDisabledForTable() {
+ String val = msTable_.getParameters().get(ICEBERG_DISABLE_READING_PUFFIN_STATS);
+ if (val == null) return false;
+ return Boolean.parseBoolean(val);
+ }
+
/**
* @throws TableLoadingException when it is unsafe to load the table.
*/
@@ -522,6 +594,7 @@ public class IcebergTable extends Table implements FeIcebergTable {
public void addColumn(Column col) {
Preconditions.checkState(col instanceof IcebergColumn);
IcebergColumn iCol = (IcebergColumn) col;
+ icebergFieldIdToCol_.put(iCol.getFieldId(), iCol);
colsByPos_.add(iCol);
colsByName_.put(iCol.getName().toLowerCase(), col);
((StructType) type_.getItemType()).addField(
@@ -529,6 +602,12 @@ public class IcebergTable extends Table implements FeIcebergTable {
iCol.getFieldId()));
}
+ @Override
+ public void clearColumns() {
+ super.clearColumns();
+ icebergFieldIdToCol_.clear();
+ }
+
private void addVirtualColumns() {
addVirtualColumn(VirtualColumn.INPUT_FILE_NAME);
addVirtualColumn(VirtualColumn.FILE_POSITION);
diff --git a/fe/src/main/java/org/apache/impala/catalog/PuffinStatsLoader.java b/fe/src/main/java/org/apache/impala/catalog/PuffinStatsLoader.java
new file mode 100644
index 000000000..29ec83672
--- /dev/null
+++ b/fe/src/main/java/org/apache/impala/catalog/PuffinStatsLoader.java
@@ -0,0 +1,196 @@
+// 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.impala.catalog;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.datasketches.common.SketchesArgumentException;
+import org.apache.datasketches.memory.Memory;
+import org.apache.datasketches.theta.Sketches;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.puffin.BlobMetadata;
+import org.apache.iceberg.puffin.FileMetadata;
+import org.apache.iceberg.puffin.Puffin;
+import org.apache.iceberg.puffin.PuffinReader;
+import org.apache.iceberg.StatisticsFile;
+import org.apache.iceberg.util.Pair;
+
+import org.apache.impala.common.FileSystemUtil;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PuffinStatsLoader {
+ private static final Logger LOG = LoggerFactory.getLogger(PuffinStatsLoader.class);
+
+ private Table iceApiTable_;
+ private String tblName_;
+ private Map result_ = new HashMap<>();
+
+ public static class PuffinStatsRecord {
+ public final StatisticsFile file;
+ public final long ndv;
+
+ public PuffinStatsRecord(StatisticsFile file, long ndv) {
+ this.file = file;
+ this.ndv = ndv;
+ }
+ }
+
+ private PuffinStatsLoader(Table iceApiTable, String tblName) {
+ iceApiTable_ = iceApiTable;
+ tblName_ = tblName;
+ }
+
+ public static Map loadPuffinStats(Table iceApiTable,
+ String tblName) {
+ PuffinStatsLoader loader = new PuffinStatsLoader(iceApiTable, tblName);
+
+ final List statsFiles = iceApiTable.statisticsFiles();
+ for (StatisticsFile statsFile : statsFiles) {
+ loader.loadStatsFromFile(statsFile);
+ }
+ return loader.result_;
+ }
+
+ private void loadStatsFromFile(StatisticsFile statsFile) {
+ final long currentSnapshotId = iceApiTable_.currentSnapshot().snapshotId();
+ if (statsFile.snapshotId() != currentSnapshotId) return;
+
+ // Keep track of the Iceberg column field ids for which we read statistics from this
+ // Puffin file. If we run into an error reading the contents of the file, the file may
+ // be corrupt so we want to remove values already read from it from the overall
+ // result.
+ List fieldIdsFromFile = new ArrayList<>();
+ try {
+ PuffinReader puffinReader = createPuffinReader(statsFile);
+ List blobs = getBlobs(puffinReader, currentSnapshotId);
+
+ // The 'UncheckedIOException' can be thrown from the 'next()' method of the
+ // iterator. Statistics that are loaded successfully before an exception is thrown
+ // are discarded because the file is probably corrupt.
+ for (Pair puffinData: puffinReader.readAll(blobs)) {
+ BlobMetadata blobMetadata = puffinData.first();
+ ByteBuffer blobData = puffinData.second();
+
+ loadStatsFromBlob(blobMetadata, blobData, statsFile, fieldIdsFromFile);
+ }
+ } catch (NotFoundException e) {
+ // 'result_' has not been touched yet.
+ logWarning(tblName_, statsFile.path(), true, e);
+ } catch (Exception e) {
+ // We restore 'result_' to the previous state because the Puffin file may be
+ // corrupt.
+ logWarning(tblName_, statsFile.path(), false, e);
+ result_.keySet().removeAll(fieldIdsFromFile);
+ }
+ }
+
+ private static void logWarning(String tableName, String statsFilePath,
+ boolean fileMissing, Exception e) {
+ String missingStr = fileMissing ? "missing " : "";
+ LOG.warn(String.format("Could not load Iceberg Puffin column statistics "
+ + "for table '%s' from %sPuffin file '%s'. Exception: %s",
+ tableName, missingStr, statsFilePath, e));
+ }
+
+ private static PuffinReader createPuffinReader(StatisticsFile statsFile) {
+ org.apache.iceberg.io.InputFile puffinFile = HadoopInputFile.fromLocation(
+ statsFile.path(), FileSystemUtil.getConfiguration());
+
+ return Puffin.read(puffinFile)
+ .withFileSize(statsFile.fileSizeInBytes())
+ .withFooterSize(statsFile.fileFooterSizeInBytes())
+ .build();
+ }
+
+ private static List getBlobs(PuffinReader puffinReader,
+ long currentSnapshotId) throws java.io.IOException {
+ FileMetadata fileMetadata = puffinReader.fileMetadata();
+ return fileMetadata.blobs().stream()
+ .filter(blob ->
+ blob.snapshotId() == currentSnapshotId &&
+ blob.type().equals("apache-datasketches-theta-v1") &&
+ blob.inputFields().size() == 1)
+ .collect(Collectors.toList());
+ }
+
+ private void loadStatsFromBlob(BlobMetadata blobMetadata, ByteBuffer blobData,
+ StatisticsFile statsFile, List fieldIdsFromFile) {
+ Preconditions.checkState(blobMetadata.inputFields().size() == 1);
+ int fieldId = blobMetadata.inputFields().get(0);
+ if (iceApiTable_.schema().findField(fieldId) == null) {
+ LOG.warn(String.format("Invalid field id %s for table '%s' found "
+ + "in Puffin stats file '%s'. Ignoring blob.",
+ fieldId, tblName_, statsFile.path()));
+ return;
+ }
+
+ double ndv = -1;
+ try {
+ // Memory.wrap(ByteBuffer) would result in an incorrect deserialisation.
+ ndv = Sketches.getEstimate(Memory.wrap(getBytes(blobData)));
+ } catch (SketchesArgumentException e) {
+ String colName = iceApiTable_.schema().idToName().get(fieldId);
+ LOG.warn(String.format("Error reading datasketch for column '%s' of table '%s' "
+ + "from Puffin stats file %s: %s", colName, tblName_, statsFile.path(), e));
+ return;
+ }
+ Preconditions.checkState(ndv != -1);
+
+ long ndvRounded = Math.round(ndv);
+ PuffinStatsRecord record = new PuffinStatsRecord(statsFile, ndvRounded);
+
+ PuffinStatsRecord prevRecord = result_.putIfAbsent(fieldId, record);
+
+ if (prevRecord == null) {
+ fieldIdsFromFile.add(fieldId);
+ } else {
+ String colName = iceApiTable_.schema().idToName().get(fieldId);
+ LOG.warn(String.format("Multiple NDV values from Puffin statistics for column '%s' "
+ + "of table '%s'. Old value (from file %s): %s; new value (from file %s): %s. "
+ + "Using the old value.", colName, tblName_, prevRecord.file.path(),
+ prevRecord.ndv, record.file.path(), record.ndv));
+ }
+ }
+
+ // Gets the bytes from the provided 'ByteBuffer' without advancing buffer position. The
+ // returned byte array may be shared with the buffer.
+ private static byte[] getBytes(ByteBuffer byteBuffer) {
+ if (byteBuffer.hasArray() && byteBuffer.arrayOffset() == 0 &&
+ byteBuffer.position() == 0) {
+ byte[] array = byteBuffer.array();
+ if (byteBuffer.remaining() == array.length) {
+ return array;
+ }
+ }
+
+ byte[] bytes = new byte[byteBuffer.remaining()];
+ byteBuffer.asReadOnlyBuffer().get(bytes);
+ return bytes;
+ }
+}
diff --git a/fe/src/main/java/org/apache/impala/service/BackendConfig.java b/fe/src/main/java/org/apache/impala/service/BackendConfig.java
index e6d26e70a..3ced187a1 100644
--- a/fe/src/main/java/org/apache/impala/service/BackendConfig.java
+++ b/fe/src/main/java/org/apache/impala/service/BackendConfig.java
@@ -448,6 +448,10 @@ public class BackendConfig {
return backendCfg_.data_stream_sender_buffer_size_used_by_planner;
}
+ public boolean disableReadingPuffinStats() {
+ return backendCfg_.disable_reading_puffin_stats;
+ }
+
public boolean isJsonScannerEnabled() {
return backendCfg_.enable_json_scanner;
}
diff --git a/java/pom.xml b/java/pom.xml
index f4a971ecf..5bb81be7e 100644
--- a/java/pom.xml
+++ b/java/pom.xml
@@ -85,6 +85,7 @@ under the License.
${env.IMPALA_LOG4J2_VERSION}
${env.IMPALA_DROPWIZARD_METRICS_VERSION}
${env.IMPALA_AIRCOMPRESSOR_VERSION}
+ ${env.IMPALA_DATASKETCHES_VERSION}
@@ -392,6 +393,7 @@ under the License.
datagenerator
+ puffin-data-generator
executor-deps
ext-data-source
../fe
diff --git a/java/puffin-data-generator/00002-3c6b1ffe-ba85-4be4-a590-7c39428931e1.metadata.json b/java/puffin-data-generator/00002-3c6b1ffe-ba85-4be4-a590-7c39428931e1.metadata.json
new file mode 100644
index 000000000..b749cf4c2
--- /dev/null
+++ b/java/puffin-data-generator/00002-3c6b1ffe-ba85-4be4-a590-7c39428931e1.metadata.json
@@ -0,0 +1,229 @@
+{
+ "format-version" : 1,
+ "table-uuid" : "e3fb99cf-b22e-43c4-b1ca-e9a47770a275",
+ "location" : "hdfs://localhost:20500/test-warehouse/ice_puffin",
+ "last-updated-ms" : 1719573340405,
+ "last-column-id" : 8,
+ "schema" : {
+ "type" : "struct",
+ "schema-id" : 0,
+ "fields" : [ {
+ "id" : 1,
+ "name" : "bool_col",
+ "required" : false,
+ "type" : "boolean"
+ }, {
+ "id" : 2,
+ "name" : "int_col",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 3,
+ "name" : "bigint_col",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 4,
+ "name" : "float_col",
+ "required" : false,
+ "type" : "float"
+ }, {
+ "id" : 5,
+ "name" : "double_col",
+ "required" : false,
+ "type" : "double"
+ }, {
+ "id" : 6,
+ "name" : "date_col",
+ "required" : false,
+ "type" : "date"
+ }, {
+ "id" : 7,
+ "name" : "string_col",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 8,
+ "name" : "timestamp_col",
+ "required" : false,
+ "type" : "timestamp"
+ } ]
+ },
+ "current-schema-id" : 0,
+ "schemas" : [ {
+ "type" : "struct",
+ "schema-id" : 0,
+ "fields" : [ {
+ "id" : 1,
+ "name" : "bool_col",
+ "required" : false,
+ "type" : "boolean"
+ }, {
+ "id" : 2,
+ "name" : "int_col",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 3,
+ "name" : "bigint_col",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 4,
+ "name" : "float_col",
+ "required" : false,
+ "type" : "float"
+ }, {
+ "id" : 5,
+ "name" : "double_col",
+ "required" : false,
+ "type" : "double"
+ }, {
+ "id" : 6,
+ "name" : "date_col",
+ "required" : false,
+ "type" : "date"
+ }, {
+ "id" : 7,
+ "name" : "string_col",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 8,
+ "name" : "timestamp_col",
+ "required" : false,
+ "type" : "timestamp"
+ } ]
+ } ],
+ "partition-spec" : [ ],
+ "default-spec-id" : 0,
+ "partition-specs" : [ {
+ "spec-id" : 0,
+ "fields" : [ ]
+ } ],
+ "last-partition-id" : 999,
+ "default-sort-order-id" : 0,
+ "sort-orders" : [ {
+ "order-id" : 0,
+ "fields" : [ ]
+ } ],
+ "properties" : {
+ "engine.hive.enabled" : "true",
+ "impala.events.catalogServiceId" : "3b153406d7cf46eb:8e8d056f492535db",
+ "external.table.purge" : "TRUE",
+ "impala.events.catalogVersion" : "2058",
+ "write.format.default" : "parquet",
+ "hive.metastore.table.owner" : "danielbecker",
+ "OBJCAPABILITIES" : "EXTREAD,EXTWRITE",
+ "storage_handler" : "org.apache.iceberg.mr.hive.HiveIcebergStorageHandler"
+ },
+ "current-snapshot-id" : 4592649074073110045,
+ "refs" : {
+ "main" : {
+ "snapshot-id" : 4592649074073110045,
+ "type" : "branch"
+ }
+ },
+ "snapshots" : [ {
+ "snapshot-id" : 4592649074073110045,
+ "timestamp-ms" : 1719573336119,
+ "summary" : {
+ "operation" : "append",
+ "added-data-files" : "1",
+ "added-records" : "1",
+ "added-files-size" : "1842",
+ "changed-partition-count" : "1",
+ "total-records" : "1",
+ "total-files-size" : "1842",
+ "total-data-files" : "1",
+ "total-delete-files" : "0",
+ "total-position-deletes" : "0",
+ "total-equality-deletes" : "0"
+ },
+ "manifest-list" : "hdfs://localhost:20500/test-warehouse/ice_puffin/metadata/snap-4592649074073110045-1-2f2c61f0-0886-465c-af09-251b06c5346d.avro",
+ "schema-id" : 0
+ } ],
+ "statistics" : [ {
+ "snapshot-id" : 4592649074073110045,
+ "statistics-path" : "hdfs://localhost:20500/test-warehouse/ice_puffin/metadata/20240628_111540_00004_ihhpt-f0149596-b20a-4156-bcfd-ff74a60996ad.stats",
+ "file-size-in-bytes" : 1721,
+ "file-footer-size-in-bytes" : 1541,
+ "blob-metadata" : [ {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 4592649074073110045,
+ "sequence-number" : 0,
+ "fields" : [ 1 ],
+ "properties" : {
+ "ndv" : "1"
+ }
+ }, {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 4592649074073110045,
+ "sequence-number" : 0,
+ "fields" : [ 2 ],
+ "properties" : {
+ "ndv" : "0"
+ }
+ }, {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 4592649074073110045,
+ "sequence-number" : 0,
+ "fields" : [ 3 ],
+ "properties" : {
+ "ndv" : "0"
+ }
+ }, {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 4592649074073110045,
+ "sequence-number" : 0,
+ "fields" : [ 4 ],
+ "properties" : {
+ "ndv" : "0"
+ }
+ }, {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 4592649074073110045,
+ "sequence-number" : 0,
+ "fields" : [ 5 ],
+ "properties" : {
+ "ndv" : "0"
+ }
+ }, {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 4592649074073110045,
+ "sequence-number" : 0,
+ "fields" : [ 6 ],
+ "properties" : {
+ "ndv" : "0"
+ }
+ }, {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 4592649074073110045,
+ "sequence-number" : 0,
+ "fields" : [ 7 ],
+ "properties" : {
+ "ndv" : "0"
+ }
+ }, {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 4592649074073110045,
+ "sequence-number" : 0,
+ "fields" : [ 8 ],
+ "properties" : {
+ "ndv" : "0"
+ }
+ } ]
+ } ],
+ "partition-statistics" : [ ],
+ "snapshot-log" : [ {
+ "timestamp-ms" : 1719573336119,
+ "snapshot-id" : 4592649074073110045
+ } ],
+ "metadata-log" : [ {
+ "timestamp-ms" : 1719572987102,
+ "metadata-file" : "hdfs://localhost:20500/test-warehouse/ice_puffin/metadata/00000-85d0bcdf-b7b7-4263-97e7-c8531f54e14e.metadata.json"
+ }, {
+ "timestamp-ms" : 1719573336148,
+ "metadata-file" : "hdfs://localhost:20500/test-warehouse/ice_puffin/metadata/00001-bd5ad524-134a-4ca8-9982-4e9eeaac7991.metadata.json"
+ } ]
+}
\ No newline at end of file
diff --git a/java/puffin-data-generator/pom.xml b/java/puffin-data-generator/pom.xml
new file mode 100644
index 000000000..9688b1f6f
--- /dev/null
+++ b/java/puffin-data-generator/pom.xml
@@ -0,0 +1,109 @@
+
+
+
+
+
+
+ org.apache.impala
+ impala-parent
+ 4.5.0-SNAPSHOT
+
+ 4.0.0
+
+ impala-puffin-data-generator
+ jar
+
+ Puffin Test Data Generator
+
+
+
+ org.apache.hadoop
+ hadoop-common
+ ${hadoop.version}
+
+
+ org.eclipse.jetty
+ *
+
+
+
+ io.netty
+ *
+
+
+ com.sun.jersey
+ jersey-server
+
+
+ com.sun.jersey
+ jersey-servlet
+
+
+
+
+
+ org.apache.iceberg
+ iceberg-api
+ ${iceberg.version}
+
+
+
+ org.apache.iceberg
+ iceberg-hive-runtime
+ ${iceberg.version}
+
+
+
+
+ org.apache.datasketches
+ datasketches-java
+ ${datasketches.version}
+
+
+
+ com.fasterxml.jackson.core
+ jackson-databind
+
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-compiler-plugin
+ 3.11.0
+
+ 1.8
+ 1.8
+
+
+
+
+ org.apache.maven.plugins
+ maven-surefire-plugin
+ 3.0.0
+
+ true
+
+
+
+
+
+
diff --git a/java/puffin-data-generator/src/main/java/org/apache/impala/puffindatagenerator/PuffinDataGenerator.java b/java/puffin-data-generator/src/main/java/org/apache/impala/puffindatagenerator/PuffinDataGenerator.java
new file mode 100644
index 000000000..c599262a7
--- /dev/null
+++ b/java/puffin-data-generator/src/main/java/org/apache/impala/puffindatagenerator/PuffinDataGenerator.java
@@ -0,0 +1,542 @@
+// 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.impala.puffindatagenerator;
+
+import java.io.BufferedWriter;
+import java.nio.ByteBuffer;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.InputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.UnsupportedEncodingException;
+import java.io.Writer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.Scanner;
+
+import org.apache.datasketches.theta.UpdateSketch;
+
+import org.apache.iceberg.puffin.Blob;
+import org.apache.iceberg.puffin.Puffin;
+import org.apache.iceberg.puffin.PuffinCompressionCodec;
+import org.apache.iceberg.puffin.PuffinWriter;
+import org.apache.iceberg.hadoop.HadoopOutputFile;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+/**
+ * This class is used to generate data for testing the reading of Puffin files.
+ *
+ * We create data for different scenarios, e.g. all statistics are in the same file,
+ * statistics are in different files, some Puffin files are corrupt etc. For each scenario
+ * we generate (one or more) Puffin files and a metadata.json file. All of these can be
+ * copied into the directory of the table. To activate a scenario, set its metadata.json
+ * file as the current metadata file of the table.
+ * Note that the UUID and the table location in the metadata.json file must match those of
+ * the table for which the metadata.json file is used. To facilitate this, the generated
+ * metadata.json files contain placeholders for these values: UUID_PLACEHOLDER and
+ * TABLE_LOCATION_PLACEHOLDER. These placeholders in the files can be easily replaced with
+ * the actual values obtained from the table, e.g. using the 'sed' tool.
+ *
+ * The metadata.json files are generated based on an existing metadata.json file (a
+ * template). This should match the schema of the table for which we are generating
+ * metadata. The statistics in the generated Puffin files do not normally match the actual
+ * data in the table. The template metadata.json file can be taken from a newly created
+ * and truncated table (truncation is needed so that a snapshot exists).
+ */
+public class PuffinDataGenerator {
+ // The table for which we generate data can be created this way:
+ // CREATE TABLE ice_puffin(
+ // int_col1 INT,
+ // int_col2 INT,
+ // bigint_col BIGINT,
+ // float_col FLOAT,
+ // double_col DOUBLE,
+ // decimal_col DECIMAL,
+ // date_col DATE,
+ // string_col STRING,
+ // timestamp_col TIMESTAMP,
+ // bool_col BOOLEAN)
+ // STORED BY ICEBERG
+
+ public static String SKETCH_TYPE = "apache-datasketches-theta-v1";
+ public static final String TABLE_LOCATION_PLACEHOLDER = "TABLE_LOCATION_PLACEHOLDER";
+ public static final String UUID_PLACEHOLDER = "UUID_PLACEHOLDER";
+ public static final long SEQUENCE_NUMBER = 0;
+
+ public static final List sketches = createSketches();
+
+ private final String localOutputDir_;
+ private final long snapshotId_;
+ private final ObjectMapper mapper_;
+ private final JsonNode metadataJsonTemplate_;
+
+ private static class FileData {
+ public final String filename;
+ public final long snapshotId;
+ public final List blobs;
+ public final boolean compressBlobs;
+ // Footer compression is not supported yet by Iceberg. The spec only allows the footer
+ // to be compressed with LZ4 but the Iceberg library can't handle LZ4 yet.
+ public final boolean compressFooter;
+ // If true, do not write file, only the 'statistics' section in metadata.json.
+ public final boolean missingFile;
+
+ public FileData(String filename, long snapshotId, List blobs,
+ boolean compressBlobs, boolean missingFile) {
+ this.filename = filename;
+ this.snapshotId = snapshotId;
+ this.blobs = blobs;
+ this.compressBlobs = compressBlobs;
+ this.compressFooter = false;
+ this.missingFile = missingFile;
+ }
+
+ public FileData(String filename, long snapshotId, List blobs,
+ boolean compressBlobs) {
+ this(filename, snapshotId, blobs, compressBlobs, false);
+ }
+ }
+
+ public static void main(String[] args) throws FileNotFoundException, IOException {
+ final String metadataJsonTemplatePath =
+ "./testdata/ice_puffin/00001-2e1ade02-35ae-4a8f-a84f-784d1e0c0790.metadata.json";
+ final String localOutputDir = "./puffin_files/";
+ PuffinDataGenerator generator = new PuffinDataGenerator(
+ metadataJsonTemplatePath, localOutputDir);
+
+ generator.writeFileWithAllStats();
+ generator.writeAllStatsTwoFiles();
+ generator.writeDuplicateStatsInFile();
+ generator.writeDuplicateStatsInTwoFiles();
+ generator.writeOneFileCurrentOneNot();
+ generator.writeNotAllBlobsCurrent();
+ generator.writeMissingFile();
+ generator.writeOneFileCorruptOneNot();
+ generator.writeAllFilesCorrupt();
+ generator.writeFileContainsInvalidFieldId();
+ generator.writeStatForUnsupportedType();
+ generator.writeFileWithInvalidAndCorruptSketches();
+ }
+
+ public PuffinDataGenerator(String metadataJsonTemplatePath, String localOutputDir)
+ throws java.io.FileNotFoundException, JsonProcessingException {
+ localOutputDir_ = localOutputDir;
+
+ String metadataJsonStr = new Scanner(new File(metadataJsonTemplatePath))
+ .useDelimiter("\\Z").next();
+
+ snapshotId_ = getSnapshotIdFromMetadataJson(metadataJsonStr);
+
+ String tableLocation = getTableLocationFromMetadataJson(metadataJsonStr);
+ metadataJsonStr = metadataJsonStr.replace(tableLocation, TABLE_LOCATION_PLACEHOLDER);
+
+ mapper_ = new ObjectMapper();
+ metadataJsonTemplate_ = mapper_.readTree(metadataJsonStr);
+ }
+
+ private static long getSnapshotIdFromMetadataJson(String metadataJsonStr) {
+ Pattern regex = Pattern.compile("\"current-snapshot-id\" ?: ?([0-9]+)");
+ Matcher matcher = regex.matcher(metadataJsonStr);
+ boolean match = matcher.find();
+ Preconditions.checkState(match);
+ String snapshotIdStr = matcher.group(1);
+ return Long.parseLong(snapshotIdStr);
+ }
+
+ private static String getTableLocationFromMetadataJson(String metadataJsonStr) {
+ Pattern regex = Pattern.compile("\"location\" ?: ?\"(.*)\"");
+ Matcher matcher = regex.matcher(metadataJsonStr);
+ boolean match = matcher.find();
+ Preconditions.checkState(match);
+ return matcher.group(1);
+ }
+
+ private String getPuffinFilePrefix() {
+ return TABLE_LOCATION_PLACEHOLDER + "/metadata/";
+ }
+
+ // All stats are in the same Puffin file.
+ private void writeFileWithAllStats()
+ throws IOException {
+ List blobs = new ArrayList<>();
+
+ blobs.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 1, 1));
+ blobs.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 2, 2));
+ blobs.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 3, 3));
+ blobs.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 4, 4));
+ blobs.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 5, 5));
+ blobs.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 6, 6));
+ blobs.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 7, 7));
+ blobs.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 8, 8));
+ blobs.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 9, 9));
+
+ List puffinFiles = new ArrayList<>();
+ puffinFiles.add(new FileData("all_stats.stats", snapshotId_, blobs, false));
+ writeFilesForScenario(puffinFiles, "all_stats_in_1_file.metadata.json");
+ }
+
+ // The stats are in two separate Puffin files.
+ private void writeAllStatsTwoFiles()
+ throws IOException {
+ List blobs1 = new ArrayList<>();
+ blobs1.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 1, 1));
+ blobs1.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 2, 2));
+ blobs1.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 3, 3));
+ blobs1.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 4, 4));
+
+ List blobs2 = new ArrayList<>();
+ blobs2.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 5, 5));
+ blobs2.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 6, 6));
+ blobs2.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 7, 7));
+ blobs2.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 8, 8));
+ blobs2.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 9, 9));
+
+ List puffinFiles = new ArrayList<>();
+ puffinFiles.add(
+ new FileData("stats_divided1.stats", snapshotId_, blobs1, false));
+ puffinFiles.add(
+ new FileData("stats_divided2.stats", snapshotId_, blobs2, true));
+ writeFilesForScenario(puffinFiles, "stats_divided.metadata.json");
+ }
+
+ // There are duplicate stats for some column(s) in the same Puffin file. The first value
+ // should be used.
+ private void writeDuplicateStatsInFile()
+ throws IOException {
+ List blobs = new ArrayList<>();
+
+ blobs.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 1, 1));
+ blobs.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 2, 2));
+ blobs.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 1, 3));
+
+ List puffinFiles = new ArrayList<>();
+ puffinFiles.add(new FileData(
+ "duplicate_stats_in_1_file.stats", snapshotId_, blobs, true));
+ writeFilesForScenario(puffinFiles, "duplicate_stats_in_1_file.metadata.json");
+ }
+
+ // There are duplicate stats for some column(s) in separate Puffin files. The first
+ // value should be used.
+ private void writeDuplicateStatsInTwoFiles()
+ throws IOException {
+ List blobs1 = new ArrayList<>();
+ blobs1.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 1, 1));
+ blobs1.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 2, 2));
+
+ List blobs2 = new ArrayList<>();
+ blobs2.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 1, 5));
+ blobs2.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 3, 3));
+
+ List puffinFiles = new ArrayList<>();
+ puffinFiles.add(new FileData(
+ "duplicate_stats_in_2_files1.stats", snapshotId_, blobs1, true));
+ puffinFiles.add(new FileData(
+ "duplicate_stats_in_2_files2.stats", snapshotId_, blobs2, false));
+ writeFilesForScenario(puffinFiles, "duplicate_stats_in_2_files.metadata.json");
+ }
+
+ // One Puffin file is for the current snapshot while another is not.
+ private void writeOneFileCurrentOneNot() throws IOException {
+ List blobs1 = new ArrayList<>();
+ blobs1.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 1, 1));
+ blobs1.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 2, 2));
+
+ List blobs2 = new ArrayList<>();
+ long notCurrentSnapshotId = snapshotId_ - 1;
+ blobs2.add(createBlob(notCurrentSnapshotId, SEQUENCE_NUMBER, 3, 3));
+ blobs2.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 4, 4));
+
+ List puffinFiles = new ArrayList<>();
+ puffinFiles.add(new FileData(
+ "current_snapshot_id.stats", snapshotId_, blobs1, true));
+ puffinFiles.add(new FileData(
+ "not_current_snapshot_id.stats", notCurrentSnapshotId, blobs2, true));
+ writeFilesForScenario(puffinFiles, "one_file_current_one_not.metadata.json");
+ }
+
+ // Some blobs are for the current snapshot while some are not.
+ private void writeNotAllBlobsCurrent() throws IOException {
+ long notCurrentSnapshotId = snapshotId_ - 1;
+ List blobs = new ArrayList<>();
+ blobs.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 1, 1));
+ blobs.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 2, 2));
+ blobs.add(createBlob(notCurrentSnapshotId, SEQUENCE_NUMBER, 3, 3));
+ blobs.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 4, 4));
+
+ List puffinFiles = new ArrayList<>();
+ puffinFiles.add(new FileData(
+ "not_all_blobs_current.stats", snapshotId_, blobs, true));
+ writeFilesForScenario(puffinFiles, "not_all_blobs_current.metadata.json");
+ }
+
+ // One of the Puffin files is missing. The other file(s) should be taken into account.
+ private void writeMissingFile() throws IOException {
+ List blobs1 = new ArrayList<>();
+ blobs1.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 1, 1));
+ blobs1.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 2, 2));
+
+ List blobs2 = new ArrayList<>();
+ blobs2.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 3, 3));
+ blobs2.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 4, 4));
+
+ List puffinFiles = new ArrayList<>();
+ puffinFiles.add(new FileData(
+ "missing_file.stats", snapshotId_, blobs1, false, true));
+ puffinFiles.add(new FileData("existing_file.stats", snapshotId_, blobs2, true));
+ writeFilesForScenario(puffinFiles, "missing_file.metadata.json");
+ }
+
+ // One of the Puffin files is corrupt, the other is not. The other file should be taken
+ // into account.
+ private void writeOneFileCorruptOneNot() throws IOException {
+ List blobs1 = new ArrayList<>();
+ blobs1.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 1, 1));
+ blobs1.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 2, 2));
+ FileData corruptFile = new FileData(
+ "corrupt_file.stats", snapshotId_, blobs1, false);
+
+ List blobs2 = new ArrayList<>();
+ blobs2.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 3, 3));
+ blobs2.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 4, 4));
+ FileData nonCorruptFile = new FileData(
+ "non_corrupt_file.stats", snapshotId_, blobs2, false);
+
+ List puffinFiles = new ArrayList<>();
+ puffinFiles.add(corruptFile);
+ puffinFiles.add(nonCorruptFile);
+ writeFilesForScenario(puffinFiles, "one_file_corrupt_one_not.metadata.json");
+
+ this.corruptFile(corruptFile.filename);
+ }
+
+ private void writeAllFilesCorrupt() throws IOException {
+ List blobs1 = new ArrayList<>();
+ blobs1.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 1, 1));
+ blobs1.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 2, 2));
+ FileData corruptFile1 = new FileData(
+ "corrupt_file1.stats", snapshotId_, blobs1, true);
+
+ List blobs2 = new ArrayList<>();
+ blobs2.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 3, 3));
+ blobs2.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 4, 4));
+ FileData corruptFile2 = new FileData(
+ "corrupt_file2.stats", snapshotId_, blobs2, true);
+
+ List puffinFiles = new ArrayList<>();
+ puffinFiles.add(corruptFile1);
+ puffinFiles.add(corruptFile2);
+ writeFilesForScenario(puffinFiles, "all_files_corrupt.metadata.json");
+
+ this.corruptFile(corruptFile1.filename);
+ this.corruptFile(corruptFile2.filename);
+ }
+
+ private void writeFileContainsInvalidFieldId() throws IOException {
+ List blobs = new ArrayList<>();
+ blobs.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 1, 1));
+ int invalid_field_id = 200;
+ blobs.add(createBlob(snapshotId_, SEQUENCE_NUMBER, invalid_field_id, 2));
+ FileData corruptFile1 = new FileData(
+ "file_contains_invalid_field_id.stats", snapshotId_, blobs, true);
+
+
+ List puffinFiles = new ArrayList<>();
+ puffinFiles.add(corruptFile1);
+ writeFilesForScenario(puffinFiles, "file_contains_invalid_field_id.metadata.json");
+ }
+
+ private void writeStatForUnsupportedType() throws IOException {
+ List blobs = new ArrayList<>();
+ blobs.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 1, 2));
+ int unsupported_field_id = 10;
+ blobs.add(createBlob(snapshotId_, SEQUENCE_NUMBER, unsupported_field_id, 2));
+ FileData corruptFile1 = new FileData(
+ "stats_for_unsupported_type.stats", snapshotId_, blobs, true);
+
+ List puffinFiles = new ArrayList<>();
+ puffinFiles.add(corruptFile1);
+ writeFilesForScenario(puffinFiles, "stats_for_unsupported_type.metadata.json");
+ }
+
+ private void writeFileWithInvalidAndCorruptSketches() throws IOException {
+ List blobs = new ArrayList<>();
+ blobs.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 1, 1));
+
+ // Sketch with invalid type
+ final String invalidSketchType = "invalidSketchType";
+ blobs.add(new Blob(invalidSketchType, Arrays.asList(2), snapshotId_,
+ SEQUENCE_NUMBER, sketches.get(1)));
+
+ blobs.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 3, 3));
+
+ // Corrupt sketch.
+ byte[] bytes = {0, 0};
+ ByteBuffer corruptSketch = ByteBuffer.wrap(bytes);
+ blobs.add(new Blob(SKETCH_TYPE, Arrays.asList(4), snapshotId_, SEQUENCE_NUMBER,
+ corruptSketch));
+
+ blobs.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 5, 5));
+
+ FileData fileData = new FileData(
+ "invalidAndCorruptSketches.stats", snapshotId_, blobs, true);
+ List puffinFiles = new ArrayList<>();
+ puffinFiles.add(fileData);
+ writeFilesForScenario(puffinFiles, "invalidAndCorruptSketches.metadata.json");
+ }
+
+ private static ByteBuffer createSketchWithNdv(int ndv) {
+ UpdateSketch sketch = UpdateSketch.builder().build();
+ for (int i = 0; i < ndv; i++) sketch.update(i);
+ return ByteBuffer.wrap(sketch.compact().toByteArray());
+ }
+
+ private static List createSketches() {
+ ImmutableList.Builder builder = new ImmutableList.Builder<>();
+ for (int i = 1; i <= 9; i++) {
+ builder.add(createSketchWithNdv(i));
+ }
+ return builder.build();
+ }
+
+ private static Blob createBlob(long snapshotId, long sequenceNumber,
+ int fieldId, int ndv) {
+ return new Blob(SKETCH_TYPE, Arrays.asList(fieldId), snapshotId, sequenceNumber,
+ sketches.get(ndv-1));
+ }
+
+ private void writeFilesForScenario(List puffinFiles, String statsJsonFile)
+ throws IOException {
+ ArrayNode jsonStatsList = mapper_.createArrayNode();
+ for (FileData fileData : puffinFiles) {
+ jsonStatsList.add(writeBlobsToFile(fileData));
+ }
+ writeMetadataJsonWithStatsToFile(statsJsonFile, jsonStatsList);
+ }
+
+ private ObjectNode writeBlobsToFile(FileData fileData) throws IOException {
+ String localOutfile = localOutputDir_ + fileData.filename;
+
+ // These values are used if we don't actually write a file ('fileData.missingFile' is
+ // true). These are the values with a file with two blobs.
+ long fileSize = 340;
+ long footerSize = 288;
+
+ if (!fileData.missingFile) {
+ Puffin.WriteBuilder writeBuilder = Puffin.write(
+ HadoopOutputFile.fromLocation(localOutfile,
+ new org.apache.hadoop.conf.Configuration()));
+ writeBuilder.createdBy("Impala Puffin Data Generator");
+ if (fileData.compressBlobs) {
+ writeBuilder.compressBlobs(PuffinCompressionCodec.ZSTD);
+ }
+ if (fileData.compressFooter) writeBuilder.compressFooter();
+
+ PuffinWriter writer = writeBuilder.build();
+ for (Blob blob : fileData.blobs) writer.add(blob);
+ writer.finish();
+ writer.close();
+
+ fileSize = writer.fileSize();
+ footerSize = writer.footerSize();
+ }
+
+
+ ObjectNode statsNode = mapper_.createObjectNode();
+ statsNode.put("snapshot-id", fileData.snapshotId);
+ statsNode.put("statistics-path", getPuffinFilePrefix() + fileData.filename);
+ statsNode.put("file-size-in-bytes", fileSize);
+ statsNode.put("file-footer-size-in-bytes", footerSize);
+
+ statsNode.put("blob-metadata", blobsToJson(fileData.blobs));
+ return statsNode;
+ }
+
+ private ArrayNode blobsToJson(List blobs) throws JsonProcessingException {
+ ArrayNode list = mapper_.createArrayNode();
+ for (Blob blob : blobs) list.add(blobMetadataToJson(blob));
+ return list;
+ }
+
+ private ObjectNode blobMetadataToJson(Blob blob) throws JsonProcessingException {
+ ObjectNode blobNode = mapper_.createObjectNode();
+
+ blobNode.put("type", blob.type());
+ blobNode.put("snapshot-id", blob.snapshotId());
+ blobNode.put("sequence-number", blob.sequenceNumber());
+
+ ArrayNode fieldsList = mapper_.createArrayNode();
+ for (int fieldId : blob.inputFields()) fieldsList.add(fieldId);
+ blobNode.set("fields", fieldsList);
+
+ return blobNode;
+ }
+
+ private void writeMetadataJsonWithStatsToFile(String outfile, ArrayNode stats)
+ throws IOException {
+ JsonNode metadataJson = metadataJsonTemplate_.deepCopy();
+
+ // Replace UUID with a placeholder.
+ String uuidKey = "table-uuid";
+ ObjectNode uuidParent = (ObjectNode) metadataJson.findParent(uuidKey);
+ uuidParent.put(uuidKey, UUID_PLACEHOLDER);
+
+ ObjectNode statsParent = (ObjectNode) metadataJson.findParent("statistics");
+ statsParent.put("statistics", stats);
+
+ String outfilePath = localOutputDir_ + outfile;
+ try (Writer writer = new BufferedWriter(new OutputStreamWriter(
+ new FileOutputStream(outfilePath), "utf-8"))) {
+ String jsonString = mapper_.writerWithDefaultPrettyPrinter()
+ .writeValueAsString(metadataJson);
+ writer.write(jsonString);
+ }
+ }
+
+ // Re-write the file without the magic (first 4 bytes) and some additional bytes.
+ private void corruptFile(String filename) throws FileNotFoundException, IOException {
+ String filePath = localOutputDir_ + filename;
+
+ int fileSize = (int) new File(filePath).length();
+ byte[] bytes = new byte[fileSize];
+
+ try (InputStream inputStream = new FileInputStream(filePath)) {
+ int bytesRead = inputStream.read(bytes);
+ }
+
+ try (OutputStream outputStream = new FileOutputStream(filePath)) {
+ final int magicLength = 4;
+ final int bytesToOmit = magicLength + 4;
+ outputStream.write(bytes, bytesToOmit, bytes.length - bytesToOmit);
+ }
+ }
+}
diff --git a/testdata/data/iceberg_test/iceberg_with_puffin_stats/data/0747babcda9277bf-954aff1b00000000_1684663509_data.0.parq b/testdata/data/iceberg_test/iceberg_with_puffin_stats/data/0747babcda9277bf-954aff1b00000000_1684663509_data.0.parq
new file mode 100644
index 000000000..f243f30d5
Binary files /dev/null and b/testdata/data/iceberg_test/iceberg_with_puffin_stats/data/0747babcda9277bf-954aff1b00000000_1684663509_data.0.parq differ
diff --git a/testdata/data/iceberg_test/iceberg_with_puffin_stats/metadata/11cd04ec-55ea-40aa-a89b-197c3c275e7a-m0.avro b/testdata/data/iceberg_test/iceberg_with_puffin_stats/metadata/11cd04ec-55ea-40aa-a89b-197c3c275e7a-m0.avro
new file mode 100644
index 000000000..6921b7290
Binary files /dev/null and b/testdata/data/iceberg_test/iceberg_with_puffin_stats/metadata/11cd04ec-55ea-40aa-a89b-197c3c275e7a-m0.avro differ
diff --git a/testdata/data/iceberg_test/iceberg_with_puffin_stats/metadata/20240906_085606_00006_wsfgs-4d9242d5-bd79-4069-be8b-2cfced8e0647.stats b/testdata/data/iceberg_test/iceberg_with_puffin_stats/metadata/20240906_085606_00006_wsfgs-4d9242d5-bd79-4069-be8b-2cfced8e0647.stats
new file mode 100644
index 000000000..98e388477
Binary files /dev/null and b/testdata/data/iceberg_test/iceberg_with_puffin_stats/metadata/20240906_085606_00006_wsfgs-4d9242d5-bd79-4069-be8b-2cfced8e0647.stats differ
diff --git a/testdata/data/iceberg_test/iceberg_with_puffin_stats/metadata/snap-1880359224532128423-1-11cd04ec-55ea-40aa-a89b-197c3c275e7a.avro b/testdata/data/iceberg_test/iceberg_with_puffin_stats/metadata/snap-1880359224532128423-1-11cd04ec-55ea-40aa-a89b-197c3c275e7a.avro
new file mode 100644
index 000000000..555d8e73b
Binary files /dev/null and b/testdata/data/iceberg_test/iceberg_with_puffin_stats/metadata/snap-1880359224532128423-1-11cd04ec-55ea-40aa-a89b-197c3c275e7a.avro differ
diff --git a/testdata/data/iceberg_test/iceberg_with_puffin_stats/metadata/v1.metadata.json b/testdata/data/iceberg_test/iceberg_with_puffin_stats/metadata/v1.metadata.json
new file mode 100644
index 000000000..fb1d1ca73
--- /dev/null
+++ b/testdata/data/iceberg_test/iceberg_with_puffin_stats/metadata/v1.metadata.json
@@ -0,0 +1,64 @@
+{
+ "format-version" : 1,
+ "table-uuid" : "1804053c-6ba0-48df-8d22-f1bd139a3635",
+ "location" : "/test-warehouse/iceberg_test/iceberg_with_puffin_stats",
+ "last-updated-ms" : 1725612488531,
+ "last-column-id" : 2,
+ "schema" : {
+ "type" : "struct",
+ "schema-id" : 0,
+ "fields" : [ {
+ "id" : 1,
+ "name" : "i",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 2,
+ "name" : "d",
+ "required" : false,
+ "type" : "decimal(9, 0)"
+ } ]
+ },
+ "current-schema-id" : 0,
+ "schemas" : [ {
+ "type" : "struct",
+ "schema-id" : 0,
+ "fields" : [ {
+ "id" : 1,
+ "name" : "i",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 2,
+ "name" : "d",
+ "required" : false,
+ "type" : "decimal(9, 0)"
+ } ]
+ } ],
+ "partition-spec" : [ ],
+ "default-spec-id" : 0,
+ "partition-specs" : [ {
+ "spec-id" : 0,
+ "fields" : [ ]
+ } ],
+ "last-partition-id" : 999,
+ "default-sort-order-id" : 0,
+ "sort-orders" : [ {
+ "order-id" : 0,
+ "fields" : [ ]
+ } ],
+ "properties" : {
+ "engine.hive.enabled" : "true",
+ "external.table.purge" : "TRUE",
+ "write.format.default" : "parquet",
+ "hive.metastore.table.owner" : "danielbecker",
+ "OBJCAPABILITIES" : "EXTREAD,EXTWRITE",
+ "storage_handler" : "org.apache.iceberg.mr.hive.HiveIcebergStorageHandler"
+ },
+ "current-snapshot-id" : -1,
+ "refs" : { },
+ "snapshots" : [ ],
+ "statistics" : [ ],
+ "snapshot-log" : [ ],
+ "metadata-log" : [ ]
+}
\ No newline at end of file
diff --git a/testdata/data/iceberg_test/iceberg_with_puffin_stats/metadata/v2.metadata.json b/testdata/data/iceberg_test/iceberg_with_puffin_stats/metadata/v2.metadata.json
new file mode 100644
index 000000000..32a167382
--- /dev/null
+++ b/testdata/data/iceberg_test/iceberg_with_puffin_stats/metadata/v2.metadata.json
@@ -0,0 +1,95 @@
+{
+ "format-version" : 1,
+ "table-uuid" : "1804053c-6ba0-48df-8d22-f1bd139a3635",
+ "location" : "/test-warehouse/iceberg_test/iceberg_with_puffin_stats",
+ "last-updated-ms" : 1725612500133,
+ "last-column-id" : 2,
+ "schema" : {
+ "type" : "struct",
+ "schema-id" : 0,
+ "fields" : [ {
+ "id" : 1,
+ "name" : "i",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 2,
+ "name" : "d",
+ "required" : false,
+ "type" : "decimal(9, 0)"
+ } ]
+ },
+ "current-schema-id" : 0,
+ "schemas" : [ {
+ "type" : "struct",
+ "schema-id" : 0,
+ "fields" : [ {
+ "id" : 1,
+ "name" : "i",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 2,
+ "name" : "d",
+ "required" : false,
+ "type" : "decimal(9, 0)"
+ } ]
+ } ],
+ "partition-spec" : [ ],
+ "default-spec-id" : 0,
+ "partition-specs" : [ {
+ "spec-id" : 0,
+ "fields" : [ ]
+ } ],
+ "last-partition-id" : 999,
+ "default-sort-order-id" : 0,
+ "sort-orders" : [ {
+ "order-id" : 0,
+ "fields" : [ ]
+ } ],
+ "properties" : {
+ "engine.hive.enabled" : "true",
+ "impala.events.catalogServiceId" : "a6b8da59a60e425a:ad0468614a61e340",
+ "external.table.purge" : "TRUE",
+ "impala.events.catalogVersion" : "4097",
+ "write.format.default" : "parquet",
+ "hive.metastore.table.owner" : "danielbecker",
+ "OBJCAPABILITIES" : "EXTREAD,EXTWRITE",
+ "storage_handler" : "org.apache.iceberg.mr.hive.HiveIcebergStorageHandler"
+ },
+ "current-snapshot-id" : 1880359224532128423,
+ "refs" : {
+ "main" : {
+ "snapshot-id" : 1880359224532128423,
+ "type" : "branch"
+ }
+ },
+ "snapshots" : [ {
+ "snapshot-id" : 1880359224532128423,
+ "timestamp-ms" : 1725612500115,
+ "summary" : {
+ "operation" : "append",
+ "added-data-files" : "1",
+ "added-records" : "2",
+ "added-files-size" : "604",
+ "changed-partition-count" : "1",
+ "total-records" : "2",
+ "total-files-size" : "604",
+ "total-data-files" : "1",
+ "total-delete-files" : "0",
+ "total-position-deletes" : "0",
+ "total-equality-deletes" : "0"
+ },
+ "manifest-list" : "/test-warehouse/iceberg_test/iceberg_with_puffin_stats/metadata/snap-1880359224532128423-1-11cd04ec-55ea-40aa-a89b-197c3c275e7a.avro",
+ "schema-id" : 0
+ } ],
+ "statistics" : [ ],
+ "snapshot-log" : [ {
+ "timestamp-ms" : 1725612500115,
+ "snapshot-id" : 1880359224532128423
+ } ],
+ "metadata-log" : [ {
+ "timestamp-ms" : 1725612488531,
+ "metadata-file" : "/test-warehouse/iceberg_test/iceberg_with_puffin_stats/metadata/00000-c1fdc549-6895-4b22-987b-b1c229294749.metadata.json"
+ } ]
+}
\ No newline at end of file
diff --git a/testdata/data/iceberg_test/iceberg_with_puffin_stats/metadata/v3.metadata.json b/testdata/data/iceberg_test/iceberg_with_puffin_stats/metadata/v3.metadata.json
new file mode 100644
index 000000000..1cf15ea77
--- /dev/null
+++ b/testdata/data/iceberg_test/iceberg_with_puffin_stats/metadata/v3.metadata.json
@@ -0,0 +1,121 @@
+{
+ "format-version" : 1,
+ "table-uuid" : "1804053c-6ba0-48df-8d22-f1bd139a3635",
+ "location" : "/test-warehouse/iceberg_test/iceberg_with_puffin_stats",
+ "last-updated-ms" : 1725612966926,
+ "last-column-id" : 2,
+ "schema" : {
+ "type" : "struct",
+ "schema-id" : 0,
+ "fields" : [ {
+ "id" : 1,
+ "name" : "i",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 2,
+ "name" : "d",
+ "required" : false,
+ "type" : "decimal(9, 0)"
+ } ]
+ },
+ "current-schema-id" : 0,
+ "schemas" : [ {
+ "type" : "struct",
+ "schema-id" : 0,
+ "fields" : [ {
+ "id" : 1,
+ "name" : "i",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 2,
+ "name" : "d",
+ "required" : false,
+ "type" : "decimal(9, 0)"
+ } ]
+ } ],
+ "partition-spec" : [ ],
+ "default-spec-id" : 0,
+ "partition-specs" : [ {
+ "spec-id" : 0,
+ "fields" : [ ]
+ } ],
+ "last-partition-id" : 999,
+ "default-sort-order-id" : 0,
+ "sort-orders" : [ {
+ "order-id" : 0,
+ "fields" : [ ]
+ } ],
+ "properties" : {
+ "engine.hive.enabled" : "true",
+ "impala.events.catalogServiceId" : "a6b8da59a60e425a:ad0468614a61e340",
+ "external.table.purge" : "TRUE",
+ "impala.events.catalogVersion" : "4097",
+ "write.format.default" : "parquet",
+ "hive.metastore.table.owner" : "danielbecker",
+ "OBJCAPABILITIES" : "EXTREAD,EXTWRITE",
+ "storage_handler" : "org.apache.iceberg.mr.hive.HiveIcebergStorageHandler"
+ },
+ "current-snapshot-id" : 1880359224532128423,
+ "refs" : {
+ "main" : {
+ "snapshot-id" : 1880359224532128423,
+ "type" : "branch"
+ }
+ },
+ "snapshots" : [ {
+ "snapshot-id" : 1880359224532128423,
+ "timestamp-ms" : 1725612500115,
+ "summary" : {
+ "operation" : "append",
+ "added-data-files" : "1",
+ "added-records" : "2",
+ "added-files-size" : "604",
+ "changed-partition-count" : "1",
+ "total-records" : "2",
+ "total-files-size" : "604",
+ "total-data-files" : "1",
+ "total-delete-files" : "0",
+ "total-position-deletes" : "0",
+ "total-equality-deletes" : "0"
+ },
+ "manifest-list" : "/test-warehouse/iceberg_test/iceberg_with_puffin_stats/metadata/snap-1880359224532128423-1-11cd04ec-55ea-40aa-a89b-197c3c275e7a.avro",
+ "schema-id" : 0
+ } ],
+ "statistics" : [ {
+ "snapshot-id" : 1880359224532128423,
+ "statistics-path" : "/test-warehouse/iceberg_test/iceberg_with_puffin_stats/metadata/20240906_085606_00006_wsfgs-4d9242d5-bd79-4069-be8b-2cfced8e0647.stats",
+ "file-size-in-bytes" : 534,
+ "file-footer-size-in-bytes" : 440,
+ "blob-metadata" : [ {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 1880359224532128423,
+ "sequence-number" : 0,
+ "fields" : [ 1 ],
+ "properties" : {
+ "ndv" : "2"
+ }
+ }, {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 1880359224532128423,
+ "sequence-number" : 0,
+ "fields" : [ 2 ],
+ "properties" : {
+ "ndv" : "2"
+ }
+ } ]
+ } ],
+ "partition-statistics" : [ ],
+ "snapshot-log" : [ {
+ "timestamp-ms" : 1725612500115,
+ "snapshot-id" : 1880359224532128423
+ } ],
+ "metadata-log" : [ {
+ "timestamp-ms" : 1725612488531,
+ "metadata-file" : "/test-warehouse/iceberg_test/iceberg_with_puffin_stats/metadata/00000-c1fdc549-6895-4b22-987b-b1c229294749.metadata.json"
+ }, {
+ "timestamp-ms" : 1725612500133,
+ "metadata-file" : "/test-warehouse/iceberg_test/iceberg_with_puffin_stats/metadata/00001-1765f110-5212-4aee-9667-3d571582e15a.metadata.json"
+ } ]
+}
\ No newline at end of file
diff --git a/testdata/data/iceberg_test/iceberg_with_puffin_stats/metadata/version-hint.txt b/testdata/data/iceberg_test/iceberg_with_puffin_stats/metadata/version-hint.txt
new file mode 100644
index 000000000..00750edc0
--- /dev/null
+++ b/testdata/data/iceberg_test/iceberg_with_puffin_stats/metadata/version-hint.txt
@@ -0,0 +1 @@
+3
diff --git a/testdata/datasets/functional/functional_schema_template.sql b/testdata/datasets/functional/functional_schema_template.sql
index 641045ea7..a02807c50 100644
--- a/testdata/datasets/functional/functional_schema_template.sql
+++ b/testdata/datasets/functional/functional_schema_template.sql
@@ -3234,6 +3234,19 @@ hadoop fs -put -f ${IMPALA_HOME}/testdata/data/iceberg_test/iceberg_non_partitio
---- DATASET
functional
---- BASE_TABLE_NAME
+iceberg_with_puffin_stats
+---- CREATE
+CREATE EXTERNAL TABLE IF NOT EXISTS {db_name}{db_suffix}.{table_name}
+STORED AS ICEBERG
+LOCATION '/test-warehouse/iceberg_test/iceberg_with_puffin_stats'
+TBLPROPERTIES('write.format.default'='parquet', 'iceberg.catalog'='hadoop.tables');
+---- DEPENDENT_LOAD
+`hadoop fs -mkdir -p /test-warehouse/iceberg_test && \
+hadoop fs -put -f ${IMPALA_HOME}/testdata/data/iceberg_test/iceberg_with_puffin_stats /test-warehouse/iceberg_test/
+====
+---- DATASET
+functional
+---- BASE_TABLE_NAME
hadoop_catalog_test_external
---- CREATE
CREATE EXTERNAL TABLE IF NOT EXISTS {db_name}{db_suffix}.{table_name}
diff --git a/testdata/datasets/functional/schema_constraints.csv b/testdata/datasets/functional/schema_constraints.csv
index a3d279df3..3dc03d87a 100644
--- a/testdata/datasets/functional/schema_constraints.csv
+++ b/testdata/datasets/functional/schema_constraints.csv
@@ -70,6 +70,7 @@ table_name:complextypestbl_iceberg_orc, constraint:restrict_to, table_format:par
table_name:hadoop_catalog_test_external, constraint:restrict_to, table_format:parquet/none/none
table_name:iceberg_int_partitioned, constraint:restrict_to, table_format:parquet/none/none
table_name:iceberg_non_partitioned, constraint:restrict_to, table_format:parquet/none/none
+table_name:iceberg_with_puffin_stats, constraint:restrict_to, table_format:parquet/none/none
table_name:iceberg_partitioned, constraint:restrict_to, table_format:parquet/none/none
table_name:iceberg_partitioned_orc_external, constraint:restrict_to, table_format:parquet/none/none
table_name:iceberg_partition_transforms_zorder, constraint:restrict_to, table_format:parquet/none/none
diff --git a/testdata/ice_puffin/00001-2e1ade02-35ae-4a8f-a84f-784d1e0c0790.metadata.json b/testdata/ice_puffin/00001-2e1ade02-35ae-4a8f-a84f-784d1e0c0790.metadata.json
new file mode 100644
index 000000000..8a57c6d0d
--- /dev/null
+++ b/testdata/ice_puffin/00001-2e1ade02-35ae-4a8f-a84f-784d1e0c0790.metadata.json
@@ -0,0 +1,172 @@
+{
+ "format-version" : 1,
+ "table-uuid" : "6c803f57-98f6-4d14-8f7d-712d6a588325",
+ "location" : "hdfs://localhost:20500/test-warehouse/ice_puffin",
+ "last-updated-ms" : 1728483427091,
+ "last-column-id" : 10,
+ "schema" : {
+ "type" : "struct",
+ "schema-id" : 0,
+ "fields" : [ {
+ "id" : 1,
+ "name" : "int_col1",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 2,
+ "name" : "int_col2",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 3,
+ "name" : "bigint_col",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 4,
+ "name" : "float_col",
+ "required" : false,
+ "type" : "float"
+ }, {
+ "id" : 5,
+ "name" : "double_col",
+ "required" : false,
+ "type" : "double"
+ }, {
+ "id" : 6,
+ "name" : "decimal_col",
+ "required" : false,
+ "type" : "decimal(9, 0)"
+ }, {
+ "id" : 7,
+ "name" : "date_col",
+ "required" : false,
+ "type" : "date"
+ }, {
+ "id" : 8,
+ "name" : "string_col",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 9,
+ "name" : "timestamp_col",
+ "required" : false,
+ "type" : "timestamp"
+ }, {
+ "id" : 10,
+ "name" : "bool_col",
+ "required" : false,
+ "type" : "boolean"
+ } ]
+ },
+ "current-schema-id" : 0,
+ "schemas" : [ {
+ "type" : "struct",
+ "schema-id" : 0,
+ "fields" : [ {
+ "id" : 1,
+ "name" : "int_col1",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 2,
+ "name" : "int_col2",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 3,
+ "name" : "bigint_col",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 4,
+ "name" : "float_col",
+ "required" : false,
+ "type" : "float"
+ }, {
+ "id" : 5,
+ "name" : "double_col",
+ "required" : false,
+ "type" : "double"
+ }, {
+ "id" : 6,
+ "name" : "decimal_col",
+ "required" : false,
+ "type" : "decimal(9, 0)"
+ }, {
+ "id" : 7,
+ "name" : "date_col",
+ "required" : false,
+ "type" : "date"
+ }, {
+ "id" : 8,
+ "name" : "string_col",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 9,
+ "name" : "timestamp_col",
+ "required" : false,
+ "type" : "timestamp"
+ }, {
+ "id" : 10,
+ "name" : "bool_col",
+ "required" : false,
+ "type" : "boolean"
+ } ]
+ } ],
+ "partition-spec" : [ ],
+ "default-spec-id" : 0,
+ "partition-specs" : [ {
+ "spec-id" : 0,
+ "fields" : [ ]
+ } ],
+ "last-partition-id" : 999,
+ "default-sort-order-id" : 0,
+ "sort-orders" : [ {
+ "order-id" : 0,
+ "fields" : [ ]
+ } ],
+ "properties" : {
+ "engine.hive.enabled" : "true",
+ "impala.events.catalogServiceId" : "95d77e9a4a3d4e27:96a993b690a3fba6",
+ "external.table.purge" : "TRUE",
+ "impala.events.catalogVersion" : "2108",
+ "write.format.default" : "parquet",
+ "hive.metastore.table.owner" : "danielbecker",
+ "OBJCAPABILITIES" : "EXTREAD,EXTWRITE",
+ "storage_handler" : "org.apache.iceberg.mr.hive.HiveIcebergStorageHandler"
+ },
+ "current-snapshot-id" : 2532372403033748214,
+ "refs" : {
+ "main" : {
+ "snapshot-id" : 2532372403033748214,
+ "type" : "branch"
+ }
+ },
+ "snapshots" : [ {
+ "snapshot-id" : 2532372403033748214,
+ "timestamp-ms" : 1728483427081,
+ "summary" : {
+ "operation" : "delete",
+ "changed-partition-count" : "0",
+ "total-records" : "0",
+ "total-files-size" : "0",
+ "total-data-files" : "0",
+ "total-delete-files" : "0",
+ "total-position-deletes" : "0",
+ "total-equality-deletes" : "0"
+ },
+ "manifest-list" : "hdfs://localhost:20500/test-warehouse/ice_puffin/metadata/snap-2532372403033748214-1-c9f94c00-2920-4a39-8e7f-c7faf7e71a7d.avro",
+ "schema-id" : 0
+ } ],
+ "statistics" : [ ],
+ "snapshot-log" : [ {
+ "timestamp-ms" : 1728483427081,
+ "snapshot-id" : 2532372403033748214
+ } ],
+ "metadata-log" : [ {
+ "timestamp-ms" : 1728483321332,
+ "metadata-file" : "hdfs://localhost:20500/test-warehouse/ice_puffin/metadata/00000-0991de67-dc39-4226-af9c-8f347dbd57e5.metadata.json"
+ } ]
+}
\ No newline at end of file
diff --git a/testdata/ice_puffin/README b/testdata/ice_puffin/README
new file mode 100644
index 000000000..577274c42
--- /dev/null
+++ b/testdata/ice_puffin/README
@@ -0,0 +1,31 @@
+This directory contains metadata files matching an Iceberg table created with the
+following command
+
+ CREATE TABLE ice_puffin (
+ int_col1 INT,
+ int_col2 INT,
+ bigint_col BIGINT,
+ float_col FLOAT,
+ double_col DOUBLE,
+ decimal_col DECIMAL,
+ date_col DATE,
+ string_col STRING,
+ timestamp_col TIMESTAMP,
+ bool_col BOOLEAN) STORED BY ICEBERG;
+
+The metadata.json and the avro files are taken from a table created with the above command
+and then truncated (truncation is needed so that a snapshot exists).
+
+The metadata files in the "generated" subdirectory are created with the
+PuffinDataGenerator tool in "java/puffin-data-generator". The metadata.json file here is
+used by that tool as a template.
+To re-generate the metadata files, run the following from the Impala home directory:
+
+ (pushd java/puffin-data-generator; mvn -DskipTests install) && \
+ mvn -f "java/puffin-data-generator/pom.xml" exec:java \
+ -Dexec.mainClass="org.apache.impala.puffindatagenerator.PuffinDataGenerator";
+
+This will generate files in the "puffin_files" directory, which must previously be empty
+or non-existent. The generated files can then be copied into the "generated" subdirectory
+here.
+
diff --git a/testdata/ice_puffin/generated/all_files_corrupt.metadata.json b/testdata/ice_puffin/generated/all_files_corrupt.metadata.json
new file mode 100644
index 000000000..df2eed9b7
--- /dev/null
+++ b/testdata/ice_puffin/generated/all_files_corrupt.metadata.json
@@ -0,0 +1,204 @@
+{
+ "format-version" : 1,
+ "table-uuid" : "UUID_PLACEHOLDER",
+ "location" : "TABLE_LOCATION_PLACEHOLDER",
+ "last-updated-ms" : 1728483427091,
+ "last-column-id" : 10,
+ "schema" : {
+ "type" : "struct",
+ "schema-id" : 0,
+ "fields" : [ {
+ "id" : 1,
+ "name" : "int_col1",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 2,
+ "name" : "int_col2",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 3,
+ "name" : "bigint_col",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 4,
+ "name" : "float_col",
+ "required" : false,
+ "type" : "float"
+ }, {
+ "id" : 5,
+ "name" : "double_col",
+ "required" : false,
+ "type" : "double"
+ }, {
+ "id" : 6,
+ "name" : "decimal_col",
+ "required" : false,
+ "type" : "decimal(9, 0)"
+ }, {
+ "id" : 7,
+ "name" : "date_col",
+ "required" : false,
+ "type" : "date"
+ }, {
+ "id" : 8,
+ "name" : "string_col",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 9,
+ "name" : "timestamp_col",
+ "required" : false,
+ "type" : "timestamp"
+ }, {
+ "id" : 10,
+ "name" : "bool_col",
+ "required" : false,
+ "type" : "boolean"
+ } ]
+ },
+ "current-schema-id" : 0,
+ "schemas" : [ {
+ "type" : "struct",
+ "schema-id" : 0,
+ "fields" : [ {
+ "id" : 1,
+ "name" : "int_col1",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 2,
+ "name" : "int_col2",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 3,
+ "name" : "bigint_col",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 4,
+ "name" : "float_col",
+ "required" : false,
+ "type" : "float"
+ }, {
+ "id" : 5,
+ "name" : "double_col",
+ "required" : false,
+ "type" : "double"
+ }, {
+ "id" : 6,
+ "name" : "decimal_col",
+ "required" : false,
+ "type" : "decimal(9, 0)"
+ }, {
+ "id" : 7,
+ "name" : "date_col",
+ "required" : false,
+ "type" : "date"
+ }, {
+ "id" : 8,
+ "name" : "string_col",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 9,
+ "name" : "timestamp_col",
+ "required" : false,
+ "type" : "timestamp"
+ }, {
+ "id" : 10,
+ "name" : "bool_col",
+ "required" : false,
+ "type" : "boolean"
+ } ]
+ } ],
+ "partition-spec" : [ ],
+ "default-spec-id" : 0,
+ "partition-specs" : [ {
+ "spec-id" : 0,
+ "fields" : [ ]
+ } ],
+ "last-partition-id" : 999,
+ "default-sort-order-id" : 0,
+ "sort-orders" : [ {
+ "order-id" : 0,
+ "fields" : [ ]
+ } ],
+ "properties" : {
+ "engine.hive.enabled" : "true",
+ "impala.events.catalogServiceId" : "95d77e9a4a3d4e27:96a993b690a3fba6",
+ "external.table.purge" : "TRUE",
+ "impala.events.catalogVersion" : "2108",
+ "write.format.default" : "parquet",
+ "hive.metastore.table.owner" : "danielbecker",
+ "OBJCAPABILITIES" : "EXTREAD,EXTWRITE",
+ "storage_handler" : "org.apache.iceberg.mr.hive.HiveIcebergStorageHandler"
+ },
+ "current-snapshot-id" : 2532372403033748214,
+ "refs" : {
+ "main" : {
+ "snapshot-id" : 2532372403033748214,
+ "type" : "branch"
+ }
+ },
+ "snapshots" : [ {
+ "snapshot-id" : 2532372403033748214,
+ "timestamp-ms" : 1728483427081,
+ "summary" : {
+ "operation" : "delete",
+ "changed-partition-count" : "0",
+ "total-records" : "0",
+ "total-files-size" : "0",
+ "total-data-files" : "0",
+ "total-delete-files" : "0",
+ "total-position-deletes" : "0",
+ "total-equality-deletes" : "0"
+ },
+ "manifest-list" : "TABLE_LOCATION_PLACEHOLDER/metadata/snap-2532372403033748214-1-c9f94c00-2920-4a39-8e7f-c7faf7e71a7d.avro",
+ "schema-id" : 0
+ } ],
+ "statistics" : [ {
+ "snapshot-id" : 2532372403033748214,
+ "statistics-path" : "TABLE_LOCATION_PLACEHOLDER/metadata/corrupt_file1.stats",
+ "file-size-in-bytes" : 479,
+ "file-footer-size-in-bytes" : 401,
+ "blob-metadata" : [ {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 1 ]
+ }, {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 2 ]
+ } ]
+ }, {
+ "snapshot-id" : 2532372403033748214,
+ "statistics-path" : "TABLE_LOCATION_PLACEHOLDER/metadata/corrupt_file2.stats",
+ "file-size-in-bytes" : 519,
+ "file-footer-size-in-bytes" : 401,
+ "blob-metadata" : [ {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 3 ]
+ }, {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 4 ]
+ } ]
+ } ],
+ "snapshot-log" : [ {
+ "timestamp-ms" : 1728483427081,
+ "snapshot-id" : 2532372403033748214
+ } ],
+ "metadata-log" : [ {
+ "timestamp-ms" : 1728483321332,
+ "metadata-file" : "TABLE_LOCATION_PLACEHOLDER/metadata/00000-0991de67-dc39-4226-af9c-8f347dbd57e5.metadata.json"
+ } ]
+}
\ No newline at end of file
diff --git a/testdata/ice_puffin/generated/all_stats.stats b/testdata/ice_puffin/generated/all_stats.stats
new file mode 100644
index 000000000..41a75af8a
Binary files /dev/null and b/testdata/ice_puffin/generated/all_stats.stats differ
diff --git a/testdata/ice_puffin/generated/all_stats_in_1_file.metadata.json b/testdata/ice_puffin/generated/all_stats_in_1_file.metadata.json
new file mode 100644
index 000000000..bbb70eabf
--- /dev/null
+++ b/testdata/ice_puffin/generated/all_stats_in_1_file.metadata.json
@@ -0,0 +1,223 @@
+{
+ "format-version" : 1,
+ "table-uuid" : "UUID_PLACEHOLDER",
+ "location" : "TABLE_LOCATION_PLACEHOLDER",
+ "last-updated-ms" : 1728483427091,
+ "last-column-id" : 10,
+ "schema" : {
+ "type" : "struct",
+ "schema-id" : 0,
+ "fields" : [ {
+ "id" : 1,
+ "name" : "int_col1",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 2,
+ "name" : "int_col2",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 3,
+ "name" : "bigint_col",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 4,
+ "name" : "float_col",
+ "required" : false,
+ "type" : "float"
+ }, {
+ "id" : 5,
+ "name" : "double_col",
+ "required" : false,
+ "type" : "double"
+ }, {
+ "id" : 6,
+ "name" : "decimal_col",
+ "required" : false,
+ "type" : "decimal(9, 0)"
+ }, {
+ "id" : 7,
+ "name" : "date_col",
+ "required" : false,
+ "type" : "date"
+ }, {
+ "id" : 8,
+ "name" : "string_col",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 9,
+ "name" : "timestamp_col",
+ "required" : false,
+ "type" : "timestamp"
+ }, {
+ "id" : 10,
+ "name" : "bool_col",
+ "required" : false,
+ "type" : "boolean"
+ } ]
+ },
+ "current-schema-id" : 0,
+ "schemas" : [ {
+ "type" : "struct",
+ "schema-id" : 0,
+ "fields" : [ {
+ "id" : 1,
+ "name" : "int_col1",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 2,
+ "name" : "int_col2",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 3,
+ "name" : "bigint_col",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 4,
+ "name" : "float_col",
+ "required" : false,
+ "type" : "float"
+ }, {
+ "id" : 5,
+ "name" : "double_col",
+ "required" : false,
+ "type" : "double"
+ }, {
+ "id" : 6,
+ "name" : "decimal_col",
+ "required" : false,
+ "type" : "decimal(9, 0)"
+ }, {
+ "id" : 7,
+ "name" : "date_col",
+ "required" : false,
+ "type" : "date"
+ }, {
+ "id" : 8,
+ "name" : "string_col",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 9,
+ "name" : "timestamp_col",
+ "required" : false,
+ "type" : "timestamp"
+ }, {
+ "id" : 10,
+ "name" : "bool_col",
+ "required" : false,
+ "type" : "boolean"
+ } ]
+ } ],
+ "partition-spec" : [ ],
+ "default-spec-id" : 0,
+ "partition-specs" : [ {
+ "spec-id" : 0,
+ "fields" : [ ]
+ } ],
+ "last-partition-id" : 999,
+ "default-sort-order-id" : 0,
+ "sort-orders" : [ {
+ "order-id" : 0,
+ "fields" : [ ]
+ } ],
+ "properties" : {
+ "engine.hive.enabled" : "true",
+ "impala.events.catalogServiceId" : "95d77e9a4a3d4e27:96a993b690a3fba6",
+ "external.table.purge" : "TRUE",
+ "impala.events.catalogVersion" : "2108",
+ "write.format.default" : "parquet",
+ "hive.metastore.table.owner" : "danielbecker",
+ "OBJCAPABILITIES" : "EXTREAD,EXTWRITE",
+ "storage_handler" : "org.apache.iceberg.mr.hive.HiveIcebergStorageHandler"
+ },
+ "current-snapshot-id" : 2532372403033748214,
+ "refs" : {
+ "main" : {
+ "snapshot-id" : 2532372403033748214,
+ "type" : "branch"
+ }
+ },
+ "snapshots" : [ {
+ "snapshot-id" : 2532372403033748214,
+ "timestamp-ms" : 1728483427081,
+ "summary" : {
+ "operation" : "delete",
+ "changed-partition-count" : "0",
+ "total-records" : "0",
+ "total-files-size" : "0",
+ "total-data-files" : "0",
+ "total-delete-files" : "0",
+ "total-position-deletes" : "0",
+ "total-equality-deletes" : "0"
+ },
+ "manifest-list" : "TABLE_LOCATION_PLACEHOLDER/metadata/snap-2532372403033748214-1-c9f94c00-2920-4a39-8e7f-c7faf7e71a7d.avro",
+ "schema-id" : 0
+ } ],
+ "statistics" : [ {
+ "snapshot-id" : 2532372403033748214,
+ "statistics-path" : "TABLE_LOCATION_PLACEHOLDER/metadata/all_stats.stats",
+ "file-size-in-bytes" : 1769,
+ "file-footer-size-in-bytes" : 1269,
+ "blob-metadata" : [ {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 1 ]
+ }, {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 2 ]
+ }, {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 3 ]
+ }, {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 4 ]
+ }, {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 5 ]
+ }, {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 6 ]
+ }, {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 7 ]
+ }, {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 8 ]
+ }, {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 9 ]
+ } ]
+ } ],
+ "snapshot-log" : [ {
+ "timestamp-ms" : 1728483427081,
+ "snapshot-id" : 2532372403033748214
+ } ],
+ "metadata-log" : [ {
+ "timestamp-ms" : 1728483321332,
+ "metadata-file" : "TABLE_LOCATION_PLACEHOLDER/metadata/00000-0991de67-dc39-4226-af9c-8f347dbd57e5.metadata.json"
+ } ]
+}
\ No newline at end of file
diff --git a/testdata/ice_puffin/generated/corrupt_file.stats b/testdata/ice_puffin/generated/corrupt_file.stats
new file mode 100644
index 000000000..4071670b2
Binary files /dev/null and b/testdata/ice_puffin/generated/corrupt_file.stats differ
diff --git a/testdata/ice_puffin/generated/corrupt_file1.stats b/testdata/ice_puffin/generated/corrupt_file1.stats
new file mode 100644
index 000000000..5d42e70d9
Binary files /dev/null and b/testdata/ice_puffin/generated/corrupt_file1.stats differ
diff --git a/testdata/ice_puffin/generated/corrupt_file2.stats b/testdata/ice_puffin/generated/corrupt_file2.stats
new file mode 100644
index 000000000..70d577114
Binary files /dev/null and b/testdata/ice_puffin/generated/corrupt_file2.stats differ
diff --git a/testdata/ice_puffin/generated/current_snapshot_id.stats b/testdata/ice_puffin/generated/current_snapshot_id.stats
new file mode 100644
index 000000000..1eec1b137
Binary files /dev/null and b/testdata/ice_puffin/generated/current_snapshot_id.stats differ
diff --git a/testdata/ice_puffin/generated/duplicate_stats_in_1_file.metadata.json b/testdata/ice_puffin/generated/duplicate_stats_in_1_file.metadata.json
new file mode 100644
index 000000000..284019b4b
--- /dev/null
+++ b/testdata/ice_puffin/generated/duplicate_stats_in_1_file.metadata.json
@@ -0,0 +1,193 @@
+{
+ "format-version" : 1,
+ "table-uuid" : "UUID_PLACEHOLDER",
+ "location" : "TABLE_LOCATION_PLACEHOLDER",
+ "last-updated-ms" : 1728483427091,
+ "last-column-id" : 10,
+ "schema" : {
+ "type" : "struct",
+ "schema-id" : 0,
+ "fields" : [ {
+ "id" : 1,
+ "name" : "int_col1",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 2,
+ "name" : "int_col2",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 3,
+ "name" : "bigint_col",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 4,
+ "name" : "float_col",
+ "required" : false,
+ "type" : "float"
+ }, {
+ "id" : 5,
+ "name" : "double_col",
+ "required" : false,
+ "type" : "double"
+ }, {
+ "id" : 6,
+ "name" : "decimal_col",
+ "required" : false,
+ "type" : "decimal(9, 0)"
+ }, {
+ "id" : 7,
+ "name" : "date_col",
+ "required" : false,
+ "type" : "date"
+ }, {
+ "id" : 8,
+ "name" : "string_col",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 9,
+ "name" : "timestamp_col",
+ "required" : false,
+ "type" : "timestamp"
+ }, {
+ "id" : 10,
+ "name" : "bool_col",
+ "required" : false,
+ "type" : "boolean"
+ } ]
+ },
+ "current-schema-id" : 0,
+ "schemas" : [ {
+ "type" : "struct",
+ "schema-id" : 0,
+ "fields" : [ {
+ "id" : 1,
+ "name" : "int_col1",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 2,
+ "name" : "int_col2",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 3,
+ "name" : "bigint_col",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 4,
+ "name" : "float_col",
+ "required" : false,
+ "type" : "float"
+ }, {
+ "id" : 5,
+ "name" : "double_col",
+ "required" : false,
+ "type" : "double"
+ }, {
+ "id" : 6,
+ "name" : "decimal_col",
+ "required" : false,
+ "type" : "decimal(9, 0)"
+ }, {
+ "id" : 7,
+ "name" : "date_col",
+ "required" : false,
+ "type" : "date"
+ }, {
+ "id" : 8,
+ "name" : "string_col",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 9,
+ "name" : "timestamp_col",
+ "required" : false,
+ "type" : "timestamp"
+ }, {
+ "id" : 10,
+ "name" : "bool_col",
+ "required" : false,
+ "type" : "boolean"
+ } ]
+ } ],
+ "partition-spec" : [ ],
+ "default-spec-id" : 0,
+ "partition-specs" : [ {
+ "spec-id" : 0,
+ "fields" : [ ]
+ } ],
+ "last-partition-id" : 999,
+ "default-sort-order-id" : 0,
+ "sort-orders" : [ {
+ "order-id" : 0,
+ "fields" : [ ]
+ } ],
+ "properties" : {
+ "engine.hive.enabled" : "true",
+ "impala.events.catalogServiceId" : "95d77e9a4a3d4e27:96a993b690a3fba6",
+ "external.table.purge" : "TRUE",
+ "impala.events.catalogVersion" : "2108",
+ "write.format.default" : "parquet",
+ "hive.metastore.table.owner" : "danielbecker",
+ "OBJCAPABILITIES" : "EXTREAD,EXTWRITE",
+ "storage_handler" : "org.apache.iceberg.mr.hive.HiveIcebergStorageHandler"
+ },
+ "current-snapshot-id" : 2532372403033748214,
+ "refs" : {
+ "main" : {
+ "snapshot-id" : 2532372403033748214,
+ "type" : "branch"
+ }
+ },
+ "snapshots" : [ {
+ "snapshot-id" : 2532372403033748214,
+ "timestamp-ms" : 1728483427081,
+ "summary" : {
+ "operation" : "delete",
+ "changed-partition-count" : "0",
+ "total-records" : "0",
+ "total-files-size" : "0",
+ "total-data-files" : "0",
+ "total-delete-files" : "0",
+ "total-position-deletes" : "0",
+ "total-equality-deletes" : "0"
+ },
+ "manifest-list" : "TABLE_LOCATION_PLACEHOLDER/metadata/snap-2532372403033748214-1-c9f94c00-2920-4a39-8e7f-c7faf7e71a7d.avro",
+ "schema-id" : 0
+ } ],
+ "statistics" : [ {
+ "snapshot-id" : 2532372403033748214,
+ "statistics-path" : "TABLE_LOCATION_PLACEHOLDER/metadata/duplicate_stats_in_1_file.stats",
+ "file-size-in-bytes" : 690,
+ "file-footer-size-in-bytes" : 559,
+ "blob-metadata" : [ {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 1 ]
+ }, {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 2 ]
+ }, {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 1 ]
+ } ]
+ } ],
+ "snapshot-log" : [ {
+ "timestamp-ms" : 1728483427081,
+ "snapshot-id" : 2532372403033748214
+ } ],
+ "metadata-log" : [ {
+ "timestamp-ms" : 1728483321332,
+ "metadata-file" : "TABLE_LOCATION_PLACEHOLDER/metadata/00000-0991de67-dc39-4226-af9c-8f347dbd57e5.metadata.json"
+ } ]
+}
\ No newline at end of file
diff --git a/testdata/ice_puffin/generated/duplicate_stats_in_1_file.stats b/testdata/ice_puffin/generated/duplicate_stats_in_1_file.stats
new file mode 100644
index 000000000..09fd0414a
Binary files /dev/null and b/testdata/ice_puffin/generated/duplicate_stats_in_1_file.stats differ
diff --git a/testdata/ice_puffin/generated/duplicate_stats_in_2_files.metadata.json b/testdata/ice_puffin/generated/duplicate_stats_in_2_files.metadata.json
new file mode 100644
index 000000000..40b688992
--- /dev/null
+++ b/testdata/ice_puffin/generated/duplicate_stats_in_2_files.metadata.json
@@ -0,0 +1,204 @@
+{
+ "format-version" : 1,
+ "table-uuid" : "UUID_PLACEHOLDER",
+ "location" : "TABLE_LOCATION_PLACEHOLDER",
+ "last-updated-ms" : 1728483427091,
+ "last-column-id" : 10,
+ "schema" : {
+ "type" : "struct",
+ "schema-id" : 0,
+ "fields" : [ {
+ "id" : 1,
+ "name" : "int_col1",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 2,
+ "name" : "int_col2",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 3,
+ "name" : "bigint_col",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 4,
+ "name" : "float_col",
+ "required" : false,
+ "type" : "float"
+ }, {
+ "id" : 5,
+ "name" : "double_col",
+ "required" : false,
+ "type" : "double"
+ }, {
+ "id" : 6,
+ "name" : "decimal_col",
+ "required" : false,
+ "type" : "decimal(9, 0)"
+ }, {
+ "id" : 7,
+ "name" : "date_col",
+ "required" : false,
+ "type" : "date"
+ }, {
+ "id" : 8,
+ "name" : "string_col",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 9,
+ "name" : "timestamp_col",
+ "required" : false,
+ "type" : "timestamp"
+ }, {
+ "id" : 10,
+ "name" : "bool_col",
+ "required" : false,
+ "type" : "boolean"
+ } ]
+ },
+ "current-schema-id" : 0,
+ "schemas" : [ {
+ "type" : "struct",
+ "schema-id" : 0,
+ "fields" : [ {
+ "id" : 1,
+ "name" : "int_col1",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 2,
+ "name" : "int_col2",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 3,
+ "name" : "bigint_col",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 4,
+ "name" : "float_col",
+ "required" : false,
+ "type" : "float"
+ }, {
+ "id" : 5,
+ "name" : "double_col",
+ "required" : false,
+ "type" : "double"
+ }, {
+ "id" : 6,
+ "name" : "decimal_col",
+ "required" : false,
+ "type" : "decimal(9, 0)"
+ }, {
+ "id" : 7,
+ "name" : "date_col",
+ "required" : false,
+ "type" : "date"
+ }, {
+ "id" : 8,
+ "name" : "string_col",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 9,
+ "name" : "timestamp_col",
+ "required" : false,
+ "type" : "timestamp"
+ }, {
+ "id" : 10,
+ "name" : "bool_col",
+ "required" : false,
+ "type" : "boolean"
+ } ]
+ } ],
+ "partition-spec" : [ ],
+ "default-spec-id" : 0,
+ "partition-specs" : [ {
+ "spec-id" : 0,
+ "fields" : [ ]
+ } ],
+ "last-partition-id" : 999,
+ "default-sort-order-id" : 0,
+ "sort-orders" : [ {
+ "order-id" : 0,
+ "fields" : [ ]
+ } ],
+ "properties" : {
+ "engine.hive.enabled" : "true",
+ "impala.events.catalogServiceId" : "95d77e9a4a3d4e27:96a993b690a3fba6",
+ "external.table.purge" : "TRUE",
+ "impala.events.catalogVersion" : "2108",
+ "write.format.default" : "parquet",
+ "hive.metastore.table.owner" : "danielbecker",
+ "OBJCAPABILITIES" : "EXTREAD,EXTWRITE",
+ "storage_handler" : "org.apache.iceberg.mr.hive.HiveIcebergStorageHandler"
+ },
+ "current-snapshot-id" : 2532372403033748214,
+ "refs" : {
+ "main" : {
+ "snapshot-id" : 2532372403033748214,
+ "type" : "branch"
+ }
+ },
+ "snapshots" : [ {
+ "snapshot-id" : 2532372403033748214,
+ "timestamp-ms" : 1728483427081,
+ "summary" : {
+ "operation" : "delete",
+ "changed-partition-count" : "0",
+ "total-records" : "0",
+ "total-files-size" : "0",
+ "total-data-files" : "0",
+ "total-delete-files" : "0",
+ "total-position-deletes" : "0",
+ "total-equality-deletes" : "0"
+ },
+ "manifest-list" : "TABLE_LOCATION_PLACEHOLDER/metadata/snap-2532372403033748214-1-c9f94c00-2920-4a39-8e7f-c7faf7e71a7d.avro",
+ "schema-id" : 0
+ } ],
+ "statistics" : [ {
+ "snapshot-id" : 2532372403033748214,
+ "statistics-path" : "TABLE_LOCATION_PLACEHOLDER/metadata/duplicate_stats_in_2_files1.stats",
+ "file-size-in-bytes" : 479,
+ "file-footer-size-in-bytes" : 401,
+ "blob-metadata" : [ {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 1 ]
+ }, {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 2 ]
+ } ]
+ }, {
+ "snapshot-id" : 2532372403033748214,
+ "statistics-path" : "TABLE_LOCATION_PLACEHOLDER/metadata/duplicate_stats_in_2_files2.stats",
+ "file-size-in-bytes" : 447,
+ "file-footer-size-in-bytes" : 347,
+ "blob-metadata" : [ {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 1 ]
+ }, {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 3 ]
+ } ]
+ } ],
+ "snapshot-log" : [ {
+ "timestamp-ms" : 1728483427081,
+ "snapshot-id" : 2532372403033748214
+ } ],
+ "metadata-log" : [ {
+ "timestamp-ms" : 1728483321332,
+ "metadata-file" : "TABLE_LOCATION_PLACEHOLDER/metadata/00000-0991de67-dc39-4226-af9c-8f347dbd57e5.metadata.json"
+ } ]
+}
\ No newline at end of file
diff --git a/testdata/ice_puffin/generated/duplicate_stats_in_2_files1.stats b/testdata/ice_puffin/generated/duplicate_stats_in_2_files1.stats
new file mode 100644
index 000000000..1eec1b137
Binary files /dev/null and b/testdata/ice_puffin/generated/duplicate_stats_in_2_files1.stats differ
diff --git a/testdata/ice_puffin/generated/duplicate_stats_in_2_files2.stats b/testdata/ice_puffin/generated/duplicate_stats_in_2_files2.stats
new file mode 100644
index 000000000..ae158379b
Binary files /dev/null and b/testdata/ice_puffin/generated/duplicate_stats_in_2_files2.stats differ
diff --git a/testdata/ice_puffin/generated/existing_file.stats b/testdata/ice_puffin/generated/existing_file.stats
new file mode 100644
index 000000000..7cf05c1c2
Binary files /dev/null and b/testdata/ice_puffin/generated/existing_file.stats differ
diff --git a/testdata/ice_puffin/generated/file_contains_invalid_field_id.metadata.json b/testdata/ice_puffin/generated/file_contains_invalid_field_id.metadata.json
new file mode 100644
index 000000000..77d09cd64
--- /dev/null
+++ b/testdata/ice_puffin/generated/file_contains_invalid_field_id.metadata.json
@@ -0,0 +1,188 @@
+{
+ "format-version" : 1,
+ "table-uuid" : "UUID_PLACEHOLDER",
+ "location" : "TABLE_LOCATION_PLACEHOLDER",
+ "last-updated-ms" : 1728483427091,
+ "last-column-id" : 10,
+ "schema" : {
+ "type" : "struct",
+ "schema-id" : 0,
+ "fields" : [ {
+ "id" : 1,
+ "name" : "int_col1",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 2,
+ "name" : "int_col2",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 3,
+ "name" : "bigint_col",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 4,
+ "name" : "float_col",
+ "required" : false,
+ "type" : "float"
+ }, {
+ "id" : 5,
+ "name" : "double_col",
+ "required" : false,
+ "type" : "double"
+ }, {
+ "id" : 6,
+ "name" : "decimal_col",
+ "required" : false,
+ "type" : "decimal(9, 0)"
+ }, {
+ "id" : 7,
+ "name" : "date_col",
+ "required" : false,
+ "type" : "date"
+ }, {
+ "id" : 8,
+ "name" : "string_col",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 9,
+ "name" : "timestamp_col",
+ "required" : false,
+ "type" : "timestamp"
+ }, {
+ "id" : 10,
+ "name" : "bool_col",
+ "required" : false,
+ "type" : "boolean"
+ } ]
+ },
+ "current-schema-id" : 0,
+ "schemas" : [ {
+ "type" : "struct",
+ "schema-id" : 0,
+ "fields" : [ {
+ "id" : 1,
+ "name" : "int_col1",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 2,
+ "name" : "int_col2",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 3,
+ "name" : "bigint_col",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 4,
+ "name" : "float_col",
+ "required" : false,
+ "type" : "float"
+ }, {
+ "id" : 5,
+ "name" : "double_col",
+ "required" : false,
+ "type" : "double"
+ }, {
+ "id" : 6,
+ "name" : "decimal_col",
+ "required" : false,
+ "type" : "decimal(9, 0)"
+ }, {
+ "id" : 7,
+ "name" : "date_col",
+ "required" : false,
+ "type" : "date"
+ }, {
+ "id" : 8,
+ "name" : "string_col",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 9,
+ "name" : "timestamp_col",
+ "required" : false,
+ "type" : "timestamp"
+ }, {
+ "id" : 10,
+ "name" : "bool_col",
+ "required" : false,
+ "type" : "boolean"
+ } ]
+ } ],
+ "partition-spec" : [ ],
+ "default-spec-id" : 0,
+ "partition-specs" : [ {
+ "spec-id" : 0,
+ "fields" : [ ]
+ } ],
+ "last-partition-id" : 999,
+ "default-sort-order-id" : 0,
+ "sort-orders" : [ {
+ "order-id" : 0,
+ "fields" : [ ]
+ } ],
+ "properties" : {
+ "engine.hive.enabled" : "true",
+ "impala.events.catalogServiceId" : "95d77e9a4a3d4e27:96a993b690a3fba6",
+ "external.table.purge" : "TRUE",
+ "impala.events.catalogVersion" : "2108",
+ "write.format.default" : "parquet",
+ "hive.metastore.table.owner" : "danielbecker",
+ "OBJCAPABILITIES" : "EXTREAD,EXTWRITE",
+ "storage_handler" : "org.apache.iceberg.mr.hive.HiveIcebergStorageHandler"
+ },
+ "current-snapshot-id" : 2532372403033748214,
+ "refs" : {
+ "main" : {
+ "snapshot-id" : 2532372403033748214,
+ "type" : "branch"
+ }
+ },
+ "snapshots" : [ {
+ "snapshot-id" : 2532372403033748214,
+ "timestamp-ms" : 1728483427081,
+ "summary" : {
+ "operation" : "delete",
+ "changed-partition-count" : "0",
+ "total-records" : "0",
+ "total-files-size" : "0",
+ "total-data-files" : "0",
+ "total-delete-files" : "0",
+ "total-position-deletes" : "0",
+ "total-equality-deletes" : "0"
+ },
+ "manifest-list" : "TABLE_LOCATION_PLACEHOLDER/metadata/snap-2532372403033748214-1-c9f94c00-2920-4a39-8e7f-c7faf7e71a7d.avro",
+ "schema-id" : 0
+ } ],
+ "statistics" : [ {
+ "snapshot-id" : 2532372403033748214,
+ "statistics-path" : "TABLE_LOCATION_PLACEHOLDER/metadata/file_contains_invalid_field_id.stats",
+ "file-size-in-bytes" : 481,
+ "file-footer-size-in-bytes" : 403,
+ "blob-metadata" : [ {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 1 ]
+ }, {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 200 ]
+ } ]
+ } ],
+ "snapshot-log" : [ {
+ "timestamp-ms" : 1728483427081,
+ "snapshot-id" : 2532372403033748214
+ } ],
+ "metadata-log" : [ {
+ "timestamp-ms" : 1728483321332,
+ "metadata-file" : "TABLE_LOCATION_PLACEHOLDER/metadata/00000-0991de67-dc39-4226-af9c-8f347dbd57e5.metadata.json"
+ } ]
+}
\ No newline at end of file
diff --git a/testdata/ice_puffin/generated/file_contains_invalid_field_id.stats b/testdata/ice_puffin/generated/file_contains_invalid_field_id.stats
new file mode 100644
index 000000000..35fdc0d31
Binary files /dev/null and b/testdata/ice_puffin/generated/file_contains_invalid_field_id.stats differ
diff --git a/testdata/ice_puffin/generated/invalidAndCorruptSketches.metadata.json b/testdata/ice_puffin/generated/invalidAndCorruptSketches.metadata.json
new file mode 100644
index 000000000..67ed425e6
--- /dev/null
+++ b/testdata/ice_puffin/generated/invalidAndCorruptSketches.metadata.json
@@ -0,0 +1,203 @@
+{
+ "format-version" : 1,
+ "table-uuid" : "UUID_PLACEHOLDER",
+ "location" : "TABLE_LOCATION_PLACEHOLDER",
+ "last-updated-ms" : 1728483427091,
+ "last-column-id" : 10,
+ "schema" : {
+ "type" : "struct",
+ "schema-id" : 0,
+ "fields" : [ {
+ "id" : 1,
+ "name" : "int_col1",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 2,
+ "name" : "int_col2",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 3,
+ "name" : "bigint_col",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 4,
+ "name" : "float_col",
+ "required" : false,
+ "type" : "float"
+ }, {
+ "id" : 5,
+ "name" : "double_col",
+ "required" : false,
+ "type" : "double"
+ }, {
+ "id" : 6,
+ "name" : "decimal_col",
+ "required" : false,
+ "type" : "decimal(9, 0)"
+ }, {
+ "id" : 7,
+ "name" : "date_col",
+ "required" : false,
+ "type" : "date"
+ }, {
+ "id" : 8,
+ "name" : "string_col",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 9,
+ "name" : "timestamp_col",
+ "required" : false,
+ "type" : "timestamp"
+ }, {
+ "id" : 10,
+ "name" : "bool_col",
+ "required" : false,
+ "type" : "boolean"
+ } ]
+ },
+ "current-schema-id" : 0,
+ "schemas" : [ {
+ "type" : "struct",
+ "schema-id" : 0,
+ "fields" : [ {
+ "id" : 1,
+ "name" : "int_col1",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 2,
+ "name" : "int_col2",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 3,
+ "name" : "bigint_col",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 4,
+ "name" : "float_col",
+ "required" : false,
+ "type" : "float"
+ }, {
+ "id" : 5,
+ "name" : "double_col",
+ "required" : false,
+ "type" : "double"
+ }, {
+ "id" : 6,
+ "name" : "decimal_col",
+ "required" : false,
+ "type" : "decimal(9, 0)"
+ }, {
+ "id" : 7,
+ "name" : "date_col",
+ "required" : false,
+ "type" : "date"
+ }, {
+ "id" : 8,
+ "name" : "string_col",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 9,
+ "name" : "timestamp_col",
+ "required" : false,
+ "type" : "timestamp"
+ }, {
+ "id" : 10,
+ "name" : "bool_col",
+ "required" : false,
+ "type" : "boolean"
+ } ]
+ } ],
+ "partition-spec" : [ ],
+ "default-spec-id" : 0,
+ "partition-specs" : [ {
+ "spec-id" : 0,
+ "fields" : [ ]
+ } ],
+ "last-partition-id" : 999,
+ "default-sort-order-id" : 0,
+ "sort-orders" : [ {
+ "order-id" : 0,
+ "fields" : [ ]
+ } ],
+ "properties" : {
+ "engine.hive.enabled" : "true",
+ "impala.events.catalogServiceId" : "95d77e9a4a3d4e27:96a993b690a3fba6",
+ "external.table.purge" : "TRUE",
+ "impala.events.catalogVersion" : "2108",
+ "write.format.default" : "parquet",
+ "hive.metastore.table.owner" : "danielbecker",
+ "OBJCAPABILITIES" : "EXTREAD,EXTWRITE",
+ "storage_handler" : "org.apache.iceberg.mr.hive.HiveIcebergStorageHandler"
+ },
+ "current-snapshot-id" : 2532372403033748214,
+ "refs" : {
+ "main" : {
+ "snapshot-id" : 2532372403033748214,
+ "type" : "branch"
+ }
+ },
+ "snapshots" : [ {
+ "snapshot-id" : 2532372403033748214,
+ "timestamp-ms" : 1728483427081,
+ "summary" : {
+ "operation" : "delete",
+ "changed-partition-count" : "0",
+ "total-records" : "0",
+ "total-files-size" : "0",
+ "total-data-files" : "0",
+ "total-delete-files" : "0",
+ "total-position-deletes" : "0",
+ "total-equality-deletes" : "0"
+ },
+ "manifest-list" : "TABLE_LOCATION_PLACEHOLDER/metadata/snap-2532372403033748214-1-c9f94c00-2920-4a39-8e7f-c7faf7e71a7d.avro",
+ "schema-id" : 0
+ } ],
+ "statistics" : [ {
+ "snapshot-id" : 2532372403033748214,
+ "statistics-path" : "TABLE_LOCATION_PLACEHOLDER/metadata/invalidAndCorruptSketches.stats",
+ "file-size-in-bytes" : 1081,
+ "file-footer-size-in-bytes" : 866,
+ "blob-metadata" : [ {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 1 ]
+ }, {
+ "type" : "invalidSketchType",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 2 ]
+ }, {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 3 ]
+ }, {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 4 ]
+ }, {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 5 ]
+ } ]
+ } ],
+ "snapshot-log" : [ {
+ "timestamp-ms" : 1728483427081,
+ "snapshot-id" : 2532372403033748214
+ } ],
+ "metadata-log" : [ {
+ "timestamp-ms" : 1728483321332,
+ "metadata-file" : "TABLE_LOCATION_PLACEHOLDER/metadata/00000-0991de67-dc39-4226-af9c-8f347dbd57e5.metadata.json"
+ } ]
+}
\ No newline at end of file
diff --git a/testdata/ice_puffin/generated/invalidAndCorruptSketches.stats b/testdata/ice_puffin/generated/invalidAndCorruptSketches.stats
new file mode 100644
index 000000000..e6ccab349
Binary files /dev/null and b/testdata/ice_puffin/generated/invalidAndCorruptSketches.stats differ
diff --git a/testdata/ice_puffin/generated/missing_file.metadata.json b/testdata/ice_puffin/generated/missing_file.metadata.json
new file mode 100644
index 000000000..095b454c8
--- /dev/null
+++ b/testdata/ice_puffin/generated/missing_file.metadata.json
@@ -0,0 +1,204 @@
+{
+ "format-version" : 1,
+ "table-uuid" : "UUID_PLACEHOLDER",
+ "location" : "TABLE_LOCATION_PLACEHOLDER",
+ "last-updated-ms" : 1728483427091,
+ "last-column-id" : 10,
+ "schema" : {
+ "type" : "struct",
+ "schema-id" : 0,
+ "fields" : [ {
+ "id" : 1,
+ "name" : "int_col1",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 2,
+ "name" : "int_col2",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 3,
+ "name" : "bigint_col",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 4,
+ "name" : "float_col",
+ "required" : false,
+ "type" : "float"
+ }, {
+ "id" : 5,
+ "name" : "double_col",
+ "required" : false,
+ "type" : "double"
+ }, {
+ "id" : 6,
+ "name" : "decimal_col",
+ "required" : false,
+ "type" : "decimal(9, 0)"
+ }, {
+ "id" : 7,
+ "name" : "date_col",
+ "required" : false,
+ "type" : "date"
+ }, {
+ "id" : 8,
+ "name" : "string_col",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 9,
+ "name" : "timestamp_col",
+ "required" : false,
+ "type" : "timestamp"
+ }, {
+ "id" : 10,
+ "name" : "bool_col",
+ "required" : false,
+ "type" : "boolean"
+ } ]
+ },
+ "current-schema-id" : 0,
+ "schemas" : [ {
+ "type" : "struct",
+ "schema-id" : 0,
+ "fields" : [ {
+ "id" : 1,
+ "name" : "int_col1",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 2,
+ "name" : "int_col2",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 3,
+ "name" : "bigint_col",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 4,
+ "name" : "float_col",
+ "required" : false,
+ "type" : "float"
+ }, {
+ "id" : 5,
+ "name" : "double_col",
+ "required" : false,
+ "type" : "double"
+ }, {
+ "id" : 6,
+ "name" : "decimal_col",
+ "required" : false,
+ "type" : "decimal(9, 0)"
+ }, {
+ "id" : 7,
+ "name" : "date_col",
+ "required" : false,
+ "type" : "date"
+ }, {
+ "id" : 8,
+ "name" : "string_col",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 9,
+ "name" : "timestamp_col",
+ "required" : false,
+ "type" : "timestamp"
+ }, {
+ "id" : 10,
+ "name" : "bool_col",
+ "required" : false,
+ "type" : "boolean"
+ } ]
+ } ],
+ "partition-spec" : [ ],
+ "default-spec-id" : 0,
+ "partition-specs" : [ {
+ "spec-id" : 0,
+ "fields" : [ ]
+ } ],
+ "last-partition-id" : 999,
+ "default-sort-order-id" : 0,
+ "sort-orders" : [ {
+ "order-id" : 0,
+ "fields" : [ ]
+ } ],
+ "properties" : {
+ "engine.hive.enabled" : "true",
+ "impala.events.catalogServiceId" : "95d77e9a4a3d4e27:96a993b690a3fba6",
+ "external.table.purge" : "TRUE",
+ "impala.events.catalogVersion" : "2108",
+ "write.format.default" : "parquet",
+ "hive.metastore.table.owner" : "danielbecker",
+ "OBJCAPABILITIES" : "EXTREAD,EXTWRITE",
+ "storage_handler" : "org.apache.iceberg.mr.hive.HiveIcebergStorageHandler"
+ },
+ "current-snapshot-id" : 2532372403033748214,
+ "refs" : {
+ "main" : {
+ "snapshot-id" : 2532372403033748214,
+ "type" : "branch"
+ }
+ },
+ "snapshots" : [ {
+ "snapshot-id" : 2532372403033748214,
+ "timestamp-ms" : 1728483427081,
+ "summary" : {
+ "operation" : "delete",
+ "changed-partition-count" : "0",
+ "total-records" : "0",
+ "total-files-size" : "0",
+ "total-data-files" : "0",
+ "total-delete-files" : "0",
+ "total-position-deletes" : "0",
+ "total-equality-deletes" : "0"
+ },
+ "manifest-list" : "TABLE_LOCATION_PLACEHOLDER/metadata/snap-2532372403033748214-1-c9f94c00-2920-4a39-8e7f-c7faf7e71a7d.avro",
+ "schema-id" : 0
+ } ],
+ "statistics" : [ {
+ "snapshot-id" : 2532372403033748214,
+ "statistics-path" : "TABLE_LOCATION_PLACEHOLDER/metadata/missing_file.stats",
+ "file-size-in-bytes" : 340,
+ "file-footer-size-in-bytes" : 288,
+ "blob-metadata" : [ {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 1 ]
+ }, {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 2 ]
+ } ]
+ }, {
+ "snapshot-id" : 2532372403033748214,
+ "statistics-path" : "TABLE_LOCATION_PLACEHOLDER/metadata/existing_file.stats",
+ "file-size-in-bytes" : 519,
+ "file-footer-size-in-bytes" : 401,
+ "blob-metadata" : [ {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 3 ]
+ }, {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 4 ]
+ } ]
+ } ],
+ "snapshot-log" : [ {
+ "timestamp-ms" : 1728483427081,
+ "snapshot-id" : 2532372403033748214
+ } ],
+ "metadata-log" : [ {
+ "timestamp-ms" : 1728483321332,
+ "metadata-file" : "TABLE_LOCATION_PLACEHOLDER/metadata/00000-0991de67-dc39-4226-af9c-8f347dbd57e5.metadata.json"
+ } ]
+}
\ No newline at end of file
diff --git a/testdata/ice_puffin/generated/non_corrupt_file.stats b/testdata/ice_puffin/generated/non_corrupt_file.stats
new file mode 100644
index 000000000..c9196cbd0
Binary files /dev/null and b/testdata/ice_puffin/generated/non_corrupt_file.stats differ
diff --git a/testdata/ice_puffin/generated/not_all_blobs_current.metadata.json b/testdata/ice_puffin/generated/not_all_blobs_current.metadata.json
new file mode 100644
index 000000000..c6c9aea69
--- /dev/null
+++ b/testdata/ice_puffin/generated/not_all_blobs_current.metadata.json
@@ -0,0 +1,198 @@
+{
+ "format-version" : 1,
+ "table-uuid" : "UUID_PLACEHOLDER",
+ "location" : "TABLE_LOCATION_PLACEHOLDER",
+ "last-updated-ms" : 1728483427091,
+ "last-column-id" : 10,
+ "schema" : {
+ "type" : "struct",
+ "schema-id" : 0,
+ "fields" : [ {
+ "id" : 1,
+ "name" : "int_col1",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 2,
+ "name" : "int_col2",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 3,
+ "name" : "bigint_col",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 4,
+ "name" : "float_col",
+ "required" : false,
+ "type" : "float"
+ }, {
+ "id" : 5,
+ "name" : "double_col",
+ "required" : false,
+ "type" : "double"
+ }, {
+ "id" : 6,
+ "name" : "decimal_col",
+ "required" : false,
+ "type" : "decimal(9, 0)"
+ }, {
+ "id" : 7,
+ "name" : "date_col",
+ "required" : false,
+ "type" : "date"
+ }, {
+ "id" : 8,
+ "name" : "string_col",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 9,
+ "name" : "timestamp_col",
+ "required" : false,
+ "type" : "timestamp"
+ }, {
+ "id" : 10,
+ "name" : "bool_col",
+ "required" : false,
+ "type" : "boolean"
+ } ]
+ },
+ "current-schema-id" : 0,
+ "schemas" : [ {
+ "type" : "struct",
+ "schema-id" : 0,
+ "fields" : [ {
+ "id" : 1,
+ "name" : "int_col1",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 2,
+ "name" : "int_col2",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 3,
+ "name" : "bigint_col",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 4,
+ "name" : "float_col",
+ "required" : false,
+ "type" : "float"
+ }, {
+ "id" : 5,
+ "name" : "double_col",
+ "required" : false,
+ "type" : "double"
+ }, {
+ "id" : 6,
+ "name" : "decimal_col",
+ "required" : false,
+ "type" : "decimal(9, 0)"
+ }, {
+ "id" : 7,
+ "name" : "date_col",
+ "required" : false,
+ "type" : "date"
+ }, {
+ "id" : 8,
+ "name" : "string_col",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 9,
+ "name" : "timestamp_col",
+ "required" : false,
+ "type" : "timestamp"
+ }, {
+ "id" : 10,
+ "name" : "bool_col",
+ "required" : false,
+ "type" : "boolean"
+ } ]
+ } ],
+ "partition-spec" : [ ],
+ "default-spec-id" : 0,
+ "partition-specs" : [ {
+ "spec-id" : 0,
+ "fields" : [ ]
+ } ],
+ "last-partition-id" : 999,
+ "default-sort-order-id" : 0,
+ "sort-orders" : [ {
+ "order-id" : 0,
+ "fields" : [ ]
+ } ],
+ "properties" : {
+ "engine.hive.enabled" : "true",
+ "impala.events.catalogServiceId" : "95d77e9a4a3d4e27:96a993b690a3fba6",
+ "external.table.purge" : "TRUE",
+ "impala.events.catalogVersion" : "2108",
+ "write.format.default" : "parquet",
+ "hive.metastore.table.owner" : "danielbecker",
+ "OBJCAPABILITIES" : "EXTREAD,EXTWRITE",
+ "storage_handler" : "org.apache.iceberg.mr.hive.HiveIcebergStorageHandler"
+ },
+ "current-snapshot-id" : 2532372403033748214,
+ "refs" : {
+ "main" : {
+ "snapshot-id" : 2532372403033748214,
+ "type" : "branch"
+ }
+ },
+ "snapshots" : [ {
+ "snapshot-id" : 2532372403033748214,
+ "timestamp-ms" : 1728483427081,
+ "summary" : {
+ "operation" : "delete",
+ "changed-partition-count" : "0",
+ "total-records" : "0",
+ "total-files-size" : "0",
+ "total-data-files" : "0",
+ "total-delete-files" : "0",
+ "total-position-deletes" : "0",
+ "total-equality-deletes" : "0"
+ },
+ "manifest-list" : "TABLE_LOCATION_PLACEHOLDER/metadata/snap-2532372403033748214-1-c9f94c00-2920-4a39-8e7f-c7faf7e71a7d.avro",
+ "schema-id" : 0
+ } ],
+ "statistics" : [ {
+ "snapshot-id" : 2532372403033748214,
+ "statistics-path" : "TABLE_LOCATION_PLACEHOLDER/metadata/not_all_blobs_current.stats",
+ "file-size-in-bytes" : 910,
+ "file-footer-size-in-bytes" : 718,
+ "blob-metadata" : [ {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 1 ]
+ }, {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 2 ]
+ }, {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748213,
+ "sequence-number" : 0,
+ "fields" : [ 3 ]
+ }, {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 4 ]
+ } ]
+ } ],
+ "snapshot-log" : [ {
+ "timestamp-ms" : 1728483427081,
+ "snapshot-id" : 2532372403033748214
+ } ],
+ "metadata-log" : [ {
+ "timestamp-ms" : 1728483321332,
+ "metadata-file" : "TABLE_LOCATION_PLACEHOLDER/metadata/00000-0991de67-dc39-4226-af9c-8f347dbd57e5.metadata.json"
+ } ]
+}
\ No newline at end of file
diff --git a/testdata/ice_puffin/generated/not_all_blobs_current.stats b/testdata/ice_puffin/generated/not_all_blobs_current.stats
new file mode 100644
index 000000000..dae30b80c
Binary files /dev/null and b/testdata/ice_puffin/generated/not_all_blobs_current.stats differ
diff --git a/testdata/ice_puffin/generated/not_current_snapshot_id.stats b/testdata/ice_puffin/generated/not_current_snapshot_id.stats
new file mode 100644
index 000000000..bda68fd0a
Binary files /dev/null and b/testdata/ice_puffin/generated/not_current_snapshot_id.stats differ
diff --git a/testdata/ice_puffin/generated/one_file_corrupt_one_not.metadata.json b/testdata/ice_puffin/generated/one_file_corrupt_one_not.metadata.json
new file mode 100644
index 000000000..f87e73458
--- /dev/null
+++ b/testdata/ice_puffin/generated/one_file_corrupt_one_not.metadata.json
@@ -0,0 +1,204 @@
+{
+ "format-version" : 1,
+ "table-uuid" : "UUID_PLACEHOLDER",
+ "location" : "TABLE_LOCATION_PLACEHOLDER",
+ "last-updated-ms" : 1728483427091,
+ "last-column-id" : 10,
+ "schema" : {
+ "type" : "struct",
+ "schema-id" : 0,
+ "fields" : [ {
+ "id" : 1,
+ "name" : "int_col1",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 2,
+ "name" : "int_col2",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 3,
+ "name" : "bigint_col",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 4,
+ "name" : "float_col",
+ "required" : false,
+ "type" : "float"
+ }, {
+ "id" : 5,
+ "name" : "double_col",
+ "required" : false,
+ "type" : "double"
+ }, {
+ "id" : 6,
+ "name" : "decimal_col",
+ "required" : false,
+ "type" : "decimal(9, 0)"
+ }, {
+ "id" : 7,
+ "name" : "date_col",
+ "required" : false,
+ "type" : "date"
+ }, {
+ "id" : 8,
+ "name" : "string_col",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 9,
+ "name" : "timestamp_col",
+ "required" : false,
+ "type" : "timestamp"
+ }, {
+ "id" : 10,
+ "name" : "bool_col",
+ "required" : false,
+ "type" : "boolean"
+ } ]
+ },
+ "current-schema-id" : 0,
+ "schemas" : [ {
+ "type" : "struct",
+ "schema-id" : 0,
+ "fields" : [ {
+ "id" : 1,
+ "name" : "int_col1",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 2,
+ "name" : "int_col2",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 3,
+ "name" : "bigint_col",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 4,
+ "name" : "float_col",
+ "required" : false,
+ "type" : "float"
+ }, {
+ "id" : 5,
+ "name" : "double_col",
+ "required" : false,
+ "type" : "double"
+ }, {
+ "id" : 6,
+ "name" : "decimal_col",
+ "required" : false,
+ "type" : "decimal(9, 0)"
+ }, {
+ "id" : 7,
+ "name" : "date_col",
+ "required" : false,
+ "type" : "date"
+ }, {
+ "id" : 8,
+ "name" : "string_col",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 9,
+ "name" : "timestamp_col",
+ "required" : false,
+ "type" : "timestamp"
+ }, {
+ "id" : 10,
+ "name" : "bool_col",
+ "required" : false,
+ "type" : "boolean"
+ } ]
+ } ],
+ "partition-spec" : [ ],
+ "default-spec-id" : 0,
+ "partition-specs" : [ {
+ "spec-id" : 0,
+ "fields" : [ ]
+ } ],
+ "last-partition-id" : 999,
+ "default-sort-order-id" : 0,
+ "sort-orders" : [ {
+ "order-id" : 0,
+ "fields" : [ ]
+ } ],
+ "properties" : {
+ "engine.hive.enabled" : "true",
+ "impala.events.catalogServiceId" : "95d77e9a4a3d4e27:96a993b690a3fba6",
+ "external.table.purge" : "TRUE",
+ "impala.events.catalogVersion" : "2108",
+ "write.format.default" : "parquet",
+ "hive.metastore.table.owner" : "danielbecker",
+ "OBJCAPABILITIES" : "EXTREAD,EXTWRITE",
+ "storage_handler" : "org.apache.iceberg.mr.hive.HiveIcebergStorageHandler"
+ },
+ "current-snapshot-id" : 2532372403033748214,
+ "refs" : {
+ "main" : {
+ "snapshot-id" : 2532372403033748214,
+ "type" : "branch"
+ }
+ },
+ "snapshots" : [ {
+ "snapshot-id" : 2532372403033748214,
+ "timestamp-ms" : 1728483427081,
+ "summary" : {
+ "operation" : "delete",
+ "changed-partition-count" : "0",
+ "total-records" : "0",
+ "total-files-size" : "0",
+ "total-data-files" : "0",
+ "total-delete-files" : "0",
+ "total-position-deletes" : "0",
+ "total-equality-deletes" : "0"
+ },
+ "manifest-list" : "TABLE_LOCATION_PLACEHOLDER/metadata/snap-2532372403033748214-1-c9f94c00-2920-4a39-8e7f-c7faf7e71a7d.avro",
+ "schema-id" : 0
+ } ],
+ "statistics" : [ {
+ "snapshot-id" : 2532372403033748214,
+ "statistics-path" : "TABLE_LOCATION_PLACEHOLDER/metadata/corrupt_file.stats",
+ "file-size-in-bytes" : 399,
+ "file-footer-size-in-bytes" : 347,
+ "blob-metadata" : [ {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 1 ]
+ }, {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 2 ]
+ } ]
+ }, {
+ "snapshot-id" : 2532372403033748214,
+ "statistics-path" : "TABLE_LOCATION_PLACEHOLDER/metadata/non_corrupt_file.stats",
+ "file-size-in-bytes" : 439,
+ "file-footer-size-in-bytes" : 347,
+ "blob-metadata" : [ {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 3 ]
+ }, {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 4 ]
+ } ]
+ } ],
+ "snapshot-log" : [ {
+ "timestamp-ms" : 1728483427081,
+ "snapshot-id" : 2532372403033748214
+ } ],
+ "metadata-log" : [ {
+ "timestamp-ms" : 1728483321332,
+ "metadata-file" : "TABLE_LOCATION_PLACEHOLDER/metadata/00000-0991de67-dc39-4226-af9c-8f347dbd57e5.metadata.json"
+ } ]
+}
\ No newline at end of file
diff --git a/testdata/ice_puffin/generated/one_file_current_one_not.metadata.json b/testdata/ice_puffin/generated/one_file_current_one_not.metadata.json
new file mode 100644
index 000000000..367b975cf
--- /dev/null
+++ b/testdata/ice_puffin/generated/one_file_current_one_not.metadata.json
@@ -0,0 +1,204 @@
+{
+ "format-version" : 1,
+ "table-uuid" : "UUID_PLACEHOLDER",
+ "location" : "TABLE_LOCATION_PLACEHOLDER",
+ "last-updated-ms" : 1728483427091,
+ "last-column-id" : 10,
+ "schema" : {
+ "type" : "struct",
+ "schema-id" : 0,
+ "fields" : [ {
+ "id" : 1,
+ "name" : "int_col1",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 2,
+ "name" : "int_col2",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 3,
+ "name" : "bigint_col",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 4,
+ "name" : "float_col",
+ "required" : false,
+ "type" : "float"
+ }, {
+ "id" : 5,
+ "name" : "double_col",
+ "required" : false,
+ "type" : "double"
+ }, {
+ "id" : 6,
+ "name" : "decimal_col",
+ "required" : false,
+ "type" : "decimal(9, 0)"
+ }, {
+ "id" : 7,
+ "name" : "date_col",
+ "required" : false,
+ "type" : "date"
+ }, {
+ "id" : 8,
+ "name" : "string_col",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 9,
+ "name" : "timestamp_col",
+ "required" : false,
+ "type" : "timestamp"
+ }, {
+ "id" : 10,
+ "name" : "bool_col",
+ "required" : false,
+ "type" : "boolean"
+ } ]
+ },
+ "current-schema-id" : 0,
+ "schemas" : [ {
+ "type" : "struct",
+ "schema-id" : 0,
+ "fields" : [ {
+ "id" : 1,
+ "name" : "int_col1",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 2,
+ "name" : "int_col2",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 3,
+ "name" : "bigint_col",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 4,
+ "name" : "float_col",
+ "required" : false,
+ "type" : "float"
+ }, {
+ "id" : 5,
+ "name" : "double_col",
+ "required" : false,
+ "type" : "double"
+ }, {
+ "id" : 6,
+ "name" : "decimal_col",
+ "required" : false,
+ "type" : "decimal(9, 0)"
+ }, {
+ "id" : 7,
+ "name" : "date_col",
+ "required" : false,
+ "type" : "date"
+ }, {
+ "id" : 8,
+ "name" : "string_col",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 9,
+ "name" : "timestamp_col",
+ "required" : false,
+ "type" : "timestamp"
+ }, {
+ "id" : 10,
+ "name" : "bool_col",
+ "required" : false,
+ "type" : "boolean"
+ } ]
+ } ],
+ "partition-spec" : [ ],
+ "default-spec-id" : 0,
+ "partition-specs" : [ {
+ "spec-id" : 0,
+ "fields" : [ ]
+ } ],
+ "last-partition-id" : 999,
+ "default-sort-order-id" : 0,
+ "sort-orders" : [ {
+ "order-id" : 0,
+ "fields" : [ ]
+ } ],
+ "properties" : {
+ "engine.hive.enabled" : "true",
+ "impala.events.catalogServiceId" : "95d77e9a4a3d4e27:96a993b690a3fba6",
+ "external.table.purge" : "TRUE",
+ "impala.events.catalogVersion" : "2108",
+ "write.format.default" : "parquet",
+ "hive.metastore.table.owner" : "danielbecker",
+ "OBJCAPABILITIES" : "EXTREAD,EXTWRITE",
+ "storage_handler" : "org.apache.iceberg.mr.hive.HiveIcebergStorageHandler"
+ },
+ "current-snapshot-id" : 2532372403033748214,
+ "refs" : {
+ "main" : {
+ "snapshot-id" : 2532372403033748214,
+ "type" : "branch"
+ }
+ },
+ "snapshots" : [ {
+ "snapshot-id" : 2532372403033748214,
+ "timestamp-ms" : 1728483427081,
+ "summary" : {
+ "operation" : "delete",
+ "changed-partition-count" : "0",
+ "total-records" : "0",
+ "total-files-size" : "0",
+ "total-data-files" : "0",
+ "total-delete-files" : "0",
+ "total-position-deletes" : "0",
+ "total-equality-deletes" : "0"
+ },
+ "manifest-list" : "TABLE_LOCATION_PLACEHOLDER/metadata/snap-2532372403033748214-1-c9f94c00-2920-4a39-8e7f-c7faf7e71a7d.avro",
+ "schema-id" : 0
+ } ],
+ "statistics" : [ {
+ "snapshot-id" : 2532372403033748214,
+ "statistics-path" : "TABLE_LOCATION_PLACEHOLDER/metadata/current_snapshot_id.stats",
+ "file-size-in-bytes" : 479,
+ "file-footer-size-in-bytes" : 401,
+ "blob-metadata" : [ {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 1 ]
+ }, {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 2 ]
+ } ]
+ }, {
+ "snapshot-id" : 2532372403033748213,
+ "statistics-path" : "TABLE_LOCATION_PLACEHOLDER/metadata/not_current_snapshot_id.stats",
+ "file-size-in-bytes" : 519,
+ "file-footer-size-in-bytes" : 401,
+ "blob-metadata" : [ {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748213,
+ "sequence-number" : 0,
+ "fields" : [ 3 ]
+ }, {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 4 ]
+ } ]
+ } ],
+ "snapshot-log" : [ {
+ "timestamp-ms" : 1728483427081,
+ "snapshot-id" : 2532372403033748214
+ } ],
+ "metadata-log" : [ {
+ "timestamp-ms" : 1728483321332,
+ "metadata-file" : "TABLE_LOCATION_PLACEHOLDER/metadata/00000-0991de67-dc39-4226-af9c-8f347dbd57e5.metadata.json"
+ } ]
+}
\ No newline at end of file
diff --git a/testdata/ice_puffin/generated/stats_divided.metadata.json b/testdata/ice_puffin/generated/stats_divided.metadata.json
new file mode 100644
index 000000000..5f1ee2107
--- /dev/null
+++ b/testdata/ice_puffin/generated/stats_divided.metadata.json
@@ -0,0 +1,229 @@
+{
+ "format-version" : 1,
+ "table-uuid" : "UUID_PLACEHOLDER",
+ "location" : "TABLE_LOCATION_PLACEHOLDER",
+ "last-updated-ms" : 1728483427091,
+ "last-column-id" : 10,
+ "schema" : {
+ "type" : "struct",
+ "schema-id" : 0,
+ "fields" : [ {
+ "id" : 1,
+ "name" : "int_col1",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 2,
+ "name" : "int_col2",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 3,
+ "name" : "bigint_col",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 4,
+ "name" : "float_col",
+ "required" : false,
+ "type" : "float"
+ }, {
+ "id" : 5,
+ "name" : "double_col",
+ "required" : false,
+ "type" : "double"
+ }, {
+ "id" : 6,
+ "name" : "decimal_col",
+ "required" : false,
+ "type" : "decimal(9, 0)"
+ }, {
+ "id" : 7,
+ "name" : "date_col",
+ "required" : false,
+ "type" : "date"
+ }, {
+ "id" : 8,
+ "name" : "string_col",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 9,
+ "name" : "timestamp_col",
+ "required" : false,
+ "type" : "timestamp"
+ }, {
+ "id" : 10,
+ "name" : "bool_col",
+ "required" : false,
+ "type" : "boolean"
+ } ]
+ },
+ "current-schema-id" : 0,
+ "schemas" : [ {
+ "type" : "struct",
+ "schema-id" : 0,
+ "fields" : [ {
+ "id" : 1,
+ "name" : "int_col1",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 2,
+ "name" : "int_col2",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 3,
+ "name" : "bigint_col",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 4,
+ "name" : "float_col",
+ "required" : false,
+ "type" : "float"
+ }, {
+ "id" : 5,
+ "name" : "double_col",
+ "required" : false,
+ "type" : "double"
+ }, {
+ "id" : 6,
+ "name" : "decimal_col",
+ "required" : false,
+ "type" : "decimal(9, 0)"
+ }, {
+ "id" : 7,
+ "name" : "date_col",
+ "required" : false,
+ "type" : "date"
+ }, {
+ "id" : 8,
+ "name" : "string_col",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 9,
+ "name" : "timestamp_col",
+ "required" : false,
+ "type" : "timestamp"
+ }, {
+ "id" : 10,
+ "name" : "bool_col",
+ "required" : false,
+ "type" : "boolean"
+ } ]
+ } ],
+ "partition-spec" : [ ],
+ "default-spec-id" : 0,
+ "partition-specs" : [ {
+ "spec-id" : 0,
+ "fields" : [ ]
+ } ],
+ "last-partition-id" : 999,
+ "default-sort-order-id" : 0,
+ "sort-orders" : [ {
+ "order-id" : 0,
+ "fields" : [ ]
+ } ],
+ "properties" : {
+ "engine.hive.enabled" : "true",
+ "impala.events.catalogServiceId" : "95d77e9a4a3d4e27:96a993b690a3fba6",
+ "external.table.purge" : "TRUE",
+ "impala.events.catalogVersion" : "2108",
+ "write.format.default" : "parquet",
+ "hive.metastore.table.owner" : "danielbecker",
+ "OBJCAPABILITIES" : "EXTREAD,EXTWRITE",
+ "storage_handler" : "org.apache.iceberg.mr.hive.HiveIcebergStorageHandler"
+ },
+ "current-snapshot-id" : 2532372403033748214,
+ "refs" : {
+ "main" : {
+ "snapshot-id" : 2532372403033748214,
+ "type" : "branch"
+ }
+ },
+ "snapshots" : [ {
+ "snapshot-id" : 2532372403033748214,
+ "timestamp-ms" : 1728483427081,
+ "summary" : {
+ "operation" : "delete",
+ "changed-partition-count" : "0",
+ "total-records" : "0",
+ "total-files-size" : "0",
+ "total-data-files" : "0",
+ "total-delete-files" : "0",
+ "total-position-deletes" : "0",
+ "total-equality-deletes" : "0"
+ },
+ "manifest-list" : "TABLE_LOCATION_PLACEHOLDER/metadata/snap-2532372403033748214-1-c9f94c00-2920-4a39-8e7f-c7faf7e71a7d.avro",
+ "schema-id" : 0
+ } ],
+ "statistics" : [ {
+ "snapshot-id" : 2532372403033748214,
+ "statistics-path" : "TABLE_LOCATION_PLACEHOLDER/metadata/stats_divided1.stats",
+ "file-size-in-bytes" : 749,
+ "file-footer-size-in-bytes" : 609,
+ "blob-metadata" : [ {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 1 ]
+ }, {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 2 ]
+ }, {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 3 ]
+ }, {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 4 ]
+ } ]
+ }, {
+ "snapshot-id" : 2532372403033748214,
+ "statistics-path" : "TABLE_LOCATION_PLACEHOLDER/metadata/stats_divided2.stats",
+ "file-size-in-bytes" : 1308,
+ "file-footer-size-in-bytes" : 879,
+ "blob-metadata" : [ {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 5 ]
+ }, {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 6 ]
+ }, {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 7 ]
+ }, {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 8 ]
+ }, {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 9 ]
+ } ]
+ } ],
+ "snapshot-log" : [ {
+ "timestamp-ms" : 1728483427081,
+ "snapshot-id" : 2532372403033748214
+ } ],
+ "metadata-log" : [ {
+ "timestamp-ms" : 1728483321332,
+ "metadata-file" : "TABLE_LOCATION_PLACEHOLDER/metadata/00000-0991de67-dc39-4226-af9c-8f347dbd57e5.metadata.json"
+ } ]
+}
\ No newline at end of file
diff --git a/testdata/ice_puffin/generated/stats_divided1.stats b/testdata/ice_puffin/generated/stats_divided1.stats
new file mode 100644
index 000000000..33aa7b550
Binary files /dev/null and b/testdata/ice_puffin/generated/stats_divided1.stats differ
diff --git a/testdata/ice_puffin/generated/stats_divided2.stats b/testdata/ice_puffin/generated/stats_divided2.stats
new file mode 100644
index 000000000..c70b71e37
Binary files /dev/null and b/testdata/ice_puffin/generated/stats_divided2.stats differ
diff --git a/testdata/ice_puffin/generated/stats_for_unsupported_type.metadata.json b/testdata/ice_puffin/generated/stats_for_unsupported_type.metadata.json
new file mode 100644
index 000000000..468534a0f
--- /dev/null
+++ b/testdata/ice_puffin/generated/stats_for_unsupported_type.metadata.json
@@ -0,0 +1,188 @@
+{
+ "format-version" : 1,
+ "table-uuid" : "UUID_PLACEHOLDER",
+ "location" : "TABLE_LOCATION_PLACEHOLDER",
+ "last-updated-ms" : 1728483427091,
+ "last-column-id" : 10,
+ "schema" : {
+ "type" : "struct",
+ "schema-id" : 0,
+ "fields" : [ {
+ "id" : 1,
+ "name" : "int_col1",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 2,
+ "name" : "int_col2",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 3,
+ "name" : "bigint_col",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 4,
+ "name" : "float_col",
+ "required" : false,
+ "type" : "float"
+ }, {
+ "id" : 5,
+ "name" : "double_col",
+ "required" : false,
+ "type" : "double"
+ }, {
+ "id" : 6,
+ "name" : "decimal_col",
+ "required" : false,
+ "type" : "decimal(9, 0)"
+ }, {
+ "id" : 7,
+ "name" : "date_col",
+ "required" : false,
+ "type" : "date"
+ }, {
+ "id" : 8,
+ "name" : "string_col",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 9,
+ "name" : "timestamp_col",
+ "required" : false,
+ "type" : "timestamp"
+ }, {
+ "id" : 10,
+ "name" : "bool_col",
+ "required" : false,
+ "type" : "boolean"
+ } ]
+ },
+ "current-schema-id" : 0,
+ "schemas" : [ {
+ "type" : "struct",
+ "schema-id" : 0,
+ "fields" : [ {
+ "id" : 1,
+ "name" : "int_col1",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 2,
+ "name" : "int_col2",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 3,
+ "name" : "bigint_col",
+ "required" : false,
+ "type" : "long"
+ }, {
+ "id" : 4,
+ "name" : "float_col",
+ "required" : false,
+ "type" : "float"
+ }, {
+ "id" : 5,
+ "name" : "double_col",
+ "required" : false,
+ "type" : "double"
+ }, {
+ "id" : 6,
+ "name" : "decimal_col",
+ "required" : false,
+ "type" : "decimal(9, 0)"
+ }, {
+ "id" : 7,
+ "name" : "date_col",
+ "required" : false,
+ "type" : "date"
+ }, {
+ "id" : 8,
+ "name" : "string_col",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 9,
+ "name" : "timestamp_col",
+ "required" : false,
+ "type" : "timestamp"
+ }, {
+ "id" : 10,
+ "name" : "bool_col",
+ "required" : false,
+ "type" : "boolean"
+ } ]
+ } ],
+ "partition-spec" : [ ],
+ "default-spec-id" : 0,
+ "partition-specs" : [ {
+ "spec-id" : 0,
+ "fields" : [ ]
+ } ],
+ "last-partition-id" : 999,
+ "default-sort-order-id" : 0,
+ "sort-orders" : [ {
+ "order-id" : 0,
+ "fields" : [ ]
+ } ],
+ "properties" : {
+ "engine.hive.enabled" : "true",
+ "impala.events.catalogServiceId" : "95d77e9a4a3d4e27:96a993b690a3fba6",
+ "external.table.purge" : "TRUE",
+ "impala.events.catalogVersion" : "2108",
+ "write.format.default" : "parquet",
+ "hive.metastore.table.owner" : "danielbecker",
+ "OBJCAPABILITIES" : "EXTREAD,EXTWRITE",
+ "storage_handler" : "org.apache.iceberg.mr.hive.HiveIcebergStorageHandler"
+ },
+ "current-snapshot-id" : 2532372403033748214,
+ "refs" : {
+ "main" : {
+ "snapshot-id" : 2532372403033748214,
+ "type" : "branch"
+ }
+ },
+ "snapshots" : [ {
+ "snapshot-id" : 2532372403033748214,
+ "timestamp-ms" : 1728483427081,
+ "summary" : {
+ "operation" : "delete",
+ "changed-partition-count" : "0",
+ "total-records" : "0",
+ "total-files-size" : "0",
+ "total-data-files" : "0",
+ "total-delete-files" : "0",
+ "total-position-deletes" : "0",
+ "total-equality-deletes" : "0"
+ },
+ "manifest-list" : "TABLE_LOCATION_PLACEHOLDER/metadata/snap-2532372403033748214-1-c9f94c00-2920-4a39-8e7f-c7faf7e71a7d.avro",
+ "schema-id" : 0
+ } ],
+ "statistics" : [ {
+ "snapshot-id" : 2532372403033748214,
+ "statistics-path" : "TABLE_LOCATION_PLACEHOLDER/metadata/stats_for_unsupported_type.stats",
+ "file-size-in-bytes" : 496,
+ "file-footer-size-in-bytes" : 402,
+ "blob-metadata" : [ {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 1 ]
+ }, {
+ "type" : "apache-datasketches-theta-v1",
+ "snapshot-id" : 2532372403033748214,
+ "sequence-number" : 0,
+ "fields" : [ 10 ]
+ } ]
+ } ],
+ "snapshot-log" : [ {
+ "timestamp-ms" : 1728483427081,
+ "snapshot-id" : 2532372403033748214
+ } ],
+ "metadata-log" : [ {
+ "timestamp-ms" : 1728483321332,
+ "metadata-file" : "TABLE_LOCATION_PLACEHOLDER/metadata/00000-0991de67-dc39-4226-af9c-8f347dbd57e5.metadata.json"
+ } ]
+}
\ No newline at end of file
diff --git a/testdata/ice_puffin/generated/stats_for_unsupported_type.stats b/testdata/ice_puffin/generated/stats_for_unsupported_type.stats
new file mode 100644
index 000000000..12490e590
Binary files /dev/null and b/testdata/ice_puffin/generated/stats_for_unsupported_type.stats differ
diff --git a/testdata/ice_puffin/snap-2532372403033748214-1-c9f94c00-2920-4a39-8e7f-c7faf7e71a7d.avro b/testdata/ice_puffin/snap-2532372403033748214-1-c9f94c00-2920-4a39-8e7f-c7faf7e71a7d.avro
new file mode 100644
index 000000000..c0cebaa16
Binary files /dev/null and b/testdata/ice_puffin/snap-2532372403033748214-1-c9f94c00-2920-4a39-8e7f-c7faf7e71a7d.avro differ
diff --git a/tests/common/impala_test_suite.py b/tests/common/impala_test_suite.py
index 9f28cc29c..f092a4375 100644
--- a/tests/common/impala_test_suite.py
+++ b/tests/common/impala_test_suite.py
@@ -1404,3 +1404,23 @@ class ImpalaTestSuite(BaseTestSuite):
"""
assert length > 0
return prefix + ''.join(choice(string.ascii_lowercase) for i in range(length))
+
+ def _get_properties(self, section_name, name, is_db=False):
+ """Extracts the db/table properties mapping from the output of DESCRIBE FORMATTED"""
+ result = self.client.execute("describe {0} formatted {1}".format(
+ "database" if is_db else "", name))
+ match = False
+ properties = dict()
+ for row in result.data:
+ fields = row.split("\t")
+ if fields[0] != '':
+ # Start of new section.
+ if match:
+ # Finished processing matching section.
+ break
+ match = section_name in fields[0]
+ elif match:
+ if fields[1] == 'NULL':
+ break
+ properties[fields[1].rstrip()] = fields[2].rstrip()
+ return properties
diff --git a/tests/custom_cluster/test_iceberg_with_puffin.py b/tests/custom_cluster/test_iceberg_with_puffin.py
new file mode 100644
index 000000000..459d1ec90
--- /dev/null
+++ b/tests/custom_cluster/test_iceberg_with_puffin.py
@@ -0,0 +1,256 @@
+# 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.
+
+from __future__ import absolute_import, division, print_function
+import pytest
+
+from subprocess import check_call
+
+import glob
+import os
+import shutil
+import sys
+import tempfile
+
+from tests.common.custom_cluster_test_suite import CustomClusterTestSuite
+
+
+class TestIcebergWithPuffinStatsStartupFlag(CustomClusterTestSuite):
+ """Tests for checking the behaviour of the startup flag
+ 'disable_reading_puffin_stats'."""
+
+ @classmethod
+ def get_workload(self):
+ return 'functional-query'
+
+ @CustomClusterTestSuite.with_args(
+ catalogd_args='--disable_reading_puffin_stats=true')
+ @pytest.mark.execute_serially
+ def test_disable_reading_puffin(self):
+ self._read_ndv_stats_expect_result([-1, -1])
+
+ @CustomClusterTestSuite.with_args(
+ catalogd_args='--disable_reading_puffin_stats=false')
+ @pytest.mark.execute_serially
+ def test_enable_reading_puffin(self, unique_database):
+ self._read_ndv_stats_expect_result([2, 2])
+
+ def _read_ndv_stats_expect_result(self, expected_ndv_stats):
+ tbl_name = "functional_parquet.iceberg_with_puffin_stats"
+ show_col_stats_stmt = "show column stats {}".format(tbl_name)
+ query_result = self.execute_query(show_col_stats_stmt)
+
+ rows = query_result.get_data().split("\n")
+ ndvs = [int(row.split()[2]) for row in rows]
+ assert ndvs == expected_ndv_stats
+
+
+class TestIcebergTableWithPuffinStats(CustomClusterTestSuite):
+ """Tests related to Puffin statistics files for Iceberg tables."""
+
+ CREATE_TBL_STMT_TEMPLATE = """CREATE TABLE {} (
+ int_col1 INT,
+ int_col2 INT,
+ bigint_col BIGINT,
+ float_col FLOAT,
+ double_col DOUBLE,
+ decimal_col DECIMAL,
+ date_col DATE,
+ string_col STRING,
+ timestamp_col TIMESTAMP,
+ bool_col BOOLEAN) STORED BY ICEBERG"""
+
+ @classmethod
+ def get_workload(cls):
+ return 'functional-query'
+
+ @classmethod
+ def add_test_dimensions(cls):
+ super(TestIcebergTableWithPuffinStats, cls).add_test_dimensions()
+
+ @CustomClusterTestSuite.with_args(
+ catalogd_args='--disable_reading_puffin_stats=false')
+ def test_puffin_stats(self, vector, unique_database):
+ """Tests that Puffin stats are correctly read. The stats we use in this test do not
+ necessarily reflect the actual state of the table."""
+ tbl_name = unique_database + ".ice_puffin_tbl"
+ create_tbl_stmt = self.CREATE_TBL_STMT_TEMPLATE.format(tbl_name)
+ self.execute_query(create_tbl_stmt)
+
+ # Set stats in HMS so we can check that we fall back to that if we don't have Puffin
+ # stats.
+ set_stats_stmt = \
+ "alter table {} set column stats timestamp_col ('numDVs'='2000')".format(tbl_name)
+ self.execute_query(set_stats_stmt)
+
+ tbl_loc = self._get_table_location(tbl_name, vector)
+
+ tbl_properties = self._get_properties("Table Parameters:", tbl_name)
+ uuid = tbl_properties["uuid"]
+ metadata_json_path = tbl_properties["metadata_location"]
+
+ self._copy_files_to_puffin_tbl(tbl_name, tbl_loc, uuid)
+
+ self._check_all_stats_in_1_file(tbl_name, tbl_loc, metadata_json_path)
+ self._check_all_stats_in_2_files(tbl_name, tbl_loc, metadata_json_path)
+ self._check_duplicate_stats_in_1_file(tbl_name, tbl_loc, metadata_json_path)
+ self._check_duplicate_stats_in_2_files(tbl_name, tbl_loc, metadata_json_path)
+ self._check_one_file_current_one_not(tbl_name, tbl_loc, metadata_json_path)
+ self._check_not_all_blobs_current(tbl_name, tbl_loc, metadata_json_path)
+ self._check_missing_file(tbl_name, tbl_loc, metadata_json_path)
+ self._check_one_file_corrupt_one_not(tbl_name, tbl_loc, metadata_json_path)
+ self._check_all_files_corrupt(tbl_name, tbl_loc, metadata_json_path)
+ self._check_file_contains_invalid_field_id(tbl_name, tbl_loc, metadata_json_path)
+ self._check_stats_for_unsupported_type(tbl_name, tbl_loc, metadata_json_path)
+ self._check_invalid_and_corrupt_sketches(tbl_name, tbl_loc, metadata_json_path)
+
+ # Disable reading Puffin stats with a table property.
+ disable_puffin_reading_tbl_prop_stmt = "alter table {} set tblproperties( \
+ 'impala.iceberg_disable_reading_puffin_stats'='true')".format(tbl_name)
+ self.execute_query(disable_puffin_reading_tbl_prop_stmt)
+ # Refresh 'metadata_json_path'.
+ tbl_properties = self._get_properties("Table Parameters:", tbl_name)
+ metadata_json_path = tbl_properties["metadata_location"]
+
+ self._check_reading_puffin_stats_disabled_by_tbl_prop(
+ tbl_name, tbl_loc, metadata_json_path)
+
+ def _copy_files_to_puffin_tbl(self, tbl_name, tbl_loc, uuid):
+ version_info = sys.version_info
+ if version_info.major >= 3 and version_info.minor >= 2:
+ with tempfile.TemporaryDirectory() as tmpdir:
+ self._copy_files_to_puffin_tbl_impl(tbl_name, tbl_loc, uuid, tmpdir)
+ else:
+ try:
+ tmpdir = tempfile.mkdtemp()
+ self._copy_files_to_puffin_tbl_impl(tbl_name, tbl_loc, uuid, tmpdir)
+ finally:
+ shutil.rmtree(tmpdir)
+
+ def _copy_files_to_puffin_tbl_impl(self, tbl_name, tbl_loc, uuid, tmpdir):
+ metadata_dir = os.path.join(os.getenv("IMPALA_HOME"), "testdata/ice_puffin")
+ tbl_loc_placeholder = "TABLE_LOCATION_PLACEHOLDER"
+ uuid_placeholder = "UUID_PLACEHOLDER"
+
+ tmp_metadata_dir = tmpdir + "/dir"
+ tmp_generated_metadata_dir = os.path.join(tmp_metadata_dir, "generated")
+ shutil.copytree(metadata_dir, tmp_metadata_dir)
+
+ sed_location_pattern = "s|{}|{}|g".format(tbl_loc_placeholder, tbl_loc)
+ sed_uuid_pattern = "s/{}/{}/g".format(uuid_placeholder, uuid)
+ metadata_json_files = glob.glob(tmp_generated_metadata_dir + "/*metadata.json")
+ for metadata_json in metadata_json_files:
+ check_call(["sed", "-i", "-e", sed_location_pattern,
+ "-e", sed_uuid_pattern, metadata_json])
+
+ # Move all files from the 'generated' subdirectory to the parent directory so that
+ # all files end up in the same directory on HDFS.
+ mv_cmd = "mv {generated_dir}/* {parent_dir} && rmdir {generated_dir}".format(
+ generated_dir=tmp_generated_metadata_dir, parent_dir=tmp_metadata_dir)
+ check_call(["bash", "-c", mv_cmd])
+
+ # Copy the files to HDFS.
+ self.filesystem_client.copy_from_local(glob.glob(tmp_metadata_dir + "/*"),
+ tbl_loc + "/metadata")
+
+ def _check_all_stats_in_1_file(self, tbl_name, tbl_loc, metadata_json_path):
+ self._check_scenario(tbl_name, tbl_loc, metadata_json_path,
+ "all_stats_in_1_file.metadata.json", [1, 2, 3, 4, 5, 6, 7, 8, 9, -1])
+
+ def _check_all_stats_in_2_files(self, tbl_name, tbl_loc, metadata_json_path):
+ self._check_scenario(tbl_name, tbl_loc, metadata_json_path,
+ "stats_divided.metadata.json", [1, 2, 3, 4, 5, 6, 7, 8, 9, -1])
+
+ def _check_duplicate_stats_in_1_file(self, tbl_name, tbl_loc, metadata_json_path):
+ self._check_scenario(tbl_name, tbl_loc, metadata_json_path,
+ "duplicate_stats_in_1_file.metadata.json",
+ [1, 2, -1, -1, -1, -1, -1, -1, 2000, -1])
+
+ def _check_duplicate_stats_in_2_files(self, tbl_name, tbl_loc, metadata_json_path):
+ self._check_scenario(tbl_name, tbl_loc, metadata_json_path,
+ "duplicate_stats_in_2_files.metadata.json",
+ [1, 2, 3, -1, -1, -1, -1, -1, 2000, -1])
+
+ def _check_one_file_current_one_not(self, tbl_name, tbl_loc, metadata_json_path):
+ self._check_scenario(tbl_name, tbl_loc, metadata_json_path,
+ "one_file_current_one_not.metadata.json",
+ [1, 2, -1, -1, -1, -1, -1, -1, 2000, -1])
+
+ def _check_not_all_blobs_current(self, tbl_name, tbl_loc, metadata_json_path):
+ self._check_scenario(tbl_name, tbl_loc, metadata_json_path,
+ "not_all_blobs_current.metadata.json",
+ [1, 2, -1, 4, -1, -1, -1, -1, 2000, -1])
+
+ def _check_missing_file(self, tbl_name, tbl_loc, metadata_json_path):
+ self._check_scenario(tbl_name, tbl_loc, metadata_json_path,
+ "missing_file.metadata.json", [-1, -1, 3, 4, -1, -1, -1, -1, 2000, -1])
+
+ def _check_one_file_corrupt_one_not(self, tbl_name, tbl_loc, metadata_json_path):
+ self._check_scenario(tbl_name, tbl_loc, metadata_json_path,
+ "one_file_corrupt_one_not.metadata.json",
+ [-1, -1, 3, 4, -1, -1, -1, -1, 2000, -1])
+
+ def _check_all_files_corrupt(self, tbl_name, tbl_loc, metadata_json_path):
+ self._check_scenario(tbl_name, tbl_loc, metadata_json_path,
+ "all_files_corrupt.metadata.json", [-1, -1, -1, -1, -1, -1, -1, -1, 2000, -1])
+
+ def _check_file_contains_invalid_field_id(self, tbl_name, tbl_loc, metadata_json_path):
+ self._check_scenario(tbl_name, tbl_loc, metadata_json_path,
+ "file_contains_invalid_field_id.metadata.json",
+ [1, -1, -1, -1, -1, -1, -1, -1, 2000, -1])
+
+ def _check_stats_for_unsupported_type(self, tbl_name, tbl_loc,
+ metadata_json_path):
+ # Ndv stats are not supported for BOOLEAN in HMS, so we don't take it into account
+ # even if it is present in a Puffin file.
+ self._check_scenario(tbl_name, tbl_loc, metadata_json_path,
+ "stats_for_unsupported_type.metadata.json",
+ [2, -1, -1, -1, -1, -1, -1, -1, 2000, -1])
+
+ def _check_invalid_and_corrupt_sketches(self, tbl_name, tbl_loc,
+ metadata_json_path):
+ self._check_scenario(tbl_name, tbl_loc, metadata_json_path,
+ "invalidAndCorruptSketches.metadata.json",
+ [1, -1, 3, -1, 5, -1, -1, -1, 2000, -1])
+
+ def _check_reading_puffin_stats_disabled_by_tbl_prop(self, tbl_name, tbl_loc,
+ metadata_json_path):
+ self._check_scenario(tbl_name, tbl_loc, metadata_json_path,
+ "all_stats_in_1_file.metadata.json", [-1, -1, -1, -1, -1, -1, -1, -1, 2000, -1])
+
+ def _check_scenario(self, tbl_name, tbl_loc, current_metadata_json_path,
+ new_metadata_json_name, expected_ndvs):
+ self._change_metadata_json_file(tbl_name, tbl_loc, current_metadata_json_path,
+ new_metadata_json_name)
+
+ invalidate_metadata_stmt = "invalidate metadata {}".format(tbl_name)
+ self.execute_query(invalidate_metadata_stmt)
+ show_col_stats_stmt = "show column stats {}".format(tbl_name)
+ res = self.execute_query(show_col_stats_stmt)
+
+ ndvs = self._get_ndvs_from_query_result(res)
+ assert expected_ndvs == ndvs
+
+ def _change_metadata_json_file(self, tbl_name, tbl_loc, current_metadata_json_path,
+ new_metadata_json_name):
+ # Overwrite the current metadata.json file with the given file.
+ new_metadata_json_path = os.path.join(tbl_loc, "metadata", new_metadata_json_name)
+ self.filesystem_client.copy(new_metadata_json_path, current_metadata_json_path, True)
+
+ def _get_ndvs_from_query_result(self, query_result):
+ rows = query_result.get_data().split("\n")
+ return [int(row.split()[2]) for row in rows]
diff --git a/tests/metadata/test_ddl_base.py b/tests/metadata/test_ddl_base.py
index 494d99315..98ab26ca1 100644
--- a/tests/metadata/test_ddl_base.py
+++ b/tests/metadata/test_ddl_base.py
@@ -78,26 +78,6 @@ class TestDdlBase(ImpalaTestSuite):
"""Extracts the DB properties mapping from the output of DESCRIBE FORMATTED"""
return self._get_properties("Owner:", db_name, True)
- def _get_properties(self, section_name, name, is_db=False):
- """Extracts the db/table properties mapping from the output of DESCRIBE FORMATTED"""
- result = self.client.execute("describe {0} formatted {1}".format(
- "database" if is_db else "", name))
- match = False
- properties = dict()
- for row in result.data:
- fields = row.split("\t")
- if fields[0] != '':
- # Start of new section.
- if match:
- # Finished processing matching section.
- break
- match = section_name in fields[0]
- elif match:
- if fields[1] == 'NULL':
- break
- properties[fields[1].rstrip()] = fields[2].rstrip()
- return properties
-
def _get_property(self, property_name, name, is_db=False):
"""Extracts a db/table property value from the output of DESCRIBE FORMATTED."""
result = self.client.execute("describe {0} formatted {1}".format(
diff --git a/tests/query_test/test_ext_data_sources.py b/tests/query_test/test_ext_data_sources.py
index c799012ee..3161c290f 100644
--- a/tests/query_test/test_ext_data_sources.py
+++ b/tests/query_test/test_ext_data_sources.py
@@ -43,26 +43,6 @@ class TestExtDataSources(ImpalaTestSuite):
"""Extracts the table properties mapping from the output of DESCRIBE FORMATTED"""
return self._get_properties('Table Parameters:', table_name)
- def _get_properties(self, section_name, name, is_db=False):
- """Extracts the db/table properties mapping from the output of DESCRIBE FORMATTED"""
- result = self.client.execute("describe {0} formatted {1}".format(
- "database" if is_db else "", name))
- match = False
- properties = dict()
- for row in result.data:
- fields = row.split("\t")
- if fields[0] != '':
- # Start of new section.
- if match:
- # Finished processing matching section.
- break
- match = section_name in fields[0]
- elif match:
- if fields[1] == 'NULL':
- break
- properties[fields[1].rstrip()] = fields[2].rstrip()
- return properties
-
def test_verify_jdbc_table_properties(self, vector):
jdbc_tbl_name = "functional.alltypes_jdbc_datasource"
properties = self._get_tbl_properties(jdbc_tbl_name)
diff --git a/tests/query_test/test_iceberg.py b/tests/query_test/test_iceberg.py
index 0cc5e72a7..53a1e1834 100644
--- a/tests/query_test/test_iceberg.py
+++ b/tests/query_test/test_iceberg.py
@@ -24,7 +24,6 @@ import os
import pytest
import pytz
import random
-
import re
import time