Skip to content

Commit b8082f6

Browse files
authored
Issue 4503: Added check into BufferedChannel's read to avoid endless loop (apache#4506)
* Added check into BufferedChannel's read to avoid endless loop if dest buffer's remaining capacity is not as much as length
1 parent f3f6956 commit b8082f6

File tree

2 files changed

+44
-1
lines changed

2 files changed

+44
-1
lines changed

bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BufferedChannel.java

Lines changed: 6 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -243,6 +243,11 @@ public long forceWrite(boolean forceMetadata) throws IOException {
243243

244244
@Override
245245
public synchronized int read(ByteBuf dest, long pos, int length) throws IOException {
246+
if (dest.writableBytes() < length) {
247+
throw new IllegalArgumentException("dest buffer remaining capacity is not enough"
248+
+ "(must be at least as \"length\"=" + length + ")");
249+
}
250+
246251
long prevPos = pos;
247252
while (length > 0) {
248253
// check if it is in the write buffer
@@ -295,4 +300,4 @@ public synchronized int getNumOfBytesInWriteBuffer() {
295300
long getUnpersistedBytes() {
296301
return unpersistedBytes.get();
297302
}
298-
}
303+
}

bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BufferedChannelTest.java

Lines changed: 38 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -21,10 +21,13 @@
2121

2222
package org.apache.bookkeeper.bookie;
2323

24+
import static org.junit.Assert.assertThrows;
25+
2426
import io.netty.buffer.ByteBuf;
2527
import io.netty.buffer.Unpooled;
2628
import io.netty.buffer.UnpooledByteBufAllocator;
2729
import java.io.File;
30+
import java.io.IOException;
2831
import java.io.RandomAccessFile;
2932
import java.nio.channels.FileChannel;
3033
import java.util.Random;
@@ -126,6 +129,41 @@ public void testBufferedChannel(int byteBufLength, int numOfWrites, int unpersis
126129
fileChannel.close();
127130
}
128131

132+
@Test
133+
public void testBufferedChannelReadWhenDestBufSizeExceedsReadLength() throws IOException {
134+
doTestBufferedChannelReadThrowing(100, 60);
135+
}
136+
137+
@Test
138+
public void testBufferedChannelReadWhenDestBufSizeDoesNotExceedReadLength() throws IOException {
139+
doTestBufferedChannelReadThrowing(100, 110);
140+
}
141+
142+
private void doTestBufferedChannelReadThrowing(int destBufSize, int readLength) throws IOException {
143+
File newLogFile = File.createTempFile("test", "log");
144+
newLogFile.deleteOnExit();
145+
146+
try (RandomAccessFile raf = new RandomAccessFile(newLogFile, "rw")) {
147+
FileChannel fileChannel = raf.getChannel();
148+
149+
try (BufferedChannel bufferedChannel = new BufferedChannel(
150+
UnpooledByteBufAllocator.DEFAULT, fileChannel,
151+
INTERNAL_BUFFER_WRITE_CAPACITY, INTERNAL_BUFFER_READ_CAPACITY, 0)) {
152+
153+
bufferedChannel.write(generateEntry(500));
154+
155+
ByteBuf destBuf = UnpooledByteBufAllocator.DEFAULT.buffer(destBufSize);
156+
157+
if (destBufSize < readLength) {
158+
assertThrows(IllegalArgumentException.class,
159+
() -> bufferedChannel.read(destBuf, 0, readLength));
160+
} else {
161+
bufferedChannel.read(destBuf, 0, readLength);
162+
}
163+
}
164+
}
165+
}
166+
129167
private static ByteBuf generateEntry(int length) {
130168
byte[] data = new byte[length];
131169
ByteBuf bb = Unpooled.buffer(length);

0 commit comments

Comments
 (0)