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 70382b129b [pvfs] Fix file status and input stream for PVFS (#6397)
70382b129b is described below

commit 70382b129babc716d271bae3162a09bba11ef60e
Author: timmyyao <[email protected]>
AuthorDate: Tue Oct 14 15:57:08 2025 +0800

    [pvfs] Fix file status and input stream for PVFS (#6397)
---
 .../apache/paimon/vfs/hadoop/PaimonVirtualFileSystem.java    | 12 +++++++-----
 .../java/org/apache/paimon/vfs/hadoop/VFSInputStream.java    |  5 +++++
 .../org/apache/paimon/vfs/hadoop/VirtualFileSystemTest.java  |  2 ++
 3 files changed, 14 insertions(+), 5 deletions(-)

diff --git 
a/paimon-vfs/paimon-vfs-hadoop/src/main/java/org/apache/paimon/vfs/hadoop/PaimonVirtualFileSystem.java
 
b/paimon-vfs/paimon-vfs-hadoop/src/main/java/org/apache/paimon/vfs/hadoop/PaimonVirtualFileSystem.java
index 3687460d2c..356d96f6f0 100644
--- 
a/paimon-vfs/paimon-vfs-hadoop/src/main/java/org/apache/paimon/vfs/hadoop/PaimonVirtualFileSystem.java
+++ 
b/paimon-vfs/paimon-vfs-hadoop/src/main/java/org/apache/paimon/vfs/hadoop/PaimonVirtualFileSystem.java
@@ -59,6 +59,7 @@ public class PaimonVirtualFileSystem extends FileSystem {
     private Configuration conf;
 
     private static final String USER_AGENT = "HadoopPVFS";
+    private static final long DEFAULT_BLOCK_SIZE = 128 * 1024 * 1024L;
 
     @Override
     public void initialize(URI uri, Configuration conf) throws IOException {
@@ -299,7 +300,7 @@ public class PaimonVirtualFileSystem extends FileSystem {
     public FileStatus getFileStatus(Path f) throws IOException {
         VFSIdentifier vfsIdentifier = 
vfsOperations.getVFSIdentifier(getVirtualPath(f));
         if (vfsIdentifier instanceof VFSCatalogIdentifier) {
-            return new FileStatus(0, true, 1, 1, 0, new Path(this.uri));
+            return new FileStatus(0, true, 1, 0, 0, new Path(this.uri));
         } else if (vfsIdentifier instanceof VFSDatabaseIdentifier) {
             String databaseName = ((VFSDatabaseIdentifier) 
vfsIdentifier).databaseName();
             GetDatabaseResponse database = 
vfsOperations.getDatabase(databaseName);
@@ -318,7 +319,7 @@ public class PaimonVirtualFileSystem extends FileSystem {
     }
 
     private FileStatus convertDatabase(GetDatabaseResponse database) {
-        return new FileStatus(0, true, 1, 1, 0, new Path(new Path(this.uri), 
database.getName()));
+        return new FileStatus(0, true, 1, 0, 0, new Path(new Path(this.uri), 
database.getName()));
     }
 
     private FileStatus convertFileStatus(
@@ -338,11 +339,12 @@ public class PaimonVirtualFileSystem extends FileSystem {
             childPath = "/" + childPath;
         }
         Path virtualPath = new Path(new Path(this.uri), databaseName + "/" + 
tableName + childPath);
+        long blockSize = fileStatus.isDir() ? 0 : DEFAULT_BLOCK_SIZE;
         return new FileStatus(
                 fileStatus.getLen(),
                 fileStatus.isDir(),
                 1,
-                1,
+                blockSize,
                 fileStatus.getModificationTime(),
                 virtualPath);
     }
@@ -375,7 +377,7 @@ public class PaimonVirtualFileSystem extends FileSystem {
         for (int i = 0; i < databases.size(); i++) {
             String database = databases.get(i);
             FileStatus fileStatus =
-                    new FileStatus(0, true, 1, 1, 0, new Path(new 
Path(this.uri), database));
+                    new FileStatus(0, true, 1, 0, 0, new Path(new 
Path(this.uri), database));
             fileStatuses[i] = fileStatus;
         }
         return fileStatuses;
@@ -387,7 +389,7 @@ public class PaimonVirtualFileSystem extends FileSystem {
             String table = tables.get(i);
             FileStatus fileStatus =
                     new FileStatus(
-                            0, true, 1, 1, 0, new Path(new Path(this.uri), 
database + "/" + table));
+                            0, true, 1, 0, 0, new Path(new Path(this.uri), 
database + "/" + table));
             fileStatuses[i] = fileStatus;
         }
         return fileStatuses;
diff --git 
a/paimon-vfs/paimon-vfs-hadoop/src/main/java/org/apache/paimon/vfs/hadoop/VFSInputStream.java
 
b/paimon-vfs/paimon-vfs-hadoop/src/main/java/org/apache/paimon/vfs/hadoop/VFSInputStream.java
index 020a58d318..7356e87bb9 100644
--- 
a/paimon-vfs/paimon-vfs-hadoop/src/main/java/org/apache/paimon/vfs/hadoop/VFSInputStream.java
+++ 
b/paimon-vfs/paimon-vfs-hadoop/src/main/java/org/apache/paimon/vfs/hadoop/VFSInputStream.java
@@ -77,4 +77,9 @@ public class VFSInputStream extends FSInputStream {
         }
         return (n == -1) ? -1 : oneByteBuf[0] & 0xff;
     }
+
+    @Override
+    public void close() throws IOException {
+        in.close();
+    }
 }
diff --git 
a/paimon-vfs/paimon-vfs-hadoop/src/test/java/org/apache/paimon/vfs/hadoop/VirtualFileSystemTest.java
 
b/paimon-vfs/paimon-vfs-hadoop/src/test/java/org/apache/paimon/vfs/hadoop/VirtualFileSystemTest.java
index d6e50d1fb0..62c893a52b 100644
--- 
a/paimon-vfs/paimon-vfs-hadoop/src/test/java/org/apache/paimon/vfs/hadoop/VirtualFileSystemTest.java
+++ 
b/paimon-vfs/paimon-vfs-hadoop/src/test/java/org/apache/paimon/vfs/hadoop/VirtualFileSystemTest.java
@@ -137,6 +137,7 @@ public abstract class VirtualFileSystemTest {
         FileStatus fileStatus = vfs.getFileStatus(vfsPath);
         Assert.assertEquals(vfsPath.toString(), 
fileStatus.getPath().toString());
         Assert.assertTrue(fileStatus.isDirectory());
+        Assert.assertEquals(0, fileStatus.getBlockSize());
 
         // Mkdir in non-existing table
         tableName = "object_table2";
@@ -195,6 +196,7 @@ public abstract class VirtualFileSystemTest {
         Assert.assertEquals(vfsPath.toString(), 
fileStatus.getPath().toString());
         Assert.assertTrue(fileStatus.isFile());
         Assert.assertEquals(5, fileStatus.getLen());
+        Assert.assertEquals(128 * 1024 * 1024L, fileStatus.getBlockSize());
 
         FSDataInputStream in = vfs.open(vfsPath);
         byte[] buffer = new byte[5];

Reply via email to