From dc0372a28e3b22d7e35047ab60925fa1734d7d16 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Wed, 9 Mar 2022 11:45:45 -0800 Subject: [PATCH] improve FileWriteOutBytes.readFully (#12323) * improve FileWriteOutBytes.readFully * no need to flush if out of bounds --- .../segment/writeout/FileWriteOutBytes.java | 6 +-- .../writeout/FileWriteOutBytesTest.java | 53 +++++++++++++++++++ 2 files changed, 56 insertions(+), 3 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/segment/writeout/FileWriteOutBytes.java b/processing/src/main/java/org/apache/druid/segment/writeout/FileWriteOutBytes.java index b12b15e518b..696e7954fde 100644 --- a/processing/src/main/java/org/apache/druid/segment/writeout/FileWriteOutBytes.java +++ b/processing/src/main/java/org/apache/druid/segment/writeout/FileWriteOutBytes.java @@ -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(); diff --git a/processing/src/test/java/org/apache/druid/segment/writeout/FileWriteOutBytesTest.java b/processing/src/test/java/org/apache/druid/segment/writeout/FileWriteOutBytesTest.java index e19faa1ee5a..8be3a4b21c4 100644 --- a/processing/src/test/java/org/apache/druid/segment/writeout/FileWriteOutBytesTest.java +++ b/processing/src/test/java/org/apache/druid/segment/writeout/FileWriteOutBytesTest.java @@ -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); + } }