You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by jo...@apache.org on 2022/12/08 04:29:20 UTC

[impala] branch master updated: IMPALA-11158: Add support for Iceberg tables with AVRO data files

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 80fc49abe IMPALA-11158: Add support for Iceberg tables with AVRO data files
80fc49abe is described below

commit 80fc49abe6c574f8d6bea8bb56ff4917f6281a41
Author: noemi <np...@cloudera.com>
AuthorDate: Wed Sep 21 19:07:59 2022 +0200

    IMPALA-11158: Add support for Iceberg tables with AVRO data files
    
    Iceberg tables containing only AVRO files or no AVRO files at all
    can now be read by Impala. Mixed file format tables with AVRO are
    currently unsupported.
    Impala uses its avro scanner to read AVRO files, therefore all the
    avro related limitations apply here as well: writes/metadata
    changes are not supported.
    
    testing:
    - created test tables: 'iceberg_avro_only' contains only AVRO files;
      'iceberg_avro_mixed' contains all file formats: avro+orc+parquet
    - added E2E test that reads Avro-only table
    - added test case to iceberg-negative.test that tries to read
      mixed file format table
    
    Change-Id: I827e5707e54bebabc614e127daa48255f86f4c4f
    Reviewed-on: http://gerrit.cloudera.org:8080/19084
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 common/thrift/CatalogObjects.thrift                |   6 +-
 .../java/org/apache/impala/catalog/HdfsTable.java  |  96 ++++++------
 .../impala/catalog/IcebergContentFileStore.java    |  29 ++++
 .../org/apache/impala/catalog/IcebergTable.java    |  12 ++
 .../apache/impala/catalog/local/LocalFsTable.java  |  16 +-
 .../impala/catalog/local/LocalIcebergTable.java    |   2 +
 .../org/apache/impala/planner/IcebergScanNode.java |  13 +-
 .../java/org/apache/impala/util/IcebergUtil.java   |   4 +
 ...19aad52ced-job_16629766502890_0016-1-00001.avro | Bin 0 -> 723 bytes
 ...24d958ebc7-job_16629766502890_0017-1-00001.avro | Bin 0 -> 723 bytes
 ...bec36d7a93c-job_16629766502890_0018-1-00001.orc | Bin 0 -> 534 bytes
 ...7912ad1-job_16629766502890_0019-1-00001.parquet | Bin 0 -> 1109 bytes
 .../13c55017-b018-4ccb-a407-08e37e28eec8-m0.avro   | Bin 0 -> 3240 bytes
 .../7b422180-e3f8-4500-b240-1424ef012246-m0.avro   | Bin 0 -> 3240 bytes
 .../80a79f8a-5a47-44c9-b16d-4bef4a5ecec3-m0.avro   | Bin 0 -> 3332 bytes
 .../8e66c338-5cd3-4b85-b986-18ec29b67d94-m0.avro   | Bin 0 -> 3323 bytes
 ...058-1-8e66c338-5cd3-4b85-b986-18ec29b67d94.avro | Bin 0 -> 2158 bytes
 ...787-1-13c55017-b018-4ccb-a407-08e37e28eec8.avro | Bin 0 -> 2009 bytes
 ...034-1-7b422180-e3f8-4500-b240-1424ef012246.avro | Bin 0 -> 2009 bytes
 ...133-1-80a79f8a-5a47-44c9-b16d-4bef4a5ecec3.avro | Bin 0 -> 2308 bytes
 .../iceberg_avro_mixed/metadata/v1.metadata.json   |  82 ++++++++++
 .../iceberg_avro_mixed/metadata/v2.metadata.json   |  83 ++++++++++
 .../iceberg_avro_mixed/metadata/v3.metadata.json   |  83 ++++++++++
 .../iceberg_avro_mixed/metadata/v4.metadata.json   | 112 ++++++++++++++
 .../iceberg_avro_mixed/metadata/v5.metadata.json   | 112 ++++++++++++++
 .../iceberg_avro_mixed/metadata/v6.metadata.json   | 117 ++++++++++++++
 .../iceberg_avro_mixed/metadata/v7.metadata.json   | 142 +++++++++++++++++
 .../iceberg_avro_mixed/metadata/v8.metadata.json   | 145 ++++++++++++++++++
 .../iceberg_avro_mixed/metadata/v9.metadata.json   | 170 +++++++++++++++++++++
 .../iceberg_avro_mixed/metadata/version-hint.txt   |   1 +
 ...cd56b94b46-job_16629766502890_0015-1-00001.avro | Bin 0 -> 604 bytes
 .../a9f8d35c-a852-49fe-996a-d94ae1896c32-m0.avro   | Bin 0 -> 3239 bytes
 ...732-1-a9f8d35c-a852-49fe-996a-d94ae1896c32.avro | Bin 0 -> 2008 bytes
 .../iceberg_avro_only/metadata/v1.metadata.json    |  72 +++++++++
 .../iceberg_avro_only/metadata/v2.metadata.json    | 101 ++++++++++++
 .../iceberg_avro_only/metadata/version-hint.txt    |   1 +
 .../ice/iceberg_avro_only/version-hint.txt         |   1 +
 .../functional/functional_schema_template.sql      |  41 +++++
 .../datasets/functional/schema_constraints.csv     |   2 +
 .../queries/QueryTest/iceberg-avro.test            |  10 ++
 .../queries/QueryTest/iceberg-negative.test        |   5 +
 tests/query_test/test_iceberg.py                   |   2 +
 42 files changed, 1412 insertions(+), 48 deletions(-)

diff --git a/common/thrift/CatalogObjects.thrift b/common/thrift/CatalogObjects.thrift
index b5a7a0b4f..0df4be794 100644
--- a/common/thrift/CatalogObjects.thrift
+++ b/common/thrift/CatalogObjects.thrift
@@ -98,10 +98,11 @@ enum THdfsCompression {
   LZ4_BLOCKED = 12
 }
 
-// Iceberg table file format identitied by table property 'write.format.default'
+// Iceberg table file format identified by table property 'write.format.default'
 enum TIcebergFileFormat {
   PARQUET = 0
   ORC = 1
+  AVRO = 2
 }
 
 // Iceberg table catalog type identified by table property 'iceberg.catalog'
