You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by bo...@apache.org on 2022/03/31 21:39:28 UTC

[impala] branch master updated (38062b4 -> 952f2af)

This is an automated email from the ASF dual-hosted git repository.

boroknagyz pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/impala.git.


    from 38062b4  IMPALA-11196 Fix a ClientCacheTest flaky test
     new 38efbee  IMPALA-11181: Improving performance of compaction checking
     new 952f2af  IMPALA-11210: Impala can only handle lowercase schema elements of Iceberg table

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../org/apache/impala/compat/MetastoreShim.java    |  39 ++--
 .../java/org/apache/impala/catalog/HdfsTable.java  |  12 ++
 .../org/apache/impala/catalog/IcebergColumn.java   |   2 +-
 .../impala/catalog/local/DirectMetaProvider.java   |   5 -
 .../apache/impala/util/IcebergSchemaConverter.java |   2 +-
 testdata/data/README                               |   3 +
 .../metadata/v1.metadata.json                      | 237 +++++++++++++++++++++
 .../metadata/version-hint.txt                      |   1 +
 .../functional/functional_schema_template.sql      |  14 ++
 .../datasets/functional/schema_constraints.csv     |   1 +
 .../queries/QueryTest/iceberg-query.test           |  14 ++
 11 files changed, 303 insertions(+), 27 deletions(-)
 create mode 100644 testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_uppercase_col/metadata/v1.metadata.json
 create mode 100644 testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_uppercase_col/metadata/version-hint.txt

[impala] 01/02: IMPALA-11181: Improving performance of compaction checking

Posted by bo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

boroknagyz pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/impala.git

commit 38efbee29d1d225d0a68eb5a6216325e12e81acd
Author: Yu-Wen Lai <yu...@cloudera.com>
AuthorDate: Tue Mar 15 20:12:52 2022 -0700

    IMPALA-11181: Improving performance of compaction checking
    
    After HIVE-25753, we don't need to explicitly set all partitions' name
    to get the latest compaction id. Besides, we can also send the last
    compaction id over to HMS so that HMS will send back compaction info
    only if there are newer compactions. In this way, we can avoid
    unnecessary data transmitted between HMS and Catalogd.
    
    Testing:
    existing tests
    
    Change-Id: I32e30ec418ad09bef862e61163539a910c96c44c
    Reviewed-on: http://gerrit.cloudera.org:8080/18324
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 .../org/apache/impala/compat/MetastoreShim.java    | 39 +++++++++++-----------
 .../java/org/apache/impala/catalog/HdfsTable.java  | 12 +++++++
 .../impala/catalog/local/DirectMetaProvider.java   |  5 ---
 3 files changed, 31 insertions(+), 25 deletions(-)

diff --git a/fe/src/compat-hive-3/java/org/apache/impala/compat/MetastoreShim.java b/fe/src/compat-hive-3/java/org/apache/impala/compat/MetastoreShim.java
index 0829106..37f35ab 100644
--- a/fe/src/compat-hive-3/java/org/apache/impala/compat/MetastoreShim.java
+++ b/fe/src/compat-hive-3/java/org/apache/impala/compat/MetastoreShim.java
@@ -108,6 +108,7 @@ import org.apache.impala.catalog.metastore.CatalogMetastoreServer;
 import org.apache.impala.catalog.metastore.ICatalogMetastoreServer;
 import org.apache.impala.common.ImpalaRuntimeException;
 import org.apache.impala.common.Metrics;
+import org.apache.impala.common.PrintUtils;
 import org.apache.impala.hive.common.MutableValidWriteIdList;
 import org.apache.impala.service.BackendConfig;
 import org.apache.impala.service.CatalogOpExecutor;
