gaborgsomogyi commented on code in PR #28112:
URL: https://github.com/apache/flink/pull/28112#discussion_r3199962035


##########
flink-filesystems/flink-s3-fs-native/src/test/java/org/apache/flink/fs/s3native/NativeS3InputStreamTest.java:
##########
@@ -57,9 +58,16 @@ private static class TrackingInputStream extends InputStream 
implements Abortabl
         private final AtomicBoolean aborted = new AtomicBoolean();
         private final AtomicBoolean closed = new AtomicBoolean();
         private volatile boolean abortedBeforeClose;
+        private final int maxAvailable;
+        private final AtomicLong bytesSkipped = new AtomicLong();
 
         TrackingInputStream(byte[] data, int offset) {
+            this(data, offset, Integer.MAX_VALUE);
+        }
+
+        TrackingInputStream(byte[] data, int offset, int maxAvailable) {
             this.delegate = new ByteArrayInputStream(data, offset, data.length 
- offset);
+            this.maxAvailable = maxAvailable;
         }
 
         TrackingInputStream(byte[] data) {

Review Comment:
   This is unused



##########
flink-filesystems/flink-s3-fs-native/src/test/java/org/apache/flink/fs/s3native/NativeS3InputStreamTest.java:
##########
@@ -333,15 +468,116 @@ void skipToEofWithOpenStreamReleasesStream() throws 
Exception {
 
             assertThat(in.skip(DATA.length)).isEqualTo(DATA.length - 1);
             assertThat(in.getPos()).isEqualTo(DATA.length);
+            assertThat(first.wasAborted()).isFalse();
 
-            assertThat(first.wasAborted()).isTrue();
-            assertThat(first.wasClosed()).isTrue();
-            assertThat(first.wasAbortedBeforeClose()).isTrue();
+            assertThat(in.read()).isEqualTo(-1);
+        }
+    }
+
+    @Test
+    void skipZeroAndNegativeAreNoOps() throws Exception {
+        StubS3Client client = new StubS3Client(DATA);
+        try (NativeS3InputStream in = new NativeS3InputStream(client, BUCKET, 
KEY, DATA.length)) {
+            in.read();
+            assertThat(in.skip(0)).isZero();
+            assertThat(in.skip(-5)).isZero();
             assertThat(client.getObjectCalls()).isEqualTo(1);
+        }
+    }
+
+    // --- read + seek integration ---
+
+    @Test
+    void readAndSeekReturnCorrectData() throws Exception {
+        StubS3Client client = new StubS3Client(DATA);
+        try (NativeS3InputStream in = new NativeS3InputStream(client, BUCKET, 
KEY, DATA.length)) {
+            assertThat(in.read()).isEqualTo(0);
+            assertThat(in.getPos()).isEqualTo(1);
+            byte[] buf = new byte[10];
+            assertThat(in.read(buf, 0, 10)).isEqualTo(10);
+            assertThat(in.getPos()).isEqualTo(11);
+            for (int i = 0; i < 10; i++) {
+                assertThat(buf[i]).isEqualTo(DATA[i + 1]);
+            }
+            assertThat(in.available()).isEqualTo(DATA.length - 11);
+            in.seek(200);
+            assertThat(in.read()).isEqualTo(200);
+            assertThat(in.getPos()).isEqualTo(201);
+            in.seek(250);
+            byte[] tail = new byte[20];
+            assertThat(in.read(tail, 0, 20)).isEqualTo(6);
+            assertThat(in.getPos()).isEqualTo(256);
             assertThat(in.read()).isEqualTo(-1);
+            assertThat(in.read(new byte[1], 0, 1)).isEqualTo(-1);
+        }
+    }
+
+    // --- buffer-efficiency: skip stays in local buffer vs. underlying stream 
---
+
+    @Test
+    void seekWithinBuffer_afterSmallRead_doesNotTouchUnderlyingStream() throws 
Exception {
+        int smallBuffer = 32;
+        StubS3Client client = new StubS3Client(DATA);
+        try (NativeS3InputStream in =
+                new NativeS3InputStream(client, BUCKET, KEY, DATA.length, 
smallBuffer)) {
+            // single-byte read fills the local buffer (up to smallBuffer 
bytes buffered)
+            in.read();
+            TrackingInputStream underlying = client.lastStream();
+
+            // forward seek of 9 bytes — fits in the local buffer, no 
underlying access needed
+            in.seek(10);
+            in.read();
+
+            assertThat(client.getObjectCalls()).isEqualTo(1);
+            assertThat(underlying.bytesSkippedFromUnderlying()).isEqualTo(0);
+        }
+    }
+
+    @Test
+    void seekWithinBuffer_afterLargeRead_touchesUnderlyingStream() throws 
Exception {
+        int smallBuffer = 16;
+        StubS3Client client = new StubS3Client(DATA);
+        try (NativeS3InputStream in =
+                new NativeS3InputStream(client, BUCKET, KEY, DATA.length, 
smallBuffer)) {
+            // bulk read >= bufferSize bypasses BufferedInputStream's local 
array entirely,
+            // leaving the local buffer empty afterward
+            byte[] buf = new byte[smallBuffer];
+            in.read(buf, 0, smallBuffer);
+            TrackingInputStream underlying = client.lastStream();
+
+            // forward seek of 10 bytes — within readBufferSize but buffer is 
empty, so
+            // BufferedInputStream.skip() delegates directly to the underlying 
stream
+            in.seek((long) smallBuffer + 10);
+            in.read();
+
+            assertThat(client.getObjectCalls()).isEqualTo(1);
+            assertThat(underlying.bytesSkippedFromUnderlying()).isEqualTo(10);
         }
     }
 
