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 18c963ca25 [pvfs] support pread in pvfs (#7247)
18c963ca25 is described below

commit 18c963ca25ba9daec56de4fdb0f4d0d6bdc03e9c
Author: shyjsarah <[email protected]>
AuthorDate: Tue Feb 10 02:25:01 2026 -0800

    [pvfs] support pread in pvfs (#7247)
---
 .../org/apache/paimon/vfs/hadoop/VFSInputStream.java    | 17 ++++++++++++++---
 1 file changed, 14 insertions(+), 3 deletions(-)

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 7356e87bb9..f8156554ff 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
@@ -19,6 +19,7 @@
 package org.apache.paimon.vfs.hadoop;
 
 import org.apache.paimon.fs.SeekableInputStream;
+import org.apache.paimon.fs.VectoredReadable;
 
 import org.apache.hadoop.fs.FSInputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -28,9 +29,7 @@ import java.io.IOException;
 /**
  * VFSInputStream wrap over paimon SeekableInputStream to support hadoop 
FSDataInputStream. TODO:
  * SeekableInputStream interface is too simple to fully support all 
FSDataInputStream operations: 1.
- * ByteBufferReadable and ByteBufferPositionedReadable should be implemented 
for full support. 2.
- * Positioned read is not supported in SeekableInputStream, so it is by 
default implemented by
- * sequence read, which is not a good solution.
+ * ByteBufferReadable and ByteBufferPositionedReadable should be implemented 
for full support.
  */
 public class VFSInputStream extends FSInputStream {
     private SeekableInputStream in;
@@ -57,6 +56,18 @@ public class VFSInputStream extends FSInputStream {
         return false;
     }
 
+    @Override
+    public int read(long position, byte[] buffer, int offset, int length) 
throws IOException {
+        if (in instanceof VectoredReadable) {
+            int byteRead = ((VectoredReadable) in).pread(position, buffer, 
offset, length);
+            if (statistics != null && byteRead >= 0) {
+                statistics.incrementBytesRead(byteRead);
+            }
+            return byteRead;
+        }
+        return super.read(position, buffer, offset, length);
+    }
+
     @Override
     public int read(byte[] b, int off, int len) throws IOException {
         int byteRead = in.read(b, off, len);

Reply via email to