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

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


The following commit(s) were added to refs/heads/master by this push:
     new b8082f61c8 Issue 4503: Added check into BufferedChannel's read to 
avoid endless loop (#4506)
b8082f61c8 is described below

commit b8082f61c87e1b9b6bcfbd367aa3911eefa5ba5b
Author: ste <[email protected]>
AuthorDate: Mon Feb 17 03:16:31 2025 +0100

    Issue 4503: Added check into BufferedChannel's read to avoid endless loop 
(#4506)
    
    * Added check into BufferedChannel's read to avoid endless loop if dest 
buffer's remaining capacity is not as much as length
---
 .../apache/bookkeeper/bookie/BufferedChannel.java  |  7 +++-
 .../bookkeeper/bookie/BufferedChannelTest.java     | 38 ++++++++++++++++++++++
 2 files changed, 44 insertions(+), 1 deletion(-)

diff --git 
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BufferedChannel.java
 
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BufferedChannel.java
index 3197165827..dbba31083d 100644
--- 
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BufferedChannel.java
+++ 
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BufferedChannel.java
@@ -243,6 +243,11 @@ public class BufferedChannel extends BufferedReadChannel 
implements Closeable {
 
     @Override
     public synchronized int read(ByteBuf dest, long pos, int length) throws 
IOException {
+        if (dest.writableBytes() < length) {
+            throw new IllegalArgumentException("dest buffer remaining capacity 
is not enough"
+                    + "(must be at least as \"length\"=" + length + ")");
+        }
+
         long prevPos = pos;
         while (length > 0) {
             // check if it is in the write buffer
@@ -295,4 +300,4 @@ public class BufferedChannel extends BufferedReadChannel 
implements Closeable {
     long getUnpersistedBytes() {
         return unpersistedBytes.get();
     }
-}
\ No newline at end of file
+}
diff --git 
a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BufferedChannelTest.java
 
b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BufferedChannelTest.java
index cd3e34d35e..81e7c62af4 100644
--- 
a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BufferedChannelTest.java
+++ 
b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BufferedChannelTest.java
@@ -21,10 +21,13 @@
 
 package org.apache.bookkeeper.bookie;
 
+import static org.junit.Assert.assertThrows;
+
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.Unpooled;
 import io.netty.buffer.UnpooledByteBufAllocator;
 import java.io.File;
+import java.io.IOException;
 import java.io.RandomAccessFile;
 import java.nio.channels.FileChannel;
 import java.util.Random;
@@ -126,6 +129,41 @@ public class BufferedChannelTest {
         fileChannel.close();
     }
 
+    @Test
+    public void testBufferedChannelReadWhenDestBufSizeExceedsReadLength() 
throws IOException {
+        doTestBufferedChannelReadThrowing(100, 60);
+    }
+
+    @Test
+    public void 
testBufferedChannelReadWhenDestBufSizeDoesNotExceedReadLength() throws 
IOException {
+        doTestBufferedChannelReadThrowing(100, 110);
+    }
+
+    private void doTestBufferedChannelReadThrowing(int destBufSize, int 
readLength) throws IOException {
+        File newLogFile = File.createTempFile("test", "log");
+        newLogFile.deleteOnExit();
+
+        try (RandomAccessFile raf = new RandomAccessFile(newLogFile, "rw")) {
+            FileChannel fileChannel = raf.getChannel();
+
+            try (BufferedChannel bufferedChannel = new BufferedChannel(
+                UnpooledByteBufAllocator.DEFAULT, fileChannel,
+                INTERNAL_BUFFER_WRITE_CAPACITY, INTERNAL_BUFFER_READ_CAPACITY, 
0)) {
+
+                bufferedChannel.write(generateEntry(500));
+
+                ByteBuf destBuf = 
UnpooledByteBufAllocator.DEFAULT.buffer(destBufSize);
+
+                if (destBufSize < readLength) {
+                    assertThrows(IllegalArgumentException.class,
+                        () -> bufferedChannel.read(destBuf, 0, readLength));
+                } else {
+                    bufferedChannel.read(destBuf, 0, readLength);
+                }
+            }
+        }
+    }
+
     private static ByteBuf generateEntry(int length) {
         byte[] data = new byte[length];
         ByteBuf bb = Unpooled.buffer(length);

Reply via email to