IMPALA-13247: Support Reading Puffin files for the current snapshot

This change adds support for reading NDV statistics from Puffin files
when they are available for the current snapshot. Puffin files or blobs
that were written for other snapshots than the current one are ignored.
Because this behaviour is different from what we have for HMS stats and
may therefore be unintuitive for users, reading Puffin stats is disabled
by default; set the "--disable_reading_puffin_stats" startup flag to
false to enable it.

When Puffin stats reading is enabled, the NDV values read from Puffin
files take precedence over NDV values stored in the HMS. This is because
we only read Puffin stats for the current snapshot, so these values are
always up-to-date, while the values in the HMS may be stale.

Note that it is currently not possible to drop Puffin stats from Impala.
For this reason, this patch also introduces two ways of disabling the
reading of Puffin stats:
  - globally, with the aforementioned "--disable_reading_puffin_stats"
    startup flag: when it is set to true, Impala will never read Puffin
    stats
  - for specific tables, by setting the
    "impala.iceberg_disable_reading_puffin_stats" table property to
    true.

Note that this change is only about reading Puffin files, Impala does
not yet support writing them.

Testing:
 - created the PuffinDataGenerator tool which can generate Puffin files
   and metadata.json files for different scenarios (e.g. all stats are
   in the same Puffin file; stats for different columns are in different
   Puffin files; some Puffin files are corrupt etc.). The generated
   files are under the "testdata/ice_puffin/generated" directory.
 - The new custom cluster test class
   'test_iceberg_with_puffin.py::TestIcebergTableWithPuffinStats' uses
   the generated data to test various scenarios.
 - Added custom cluster tests that test the
   'disable_reading_puffin_stats' startup flag.

Change-Id: I50c1228988960a686d08a9b2942e01e366678866
Reviewed-on: http://gerrit.cloudera.org:8080/21605
Reviewed-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
This commit is contained in:
Daniel Becker
2024-06-24 16:29:31 +02:00
committed by Impala Public Jenkins
parent 9ca3ccb347
commit b05b408f17
60 changed files with 4400 additions and 42 deletions

View File

@@ -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[] = {

View File

@@ -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);

View File

@@ -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

View File

@@ -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
}

View File

@@ -435,6 +435,13 @@ under the License.
<version>${iceberg.version}</version>
</dependency>
<!-- Needed for reading Iceberg Puffin files. -->
<dependency>
<groupId>org.apache.datasketches</groupId>
<artifactId>datasketches-java</artifactId>
<version>${datasketches.version}</version>
</dependency>
<!-- Needed for tests like JdbcTest which instantiates HiveDriver -->
<dependency>
<groupId>org.apache.hive</groupId>

View File

@@ -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) {

View File

@@ -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<Integer, IcebergColumn> icebergFieldIdToCol_;
private Map<String, TIcebergPartitionStats> 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<Integer, PuffinStatsLoader.PuffinStatsRecord> puffinNdvs =
PuffinStatsLoader.loadPuffinStats(icebergApiTable_, getFullName());
for (Map.Entry<Integer, PuffinStatsLoader.PuffinStatsRecord> 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);

View File

@@ -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<Integer, PuffinStatsRecord> 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<Integer, PuffinStatsRecord> loadPuffinStats(Table iceApiTable,
String tblName) {
PuffinStatsLoader loader = new PuffinStatsLoader(iceApiTable, tblName);
final List<StatisticsFile> 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<Integer> fieldIdsFromFile = new ArrayList<>();
try {
PuffinReader puffinReader = createPuffinReader(statsFile);
List<BlobMetadata> 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<BlobMetadata, ByteBuffer> 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<BlobMetadata> 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<Integer> 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;
}
}

View File

@@ -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;
}

View File

@@ -85,6 +85,7 @@ under the License.
<log4j2.version>${env.IMPALA_LOG4J2_VERSION}</log4j2.version>
<dropwizard-metrics.version>${env.IMPALA_DROPWIZARD_METRICS_VERSION}</dropwizard-metrics.version>
<aircompressor.version>${env.IMPALA_AIRCOMPRESSOR_VERSION}</aircompressor.version>
<datasketches.version>${env.IMPALA_DATASKETCHES_VERSION}</datasketches.version>
</properties>
<repositories>
@@ -392,6 +393,7 @@ under the License.
<modules>
<module>datagenerator</module>
<module>puffin-data-generator</module>
<module>executor-deps</module>
<module>ext-data-source</module>
<module>../fe</module>

View File

@@ -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"
} ]
}

View File

@@ -0,0 +1,109 @@
<?xml version="1.0"?>
<!--
Licensed to the Apache Software Foundation (ASF) under one
or more contributor license agreements. See the NOTICE file
distributed with this work for additional information
regarding copyright ownership. The ASF licenses this file
to you under the Apache License, Version 2.0 (the
"License"); you may not use this file except in compliance
with the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing,
software distributed under the License is distributed on an
"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
KIND, either express or implied. See the License for the
specific language governing permissions and limitations
under the License.
-->
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<parent>
<groupId>org.apache.impala</groupId>
<artifactId>impala-parent</artifactId>
<version>4.5.0-SNAPSHOT</version>
</parent>
<modelVersion>4.0.0</modelVersion>
<artifactId>impala-puffin-data-generator</artifactId>
<packaging>jar</packaging>
<name>Puffin Test Data Generator</name>
<dependencies>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-common</artifactId>
<version>${hadoop.version}</version>
<exclusions>
<exclusion>
<groupId>org.eclipse.jetty</groupId>
<artifactId>*</artifactId>
</exclusion>
<exclusion>
<!-- IMPALA-9468: Avoid pulling in netty for security reasons -->
<groupId>io.netty</groupId>
<artifactId>*</artifactId>
</exclusion>
<exclusion>
<groupId>com.sun.jersey</groupId>
<artifactId>jersey-server</artifactId>
</exclusion>
<exclusion>
<groupId>com.sun.jersey</groupId>
<artifactId>jersey-servlet</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.iceberg</groupId>
<artifactId>iceberg-api</artifactId>
<version>${iceberg.version}</version>
</dependency>
<dependency>
<groupId>org.apache.iceberg</groupId>
<artifactId>iceberg-hive-runtime</artifactId>
<version>${iceberg.version}</version>
</dependency>
<!-- Needed for reading Iceberg Puffin files. -->
<dependency>
<groupId>org.apache.datasketches</groupId>
<artifactId>datasketches-java</artifactId>
<version>${datasketches.version}</version>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-databind</artifactId>
</dependency>
</dependencies>
<build>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-compiler-plugin</artifactId>
<version>3.11.0</version>
<configuration>
<source>1.8</source>
<target>1.8</target>
</configuration>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-surefire-plugin</artifactId>
<version>3.0.0</version>
<configuration>
<redirectTestOutputToFile>true</redirectTestOutputToFile>
</configuration>
</plugin>
</plugins>
</build>
</project>