@@ -609,19 +610,12 @@ public class MetastoreShim extends Hive3MetastoreShimBase {
   public static List<HdfsPartition.Builder> getPartitionsForRefreshingFileMetadata(
       CatalogServiceCatalog catalog, HdfsTable hdfsTable) throws CatalogException {
     List<HdfsPartition.Builder> partBuilders = new ArrayList<>();
-    List<HdfsPartition> hdfsPartitions = hdfsTable.getPartitions()
-        .stream()
-        .map(p -> (HdfsPartition) p)
-        .collect(Collectors.toList());
     // fetch the latest compaction info from HMS
     GetLatestCommittedCompactionInfoRequest request =
         new GetLatestCommittedCompactionInfoRequest(
             hdfsTable.getDb().getName(), hdfsTable.getName());
-    if (hdfsTable.isPartitioned()) {
-      List<String> partNames = hdfsPartitions.stream()
-          .map(HdfsPartition::getPartitionName)
-          .collect(Collectors.toList());
-      request.setPartitionnames(partNames);
+    if (hdfsTable.getLastCompactionId() > 0) {
+      request.setLastCompactionId(hdfsTable.getLastCompactionId());
     }
 
     GetLatestCommittedCompactionInfoResponse response;
@@ -645,12 +639,9 @@ public class MetastoreShim extends Hive3MetastoreShimBase {
       }
     }
 
-    for (HdfsPartition partition : hdfsPartitions) {
-      long latestCompactionId =
-          partNameToCompactionId.getOrDefault(partition.getPartitionName(), -1L);
-      if (partition.getLastCompactionId() >= latestCompactionId) {
-        continue;
-      }
+    for (HdfsPartition partition : hdfsTable.getPartitionsForNames(
+        partNameToCompactionId.keySet())) {
+      long latestCompactionId = partNameToCompactionId.get(partition.getPartitionName());
       HdfsPartition.Builder builder = new HdfsPartition.Builder(partition);
       LOG.debug(
           "Cached compaction id for {} partition {}: {} but the latest compaction id: {}",
@@ -672,6 +663,10 @@ public class MetastoreShim extends Hive3MetastoreShimBase {
       throws TException {
     Preconditions.checkNotNull(table, "TableMetaRef must be non-null");
     Preconditions.checkNotNull(metas, "Partition map must be non-null");
+    if (metas.isEmpty()) {
+      return Collections.emptyList();
+    }
+    Stopwatch sw = Stopwatch.createStarted();
     List<PartitionRef> stalePartitions = new ArrayList<>();
     if (!table.isTransactional() || metas.isEmpty()) return stalePartitions;
     GetLatestCommittedCompactionInfoRequest request =
@@ -680,13 +675,17 @@ public class MetastoreShim extends Hive3MetastoreShimBase {
       request.setPartitionnames(metas.keySet().stream()
           .map(PartitionRef::getName).collect(Collectors.toList()));
     }
+    long lastCompactionId = metas.values().stream()
+        .mapToLong(p -> p.getLastCompactionId()).max().orElse(-1);
+    if (lastCompactionId > 0) {
+      request.setLastCompactionId(lastCompactionId);
+    }
+
     GetLatestCommittedCompactionInfoResponse response;
     try (MetaStoreClientPool.MetaStoreClient client = msClientPool.getClient()) {
       response = CompactionInfoLoader.getLatestCompactionInfo(client, request);
     }
     Map<String, Long> partNameToCompactionId = new HashMap<>();
-    // If the table is partitioned, we must set partition name, otherwise empty result
-    // will be returned.
     if (table.isPartitioned()) {
       for (CompactionInfoStruct ci : response.getCompactions()) {
         partNameToCompactionId.put(
@@ -703,13 +702,13 @@ public class MetastoreShim extends Hive3MetastoreShimBase {
         metas.entrySet().iterator();
     while (iter.hasNext()) {
       Map.Entry<PartitionRef, PartitionMetadata> entry = iter.next();
-      long latestCompactionId = partNameToCompactionId.getOrDefault(
-          entry.getKey().getName(), -1L);
-      if (entry.getValue().getLastCompactionId() < latestCompactionId) {
+      if (partNameToCompactionId.containsKey(entry.getKey().getName())) {
         stalePartitions.add(entry.getKey());
         iter.remove();
       }
     }
+    LOG.debug("Checked the latest compaction info for {}.{}. Time taken: {}", dbName,
+        tableName, PrintUtils.printTimeMs(sw.stop().elapsed(TimeUnit.MILLISECONDS)));
     return stalePartitions;
   }
 
diff --git a/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java b/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java
index 95917a1..5867b32 100644
--- a/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java
+++ b/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java
@@ -292,6 +292,13 @@ public class HdfsTable extends Table implements FeFsTable {
   // null in the case that this table is not transactional.
   protected MutableValidWriteIdList validWriteIds_ = null;
 
+  // The last committed compaction id in the table level. It will be sent as a filter to
+  // retrieve only the latest compaction that is not seen by this instance. This value is
+  // updated whenever a partition is added to the table so that it is guaranteed to be
+  // up-to-date.
+  // -1 means there is no previous compaction event or compaction is not supported.
+  private long lastCompactionId_ = -1;
+
   // Partitions are marked as "dirty" indicating there are in-progress modifications on
   // their metadata. The corresponding partition builder contains the new version of the
   // metadata so represents the in-progress modifications. The modifications will be
@@ -967,6 +974,7 @@ public class HdfsTable extends Table implements FeFsTable {
     fileMetadataStats_.totalFileBytes += partition.getSize();
     fileMetadataStats_.numFiles += partition.getNumFileDescriptors();
     updatePartitionMdAndColStats(partition);
+    lastCompactionId_ = Math.max(lastCompactionId_, partition.getLastCompactionId());
     return true;
   }
 
@@ -3015,6 +3023,10 @@ public class HdfsTable extends Table implements FeFsTable {
     }
   }
 
+  public long getLastCompactionId() {
+    return lastCompactionId_;
+  }
+
   /**
    * Updates the pending version of this table if the tbl version matches with the
    * expectedTblVersion.
diff --git a/fe/src/main/java/org/apache/impala/catalog/local/DirectMetaProvider.java b/fe/src/main/java/org/apache/impala/catalog/local/DirectMetaProvider.java
index 41c7d15..8b741fc 100644
--- a/fe/src/main/java/org/apache/impala/catalog/local/DirectMetaProvider.java
+++ b/fe/src/main/java/org/apache/impala/catalog/local/DirectMetaProvider.java
@@ -29,7 +29,6 @@ import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
 
-import com.google.common.base.Stopwatch;
 import com.google.common.collect.Iterables;
 import org.apache.commons.lang.NotImplementedException;
 import org.apache.hadoop.fs.Path;
@@ -55,7 +54,6 @@ import org.apache.impala.catalog.MetaStoreClientPool;
 import org.apache.impala.catalog.MetaStoreClientPool.MetaStoreClient;
 import org.apache.impala.catalog.SqlConstraints;
 import org.apache.impala.common.Pair;
-import org.apache.impala.common.PrintUtils;
 import org.apache.impala.compat.MetastoreShim;
 import org.apache.impala.service.BackendConfig;
 import org.apache.impala.thrift.TBackendGflags;
@@ -535,13 +533,10 @@ class DirectMetaProvider implements MetaProvider {
       TableMetaRef table, Map<PartitionRef, PartitionMetadata> metas) throws TException {
     Preconditions.checkNotNull(table, "TableMetaRef must be non-null");
     Preconditions.checkNotNull(metas, "Partition map must be non-null");
-    Stopwatch sw = Stopwatch.createStarted();
 
     List<PartitionRef> stalePartitions = MetastoreShim.checkLatestCompaction(
         msClientPool_, dbName, tableName, table, metas,
         PartitionRefImpl.UNPARTITIONED_NAME);
-    LOG.debug("Checked the latest compaction id for {}.{} Time taken: {}", dbName,
-        tableName, PrintUtils.printTimeMs(sw.stop().elapsed(TimeUnit.MILLISECONDS)));
     return stalePartitions;
   }
 

[impala] 02/02: IMPALA-11210: Impala can only handle lowercase schema elements of Iceberg table

Posted by bo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

boroknagyz pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/impala.git

commit 952f2af0ca5afd48bac828b66db467502da76ff2
Author: Zoltan Borok-Nagy <bo...@cloudera.com>
AuthorDate: Wed Mar 30 16:18:46 2022 +0200

    IMPALA-11210: Impala can only handle lowercase schema elements of Iceberg table
    
    When Impala/Hive creates a table they lowercase the schema elements.
    When Spark creates an Iceberg table it doesn't lowercase the names
    of the columns in the Iceberg metadata. This triggers a precondition
    check in Impala which makes such Iceberg tables unloadable.
    
    This patch converts column names to lowercase when converting Iceberg
    schemas to Hive/Impala schemas.
    
    Testing:
     * added e2e test
    
    Change-Id: Iffd910f76844fbf34db805dda6c3053c5ad1cf79
    Reviewed-on: http://gerrit.cloudera.org:8080/18368
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 .../org/apache/impala/catalog/IcebergColumn.java   |   2 +-
 .../apache/impala/util/IcebergSchemaConverter.java |   2 +-
 testdata/data/README                               |   3 +
 .../metadata/v1.metadata.json                      | 237 +++++++++++++++++++++
 .../metadata/version-hint.txt                      |   1 +
 .../functional/functional_schema_template.sql      |  14 ++
 .../datasets/functional/schema_constraints.csv     |   1 +
 .../queries/QueryTest/iceberg-query.test           |  14 ++
 8 files changed, 272 insertions(+), 2 deletions(-)

diff --git a/fe/src/main/java/org/apache/impala/catalog/IcebergColumn.java b/fe/src/main/java/org/apache/impala/catalog/IcebergColumn.java
index a9be0d9..1b6f953 100644
--- a/fe/src/main/java/org/apache/impala/catalog/IcebergColumn.java
+++ b/fe/src/main/java/org/apache/impala/catalog/IcebergColumn.java
@@ -41,7 +41,7 @@ public class IcebergColumn extends Column {
 
   public IcebergColumn(String name, Type type, String comment, int position,
       int fieldId, int fieldMapKeyId, int fieldMapValueId, boolean isNullable) {
-    super(name, type, comment, position);
+    super(name.toLowerCase(), type, comment, position);
     fieldId_ = fieldId;
     fieldMapKeyId_ = fieldMapKeyId;
     fieldMapValueId_ = fieldMapValueId;
diff --git a/fe/src/main/java/org/apache/impala/util/IcebergSchemaConverter.java b/fe/src/main/java/org/apache/impala/util/IcebergSchemaConverter.java
index 131a7f0..9757210 100644
--- a/fe/src/main/java/org/apache/impala/util/IcebergSchemaConverter.java
+++ b/fe/src/main/java/org/apache/impala/util/IcebergSchemaConverter.java
@@ -115,7 +115,7 @@ public class IcebergSchemaConverter {
     for (Types.NestedField column : schema.columns()) {
       Type colType = toImpalaType(column.type());
       // Update sd cols by iceberg NestedField
-      ret.add(new FieldSchema(column.name(), colType.toSql().toLowerCase(),
+      ret.add(new FieldSchema(column.name().toLowerCase(), colType.toSql().toLowerCase(),
           column.doc()));
     }
     return ret;
diff --git a/testdata/data/README b/testdata/data/README
index 804c5a1..af49e93 100644
--- a/testdata/data/README
+++ b/testdata/data/README
@@ -715,3 +715,6 @@ The tables that have the following schema changes since table migration:
 * Partition FLOAT column to DOUBLE
 * Partition DECIMAL(5,3) column to DECIMAL(8,3)
 * Non-partition column has been moved to end of the schema
+
+iceberg_test/hadoop_catalog/ice/iceberg_uppercase_col:
+Generated by Impala, then modified the metadata.json file to contain uppercase characters.
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_uppercase_col/metadata/v1.metadata.json b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_uppercase_col/metadata/v1.metadata.json
new file mode 100644
index 0000000..5de1476
--- /dev/null
+++ b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_uppercase_col/metadata/v1.metadata.json
@@ -0,0 +1,237 @@
+{
+  "format-version" : 1,
+  "table-uuid" : "3a93e4c0-5357-4203-a7e1-242168207af8",
+  "location" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_uppercase_col",
+  "last-updated-ms" : 1648649057966,
+  "last-column-id" : 1,
+  "schema" : {
+    "type" : "struct",
+    "schema-id" : 0,
+    "fields" : [ {
+      "id" : 1,
+      "name" : "Region",
+      "required" : false,
+      "type" : "string",
+      "doc" : "from deserializer"
+    }, {
+      "id" : 2,
+      "name" : "Nested_Struct",
+      "required" : false,
+      "type" : {
+        "type" : "struct",
+        "fields" : [ {
+          "id" : 3,
+          "name" : "A",
+          "required" : true,
+          "type" : "int"
+        }, {
+          "id" : 4,
+          "name" : "B",
+          "required" : true,
+          "type" : {
+            "type" : "list",
+            "element-id" : 7,
+            "element" : "int",
+            "element-required" : true
+          }
+        }, {
+          "id" : 5,
+          "name" : "C",
+          "required" : true,
+          "type" : {
+            "type" : "struct",
+            "fields" : [ {
+              "id" : 8,
+              "name" : "D",
+              "required" : true,
+              "type" : {
+                "type" : "list",
+                "element-id" : 9,
+                "element" : {
+                  "type" : "list",
+                  "element-id" : 10,
+                  "element" : {
+                    "type" : "struct",
+                    "fields" : [ {
+                      "id" : 11,
+                      "name" : "E",
+                      "required" : true,
+                      "type" : "int"
+                    }, {
+                      "id" : 12,
+                      "name" : "F",
+                      "required" : true,
+                      "type" : "string"
+                    } ]
+                  },
+                  "element-required" : true
+                },
+                "element-required" : true
+              }
+            } ]
+          }
+        }, {
+          "id" : 6,
+          "name" : "G",
+          "required" : true,
+          "type" : {
+            "type" : "map",
+            "key-id" : 13,
+            "key" : "string",
+            "value-id" : 14,
+            "value" : {
+              "type" : "struct",
+              "fields" : [ {
+                "id" : 15,
+                "name" : "H",
+                "required" : true,
+                "type" : {
+                  "type" : "struct",
+                  "fields" : [ {
+                    "id" : 16,
+                    "name" : "I",
+                    "required" : true,
+                    "type" : {
+                      "type" : "list",
+                      "element-id" : 17,
+                      "element" : "double",
+                      "element-required" : true
+                    }
+                  } ]
+                }
+              } ]
+            },
+            "value-required" : true
+          }
+        } ]
+      }
+    } ]
+  },
+  "current-schema-id" : 0,
+  "schemas" : [ {
+    "type" : "struct",
+    "schema-id" : 0,
+    "fields" : [ {
+      "id" : 1,
+      "name" : "region",
+      "required" : false,
+      "type" : "string",
+      "doc" : "from deserializer"
+    }, {
+      "id" : 2,
+      "name" : "Nested_Struct",
+      "required" : false,
+      "type" : {
+        "type" : "struct",
+        "fields" : [ {
+          "id" : 3,
+          "name" : "A",
+          "required" : true,
+          "type" : "int"
+        }, {
+          "id" : 4,
+          "name" : "B",
+          "required" : true,
+          "type" : {
+            "type" : "list",
+            "element-id" : 7,
+            "element" : "int",
+            "element-required" : true
+          }
+        }, {
+          "id" : 5,
+          "name" : "C",
+          "required" : true,
+          "type" : {
+            "type" : "struct",
+            "fields" : [ {
+              "id" : 8,
+              "name" : "D",
+              "required" : true,
+              "type" : {
+                "type" : "list",
+                "element-id" : 9,
+                "element" : {
+                  "type" : "list",
+                  "element-id" : 10,
+                  "element" : {
+                    "type" : "struct",
+                    "fields" : [ {
+                      "id" : 11,
+                      "name" : "E",
+                      "required" : true,
+                      "type" : "int"
+                    }, {
+                      "id" : 12,
+                      "name" : "F",
+                      "required" : true,
+                      "type" : "string"
+                    } ]
+                  },
+                  "element-required" : true
+                },
+                "element-required" : true
+              }
+            } ]
+          }
+        }, {
+          "id" : 6,
+          "name" : "G",
+          "required" : true,
+          "type" : {
+            "type" : "map",
+            "key-id" : 13,
+            "key" : "string",
+            "value-id" : 14,
+            "value" : {
+              "type" : "struct",
+              "fields" : [ {
+                "id" : 15,
+                "name" : "H",
+                "required" : true,
+                "type" : {
+                  "type" : "struct",
+                  "fields" : [ {
+                    "id" : 16,
+                    "name" : "I",
+                    "required" : true,
+                    "type" : {
+                      "type" : "list",
+                      "element-id" : 17,
+                      "element" : "double",
+                      "element-required" : true
+                    }
+                  } ]
+                }
+              } ]
+            },
+            "value-required" : true
+          }
+        } ]
+      }
+    } ]
+  } ],
+  "partition-spec" : [ ],
+  "default-spec-id" : 0,
+  "partition-specs" : [ {
+    "spec-id" : 0,
+    "fields" : [ ]
+  } ],
+  "last-partition-id" : 999,
+  "default-sort-order-id" : 0,
+  "sort-orders" : [ {
+    "order-id" : 0,
+    "fields" : [ ]
+  } ],
+  "properties" : {
+    "engine.hive.enabled" : "true",
+    "external.table.purge" : "TRUE",
+    "write.format.default" : "parquet",
+    "OBJCAPABILITIES" : "EXTREAD,EXTWRITE",
+    "storage_handler" : "org.apache.iceberg.mr.hive.HiveIcebergStorageHandler"
+  },
+  "current-snapshot-id" : -1,
+  "snapshots" : [ ],
+  "snapshot-log" : [ ],
+  "metadata-log" : [ ]
+}
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_uppercase_col/metadata/version-hint.txt b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_uppercase_col/metadata/version-hint.txt
new file mode 100644
index 0000000..d00491f
--- /dev/null
+++ b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_uppercase_col/metadata/version-hint.txt
@@ -0,0 +1 @@
+1
diff --git a/testdata/datasets/functional/functional_schema_template.sql b/testdata/datasets/functional/functional_schema_template.sql
index 21d5c16..a81f76d 100644
--- a/testdata/datasets/functional/functional_schema_template.sql
+++ b/testdata/datasets/functional/functional_schema_template.sql
@@ -3297,6 +3297,20 @@ STORED AS ICEBERG;
 ---- DATASET
 functional
 ---- BASE_TABLE_NAME
+iceberg_uppercase_col
+---- CREATE
+CREATE EXTERNAL TABLE IF NOT EXISTS {db_name}{db_suffix}.{table_name}
+STORED AS ICEBERG
+TBLPROPERTIES('write.format.default'='parquet', 'iceberg.catalog'='hadoop.catalog',
+              'iceberg.catalog_location'='/test-warehouse/iceberg_test/hadoop_catalog',
+              'iceberg.table_identifier'='ice.iceberg_uppercase_col');
+---- DEPENDENT_LOAD
+`hadoop fs -mkdir -p /test-warehouse/iceberg_test/hadoop_catalog/ice && \
+hadoop fs -put -f ${IMPALA_HOME}/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_uppercase_col /test-warehouse/iceberg_test/hadoop_catalog/ice
+====
+---- DATASET
+functional
+---- BASE_TABLE_NAME
 alltypes_date_partition_2
 ---- PARTITION_COLUMNS
 date_col date
diff --git a/testdata/datasets/functional/schema_constraints.csv b/testdata/datasets/functional/schema_constraints.csv
index 0bb7711..c462d4a 100644
--- a/testdata/datasets/functional/schema_constraints.csv
+++ b/testdata/datasets/functional/schema_constraints.csv
@@ -77,6 +77,7 @@ table_name:iceberg_alltypes_part, constraint:restrict_to, table_format:parquet/n
 table_name:iceberg_alltypes_part_orc, constraint:restrict_to, table_format:parquet/none/none
 table_name:iceberg_legacy_partition_schema_evolution, constraint:restrict_to, table_format:parquet/none/none
 table_name:iceberg_legacy_partition_schema_evolution_orc, constraint:restrict_to, table_format:parquet/none/none
+table_name:iceberg_uppercase_col, constraint:restrict_to, table_format:parquet/none/none
 
 # TODO: Support Avro. Data loading currently fails for Avro because complex types
 # cannot be converted to the corresponding Avro types yet.
diff --git a/testdata/workloads/functional-query/queries/QueryTest/iceberg-query.test b/testdata/workloads/functional-query/queries/QueryTest/iceberg-query.test
index 4d2b590..8adb1d8 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/iceberg-query.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/iceberg-query.test
@@ -507,3 +507,17 @@ Path,Size,Partition
 ---- TYPES
 STRING,STRING,STRING
 ====
+---- QUERY
+describe formatted iceberg_uppercase_col;
+---- RESULTS: VERIFY_IS_SUBSET
+'region','string','from deserializer'
+'nested_struct','struct<a:int,b:array<int>,c:struct<d:array<array<struct<e:int,f:string>>>>,g:map<string,struct<h:struct<i:array<double>>>>>','NULL'
+---- TYPES
+string, string, string
+====
+---- QUERY
+SELECT * FROM iceberg_uppercase_col;
+---- RESULTS
+---- TYPES
+STRING
+====