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

commit 1e1b8f25b686471b088148dd296a2eb731160302
Author: Zoltan Borok-Nagy <[email protected]>
AuthorDate: Thu Jan 12 15:51:58 2023 +0100

    IMPALA-11826: Avoid calling planFiles() on Iceberg V2 tables when there are 
no predicates
    
    Similar to IMPALA-11591 but this Jira extends it to V2 tables. With
    this patch we group data files into two categories in
    IcebergContentFileStore:
     * data files without deletes
     * data files with deletes
    
    With this information we can avoid calling planFiles() when planning
    the scans of Iceberg tables. We can just set the lists of the file
    descriptors based on IcebergContentFileStore then invoke the regular
    planning methods.
    
    iceberg-v2-tables.test had to be updated a bit because now we are
    calculating the lengths of the file paths based on Impala's file
    descriptor objects + table location, and not based on data file
    information in Iceberg metadata (which has the file system prefix
    stripped)
    
    Testing:
      * executed existing tests
      * Updated plan tests
    
    Change-Id: Ia46bd2dce248a9e096fc1c0bd914fc3fa4686fb0
    Reviewed-on: http://gerrit.cloudera.org:8080/19419
    Reviewed-by: Impala Public Jenkins <[email protected]>
    Tested-by: Impala Public Jenkins <[email protected]>
---
 common/thrift/CatalogObjects.thrift                |  11 +-
 .../org/apache/impala/catalog/FeIcebergTable.java  |  48 +++---
 .../impala/catalog/IcebergContentFileStore.java    | 166 +++++++++++++--------
 .../org/apache/impala/catalog/IcebergTable.java    |   7 +-
 .../catalog/iceberg/GroupedContentFiles.java       |  58 +++++++
 .../apache/impala/planner/IcebergScanPlanner.java  |  79 +++++-----
 .../java/org/apache/impala/util/IcebergUtil.java   |  11 +-
 .../impala/catalog/local/LocalCatalogTest.java     |   9 +-
 .../queries/PlannerTest/iceberg-v2-tables.test     |  60 ++++----
 9 files changed, 277 insertions(+), 172 deletions(-)