View File

@@ -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<ByteBuffer> 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<Blob> 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<Blob> 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<Blob> 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<Blob> 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<FileData> 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<Blob> 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<Blob> 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<FileData> 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<Blob> 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<FileData> 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<Blob> blobs1 = new ArrayList<>();
blobs1.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 1, 1));
blobs1.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 2, 2));
List<Blob> blobs2 = new ArrayList<>();
blobs2.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 1, 5));
blobs2.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 3, 3));
List<FileData> 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<Blob> blobs1 = new ArrayList<>();
blobs1.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 1, 1));
blobs1.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 2, 2));
List<Blob> blobs2 = new ArrayList<>();
long notCurrentSnapshotId = snapshotId_ - 1;
blobs2.add(createBlob(notCurrentSnapshotId, SEQUENCE_NUMBER, 3, 3));
blobs2.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 4, 4));
List<FileData> 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<Blob> 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<FileData> 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<Blob> blobs1 = new ArrayList<>();
blobs1.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 1, 1));
blobs1.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 2, 2));
List<Blob> blobs2 = new ArrayList<>();
blobs2.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 3, 3));
blobs2.add(createBlob(snapshotId_, SEQUENCE_NUMBER, 4, 4));
List<FileData> 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<Blob> 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<Blob> 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<FileData> 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<Blob> 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<Blob> 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<FileData> 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<Blob> 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<FileData> puffinFiles = new ArrayList<>();
puffinFiles.add(corruptFile1);
writeFilesForScenario(puffinFiles, "file_contains_invalid_field_id.metadata.json");
}
private void writeStatForUnsupportedType() throws IOException {
List<Blob> 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<FileData> puffinFiles = new ArrayList<>();
puffinFiles.add(corruptFile1);
writeFilesForScenario(puffinFiles, "stats_for_unsupported_type.metadata.json");
}
private void writeFileWithInvalidAndCorruptSketches() throws IOException {
List<Blob> 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<FileData> 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<ByteBuffer> createSketches() {
ImmutableList.Builder<ByteBuffer> 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<FileData> 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<Blob> 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);
}
}
}

View File

@@ -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" : [ ]
}

View File

@@ -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"
} ]
}

View File

@@ -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"
} ]
}

View File

@@ -0,0 +1 @@
3

View File

@@ -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}

View File

@@ -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
1 # Table level constraints:
70 table_name:iceberg_partition_transforms_zorder, constraint:restrict_to, table_format:parquet/none/none table_name:iceberg_partitioned_orc_external, constraint:restrict_to, table_format:parquet/none/none
71 table_name:iceberg_resolution_test_external, constraint:restrict_to, table_format:parquet/none/none table_name:iceberg_partition_transforms_zorder, constraint:restrict_to, table_format:parquet/none/none
72 table_name:iceberg_alltypes_part, constraint:restrict_to, table_format:parquet/none/none table_name:iceberg_resolution_test_external, constraint:restrict_to, table_format:parquet/none/none
73 table_name:iceberg_alltypes_part, constraint:restrict_to, table_format:parquet/none/none
74 table_name:iceberg_alltypes_part_orc, constraint:restrict_to, table_format:parquet/none/none
75 table_name:iceberg_legacy_partition_schema_evolution, constraint:restrict_to, table_format:parquet/none/none
76 table_name:iceberg_legacy_partition_schema_evolution_orc, constraint:restrict_to, table_format:parquet/none/none

View File

@@ -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"
} ]
}

31
testdata/ice_puffin/README vendored Normal file
View File

@@ -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.

View File

@@ -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"
} ]
}

Binary file not shown.

View File

@@ -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"
} ]
}

Binary file not shown.

Binary file not shown.

Binary file not shown.

Binary file not shown.

View File

@@ -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"
} ]
}

Binary file not shown.

View File

@@ -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"
} ]
}

Binary file not shown.

Binary file not shown.

Binary file not shown.

View File

@@ -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"
} ]
}

Binary file not shown.

View File

@@ -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"
} ]
}

Binary file not shown.

View File

@@ -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"
} ]
}

Binary file not shown.

View File

@@ -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"
} ]
}

Binary file not shown.

Binary file not shown.

View File

@@ -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"
} ]
}

View File

@@ -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"
} ]
}

View File

@@ -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"
} ]
}

Binary file not shown.

Binary file not shown.

View File

@@ -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"
} ]
}

Binary file not shown.

View File

@@ -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

View File

@@ -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]

View File

@@ -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(

View File

@@ -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)

View File

@@ -24,7 +24,6 @@ import os
import pytest
import pytz
import random
import re
import time