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


##########
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:
   made changes PTAL



##########
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:
   added one more extra test to validate it. PTAL 



-- 
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