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

zhangstar333 pushed a commit to branch iceberg-v3-test-branch-4.1
in repository https://gitbox.apache.org/repos/asf/doris.git


The following commit(s) were added to refs/heads/iceberg-v3-test-branch-4.1 by 
this push:
     new f5e50abd2e1 fix iceberg getFileFormat
f5e50abd2e1 is described below

commit f5e50abd2e1895dd6b60c996f9a1ae286aa1d133
Author: zhangstar333 <[email protected]>
AuthorDate: Thu Jun 4 19:59:05 2026 +0800

    fix iceberg getFileFormat
---
 .../doris/datasource/iceberg/IcebergUtils.java     | 51 +++++++++++++++++-----
 1 file changed, 41 insertions(+), 10 deletions(-)

diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergUtils.java
 
b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergUtils.java
index e162370e732..12b51c0b7c2 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergUtils.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergUtils.java
@@ -168,6 +168,8 @@ public class IcebergUtils {
     // nickname in flink and spark
     public static final String WRITE_FORMAT = "write-format";
     public static final String COMPRESSION_CODEC = "compression-codec";
+    // For migrated Iceberg tables, the format may be stored as 
"iceberg/parquet" or "iceberg/orc"
+    public static final String FORMAT = "format";
 
     // nickname in spark
     public static final String SPARK_SQL_COMPRESSION_CODEC = 
"spark.sql.iceberg.compression-codec";
@@ -1089,16 +1091,7 @@ public class IcebergUtils {
 
     public static FileFormat getFileFormat(Table icebergTable) {
         Map<String, String> properties = icebergTable.properties();
-        for (Map.Entry<String, String> entry : properties.entrySet()) {
-            LOG.info("asd Iceberg table property: {}={}", entry.getKey(), 
entry.getValue());
-        }
-        String fileFormatName;
-        if (properties.containsKey(WRITE_FORMAT)) {
-            fileFormatName = properties.get(WRITE_FORMAT);
-        } else {
-            fileFormatName = 
properties.getOrDefault(TableProperties.DEFAULT_FILE_FORMAT, PARQUET_NAME);
-        }
-        LOG.info("asd Iceberg table {}.{} file format is {}", 
icebergTable.name(), fileFormatName);
+        String fileFormatName = resolveFileFormatName(icebergTable, 
properties);
         FileFormat fileFormat;
         if (fileFormatName.toLowerCase().contains(ORC_NAME)) {
             fileFormat = FileFormat.ORC;
@@ -1110,6 +1103,44 @@ public class IcebergUtils {
         return fileFormat;
     }
 
+    private static String resolveFileFormatName(Table icebergTable, 
Map<String, String> properties) {
+        // 1. Check "write-format" (nickname in Flink and Spark)
+        if (properties.containsKey(WRITE_FORMAT)) {
+            return properties.get(WRITE_FORMAT);
+        }
+        // 2. Check "write.format.default" (standard Iceberg property)
+        if (properties.containsKey(TableProperties.DEFAULT_FILE_FORMAT)) {
+            return properties.get(TableProperties.DEFAULT_FILE_FORMAT);
+        }
+        // 3. Check "format" property (e.g., "iceberg/parquet", "iceberg/orc")
+        //    This is commonly set on migrated Iceberg tables.
+        if (properties.containsKey(FORMAT)) {
+            return properties.get(FORMAT);
+        }
+        // 4. Last resort: infer from the actual data files in the current 
snapshot.
+        //    This handles migrated tables where none of the above properties 
are set.
+        return inferFileFormatFromDataFiles(icebergTable);
+    }
+
+    private static String inferFileFormatFromDataFiles(Table icebergTable) {
+        if (icebergTable.currentSnapshot() == null) {
+            LOG.info("asd Iceberg table {} has no snapshot, defaulting to {}", 
icebergTable.name(), PARQUET_NAME);
+            return PARQUET_NAME;
+        }
+        try (CloseableIterable<FileScanTask> files = 
icebergTable.newScan().planFiles()) {
+            java.util.Iterator<FileScanTask> it = files.iterator();
+            if (it.hasNext()) {
+                String format = it.next().file().format().name().toLowerCase();
+                LOG.info("asd Iceberg table {} inferred file format {} from 
data files", icebergTable.name(), format);
+                return format;
+            }
+        } catch (Exception e) {
+            LOG.warn("asd Failed to infer file format from data files for 
table {}, defaulting to {}",
+                    icebergTable.name(), PARQUET_NAME, e);
+        }
+        return PARQUET_NAME;
+    }
+
 
     public static String getFileCompress(Table table) {
         Map<String, String> properties = table.properties();


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to