@@ -604,6 +605,9 @@ struct TIcebergPartitionStats {
 struct TIcebergContentFileStore {
   1: optional map<string, THdfsFileDesc> path_hash_to_data_file
   2: optional map<string, THdfsFileDesc> path_hash_to_delete_file
+  3: optional bool has_avro
+  4: optional bool has_orc
+  5: optional bool has_parquet
 }
 
 struct TIcebergTable {
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 7a9913828..f4ea37c45 100644
--- a/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java
+++ b/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java
@@ -1747,9 +1747,8 @@ public class HdfsTable extends Table implements FeFsTable {
   }
 
   /**
-   * Sets avroSchema_ if the table or any of the partitions in the table are stored
-   * as Avro. Additionally, this method also reconciles the schema if the column
-   * definitions from the metastore differ from the Avro schema.
+   * Checks if the table or any of the partitions in the table are stored as Avro.
+   * If so, calls setAvroSchemaInternal to set avroSchema_.
    */
   protected void setAvroSchema(IMetaStoreClient client,
       org.apache.hadoop.hive.metastore.api.Table msTbl) throws Exception {
@@ -1758,47 +1757,56 @@ public class HdfsTable extends Table implements FeFsTable {
     String serDeLib = msTbl.getSd().getSerdeInfo().getSerializationLib();
     if (HdfsFileFormat.fromJavaClassName(inputFormat, serDeLib) == HdfsFileFormat.AVRO
         || hasAvroData_) {
-      // Look for Avro schema in TBLPROPERTIES and in SERDEPROPERTIES, with the latter
-      // taking precedence.
-      List<Map<String, String>> schemaSearchLocations = new ArrayList<>();
-      schemaSearchLocations.add(
-          getMetaStoreTable().getSd().getSerdeInfo().getParameters());
-      schemaSearchLocations.add(getMetaStoreTable().getParameters());
-
-      avroSchema_ = AvroSchemaUtils.getAvroSchema(schemaSearchLocations);
-
-      if (avroSchema_ == null) {
-        // No Avro schema was explicitly set in the table metadata, so infer the Avro
-        // schema from the column definitions.
-        Schema inferredSchema = AvroSchemaConverter.convertFieldSchemas(
-            msTbl.getSd().getCols(), getFullName());
-        avroSchema_ = inferredSchema.toString();
-        // NOTE: below we reconcile this inferred schema back into the table
-        // schema in the case of Avro-formatted tables. This has the side effect
-        // of promoting types like TINYINT to INT.
-      }
-      String serdeLib = msTbl.getSd().getSerdeInfo().getSerializationLib();
-      if (serdeLib == null ||
-          serdeLib.equals("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe")) {
-        // If the SerDe library is null or set to LazySimpleSerDe or is null, it
-        // indicates there is an issue with the table metadata since Avro table need a
-        // non-native serde. Instead of failing to load the table, fall back to
-        // using the fields from the storage descriptor (same as Hive).
-        return;
-      } else {
-        List<FieldSchema> reconciledFieldSchemas = AvroSchemaUtils.reconcileAvroSchema(
-            msTbl, avroSchema_);
-
-        // Reset and update nonPartFieldSchemas_ to the reconciled colDefs.
-        nonPartFieldSchemas_.clear();
-        nonPartFieldSchemas_.addAll(reconciledFieldSchemas);
-        // Update the columns as per the reconciled colDefs and re-load stats.
-        clearColumns();
-        addColumnsFromFieldSchemas(msTbl.getPartitionKeys());
-        addColumnsFromFieldSchemas(nonPartFieldSchemas_);
-        addVirtualColumns();
-        loadAllColumnStats(client);
-      }
+      setAvroSchemaInternal(client, msTable_);
+    }
+  }
+
+  /**
+   * Sets avroSchema_. Additionally, this method also reconciles the schema if the column
+   * definitions from the metastore differ from the Avro schema.
+   */
+  protected void setAvroSchemaInternal(IMetaStoreClient client,
+      org.apache.hadoop.hive.metastore.api.Table msTbl) throws Exception {
+    // Look for Avro schema in TBLPROPERTIES and in SERDEPROPERTIES, with the latter
+    // taking precedence.
+    List<Map<String, String>> schemaSearchLocations = new ArrayList<>();
+    schemaSearchLocations.add(
+        getMetaStoreTable().getSd().getSerdeInfo().getParameters());
+    schemaSearchLocations.add(getMetaStoreTable().getParameters());
+
+    avroSchema_ = AvroSchemaUtils.getAvroSchema(schemaSearchLocations);
+
+    if (avroSchema_ == null) {
+      // No Avro schema was explicitly set in the table metadata, so infer the Avro
+      // schema from the column definitions.
+      Schema inferredSchema = AvroSchemaConverter.convertFieldSchemas(
+          msTbl.getSd().getCols(), getFullName());
+      avroSchema_ = inferredSchema.toString();
+      // NOTE: below we reconcile this inferred schema back into the table
+      // schema in the case of Avro-formatted tables. This has the side effect
+      // of promoting types like TINYINT to INT.
+    }
+    String serdeLib = msTbl.getSd().getSerdeInfo().getSerializationLib();
+    if (serdeLib == null ||
+        serdeLib.equals("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe")) {
+      // If the SerDe library is null or set to LazySimpleSerDe or is null, it
+      // indicates there is an issue with the table metadata since Avro table need a
+      // non-native serde. Instead of failing to load the table, fall back to
+      // using the fields from the storage descriptor (same as Hive).
+      return;
+    } else {
+      List<FieldSchema> reconciledFieldSchemas = AvroSchemaUtils.reconcileAvroSchema(
+          msTbl, avroSchema_);
+
+      // Reset and update nonPartFieldSchemas_ to the reconciled colDefs.
+      nonPartFieldSchemas_.clear();
+      nonPartFieldSchemas_.addAll(reconciledFieldSchemas);
+      // Update the columns as per the reconciled colDefs and re-load stats.
+      clearColumns();
+      addColumnsFromFieldSchemas(msTbl.getPartitionKeys());
+      addColumnsFromFieldSchemas(nonPartFieldSchemas_);
+      addVirtualColumns();
+      loadAllColumnStats(client);
     }
   }
 
diff --git a/fe/src/main/java/org/apache/impala/catalog/IcebergContentFileStore.java b/fe/src/main/java/org/apache/impala/catalog/IcebergContentFileStore.java
index ee7cc0974..4101acebb 100644
--- a/fe/src/main/java/org/apache/impala/catalog/IcebergContentFileStore.java
+++ b/fe/src/main/java/org/apache/impala/catalog/IcebergContentFileStore.java
@@ -28,6 +28,7 @@ import java.util.concurrent.ConcurrentMap;
 
 import org.apache.curator.shaded.com.google.common.base.Preconditions;
 import org.apache.impala.catalog.HdfsPartition.FileDescriptor;
+import org.apache.impala.fb.FbIcebergDataFileFormat;
 import org.apache.impala.thrift.THdfsFileDesc;
 import org.apache.impala.thrift.TIcebergContentFileStore;
 import org.apache.impala.thrift.TNetworkAddress;
@@ -53,17 +54,24 @@ public class IcebergContentFileStore {
   private final ConcurrentMap<String, FileDescriptor> oldFileDescMap_ =
       new ConcurrentHashMap<>();
 
+  // Flags to indicate file formats used in the table.
+  private boolean hasAvro_ = false;
+  private boolean hasOrc_ = false;
+  private boolean hasParquet_ = false;
+
   public IcebergContentFileStore() {}
 
   public void addDataFileDescriptor(String pathHash, FileDescriptor desc) {
     if (dataFileDescMap_.put(pathHash, desc) == null) {
       dataFiles_.add(desc);
+      updateFileFormats(desc);
     }
   }
 
   public void addDeleteFileDescriptor(String pathHash, FileDescriptor desc) {
     if (deleteFileDescMap_.put(pathHash, desc) == null) {
       deleteFiles_.add(desc);
+      updateFileFormats(desc);
     }
   }
 
@@ -103,10 +111,28 @@ public class IcebergContentFileStore {
     return Iterables.concat(dataFiles_, deleteFiles_);
   }
 
+  public boolean hasAvro() { return hasAvro_; }
+  public boolean hasOrc() { return hasOrc_; }
+  public boolean hasParquet() { return hasParquet_; }
+
+  private void updateFileFormats(FileDescriptor desc) {
+    byte fileFormat = desc.getFbFileMetadata().icebergMetadata().fileFormat();
+    if (fileFormat == FbIcebergDataFileFormat.PARQUET) {
+      hasParquet_ = true;
+    } else if (fileFormat == FbIcebergDataFileFormat.ORC) {
+      hasOrc_ = true;
+    } else if (fileFormat == FbIcebergDataFileFormat.AVRO) {
+      hasAvro_ = true;
+    }
+  }
+
   public TIcebergContentFileStore toThrift() {
     TIcebergContentFileStore ret = new TIcebergContentFileStore();
     ret.setPath_hash_to_data_file(convertFileMapToThrift(dataFileDescMap_));
     ret.setPath_hash_to_delete_file(convertFileMapToThrift(deleteFileDescMap_));
+    ret.setHas_avro(hasAvro_);
+    ret.setHas_orc(hasOrc_);
+    ret.setHas_parquet(hasParquet_);
     return ret;
   }
 
@@ -122,6 +148,9 @@ public class IcebergContentFileStore {
       convertFileMapFromThrift(tFileStore.getPath_hash_to_delete_file(),
           ret.deleteFileDescMap_, ret.deleteFiles_, networkAddresses, hostIndex);
     }
+    ret.hasAvro_ = tFileStore.isSetHas_avro() ? tFileStore.isHas_avro() : false;
+    ret.hasOrc_ = tFileStore.isSetHas_orc() ? tFileStore.isHas_orc() : false;
+    ret.hasParquet_ = tFileStore.isSetHas_parquet() ? tFileStore.isHas_parquet() : false;
     return ret;
   }
 
diff --git a/fe/src/main/java/org/apache/impala/catalog/IcebergTable.java b/fe/src/main/java/org/apache/impala/catalog/IcebergTable.java
index 6d239e8a5..04e418e1c 100644
--- a/fe/src/main/java/org/apache/impala/catalog/IcebergTable.java
+++ b/fe/src/main/java/org/apache/impala/catalog/IcebergTable.java
@@ -359,6 +359,7 @@ public class IcebergTable extends Table implements FeIcebergTable {
         partitionStats_ = Utils.loadPartitionStats(this);
         setIcebergTableStats();
         loadAllColumnStats(msClient);
+        setAvroSchema(msClient, msTbl, fileStore_);
       } catch (Exception e) {
         throw new IcebergTableLoadingException("Error loading metadata for Iceberg table "
             + icebergTableLocation_, e);
@@ -407,6 +408,17 @@ public class IcebergTable extends Table implements FeIcebergTable {
     }
   }
 
+  /**
+   * Loads the AVRO schema if the table contains AVRO files.
+   */
+  private void setAvroSchema(IMetaStoreClient msClient,
+      org.apache.hadoop.hive.metastore.api.Table msTbl,
+      IcebergContentFileStore fileStore) throws Exception {
+    if (fileStore.hasAvro()) {
+      hdfsTable_.setAvroSchemaInternal(msClient, msTbl);
+    }
+  }
+
   @Override
   public void addColumn(Column col) {
     Preconditions.checkState(col instanceof IcebergColumn);
diff --git a/fe/src/main/java/org/apache/impala/catalog/local/LocalFsTable.java b/fe/src/main/java/org/apache/impala/catalog/local/LocalFsTable.java
index eca95765d..d5768511a 100644
--- a/fe/src/main/java/org/apache/impala/catalog/local/LocalFsTable.java
+++ b/fe/src/main/java/org/apache/impala/catalog/local/LocalFsTable.java
@@ -122,7 +122,7 @@ public class LocalFsTable extends LocalTable implements FeFsTable {
    * as a table property. Such a schema is used when querying Avro partitions
    * of non-Avro tables.
    */
-  private final String avroSchema_;
+  private String avroSchema_;
 
   /**
    * True if this table is marked as cached by hdfs caching. Does not necessarily mean
@@ -373,6 +373,20 @@ public class LocalFsTable extends LocalTable implements FeFsTable {
     return false;
   }
 
+  protected void setAvroSchema(Table msTbl) {
+    if (avroSchema_ == null) {
+      // No Avro schema was explicitly set in the table metadata, so infer the Avro
+      // schema from the column definitions.
+      Schema inferredSchema = AvroSchemaConverter.convertFieldSchemas(
+          msTbl.getSd().getCols(), msTbl.getDbName() + "." + msTbl.getTableName());
+      avroSchema_ = inferredSchema.toString();
+    }
+  }
+
+  protected String getAvroSchema() {
+    return avroSchema_;
+  }
+
   public LocalFsPartition createPrototypePartition() {
     // The prototype partition should not have a location set in its storage
     // descriptor, or else all inserted files will end up written into the
diff --git a/fe/src/main/java/org/apache/impala/catalog/local/LocalIcebergTable.java b/fe/src/main/java/org/apache/impala/catalog/local/LocalIcebergTable.java
index 219da5e85..b0b2a489f 100644
--- a/fe/src/main/java/org/apache/impala/catalog/local/LocalIcebergTable.java
+++ b/fe/src/main/java/org/apache/impala/catalog/local/LocalIcebergTable.java
@@ -147,6 +147,7 @@ public class LocalIcebergTable extends LocalTable implements FeIcebergTable {
         tableInfo.getIceberg_table().getContent_files(),
         tableInfo.getNetwork_addresses(),
         getHostIndex());
+    if (fileStore_.hasAvro()) localFsTable_.setAvroSchema(msTable);
     icebergApiTable_ = icebergApiTable;
     catalogSnapshotId_ = tableInfo.getIceberg_table().getCatalog_snapshot_id();
     partitionSpecs_ = Utils.loadPartitionSpecByIceberg(this);
@@ -273,6 +274,7 @@ public class LocalIcebergTable extends LocalTable implements FeIcebergTable {
     THdfsTable hdfsTable = new THdfsTable(localFsTable_.getHdfsBaseDir(),
         getColumnNames(), localFsTable_.getNullPartitionKeyValue(),
         FeFsTable.DEFAULT_NULL_COLUMN_VALUE, idToPartition, tPrototypePartition);
+    hdfsTable.setAvroSchema(localFsTable_.getAvroSchema());
     Utils.updateIcebergPartitionFileFormat(this, hdfsTable);
     hdfsTable.setPartition_prefixes(localFsTable_.getPartitionPrefixes());
     return hdfsTable;
diff --git a/fe/src/main/java/org/apache/impala/planner/IcebergScanNode.java b/fe/src/main/java/org/apache/impala/planner/IcebergScanNode.java
index c023c417d..d85ada46c 100644
--- a/fe/src/main/java/org/apache/impala/planner/IcebergScanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/IcebergScanNode.java
@@ -58,7 +58,7 @@ public class IcebergScanNode extends HdfsScanNode {
     Preconditions.checkState(partitions_.size() == 1);
 
     fileDescs_ = fileDescs;
-
+    //TODO IMPALA-11577: optimize file format counting
     boolean hasParquet = false;
     boolean hasOrc = false;
     boolean hasAvro = false;
@@ -77,7 +77,16 @@ public class IcebergScanNode extends HdfsScanNode {
     }
     if (hasParquet) fileFormats_.add(HdfsFileFormat.PARQUET);
     if (hasOrc) fileFormats_.add(HdfsFileFormat.ORC);
-    if (hasAvro) fileFormats_.add(HdfsFileFormat.AVRO);
+
+    //TODO IMPALA-11708: Currently mixed file format Iceberg tables containing AVRO files
+    // cannot be read.
+    if (hasAvro) {
+      fileFormats_.add(HdfsFileFormat.AVRO);
+      if (hasOrc || hasParquet) {
+        throw new ImpalaRuntimeException("Iceberg tables containing multiple file "
+            + "formats are only supported if they do not contain AVRO files.");
+      }
+    }
   }
 
   /**
diff --git a/fe/src/main/java/org/apache/impala/util/IcebergUtil.java b/fe/src/main/java/org/apache/impala/util/IcebergUtil.java
index b7d8645d7..3f9dc7314 100644
--- a/fe/src/main/java/org/apache/impala/util/IcebergUtil.java
+++ b/fe/src/main/java/org/apache/impala/util/IcebergUtil.java
@@ -326,6 +326,8 @@ public class IcebergUtil {
       return TIcebergFileFormat.PARQUET;
     } else if ("ORC".equalsIgnoreCase(format)) {
       return TIcebergFileFormat.ORC;
+    } else if ("AVRO".equalsIgnoreCase(format)) {
+      return TIcebergFileFormat.AVRO;
     }
     return null;
   }
@@ -507,6 +509,8 @@ public class IcebergUtil {
     switch (format) {
       case ORC:
         return THdfsFileFormat.ORC;
+      case AVRO:
+        return THdfsFileFormat.AVRO;
       case PARQUET:
       default:
         return THdfsFileFormat.PARQUET;
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/data/00000-0-data-noemi_20221026130844_b228ff88-5625-494b-b27a-7819aad52ced-job_16629766502890_0016-1-00001.avro b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/data/00000-0-data-noemi_20221026130844_b228ff88-5625-494b-b27a-7819aad52ced-job_16629766502890_0016-1-00001.avro
new file mode 100644
index 000000000..5c71b0b7b
Binary files /dev/null and b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/data/00000-0-data-noemi_20221026130844_b228ff88-5625-494b-b27a-7819aad52ced-job_16629766502890_0016-1-00001.avro differ
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/data/00000-0-data-noemi_20221028111610_c7e89043-49e0-40fe-95a5-bf24d958ebc7-job_16629766502890_0017-1-00001.avro b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/data/00000-0-data-noemi_20221028111610_c7e89043-49e0-40fe-95a5-bf24d958ebc7-job_16629766502890_0017-1-00001.avro
new file mode 100644
index 000000000..623b428f4
Binary files /dev/null and b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/data/00000-0-data-noemi_20221028111610_c7e89043-49e0-40fe-95a5-bf24d958ebc7-job_16629766502890_0017-1-00001.avro differ
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/data/00000-0-data-noemi_20221028113321_fbfa5f31-421d-406a-9d46-6bec36d7a93c-job_16629766502890_0018-1-00001.orc b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/data/00000-0-data-noemi_20221028113321_fbfa5f31-421d-406a-9d46-6bec36d7a93c-job_16629766502890_0018-1-00001.orc
new file mode 100644
index 000000000..09837e843
Binary files /dev/null and b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/data/00000-0-data-noemi_20221028113321_fbfa5f31-421d-406a-9d46-6bec36d7a93c-job_16629766502890_0018-1-00001.orc differ
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/data/00000-0-data-noemi_20221028114730_e2f7d99d-7ad8-478c-a814-19e2d7912ad1-job_16629766502890_0019-1-00001.parquet b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/data/00000-0-data-noemi_20221028114730_e2f7d99d-7ad8-478c-a814-19e2d7912ad1-job_16629766502890_0019-1-00001.parquet
new file mode 100644
index 000000000..d927f7882
Binary files /dev/null and b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/data/00000-0-data-noemi_20221028114730_e2f7d99d-7ad8-478c-a814-19e2d7912ad1-job_16629766502890_0019-1-00001.parquet differ
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/13c55017-b018-4ccb-a407-08e37e28eec8-m0.avro b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/13c55017-b018-4ccb-a407-08e37e28eec8-m0.avro
new file mode 100644
index 000000000..b799ffed5
Binary files /dev/null and b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/13c55017-b018-4ccb-a407-08e37e28eec8-m0.avro differ
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/7b422180-e3f8-4500-b240-1424ef012246-m0.avro b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/7b422180-e3f8-4500-b240-1424ef012246-m0.avro
new file mode 100644
index 000000000..e09e5242f
Binary files /dev/null and b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/7b422180-e3f8-4500-b240-1424ef012246-m0.avro differ
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/80a79f8a-5a47-44c9-b16d-4bef4a5ecec3-m0.avro b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/80a79f8a-5a47-44c9-b16d-4bef4a5ecec3-m0.avro
new file mode 100644
index 000000000..bde2cf8b7
Binary files /dev/null and b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/80a79f8a-5a47-44c9-b16d-4bef4a5ecec3-m0.avro differ
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/8e66c338-5cd3-4b85-b986-18ec29b67d94-m0.avro b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/8e66c338-5cd3-4b85-b986-18ec29b67d94-m0.avro
new file mode 100644
index 000000000..c74e2096e
Binary files /dev/null and b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/8e66c338-5cd3-4b85-b986-18ec29b67d94-m0.avro differ
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/snap-1131576191504541058-1-8e66c338-5cd3-4b85-b986-18ec29b67d94.avro b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/snap-1131576191504541058-1-8e66c338-5cd3-4b85-b986-18ec29b67d94.avro
new file mode 100644
index 000000000..8616a6aaf
Binary files /dev/null and b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/snap-1131576191504541058-1-8e66c338-5cd3-4b85-b986-18ec29b67d94.avro differ
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/snap-1744181916149214787-1-13c55017-b018-4ccb-a407-08e37e28eec8.avro b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/snap-1744181916149214787-1-13c55017-b018-4ccb-a407-08e37e28eec8.avro
new file mode 100644
index 000000000..f0f2d3b07
Binary files /dev/null and b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/snap-1744181916149214787-1-13c55017-b018-4ccb-a407-08e37e28eec8.avro differ
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/snap-3243718219085059034-1-7b422180-e3f8-4500-b240-1424ef012246.avro b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/snap-3243718219085059034-1-7b422180-e3f8-4500-b240-1424ef012246.avro
new file mode 100644
index 000000000..84d945b5f
Binary files /dev/null and b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/snap-3243718219085059034-1-7b422180-e3f8-4500-b240-1424ef012246.avro differ
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/snap-5089000375160183133-1-80a79f8a-5a47-44c9-b16d-4bef4a5ecec3.avro b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/snap-5089000375160183133-1-80a79f8a-5a47-44c9-b16d-4bef4a5ecec3.avro
new file mode 100644
index 000000000..81bad5b85
Binary files /dev/null and b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/snap-5089000375160183133-1-80a79f8a-5a47-44c9-b16d-4bef4a5ecec3.avro differ
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/v1.metadata.json b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/v1.metadata.json
new file mode 100644
index 000000000..af0442032
--- /dev/null
+++ b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/v1.metadata.json
@@ -0,0 +1,82 @@
+{
+  "format-version" : 1,
+  "table-uuid" : "a7acd8de-5471-4015-88aa-d861d75f7967",
+  "location" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed",
+  "last-updated-ms" : 1666948270383,
+  "last-column-id" : 4,
+  "schema" : {
+    "type" : "struct",
+    "schema-id" : 0,
+    "fields" : [ {
+      "id" : 1,
+      "name" : "int_col",
+      "required" : false,
+      "type" : "int"
+    }, {
+      "id" : 2,
+      "name" : "string_col",
+      "required" : false,
+      "type" : "string"
+    }, {
+      "id" : 3,
+      "name" : "double_col",
+      "required" : false,
+      "type" : "double"
+    }, {
+      "id" : 4,
+      "name" : "bool_col",
+      "required" : false,
+      "type" : "boolean"
+    } ]
+  },
+  "current-schema-id" : 0,
+  "schemas" : [ {
+    "type" : "struct",
+    "schema-id" : 0,
+    "fields" : [ {
+      "id" : 1,
+      "name" : "int_col",
+      "required" : false,
+      "type" : "int"
+    }, {
+      "id" : 2,
+      "name" : "string_col",
+      "required" : false,
+      "type" : "string"
+    }, {
+      "id" : 3,
+      "name" : "double_col",
+      "required" : false,
+      "type" : "double"
+    }, {
+      "id" : 4,
+      "name" : "bool_col",
+      "required" : false,
+      "type" : "boolean"
+    } ]
+  } ],
+  "partition-spec" : [ ],
+  "default-spec-id" : 0,
+  "partition-specs" : [ {
+    "spec-id" : 0,
+    "fields" : [ ]
+  } ],
+  "last-partition-id" : 999,
+  "default-sort-order-id" : 0,
+  "sort-orders" : [ {
+    "order-id" : 0,
+    "fields" : [ ]
+  } ],
+  "properties" : {
+    "engine.hive.enabled" : "true",
+    "bucketing_version" : "2",
+    "EXTERNAL" : "TRUE",
+    "serialization.format" : "1",
+    "storage_handler" : "org.apache.iceberg.mr.hive.HiveIcebergStorageHandler"
+  },
+  "current-snapshot-id" : -1,
+  "refs" : { },
+  "snapshots" : [ ],
+  "snapshot-log" : [ ],
+  "metadata-log" : [ ]
+}
\ No newline at end of file
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/v2.metadata.json b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/v2.metadata.json
new file mode 100644
index 000000000..611e1e9b5
--- /dev/null
+++ b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/v2.metadata.json
@@ -0,0 +1,83 @@
+{
+  "format-version" : 1,
+  "table-uuid" : "c84d8735-8499-4240-aeaf-63a1e58a7c9a",
+  "location" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed",
+  "last-updated-ms" : 1666782261541,
+  "last-column-id" : 4,
+  "schema" : {
+    "type" : "struct",
+    "schema-id" : 0,
+    "fields" : [ {
+      "id" : 1,
+      "name" : "int_col",
+      "required" : false,
+      "type" : "int"
+    }, {
+      "id" : 2,
+      "name" : "string_col",
+      "required" : false,
+      "type" : "string"
+    }, {
+      "id" : 3,
+      "name" : "double_col",
+      "required" : false,
+      "type" : "double"
+    }, {
+      "id" : 4,
+      "name" : "bool_col",
+      "required" : false,
+      "type" : "boolean"
+    } ]
+  },
+  "current-schema-id" : 0,
+  "schemas" : [ {
+    "type" : "struct",
+    "schema-id" : 0,
+    "fields" : [ {
+      "id" : 1,
+      "name" : "int_col",
+      "required" : false,
+      "type" : "int"
+    }, {
+      "id" : 2,
+      "name" : "string_col",
+      "required" : false,
+      "type" : "string"
+    }, {
+      "id" : 3,
+      "name" : "double_col",
+      "required" : false,
+      "type" : "double"
+    }, {
+      "id" : 4,
+      "name" : "bool_col",
+      "required" : false,
+      "type" : "boolean"
+    } ]
+  } ],
+  "partition-spec" : [ ],
+  "default-spec-id" : 0,
+  "partition-specs" : [ {
+    "spec-id" : 0,
+    "fields" : [ ]
+  } ],
+  "last-partition-id" : 999,
+  "default-sort-order-id" : 0,
+  "sort-orders" : [ {
+    "order-id" : 0,
+    "fields" : [ ]
+  } ],
+  "properties" : {
+    "engine.hive.enabled" : "true",
+    "EXTERNAL" : "TRUE",
+    "write.format.default" : "avro",
+    "bucketing_version" : "2",
+    "serialization.format" : "1",
+    "storage_handler" : "org.apache.iceberg.mr.hive.HiveIcebergStorageHandler"
+  },
+  "current-snapshot-id" : -1,
+  "refs" : { },
+  "snapshots" : [ ],
+  "snapshot-log" : [ ],
+  "metadata-log" : [ ]
+}
\ No newline at end of file
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/v3.metadata.json b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/v3.metadata.json
new file mode 100644
index 000000000..6fad5e1dd
--- /dev/null
+++ b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/v3.metadata.json
@@ -0,0 +1,83 @@
+{
+  "format-version" : 1,
+  "table-uuid" : "ab152578-db4a-493a-8796-5dc9e3e36004",
+  "location" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed",
+  "last-updated-ms" : 1666948531175,
+  "last-column-id" : 4,
+  "schema" : {
+    "type" : "struct",
+    "schema-id" : 0,
+    "fields" : [ {
+      "id" : 1,
+      "name" : "int_col",
+      "required" : false,
+      "type" : "int"
+    }, {
+      "id" : 2,
+      "name" : "string_col",
+      "required" : false,
+      "type" : "string"
+    }, {
+      "id" : 3,
+      "name" : "double_col",
+      "required" : false,
+      "type" : "double"
+    }, {
+      "id" : 4,
+      "name" : "bool_col",
+      "required" : false,
+      "type" : "boolean"
+    } ]
+  },
+  "current-schema-id" : 0,
+  "schemas" : [ {
+    "type" : "struct",
+    "schema-id" : 0,
+    "fields" : [ {
+      "id" : 1,
+      "name" : "int_col",
+      "required" : false,
+      "type" : "int"
+    }, {
+      "id" : 2,
+      "name" : "string_col",
+      "required" : false,
+      "type" : "string"
+    }, {
+      "id" : 3,
+      "name" : "double_col",
+      "required" : false,
+      "type" : "double"
+    }, {
+      "id" : 4,
+      "name" : "bool_col",
+      "required" : false,
+      "type" : "boolean"
+    } ]
+  } ],
+  "partition-spec" : [ ],
+  "default-spec-id" : 0,
+  "partition-specs" : [ {
+    "spec-id" : 0,
+    "fields" : [ ]
+  } ],
+  "last-partition-id" : 999,
+  "default-sort-order-id" : 0,
+  "sort-orders" : [ {
+    "order-id" : 0,
+    "fields" : [ ]
+  } ],
+  "properties" : {
+    "engine.hive.enabled" : "true",
+    "EXTERNAL" : "TRUE",
+    "write.format.default" : "avro",
+    "bucketing_version" : "2",
+    "serialization.format" : "1",
+    "storage_handler" : "org.apache.iceberg.mr.hive.HiveIcebergStorageHandler"
+  },
+  "current-snapshot-id" : -1,
+  "refs" : { },
+  "snapshots" : [ ],
+  "snapshot-log" : [ ],
+  "metadata-log" : [ ]
+}
\ No newline at end of file
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/v4.metadata.json b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/v4.metadata.json
new file mode 100644
index 000000000..782a388ac
--- /dev/null
+++ b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/v4.metadata.json
@@ -0,0 +1,112 @@
+{
+  "format-version" : 1,
+  "table-uuid" : "ab152578-db4a-493a-8796-5dc9e3e36004",
+  "location" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed",
+  "last-updated-ms" : 1666948579101,
+  "last-column-id" : 4,
+  "schema" : {
+    "type" : "struct",
+    "schema-id" : 0,
+    "fields" : [ {
+      "id" : 1,
+      "name" : "int_col",
+      "required" : false,
+      "type" : "int"
+    }, {
+      "id" : 2,
+      "name" : "string_col",
+      "required" : false,
+      "type" : "string"
+    }, {
+      "id" : 3,
+      "name" : "double_col",
+      "required" : false,
+      "type" : "double"
+    }, {
+      "id" : 4,
+      "name" : "bool_col",
+      "required" : false,
+      "type" : "boolean"
+    } ]
+  },
+  "current-schema-id" : 0,
+  "schemas" : [ {
+    "type" : "struct",
+    "schema-id" : 0,
+    "fields" : [ {
+      "id" : 1,
+      "name" : "int_col",
+      "required" : false,
+      "type" : "int"
+    }, {
+      "id" : 2,
+      "name" : "string_col",
+      "required" : false,
+      "type" : "string"
+    }, {
+      "id" : 3,
+      "name" : "double_col",
+      "required" : false,
+      "type" : "double"
+    }, {
+      "id" : 4,
+      "name" : "bool_col",
+      "required" : false,
+      "type" : "boolean"
+    } ]
+  } ],
+  "partition-spec" : [ ],
+  "default-spec-id" : 0,
+  "partition-specs" : [ {
+    "spec-id" : 0,
+    "fields" : [ ]
+  } ],
+  "last-partition-id" : 999,
+  "default-sort-order-id" : 0,
+  "sort-orders" : [ {
+    "order-id" : 0,
+    "fields" : [ ]
+  } ],
+  "properties" : {
+    "engine.hive.enabled" : "true",
+    "EXTERNAL" : "TRUE",
+    "write.format.default" : "avro",
+    "bucketing_version" : "2",
+    "serialization.format" : "1",
+    "storage_handler" : "org.apache.iceberg.mr.hive.HiveIcebergStorageHandler"
+  },
+  "current-snapshot-id" : 3243718219085059034,
+  "refs" : {
+    "main" : {
+      "snapshot-id" : 3243718219085059034,
+      "type" : "branch"
+    }
+  },
+  "snapshots" : [ {
+    "snapshot-id" : 3243718219085059034,
+    "timestamp-ms" : 1666948579101,
+    "summary" : {
+      "operation" : "append",
+      "added-data-files" : "1",
+      "added-records" : "1",
+      "added-files-size" : "723",
+      "changed-partition-count" : "1",
+      "total-records" : "1",
+      "total-files-size" : "723",
+      "total-data-files" : "1",
+      "total-delete-files" : "0",
+      "total-position-deletes" : "0",
+      "total-equality-deletes" : "0"
+    },
+    "manifest-list" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/snap-3243718219085059034-1-7b422180-e3f8-4500-b240-1424ef012246.avro",
+    "schema-id" : 0
+  } ],
+  "snapshot-log" : [ {
+    "timestamp-ms" : 1666948579101,
+    "snapshot-id" : 3243718219085059034
+  } ],
+  "metadata-log" : [ {
+    "timestamp-ms" : 1666948531175,
+    "metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/00000-58a6b29d-3a84-4192-bb65-8382a5222c1a.metadata.json"
+  } ]
+}
\ No newline at end of file
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/v5.metadata.json b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/v5.metadata.json
new file mode 100644
index 000000000..6d7eb6b6e
--- /dev/null
+++ b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/v5.metadata.json
@@ -0,0 +1,112 @@
+{
+  "format-version" : 1,
+  "table-uuid" : "c84d8735-8499-4240-aeaf-63a1e58a7c9a",
+  "location" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed",
+  "last-updated-ms" : 1666782533443,
+  "last-column-id" : 4,
+  "schema" : {
+    "type" : "struct",
+    "schema-id" : 0,
+    "fields" : [ {
+      "id" : 1,
+      "name" : "int_col",
+      "required" : false,
+      "type" : "int"
+    }, {
+      "id" : 2,
+      "name" : "string_col",
+      "required" : false,
+      "type" : "string"
+    }, {
+      "id" : 3,
+      "name" : "double_col",
+      "required" : false,
+      "type" : "double"
+    }, {
+      "id" : 4,
+      "name" : "bool_col",
+      "required" : false,
+      "type" : "boolean"
+    } ]
+  },
+  "current-schema-id" : 0,
+  "schemas" : [ {
+    "type" : "struct",
+    "schema-id" : 0,
+    "fields" : [ {
+      "id" : 1,
+      "name" : "int_col",
+      "required" : false,
+      "type" : "int"
+    }, {
+      "id" : 2,
+      "name" : "string_col",
+      "required" : false,
+      "type" : "string"
+    }, {
+      "id" : 3,
+      "name" : "double_col",
+      "required" : false,
+      "type" : "double"
+    }, {
+      "id" : 4,
+      "name" : "bool_col",
+      "required" : false,
+      "type" : "boolean"
+    } ]
+  } ],
+  "partition-spec" : [ ],
+  "default-spec-id" : 0,
+  "partition-specs" : [ {
+    "spec-id" : 0,
+    "fields" : [ ]
+  } ],
+  "last-partition-id" : 999,
+  "default-sort-order-id" : 0,
+  "sort-orders" : [ {
+    "order-id" : 0,
+    "fields" : [ ]
+  } ],
+  "properties" : {
+    "engine.hive.enabled" : "true",
+    "EXTERNAL" : "TRUE",
+    "write.format.default" : "avro",
+    "bucketing_version" : "2",
+    "serialization.format" : "1",
+    "storage_handler" : "org.apache.iceberg.mr.hive.HiveIcebergStorageHandler"
+  },
+  "current-snapshot-id" : 1744181916149214787,
+  "refs" : {
+    "main" : {
+      "snapshot-id" : 1744181916149214787,
+      "type" : "branch"
+    }
+  },
+  "snapshots" : [ {
+    "snapshot-id" : 1744181916149214787,
+    "timestamp-ms" : 1666782533443,
+    "summary" : {
+      "operation" : "append",
+      "added-data-files" : "1",
+      "added-records" : "1",
+      "added-files-size" : "723",
+      "changed-partition-count" : "1",
+      "total-records" : "1",
+      "total-files-size" : "723",
+      "total-data-files" : "1",
+      "total-delete-files" : "0",
+      "total-position-deletes" : "0",
+      "total-equality-deletes" : "0"
+    },
+    "manifest-list" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/snap-1744181916149214787-1-13c55017-b018-4ccb-a407-08e37e28eec8.avro",
+    "schema-id" : 0
+  } ],
+  "snapshot-log" : [ {
+    "timestamp-ms" : 1666782533443,
+    "snapshot-id" : 1744181916149214787
+  } ],
+  "metadata-log" : [ {
+    "timestamp-ms" : 1666782261541,
+    "metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/00000-547c9de5-692a-489f-ab78-751cf8952229.metadata.json"
+  } ]
+}
\ No newline at end of file
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/v6.metadata.json b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/v6.metadata.json
new file mode 100644
index 000000000..39320de7e
--- /dev/null
+++ b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/v6.metadata.json
@@ -0,0 +1,117 @@
+{
+  "format-version" : 1,
+  "table-uuid" : "ab152578-db4a-493a-8796-5dc9e3e36004",
+  "location" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed",
+  "last-updated-ms" : 1666949502040,
+  "last-column-id" : 4,
+  "schema" : {
+    "type" : "struct",
+    "schema-id" : 0,
+    "fields" : [ {
+      "id" : 1,
+      "name" : "int_col",
+      "required" : false,
+      "type" : "int"
+    }, {
+      "id" : 2,
+      "name" : "string_col",
+      "required" : false,
+      "type" : "string"
+    }, {
+      "id" : 3,
+      "name" : "double_col",
+      "required" : false,
+      "type" : "double"
+    }, {
+      "id" : 4,
+      "name" : "bool_col",
+      "required" : false,
+      "type" : "boolean"
+    } ]
+  },
+  "current-schema-id" : 0,
+  "schemas" : [ {
+    "type" : "struct",
+    "schema-id" : 0,
+    "fields" : [ {
+      "id" : 1,
+      "name" : "int_col",
+      "required" : false,
+      "type" : "int"
+    }, {
+      "id" : 2,
+      "name" : "string_col",
+      "required" : false,
+      "type" : "string"
+    }, {
+      "id" : 3,
+      "name" : "double_col",
+      "required" : false,
+      "type" : "double"
+    }, {
+      "id" : 4,
+      "name" : "bool_col",
+      "required" : false,
+      "type" : "boolean"
+    } ]
+  } ],
+  "partition-spec" : [ ],
+  "default-spec-id" : 0,
+  "partition-specs" : [ {
+    "spec-id" : 0,
+    "fields" : [ ]
+  } ],
+  "last-partition-id" : 999,
+  "default-sort-order-id" : 0,
+  "sort-orders" : [ {
+    "order-id" : 0,
+    "fields" : [ ]
+  } ],
+  "properties" : {
+    "engine.hive.enabled" : "true",
+    "last_modified_time" : "1666949501",
+    "EXTERNAL" : "TRUE",
+    "write.format.default" : "orc",
+    "bucketing_version" : "2",
+    "last_modified_by" : "noemi",
+    "serialization.format" : "1",
+    "storage_handler" : "org.apache.iceberg.mr.hive.HiveIcebergStorageHandler"
+  },
+  "current-snapshot-id" : 3243718219085059034,
+  "refs" : {
+    "main" : {
+      "snapshot-id" : 3243718219085059034,
+      "type" : "branch"
+    }
+  },
+  "snapshots" : [ {
+    "snapshot-id" : 3243718219085059034,
+    "timestamp-ms" : 1666948579101,
+    "summary" : {
+      "operation" : "append",
+      "added-data-files" : "1",
+      "added-records" : "1",
+      "added-files-size" : "723",
+      "changed-partition-count" : "1",
+      "total-records" : "1",
+      "total-files-size" : "723",
+      "total-data-files" : "1",
+      "total-delete-files" : "0",
+      "total-position-deletes" : "0",
+      "total-equality-deletes" : "0"
+    },
+    "manifest-list" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/snap-3243718219085059034-1-7b422180-e3f8-4500-b240-1424ef012246.avro",
+    "schema-id" : 0
+  } ],
+  "snapshot-log" : [ {
+    "timestamp-ms" : 1666948579101,
+    "snapshot-id" : 3243718219085059034
+  } ],
+  "metadata-log" : [ {
+    "timestamp-ms" : 1666948531175,
+    "metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/00000-58a6b29d-3a84-4192-bb65-8382a5222c1a.metadata.json"
+  }, {
+    "timestamp-ms" : 1666948579101,
+    "metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/00001-9a74d7ab-e873-4d72-8bd3-25baeb6a1f5c.metadata.json"
+  } ]
+}
\ No newline at end of file
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/v7.metadata.json b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/v7.metadata.json
new file mode 100644
index 000000000..5ea19a21c
--- /dev/null
+++ b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/v7.metadata.json
@@ -0,0 +1,142 @@
+{
+  "format-version" : 1,
+  "table-uuid" : "ab152578-db4a-493a-8796-5dc9e3e36004",
+  "location" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed",
+  "last-updated-ms" : 1666949609311,
+  "last-column-id" : 4,
+  "schema" : {
+    "type" : "struct",
+    "schema-id" : 0,
+    "fields" : [ {
+      "id" : 1,
+      "name" : "int_col",
+      "required" : false,
+      "type" : "int"
+    }, {
+      "id" : 2,
+      "name" : "string_col",
+      "required" : false,
+      "type" : "string"
+    }, {
+      "id" : 3,
+      "name" : "double_col",
+      "required" : false,
+      "type" : "double"
+    }, {
+      "id" : 4,
+      "name" : "bool_col",
+      "required" : false,
+      "type" : "boolean"
+    } ]
+  },
+  "current-schema-id" : 0,
+  "schemas" : [ {
+    "type" : "struct",
+    "schema-id" : 0,
+    "fields" : [ {
+      "id" : 1,
+      "name" : "int_col",
+      "required" : false,
+      "type" : "int"
+    }, {
+      "id" : 2,
+      "name" : "string_col",
+      "required" : false,
+      "type" : "string"
+    }, {
+      "id" : 3,
+      "name" : "double_col",
+      "required" : false,
+      "type" : "double"
+    }, {
+      "id" : 4,
+      "name" : "bool_col",
+      "required" : false,
+      "type" : "boolean"
+    } ]
+  } ],
+  "partition-spec" : [ ],
+  "default-spec-id" : 0,
+  "partition-specs" : [ {
+    "spec-id" : 0,
+    "fields" : [ ]
+  } ],
+  "last-partition-id" : 999,
+  "default-sort-order-id" : 0,
+  "sort-orders" : [ {
+    "order-id" : 0,
+    "fields" : [ ]
+  } ],
+  "properties" : {
+    "engine.hive.enabled" : "true",
+    "last_modified_time" : "1666949501",
+    "EXTERNAL" : "TRUE",
+    "write.format.default" : "orc",
+    "bucketing_version" : "2",
+    "last_modified_by" : "noemi",
+    "serialization.format" : "1",
+    "storage_handler" : "org.apache.iceberg.mr.hive.HiveIcebergStorageHandler"
+  },
+  "current-snapshot-id" : 1131576191504541058,
+  "refs" : {
+    "main" : {
+      "snapshot-id" : 1131576191504541058,
+      "type" : "branch"
+    }
+  },
+  "snapshots" : [ {
+    "snapshot-id" : 3243718219085059034,
+    "timestamp-ms" : 1666948579101,
+    "summary" : {
+      "operation" : "append",
+      "added-data-files" : "1",
+      "added-records" : "1",
+      "added-files-size" : "723",
+      "changed-partition-count" : "1",
+      "total-records" : "1",
+      "total-files-size" : "723",
+      "total-data-files" : "1",
+      "total-delete-files" : "0",
+      "total-position-deletes" : "0",
+      "total-equality-deletes" : "0"
+    },
+    "manifest-list" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/snap-3243718219085059034-1-7b422180-e3f8-4500-b240-1424ef012246.avro",
+    "schema-id" : 0
+  }, {
+    "snapshot-id" : 1131576191504541058,
+    "parent-snapshot-id" : 3243718219085059034,
+    "timestamp-ms" : 1666949609311,
+    "summary" : {
+      "operation" : "append",
+      "added-data-files" : "1",
+      "added-records" : "1",
+      "added-files-size" : "534",
+      "changed-partition-count" : "1",
+      "total-records" : "2",
+      "total-files-size" : "1257",
+      "total-data-files" : "2",
+      "total-delete-files" : "0",
+      "total-position-deletes" : "0",
+      "total-equality-deletes" : "0"
+    },
+    "manifest-list" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/snap-1131576191504541058-1-8e66c338-5cd3-4b85-b986-18ec29b67d94.avro",
+    "schema-id" : 0
+  } ],
+  "snapshot-log" : [ {
+    "timestamp-ms" : 1666948579101,
+    "snapshot-id" : 3243718219085059034
+  }, {
+    "timestamp-ms" : 1666949609311,
+    "snapshot-id" : 1131576191504541058
+  } ],
+  "metadata-log" : [ {
+    "timestamp-ms" : 1666948531175,
+    "metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/00000-58a6b29d-3a84-4192-bb65-8382a5222c1a.metadata.json"
+  }, {
+    "timestamp-ms" : 1666948579101,
+    "metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/00001-9a74d7ab-e873-4d72-8bd3-25baeb6a1f5c.metadata.json"
+  }, {
+    "timestamp-ms" : 1666949502040,
+    "metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/00002-e1863a7b-6666-46df-9015-e78e145e7878.metadata.json"
+  } ]
+}
\ No newline at end of file
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/v8.metadata.json b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/v8.metadata.json
new file mode 100644
index 000000000..44d99a06a
--- /dev/null
+++ b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/v8.metadata.json
@@ -0,0 +1,145 @@
+{
+  "format-version" : 1,
+  "table-uuid" : "ab152578-db4a-493a-8796-5dc9e3e36004",
+  "location" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed",
+  "last-updated-ms" : 1666949895636,
+  "last-column-id" : 4,
+  "schema" : {
+    "type" : "struct",
+    "schema-id" : 0,
+    "fields" : [ {
+      "id" : 1,
+      "name" : "int_col",
+      "required" : false,
+      "type" : "int"
+    }, {
+      "id" : 2,
+      "name" : "string_col",
+      "required" : false,
+      "type" : "string"
+    }, {
+      "id" : 3,
+      "name" : "double_col",
+      "required" : false,
+      "type" : "double"
+    }, {
+      "id" : 4,
+      "name" : "bool_col",
+      "required" : false,
+      "type" : "boolean"
+    } ]
+  },
+  "current-schema-id" : 0,
+  "schemas" : [ {
+    "type" : "struct",
+    "schema-id" : 0,
+    "fields" : [ {
+      "id" : 1,
+      "name" : "int_col",
+      "required" : false,
+      "type" : "int"
+    }, {
+      "id" : 2,
+      "name" : "string_col",
+      "required" : false,
+      "type" : "string"
+    }, {
+      "id" : 3,
+      "name" : "double_col",
+      "required" : false,
+      "type" : "double"
+    }, {
+      "id" : 4,
+      "name" : "bool_col",
+      "required" : false,
+      "type" : "boolean"
+    } ]
+  } ],
+  "partition-spec" : [ ],
+  "default-spec-id" : 0,
+  "partition-specs" : [ {
+    "spec-id" : 0,
+    "fields" : [ ]
+  } ],
+  "last-partition-id" : 999,
+  "default-sort-order-id" : 0,
+  "sort-orders" : [ {
+    "order-id" : 0,
+    "fields" : [ ]
+  } ],
+  "properties" : {
+    "engine.hive.enabled" : "true",
+    "last_modified_time" : "1666949895",
+    "EXTERNAL" : "TRUE",
+    "write.format.default" : "parquet",
+    "bucketing_version" : "2",
+    "last_modified_by" : "noemi",
+    "serialization.format" : "1",
+    "storage_handler" : "org.apache.iceberg.mr.hive.HiveIcebergStorageHandler"
+  },
+  "current-snapshot-id" : 1131576191504541058,
+  "refs" : {
+    "main" : {
+      "snapshot-id" : 1131576191504541058,
+      "type" : "branch"
+    }
+  },
+  "snapshots" : [ {
+    "snapshot-id" : 3243718219085059034,
+    "timestamp-ms" : 1666948579101,
+    "summary" : {
+      "operation" : "append",
+      "added-data-files" : "1",
+      "added-records" : "1",
+      "added-files-size" : "723",
+      "changed-partition-count" : "1",
+      "total-records" : "1",
+      "total-files-size" : "723",
+      "total-data-files" : "1",
+      "total-delete-files" : "0",
+      "total-position-deletes" : "0",
+      "total-equality-deletes" : "0"
+    },
+    "manifest-list" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/snap-3243718219085059034-1-7b422180-e3f8-4500-b240-1424ef012246.avro",
+    "schema-id" : 0
+  }, {
+    "snapshot-id" : 1131576191504541058,
+    "parent-snapshot-id" : 3243718219085059034,
+    "timestamp-ms" : 1666949609311,
+    "summary" : {
+      "operation" : "append",
+      "added-data-files" : "1",
+      "added-records" : "1",
+      "added-files-size" : "534",
+      "changed-partition-count" : "1",
+      "total-records" : "2",
+      "total-files-size" : "1257",
+      "total-data-files" : "2",
+      "total-delete-files" : "0",
+      "total-position-deletes" : "0",
+      "total-equality-deletes" : "0"
+    },
+    "manifest-list" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/snap-1131576191504541058-1-8e66c338-5cd3-4b85-b986-18ec29b67d94.avro",
+    "schema-id" : 0
+  } ],
+  "snapshot-log" : [ {
+    "timestamp-ms" : 1666948579101,
+    "snapshot-id" : 3243718219085059034
+  }, {
+    "timestamp-ms" : 1666949609311,
+    "snapshot-id" : 1131576191504541058
+  } ],
+  "metadata-log" : [ {
+    "timestamp-ms" : 1666948531175,
+    "metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/00000-58a6b29d-3a84-4192-bb65-8382a5222c1a.metadata.json"
+  }, {
+    "timestamp-ms" : 1666948579101,
+    "metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/00001-9a74d7ab-e873-4d72-8bd3-25baeb6a1f5c.metadata.json"
+  }, {
+    "timestamp-ms" : 1666949502040,
+    "metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/00002-e1863a7b-6666-46df-9015-e78e145e7878.metadata.json"
+  }, {
+    "timestamp-ms" : 1666949609311,
+    "metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/00003-5ba0f9e4-4743-48af-918d-252319c4e055.metadata.json"
+  } ]
+}
\ No newline at end of file
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/v9.metadata.json b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/v9.metadata.json
new file mode 100644
index 000000000..2b79f43aa
--- /dev/null
+++ b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/v9.metadata.json
@@ -0,0 +1,170 @@
+{
+  "format-version" : 1,
+  "table-uuid" : "ab152578-db4a-493a-8796-5dc9e3e36004",
+  "location" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed",
+  "last-updated-ms" : 1666950458526,
+  "last-column-id" : 4,
+  "schema" : {
+    "type" : "struct",
+    "schema-id" : 0,
+    "fields" : [ {
+      "id" : 1,
+      "name" : "int_col",
+      "required" : false,
+      "type" : "int"
+    }, {
+      "id" : 2,
+      "name" : "string_col",
+      "required" : false,
+      "type" : "string"
+    }, {
+      "id" : 3,
+      "name" : "double_col",
+      "required" : false,
+      "type" : "double"
+    }, {
+      "id" : 4,
+      "name" : "bool_col",
+      "required" : false,
+      "type" : "boolean"
+    } ]
+  },
+  "current-schema-id" : 0,
+  "schemas" : [ {
+    "type" : "struct",
+    "schema-id" : 0,
+    "fields" : [ {
+      "id" : 1,
+      "name" : "int_col",
+      "required" : false,
+      "type" : "int"
+    }, {
+      "id" : 2,
+      "name" : "string_col",
+      "required" : false,
+      "type" : "string"
+    }, {
+      "id" : 3,
+      "name" : "double_col",
+      "required" : false,
+      "type" : "double"
+    }, {
+      "id" : 4,
+      "name" : "bool_col",
+      "required" : false,
+      "type" : "boolean"
+    } ]
+  } ],
+  "partition-spec" : [ ],
+  "default-spec-id" : 0,
+  "partition-specs" : [ {
+    "spec-id" : 0,
+    "fields" : [ ]
+  } ],
+  "last-partition-id" : 999,
+  "default-sort-order-id" : 0,
+  "sort-orders" : [ {
+    "order-id" : 0,
+    "fields" : [ ]
+  } ],
+  "properties" : {
+    "engine.hive.enabled" : "true",
+    "last_modified_time" : "1666949895",
+    "EXTERNAL" : "TRUE",
+    "write.format.default" : "parquet",
+    "bucketing_version" : "2",
+    "last_modified_by" : "noemi",
+    "serialization.format" : "1",
+    "storage_handler" : "org.apache.iceberg.mr.hive.HiveIcebergStorageHandler"
+  },
+  "current-snapshot-id" : 5089000375160183133,
+  "refs" : {
+    "main" : {
+      "snapshot-id" : 5089000375160183133,
+      "type" : "branch"
+    }
+  },
+  "snapshots" : [ {
+    "snapshot-id" : 3243718219085059034,
+    "timestamp-ms" : 1666948579101,
+    "summary" : {
+      "operation" : "append",
+      "added-data-files" : "1",
+      "added-records" : "1",
+      "added-files-size" : "723",
+      "changed-partition-count" : "1",
+      "total-records" : "1",
+      "total-files-size" : "723",
+      "total-data-files" : "1",
+      "total-delete-files" : "0",
+      "total-position-deletes" : "0",
+      "total-equality-deletes" : "0"
+    },
+    "manifest-list" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/snap-3243718219085059034-1-7b422180-e3f8-4500-b240-1424ef012246.avro",
+    "schema-id" : 0
+  }, {
+    "snapshot-id" : 1131576191504541058,
+    "parent-snapshot-id" : 3243718219085059034,
+    "timestamp-ms" : 1666949609311,
+    "summary" : {
+      "operation" : "append",
+      "added-data-files" : "1",
+      "added-records" : "1",
+      "added-files-size" : "534",
+      "changed-partition-count" : "1",
+      "total-records" : "2",
+      "total-files-size" : "1257",
+      "total-data-files" : "2",
+      "total-delete-files" : "0",
+      "total-position-deletes" : "0",
+      "total-equality-deletes" : "0"
+    },
+    "manifest-list" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/snap-1131576191504541058-1-8e66c338-5cd3-4b85-b986-18ec29b67d94.avro",
+    "schema-id" : 0
+  }, {
+    "snapshot-id" : 5089000375160183133,
+    "parent-snapshot-id" : 1131576191504541058,
+    "timestamp-ms" : 1666950458526,
+    "summary" : {
+      "operation" : "append",
+      "added-data-files" : "1",
+      "added-records" : "1",
+      "added-files-size" : "1109",
+      "changed-partition-count" : "1",
+      "total-records" : "3",
+      "total-files-size" : "2366",
+      "total-data-files" : "3",
+      "total-delete-files" : "0",
+      "total-position-deletes" : "0",
+      "total-equality-deletes" : "0"
+    },
+    "manifest-list" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/snap-5089000375160183133-1-80a79f8a-5a47-44c9-b16d-4bef4a5ecec3.avro",
+    "schema-id" : 0
+  } ],
+  "snapshot-log" : [ {
+    "timestamp-ms" : 1666948579101,
+    "snapshot-id" : 3243718219085059034
+  }, {
+    "timestamp-ms" : 1666949609311,
+    "snapshot-id" : 1131576191504541058
+  }, {
+    "timestamp-ms" : 1666950458526,
+    "snapshot-id" : 5089000375160183133
+  } ],
+  "metadata-log" : [ {
+    "timestamp-ms" : 1666948531175,
+    "metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/00000-58a6b29d-3a84-4192-bb65-8382a5222c1a.metadata.json"
+  }, {
+    "timestamp-ms" : 1666948579101,
+    "metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/00001-9a74d7ab-e873-4d72-8bd3-25baeb6a1f5c.metadata.json"
+  }, {
+    "timestamp-ms" : 1666949502040,
+    "metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/00002-e1863a7b-6666-46df-9015-e78e145e7878.metadata.json"
+  }, {
+    "timestamp-ms" : 1666949609311,
+    "metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/00003-5ba0f9e4-4743-48af-918d-252319c4e055.metadata.json"
+  }, {
+    "timestamp-ms" : 1666949895636,
+    "metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/00004-e680e6e3-1dbd-4f48-b3d0-e676350ade56.metadata.json"
+  } ]
+}
\ No newline at end of file
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/version-hint.txt b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/version-hint.txt
new file mode 100644
index 000000000..ec635144f
--- /dev/null
+++ b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_mixed/metadata/version-hint.txt
@@ -0,0 +1 @@
+9
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_only/data/00000-0-data-noemi_20221021195331_77fbb37f-2393-4a66-9656-61cd56b94b46-job_16629766502890_0015-1-00001.avro b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_only/data/00000-0-data-noemi_20221021195331_77fbb37f-2393-4a66-9656-61cd56b94b46-job_16629766502890_0015-1-00001.avro
new file mode 100644
index 000000000..2a454d5f3
Binary files /dev/null and b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_only/data/00000-0-data-noemi_20221021195331_77fbb37f-2393-4a66-9656-61cd56b94b46-job_16629766502890_0015-1-00001.avro differ
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_only/metadata/a9f8d35c-a852-49fe-996a-d94ae1896c32-m0.avro b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_only/metadata/a9f8d35c-a852-49fe-996a-d94ae1896c32-m0.avro
new file mode 100644
index 000000000..bd8d76fbd
Binary files /dev/null and b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_only/metadata/a9f8d35c-a852-49fe-996a-d94ae1896c32-m0.avro differ
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_only/metadata/snap-725782911885631732-1-a9f8d35c-a852-49fe-996a-d94ae1896c32.avro b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_only/metadata/snap-725782911885631732-1-a9f8d35c-a852-49fe-996a-d94ae1896c32.avro
new file mode 100644
index 000000000..799d0f2b9
Binary files /dev/null and b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_only/metadata/snap-725782911885631732-1-a9f8d35c-a852-49fe-996a-d94ae1896c32.avro differ
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_only/metadata/v1.metadata.json b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_only/metadata/v1.metadata.json
new file mode 100644
index 000000000..d1fa4e378
--- /dev/null
+++ b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_only/metadata/v1.metadata.json
@@ -0,0 +1,72 @@
+{
+  "format-version" : 1,
+  "table-uuid" : "a7d11596-b69a-4cb8-ba17-faef632ba9ec",
+  "location" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_avro_only",
+  "last-updated-ms" : 1666374749642,
+  "last-column-id" : 3,
+  "schema" : {
+    "type" : "struct",
+    "schema-id" : 0,
+    "fields" : [ {
+      "id" : 1,
+      "name" : "int_col",
+      "required" : false,
+      "type" : "int"
+    }, {
+      "id" : 2,
+      "name" : "string_col",
+      "required" : false,
+      "type" : "string"
+    }, {
+      "id" : 3,
+      "name" : "double_col",
+      "required" : false,
+      "type" : "double"
+    } ]
+  },
+  "current-schema-id" : 0,
+  "schemas" : [ {
+    "type" : "struct",
+    "schema-id" : 0,
+    "fields" : [ {
+      "id" : 1,
+      "name" : "int_col",
+      "required" : false,
+      "type" : "int"
+    }, {
+      "id" : 2,
+      "name" : "string_col",
+      "required" : false,
+      "type" : "string"
+    }, {
+      "id" : 3,
+      "name" : "double_col",
+      "required" : false,
+      "type" : "double"
+    } ]
+  } ],
+  "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",
+    "bucketing_version" : "2",
+    "serialization.format" : "1",
+    "storage_handler" : "org.apache.iceberg.mr.hive.HiveIcebergStorageHandler",
+    "write.format.default" : "avro"
+  },
+  "current-snapshot-id" : -1,
+  "refs" : { },
+  "snapshots" : [ ],
+  "snapshot-log" : [ ],
+  "metadata-log" : [ ]
+}
\ No newline at end of file
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_only/metadata/v2.metadata.json b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_only/metadata/v2.metadata.json
new file mode 100644
index 000000000..dbd8868a0
--- /dev/null
+++ b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_only/metadata/v2.metadata.json
@@ -0,0 +1,101 @@
+{
+  "format-version" : 1,
+  "table-uuid" : "a7d11596-b69a-4cb8-ba17-faef632ba9ec",
+  "location" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_avro_only",
+  "last-updated-ms" : 1666374820147,
+  "last-column-id" : 3,
+  "schema" : {
+    "type" : "struct",
+    "schema-id" : 0,
+    "fields" : [ {
+      "id" : 1,
+      "name" : "int_col",
+      "required" : false,
+      "type" : "int"
+    }, {
+      "id" : 2,
+      "name" : "string_col",
+      "required" : false,
+      "type" : "string"
+    }, {
+      "id" : 3,
+      "name" : "double_col",
+      "required" : false,
+      "type" : "double"
+    } ]
+  },
+  "current-schema-id" : 0,
+  "schemas" : [ {
+    "type" : "struct",
+    "schema-id" : 0,
+    "fields" : [ {
+      "id" : 1,
+      "name" : "int_col",
+      "required" : false,
+      "type" : "int"
+    }, {
+      "id" : 2,
+      "name" : "string_col",
+      "required" : false,
+      "type" : "string"
+    }, {
+      "id" : 3,
+      "name" : "double_col",
+      "required" : false,
+      "type" : "double"
+    } ]
+  } ],
+  "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",
+    "bucketing_version" : "2",
+    "serialization.format" : "1",
+    "storage_handler" : "org.apache.iceberg.mr.hive.HiveIcebergStorageHandler",
+    "write.format.default" : "avro"
+  },
+  "current-snapshot-id" : 725782911885631732,
+  "refs" : {
+    "main" : {
+      "snapshot-id" : 725782911885631732,
+      "type" : "branch"
+    }
+  },
+  "snapshots" : [ {
+    "snapshot-id" : 725782911885631732,
+    "timestamp-ms" : 1666374820147,
+    "summary" : {
+      "operation" : "append",
+      "added-data-files" : "1",
+      "added-records" : "3",
+      "added-files-size" : "604",
+      "changed-partition-count" : "1",
+      "total-records" : "3",
+      "total-files-size" : "604",
+      "total-data-files" : "1",
+      "total-delete-files" : "0",
+      "total-position-deletes" : "0",
+      "total-equality-deletes" : "0"
+    },
+    "manifest-list" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_avro_only/metadata/snap-725782911885631732-1-a9f8d35c-a852-49fe-996a-d94ae1896c32.avro",
+    "schema-id" : 0
+  } ],
+  "snapshot-log" : [ {
+    "timestamp-ms" : 1666374820147,
+    "snapshot-id" : 725782911885631732
+  } ],
+  "metadata-log" : [ {
+    "timestamp-ms" : 1666374749642,
+    "metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_avro_only/metadata/00000-6f456c5b-0115-4a6b-8e66-2577ab899c87.metadata.json"
+  } ]
+}
\ No newline at end of file
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_only/metadata/version-hint.txt b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_only/metadata/version-hint.txt
new file mode 100644
index 000000000..0cfbf0888
--- /dev/null
+++ b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_only/metadata/version-hint.txt
@@ -0,0 +1 @@
+2
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_only/version-hint.txt b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_only/version-hint.txt
new file mode 100644
index 000000000..d00491fd7
--- /dev/null
+++ b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_avro_only/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 d2a0941bd..5f52e6d51 100644
--- a/testdata/datasets/functional/functional_schema_template.sql
+++ b/testdata/datasets/functional/functional_schema_template.sql
@@ -3603,6 +3603,47 @@ TBLPROPERTIES('iceberg.catalog'='hadoop.catalog',
 ---- 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_v2_partitioned_position_deletes_orc /test-warehouse/iceberg_test/hadoop_catalog/ice
+
+====
+---- DATASET
+functional
+---- BASE_TABLE_NAME
+iceberg_avro_only
+---- CREATE
+CREATE EXTERNAL TABLE IF NOT EXISTS {db_name}{db_suffix}.{table_name} (
+  int_col int,
+  string_col string,
+  double_col double
+)
+STORED AS ICEBERG
+TBLPROPERTIES('iceberg.catalog'='hadoop.catalog',
+              'iceberg.catalog_location'='/test-warehouse/iceberg_test/hadoop_catalog',
+              'iceberg.table_identifier'='ice.iceberg_avro_only',
+              'write.format.default'='avro');
+---- 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_avro_only /test-warehouse/iceberg_test/hadoop_catalog/ice
+
+====
+---- DATASET
+functional
+---- BASE_TABLE_NAME
+iceberg_avro_mixed
+---- CREATE
+CREATE EXTERNAL TABLE IF NOT EXISTS {db_name}{db_suffix}.{table_name} (
+  int_col int,
+  string_col string,
+  double_col double,
+  bool_col boolean
+)
+STORED AS ICEBERG
+TBLPROPERTIES('iceberg.catalog'='hadoop.catalog',
+              'iceberg.catalog_location'='/test-warehouse/iceberg_test/hadoop_catalog',
+              'iceberg.table_identifier'='ice.iceberg_avro_mixed',
+              'write.format.default'='avro');
+---- 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_avro_mixed /test-warehouse/iceberg_test/hadoop_catalog/ice
 ====
 ---- DATASET
 functional
diff --git a/testdata/datasets/functional/schema_constraints.csv b/testdata/datasets/functional/schema_constraints.csv
index 55e0d17d1..dee5088e4 100644
--- a/testdata/datasets/functional/schema_constraints.csv
+++ b/testdata/datasets/functional/schema_constraints.csv
@@ -92,6 +92,8 @@ table_name:iceberg_v2_positional_not_all_data_files_have_delete_files_orc, const
 table_name:iceberg_v2_partitioned_position_deletes, constraint:restrict_to, table_format:parquet/none/none
 table_name:iceberg_v2_partitioned_position_deletes_orc, constraint:restrict_to, table_format:parquet/none/none
 table_name:iceberg_multiple_storage_locations, constraint:restrict_to, table_format:parquet/none/none
+table_name:iceberg_avro_only, constraint:restrict_to, table_format:parquet/none/none
+table_name:iceberg_avro_mixed, 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-avro.test b/testdata/workloads/functional-query/queries/QueryTest/iceberg-avro.test
new file mode 100644
index 000000000..fde274b7f
--- /dev/null
+++ b/testdata/workloads/functional-query/queries/QueryTest/iceberg-avro.test
@@ -0,0 +1,10 @@
+====
+---- QUERY
+select * from functional_parquet.iceberg_avro_only;
+---- RESULTS
+1,'A',0.5
+2,'B',1.5
+3,'C',2.5
+---- TYPES
+INT, STRING, DOUBLE
+====
\ No newline at end of file
diff --git a/testdata/workloads/functional-query/queries/QueryTest/iceberg-negative.test b/testdata/workloads/functional-query/queries/QueryTest/iceberg-negative.test
index 2231c8e15..d55a05d41 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/iceberg-negative.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/iceberg-negative.test
@@ -662,3 +662,8 @@ select * from functional_parquet.iceberg_alltypes_part for system_time as of '20
 ---- CATCH
 IllegalArgumentException: Cannot find a snapshot older than 2000-01-01 01:02:03
 ====
+---- QUERY
+select * from functional_parquet.iceberg_avro_mixed;
+---- CATCH
+ImpalaRuntimeException: Iceberg tables containing multiple file formats are only supported if they do not contain AVRO files.
+====
diff --git a/tests/query_test/test_iceberg.py b/tests/query_test/test_iceberg.py
index c737b9fde..07e3f309c 100644
--- a/tests/query_test/test_iceberg.py
+++ b/tests/query_test/test_iceberg.py
@@ -854,6 +854,8 @@ class TestIcebergTable(IcebergTestSuite):
   def test_virtual_columns(self, vector, unique_database):
     self.run_test_case('QueryTest/iceberg-virtual-columns', vector, unique_database)
 
+  def test_avro_file_format(self, vector, unique_database):
+    self.run_test_case('QueryTest/iceberg-avro', vector, unique_database)
 
 class TestIcebergV2Table(IcebergTestSuite):
   """Tests related to Iceberg V2 tables."""