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

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


The following commit(s) were added to refs/heads/master by this push:
     new 0cc45257a3 [jindo] Add listFilesIterative implementation in 
HadoopCompliantFileIO
0cc45257a3 is described below

commit 0cc45257a35782a26e51dca55baa950e70965bea
Author: JingsongLi <[email protected]>
AuthorDate: Wed Nov 19 22:41:09 2025 +0800

    [jindo] Add listFilesIterative implementation in HadoopCompliantFileIO
---
 .../apache/paimon/jindo/HadoopCompliantFileIO.java  | 21 +++++++++++++++++++++
 1 file changed, 21 insertions(+)

diff --git 
a/paimon-filesystems/paimon-jindo/src/main/java/org/apache/paimon/jindo/HadoopCompliantFileIO.java
 
b/paimon-filesystems/paimon-jindo/src/main/java/org/apache/paimon/jindo/HadoopCompliantFileIO.java
index 7356b9687a..3ced092f1d 100644
--- 
a/paimon-filesystems/paimon-jindo/src/main/java/org/apache/paimon/jindo/HadoopCompliantFileIO.java
+++ 
b/paimon-filesystems/paimon-jindo/src/main/java/org/apache/paimon/jindo/HadoopCompliantFileIO.java
@@ -22,6 +22,7 @@ import org.apache.paimon.fs.FileIO;
 import org.apache.paimon.fs.FileStatus;
 import org.apache.paimon.fs.Path;
 import org.apache.paimon.fs.PositionOutputStream;
+import org.apache.paimon.fs.RemoteIterator;
 import org.apache.paimon.fs.SeekableInputStream;
 import org.apache.paimon.fs.VectoredReadable;
 import org.apache.paimon.utils.Pair;
@@ -87,6 +88,26 @@ public abstract class HadoopCompliantFileIO implements 
FileIO {
         return statuses;
     }
 
+    @Override
+    public RemoteIterator<FileStatus> listFilesIterative(Path path, boolean 
recursive)
+            throws IOException {
+        org.apache.hadoop.fs.Path hadoopPath = path(path);
+        
org.apache.hadoop.fs.RemoteIterator<org.apache.hadoop.fs.LocatedFileStatus> 
hadoopIter =
+                getFileSystem(hadoopPath).listFiles(hadoopPath, recursive);
+        return new RemoteIterator<FileStatus>() {
+            @Override
+            public boolean hasNext() throws IOException {
+                return hadoopIter.hasNext();
+            }
+
+            @Override
+            public FileStatus next() throws IOException {
+                org.apache.hadoop.fs.FileStatus hadoopStatus = 
hadoopIter.next();
+                return new HadoopFileStatus(hadoopStatus);
+            }
+        };
+    }
+
     @Override
     public boolean exists(Path path) throws IOException {
         org.apache.hadoop.fs.Path hadoopPath = path(path);

Reply via email to