diff --git a/common/thrift/CatalogObjects.thrift 
b/common/thrift/CatalogObjects.thrift
index 5ba8c4edb..60bf88164 100644
--- a/common/thrift/CatalogObjects.thrift
+++ b/common/thrift/CatalogObjects.thrift
@@ -603,11 +603,12 @@ struct TIcebergPartitionStats {
 
 // Contains maps from 128-bit Murmur3 hash of file path to its file descriptor
 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
+  1: optional map<string, THdfsFileDesc> path_hash_to_data_file_without_deletes
+  2: optional map<string, THdfsFileDesc> path_hash_to_data_file_with_deletes
+  3: optional map<string, THdfsFileDesc> path_hash_to_delete_file
+  4: optional bool has_avro
+  5: optional bool has_orc
+  6: optional bool has_parquet
 }
 
 struct TIcebergTable {
diff --git a/fe/src/main/java/org/apache/impala/catalog/FeIcebergTable.java 
b/fe/src/main/java/org/apache/impala/catalog/FeIcebergTable.java
index 21b79f285..043cb25ae 100644
--- a/fe/src/main/java/org/apache/impala/catalog/FeIcebergTable.java
+++ b/fe/src/main/java/org/apache/impala/catalog/FeIcebergTable.java
@@ -19,7 +19,6 @@ package org.apache.impala.catalog;
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
-import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
 import com.google.common.primitives.Ints;
 
@@ -63,6 +62,7 @@ import org.apache.impala.analysis.TimeTravelSpec;
 import org.apache.impala.analysis.TimeTravelSpec.Kind;
 import org.apache.impala.catalog.HdfsPartition.FileBlock;
 import org.apache.impala.catalog.HdfsPartition.FileDescriptor;
+import org.apache.impala.catalog.iceberg.GroupedContentFiles;
 import org.apache.impala.common.FileSystemUtil;
 import org.apache.impala.common.Pair;
 import org.apache.impala.common.PrintUtils;
@@ -671,44 +671,50 @@ public interface FeIcebergTable extends FeFsTable {
 
     /**
      * Returns the FileDescriptors loaded by the internal HdfsTable. To avoid 
returning
-     * the metadata files the resulset is limited to the files that are 
tracked by
+     * the metadata files the resultset is limited to the files that are 
tracked by
      * Iceberg. Both the HdfsBaseDir and the DataFile path can contain the 
scheme in their
      * path, using org.apache.hadoop.fs.Path to normalize the paths.
      */
     public static IcebergContentFileStore loadAllPartition(
-        IcebergTable table, Pair<List<DataFile>, Set<DeleteFile>> icebergFiles)
+        IcebergTable table, GroupedContentFiles icebergFiles)
         throws IOException, TableLoadingException {
       Map<String, HdfsPartition.FileDescriptor> hdfsFileDescMap = new 
HashMap<>();
       Collection<HdfsPartition> partitions =
           ((HdfsTable)table.getFeFsTable()).partitionMap_.values();
       for (HdfsPartition partition : partitions) {
         for (FileDescriptor fileDesc : partition.getFileDescriptors()) {
-            Path path = new 
Path(fileDesc.getAbsolutePath(table.getHdfsBaseDir()));
-            hdfsFileDescMap.put(path.toUri().getPath(), fileDesc);
+          Path path = new 
Path(fileDesc.getAbsolutePath(table.getHdfsBaseDir()));
+          hdfsFileDescMap.put(path.toUri().getPath(), fileDesc);
         }
       }
       IcebergContentFileStore fileStore = new IcebergContentFileStore();
-      for (ContentFile contentFile : Iterables.concat(icebergFiles.first,
-                                                      icebergFiles.second)) {
-        addContentFileToFileStore(contentFile, fileStore, table, 
hdfsFileDescMap);
+      Pair<String, HdfsPartition.FileDescriptor> pathHashAndFd;
+      for (DataFile dataFile : icebergFiles.dataFilesWithoutDeletes) {
+        pathHashAndFd = getPathHashAndFd(dataFile, table, hdfsFileDescMap);
+        fileStore.addDataFileWithoutDeletes(pathHashAndFd.first, 
pathHashAndFd.second);
+      }
+      for (DataFile dataFile : icebergFiles.dataFilesWithDeletes) {
+        pathHashAndFd = getPathHashAndFd(dataFile, table, hdfsFileDescMap);
+        fileStore.addDataFileWithDeletes(pathHashAndFd.first, 
pathHashAndFd.second);
+      }
+      for (DeleteFile deleteFile : icebergFiles.deleteFiles) {
+        Preconditions.checkState(
+            deleteFile.content().equals(FileContent.EQUALITY_DELETES) ||
+            deleteFile.content().equals(FileContent.POSITION_DELETES));
+        pathHashAndFd = getPathHashAndFd(deleteFile, table, hdfsFileDescMap);
+        fileStore.addDeleteFileDescriptor(pathHashAndFd.first, 
pathHashAndFd.second);
       }
       return fileStore;
     }
 
-    private static void addContentFileToFileStore(ContentFile contentFile,
-        IcebergContentFileStore fileStore, IcebergTable table,
+    private static Pair<String, HdfsPartition.FileDescriptor> getPathHashAndFd(
+        ContentFile contentFile,
+        IcebergTable table,
         Map<String, HdfsPartition.FileDescriptor> hdfsFileDescMap) throws 
IOException {
       String pathHash = IcebergUtil.getFilePathHash(contentFile);
       HdfsPartition.FileDescriptor fd = getOrCreateIcebergFd(
           table, hdfsFileDescMap, contentFile);
-      if (contentFile.content().equals(FileContent.DATA)) {
-        fileStore.addDataFileDescriptor(pathHash, fd);
-      } else {
-        Preconditions.checkState(
-            contentFile.content().equals(FileContent.EQUALITY_DELETES) ||
-            contentFile.content().equals(FileContent.POSITION_DELETES));
-        fileStore.addDeleteFileDescriptor(pathHash, fd);
-      }
+      return new Pair<>(pathHash, fd);
     }
 
     private static FileDescriptor getOrCreateIcebergFd(IcebergTable table,
@@ -743,12 +749,10 @@ public interface FeIcebergTable extends FeFsTable {
      * TODO(IMPALA-11516): Return better partition stats for V2 tables.
      */
     public static Map<String, TIcebergPartitionStats> loadPartitionStats(
-        IcebergTable table, Pair<List<DataFile>, Set<DeleteFile>> icebergFiles)
+        IcebergTable table, GroupedContentFiles icebergFiles)
         throws TableLoadingException {
-      List<DataFile> dataFileList = icebergFiles.first;
-      Set<DeleteFile> deleteFileList = icebergFiles.second;
       Map<String, TIcebergPartitionStats> nameToStats = new HashMap<>();
-      for (ContentFile<?> contentFile : Iterables.concat(dataFileList, 
deleteFileList)) {
+      for (ContentFile<?> contentFile : icebergFiles.getAllContentFiles()) {
         String name = getPartitionKey(table, contentFile);
         nameToStats.put(name, mergePartitionStats(nameToStats, contentFile, 
name));
       }
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 4101acebb..d869665d5 100644
--- a/fe/src/main/java/org/apache/impala/catalog/IcebergContentFileStore.java
+++ b/fe/src/main/java/org/apache/impala/catalog/IcebergContentFileStore.java
@@ -34,21 +34,68 @@ import org.apache.impala.thrift.TIcebergContentFileStore;
 import org.apache.impala.thrift.TNetworkAddress;
 import org.apache.impala.util.ListMap;
 
-
 /**
  * Helper class for storing Iceberg file descriptors. It stores data and 
delete files
  * separately, while also storing file descriptors belonging to earlier 
snapshots.
  */
 public class IcebergContentFileStore {
-  // Key is the DataFile path hash, value is FileDescriptor transformed from 
DataFile
-  private final Map<String, FileDescriptor> dataFileDescMap_ = new HashMap<>();
-  private final Map<String, FileDescriptor> deleteFileDescMap_ = new 
HashMap<>();
 
-  // List of Iceberg data files (doesn't include delete files)
-  private final List<FileDescriptor> dataFiles_ = new ArrayList<>();
+  // Auxiliary class for holding file descriptors in both a map and a list.
+  private static class MapListContainer {
+    // Key is the ContentFile path hash, value is FileDescriptor transformed 
from DataFile
+    private final Map<String, FileDescriptor> fileDescMap_ = new HashMap<>();
+    private final List<FileDescriptor> fileDescList_ = new ArrayList<>();
+
+    // Adds a file to the map. If this is a new entry, then add it to the list 
as well.
+    // Return true if 'desc' was a new entry.
+    public boolean add(String pathHash, FileDescriptor desc) {
+      if (fileDescMap_.put(pathHash, desc) == null) {
+        fileDescList_.add(desc);
+        return true;
+      }
+      return false;
+    }
+
+    public FileDescriptor get(String pathHash) {
+      return fileDescMap_.get(pathHash);
+    }
+
+    public long getNumFiles() {
+      return fileDescList_.size();
+    }
+
+    List<FileDescriptor> getList() { return fileDescList_; }
+
+    // It's enough to only convert the map part to thrift.
+    Map<String, THdfsFileDesc> toThrift() {
+      Map<String, THdfsFileDesc> ret = new HashMap<>();
+      for (Map.Entry<String, HdfsPartition.FileDescriptor> entry :
+          fileDescMap_.entrySet()) {
+        ret.put(entry.getKey(), entry.getValue().toThrift());
+      }
+      return ret;
+    }
 
-  // List of Iceberg delete files (equality and position delete files)
-  private final List<FileDescriptor> deleteFiles_ = new ArrayList<>();
+    static MapListContainer fromThrift(Map<String, THdfsFileDesc> thriftMap,
+        List<TNetworkAddress> networkAddresses, ListMap<TNetworkAddress> 
hostIndex) {
+      MapListContainer ret = new MapListContainer();
+      for (Map.Entry<String, THdfsFileDesc> entry : thriftMap.entrySet()) {
+        FileDescriptor fd = FileDescriptor.fromThrift(entry.getValue());
+        Preconditions.checkNotNull(fd);
+        if (networkAddresses != null) {
+          Preconditions.checkNotNull(hostIndex);
+          fd = fd.cloneWithNewHostIndex(networkAddresses, hostIndex);
+        }
+        ret.add(entry.getKey(), fd);
+      }
+      return ret;
+    }
+  }
+
+  // Separate map-list containers for the different content files.
+  private MapListContainer dataFilesWithoutDeletes_ = new MapListContainer();
+  private MapListContainer dataFilesWithDeletes_ = new MapListContainer();
+  private MapListContainer deleteFiles_ = new MapListContainer();
 
   // Caches file descriptors loaded during time-travel queries.
   private final ConcurrentMap<String, FileDescriptor> oldFileDescMap_ =
@@ -61,54 +108,71 @@ public class IcebergContentFileStore {
 
   public IcebergContentFileStore() {}
 
-  public void addDataFileDescriptor(String pathHash, FileDescriptor desc) {
-    if (dataFileDescMap_.put(pathHash, desc) == null) {
-      dataFiles_.add(desc);
+  public void addDataFileWithoutDeletes(String pathHash, FileDescriptor desc) {
+    if (dataFilesWithoutDeletes_.add(pathHash, desc)) {
+      updateFileFormats(desc);
+    }
+  }
+
+  public void addDataFileWithDeletes(String pathHash, FileDescriptor desc) {
+    if (dataFilesWithDeletes_.add(pathHash, desc)) {
       updateFileFormats(desc);
     }
   }
 
   public void addDeleteFileDescriptor(String pathHash, FileDescriptor desc) {
-    if (deleteFileDescMap_.put(pathHash, desc) == null) {
-      deleteFiles_.add(desc);
+    if (deleteFiles_.add(pathHash, desc)) {
       updateFileFormats(desc);
     }
   }
 
+  // This is only invoked during time travel, when we are querying a snapshot 
that has
+  // data files which have been removed since.
   public void addOldFileDescriptor(String pathHash, FileDescriptor desc) {
     oldFileDescMap_.put(pathHash, desc);
   }
 
   public FileDescriptor getDataFileDescriptor(String pathHash) {
-    return dataFileDescMap_.get(pathHash);
+    FileDescriptor desc = dataFilesWithoutDeletes_.get(pathHash);
+    if (desc != null) return desc;
+    return dataFilesWithDeletes_.get(pathHash);
   }
 
   public FileDescriptor getDeleteFileDescriptor(String pathHash) {
-    return deleteFileDescMap_.get(pathHash);
+    return deleteFiles_.get(pathHash);
   }
 
   public FileDescriptor getOldFileDescriptor(String pathHash) {
     return oldFileDescMap_.get(pathHash);
   }
 
-  public FileDescriptor getFileDescriptor(String pathHash) {
-    FileDescriptor desc = null;
-    desc = dataFileDescMap_.get(pathHash);
-    if (desc != null) return desc;
-    desc = deleteFileDescMap_.get(pathHash);
-    if (desc != null) return desc;
-    desc = oldFileDescMap_.get(pathHash);
-    return desc;
+  public List<FileDescriptor> getDataFilesWithoutDeletes() {
+    return dataFilesWithoutDeletes_.getList();
   }
 
-  public List<FileDescriptor> getDataFiles() { return dataFiles_; }
+  public List<FileDescriptor> getDataFilesWithDeletes() {
+    return dataFilesWithDeletes_.getList();
+  }
 
-  public List<FileDescriptor> getDeleteFiles() { return deleteFiles_; }
+  public List<FileDescriptor> getDeleteFiles() { return 
deleteFiles_.getList(); }
 
-  public long getNumFiles() { return dataFiles_.size() + deleteFiles_.size(); }
+  public long getNumFiles() {
+    return dataFilesWithoutDeletes_.getNumFiles() +
+           dataFilesWithDeletes_.getNumFiles() +
+           deleteFiles_.getNumFiles();
+  }
 
   public Iterable<FileDescriptor> getAllFiles() {
-    return Iterables.concat(dataFiles_, deleteFiles_);
+    return Iterables.concat(
+        dataFilesWithoutDeletes_.getList(),
+        dataFilesWithDeletes_.getList(),
+        deleteFiles_.getList());
+  }
+
+  public Iterable<FileDescriptor> getAllDataFiles() {
+    return Iterables.concat(
+        dataFilesWithoutDeletes_.getList(),
+        dataFilesWithDeletes_.getList());
   }
 
   public boolean hasAvro() { return hasAvro_; }
@@ -128,8 +192,9 @@ public class IcebergContentFileStore {
 
   public TIcebergContentFileStore toThrift() {
     TIcebergContentFileStore ret = new TIcebergContentFileStore();
-    ret.setPath_hash_to_data_file(convertFileMapToThrift(dataFileDescMap_));
-    
ret.setPath_hash_to_delete_file(convertFileMapToThrift(deleteFileDescMap_));
+    
ret.setPath_hash_to_data_file_without_deletes(dataFilesWithoutDeletes_.toThrift());
+    
ret.setPath_hash_to_data_file_with_deletes(dataFilesWithDeletes_.toThrift());
+    ret.setPath_hash_to_delete_file(deleteFiles_.toThrift());
     ret.setHas_avro(hasAvro_);
     ret.setHas_orc(hasOrc_);
     ret.setHas_parquet(hasParquet_);
@@ -140,43 +205,24 @@ public class IcebergContentFileStore {
       List<TNetworkAddress> networkAddresses,
       ListMap<TNetworkAddress> hostIndex) {
     IcebergContentFileStore ret = new IcebergContentFileStore();
-    if (tFileStore.isSetPath_hash_to_data_file()) {
-      convertFileMapFromThrift(tFileStore.getPath_hash_to_data_file(),
-          ret.dataFileDescMap_, ret.dataFiles_, networkAddresses, hostIndex);
+    if (tFileStore.isSetPath_hash_to_data_file_without_deletes()) {
+      ret.dataFilesWithoutDeletes_ = MapListContainer.fromThrift(
+          tFileStore.getPath_hash_to_data_file_without_deletes(),
+          networkAddresses, hostIndex);
+    }
+    if (tFileStore.isSetPath_hash_to_data_file_with_deletes()) {
+      ret.dataFilesWithDeletes_ = MapListContainer.fromThrift(
+          tFileStore.getPath_hash_to_data_file_with_deletes(),
+          networkAddresses, hostIndex);
     }
     if (tFileStore.isSetPath_hash_to_delete_file()) {
-      convertFileMapFromThrift(tFileStore.getPath_hash_to_delete_file(),
-          ret.deleteFileDescMap_, ret.deleteFiles_, networkAddresses, 
hostIndex);
+      ret.deleteFiles_ = MapListContainer.fromThrift(
+          tFileStore.getPath_hash_to_delete_file(),
+          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;
   }
-
-  private static Map<String, THdfsFileDesc> convertFileMapToThrift(
-      Map<String, FileDescriptor> fileDescMap) {
-    Map<String, THdfsFileDesc> ret = new HashMap<>();
-    for (Map.Entry<String, HdfsPartition.FileDescriptor> entry : 
fileDescMap.entrySet()) {
-      ret.put(entry.getKey(), entry.getValue().toThrift());
-    }
-    return ret;
-  }
-
-  private static void convertFileMapFromThrift(Map<String, THdfsFileDesc> 
thriftMap,
-      Map<String, FileDescriptor> outMap, List<FileDescriptor> outList,
-      List<TNetworkAddress> networkAddresses, ListMap<TNetworkAddress> 
hostIndex) {
-    Preconditions.checkNotNull(outMap);
-    Preconditions.checkNotNull(outList);
-    for (Map.Entry<String, THdfsFileDesc> entry : thriftMap.entrySet()) {
-      FileDescriptor fd = FileDescriptor.fromThrift(entry.getValue());
-      Preconditions.checkNotNull(fd);
-      if (networkAddresses != null) {
-        Preconditions.checkNotNull(hostIndex);
-        fd = fd.cloneWithNewHostIndex(networkAddresses, hostIndex);
-      }
-      outMap.put(entry.getKey(), fd);
-      outList.add(fd);
-    }
-  }
 }
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 77e7b49c0..f66efd828 100644
--- a/fe/src/main/java/org/apache/impala/catalog/IcebergTable.java
+++ b/fe/src/main/java/org/apache/impala/catalog/IcebergTable.java
@@ -29,13 +29,10 @@ import org.apache.hadoop.hive.common.StatsSetupConst;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
 import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
-import org.apache.iceberg.DataFile;
-import org.apache.iceberg.DeleteFile;
 import org.apache.impala.analysis.IcebergPartitionField;
 import org.apache.impala.analysis.IcebergPartitionSpec;
 import org.apache.impala.analysis.IcebergPartitionTransform;
-import org.apache.impala.catalog.HdfsPartition.FileDescriptor;
-import org.apache.impala.common.Pair;
+import org.apache.impala.catalog.iceberg.GroupedContentFiles;
 import org.apache.impala.thrift.TCatalogObjectType;
 import org.apache.impala.thrift.TCompressionCodec;
 import org.apache.impala.thrift.TGetPartialCatalogObjectRequest;
@@ -358,7 +355,7 @@ public class IcebergTable extends Table implements 
FeIcebergTable {
         icebergParquetDictPageSize_ = 
Utils.getIcebergParquetDictPageSize(msTbl);
         hdfsTable_
             .load(false, msClient, msTable_, true, true, false, null, 
null,null, reason);
-        Pair<List<DataFile>, Set<DeleteFile>> icebergFiles = IcebergUtil
+        GroupedContentFiles icebergFiles = IcebergUtil
             .getIcebergFiles(this, new ArrayList<>(), /*timeTravelSpec=*/null);
         fileStore_ = Utils.loadAllPartition(this, icebergFiles);
         partitionStats_ = Utils.loadPartitionStats(this, icebergFiles);
diff --git 
a/fe/src/main/java/org/apache/impala/catalog/iceberg/GroupedContentFiles.java 
b/fe/src/main/java/org/apache/impala/catalog/iceberg/GroupedContentFiles.java
new file mode 100644
index 000000000..65e06494b
--- /dev/null
+++ 
b/fe/src/main/java/org/apache/impala/catalog/iceberg/GroupedContentFiles.java
@@ -0,0 +1,58 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.impala.catalog.iceberg;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.io.CloseableIterable;
+
+/**
+ * Struct-like object to group different Iceberg content files:
+ * - data files without deleted rows
+ * - data files with deleted rows
+ * - delete files
+ */
+public class GroupedContentFiles {
+  public List<DataFile> dataFilesWithoutDeletes = new ArrayList<>();
+  public List<DataFile> dataFilesWithDeletes = new ArrayList<>();
+  public Set<DeleteFile> deleteFiles = new HashSet<>();
+
+  public GroupedContentFiles(CloseableIterable<FileScanTask> fileScanTasks) {
+    for (FileScanTask scanTask : fileScanTasks) {
+      if (scanTask.deletes().isEmpty()) {
+        dataFilesWithoutDeletes.add(scanTask.file());
+      } else {
+        dataFilesWithDeletes.add(scanTask.file());
+        deleteFiles.addAll(scanTask.deletes());
+      }
+    }
+  }
+
+  public Iterable<ContentFile> getAllContentFiles() {
+    return Iterables.concat(dataFilesWithoutDeletes, dataFilesWithDeletes, 
deleteFiles);
+  }
+}
diff --git a/fe/src/main/java/org/apache/impala/planner/IcebergScanPlanner.java 
b/fe/src/main/java/org/apache/impala/planner/IcebergScanPlanner.java
index 4b289e500..d3d5f7122 100644
--- a/fe/src/main/java/org/apache/impala/planner/IcebergScanPlanner.java
+++ b/fe/src/main/java/org/apache/impala/planner/IcebergScanPlanner.java
@@ -139,23 +139,12 @@ public class IcebergScanPlanner {
     analyzer_.materializeSlots(conjuncts_);
 
     if (!needIcebergForPlanning()) {
-      return planWithoutIceberg();
+      setFileDescriptorsBasedOnFileStore();
+      return createIcebergScanPlanImpl();
     }
 
     filterFileDescriptors();
-
-    PlanNode ret;
-    if (deleteFiles_.isEmpty()) {
-      // If there are no delete files we can just create a single SCAN node.
-      Preconditions.checkState(dataFilesWithDeletes_.isEmpty());
-      ret = new IcebergScanNode(ctx_.getNextNodeId(), tblRef_, conjuncts_,
-          aggInfo_, dataFilesWithoutDeletes_, nonIdentityConjuncts_);
-      ret.init(analyzer_);
-    } else {
-      // Let's create a bit more complex plan in the presence of delete files.
-      ret = createComplexIcebergScanPlan();
-    }
-    return ret;
+    return createIcebergScanPlanImpl();
   }
 
   /**
@@ -163,28 +152,30 @@ public class IcebergScanPlanner {
    * true:
    *  - we don't push down predicates
    *  - no time travel
-   *  - no delete files
-   * TODO: we should still avoid calling planFiles() if there are delete 
files. To do that
-   * we either need to track which delete files have corresponding data files 
so we
-   * can create the UNION ALL node. Or, if we have an optimized, 
Iceberg-specific
-   * ANTI-JOIN operator, then it wouldn't hurt too much to transfer all rows 
through it.
    */
   private boolean needIcebergForPlanning() {
     return
         !icebergPredicates_.isEmpty() ||
-        tblRef_.getTimeTravelSpec() != null ||
-        !getIceTable().getContentFileStore().getDeleteFiles().isEmpty();
+        tblRef_.getTimeTravelSpec() != null;
   }
 
-  private PlanNode planWithoutIceberg() throws ImpalaException {
-    PlanNode ret = new IcebergScanNode(ctx_.getNextNodeId(), tblRef_, 
conjuncts_,
-        aggInfo_, getIceTable().getContentFileStore().getDataFiles(),
-        nonIdentityConjuncts_);
-    ret.init(analyzer_);
-    return ret;
+  private void setFileDescriptorsBasedOnFileStore() {
+    IcebergContentFileStore fileStore = getIceTable().getContentFileStore();
+    dataFilesWithoutDeletes_ = fileStore.getDataFilesWithoutDeletes();
+    dataFilesWithDeletes_ = fileStore.getDataFilesWithDeletes();
+    deleteFiles_ = new HashSet<>(fileStore.getDeleteFiles());
+    updateDeleteStatistics();
   }
 
-  private PlanNode createComplexIcebergScanPlan() throws ImpalaException {
+  private PlanNode createIcebergScanPlanImpl() throws ImpalaException {
+    if (deleteFiles_.isEmpty()) {
+      // If there are no delete files we can just create a single SCAN node.
+      Preconditions.checkState(dataFilesWithDeletes_.isEmpty());
+      PlanNode ret = new IcebergScanNode(ctx_.getNextNodeId(), tblRef_, 
conjuncts_,
+          aggInfo_, dataFilesWithoutDeletes_, nonIdentityConjuncts_);
+      ret.init(analyzer_);
+      return ret;
+    }
     PlanNode joinNode = createPositionJoinNode();
     if (dataFilesWithoutDeletes_.isEmpty()) {
       // All data files has corresponding delete files, so we just return an 
ANTI JOIN
@@ -211,6 +202,9 @@ public class IcebergScanPlanner {
   }
 
   private PlanNode createPositionJoinNode() throws ImpalaException {
+    Preconditions.checkState(deletesRecordCount_ != 0);
+    Preconditions.checkState(dataFilesWithDeletesSumPaths_ != 0);
+    Preconditions.checkState(dataFilesWithDeletesMaxPath_ != 0);
     // The followings just create separate scan nodes for data files and 
position delete
     // files, plus adds a LEFT ANTI HASH JOIN above them.
     PlanNodeId dataScanNodeId = ctx_.getNextNodeId();
@@ -338,7 +332,6 @@ public class IcebergScanPlanner {
         if (fileScanTask.deletes().isEmpty()) {
           dataFilesWithoutDeletes_.add(fileDesc.first);
         } else {
-          updateDataFilesWithDeletesStatistics(fileScanTask.file());
           dataFilesWithDeletes_.add(fileDesc.first);
           for (DeleteFile delFile : fileScanTask.deletes()) {
             // TODO(IMPALA-11388): Add support for equality deletes.
@@ -350,10 +343,7 @@ public class IcebergScanPlanner {
             }
             Pair<FileDescriptor, Boolean> delFileDesc = 
getFileDescriptor(delFile);
             if (!delFileDesc.second) ++dataFilesCacheMisses;
-            if (deleteFiles_.add(delFileDesc.first)) {
-              // New delete file added to 'deleteFiles_'.
-              updateDeleteFilesStatistics(delFile);
-            }
+            deleteFiles_.add(delFileDesc.first);
           }
         }
       }
@@ -368,22 +358,33 @@ public class IcebergScanPlanner {
           "Failed to load data files for Iceberg table: %s", 
getIceTable().getFullName()),
           e);
     }
+    updateDeleteStatistics();
+  }
+
+  private void updateDeleteStatistics() {
+    for (FileDescriptor fd : dataFilesWithDeletes_) {
+      updateDataFilesWithDeletesStatistics(fd);
+    }
+    for (FileDescriptor fd : deleteFiles_) {
+      updateDeleteFilesStatistics(fd);
+    }
   }
 
-  private void updateDataFilesWithDeletesStatistics(ContentFile cf) {
-    long pathSize = cf.path().toString().length();
+  private void updateDataFilesWithDeletesStatistics(FileDescriptor fd) {
+    String path = fd.getAbsolutePath(getIceTable().getLocation());
+    long pathSize = path.length();
     dataFilesWithDeletesSumPaths_ += pathSize;
     if (pathSize > dataFilesWithDeletesMaxPath_) {
       dataFilesWithDeletesMaxPath_ = pathSize;
     }
   }
 
-  private void updateDeleteFilesStatistics(DeleteFile delFile) {
-    deletesRecordCount_ += getRecordCount(delFile);
+  private void updateDeleteFilesStatistics(FileDescriptor fd) {
+    deletesRecordCount_ += getRecordCount(fd);
   }
 
-  private long getRecordCount(DeleteFile delFile) {
-    long recordCount = delFile.recordCount();
+  private long getRecordCount(FileDescriptor fd) {
+    long recordCount = fd.getFbFileMetadata().icebergMetadata().recordCount();
     // 'record_count' is a required field for Iceberg data files, but let's 
still
     // prepare for the case when a compute engine doesn't fill it.
     if (recordCount == -1) return 1000;
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 6d08e4866..72859527e 100644
--- a/fe/src/main/java/org/apache/impala/util/IcebergUtil.java
+++ b/fe/src/main/java/org/apache/impala/util/IcebergUtil.java
@@ -73,6 +73,7 @@ import org.apache.impala.catalog.HdfsFileFormat;
 import org.apache.impala.catalog.IcebergColumn;
 import org.apache.impala.catalog.IcebergTable;
 import org.apache.impala.catalog.TableLoadingException;
+import org.apache.impala.catalog.iceberg.GroupedContentFiles;
 import org.apache.impala.catalog.iceberg.IcebergCatalog;
 import org.apache.impala.catalog.iceberg.IcebergCatalogs;
 import org.apache.impala.catalog.iceberg.IcebergHadoopCatalog;
@@ -551,21 +552,15 @@ public class IcebergUtil {
   /**
    * Returns lists of data and delete files in the Iceberg table.
    */
-  public static Pair<List<DataFile>, Set<DeleteFile>> getIcebergFiles(
+  public static GroupedContentFiles getIcebergFiles(
       FeIcebergTable table, List<Expression> predicates, TimeTravelSpec 
timeTravelSpec)
         throws TableLoadingException {
-    List<DataFile> dataFiles = new ArrayList<>();
-    Set<DeleteFile> deleteFiles = new HashSet<>();
     try (CloseableIterable<FileScanTask> fileScanTasks = planFiles(
         table, predicates, timeTravelSpec)) {
-      for (FileScanTask scanTask : fileScanTasks) {
-        dataFiles.add(scanTask.file());
-        deleteFiles.addAll(scanTask.deletes());
-      }
+      return new GroupedContentFiles(fileScanTasks);
     } catch (IOException e) {
       throw new TableLoadingException("Error during reading Iceberg manifest 
files.", e);
     }
-    return new Pair<>(dataFiles, deleteFiles);
   }
 
   private static TableScan createScanAsOf(FeIcebergTable table,
diff --git 
a/fe/src/test/java/org/apache/impala/catalog/local/LocalCatalogTest.java 
b/fe/src/test/java/org/apache/impala/catalog/local/LocalCatalogTest.java
index 2c577447d..5f6e78fd2 100644
--- a/fe/src/test/java/org/apache/impala/catalog/local/LocalCatalogTest.java
+++ b/fe/src/test/java/org/apache/impala/catalog/local/LocalCatalogTest.java
@@ -46,6 +46,7 @@ import org.apache.impala.service.FeSupport;
 import org.apache.impala.service.Frontend;
 import org.apache.impala.testutil.TestUtils;
 import org.apache.impala.thrift.TCatalogObjectType;
+import org.apache.impala.thrift.TIcebergContentFileStore;
 import org.apache.impala.thrift.TMetadataOpRequest;
 import org.apache.impala.thrift.TMetadataOpcode;
 import org.apache.impala.thrift.TNetworkAddress;
@@ -283,9 +284,11 @@ public class LocalCatalogTest {
     ListMap<TNetworkAddress> catalogdHostIndexes = new ListMap<>();
     catalogdHostIndexes.populate(tblInfo.getNetwork_addresses());
     IcebergContentFileStore catalogFileStore = 
IcebergContentFileStore.fromThrift(
-            tblInfo.getIceberg_table().getContent_files(),
-            null, null);
-    for (FileDescriptor localFd : fileStore.getDataFiles()) {
+        tblInfo.getIceberg_table().getContent_files(),
+        null, null);
+    TIcebergContentFileStore icebergContentFileStore = 
catalogFileStore.toThrift();
+    assertEquals(tblInfo.getIceberg_table().getContent_files(), 
icebergContentFileStore);
+    for (FileDescriptor localFd : fileStore.getAllDataFiles()) {
       String path = localFd.getAbsolutePath(t.getLocation());
       // For this test table the manifest files contain data paths without 
FS-scheme, so
       // they are loaded to the file content store without them.
diff --git 
a/testdata/workloads/functional-planner/queries/PlannerTest/iceberg-v2-tables.test
 
b/testdata/workloads/functional-planner/queries/PlannerTest/iceberg-v2-tables.test
index e02ca0d7f..7186114ef 100644
--- 
a/testdata/workloads/functional-planner/queries/PlannerTest/iceberg-v2-tables.test
+++ 
b/testdata/workloads/functional-planner/queries/PlannerTest/iceberg-v2-tables.test
@@ -55,7 +55,7 @@ PLAN-ROOT SINK
 |
 |--01:SCAN HDFS 
[functional_parquet.iceberg_v2_delete_positional-POSITION-DELETE-01 
functional_parquet.iceberg_v2_delete_positional-position-delete]
 |     HDFS partitions=1/1 files=1 size=1.54KB
-|     row-size=160B cardinality=1
+|     row-size=182B cardinality=1
 |
 00:SCAN HDFS [functional_parquet.iceberg_v2_delete_positional]
    HDFS partitions=1/1 files=1 size=662B
@@ -80,7 +80,7 @@ PLAN-ROOT SINK
 |  |
 |  01:SCAN HDFS 
[functional_parquet.iceberg_v2_delete_positional-POSITION-DELETE-01 
functional_parquet.iceberg_v2_delete_positional-position-delete]
 |     HDFS partitions=1/1 files=1 size=1.54KB
-|     row-size=160B cardinality=1
+|     row-size=182B cardinality=1
 |
 00:SCAN HDFS [functional_parquet.iceberg_v2_delete_positional]
    HDFS partitions=1/1 files=1 size=662B
@@ -95,7 +95,7 @@ PLAN-ROOT SINK
 |
 |--01:SCAN HDFS 
[functional_parquet.iceberg_v2_delete_positional-POSITION-DELETE-01 
functional_parquet.iceberg_v2_delete_positional-position-delete]
 |     HDFS partitions=1/1 files=1 size=1.54KB
-|     row-size=160B cardinality=1
+|     row-size=182B cardinality=1
 |
 00:SCAN HDFS [functional_parquet.iceberg_v2_delete_positional]
    HDFS partitions=1/1 files=1 size=662B
@@ -112,7 +112,7 @@ PLAN-ROOT SINK
 |  |
 |  01:SCAN HDFS 
[functional_parquet.iceberg_v2_delete_positional-POSITION-DELETE-01 
functional_parquet.iceberg_v2_delete_positional-position-delete]
 |     HDFS partitions=1/1 files=1 size=1.54KB
-|     row-size=160B cardinality=1
+|     row-size=182B cardinality=1
 |
 00:SCAN HDFS [functional_parquet.iceberg_v2_delete_positional]
    HDFS partitions=1/1 files=1 size=662B
@@ -127,7 +127,7 @@ PLAN-ROOT SINK
 |
 |--01:SCAN HDFS 
[functional_parquet.iceberg_v2_positional_delete_all_rows-POSITION-DELETE-01 
functional_parquet.iceberg_v2_positional_delete_all_rows-position-delete]
 |     HDFS partitions=1/1 files=1 size=2.60KB
-|     row-size=224B cardinality=3
+|     row-size=246B cardinality=3
 |
 00:SCAN HDFS [functional_parquet.iceberg_v2_positional_delete_all_rows]
    HDFS partitions=1/1 files=1 size=625B
@@ -144,7 +144,7 @@ PLAN-ROOT SINK
 |  |
 |  01:SCAN HDFS 
[functional_parquet.iceberg_v2_positional_delete_all_rows-POSITION-DELETE-01 
functional_parquet.iceberg_v2_positional_delete_all_rows-position-delete]
 |     HDFS partitions=1/1 files=1 size=2.60KB
-|     row-size=224B cardinality=3
+|     row-size=246B cardinality=3
 |
 00:SCAN HDFS [functional_parquet.iceberg_v2_positional_delete_all_rows]
    HDFS partitions=1/1 files=1 size=625B
@@ -179,7 +179,7 @@ PLAN-ROOT SINK
 |
 |--01:SCAN HDFS 
[functional_parquet.iceberg_v2_positional_delete_all_rows-POSITION-DELETE-01 
functional_parquet.iceberg_v2_positional_delete_all_rows-position-delete]
 |     HDFS partitions=1/1 files=1 size=2.60KB
-|     row-size=224B cardinality=3
+|     row-size=246B cardinality=3
 |
 00:SCAN HDFS [functional_parquet.iceberg_v2_positional_delete_all_rows]
    HDFS partitions=1/1 files=1 size=625B
@@ -198,7 +198,7 @@ PLAN-ROOT SINK
 |  |
 |  01:SCAN HDFS 
[functional_parquet.iceberg_v2_positional_delete_all_rows-POSITION-DELETE-01 
functional_parquet.iceberg_v2_positional_delete_all_rows-position-delete]
 |     HDFS partitions=1/1 files=1 size=2.60KB
-|     row-size=224B cardinality=3
+|     row-size=246B cardinality=3
 |
 00:SCAN HDFS [functional_parquet.iceberg_v2_positional_delete_all_rows]
    HDFS partitions=1/1 files=1 size=625B
@@ -218,7 +218,7 @@ PLAN-ROOT SINK
 |  |
 |  |--01:SCAN HDFS 
[functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-POSITION-DELETE-01
 
functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-position-delete]
 |  |     HDFS partitions=1/1 files=2 size=5.33KB
-|  |     row-size=245B cardinality=4
+|  |     row-size=267B cardinality=4
 |  |
 |  00:SCAN HDFS 
[functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files]
 |     HDFS partitions=1/1 files=2 size=1.22KB
@@ -245,7 +245,7 @@ PLAN-ROOT SINK
 |  |  |
 |  |  01:SCAN HDFS 
[functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-POSITION-DELETE-01
 
functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-position-delete]
 |  |     HDFS partitions=1/1 files=2 size=5.33KB
-|  |     row-size=245B cardinality=4
+|  |     row-size=267B cardinality=4
 |  |
 |  05:EXCHANGE 
[HASH(functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files.file__position,functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files.input__file__name)]
 |  |
@@ -270,7 +270,7 @@ PLAN-ROOT SINK
 |  |
 |  |--01:SCAN HDFS 
[functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-POSITION-DELETE-01
 
functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-position-delete]
 |  |     HDFS partitions=1/1 files=2 size=5.33KB
-|  |     row-size=245B cardinality=4
+|  |     row-size=267B cardinality=4
 |  |
 |  00:SCAN HDFS 
[functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files]
 |     HDFS partitions=1/1 files=2 size=1.22KB
@@ -295,7 +295,7 @@ PLAN-ROOT SINK
 |  |  |
 |  |  01:SCAN HDFS 
[functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-POSITION-DELETE-01
 
functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-position-delete]
 |  |     HDFS partitions=1/1 files=2 size=5.33KB
-|  |     row-size=245B cardinality=4
+|  |     row-size=267B cardinality=4
 |  |
 |  05:EXCHANGE 
[HASH(functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files.file__position,functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files.input__file__name)]
 |  |
@@ -320,7 +320,7 @@ PLAN-ROOT SINK
 |  |
 |  |--01:SCAN HDFS 
[functional_parquet.iceberg_v2_positional_update_all_rows-POSITION-DELETE-01 
functional_parquet.iceberg_v2_positional_update_all_rows-position-delete]
 |  |     HDFS partitions=1/1 files=1 size=2.60KB
-|  |     row-size=224B cardinality=3
+|  |     row-size=246B cardinality=3
 |  |
 |  00:SCAN HDFS [functional_parquet.iceberg_v2_positional_update_all_rows]
 |     HDFS partitions=1/1 files=1 size=625B
@@ -345,7 +345,7 @@ PLAN-ROOT SINK
 |  |  |
 |  |  01:SCAN HDFS 
[functional_parquet.iceberg_v2_positional_update_all_rows-POSITION-DELETE-01 
functional_parquet.iceberg_v2_positional_update_all_rows-position-delete]
 |  |     HDFS partitions=1/1 files=1 size=2.60KB
-|  |     row-size=224B cardinality=3
+|  |     row-size=246B cardinality=3
 |  |
 |  00:SCAN HDFS [functional_parquet.iceberg_v2_positional_update_all_rows]
 |     HDFS partitions=1/1 files=1 size=625B
@@ -364,7 +364,7 @@ PLAN-ROOT SINK
 |
 |--01:SCAN HDFS 
[functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 
functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
 |     HDFS partitions=1/1 files=3 size=9.47KB
-|     row-size=182B cardinality=10
+|     row-size=204B cardinality=10
 |
 00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
    HDFS partitions=1/1 files=3 size=3.48KB
@@ -381,7 +381,7 @@ PLAN-ROOT SINK
 |  |
 |  01:SCAN HDFS 
[functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 
functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
 |     HDFS partitions=1/1 files=3 size=9.47KB
-|     row-size=182B cardinality=10
+|     row-size=204B cardinality=10
 |
 03:EXCHANGE 
[HASH(functional_parquet.iceberg_v2_partitioned_position_deletes.file__position,functional_parquet.iceberg_v2_partitioned_position_deletes.input__file__name)]
 |
@@ -403,7 +403,7 @@ PLAN-ROOT SINK
 |  |
 |  |--01:SCAN HDFS 
[functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-POSITION-DELETE-01
 
functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-position-delete]
 |  |     HDFS partitions=1/1 files=2 size=5.33KB
-|  |     row-size=245B cardinality=4
+|  |     row-size=267B cardinality=4
 |  |
 |  00:SCAN HDFS 
[functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files]
 |     HDFS partitions=1/1 files=2 size=1.22KB
@@ -430,7 +430,7 @@ PLAN-ROOT SINK
 |  |  |
 |  |  01:SCAN HDFS 
[functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-POSITION-DELETE-01
 
functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-position-delete]
 |  |     HDFS partitions=1/1 files=2 size=5.33KB
-|  |     row-size=245B cardinality=4
+|  |     row-size=267B cardinality=4
 |  |
 |  05:EXCHANGE 
[HASH(functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files.file__position,functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files.input__file__name)]
 |  |
@@ -471,7 +471,7 @@ PLAN-ROOT SINK
 |  |
 |  |--01:SCAN HDFS 
[functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-POSITION-DELETE-01
 
functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-position-delete]
 |  |     HDFS partitions=1/1 files=2 size=5.33KB
-|  |     row-size=245B cardinality=4
+|  |     row-size=267B cardinality=4
 |  |
 |  00:SCAN HDFS 
[functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files]
 |     HDFS partitions=1/1 files=2 size=1.22KB
@@ -516,7 +516,7 @@ PLAN-ROOT SINK
 |  |  |
 |  |  01:SCAN HDFS 
[functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-POSITION-DELETE-01
 
functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-position-delete]
 |  |     HDFS partitions=1/1 files=2 size=5.33KB
-|  |     row-size=245B cardinality=4
+|  |     row-size=267B cardinality=4
 |  |
 |  08:EXCHANGE 
[HASH(functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files.file__position,functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files.input__file__name)]
 |  |
@@ -542,7 +542,7 @@ PLAN-ROOT SINK
 |  |
 |  |--01:SCAN HDFS 
[functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-POSITION-DELETE-01
 
functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-position-delete]
 |  |     HDFS partitions=1/1 files=2 size=5.33KB
-|  |     row-size=245B cardinality=4
+|  |     row-size=267B cardinality=4
 |  |
 |  00:SCAN HDFS 
[functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files]
 |     HDFS partitions=1/1 files=2 size=1.22KB
@@ -569,7 +569,7 @@ PLAN-ROOT SINK
 |  |  |
 |  |  01:SCAN HDFS 
[functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-POSITION-DELETE-01
 
functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-position-delete]
 |  |     HDFS partitions=1/1 files=2 size=5.33KB
-|  |     row-size=245B cardinality=4
+|  |     row-size=267B cardinality=4
 |  |
 |  05:EXCHANGE 
[HASH(functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files.file__position,functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files.input__file__name)]
 |  |
@@ -607,7 +607,7 @@ PLAN-ROOT SINK
 |  |  |
 |  |  |--06:SCAN HDFS 
[functional_parquet.iceberg_v2_positional_update_all_rows-POSITION-DELETE-06 
functional_parquet.iceberg_v2_positional_update_all_rows-position-delete]
 |  |  |     HDFS partitions=1/1 files=1 size=2.60KB
-|  |  |     row-size=224B cardinality=3
+|  |  |     row-size=246B cardinality=3
 |  |  |
 |  |  05:SCAN HDFS [functional_parquet.iceberg_v2_positional_update_all_rows]
 |  |     HDFS partitions=1/1 files=1 size=625B
@@ -626,7 +626,7 @@ PLAN-ROOT SINK
 |  |
 |  |--01:SCAN HDFS 
[functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-POSITION-DELETE-01
 
functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-position-delete]
 |  |     HDFS partitions=1/1 files=2 size=5.33KB
-|  |     row-size=245B cardinality=4
+|  |     row-size=267B cardinality=4
 |  |
 |  00:SCAN HDFS 
[functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files]
 |     HDFS partitions=1/1 files=2 size=1.22KB
@@ -670,7 +670,7 @@ PLAN-ROOT SINK
 |  |  |  |
 |  |  |  06:SCAN HDFS 
[functional_parquet.iceberg_v2_positional_update_all_rows-POSITION-DELETE-06 
functional_parquet.iceberg_v2_positional_update_all_rows-position-delete]
 |  |  |     HDFS partitions=1/1 files=1 size=2.60KB
-|  |  |     row-size=224B cardinality=3
+|  |  |     row-size=246B cardinality=3
 |  |  |
 |  |  05:SCAN HDFS [functional_parquet.iceberg_v2_positional_update_all_rows]
 |  |     HDFS partitions=1/1 files=1 size=625B
@@ -691,7 +691,7 @@ PLAN-ROOT SINK
 |  |  |
 |  |  01:SCAN HDFS 
[functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-POSITION-DELETE-01
 
functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-position-delete]
 |  |     HDFS partitions=1/1 files=2 size=5.33KB
-|  |     row-size=245B cardinality=4
+|  |     row-size=267B cardinality=4
 |  |
 |  12:EXCHANGE 
[HASH(functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files.file__position,functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files.input__file__name)]
 |  |
@@ -714,7 +714,7 @@ PLAN-ROOT SINK
 |
 |--01:SCAN HDFS 
[functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 
functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
 |     HDFS partitions=1/1 files=1 size=3.18KB
-|     row-size=185B cardinality=2
+|     row-size=207B cardinality=2
 |
 00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
    HDFS partitions=1/1 files=1 size=1.17KB
@@ -732,7 +732,7 @@ PLAN-ROOT SINK
 |  |
 |  01:SCAN HDFS 
[functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 
functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
 |     HDFS partitions=1/1 files=1 size=3.18KB
-|     row-size=185B cardinality=2
+|     row-size=207B cardinality=2
 |
 00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
    HDFS partitions=1/1 files=1 size=1.17KB
@@ -749,7 +749,7 @@ PLAN-ROOT SINK
 |
 |--01:SCAN HDFS 
[functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 
functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
 |     HDFS partitions=1/1 files=1 size=3.18KB
-|     row-size=185B cardinality=2
+|     row-size=207B cardinality=2
 |
 00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
    HDFS partitions=1/1 files=1 size=1.17KB
@@ -767,7 +767,7 @@ PLAN-ROOT SINK
 |  |
 |  01:SCAN HDFS 
[functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 
functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
 |     HDFS partitions=1/1 files=1 size=3.18KB
-|     row-size=185B cardinality=2
+|     row-size=207B cardinality=2
 |
 00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
    HDFS partitions=1/1 files=1 size=1.17KB


Reply via email to