mirror of
https://github.com/apache/druid.git
synced 2025-03-06 17:29:45 +00:00
improve FileWriteOutBytes.readFully (#12323)
* improve FileWriteOutBytes.readFully * no need to flush if out of bounds
This commit is contained in:
parent
7bf1d8c5c0
commit
dc0372a28e
@ -131,10 +131,10 @@ final class FileWriteOutBytes extends WriteOutBytes
|
||||
@Override
|
||||
public void readFully(long pos, ByteBuffer buffer) throws IOException
|
||||
{
|
||||
flush();
|
||||
if (pos < 0 || pos > ch.size()) {
|
||||
throw new IAE("pos %d out of range [%d, %d]", pos, 0, ch.size());
|
||||
if (pos < 0 || pos > writeOutBytes) {
|
||||
throw new IAE("pos %d out of range [%d, %d]", pos, 0, writeOutBytes);
|
||||
}
|
||||
flush();
|
||||
ch.read(buffer, pos);
|
||||
if (buffer.remaining() > 0) {
|
||||
throw new BufferUnderflowException();
|
||||
|
@ -19,6 +19,7 @@
|
||||
|
||||
package org.apache.druid.segment.writeout;
|
||||
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.easymock.EasyMock;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
@ -139,4 +140,56 @@ public class FileWriteOutBytesTest
|
||||
size = fileWriteOutBytes.size();
|
||||
Assert.assertEquals(4, size);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReadFullyWorks() throws IOException
|
||||
{
|
||||
int fileSize = 4096;
|
||||
int numOfInt = fileSize / Integer.BYTES;
|
||||
ByteBuffer destination = ByteBuffer.allocate(Integer.BYTES);
|
||||
ByteBuffer underlying = ByteBuffer.allocate(fileSize);
|
||||
// Write 4KiB of ints and expect the write operation of the file channel will be triggered only once.
|
||||
EasyMock.expect(mockFileChannel.write(EasyMock.anyObject(ByteBuffer.class)))
|
||||
.andAnswer(() -> {
|
||||
ByteBuffer buffer = (ByteBuffer) EasyMock.getCurrentArguments()[0];
|
||||
underlying.position(0);
|
||||
underlying.put(buffer);
|
||||
return 0;
|
||||
}).times(1);
|
||||
EasyMock.expect(mockFileChannel.read(EasyMock.eq(destination), EasyMock.eq(100L * Integer.BYTES)))
|
||||
.andAnswer(() -> {
|
||||
ByteBuffer buffer = (ByteBuffer) EasyMock.getCurrentArguments()[0];
|
||||
long pos = (long) EasyMock.getCurrentArguments()[1];
|
||||
buffer.putInt(underlying.getInt((int) pos));
|
||||
return Integer.BYTES;
|
||||
}).times(1);
|
||||
EasyMock.replay(mockFileChannel);
|
||||
for (int i = 0; i < numOfInt; i++) {
|
||||
fileWriteOutBytes.writeInt(i);
|
||||
}
|
||||
Assert.assertEquals(underlying.capacity(), fileWriteOutBytes.size());
|
||||
|
||||
destination.position(0);
|
||||
fileWriteOutBytes.readFully(100L * Integer.BYTES, destination);
|
||||
destination.position(0);
|
||||
Assert.assertEquals(100, destination.getInt());
|
||||
EasyMock.verify(mockFileChannel);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReadFullyOutOfBoundsDoesnt() throws IOException
|
||||
{
|
||||
int fileSize = 4096;
|
||||
int numOfInt = fileSize / Integer.BYTES;
|
||||
ByteBuffer destination = ByteBuffer.allocate(Integer.BYTES);
|
||||
EasyMock.replay(mockFileChannel);
|
||||
for (int i = 0; i < numOfInt; i++) {
|
||||
fileWriteOutBytes.writeInt(i);
|
||||
}
|
||||
Assert.assertEquals(fileSize, fileWriteOutBytes.size());
|
||||
|
||||
destination.position(0);
|
||||
Assert.assertThrows(IAE.class, () -> fileWriteOutBytes.readFully(5000, destination));
|
||||
EasyMock.verify(mockFileChannel);
|
||||
}
|
||||
}
|
||||
|
Loading…
x
Reference in New Issue
Block a user