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