+    @Test
+    void 
seekBeyondReadBufferSize_inflatedByUnderlyingAvailable_reopensWithRangeRequest()
+            throws Exception {
+        int smallBuffer = 16;
+        // default maxAvailable = Integer.MAX_VALUE: underlying stream reports 
a large available(),
+        // which used to inflate forwardSeekLimit and incorrectly suppress 
range-request reopens

Review Comment:
   This will tell almost nothing to devs who is not read this discussion so I 
think we can remove it.



##########
flink-filesystems/flink-s3-fs-native/src/main/java/org/apache/flink/fs/s3native/NativeS3InputStream.java:
##########
@@ -98,39 +110,60 @@ public NativeS3InputStream(
                 this.readBufferSize / 1024);
     }
 
+    /** Reconciles {@link #nextReadPos} and {@link #streamPos} before reading 
bytes. */
     @GuardedBy("lock")
-    private void lazyInitialize() throws IOException {
-        assert lock.isHeldByCurrentThread() : "lazyInitialize() requires lock 
to be held";
-        if (currentStream == null && !closed) {
-            openStreamAtCurrentPosition();
-        }
-    }
+    private void lazySeek() throws IOException {
+        assert lock.isHeldByCurrentThread() : "lazySeek() requires lock to be 
held";
+        long targetPos = nextReadPos;
 
-    /** At EOF, release instead of reopening: {@code bytes=contentLength-} 
returns S3 416. */
-    @GuardedBy("lock")
-    private void repositionOpenStream() throws IOException {
-        assert lock.isHeldByCurrentThread() : "repositionOpenStream() requires 
lock to be held";
         if (currentStream == null) {
+            streamPos = targetPos;
             return;
         }
-        if (position >= contentLength) {
+
+        if (targetPos == streamPos) {
+            return;
+        }
+
+        long diff = targetPos - streamPos;
+        streamPos = targetPos;
+
+        if (targetPos >= contentLength) {
             releaseStreams();
-        } else {
+            return;
+        }
+
+        if (diff > 0 && diff <= (long) readBufferSize) {
+            skipBytesInBuffer(diff);
+            return;
+        }
+
+        openStreamAtCurrentPosition();
+    }
+
+    @GuardedBy("lock")
+    private void ensureStreamOpen() throws IOException {
+        assert lock.isHeldByCurrentThread() : "ensureStreamOpen() requires 
lock to be held";
+        if (currentStream == null && !closed) {
             openStreamAtCurrentPosition();
         }
     }
 
-    /**
-     * Opens (or reopens) the S3 stream at the current position.
-     *
-     * <p>This method:
-     *
-     * <ul>
-     *   <li>Closes any existing stream
-     *   <li>Opens a new stream starting at {@link #position}
-     *   <li>Uses HTTP range requests for non-zero positions
-     * </ul>
-     */
+    @GuardedBy("lock")
+    private void skipBytesInBuffer(long n) throws IOException {
+        assert lock.isHeldByCurrentThread() : "skipBytesInBuffer() requires 
lock to be held";
+        long remaining = n;
+        while (remaining > 0) {
+            long skipped = bufferedStream.skip(remaining);
+            if (skipped <= 0) {
+                openStreamAtCurrentPosition();

Review Comment:
   `skipBytesInBuffer` has an untested recovery path: if 
`bufferedStream.skip()` returns <= 0 mid-skip (underlying HTTP connection 
stalled or closed), it falls back to `openStreamAtCurrentPosition()`. This is 
correct - streamPos is already set to targetPos before skipBytesInBuffer is 
called, so the fresh range request lands at the right position - but it is 
never exercised by the current test suite.
   
   A targeted test would extend `TrackingInputStream` to `return 0` from 
`skip()` on the first call, then delegate normally. The setup: do a small read 
(fills buffer), seek forward within `readBufferSize`, trigger the skip failure, 
then assert the correct byte is returned and `getObjectCalls() == 2` (new range 
request was issued at the seek target).
   



##########
flink-filesystems/flink-s3-fs-native/src/main/java/org/apache/flink/fs/s3native/NativeS3InputStream.java:
##########
@@ -98,39 +110,60 @@ public NativeS3InputStream(
                 this.readBufferSize / 1024);
     }
 
+    /** Reconciles {@link #nextReadPos} and {@link #streamPos} before reading 
bytes. */
     @GuardedBy("lock")
-    private void lazyInitialize() throws IOException {
-        assert lock.isHeldByCurrentThread() : "lazyInitialize() requires lock 
to be held";
-        if (currentStream == null && !closed) {
-            openStreamAtCurrentPosition();
-        }
-    }
+    private void lazySeek() throws IOException {
+        assert lock.isHeldByCurrentThread() : "lazySeek() requires lock to be 
held";
+        long targetPos = nextReadPos;
 
-    /** At EOF, release instead of reopening: {@code bytes=contentLength-} 
returns S3 416. */
-    @GuardedBy("lock")
-    private void repositionOpenStream() throws IOException {
-        assert lock.isHeldByCurrentThread() : "repositionOpenStream() requires 
lock to be held";
         if (currentStream == null) {
+            streamPos = targetPos;
             return;
         }
-        if (position >= contentLength) {
+
+        if (targetPos == streamPos) {
+            return;
+        }
+
+        long diff = targetPos - streamPos;
+        streamPos = targetPos;
+
+        if (targetPos >= contentLength) {
             releaseStreams();
-        } else {
+            return;
+        }
+
+        if (diff > 0 && diff <= (long) readBufferSize) {

Review Comment:
   I would put comment here explaining that we can't really find out the data 
size sitting inside the buffer without heavy or shaky development 
effort(extending JDK class) so the tradeoff is to accept max buffer size amount 
of skips during seeks.
   
   As a side note which we don't need to mention as comment. If that's not 
performing well we can add a new config which controls the seek behavior, like 
`s3.max-forward-seek-bytes` or something similar.



##########
flink-filesystems/flink-s3-fs-native/src/test/java/org/apache/flink/fs/s3native/NativeS3InputStreamTest.java:
##########
@@ -333,15 +468,116 @@ void skipToEofWithOpenStreamReleasesStream() throws 
Exception {
 
             assertThat(in.skip(DATA.length)).isEqualTo(DATA.length - 1);
             assertThat(in.getPos()).isEqualTo(DATA.length);
+            assertThat(first.wasAborted()).isFalse();
 
-            assertThat(first.wasAborted()).isTrue();
-            assertThat(first.wasClosed()).isTrue();
-            assertThat(first.wasAbortedBeforeClose()).isTrue();
+            assertThat(in.read()).isEqualTo(-1);
+        }
+    }
+
+    @Test
+    void skipZeroAndNegativeAreNoOps() throws Exception {
+        StubS3Client client = new StubS3Client(DATA);
+        try (NativeS3InputStream in = new NativeS3InputStream(client, BUCKET, 
KEY, DATA.length)) {
+            in.read();
+            assertThat(in.skip(0)).isZero();
+            assertThat(in.skip(-5)).isZero();
             assertThat(client.getObjectCalls()).isEqualTo(1);
+        }
+    }
+
+    // --- read + seek integration ---
+
+    @Test
+    void readAndSeekReturnCorrectData() throws Exception {
+        StubS3Client client = new StubS3Client(DATA);
+        try (NativeS3InputStream in = new NativeS3InputStream(client, BUCKET, 
KEY, DATA.length)) {
+            assertThat(in.read()).isEqualTo(0);
+            assertThat(in.getPos()).isEqualTo(1);
+            byte[] buf = new byte[10];
+            assertThat(in.read(buf, 0, 10)).isEqualTo(10);
+            assertThat(in.getPos()).isEqualTo(11);
+            for (int i = 0; i < 10; i++) {
+                assertThat(buf[i]).isEqualTo(DATA[i + 1]);
+            }
+            assertThat(in.available()).isEqualTo(DATA.length - 11);
+            in.seek(200);
+            assertThat(in.read()).isEqualTo(200);
+            assertThat(in.getPos()).isEqualTo(201);
+            in.seek(250);
+            byte[] tail = new byte[20];
+            assertThat(in.read(tail, 0, 20)).isEqualTo(6);
+            assertThat(in.getPos()).isEqualTo(256);
             assertThat(in.read()).isEqualTo(-1);
+            assertThat(in.read(new byte[1], 0, 1)).isEqualTo(-1);
+        }
+    }
+
+    // --- buffer-efficiency: skip stays in local buffer vs. underlying stream 
---
+
+    @Test
+    void seekWithinBuffer_afterSmallRead_doesNotTouchUnderlyingStream() throws 
Exception {
+        int smallBuffer = 32;
+        StubS3Client client = new StubS3Client(DATA);
+        try (NativeS3InputStream in =
+                new NativeS3InputStream(client, BUCKET, KEY, DATA.length, 
smallBuffer)) {
+            // single-byte read fills the local buffer (up to smallBuffer 
bytes buffered)
+            in.read();

Review Comment:
   I think for correctness we should assert on read values in all tests (we 
read from proper place and proper values). Additionally this comment is 
promising that the buffer is filled up which is not asserted.



##########
flink-filesystems/flink-s3-fs-native/src/test/java/org/apache/flink/fs/s3native/NativeS3InputStreamTest.java:
##########
@@ -333,15 +468,116 @@ void skipToEofWithOpenStreamReleasesStream() throws 
Exception {
 
             assertThat(in.skip(DATA.length)).isEqualTo(DATA.length - 1);
             assertThat(in.getPos()).isEqualTo(DATA.length);
+            assertThat(first.wasAborted()).isFalse();
 
-            assertThat(first.wasAborted()).isTrue();
-            assertThat(first.wasClosed()).isTrue();
-            assertThat(first.wasAbortedBeforeClose()).isTrue();
+            assertThat(in.read()).isEqualTo(-1);
+        }
+    }
+
+    @Test
+    void skipZeroAndNegativeAreNoOps() throws Exception {
+        StubS3Client client = new StubS3Client(DATA);
+        try (NativeS3InputStream in = new NativeS3InputStream(client, BUCKET, 
KEY, DATA.length)) {
+            in.read();
+            assertThat(in.skip(0)).isZero();
+            assertThat(in.skip(-5)).isZero();
             assertThat(client.getObjectCalls()).isEqualTo(1);
+        }
+    }
+
+    // --- read + seek integration ---
+
+    @Test
+    void readAndSeekReturnCorrectData() throws Exception {
+        StubS3Client client = new StubS3Client(DATA);
+        try (NativeS3InputStream in = new NativeS3InputStream(client, BUCKET, 
KEY, DATA.length)) {
+            assertThat(in.read()).isEqualTo(0);
+            assertThat(in.getPos()).isEqualTo(1);
+            byte[] buf = new byte[10];
+            assertThat(in.read(buf, 0, 10)).isEqualTo(10);
+            assertThat(in.getPos()).isEqualTo(11);
+            for (int i = 0; i < 10; i++) {
+                assertThat(buf[i]).isEqualTo(DATA[i + 1]);
+            }
+            assertThat(in.available()).isEqualTo(DATA.length - 11);
+            in.seek(200);
+            assertThat(in.read()).isEqualTo(200);
+            assertThat(in.getPos()).isEqualTo(201);
+            in.seek(250);
+            byte[] tail = new byte[20];
+            assertThat(in.read(tail, 0, 20)).isEqualTo(6);
+            assertThat(in.getPos()).isEqualTo(256);
             assertThat(in.read()).isEqualTo(-1);
+            assertThat(in.read(new byte[1], 0, 1)).isEqualTo(-1);
+        }
+    }
+
+    // --- buffer-efficiency: skip stays in local buffer vs. underlying stream 
---
+
+    @Test
+    void seekWithinBuffer_afterSmallRead_doesNotTouchUnderlyingStream() throws 
Exception {
+        int smallBuffer = 32;
+        StubS3Client client = new StubS3Client(DATA);
+        try (NativeS3InputStream in =
+                new NativeS3InputStream(client, BUCKET, KEY, DATA.length, 
smallBuffer)) {
+            // single-byte read fills the local buffer (up to smallBuffer 
bytes buffered)
+            in.read();
+            TrackingInputStream underlying = client.lastStream();
+
+            // forward seek of 9 bytes — fits in the local buffer, no 
underlying access needed
+            in.seek(10);
+            in.read();
+
+            assertThat(client.getObjectCalls()).isEqualTo(1);
+            assertThat(underlying.bytesSkippedFromUnderlying()).isEqualTo(0);

Review Comment:
   Here it's not checked that more data is not read.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to