You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by dk...@apache.org on 2023/02/24 09:13:45 UTC
[hive] branch master updated: HIVE-26808: Port iceberg catalog changes (Zsolt Miskolczi, reviewed by Denys Kuzmenko)
This is an automated email from the ASF dual-hosted git repository.
dkuzmenko pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git
The following commit(s) were added to refs/heads/master by this push:
new 89fd959a04f HIVE-26808: Port iceberg catalog changes (Zsolt Miskolczi, reviewed by Denys Kuzmenko)
89fd959a04f is described below
commit 89fd959a04f8871ef8ea0f47544a96a4651baaf0
Author: InvisibleProgrammer <zs...@gmail.com>
AuthorDate: Fri Feb 24 10:13:36 2023 +0100
HIVE-26808: Port iceberg catalog changes (Zsolt Miskolczi, reviewed by Denys Kuzmenko)
Closes #3907
---
.../java/org/apache/iceberg/util/LocationUtil.java | 38 +++
iceberg/iceberg-catalog/pom.xml | 6 +
.../java/org/apache/iceberg/hive/HiveCatalog.java | 68 +++--
.../apache/iceberg/hive/HiveTableOperations.java | 93 ++++++-
.../org/apache/iceberg/hive/TestHiveCatalog.java | 280 ++++++++++++++++++++-
.../java/org/apache/iceberg/data/FileHelpers.java | 9 +-
.../apache/iceberg/mr/TestIcebergInputFormats.java | 3 +-
.../iceberg/mr/hive/HiveIcebergTestUtils.java | 2 +-
.../hive/TestHiveIcebergStorageHandlerNoScan.java | 4 +-
.../apache/iceberg/util/TestCharSequenceSet.java | 53 ++++
.../org/apache/iceberg/util/TestLocationUtil.java | 57 +++++
.../positive/alter_multi_part_table_to_iceberg.q | 10 +-
.../queries/positive/alter_part_table_to_iceberg.q | 10 +-
.../test/queries/positive/alter_table_to_iceberg.q | 10 +-
.../positive/ctas_iceberg_partitioned_orc.q | 10 +-
.../src/test/queries/positive/iceberg_v2_deletes.q | 10 +-
.../queries/positive/mv_iceberg_partitioned_orc.q | 6 +
.../queries/positive/mv_iceberg_partitioned_orc2.q | 6 +
.../queries/positive/show_create_iceberg_table.q | 2 +
.../positive/truncate_force_iceberg_table.q | 12 +-
.../test/queries/positive/truncate_iceberg_table.q | 12 +-
.../positive/truncate_partitioned_iceberg_table.q | 12 +-
.../alter_multi_part_table_to_iceberg.q.out | 21 +-
.../positive/alter_part_table_to_iceberg.q.out | 21 +-
.../results/positive/alter_table_to_iceberg.q.out | 18 +-
.../results/positive/create_iceberg_table.q.out | 2 +
...create_iceberg_table_stored_as_fileformat.q.out | 10 +
.../create_iceberg_table_stored_by_iceberg.q.out | 2 +
...le_stored_by_iceberg_with_serdeproperties.q.out | 2 +
.../positive/ctas_iceberg_partitioned_orc.q.out | 7 +-
.../src/test/results/positive/ctlt_iceberg.q.out | 11 +
.../results/positive/describe_iceberg_table.q.out | 11 +
.../test/results/positive/iceberg_v2_deletes.q.out | 15 +-
.../positive/mv_iceberg_partitioned_orc.q.out | 10 +
.../positive/mv_iceberg_partitioned_orc2.q.out | 10 +
.../positive/show_create_iceberg_table.q.out | 15 ++
.../positive/truncate_force_iceberg_table.q.out | 12 +-
.../results/positive/truncate_iceberg_table.q.out | 30 ++-
.../truncate_partitioned_iceberg_table.q.out | 14 +-
.../positive/use_basic_stats_from_iceberg.q.out | 4 +
40 files changed, 837 insertions(+), 91 deletions(-)
diff --git a/core/src/main/java/org/apache/iceberg/util/LocationUtil.java b/core/src/main/java/org/apache/iceberg/util/LocationUtil.java
new file mode 100644
index 00000000000..42c26524f28
--- /dev/null
+++ b/core/src/main/java/org/apache/iceberg/util/LocationUtil.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.util;
+
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class LocationUtil {
+ private LocationUtil() {
+
+ }
+
+ public static String stripTrailingSlash(String path) {
+ Preconditions.checkArgument(path != null && path.length() > 0, "path must not be null or empty");
+
+ String result = path;
+ while (result.endsWith("/")) {
+ result = result.substring(0, result.length() - 1);
+ }
+ return result;
+ }
+}
diff --git a/iceberg/iceberg-catalog/pom.xml b/iceberg/iceberg-catalog/pom.xml
index 3b41f886870..e79ec65f0fe 100644
--- a/iceberg/iceberg-catalog/pom.xml
+++ b/iceberg/iceberg-catalog/pom.xml
@@ -42,6 +42,12 @@
<artifactId>hadoop-hdfs</artifactId>
<scope>test</scope>
</dependency>
+ <dependency>
+ <groupId>org.assertj</groupId>
+ <artifactId>assertj-core</artifactId>
+ <version>${assertj.version}</version>
+ <scope>test</scope>
+ </dependency>
<dependency>
<groupId>junit</groupId>
<artifactId>junit</artifactId>
diff --git a/iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveCatalog.java b/iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
index b97ff3daa8b..80f7d4065bb 100644
--- a/iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
+++ b/iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
@@ -36,12 +36,10 @@ import org.apache.hadoop.hive.metastore.api.Table;
import org.apache.hadoop.hive.metastore.api.UnknownDBException;
import org.apache.iceberg.BaseMetastoreCatalog;
import org.apache.iceberg.BaseMetastoreTableOperations;
-import org.apache.iceberg.BaseTable;
import org.apache.iceberg.CatalogProperties;
import org.apache.iceberg.CatalogUtil;
import org.apache.iceberg.ClientPool;
import org.apache.iceberg.TableMetadata;
-import org.apache.iceberg.TableMetadataParser;
import org.apache.iceberg.TableOperations;
import org.apache.iceberg.catalog.Namespace;
import org.apache.iceberg.catalog.SupportsNamespaces;
@@ -49,14 +47,16 @@ import org.apache.iceberg.catalog.TableIdentifier;
import org.apache.iceberg.exceptions.NamespaceNotEmptyException;
import org.apache.iceberg.exceptions.NoSuchNamespaceException;
import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.NotFoundException;
import org.apache.iceberg.hadoop.HadoopFileIO;
import org.apache.iceberg.io.FileIO;
-import org.apache.iceberg.io.InputFile;
import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.LocationUtil;
import org.apache.thrift.TException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -72,12 +72,14 @@ public class HiveCatalog extends BaseMetastoreCatalog implements SupportsNamespa
private FileIO fileIO;
private ClientPool<IMetaStoreClient, TException> clients;
private boolean listAllTables = false;
+ private Map<String, String> catalogProperties;
public HiveCatalog() {
}
@Override
public void initialize(String inputName, Map<String, String> properties) {
+ this.catalogProperties = ImmutableMap.copyOf(properties);
this.name = inputName;
if (conf == null) {
LOG.warn("No Hadoop Configuration was set, using the default environment Configuration");
@@ -89,7 +91,8 @@ public class HiveCatalog extends BaseMetastoreCatalog implements SupportsNamespa
}
if (properties.containsKey(CatalogProperties.WAREHOUSE_LOCATION)) {
- this.conf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, properties.get(CatalogProperties.WAREHOUSE_LOCATION));
+ this.conf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname,
+ LocationUtil.stripTrailingSlash(properties.get(CatalogProperties.WAREHOUSE_LOCATION)));
}
this.listAllTables = Boolean.parseBoolean(properties.getOrDefault(LIST_ALL_TABLES, LIST_ALL_TABLES_DEFAULT));
@@ -152,11 +155,16 @@ public class HiveCatalog extends BaseMetastoreCatalog implements SupportsNamespa
String database = identifier.namespace().level(0);
TableOperations ops = newTableOps(identifier);
- TableMetadata lastMetadata;
- if (purge && ops.current() != null) {
- lastMetadata = ops.current();
- } else {
- lastMetadata = null;
+ TableMetadata lastMetadata = null;
+ if (purge) {
+ try {
+ lastMetadata = ops.current();
+ } catch (NotFoundException e) {
+ LOG.warn(
+ "Failed to load table metadata for table: {}, continuing drop without purge",
+ identifier,
+ e);
+ }
}
try {
@@ -229,23 +237,6 @@ public class HiveCatalog extends BaseMetastoreCatalog implements SupportsNamespa
}
}
- @Override
- public org.apache.iceberg.Table registerTable(TableIdentifier identifier, String metadataFileLocation) {
- Preconditions.checkArgument(isValidIdentifier(identifier), "Invalid identifier: %s", identifier);
-
- // Throw an exception if this table already exists in the catalog.
- if (tableExists(identifier)) {
- throw new org.apache.iceberg.exceptions.AlreadyExistsException("Table already exists: %s", identifier);
- }
-
- TableOperations ops = newTableOps(identifier);
- InputFile metadataFile = fileIO.newInputFile(metadataFileLocation);
- TableMetadata metadata = TableMetadataParser.read(ops.io(), metadataFile);
- ops.commit(null, metadata);
-
- return new BaseTable(ops, identifier.toString());
- }
-
@Override
public void createNamespace(Namespace namespace, Map<String, String> meta) {
Preconditions.checkArgument(
@@ -465,19 +456,17 @@ public class HiveCatalog extends BaseMetastoreCatalog implements SupportsNamespa
throw new RuntimeException("Interrupted during commit", e);
}
- // Otherwise stick to the {WAREHOUSE_DIR}/{DB_NAME}.db/{TABLE_NAME} path
- String warehouseLocation = getWarehouseLocation();
- return String.format(
- "%s/%s.db/%s",
- warehouseLocation,
- tableIdentifier.namespace().levels()[0],
- tableIdentifier.name());
+ // Otherwise, stick to the {WAREHOUSE_DIR}/{DB_NAME}.db/{TABLE_NAME} path
+ String databaseLocation = databaseLocation(tableIdentifier.namespace().levels()[0]);
+ return String.format("%s/%s", databaseLocation, tableIdentifier.name());
}
- private String getWarehouseLocation() {
+ private String databaseLocation(String databaseName) {
String warehouseLocation = conf.get(HiveConf.ConfVars.METASTOREWAREHOUSE.varname);
- Preconditions.checkNotNull(warehouseLocation, "Warehouse location is not set: hive.metastore.warehouse.dir=null");
- return warehouseLocation;
+ Preconditions.checkNotNull(
+ warehouseLocation, "Warehouse location is not set: hive.metastore.warehouse.dir=null");
+ warehouseLocation = LocationUtil.stripTrailingSlash(warehouseLocation);
+ return String.format("%s/%s.db", warehouseLocation, databaseName);
}
private String getExternalWarehouseLocation() {
@@ -510,7 +499,7 @@ public class HiveCatalog extends BaseMetastoreCatalog implements SupportsNamespa
database.setName(namespace.level(0));
database.setLocationUri(new Path(getExternalWarehouseLocation(), namespace.level(0)).toString() + ".db");
- database.setManagedLocationUri(new Path(getWarehouseLocation(), namespace.level(0)).toString() + ".db");
+ database.setManagedLocationUri(databaseLocation(namespace.level(0)));
meta.forEach((key, value) -> {
if (key.equals("comment")) {
@@ -545,6 +534,11 @@ public class HiveCatalog extends BaseMetastoreCatalog implements SupportsNamespa
return conf;
}
+ @Override
+ protected Map<String, String> properties() {
+ return catalogProperties == null ? ImmutableMap.of() : catalogProperties;
+ }
+
@VisibleForTesting
void setListAllTables(boolean listAllTables) {
this.listAllTables = listAllTables;
diff --git a/iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java b/iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
index 9fe617de825..43f7e52382b 100644
--- a/iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
+++ b/iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
@@ -37,10 +37,14 @@ import org.apache.hadoop.hive.metastore.api.SerDeInfo;
import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
import org.apache.hadoop.hive.metastore.api.Table;
import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
+import org.apache.hive.iceberg.com.fasterxml.jackson.core.JsonProcessingException;
import org.apache.iceberg.BaseMetastoreTableOperations;
import org.apache.iceberg.ClientPool;
+import org.apache.iceberg.PartitionSpecParser;
+import org.apache.iceberg.SchemaParser;
import org.apache.iceberg.Snapshot;
import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.SortOrderParser;
import org.apache.iceberg.TableMetadata;
import org.apache.iceberg.TableProperties;
import org.apache.iceberg.exceptions.AlreadyExistsException;
@@ -56,6 +60,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.BiMap;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableBiMap;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.JsonUtil;
import org.apache.parquet.hadoop.ParquetOutputFormat;
import org.apache.thrift.TException;
import org.slf4j.Logger;
@@ -72,6 +77,12 @@ public class HiveTableOperations extends BaseMetastoreTableOperations {
private static final String HIVE_ICEBERG_METADATA_REFRESH_MAX_RETRIES = "iceberg.hive.metadata-refresh-max-retries";
private static final int HIVE_ICEBERG_METADATA_REFRESH_MAX_RETRIES_DEFAULT = 2;
+ // the max size is based on HMS backend database. For Hive versions below 2.3, the max table parameter size is 4000
+ // characters, see https://issues.apache.org/jira/browse/HIVE-12274
+ // set to 0 to not expose Iceberg metadata in HMS Table properties.
+ private static final String HIVE_TABLE_PROPERTY_MAX_SIZE = "iceberg.hive.table-property-max-size";
+ private static final long HIVE_TABLE_PROPERTY_MAX_SIZE_DEFAULT = 32672;
+
private static final BiMap<String, String> ICEBERG_TO_HMS_TRANSLATION = ImmutableBiMap.of(
// gc.enabled in Iceberg and external.table.purge in Hive are meant to do the same things but with different names
GC_ENABLED, "external.table.purge",
@@ -100,6 +111,7 @@ public class HiveTableOperations extends BaseMetastoreTableOperations {
private final String database;
private final String tableName;
private final Configuration conf;
+ private final long maxHiveTablePropertySize;
private final int metadataRefreshMaxRetries;
private final FileIO fileIO;
private final ClientPool<IMetaStoreClient, TException> metaClients;
@@ -115,6 +127,7 @@ public class HiveTableOperations extends BaseMetastoreTableOperations {
this.tableName = table;
this.metadataRefreshMaxRetries =
conf.getInt(HIVE_ICEBERG_METADATA_REFRESH_MAX_RETRIES, HIVE_ICEBERG_METADATA_REFRESH_MAX_RETRIES_DEFAULT);
+ this.maxHiveTablePropertySize = conf.getLong(HIVE_TABLE_PROPERTY_MAX_SIZE, HIVE_TABLE_PROPERTY_MAX_SIZE_DEFAULT);
}
@Override
@@ -250,6 +263,8 @@ public class HiveTableOperations extends BaseMetastoreTableOperations {
} finally {
cleanupMetadataAndUnlock(commitStatus, newMetadataLocation, commitLock);
}
+
+ LOG.info("Committed to table {} with the new metadata location {}", fullName, newMetadataLocation);
}
@VisibleForTesting
@@ -341,9 +356,84 @@ public class HiveTableOperations extends BaseMetastoreTableOperations {
parameters.put(StatsSetupConst.TOTAL_SIZE, summary.get(SnapshotSummary.TOTAL_FILE_SIZE_PROP));
}
+ setSnapshotStats(metadata, parameters);
+ setSchema(metadata, parameters);
+ setPartitionSpec(metadata, parameters);
+ setSortOrder(metadata, parameters);
+
tbl.setParameters(parameters);
}
+ @VisibleForTesting
+ void setSnapshotStats(TableMetadata metadata, Map<String, String> parameters) {
+ parameters.remove(TableProperties.CURRENT_SNAPSHOT_ID);
+ parameters.remove(TableProperties.CURRENT_SNAPSHOT_TIMESTAMP);
+ parameters.remove(TableProperties.CURRENT_SNAPSHOT_SUMMARY);
+
+ Snapshot currentSnapshot = metadata.currentSnapshot();
+ if (exposeInHmsProperties() && currentSnapshot != null) {
+ parameters.put(TableProperties.CURRENT_SNAPSHOT_ID, String.valueOf(currentSnapshot.snapshotId()));
+ parameters.put(TableProperties.CURRENT_SNAPSHOT_TIMESTAMP, String.valueOf(currentSnapshot.timestampMillis()));
+ setSnapshotSummary(parameters, currentSnapshot);
+ }
+
+ parameters.put(TableProperties.SNAPSHOT_COUNT, String.valueOf(metadata.snapshots().size()));
+ }
+
+ @VisibleForTesting
+ void setSnapshotSummary(Map<String, String> parameters, Snapshot currentSnapshot) {
+ try {
+ String summary = JsonUtil.mapper().writeValueAsString(currentSnapshot.summary());
+ if (summary.length() <= maxHiveTablePropertySize) {
+ parameters.put(TableProperties.CURRENT_SNAPSHOT_SUMMARY, summary);
+ } else {
+ LOG.warn("Not exposing the current snapshot({}) summary in HMS since it exceeds {} characters",
+ currentSnapshot.snapshotId(), maxHiveTablePropertySize);
+ }
+ } catch (JsonProcessingException e) {
+ LOG.warn("Failed to convert current snapshot({}) summary to a json string", currentSnapshot.snapshotId(), e);
+ }
+ }
+
+ @VisibleForTesting
+ void setSchema(TableMetadata metadata, Map<String, String> parameters) {
+ parameters.remove(TableProperties.CURRENT_SCHEMA);
+ if (exposeInHmsProperties() && metadata.schema() != null) {
+ String schema = SchemaParser.toJson(metadata.schema());
+ setField(parameters, TableProperties.CURRENT_SCHEMA, schema);
+ }
+ }
+
+ @VisibleForTesting
+ void setPartitionSpec(TableMetadata metadata, Map<String, String> parameters) {
+ parameters.remove(TableProperties.DEFAULT_PARTITION_SPEC);
+ if (exposeInHmsProperties() && metadata.spec() != null && metadata.spec().isPartitioned()) {
+ String spec = PartitionSpecParser.toJson(metadata.spec());
+ setField(parameters, TableProperties.DEFAULT_PARTITION_SPEC, spec);
+ }
+ }
+
+ @VisibleForTesting
+ void setSortOrder(TableMetadata metadata, Map<String, String> parameters) {
+ parameters.remove(TableProperties.DEFAULT_SORT_ORDER);
+ if (exposeInHmsProperties() && metadata.sortOrder() != null && metadata.sortOrder().isSorted()) {
+ String sortOrder = SortOrderParser.toJson(metadata.sortOrder());
+ setField(parameters, TableProperties.DEFAULT_SORT_ORDER, sortOrder);
+ }
+ }
+
+ private void setField(Map<String, String> parameters, String key, String value) {
+ if (value.length() <= maxHiveTablePropertySize) {
+ parameters.put(key, value);
+ } else {
+ LOG.warn("Not exposing {} in HMS since it exceeds {} characters", key, maxHiveTablePropertySize);
+ }
+ }
+
+ private boolean exposeInHmsProperties() {
+ return maxHiveTablePropertySize > 0;
+ }
+
private StorageDescriptor storageDescriptor(TableMetadata metadata, boolean hiveEngineEnabled) {
final StorageDescriptor storageDescriptor = new StorageDescriptor();
@@ -377,8 +467,7 @@ public class HiveTableOperations extends BaseMetastoreTableOperations {
io().deleteFile(metadataLocation);
}
} catch (RuntimeException e) {
- LOG.error("Fail to cleanup metadata file at {}", metadataLocation, e);
- throw e;
+ LOG.error("Failed to cleanup metadata file at {}", metadataLocation, e);
} finally {
doUnlock(lock);
}
diff --git a/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestHiveCatalog.java b/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestHiveCatalog.java
index 6ba3a46a27a..00d7468cf4a 100644
--- a/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestHiveCatalog.java
+++ b/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestHiveCatalog.java
@@ -21,27 +21,44 @@ package org.apache.iceberg.hive;
import java.util.List;
import java.util.Map;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hadoop.hive.metastore.api.Database;
import org.apache.iceberg.AssertHelpers;
import org.apache.iceberg.CachingCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.FileFormat;
import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.PartitionSpecParser;
import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.Snapshot;
import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.SortOrderParser;
import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
import org.apache.iceberg.TableProperties;
import org.apache.iceberg.Transaction;
+import org.apache.iceberg.UpdateSchema;
import org.apache.iceberg.catalog.Catalog;
import org.apache.iceberg.catalog.Namespace;
import org.apache.iceberg.catalog.TableIdentifier;
import org.apache.iceberg.exceptions.AlreadyExistsException;
import org.apache.iceberg.exceptions.NamespaceNotEmptyException;
import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
import org.apache.iceberg.transforms.Transform;
import org.apache.iceberg.transforms.Transforms;
import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.JsonUtil;
import org.apache.thrift.TException;
import org.junit.Assert;
import org.junit.Rule;
@@ -51,7 +68,16 @@ import org.junit.rules.TemporaryFolder;
import static org.apache.iceberg.NullOrder.NULLS_FIRST;
import static org.apache.iceberg.SortDirection.ASC;
+import static org.apache.iceberg.TableProperties.CURRENT_SCHEMA;
+import static org.apache.iceberg.TableProperties.CURRENT_SNAPSHOT_ID;
+import static org.apache.iceberg.TableProperties.CURRENT_SNAPSHOT_SUMMARY;
+import static org.apache.iceberg.TableProperties.CURRENT_SNAPSHOT_TIMESTAMP;
+import static org.apache.iceberg.TableProperties.DEFAULT_PARTITION_SPEC;
+import static org.apache.iceberg.TableProperties.DEFAULT_SORT_ORDER;
+import static org.apache.iceberg.expressions.Expressions.bucket;
import static org.apache.iceberg.types.Types.NestedField.required;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
public class TestHiveCatalog extends HiveMetastoreTest {
private static ImmutableMap meta = ImmutableMap.of(
@@ -220,7 +246,7 @@ public class TestHiveCatalog extends HiveMetastoreTest {
}
@Test
- public void testCreateTableDefaultSortOrder() {
+ public void testCreateTableDefaultSortOrder() throws Exception {
Schema schema = new Schema(
required(1, "id", Types.IntegerType.get(), "unique ID"),
required(2, "data", Types.StringType.get())
@@ -234,13 +260,16 @@ public class TestHiveCatalog extends HiveMetastoreTest {
Table table = catalog.createTable(tableIdent, schema, spec);
Assert.assertEquals("Order ID must match", 0, table.sortOrder().orderId());
Assert.assertTrue("Order must unsorted", table.sortOrder().isUnsorted());
+
+ Assert.assertFalse("Must not have default sort order in catalog",
+ hmsTableParameters().containsKey(DEFAULT_SORT_ORDER));
} finally {
catalog.dropTable(tableIdent);
}
}
@Test
- public void testCreateTableCustomSortOrder() {
+ public void testCreateTableCustomSortOrder() throws Exception {
Schema schema = new Schema(
required(1, "id", Types.IntegerType.get(), "unique ID"),
required(2, "data", Types.StringType.get())
@@ -265,6 +294,8 @@ public class TestHiveCatalog extends HiveMetastoreTest {
Assert.assertEquals("Null order must match ", NULLS_FIRST, sortOrder.fields().get(0).nullOrder());
Transform<?, ?> transform = Transforms.identity(Types.IntegerType.get());
Assert.assertEquals("Transform must match", transform, sortOrder.fields().get(0).transform());
+
+ Assert.assertEquals(SortOrderParser.toJson(table.sortOrder()), hmsTableParameters().get(DEFAULT_SORT_ORDER));
} finally {
catalog.dropTable(tableIdent);
}
@@ -411,6 +442,22 @@ public class TestHiveCatalog extends HiveMetastoreTest {
});
}
+ @Test
+ public void testDropTableWithoutMetadataFile() {
+ TableIdentifier identifier = TableIdentifier.of(DB_NAME, "tbl");
+ Schema tableSchema =
+ new Schema(Types.StructType.of(required(1, "id", Types.LongType.get())).fields());
+ catalog.createTable(identifier, tableSchema);
+ String metadataFileLocation = catalog.newTableOps(identifier).current().metadataFileLocation();
+ TableOperations ops = catalog.newTableOps(identifier);
+ ops.io().deleteFile(metadataFileLocation);
+ Assert.assertTrue(catalog.dropTable(identifier));
+
+ org.assertj.core.api.Assertions.assertThatThrownBy(() -> catalog.loadTable(identifier))
+ .isInstanceOf(NoSuchTableException.class)
+ .hasMessageContaining("Table does not exist:");
+ }
+
@Test
public void testTableName() {
Schema schema = new Schema(
@@ -452,6 +499,26 @@ public class TestHiveCatalog extends HiveMetastoreTest {
TableIdentifier tableIdentifier = TableIdentifier.of(DB_NAME, "tbl");
String location = temp.newFolder("tbl").toString();
+ try {
+ catalog.buildTable(tableIdentifier, schema)
+ .withLocation(location)
+ .create();
+
+ Assert.assertNotNull(hmsTableParameters().get(TableProperties.UUID));
+ } finally {
+ catalog.dropTable(tableIdentifier);
+ }
+ }
+
+ @Test
+ public void testSnapshotStatsTableProperties() throws Exception {
+ Schema schema = new Schema(
+ required(1, "id", Types.IntegerType.get(), "unique ID"),
+ required(2, "data", Types.StringType.get())
+ );
+ TableIdentifier tableIdentifier = TableIdentifier.of(DB_NAME, "tbl");
+ String location = temp.newFolder("tbl").toString();
+
try {
catalog.buildTable(tableIdentifier, schema)
.withLocation(location)
@@ -461,11 +528,214 @@ public class TestHiveCatalog extends HiveMetastoreTest {
org.apache.hadoop.hive.metastore.api.Table hmsTable =
metastoreClient.getTable(tableIdentifier.namespace().level(0), tableName);
- // check parameters are in expected state
- Map<String, String> parameters = hmsTable.getParameters();
- Assert.assertNotNull(parameters.get(TableProperties.UUID));
+ // check whether parameters are in expected state
+ Map<String, String> parameters = hmsTableParameters();
+ Assert.assertEquals("0", parameters.get(TableProperties.SNAPSHOT_COUNT));
+ Assert.assertNull(parameters.get(CURRENT_SNAPSHOT_SUMMARY));
+ Assert.assertNull(parameters.get(CURRENT_SNAPSHOT_ID));
+ Assert.assertNull(parameters.get(CURRENT_SNAPSHOT_TIMESTAMP));
+
+ // create a snapshot
+ Table icebergTable = catalog.loadTable(tableIdentifier);
+ String fileName = UUID.randomUUID().toString();
+ DataFile file = DataFiles.builder(icebergTable.spec())
+ .withPath(FileFormat.PARQUET.addExtension(fileName))
+ .withRecordCount(2)
+ .withFileSizeInBytes(0)
+ .build();
+ icebergTable.newFastAppend().appendFile(file).commit();
+
+ // check whether parameters are in expected state
+ parameters = hmsTableParameters();
+ Assert.assertEquals("1", parameters.get(TableProperties.SNAPSHOT_COUNT));
+ String summary = JsonUtil.mapper().writeValueAsString(icebergTable.currentSnapshot().summary());
+ Assert.assertEquals(summary, parameters.get(CURRENT_SNAPSHOT_SUMMARY));
+ long snapshotId = icebergTable.currentSnapshot().snapshotId();
+ Assert.assertEquals(String.valueOf(snapshotId), parameters.get(CURRENT_SNAPSHOT_ID));
+ Assert.assertEquals(String.valueOf(icebergTable.currentSnapshot().timestampMillis()),
+ parameters.get(CURRENT_SNAPSHOT_TIMESTAMP));
} finally {
catalog.dropTable(tableIdentifier);
}
}
+
+ @Test
+ public void testSetSnapshotSummary() throws Exception {
+ Configuration conf = new Configuration();
+ conf.set("iceberg.hive.table-property-max-size", "4000");
+ HiveTableOperations ops = new HiveTableOperations(conf, null, null, catalog.name(), DB_NAME, "tbl");
+ Snapshot snapshot = mock(Snapshot.class);
+ Map<String, String> summary = Maps.newHashMap();
+ when(snapshot.summary()).thenReturn(summary);
+
+ // create a snapshot summary whose json string size is less than the limit
+ for (int i = 0; i < 100; i++) {
+ summary.put(String.valueOf(i), "value");
+ }
+ Assert.assertTrue(JsonUtil.mapper().writeValueAsString(summary).length() < 4000);
+ Map<String, String> parameters = Maps.newHashMap();
+ ops.setSnapshotSummary(parameters, snapshot);
+ Assert.assertEquals("The snapshot summary must be in parameters", 1, parameters.size());
+
+ // create a snapshot summary whose json string size exceeds the limit
+ for (int i = 0; i < 1000; i++) {
+ summary.put(String.valueOf(i), "value");
+ }
+ long summarySize = JsonUtil.mapper().writeValueAsString(summary).length();
+ // the limit has been updated to 4000 instead of the default value(32672)
+ Assert.assertTrue(summarySize > 4000 && summarySize < 32672);
+ parameters.remove(CURRENT_SNAPSHOT_SUMMARY);
+ ops.setSnapshotSummary(parameters, snapshot);
+ Assert.assertEquals("The snapshot summary must not be in parameters due to the size limit", 0, parameters.size());
+ }
+
+ @Test
+ public void testNotExposeTableProperties() {
+ Configuration conf = new Configuration();
+ conf.set("iceberg.hive.table-property-max-size", "0");
+ HiveTableOperations ops = new HiveTableOperations(conf, null, null, catalog.name(), DB_NAME, "tbl");
+ TableMetadata metadata = mock(TableMetadata.class);
+ Map<String, String> parameters = Maps.newHashMap();
+ parameters.put(CURRENT_SNAPSHOT_SUMMARY, "summary");
+ parameters.put(CURRENT_SNAPSHOT_ID, "snapshotId");
+ parameters.put(CURRENT_SNAPSHOT_TIMESTAMP, "timestamp");
+ parameters.put(CURRENT_SCHEMA, "schema");
+ parameters.put(DEFAULT_PARTITION_SPEC, "partitionSpec");
+ parameters.put(DEFAULT_SORT_ORDER, "sortOrder");
+
+ ops.setSnapshotStats(metadata, parameters);
+ Assert.assertNull(parameters.get(CURRENT_SNAPSHOT_SUMMARY));
+ Assert.assertNull(parameters.get(CURRENT_SNAPSHOT_ID));
+ Assert.assertNull(parameters.get(CURRENT_SNAPSHOT_TIMESTAMP));
+
+ ops.setSchema(metadata, parameters);
+ Assert.assertNull(parameters.get(CURRENT_SCHEMA));
+
+ ops.setPartitionSpec(metadata, parameters);
+ Assert.assertNull(parameters.get(DEFAULT_PARTITION_SPEC));
+
+ ops.setSortOrder(metadata, parameters);
+ Assert.assertNull(parameters.get(DEFAULT_SORT_ORDER));
+ }
+
+ @Test
+ public void testSetDefaultPartitionSpec() throws Exception {
+ Schema schema = new Schema(
+ required(1, "id", Types.IntegerType.get(), "unique ID"),
+ required(2, "data", Types.StringType.get())
+ );
+ TableIdentifier tableIdent = TableIdentifier.of(DB_NAME, "tbl");
+
+ try {
+ Table table = catalog.buildTable(tableIdent, schema).create();
+ Assert.assertFalse("Must not have default partition spec",
+ hmsTableParameters().containsKey(TableProperties.DEFAULT_PARTITION_SPEC));
+
+ table.updateSpec().addField(bucket("data", 16)).commit();
+ Assert.assertEquals(PartitionSpecParser.toJson(table.spec()),
+ hmsTableParameters().get(TableProperties.DEFAULT_PARTITION_SPEC));
+ } finally {
+ catalog.dropTable(tableIdent);
+ }
+ }
+
+ @Test
+ public void testSetCurrentSchema() throws Exception {
+ Schema schema = new Schema(
+ required(1, "id", Types.IntegerType.get(), "unique ID"),
+ required(2, "data", Types.StringType.get())
+ );
+ TableIdentifier tableIdent = TableIdentifier.of(DB_NAME, "tbl");
+
+ try {
+ Table table = catalog.buildTable(tableIdent, schema).create();
+
+ Assert.assertEquals(SchemaParser.toJson(table.schema()), hmsTableParameters().get(CURRENT_SCHEMA));
+
+ // add many new fields to make the schema json string exceed the limit
+ UpdateSchema updateSchema = table.updateSchema();
+ for (int i = 0; i < 600; i++) {
+ updateSchema.addColumn("new_col_" + i, Types.StringType.get());
+ }
+ updateSchema.commit();
+
+ Assert.assertTrue(SchemaParser.toJson(table.schema()).length() > 32672);
+ Assert.assertNull(hmsTableParameters().get(CURRENT_SCHEMA));
+ } finally {
+ catalog.dropTable(tableIdent);
+ }
+ }
+
+ private Map<String, String> hmsTableParameters() throws TException {
+ org.apache.hadoop.hive.metastore.api.Table hmsTable = metastoreClient.getTable(DB_NAME, "tbl");
+ return hmsTable.getParameters();
+ }
+
+ @Test
+ public void testConstructorWarehousePathWithEndSlash() {
+ HiveCatalog catalogWithSlash = new HiveCatalog();
+ String wareHousePath = "s3://bucket/db/tbl";
+
+ catalogWithSlash.initialize(
+ "hive_catalog", ImmutableMap.of(CatalogProperties.WAREHOUSE_LOCATION, wareHousePath + "/"));
+ Assert.assertEquals(
+ "Should have trailing slash stripped",
+ wareHousePath,
+ catalogWithSlash.getConf().get(HiveConf.ConfVars.METASTOREWAREHOUSE.varname));
+ }
+
+ @Test
+ public void testTablePropsDefinedAtCatalogLevel() {
+ Schema schema = new Schema(required(1, "id", Types.IntegerType.get(), "unique ID"));
+ TableIdentifier tableIdent = TableIdentifier.of(DB_NAME, "tbl");
+
+ ImmutableMap<String, String> catalogProps =
+ ImmutableMap.of(
+ "table-default.key1", "catalog-default-key1",
+ "table-default.key2", "catalog-default-key2",
+ "table-default.key3", "catalog-default-key3",
+ "table-override.key3", "catalog-override-key3",
+ "table-override.key4", "catalog-override-key4");
+ Catalog hiveCatalog =
+ CatalogUtil.loadCatalog(
+ HiveCatalog.class.getName(),
+ CatalogUtil.ICEBERG_CATALOG_TYPE_HIVE,
+ catalogProps,
+ hiveConf);
+
+ try {
+ Table table =
+ hiveCatalog
+ .buildTable(tableIdent, schema)
+ .withProperty("key2", "table-key2")
+ .withProperty("key3", "table-key3")
+ .withProperty("key5", "table-key5")
+ .create();
+
+ Assert.assertEquals(
+ "Table defaults set for the catalog must be added to the table properties.",
+ "catalog-default-key1",
+ table.properties().get("key1"));
+ Assert.assertEquals(
+ "Table property must override table default properties set at catalog level.",
+ "table-key2",
+ table.properties().get("key2"));
+ Assert.assertEquals(
+ "Table property override set at catalog level must override table default" +
+ " properties set at catalog level and table property specified.",
+ "catalog-override-key3",
+ table.properties().get("key3"));
+ Assert.assertEquals(
+ "Table override not in table props or defaults should be added to table properties",
+ "catalog-override-key4",
+ table.properties().get("key4"));
+ Assert.assertEquals(
+ "Table properties without any catalog level default or override should be added to table" +
+ " properties.",
+ "table-key5",
+ table.properties().get("key5"));
+ } finally {
+ hiveCatalog.dropTable(tableIdent);
+ }
+ }
}
diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/data/FileHelpers.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/data/FileHelpers.java
index 4919e6b060e..a68fcde2c92 100644
--- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/data/FileHelpers.java
+++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/data/FileHelpers.java
@@ -22,7 +22,6 @@ package org.apache.iceberg.data;
import java.io.Closeable;
import java.io.IOException;
import java.util.List;
-import java.util.Locale;
import java.util.Map;
import org.apache.iceberg.DataFile;
import org.apache.iceberg.DataFiles;
@@ -52,22 +51,22 @@ public class FileHelpers {
}
public static Pair<DeleteFile, CharSequenceSet> writeDeleteFile(Table table, OutputFile out,
- List<Pair<CharSequence, Long>> deletes)
+ List<Pair<CharSequence, Long>> deletes)
throws IOException {
return writeDeleteFile(table, out, null, deletes);
}
public static Pair<DeleteFile, CharSequenceSet> writeDeleteFile(Table table, OutputFile out, StructLike partition,
- List<Pair<CharSequence, Long>> deletes)
+ List<Pair<CharSequence, Long>> deletes)
throws IOException {
FileFormat format = defaultFormat(table.properties());
FileAppenderFactory<Record> factory = new GenericAppenderFactory(table.schema(), table.spec());
PositionDeleteWriter<Record> writer =
factory.newPosDeleteWriter(encrypt(out), format, partition);
+ PositionDelete<Record> posDelete = PositionDelete.create();
try (Closeable toClose = writer) {
for (Pair<CharSequence, Long> delete : deletes) {
- PositionDelete<Record> posDelete = PositionDelete.create();
writer.write(posDelete.set(delete.first(), delete.second(), null));
}
}
@@ -140,6 +139,6 @@ public class FileHelpers {
private static FileFormat defaultFormat(Map<String, String> properties) {
String formatString = properties.getOrDefault(DEFAULT_FILE_FORMAT, DEFAULT_FILE_FORMAT_DEFAULT);
- return FileFormat.valueOf(formatString.toUpperCase(Locale.ENGLISH));
+ return FileFormat.fromString(formatString);
}
}
diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/TestIcebergInputFormats.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/TestIcebergInputFormats.java
index 18e15d45e87..0f749f59b24 100644
--- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/TestIcebergInputFormats.java
+++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/TestIcebergInputFormats.java
@@ -23,7 +23,6 @@ import java.io.File;
import java.io.IOException;
import java.io.UncheckedIOException;
import java.util.List;
-import java.util.Locale;
import java.util.Map;
import java.util.Set;
import java.util.function.Function;
@@ -138,7 +137,7 @@ public class TestIcebergInputFormats {
public TestIcebergInputFormats(TestInputFormat.Factory<Record> testInputFormat, String fileFormat) {
this.testInputFormat = testInputFormat;
- this.fileFormat = FileFormat.valueOf(fileFormat.toUpperCase(Locale.ENGLISH));
+ this.fileFormat = FileFormat.fromString(fileFormat);
}
@Test
diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergTestUtils.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergTestUtils.java
index 1bb12189dc1..d84fc225ad8 100644
--- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergTestUtils.java
+++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergTestUtils.java
@@ -387,7 +387,7 @@ public class HiveIcebergTestUtils {
PositionDeleteWriter<Record> posWriter = appenderFactory.newPosDeleteWriter(outputFile, fileFormat, partitionKey);
try (PositionDeleteWriter<Record> writer = posWriter) {
deletes.forEach(del -> {
- PositionDelete<Record> positionDelete = PositionDelete.create();
+ PositionDelete positionDelete = PositionDelete.create();
positionDelete.set(del.path(), del.pos(), del.row());
writer.write(positionDelete);
});
diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java
index 07d4c2a78f0..c60c3183eb0 100644
--- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java
+++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java
@@ -980,7 +980,7 @@ public class TestHiveIcebergStorageHandlerNoScan {
Assert.assertEquals(expectedIcebergProperties, icebergTable.properties());
if (Catalogs.hiveCatalog(shell.getHiveConf(), tableProperties)) {
- Assert.assertEquals(11, hmsParams.size());
+ Assert.assertEquals(13, hmsParams.size());
Assert.assertEquals("initial_val", hmsParams.get("custom_property"));
Assert.assertEquals("TRUE", hmsParams.get("EXTERNAL"));
Assert.assertEquals("true", hmsParams.get(TableProperties.ENGINE_HIVE_ENABLED));
@@ -1017,7 +1017,7 @@ public class TestHiveIcebergStorageHandlerNoScan {
.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
if (Catalogs.hiveCatalog(shell.getHiveConf(), tableProperties)) {
- Assert.assertEquals(14, hmsParams.size()); // 2 newly-added properties + previous_metadata_location prop
+ Assert.assertEquals(16, hmsParams.size()); // 2 newly-added properties + previous_metadata_location prop
Assert.assertEquals("true", hmsParams.get("new_prop_1"));
Assert.assertEquals("false", hmsParams.get("new_prop_2"));
Assert.assertEquals("new_val", hmsParams.get("custom_property"));
diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/util/TestCharSequenceSet.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/util/TestCharSequenceSet.java
new file mode 100644
index 00000000000..d208de20db0
--- /dev/null
+++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/util/TestCharSequenceSet.java
@@ -0,0 +1,53 @@
+/*
+ * 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.
+ */
+
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.iceberg.util;
+
+import java.util.Arrays;
+import java.util.Set;
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+
+public class TestCharSequenceSet {
+
+ // This test just verifies https://errorprone.info/bugpattern/CollectionUndefinedEquality
+ @Test
+ public void testSearchingInCharSequenceCollection() {
+ Set<CharSequence> set = CharSequenceSet.of(Arrays.asList("abc", new StringBuffer("def")));
+ Assertions.assertThat(set).contains("abc");
+ Assertions.assertThat(set.stream().anyMatch("def"::contains)).isTrue();
+
+ // this would fail with a normal Set<CharSequence>
+ Assertions.assertThat(set.contains("def")).isTrue();
+ }
+}
diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/util/TestLocationUtil.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/util/TestLocationUtil.java
new file mode 100644
index 00000000000..69ba60d6d10
--- /dev/null
+++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/util/TestLocationUtil.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.util;
+
+import org.apache.iceberg.AssertHelpers;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestLocationUtil {
+
+ @Test
+ public void testStripTrailingSlash() {
+ String pathWithoutTrailingSlash = "s3://bucket/db/tbl";
+ Assert.assertEquals("Should have no trailing slashes", pathWithoutTrailingSlash,
+ LocationUtil.stripTrailingSlash(pathWithoutTrailingSlash));
+
+ String pathWithSingleTrailingSlash = pathWithoutTrailingSlash + "/";
+ Assert.assertEquals("Should have no trailing slashes", pathWithoutTrailingSlash,
+ LocationUtil.stripTrailingSlash(pathWithSingleTrailingSlash));
+
+ String pathWithMultipleTrailingSlash = pathWithoutTrailingSlash + "////";
+ Assert.assertEquals("Should have no trailing slashes", pathWithoutTrailingSlash,
+ LocationUtil.stripTrailingSlash(pathWithMultipleTrailingSlash));
+
+ String pathWithOnlySlash = "////";
+ Assert.assertEquals("Should have no trailing slashes", "",
+ LocationUtil.stripTrailingSlash(pathWithOnlySlash));
+ }
+
+ @Test
+ public void testStripTrailingSlashWithInvalidPath() {
+ String [] invalidPaths = new String[] {null, ""};
+
+ for (String invalidPath : invalidPaths) {
+ AssertHelpers.assertThrows("path must be valid", IllegalArgumentException.class, "path must not be null or empty",
+ () -> LocationUtil.stripTrailingSlash(invalidPath));
+ }
+
+ }
+}
diff --git a/iceberg/iceberg-handler/src/test/queries/positive/alter_multi_part_table_to_iceberg.q b/iceberg/iceberg-handler/src/test/queries/positive/alter_multi_part_table_to_iceberg.q
index 54794d82150..7a561c800f3 100644
--- a/iceberg/iceberg-handler/src/test/queries/positive/alter_multi_part_table_to_iceberg.q
+++ b/iceberg/iceberg-handler/src/test/queries/positive/alter_multi_part_table_to_iceberg.q
@@ -2,8 +2,14 @@
--! qt:replace:/(\s+totalSize\s+)\S+(\s+)/$1#Masked#$2/
-- Mask random uuid
--! qt:replace:/(\s+uuid\s+)\S+(\s*)/$1#Masked#$2/
--- Mask random snapshot id
---! qt:replace:/(\s+current-snapshot-id\s+)\d+(\s*)/$1#SnapshotId#/
+-- Mask a random snapshot id
+--! qt:replace:/(\s+current-snapshot-id\s+)\S+(\s*)/$1#Masked#/
+-- Mask added file size
+--! qt:replace:/(\S\"added-files-size\\\":\\\")(\d+)(\\\")/$1#Masked#$3/
+-- Mask total file size
+--! qt:replace:/(\S\"total-files-size\\\":\\\")(\d+)(\\\")/$1#Masked#$3/
+-- Mask current-snapshot-timestamp-ms
+--! qt:replace:/(\s+current-snapshot-timestamp-ms\s+)\S+(\s*)/$1#Masked#$2/
set hive.vectorized.execution.enabled=false;
diff --git a/iceberg/iceberg-handler/src/test/queries/positive/alter_part_table_to_iceberg.q b/iceberg/iceberg-handler/src/test/queries/positive/alter_part_table_to_iceberg.q
index 1111a0f995d..b9bf9e62b33 100644
--- a/iceberg/iceberg-handler/src/test/queries/positive/alter_part_table_to_iceberg.q
+++ b/iceberg/iceberg-handler/src/test/queries/positive/alter_part_table_to_iceberg.q
@@ -2,8 +2,14 @@
--! qt:replace:/(\s+totalSize\s+)\S+(\s+)/$1#Masked#$2/
-- Mask random uuid
--! qt:replace:/(\s+uuid\s+)\S+/$1#Masked#/
--- Mask random snapshot id
---! qt:replace:/(\s+current-snapshot-id\s+)\d+(\s*)/$1#SnapshotId#/
+-- Mask a random snapshot id
+--! qt:replace:/(\s+current-snapshot-id\s+)\S+(\s*)/$1#Masked#/
+-- Mask added file size
+--! qt:replace:/(\S\"added-files-size\\\":\\\")(\d+)(\\\")/$1#Masked#$3/
+-- Mask total file size
+--! qt:replace:/(\S\"total-files-size\\\":\\\")(\d+)(\\\")/$1#Masked#$3/
+-- Mask current-snapshot-timestamp-ms
+--! qt:replace:/(\s+current-snapshot-timestamp-ms\s+)\S+(\s*)/$1#Masked#$2/
set hive.vectorized.execution.enabled=false;
diff --git a/iceberg/iceberg-handler/src/test/queries/positive/alter_table_to_iceberg.q b/iceberg/iceberg-handler/src/test/queries/positive/alter_table_to_iceberg.q
index b0720c8b92d..775351f647c 100644
--- a/iceberg/iceberg-handler/src/test/queries/positive/alter_table_to_iceberg.q
+++ b/iceberg/iceberg-handler/src/test/queries/positive/alter_table_to_iceberg.q
@@ -2,8 +2,14 @@
--! qt:replace:/(\s+totalSize\s+)\S+(\s+)/$1#Masked#$2/
-- Mask random uuid
--! qt:replace:/(\s+uuid\s+)\S+(\s*)/$1#Masked#$2/
--- Mask random snapshot id
---! qt:replace:/(\s+current-snapshot-id\s+)\d+(\s*)/$1#SnapshotId#/
+-- Mask a random snapshot id
+--! qt:replace:/(\s+current-snapshot-id\s+)\S+(\s*)/$1#Masked#/
+-- Mask added file size
+--! qt:replace:/(\S\"added-files-size\\\":\\\")(\d+)(\\\")/$1#Masked#$3/
+-- Mask total file size
+--! qt:replace:/(\S\"total-files-size\\\":\\\")(\d+)(\\\")/$1#Masked#$3/
+-- Mask current-snapshot-timestamp-ms
+--! qt:replace:/(\s+current-snapshot-timestamp-ms\s+)\S+(\s*)/$1#Masked#$2/
set hive.vectorized.execution.enabled=false;
diff --git a/iceberg/iceberg-handler/src/test/queries/positive/ctas_iceberg_partitioned_orc.q b/iceberg/iceberg-handler/src/test/queries/positive/ctas_iceberg_partitioned_orc.q
index 4328a938b17..2cb495a6ef2 100644
--- a/iceberg/iceberg-handler/src/test/queries/positive/ctas_iceberg_partitioned_orc.q
+++ b/iceberg/iceberg-handler/src/test/queries/positive/ctas_iceberg_partitioned_orc.q
@@ -1,7 +1,13 @@
set hive.query.lifetime.hooks=org.apache.iceberg.mr.hive.HiveIcebergQueryLifeTimeHook;
--! qt:replace:/(\s+uuid\s+)\S+(\s*)/$1#Masked#$2/
--- Mask random snapshot id
---! qt:replace:/(\s+current-snapshot-id\s+)\d+(\s*)/$1#SnapshotId#/
+-- Mask a random snapshot id
+--! qt:replace:/(\s+current-snapshot-id\s+)\S+(\s*)/$1#Masked#/
+-- Mask added file size
+--! qt:replace:/(\S\"added-files-size\\\":\\\")(\d+)(\\\")/$1#Masked#$3/
+-- Mask total file size
+--! qt:replace:/(\S\"total-files-size\\\":\\\")(\d+)(\\\")/$1#Masked#$3/
+-- Mask current-snapshot-timestamp-ms
+--! qt:replace:/(\s+current-snapshot-timestamp-ms\s+)\S+(\s*)/$1#Masked#$2/
set hive.explain.user=false;
diff --git a/iceberg/iceberg-handler/src/test/queries/positive/iceberg_v2_deletes.q b/iceberg/iceberg-handler/src/test/queries/positive/iceberg_v2_deletes.q
index 0e0e6a9d8d8..9000c9bbc1e 100644
--- a/iceberg/iceberg-handler/src/test/queries/positive/iceberg_v2_deletes.q
+++ b/iceberg/iceberg-handler/src/test/queries/positive/iceberg_v2_deletes.q
@@ -1,7 +1,13 @@
-- Mask random uuid
--! qt:replace:/(\s+'uuid'=')\S+('\s*)/$1#Masked#$2/
--- Mask random snapshot id
---! qt:replace:/('current-snapshot-id'=')\d+/$1#SnapshotId#/
+-- Mask a random snapshot id
+--! qt:replace:/(\s\'current-snapshot-id\'=\')(\d+)(\')/$1#Masked#$3/
+-- Mask added file size
+--! qt:replace:/(\S+\"added-files-size\":\")(\d+)(\")/$1#Masked#$3/
+-- Mask total file size
+--! qt:replace:/(\S+\"total-files-size\":\")(\d+)(\")/$1#Masked#$3/
+-- Mask current-snapshot-timestamp-ms
+--! qt:replace:/(\s\'current-snapshot-timestamp-ms\'=\')(\d+)(\')/$1#Masked#$3/
-- create an unpartitioned table with skip delete data set to false
create table ice01 (id int) Stored by Iceberg stored as ORC
diff --git a/iceberg/iceberg-handler/src/test/queries/positive/mv_iceberg_partitioned_orc.q b/iceberg/iceberg-handler/src/test/queries/positive/mv_iceberg_partitioned_orc.q
index 44a5561f926..ad01a992bff 100644
--- a/iceberg/iceberg-handler/src/test/queries/positive/mv_iceberg_partitioned_orc.q
+++ b/iceberg/iceberg-handler/src/test/queries/positive/mv_iceberg_partitioned_orc.q
@@ -2,6 +2,12 @@
--! qt:replace:/(\s+uuid\s+)\S+(\s*)/$1#Masked#$2/
-- Mask random snapshot id
--! qt:replace:/(\s+current-snapshot-id\s+)\d+(\s*)/$1#SnapshotId#/
+-- Mask added file size
+--! qt:replace:/(\S\"added-files-size\\\":\\\")(\d+)(\\\")/$1#Masked#$3/
+-- Mask total file size
+--! qt:replace:/(\S\"total-files-size\\\":\\\")(\d+)(\\\")/$1#Masked#$3/
+-- Mask current-snapshot-timestamp-ms
+--! qt:replace:/(\s+current-snapshot-timestamp-ms\s+)\S+(\s*)/$1#Masked#$2/
-- Mask the totalSize value as it can change at file format library update
--! qt:replace:/(\s+totalSize\s+)\S+(\s+)/$1#Masked#$2/
-- SORT_QUERY_RESULTS
diff --git a/iceberg/iceberg-handler/src/test/queries/positive/mv_iceberg_partitioned_orc2.q b/iceberg/iceberg-handler/src/test/queries/positive/mv_iceberg_partitioned_orc2.q
index 3cb8c04f1d0..7d0a6efaff1 100644
--- a/iceberg/iceberg-handler/src/test/queries/positive/mv_iceberg_partitioned_orc2.q
+++ b/iceberg/iceberg-handler/src/test/queries/positive/mv_iceberg_partitioned_orc2.q
@@ -1,6 +1,12 @@
-- MV data is stored by partitioned iceberg with partition spec
--! qt:replace:/(\s+uuid\s+)\S+(\s*)/$1#Masked#$2/
--! qt:replace:/(\s+current-snapshot-id\s+)\d+(\s*)/$1#SnapshotId#/
+-- Mask added file size
+--! qt:replace:/(\S\"added-files-size\\\":\\\")(\d+)(\\\")/$1#Masked#$3/
+-- Mask total file size
+--! qt:replace:/(\S\"total-files-size\\\":\\\")(\d+)(\\\")/$1#Masked#$3/
+-- Mask current-snapshot-timestamp-ms
+--! qt:replace:/(\s+current-snapshot-timestamp-ms\s+)\S+(\s*)/$1#Masked#$2/
-- Mask the totalSize value as it can change at file format library update
--! qt:replace:/(\s+totalSize\s+)\S+(\s+)/$1#Masked#$2/
-- SORT_QUERY_RESULTS
diff --git a/iceberg/iceberg-handler/src/test/queries/positive/show_create_iceberg_table.q b/iceberg/iceberg-handler/src/test/queries/positive/show_create_iceberg_table.q
index 1243220ae8e..ed1c63b639a 100644
--- a/iceberg/iceberg-handler/src/test/queries/positive/show_create_iceberg_table.q
+++ b/iceberg/iceberg-handler/src/test/queries/positive/show_create_iceberg_table.q
@@ -2,6 +2,8 @@
--! qt:replace:/(\s+'uuid'=')\S+('\s*)/$1#Masked#$2/
-- Mask random snapshot id
--! qt:replace:/('current-snapshot-id'=')\d+/$1#SnapshotId#/
+-- Mask current-snapshot-timestamp-ms
+--! qt:replace:/('current-snapshot-timestamp-ms'=')\d+/$1#Masked#/
DROP TABLE IF EXISTS ice_t;
CREATE EXTERNAL TABLE ice_t (i int, s string, ts timestamp, d date) STORED BY ICEBERG;
diff --git a/iceberg/iceberg-handler/src/test/queries/positive/truncate_force_iceberg_table.q b/iceberg/iceberg-handler/src/test/queries/positive/truncate_force_iceberg_table.q
index 08d8a9a811a..47526271d5f 100644
--- a/iceberg/iceberg-handler/src/test/queries/positive/truncate_force_iceberg_table.q
+++ b/iceberg/iceberg-handler/src/test/queries/positive/truncate_force_iceberg_table.q
@@ -3,8 +3,16 @@
--! qt:replace:/(\s+totalSize\s+)\S+(\s+)/$1#Masked#$2/
-- Mask random uuid
--! qt:replace:/(\s+uuid\s+)\S+(\s*)/$1#Masked#$2/
--- Mask random snapshot id
---! qt:replace:/(\s+current-snapshot-id\s+)\d+(\s*)/$1#SnapshotId#/
+-- Mask a random snapshot id
+--! qt:replace:/(\s+current-snapshot-id\s+)\S+(\s*)/$1#Masked#/
+-- Mask added file size
+--! qt:replace:/(\S\"added-files-size\\\":\\\")(\d+)(\\\")/$1#Masked#$3/
+-- Mask total file size
+--! qt:replace:/(\S\"total-files-size\\\":\\\")(\d+)(\\\")/$1#Masked#$3/
+-- Mask current-snapshot-timestamp-ms
+--! qt:replace:/(\s+current-snapshot-timestamp-ms\s+)\S+(\s*)/$1#Masked#$2/
+-- Mask removed file size
+--! qt:replace:/(\S\"removed-files-size\\\":\\\")(\d+)(\\\")/$1#Masked#$3/
set hive.vectorized.execution.enabled=false;
diff --git a/iceberg/iceberg-handler/src/test/queries/positive/truncate_iceberg_table.q b/iceberg/iceberg-handler/src/test/queries/positive/truncate_iceberg_table.q
index ef26f188fe9..775a06c3483 100644
--- a/iceberg/iceberg-handler/src/test/queries/positive/truncate_iceberg_table.q
+++ b/iceberg/iceberg-handler/src/test/queries/positive/truncate_iceberg_table.q
@@ -3,8 +3,16 @@
--! qt:replace:/(\s+totalSize\s+)\S+(\s+)/$1#Masked#$2/
-- Mask random uuid
--! qt:replace:/(\s+uuid\s+)\S+(\s*)/$1#Masked#$2/
--- Mask random snapshot id
---! qt:replace:/(\s+current-snapshot-id\s+)\d+(\s*)/$1#SnapshotId#/
+-- Mask a random snapshot id
+--! qt:replace:/(\s+current-snapshot-id\s+)\S+(\s*)/$1#Masked#/
+-- Mask added file size
+--! qt:replace:/(\S\"added-files-size\\\":\\\")(\d+)(\\\")/$1#Masked#$3/
+-- Mask total file size
+--! qt:replace:/(\S\"total-files-size\\\":\\\")(\d+)(\\\")/$1#Masked#$3/
+-- Mask current-snapshot-timestamp-ms
+--! qt:replace:/(\s+current-snapshot-timestamp-ms\s+)\S+(\s*)/$1#Masked#$2/
+-- Mask removed file size
+--! qt:replace:/(\S\"removed-files-size\\\":\\\")(\d+)(\\\")/$1#Masked#$3/
set hive.vectorized.execution.enabled=false;
diff --git a/iceberg/iceberg-handler/src/test/queries/positive/truncate_partitioned_iceberg_table.q b/iceberg/iceberg-handler/src/test/queries/positive/truncate_partitioned_iceberg_table.q
index 96b07140856..ad6b3773510 100644
--- a/iceberg/iceberg-handler/src/test/queries/positive/truncate_partitioned_iceberg_table.q
+++ b/iceberg/iceberg-handler/src/test/queries/positive/truncate_partitioned_iceberg_table.q
@@ -3,8 +3,16 @@
--! qt:replace:/(\s+totalSize\s+)\S+(\s+)/$1#Masked#$2/
-- Mask random uuid
--! qt:replace:/(\s+uuid\s+)\S+(\s*)/$1#Masked#$2/
--- Mask random snapshot id
---! qt:replace:/(\s+current-snapshot-id\s+)\d+(\s*)/$1#SnapshotId#/
+-- Mask a random snapshot id
+--! qt:replace:/(\s+current-snapshot-id\s+)\S+(\s*)/$1#Masked#/
+-- Mask added file size
+--! qt:replace:/(\S\"added-files-size\\\":\\\")(\d+)(\\\")/$1#Masked#$3/
+-- Mask total file size
+--! qt:replace:/(\S\"total-files-size\\\":\\\")(\d+)(\\\")/$1#Masked#$3/
+-- Mask current-snapshot-timestamp-ms
+--! qt:replace:/(\s+current-snapshot-timestamp-ms\s+)\S+(\s*)/$1#Masked#$2/
+-- Mask removed file size
+--! qt:replace:/(\S\"removed-files-size\\\":\\\")(\d+)(\\\")/$1#Masked#$3/
set hive.vectorized.execution.enabled=false;
diff --git a/iceberg/iceberg-handler/src/test/results/positive/alter_multi_part_table_to_iceberg.q.out b/iceberg/iceberg-handler/src/test/results/positive/alter_multi_part_table_to_iceberg.q.out
index 0694e4e732d..5d876e2adc9 100644
--- a/iceberg/iceberg-handler/src/test/results/positive/alter_multi_part_table_to_iceberg.q.out
+++ b/iceberg/iceberg-handler/src/test/results/positive/alter_multi_part_table_to_iceberg.q.out
@@ -185,7 +185,11 @@ Table Parameters:
EXTERNAL TRUE
MIGRATED_TO_ICEBERG true
bucketing_version 2
- current-snapshot-id #SnapshotId#
+ current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"a\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"b\",\"required\":false,\"type\":\"string\"},{\"id\":3,\"name\":\"c\",\"required\":false,\"type\":\"string\"}]}
+ current-snapshot-id #Masked#
+ current-snapshot-summary {\"added-data-files\":\"7\",\"added-records\":\"15\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"7\",\"total-records\":\"15\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"7\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"}
+ current-snapshot-timestamp-ms #Masked#
+ default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"b\",\"transform\":\"identity\",\"source-id\":2,\"field-id\":1000},{\"name\":\"c\",\"transform\":\"identity\",\"source-id\":3,\"field-id\":1001}]}
engine.hive.enabled true
iceberg.orc.files.only true
#### A masked pattern was here ####
@@ -203,6 +207,7 @@ Table Parameters:
\"field-id\" : 3,
\"names\" : [ \"c\" ]
} ]
+ snapshot-count 1
storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler
table_type ICEBERG
totalSize #Masked#
@@ -434,7 +439,11 @@ Table Parameters:
EXTERNAL TRUE
MIGRATED_TO_ICEBERG true
bucketing_version 2
- current-snapshot-id #SnapshotId#
+ current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"a\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"b\",\"required\":false,\"type\":\"string\"},{\"id\":3,\"name\":\"c\",\"required\":false,\"type\":\"string\"}]}
+ current-snapshot-id #Masked#
+ current-snapshot-summary {\"added-data-files\":\"7\",\"added-records\":\"15\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"7\",\"total-records\":\"15\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"7\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"}
+ current-snapshot-timestamp-ms #Masked#
+ default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"b\",\"transform\":\"identity\",\"source-id\":2,\"field-id\":1000},{\"name\":\"c\",\"transform\":\"identity\",\"source-id\":3,\"field-id\":1001}]}
engine.hive.enabled true
iceberg.orc.files.only false
#### A masked pattern was here ####
@@ -452,6 +461,7 @@ Table Parameters:
\"field-id\" : 3,
\"names\" : [ \"c\" ]
} ]
+ snapshot-count 1
storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler
table_type ICEBERG
totalSize #Masked#
@@ -683,7 +693,11 @@ Table Parameters:
EXTERNAL TRUE
MIGRATED_TO_ICEBERG true
bucketing_version 2
- current-snapshot-id #SnapshotId#
+ current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"a\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"b\",\"required\":false,\"type\":\"string\"},{\"id\":3,\"name\":\"c\",\"required\":false,\"type\":\"string\"}]}
+ current-snapshot-id #Masked#
+ current-snapshot-summary {\"added-data-files\":\"7\",\"added-records\":\"15\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"7\",\"total-records\":\"15\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"7\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"}
+ current-snapshot-timestamp-ms #Masked#
+ default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"b\",\"transform\":\"identity\",\"source-id\":2,\"field-id\":1000},{\"name\":\"c\",\"transform\":\"identity\",\"source-id\":3,\"field-id\":1001}]}
engine.hive.enabled true
iceberg.orc.files.only false
#### A masked pattern was here ####
@@ -701,6 +715,7 @@ Table Parameters:
\"field-id\" : 3,
\"names\" : [ \"c\" ]
} ]
+ snapshot-count 1
storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler
table_type ICEBERG
totalSize #Masked#
diff --git a/iceberg/iceberg-handler/src/test/results/positive/alter_part_table_to_iceberg.q.out b/iceberg/iceberg-handler/src/test/results/positive/alter_part_table_to_iceberg.q.out
index aa09c38d82a..5c8b9ee857f 100644
--- a/iceberg/iceberg-handler/src/test/results/positive/alter_part_table_to_iceberg.q.out
+++ b/iceberg/iceberg-handler/src/test/results/positive/alter_part_table_to_iceberg.q.out
@@ -143,7 +143,11 @@ Table Parameters:
EXTERNAL TRUE
MIGRATED_TO_ICEBERG true
bucketing_version 2
- current-snapshot-id #SnapshotId#
+ current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"a\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"b\",\"required\":false,\"type\":\"string\"}]}
+ current-snapshot-id #Masked#
+ current-snapshot-summary {\"added-data-files\":\"4\",\"added-records\":\"9\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"4\",\"total-records\":\"9\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"4\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"}
+ current-snapshot-timestamp-ms #Masked#
+ default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"b\",\"transform\":\"identity\",\"source-id\":2,\"field-id\":1000}]}
engine.hive.enabled true
iceberg.orc.files.only true
#### A masked pattern was here ####
@@ -158,6 +162,7 @@ Table Parameters:
\"field-id\" : 2,
\"names\" : [ \"b\" ]
} ]
+ snapshot-count 1
storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler
table_type ICEBERG
totalSize #Masked#
@@ -341,7 +346,11 @@ Table Parameters:
EXTERNAL TRUE
MIGRATED_TO_ICEBERG true
bucketing_version 2
- current-snapshot-id #SnapshotId#
+ current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"a\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"b\",\"required\":false,\"type\":\"string\"}]}
+ current-snapshot-id #Masked#
+ current-snapshot-summary {\"added-data-files\":\"4\",\"added-records\":\"9\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"4\",\"total-records\":\"9\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"4\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"}
+ current-snapshot-timestamp-ms #Masked#
+ default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"b\",\"transform\":\"identity\",\"source-id\":2,\"field-id\":1000}]}
engine.hive.enabled true
iceberg.orc.files.only false
#### A masked pattern was here ####
@@ -356,6 +365,7 @@ Table Parameters:
\"field-id\" : 2,
\"names\" : [ \"b\" ]
} ]
+ snapshot-count 1
storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler
table_type ICEBERG
totalSize #Masked#
@@ -539,7 +549,11 @@ Table Parameters:
EXTERNAL TRUE
MIGRATED_TO_ICEBERG true
bucketing_version 2
- current-snapshot-id #SnapshotId#
+ current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"a\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"b\",\"required\":false,\"type\":\"string\"}]}
+ current-snapshot-id #Masked#
+ current-snapshot-summary {\"added-data-files\":\"4\",\"added-records\":\"9\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"4\",\"total-records\":\"9\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"4\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"}
+ current-snapshot-timestamp-ms #Masked#
+ default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"b\",\"transform\":\"identity\",\"source-id\":2,\"field-id\":1000}]}
engine.hive.enabled true
iceberg.orc.files.only false
#### A masked pattern was here ####
@@ -554,6 +568,7 @@ Table Parameters:
\"field-id\" : 2,
\"names\" : [ \"b\" ]
} ]
+ snapshot-count 1
storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler
table_type ICEBERG
totalSize #Masked#
diff --git a/iceberg/iceberg-handler/src/test/results/positive/alter_table_to_iceberg.q.out b/iceberg/iceberg-handler/src/test/results/positive/alter_table_to_iceberg.q.out
index 6f3009324fb..96882292e58 100644
--- a/iceberg/iceberg-handler/src/test/results/positive/alter_table_to_iceberg.q.out
+++ b/iceberg/iceberg-handler/src/test/results/positive/alter_table_to_iceberg.q.out
@@ -98,7 +98,10 @@ Table Parameters:
EXTERNAL TRUE
MIGRATED_TO_ICEBERG true
bucketing_version 2
- current-snapshot-id #SnapshotId#
+ current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"a\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"b\",\"required\":false,\"type\":\"string\"}]}
+ current-snapshot-id #Masked#
+ current-snapshot-summary {\"added-data-files\":\"1\",\"added-records\":\"5\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"5\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"1\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"}
+ current-snapshot-timestamp-ms #Masked#
engine.hive.enabled true
iceberg.orc.files.only true
#### A masked pattern was here ####
@@ -114,6 +117,7 @@ Table Parameters:
\"field-id\" : 2,
\"names\" : [ \"b\" ]
} ]
+ snapshot-count 1
storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler
table_type ICEBERG
totalSize #Masked#
@@ -248,7 +252,10 @@ Table Parameters:
EXTERNAL TRUE
MIGRATED_TO_ICEBERG true
bucketing_version 2
- current-snapshot-id #SnapshotId#
+ current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"a\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"b\",\"required\":false,\"type\":\"string\"}]}
+ current-snapshot-id #Masked#
+ current-snapshot-summary {\"added-data-files\":\"1\",\"added-records\":\"5\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"5\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"1\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"}
+ current-snapshot-timestamp-ms #Masked#
engine.hive.enabled true
iceberg.orc.files.only false
#### A masked pattern was here ####
@@ -264,6 +271,7 @@ Table Parameters:
\"field-id\" : 2,
\"names\" : [ \"b\" ]
} ]
+ snapshot-count 1
storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler
table_type ICEBERG
totalSize #Masked#
@@ -398,7 +406,10 @@ Table Parameters:
EXTERNAL TRUE
MIGRATED_TO_ICEBERG true
bucketing_version 2
- current-snapshot-id #SnapshotId#
+ current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"a\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"b\",\"required\":false,\"type\":\"string\"}]}
+ current-snapshot-id #Masked#
+ current-snapshot-summary {\"added-data-files\":\"1\",\"added-records\":\"5\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"5\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"1\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"}
+ current-snapshot-timestamp-ms #Masked#
engine.hive.enabled true
iceberg.orc.files.only false
#### A masked pattern was here ####
@@ -414,6 +425,7 @@ Table Parameters:
\"field-id\" : 2,
\"names\" : [ \"b\" ]
} ]
+ snapshot-count 1
storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler
table_type ICEBERG
totalSize #Masked#
diff --git a/iceberg/iceberg-handler/src/test/results/positive/create_iceberg_table.q.out b/iceberg/iceberg-handler/src/test/results/positive/create_iceberg_table.q.out
index d8952502e64..ba61c9535a6 100644
--- a/iceberg/iceberg-handler/src/test/results/positive/create_iceberg_table.q.out
+++ b/iceberg/iceberg-handler/src/test/results/positive/create_iceberg_table.q.out
@@ -28,6 +28,7 @@ Table Parameters:
COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"d\":\"true\",\"i\":\"true\",\"s\":\"true\",\"ts\":\"true\"}}
EXTERNAL TRUE
bucketing_version 2
+ current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"i\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"s\",\"required\":false,\"type\":\"string\"},{\"id\":3,\"name\":\"ts\",\"required\":false,\"type\":\"timestamp\"},{\"id\":4,\"name\":\"d\",\"required\":false,\"type\":\"date\"}]}
engine.hive.enabled true
iceberg.orc.files.only false
metadata_location hdfs://### HDFS PATH ###
@@ -35,6 +36,7 @@ Table Parameters:
numRows 0
rawDataSize 0
serialization.format 1
+ snapshot-count 0
storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler
table_type ICEBERG
totalSize #Masked#
diff --git a/iceberg/iceberg-handler/src/test/results/positive/create_iceberg_table_stored_as_fileformat.q.out b/iceberg/iceberg-handler/src/test/results/positive/create_iceberg_table_stored_as_fileformat.q.out
index 82a76919e09..4c290be51ea 100644
--- a/iceberg/iceberg-handler/src/test/results/positive/create_iceberg_table_stored_as_fileformat.q.out
+++ b/iceberg/iceberg-handler/src/test/results/positive/create_iceberg_table_stored_as_fileformat.q.out
@@ -32,6 +32,7 @@ Table Parameters:
COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"d\":\"true\",\"i\":\"true\",\"s\":\"true\",\"ts\":\"true\"}}
EXTERNAL TRUE
bucketing_version 2
+ current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"i\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"s\",\"required\":false,\"type\":\"string\"},{\"id\":3,\"name\":\"ts\",\"required\":false,\"type\":\"timestamp\"},{\"id\":4,\"name\":\"d\",\"required\":false,\"type\":\"date\"}]}
engine.hive.enabled true
iceberg.orc.files.only true
metadata_location hdfs://### HDFS PATH ###
@@ -39,6 +40,7 @@ Table Parameters:
numRows 0
rawDataSize 0
serialization.format 1
+ snapshot-count 0
storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler
table_type ICEBERG
totalSize #Masked#
@@ -94,6 +96,7 @@ Table Parameters:
COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"d\":\"true\",\"i\":\"true\",\"s\":\"true\",\"ts\":\"true\"}}
EXTERNAL TRUE
bucketing_version 2
+ current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"i\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"s\",\"required\":false,\"type\":\"string\"},{\"id\":3,\"name\":\"ts\",\"required\":false,\"type\":\"timestamp\"},{\"id\":4,\"name\":\"d\",\"required\":false,\"type\":\"date\"}]}
engine.hive.enabled true
iceberg.orc.files.only false
metadata_location hdfs://### HDFS PATH ###
@@ -101,6 +104,7 @@ Table Parameters:
numRows 0
rawDataSize 0
serialization.format 1
+ snapshot-count 0
storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler
table_type ICEBERG
totalSize #Masked#
@@ -156,6 +160,7 @@ Table Parameters:
COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"d\":\"true\",\"i\":\"true\",\"s\":\"true\",\"ts\":\"true\"}}
EXTERNAL TRUE
bucketing_version 2
+ current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"i\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"s\",\"required\":false,\"type\":\"string\"},{\"id\":3,\"name\":\"ts\",\"required\":false,\"type\":\"timestamp\"},{\"id\":4,\"name\":\"d\",\"required\":false,\"type\":\"date\"}]}
engine.hive.enabled true
iceberg.orc.files.only false
metadata_location hdfs://### HDFS PATH ###
@@ -163,6 +168,7 @@ Table Parameters:
numRows 0
rawDataSize 0
serialization.format 1
+ snapshot-count 0
storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler
table_type ICEBERG
totalSize #Masked#
@@ -218,6 +224,7 @@ Table Parameters:
COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"d\":\"true\",\"i\":\"true\",\"s\":\"true\",\"ts\":\"true\"}}
EXTERNAL TRUE
bucketing_version 2
+ current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"i\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"s\",\"required\":false,\"type\":\"string\"},{\"id\":3,\"name\":\"ts\",\"required\":false,\"type\":\"timestamp\"},{\"id\":4,\"name\":\"d\",\"required\":false,\"type\":\"date\"}]}
engine.hive.enabled true
iceberg.orc.files.only false
metadata_location hdfs://### HDFS PATH ###
@@ -225,6 +232,7 @@ Table Parameters:
numRows 0
rawDataSize 0
serialization.format 1
+ snapshot-count 0
storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler
table_type ICEBERG
totalSize #Masked#
@@ -276,6 +284,7 @@ Table Parameters:
COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"d\":\"true\",\"i\":\"true\",\"s\":\"true\",\"ts\":\"true\"}}
EXTERNAL TRUE
bucketing_version 2
+ current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"i\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"s\",\"required\":false,\"type\":\"string\"},{\"id\":3,\"name\":\"ts\",\"required\":false,\"type\":\"timestamp\"},{\"id\":4,\"name\":\"d\",\"required\":false,\"type\":\"date\"}]}
dummy dummy_value
engine.hive.enabled true
iceberg.orc.files.only true
@@ -284,6 +293,7 @@ Table Parameters:
numRows 0
rawDataSize 0
serialization.format 1
+ snapshot-count 0
storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler
table_type ICEBERG
totalSize #Masked#
diff --git a/iceberg/iceberg-handler/src/test/results/positive/create_iceberg_table_stored_by_iceberg.q.out b/iceberg/iceberg-handler/src/test/results/positive/create_iceberg_table_stored_by_iceberg.q.out
index d8952502e64..ba61c9535a6 100644
--- a/iceberg/iceberg-handler/src/test/results/positive/create_iceberg_table_stored_by_iceberg.q.out
+++ b/iceberg/iceberg-handler/src/test/results/positive/create_iceberg_table_stored_by_iceberg.q.out
@@ -28,6 +28,7 @@ Table Parameters:
COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"d\":\"true\",\"i\":\"true\",\"s\":\"true\",\"ts\":\"true\"}}
EXTERNAL TRUE
bucketing_version 2
+ current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"i\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"s\",\"required\":false,\"type\":\"string\"},{\"id\":3,\"name\":\"ts\",\"required\":false,\"type\":\"timestamp\"},{\"id\":4,\"name\":\"d\",\"required\":false,\"type\":\"date\"}]}
engine.hive.enabled true
iceberg.orc.files.only false
metadata_location hdfs://### HDFS PATH ###
@@ -35,6 +36,7 @@ Table Parameters:
numRows 0
rawDataSize 0
serialization.format 1
+ snapshot-count 0
storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler
table_type ICEBERG
totalSize #Masked#
diff --git a/iceberg/iceberg-handler/src/test/results/positive/create_iceberg_table_stored_by_iceberg_with_serdeproperties.q.out b/iceberg/iceberg-handler/src/test/results/positive/create_iceberg_table_stored_by_iceberg_with_serdeproperties.q.out
index 610eb67907f..a6114e7750b 100644
--- a/iceberg/iceberg-handler/src/test/results/positive/create_iceberg_table_stored_by_iceberg_with_serdeproperties.q.out
+++ b/iceberg/iceberg-handler/src/test/results/positive/create_iceberg_table_stored_by_iceberg_with_serdeproperties.q.out
@@ -28,6 +28,7 @@ Table Parameters:
COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"d\":\"true\",\"i\":\"true\",\"s\":\"true\",\"ts\":\"true\"}}
EXTERNAL TRUE
bucketing_version 2
+ current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"i\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"s\",\"required\":false,\"type\":\"string\"},{\"id\":3,\"name\":\"ts\",\"required\":false,\"type\":\"timestamp\"},{\"id\":4,\"name\":\"d\",\"required\":false,\"type\":\"date\"}]}
engine.hive.enabled true
iceberg.orc.files.only true
metadata_location hdfs://### HDFS PATH ###
@@ -35,6 +36,7 @@ Table Parameters:
numRows 0
rawDataSize 0
serialization.format 1
+ snapshot-count 0
storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler
table_type ICEBERG
totalSize #Masked#
diff --git a/iceberg/iceberg-handler/src/test/results/positive/ctas_iceberg_partitioned_orc.q.out b/iceberg/iceberg-handler/src/test/results/positive/ctas_iceberg_partitioned_orc.q.out
index ea415853746..5f2c545d3af 100644
--- a/iceberg/iceberg-handler/src/test/results/positive/ctas_iceberg_partitioned_orc.q.out
+++ b/iceberg/iceberg-handler/src/test/results/positive/ctas_iceberg_partitioned_orc.q.out
@@ -295,7 +295,11 @@ Table Parameters:
COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"a\":\"true\",\"b\":\"true\",\"c\":\"true\"}}
EXTERNAL TRUE
bucketing_version 2
- current-snapshot-id #SnapshotId#
+ current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"a\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"b\",\"required\":false,\"type\":\"string\"},{\"id\":3,\"name\":\"c\",\"required\":false,\"type\":\"int\"}]}
+ current-snapshot-id #Masked#
+ current-snapshot-summary {\"added-data-files\":\"2\",\"added-records\":\"2\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"2\",\"total-records\":\"2\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"2\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"}
+ current-snapshot-timestamp-ms #Masked#
+ default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"a_bucket\",\"transform\":\"bucket[16]\",\"source-id\":1,\"field-id\":1000},{\"name\":\"b_trunc\",\"transform\":\"truncate[3]\",\"source-id\":2,\"field-id\":1001}]}
engine.hive.enabled true
format-version 2
iceberg.orc.files.only true
@@ -304,6 +308,7 @@ Table Parameters:
numRows 2
previous_metadata_location hdfs://### HDFS PATH ###
serialization.format 1
+ snapshot-count 1
storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler
table_type ICEBERG
totalSize 812
diff --git a/iceberg/iceberg-handler/src/test/results/positive/ctlt_iceberg.q.out b/iceberg/iceberg-handler/src/test/results/positive/ctlt_iceberg.q.out
index 389f59df599..d806a28de73 100644
--- a/iceberg/iceberg-handler/src/test/results/positive/ctlt_iceberg.q.out
+++ b/iceberg/iceberg-handler/src/test/results/positive/ctlt_iceberg.q.out
@@ -41,10 +41,12 @@ TBLPROPERTIES (
'TRANSLATED_TO_EXTERNAL'='TRUE',
'bucketing_version'='2',
'created_with_ctlt'='true',
+ 'current-schema'='{"type":"struct","schema-id":0,"fields":[{"id":1,"name":"a","required":false,"type":"int"}]}',
'engine.hive.enabled'='true',
'format-version'='2',
'iceberg.orc.files.only'='false',
'metadata_location'='hdfs://### HDFS PATH ###',
+ 'snapshot-count'='0',
'table_type'='ICEBERG',
#### A masked pattern was here ####
'uuid'='#Masked#',
@@ -123,10 +125,13 @@ LOCATION
'hdfs://### HDFS PATH ###'
TBLPROPERTIES (
'bucketing_version'='2',
+ 'current-schema'='{"type":"struct","schema-id":0,"fields":[{"id":1,"name":"id","required":false,"type":"int"},{"id":2,"name":"company","required":false,"type":"string"}]}',
+ 'default-partition-spec'='{"spec-id":0,"fields":[{"name":"company","transform":"identity","source-id":2,"field-id":1000}]}',
'engine.hive.enabled'='true',
'iceberg.orc.files.only'='false',
'metadata_location'='hdfs://### HDFS PATH ###',
'serialization.format'='1',
+ 'snapshot-count'='0',
'table_type'='ICEBERG',
#### A masked pattern was here ####
'uuid'='#Masked#')
@@ -160,9 +165,12 @@ TBLPROPERTIES (
'TRANSLATED_TO_EXTERNAL'='TRUE',
'bucketing_version'='2',
'created_with_ctlt'='true',
+ 'current-schema'='{"type":"struct","schema-id":0,"fields":[{"id":1,"name":"id","required":false,"type":"int"},{"id":2,"name":"company","required":false,"type":"string"}]}',
+ 'default-partition-spec'='{"spec-id":0,"fields":[{"name":"company","transform":"identity","source-id":2,"field-id":1000}]}',
'engine.hive.enabled'='true',
'iceberg.orc.files.only'='false',
'metadata_location'='hdfs://### HDFS PATH ###',
+ 'snapshot-count'='0',
'table_type'='ICEBERG',
#### A masked pattern was here ####
'uuid'='#Masked#')
@@ -225,9 +233,12 @@ TBLPROPERTIES (
'TRANSLATED_TO_EXTERNAL'='TRUE',
'bucketing_version'='2',
'created_with_ctlt'='true',
+ 'current-schema'='{"type":"struct","schema-id":0,"fields":[{"id":1,"name":"id","required":false,"type":"int"},{"id":2,"name":"company","required":false,"type":"string"}]}',
+ 'default-partition-spec'='{"spec-id":0,"fields":[{"name":"company","transform":"identity","source-id":2,"field-id":1000}]}',
'engine.hive.enabled'='true',
'iceberg.orc.files.only'='false',
'metadata_location'='hdfs://### HDFS PATH ###',
+ 'snapshot-count'='0',
'table_type'='ICEBERG',
#### A masked pattern was here ####
'uuid'='#Masked#')
diff --git a/iceberg/iceberg-handler/src/test/results/positive/describe_iceberg_table.q.out b/iceberg/iceberg-handler/src/test/results/positive/describe_iceberg_table.q.out
index c1251f81f62..92b9e57aee2 100644
--- a/iceberg/iceberg-handler/src/test/results/positive/describe_iceberg_table.q.out
+++ b/iceberg/iceberg-handler/src/test/results/positive/describe_iceberg_table.q.out
@@ -68,6 +68,7 @@ Table Parameters:
COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"d\":\"true\",\"i\":\"true\",\"s\":\"true\",\"ts\":\"true\"}}
EXTERNAL TRUE
bucketing_version 2
+ current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"i\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"s\",\"required\":false,\"type\":\"string\"},{\"id\":3,\"name\":\"ts\",\"required\":false,\"type\":\"timestamp\"},{\"id\":4,\"name\":\"d\",\"required\":false,\"type\":\"date\"}]}
engine.hive.enabled true
iceberg.orc.files.only false
metadata_location hdfs://### HDFS PATH ###
@@ -75,6 +76,7 @@ Table Parameters:
numRows 0
rawDataSize 0
serialization.format 1
+ snapshot-count 0
storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler
table_type ICEBERG
totalSize #Masked#
@@ -122,6 +124,8 @@ Table Parameters:
COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"bucket_field\":\"true\",\"day_field\":\"true\",\"hour_field\":\"true\",\"identity_field\":\"true\",\"month_field\":\"true\",\"truncate_field\":\"true\",\"year_field\":\"true\"}}
EXTERNAL TRUE
bucketing_version 2
+ current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"year_field\",\"required\":false,\"type\":\"date\"},{\"id\":2,\"name\":\"month_field\",\"required\":false,\"type\":\"date\"},{\"id\":3,\"name\":\"day_field\",\"required\":false,\"type\":\"date\"},{\"id\":4,\"name\":\"hour_field\",\"required\":false,\"type\":\"timestamp\"},{\"id\":5,\"name\":\"truncate_field\",\"required\":false,\"type\":\"string\"},{\"id\":6,\"name\":\"bucket_field\",\"required\":f [...]
+ default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"year_field_year\",\"transform\":\"year\",\"source-id\":1,\"field-id\":1000},{\"name\":\"month_field_month\",\"transform\":\"month\",\"source-id\":2,\"field-id\":1001},{\"name\":\"day_field_day\",\"transform\":\"day\",\"source-id\":3,\"field-id\":1002},{\"name\":\"hour_field_hour\",\"transform\":\"hour\",\"source-id\":4,\"field-id\":1003},{\"name\":\"truncate_field_trunc\",\"transform\":\"truncate[2]\",\"source-id\":5,\"field- [...]
engine.hive.enabled true
iceberg.orc.files.only false
metadata_location hdfs://### HDFS PATH ###
@@ -129,6 +133,7 @@ Table Parameters:
numRows 0
rawDataSize 0
serialization.format 1
+ snapshot-count 0
storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler
table_type ICEBERG
totalSize #Masked#
@@ -177,6 +182,8 @@ Table Parameters:
COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"bucket_field\":\"true\",\"day_field\":\"true\",\"hour_field\":\"true\",\"id\":\"true\",\"identity_field\":\"true\",\"month_field\":\"true\",\"truncate_field\":\"true\",\"year_field\":\"true\"}}
EXTERNAL TRUE
bucketing_version 2
+ current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"id\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"year_field\",\"required\":false,\"type\":\"date\"},{\"id\":3,\"name\":\"month_field\",\"required\":false,\"type\":\"date\"},{\"id\":4,\"name\":\"day_field\",\"required\":false,\"type\":\"date\"},{\"id\":5,\"name\":\"hour_field\",\"required\":false,\"type\":\"timestamp\"},{\"id\":6,\"name\":\"truncate_field\",\"required\":false,\"type\" [...]
+ default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"year_field_year\",\"transform\":\"year\",\"source-id\":2,\"field-id\":1000},{\"name\":\"month_field_month\",\"transform\":\"month\",\"source-id\":3,\"field-id\":1001},{\"name\":\"day_field_day\",\"transform\":\"day\",\"source-id\":4,\"field-id\":1002},{\"name\":\"hour_field_hour\",\"transform\":\"hour\",\"source-id\":5,\"field-id\":1003},{\"name\":\"truncate_field_trunc\",\"transform\":\"truncate[2]\",\"source-id\":6,\"field- [...]
engine.hive.enabled true
iceberg.orc.files.only false
metadata_location hdfs://### HDFS PATH ###
@@ -184,6 +191,7 @@ Table Parameters:
numRows 0
rawDataSize 0
serialization.format 1
+ snapshot-count 0
storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler
table_type ICEBERG
totalSize #Masked#
@@ -220,6 +228,8 @@ Table Parameters:
COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"a\":\"true\",\"b\":\"true\"}}
EXTERNAL TRUE
bucketing_version 2
+ current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"a\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"b\",\"required\":false,\"type\":\"string\"}]}
+ default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"b\",\"transform\":\"identity\",\"source-id\":2,\"field-id\":1000}]}
engine.hive.enabled true
iceberg.orc.files.only false
metadata_location hdfs://### HDFS PATH ###
@@ -227,6 +237,7 @@ Table Parameters:
numRows 0
rawDataSize 0
serialization.format 1
+ snapshot-count 0
storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler
table_type ICEBERG
totalSize #Masked#
diff --git a/iceberg/iceberg-handler/src/test/results/positive/iceberg_v2_deletes.q.out b/iceberg/iceberg-handler/src/test/results/positive/iceberg_v2_deletes.q.out
index 118f9120400..10d183f43d8 100644
--- a/iceberg/iceberg-handler/src/test/results/positive/iceberg_v2_deletes.q.out
+++ b/iceberg/iceberg-handler/src/test/results/positive/iceberg_v2_deletes.q.out
@@ -25,12 +25,14 @@ LOCATION
'hdfs://### HDFS PATH ###'
TBLPROPERTIES (
'bucketing_version'='2',
+ 'current-schema'='{"type":"struct","schema-id":0,"fields":[{"id":1,"name":"id","required":false,"type":"int"}]}',
'engine.hive.enabled'='true',
'format-version'='2',
'iceberg.delete.skiprowdata'='false',
'iceberg.orc.files.only'='true',
'metadata_location'='hdfs://### HDFS PATH ###',
'serialization.format'='1',
+ 'snapshot-count'='0',
'table_type'='ICEBERG',
#### A masked pattern was here ####
'uuid'='#Masked#',
@@ -131,7 +133,10 @@ LOCATION
'hdfs://### HDFS PATH ###'
TBLPROPERTIES (
'bucketing_version'='2',
- 'current-snapshot-id'='#SnapshotId#',
+ 'current-schema'='{"type":"struct","schema-id":0,"fields":[{"id":1,"name":"id","required":false,"type":"int"}]}',
+ 'current-snapshot-id'='#Masked#',
+ 'current-snapshot-summary'='{"added-position-delete-files":"1","added-delete-files":"1","added-files-size":"#Masked#","added-position-deletes":"1","changed-partition-count":"1","total-records":"8","total-files-size":"#Masked#","total-data-files":"2","total-delete-files":"2","total-position-deletes":"3","total-equality-deletes":"0"}',
+ 'current-snapshot-timestamp-ms'='#Masked#',
'engine.hive.enabled'='true',
'format-version'='2',
'iceberg.delete.skiprowdata'='true',
@@ -140,6 +145,7 @@ TBLPROPERTIES (
'metadata_location'='hdfs://### HDFS PATH ###',
'previous_metadata_location'='hdfs://### HDFS PATH ###',
'serialization.format'='1',
+ 'snapshot-count'='4',
'table_type'='ICEBERG',
#### A masked pattern was here ####
'uuid'='#Masked#',
@@ -273,7 +279,11 @@ LOCATION
'hdfs://### HDFS PATH ###'
TBLPROPERTIES (
'bucketing_version'='2',
- 'current-snapshot-id'='#SnapshotId#',
+ 'current-schema'='{"type":"struct","schema-id":0,"fields":[{"id":1,"name":"id","required":false,"type":"int"},{"id":2,"name":"part","required":false,"type":"int"}]}',
+ 'current-snapshot-id'='#Masked#',
+ 'current-snapshot-summary'='{"added-position-delete-files":"1","added-delete-files":"1","added-files-size":"#Masked#","added-position-deletes":"1","changed-partition-count":"1","total-records":"8","total-files-size":"#Masked#","total-data-files":"4","total-delete-files":"3","total-position-deletes":"3","total-equality-deletes":"0"}',
+ 'current-snapshot-timestamp-ms'='#Masked#',
+ 'default-partition-spec'='{"spec-id":0,"fields":[{"name":"part","transform":"identity","source-id":2,"field-id":1000}]}',
'engine.hive.enabled'='true',
'format-version'='2',
'iceberg.delete.skiprowdata'='true',
@@ -282,6 +292,7 @@ TBLPROPERTIES (
'metadata_location'='hdfs://### HDFS PATH ###',
'previous_metadata_location'='hdfs://### HDFS PATH ###',
'serialization.format'='1',
+ 'snapshot-count'='4',
'table_type'='ICEBERG',
#### A masked pattern was here ####
'uuid'='#Masked#',
diff --git a/iceberg/iceberg-handler/src/test/results/positive/mv_iceberg_partitioned_orc.q.out b/iceberg/iceberg-handler/src/test/results/positive/mv_iceberg_partitioned_orc.q.out
index 8cda12100b4..f4dbe3b5650 100644
--- a/iceberg/iceberg-handler/src/test/results/positive/mv_iceberg_partitioned_orc.q.out
+++ b/iceberg/iceberg-handler/src/test/results/positive/mv_iceberg_partitioned_orc.q.out
@@ -61,7 +61,11 @@ Table Type: MATERIALIZED_VIEW
Table Parameters:
COLUMN_STATS_ACCURATE {\"COLUMN_STATS\":{\"b\":\"true\",\"c\":\"true\"}}
bucketing_version 2
+ current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"c\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"b\",\"required\":false,\"type\":\"string\"}]}
current-snapshot-id #SnapshotId#
+ current-snapshot-summary {\"added-data-files\":\"2\",\"added-records\":\"2\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"2\",\"total-records\":\"2\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"2\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"}
+ current-snapshot-timestamp-ms #Masked#
+ default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"b\",\"transform\":\"identity\",\"source-id\":2,\"field-id\":1000}]}
engine.hive.enabled true
format-version 1
iceberg.orc.files.only true
@@ -69,6 +73,7 @@ Table Parameters:
numFiles 2
numRows 2
previous_metadata_location hdfs://### HDFS PATH ###
+ snapshot-count 1
storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler
table_type ICEBERG
totalSize #Masked#
@@ -141,7 +146,11 @@ Table Type: MATERIALIZED_VIEW
Table Parameters:
COLUMN_STATS_ACCURATE {\"COLUMN_STATS\":{\"b\":\"true\",\"c\":\"true\"}}
bucketing_version 2
+ current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"c\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"b\",\"required\":false,\"type\":\"string\"}]}
current-snapshot-id #SnapshotId#
+ current-snapshot-summary {\"added-data-files\":\"2\",\"added-records\":\"2\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"2\",\"total-records\":\"2\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"2\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"}
+ current-snapshot-timestamp-ms #Masked#
+ default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"b\",\"transform\":\"identity\",\"source-id\":2,\"field-id\":1000}]}
engine.hive.enabled true
format-version 2
iceberg.orc.files.only true
@@ -149,6 +158,7 @@ Table Parameters:
numFiles 2
numRows 2
previous_metadata_location hdfs://### HDFS PATH ###
+ snapshot-count 1
storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler
table_type ICEBERG
totalSize #Masked#
diff --git a/iceberg/iceberg-handler/src/test/results/positive/mv_iceberg_partitioned_orc2.q.out b/iceberg/iceberg-handler/src/test/results/positive/mv_iceberg_partitioned_orc2.q.out
index 6a31b96ef77..d83c099aee4 100644
--- a/iceberg/iceberg-handler/src/test/results/positive/mv_iceberg_partitioned_orc2.q.out
+++ b/iceberg/iceberg-handler/src/test/results/positive/mv_iceberg_partitioned_orc2.q.out
@@ -62,7 +62,11 @@ Table Type: MATERIALIZED_VIEW
Table Parameters:
COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"b\":\"true\",\"c\":\"true\"}}
bucketing_version 2
+ current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"b\",\"required\":false,\"type\":\"string\"},{\"id\":2,\"name\":\"c\",\"required\":false,\"type\":\"int\"}]}
current-snapshot-id #SnapshotId#
+ current-snapshot-summary {\"added-data-files\":\"2\",\"added-records\":\"2\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"2\",\"total-records\":\"2\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"2\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"}
+ current-snapshot-timestamp-ms #Masked#
+ default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"b_bucket\",\"transform\":\"bucket[16]\",\"source-id\":1,\"field-id\":1000},{\"name\":\"c_trunc\",\"transform\":\"truncate[3]\",\"source-id\":2,\"field-id\":1001}]}
engine.hive.enabled true
format-version 1
iceberg.orc.files.only true
@@ -70,6 +74,7 @@ Table Parameters:
numFiles 2
numRows 2
previous_metadata_location hdfs://### HDFS PATH ###
+ snapshot-count 1
storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler
table_type ICEBERG
totalSize #Masked#
@@ -143,7 +148,11 @@ Table Type: MATERIALIZED_VIEW
Table Parameters:
COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"b\":\"true\",\"c\":\"true\"}}
bucketing_version 2
+ current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"b\",\"required\":false,\"type\":\"string\"},{\"id\":2,\"name\":\"c\",\"required\":false,\"type\":\"int\"}]}
current-snapshot-id #SnapshotId#
+ current-snapshot-summary {\"added-data-files\":\"2\",\"added-records\":\"2\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"2\",\"total-records\":\"2\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"2\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"}
+ current-snapshot-timestamp-ms #Masked#
+ default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"b_bucket\",\"transform\":\"bucket[16]\",\"source-id\":1,\"field-id\":1000},{\"name\":\"c_trunc\",\"transform\":\"truncate[3]\",\"source-id\":2,\"field-id\":1001}]}
engine.hive.enabled true
format-version 2
iceberg.orc.files.only true
@@ -151,6 +160,7 @@ Table Parameters:
numFiles 2
numRows 2
previous_metadata_location hdfs://### HDFS PATH ###
+ snapshot-count 1
storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler
table_type ICEBERG
totalSize #Masked#
diff --git a/iceberg/iceberg-handler/src/test/results/positive/show_create_iceberg_table.q.out b/iceberg/iceberg-handler/src/test/results/positive/show_create_iceberg_table.q.out
index 50a9faa273a..38b678501a4 100644
--- a/iceberg/iceberg-handler/src/test/results/positive/show_create_iceberg_table.q.out
+++ b/iceberg/iceberg-handler/src/test/results/positive/show_create_iceberg_table.q.out
@@ -30,10 +30,12 @@ LOCATION
'hdfs://### HDFS PATH ###'
TBLPROPERTIES (
'bucketing_version'='2',
+ 'current-schema'='{"type":"struct","schema-id":0,"fields":[{"id":1,"name":"i","required":false,"type":"int"},{"id":2,"name":"s","required":false,"type":"string"},{"id":3,"name":"ts","required":false,"type":"timestamp"},{"id":4,"name":"d","required":false,"type":"date"}]}',
'engine.hive.enabled'='true',
'iceberg.orc.files.only'='false',
'metadata_location'='hdfs://### HDFS PATH ###',
'serialization.format'='1',
+ 'snapshot-count'='0',
'table_type'='ICEBERG',
#### A masked pattern was here ####
'uuid'='#Masked#')
@@ -80,10 +82,13 @@ LOCATION
'hdfs://### HDFS PATH ###'
TBLPROPERTIES (
'bucketing_version'='2',
+ 'current-schema'='{"type":"struct","schema-id":0,"fields":[{"id":1,"name":"year_field","required":false,"type":"date"},{"id":2,"name":"month_field","required":false,"type":"date"},{"id":3,"name":"day_field","required":false,"type":"date"},{"id":4,"name":"hour_field","required":false,"type":"timestamp"},{"id":5,"name":"truncate_field","required":false,"type":"string"},{"id":6,"name":"bucket_field","required":false,"type":"int"},{"id":7,"name":"identity_field","required":false,"type":"in [...]
+ 'default-partition-spec'='{"spec-id":0,"fields":[{"name":"year_field_year","transform":"year","source-id":1,"field-id":1000},{"name":"month_field_month","transform":"month","source-id":2,"field-id":1001},{"name":"day_field_day","transform":"day","source-id":3,"field-id":1002},{"name":"hour_field_hour","transform":"hour","source-id":4,"field-id":1003},{"name":"truncate_field_trunc","transform":"truncate[2]","source-id":5,"field-id":1004},{"name":"bucket_field_bucket","transform":"bucket [...]
'engine.hive.enabled'='true',
'iceberg.orc.files.only'='false',
'metadata_location'='hdfs://### HDFS PATH ###',
'serialization.format'='1',
+ 'snapshot-count'='0',
'table_type'='ICEBERG',
#### A masked pattern was here ####
'uuid'='#Masked#')
@@ -131,10 +136,13 @@ LOCATION
'hdfs://### HDFS PATH ###'
TBLPROPERTIES (
'bucketing_version'='2',
+ 'current-schema'='{"type":"struct","schema-id":0,"fields":[{"id":1,"name":"id","required":false,"type":"int"},{"id":2,"name":"year_field","required":false,"type":"date"},{"id":3,"name":"month_field","required":false,"type":"date"},{"id":4,"name":"day_field","required":false,"type":"date"},{"id":5,"name":"hour_field","required":false,"type":"timestamp"},{"id":6,"name":"truncate_field","required":false,"type":"string"},{"id":7,"name":"bucket_field","required":false,"type":"int"},{"id":8, [...]
+ 'default-partition-spec'='{"spec-id":0,"fields":[{"name":"year_field_year","transform":"year","source-id":2,"field-id":1000},{"name":"month_field_month","transform":"month","source-id":3,"field-id":1001},{"name":"day_field_day","transform":"day","source-id":4,"field-id":1002},{"name":"hour_field_hour","transform":"hour","source-id":5,"field-id":1003},{"name":"truncate_field_trunc","transform":"truncate[2]","source-id":6,"field-id":1004},{"name":"bucket_field_bucket","transform":"bucket [...]
'engine.hive.enabled'='true',
'iceberg.orc.files.only'='false',
'metadata_location'='hdfs://### HDFS PATH ###',
'serialization.format'='1',
+ 'snapshot-count'='0',
'table_type'='ICEBERG',
#### A masked pattern was here ####
'uuid'='#Masked#')
@@ -170,10 +178,13 @@ LOCATION
'hdfs://### HDFS PATH ###'
TBLPROPERTIES (
'bucketing_version'='2',
+ 'current-schema'='{"type":"struct","schema-id":0,"fields":[{"id":1,"name":"a","required":false,"type":"int"},{"id":2,"name":"b","required":false,"type":"string"}]}',
+ 'default-partition-spec'='{"spec-id":0,"fields":[{"name":"b","transform":"identity","source-id":2,"field-id":1000}]}',
'engine.hive.enabled'='true',
'iceberg.orc.files.only'='false',
'metadata_location'='hdfs://### HDFS PATH ###',
'serialization.format'='1',
+ 'snapshot-count'='0',
'table_type'='ICEBERG',
#### A masked pattern was here ####
'uuid'='#Masked#')
@@ -215,12 +226,16 @@ LOCATION
'hdfs://### HDFS PATH ###'
TBLPROPERTIES (
'bucketing_version'='2',
+ 'current-schema'='{"type":"struct","schema-id":0,"fields":[{"id":1,"name":"i","required":false,"type":"int"},{"id":2,"name":"s","required":false,"type":"string"}]}',
'current-snapshot-id'='#SnapshotId#',
+ 'current-snapshot-summary'='{"added-data-files":"1","added-records":"3","added-files-size":"638","changed-partition-count":"1","total-records":"3","total-files-size":"638","total-data-files":"1","total-delete-files":"0","total-position-deletes":"0","total-equality-deletes":"0"}',
+ 'current-snapshot-timestamp-ms'='#Masked#',
'engine.hive.enabled'='true',
'iceberg.orc.files.only'='false',
'metadata_location'='hdfs://### HDFS PATH ###',
'previous_metadata_location'='hdfs://### HDFS PATH ###',
'serialization.format'='1',
+ 'snapshot-count'='1',
'table_type'='ICEBERG',
#### A masked pattern was here ####
'uuid'='#Masked#')
diff --git a/iceberg/iceberg-handler/src/test/results/positive/truncate_force_iceberg_table.q.out b/iceberg/iceberg-handler/src/test/results/positive/truncate_force_iceberg_table.q.out
index 1cb30e82c40..d7abd58a001 100644
--- a/iceberg/iceberg-handler/src/test/results/positive/truncate_force_iceberg_table.q.out
+++ b/iceberg/iceberg-handler/src/test/results/positive/truncate_force_iceberg_table.q.out
@@ -88,7 +88,10 @@ Table Parameters:
COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"id\":\"true\",\"value\":\"true\"}}
EXTERNAL TRUE
bucketing_version 2
- current-snapshot-id #SnapshotId#
+ current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"id\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"value\",\"required\":false,\"type\":\"string\"}]}
+ current-snapshot-id #Masked#
+ current-snapshot-summary {\"added-data-files\":\"1\",\"added-records\":\"3\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"10\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"3\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"}
+ current-snapshot-timestamp-ms #Masked#
engine.hive.enabled true
external.table.purge false
iceberg.orc.files.only false
@@ -99,6 +102,7 @@ Table Parameters:
previous_metadata_location hdfs://### HDFS PATH ###
rawDataSize 0
serialization.format 1
+ snapshot-count 3
storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler
table_type ICEBERG
totalSize #Masked#
@@ -155,7 +159,10 @@ Table Parameters:
COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"}
EXTERNAL TRUE
bucketing_version 2
- current-snapshot-id #SnapshotId#
+ current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"id\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"value\",\"required\":false,\"type\":\"string\"}]}
+ current-snapshot-id #Masked#
+ current-snapshot-summary {\"deleted-data-files\":\"3\",\"deleted-records\":\"10\",\"removed-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"0\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"0\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"}
+ current-snapshot-timestamp-ms #Masked#
engine.hive.enabled true
external.table.purge false
iceberg.orc.files.only false
@@ -166,6 +173,7 @@ Table Parameters:
previous_metadata_location hdfs://### HDFS PATH ###
rawDataSize 0
serialization.format 1
+ snapshot-count 4
storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler
table_type ICEBERG
totalSize #Masked#
diff --git a/iceberg/iceberg-handler/src/test/results/positive/truncate_iceberg_table.q.out b/iceberg/iceberg-handler/src/test/results/positive/truncate_iceberg_table.q.out
index 2f1376d1f01..dfab2edec75 100644
--- a/iceberg/iceberg-handler/src/test/results/positive/truncate_iceberg_table.q.out
+++ b/iceberg/iceberg-handler/src/test/results/positive/truncate_iceberg_table.q.out
@@ -88,7 +88,10 @@ Table Parameters:
COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"id\":\"true\",\"value\":\"true\"}}
EXTERNAL TRUE
bucketing_version 2
- current-snapshot-id #SnapshotId#
+ current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"id\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"value\",\"required\":false,\"type\":\"string\"}]}
+ current-snapshot-id #Masked#
+ current-snapshot-summary {\"added-data-files\":\"1\",\"added-records\":\"3\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"10\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"3\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"}
+ current-snapshot-timestamp-ms #Masked#
engine.hive.enabled true
external.table.purge true
iceberg.orc.files.only true
@@ -99,6 +102,7 @@ Table Parameters:
previous_metadata_location hdfs://### HDFS PATH ###
rawDataSize 0
serialization.format 1
+ snapshot-count 3
storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler
table_type ICEBERG
totalSize #Masked#
@@ -155,7 +159,10 @@ Table Parameters:
COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"}
EXTERNAL TRUE
bucketing_version 2
- current-snapshot-id #SnapshotId#
+ current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"id\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"value\",\"required\":false,\"type\":\"string\"}]}
+ current-snapshot-id #Masked#
+ current-snapshot-summary {\"deleted-data-files\":\"3\",\"deleted-records\":\"10\",\"removed-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"0\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"0\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"}
+ current-snapshot-timestamp-ms #Masked#
engine.hive.enabled true
external.table.purge true
iceberg.orc.files.only true
@@ -166,6 +173,7 @@ Table Parameters:
previous_metadata_location hdfs://### HDFS PATH ###
rawDataSize 0
serialization.format 1
+ snapshot-count 4
storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler
table_type ICEBERG
totalSize #Masked#
@@ -220,7 +228,10 @@ Table Parameters:
COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"id\":\"true\",\"value\":\"true\"}}
EXTERNAL TRUE
bucketing_version 2
- current-snapshot-id #SnapshotId#
+ current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"id\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"value\",\"required\":false,\"type\":\"string\"}]}
+ current-snapshot-id #Masked#
+ current-snapshot-summary {\"added-data-files\":\"1\",\"added-records\":\"5\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"5\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"1\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"}
+ current-snapshot-timestamp-ms #Masked#
engine.hive.enabled true
external.table.purge true
iceberg.orc.files.only true
@@ -231,6 +242,7 @@ Table Parameters:
previous_metadata_location hdfs://### HDFS PATH ###
rawDataSize 0
serialization.format 1
+ snapshot-count 5
storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler
table_type ICEBERG
totalSize #Masked#
@@ -287,7 +299,10 @@ Table Parameters:
COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"}
EXTERNAL TRUE
bucketing_version 2
- current-snapshot-id #SnapshotId#
+ current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"id\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"value\",\"required\":false,\"type\":\"string\"}]}
+ current-snapshot-id #Masked#
+ current-snapshot-summary {\"deleted-data-files\":\"1\",\"deleted-records\":\"5\",\"removed-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"0\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"0\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"}
+ current-snapshot-timestamp-ms #Masked#
engine.hive.enabled true
external.table.purge true
iceberg.orc.files.only true
@@ -298,6 +313,7 @@ Table Parameters:
previous_metadata_location hdfs://### HDFS PATH ###
rawDataSize 0
serialization.format 1
+ snapshot-count 6
storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler
table_type ICEBERG
totalSize #Masked#
@@ -370,7 +386,10 @@ Table Parameters:
COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"}
EXTERNAL TRUE
bucketing_version 2
- current-snapshot-id #SnapshotId#
+ current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"id\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"value\",\"required\":false,\"type\":\"string\"}]}
+ current-snapshot-id #Masked#
+ current-snapshot-summary {\"deleted-data-files\":\"1\",\"deleted-records\":\"5\",\"removed-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"0\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"0\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"}
+ current-snapshot-timestamp-ms #Masked#
engine.hive.enabled true
external.table.purge false
iceberg.orc.files.only true
@@ -381,6 +400,7 @@ Table Parameters:
previous_metadata_location hdfs://### HDFS PATH ###
rawDataSize 0
serialization.format 1
+ snapshot-count 8
storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler
table_type ICEBERG
totalSize #Masked#
diff --git a/iceberg/iceberg-handler/src/test/results/positive/truncate_partitioned_iceberg_table.q.out b/iceberg/iceberg-handler/src/test/results/positive/truncate_partitioned_iceberg_table.q.out
index d6872db944f..ebcd4442a46 100644
--- a/iceberg/iceberg-handler/src/test/results/positive/truncate_partitioned_iceberg_table.q.out
+++ b/iceberg/iceberg-handler/src/test/results/positive/truncate_partitioned_iceberg_table.q.out
@@ -95,7 +95,11 @@ Table Parameters:
EXTERNAL TRUE
MIGRATED_TO_ICEBERG true
bucketing_version 2
- current-snapshot-id #SnapshotId#
+ current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"a\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"b\",\"required\":false,\"type\":\"string\"}]}
+ current-snapshot-id #Masked#
+ current-snapshot-summary {\"added-data-files\":\"4\",\"added-records\":\"9\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"4\",\"total-records\":\"9\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"4\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"}
+ current-snapshot-timestamp-ms #Masked#
+ default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"b\",\"transform\":\"identity\",\"source-id\":2,\"field-id\":1000}]}
engine.hive.enabled true
external.table.purge true
iceberg.orc.files.only false
@@ -111,6 +115,7 @@ Table Parameters:
\"field-id\" : 2,
\"names\" : [ \"b\" ]
} ]
+ snapshot-count 1
storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler
table_type ICEBERG
totalSize #Masked#
@@ -189,7 +194,11 @@ Table Parameters:
EXTERNAL TRUE
MIGRATED_TO_ICEBERG true
bucketing_version 2
- current-snapshot-id #SnapshotId#
+ current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"a\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"b\",\"required\":false,\"type\":\"string\"}]}
+ current-snapshot-id #Masked#
+ current-snapshot-summary {\"deleted-data-files\":\"4\",\"deleted-records\":\"9\",\"removed-files-size\":\"#Masked#\",\"changed-partition-count\":\"4\",\"total-records\":\"0\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"0\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"}
+ current-snapshot-timestamp-ms #Masked#
+ default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"b\",\"transform\":\"identity\",\"source-id\":2,\"field-id\":1000}]}
engine.hive.enabled true
external.table.purge true
iceberg.orc.files.only false
@@ -205,6 +214,7 @@ Table Parameters:
\"field-id\" : 2,
\"names\" : [ \"b\" ]
} ]
+ snapshot-count 2
storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler
table_type ICEBERG
totalSize #Masked#
diff --git a/iceberg/iceberg-handler/src/test/results/positive/use_basic_stats_from_iceberg.q.out b/iceberg/iceberg-handler/src/test/results/positive/use_basic_stats_from_iceberg.q.out
index 29f7fff01e8..d24cbaf5e2d 100644
--- a/iceberg/iceberg-handler/src/test/results/positive/use_basic_stats_from_iceberg.q.out
+++ b/iceberg/iceberg-handler/src/test/results/positive/use_basic_stats_from_iceberg.q.out
@@ -148,6 +148,7 @@ Table Type: MANAGED_TABLE
Table Parameters:
COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"a\":\"true\"}}
bucketing_version 2
+ current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"a\",\"required\":false,\"type\":\"int\"}]}
engine.hive.enabled true
format-version 2
iceberg.orc.files.only false
@@ -156,6 +157,7 @@ Table Parameters:
numRows 0
rawDataSize 0
serialization.format 1
+ snapshot-count 0
storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler
table_type ICEBERG
totalSize 0
@@ -189,6 +191,7 @@ Table Type: MANAGED_TABLE
Table Parameters:
COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"b\":\"true\"}}
bucketing_version 2
+ current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"b\",\"required\":false,\"type\":\"int\"}]}
engine.hive.enabled true
format-version 2
iceberg.orc.files.only false
@@ -197,6 +200,7 @@ Table Parameters:
numRows 0
rawDataSize 0
serialization.format 1
+ snapshot-count 0
storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler
table_type ICEBERG
totalSize 0