mirror of https://github.com/apache/druid.git
Optimize FileWriteOutBytes to avoid high system cpu usage (#9722)
* optimize FileWriteOutBytes to avoid high sys cpu * optimize FileWriteOutBytes to avoid high sys cpu -- remove IOException * optimize FileWriteOutBytes to avoid high sys cpu -- remove IOException in writeOutBytes.size * Revert "optimize FileWriteOutBytes to avoid high sys cpu -- remove IOException in writeOutBytes.size" This reverts commit965f7421
* Revert "optimize FileWriteOutBytes to avoid high sys cpu -- remove IOException" This reverts commit149e08c0
* optimize FileWriteOutBytes to avoid high sys cpu -- avoid IOEception never thrown check * Fix size counting to handle IOE in FileWriteOutBytes + tests * remove unused throws IOException in WriteOutBytes.size() * Remove redundant throws IOExcpetion clauses * Parameterize IndexMergeBenchmark Co-authored-by: huanghui.bigrey <huanghui.bigrey@bytedance.com> Co-authored-by: Suneet Saldanha <suneet.saldanha@imply.io>
This commit is contained in:
parent
4087a015e8
commit
c5bfe36011
|
@ -35,11 +35,13 @@ import org.apache.druid.segment.IndexIO;
|
||||||
import org.apache.druid.segment.IndexMergerV9;
|
import org.apache.druid.segment.IndexMergerV9;
|
||||||
import org.apache.druid.segment.IndexSpec;
|
import org.apache.druid.segment.IndexSpec;
|
||||||
import org.apache.druid.segment.QueryableIndex;
|
import org.apache.druid.segment.QueryableIndex;
|
||||||
import org.apache.druid.segment.column.ColumnConfig;
|
|
||||||
import org.apache.druid.segment.incremental.IncrementalIndex;
|
import org.apache.druid.segment.incremental.IncrementalIndex;
|
||||||
import org.apache.druid.segment.incremental.IncrementalIndexSchema;
|
import org.apache.druid.segment.incremental.IncrementalIndexSchema;
|
||||||
import org.apache.druid.segment.serde.ComplexMetrics;
|
import org.apache.druid.segment.serde.ComplexMetrics;
|
||||||
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
|
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
|
||||||
|
import org.apache.druid.segment.writeout.OnHeapMemorySegmentWriteOutMediumFactory;
|
||||||
|
import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory;
|
||||||
|
import org.apache.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory;
|
||||||
import org.openjdk.jmh.annotations.Benchmark;
|
import org.openjdk.jmh.annotations.Benchmark;
|
||||||
import org.openjdk.jmh.annotations.BenchmarkMode;
|
import org.openjdk.jmh.annotations.BenchmarkMode;
|
||||||
import org.openjdk.jmh.annotations.Fork;
|
import org.openjdk.jmh.annotations.Fork;
|
||||||
|
@ -66,6 +68,7 @@ import java.util.concurrent.TimeUnit;
|
||||||
@Measurement(iterations = 25)
|
@Measurement(iterations = 25)
|
||||||
public class IndexMergeBenchmark
|
public class IndexMergeBenchmark
|
||||||
{
|
{
|
||||||
|
|
||||||
@Param({"5"})
|
@Param({"5"})
|
||||||
private int numSegments;
|
private int numSegments;
|
||||||
|
|
||||||
|
@ -78,9 +81,13 @@ public class IndexMergeBenchmark
|
||||||
@Param({"true", "false"})
|
@Param({"true", "false"})
|
||||||
private boolean rollup;
|
private boolean rollup;
|
||||||
|
|
||||||
|
@Param({"OFF_HEAP", "TMP_FILE", "ON_HEAP"})
|
||||||
|
private SegmentWriteOutType factoryType;
|
||||||
|
|
||||||
|
|
||||||
private static final Logger log = new Logger(IndexMergeBenchmark.class);
|
private static final Logger log = new Logger(IndexMergeBenchmark.class);
|
||||||
private static final int RNG_SEED = 9999;
|
private static final int RNG_SEED = 9999;
|
||||||
private static final IndexMergerV9 INDEX_MERGER_V9;
|
|
||||||
private static final IndexIO INDEX_IO;
|
private static final IndexIO INDEX_IO;
|
||||||
public static final ObjectMapper JSON_MAPPER;
|
public static final ObjectMapper JSON_MAPPER;
|
||||||
|
|
||||||
|
@ -91,6 +98,7 @@ public class IndexMergeBenchmark
|
||||||
private List<QueryableIndex> indexesToMerge;
|
private List<QueryableIndex> indexesToMerge;
|
||||||
private BenchmarkSchemaInfo schemaInfo;
|
private BenchmarkSchemaInfo schemaInfo;
|
||||||
private File tmpDir;
|
private File tmpDir;
|
||||||
|
private IndexMergerV9 indexMergerV9;
|
||||||
|
|
||||||
static {
|
static {
|
||||||
JSON_MAPPER = new DefaultObjectMapper();
|
JSON_MAPPER = new DefaultObjectMapper();
|
||||||
|
@ -99,23 +107,16 @@ public class IndexMergeBenchmark
|
||||||
JSON_MAPPER.setInjectableValues(injectableValues);
|
JSON_MAPPER.setInjectableValues(injectableValues);
|
||||||
INDEX_IO = new IndexIO(
|
INDEX_IO = new IndexIO(
|
||||||
JSON_MAPPER,
|
JSON_MAPPER,
|
||||||
new ColumnConfig()
|
() -> 0
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public int columnCacheSizeBytes()
|
|
||||||
{
|
|
||||||
return 0;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
);
|
);
|
||||||
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Setup
|
@Setup
|
||||||
public void setup() throws IOException
|
public void setup() throws IOException
|
||||||
{
|
{
|
||||||
log.info("SETUP CALLED AT " + System.currentTimeMillis());
|
|
||||||
|
|
||||||
|
log.info("SETUP CALLED AT " + System.currentTimeMillis());
|
||||||
|
indexMergerV9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, getSegmentWriteOutMediumFactory(factoryType));
|
||||||
ComplexMetrics.registerSerde("hyperUnique", new HyperUniquesSerde());
|
ComplexMetrics.registerSerde("hyperUnique", new HyperUniquesSerde());
|
||||||
|
|
||||||
indexesToMerge = new ArrayList<>();
|
indexesToMerge = new ArrayList<>();
|
||||||
|
@ -143,7 +144,7 @@ public class IndexMergeBenchmark
|
||||||
tmpDir = FileUtils.createTempDir();
|
tmpDir = FileUtils.createTempDir();
|
||||||
log.info("Using temp dir: " + tmpDir.getAbsolutePath());
|
log.info("Using temp dir: " + tmpDir.getAbsolutePath());
|
||||||
|
|
||||||
File indexFile = INDEX_MERGER_V9.persist(
|
File indexFile = indexMergerV9.persist(
|
||||||
incIndex,
|
incIndex,
|
||||||
tmpDir,
|
tmpDir,
|
||||||
new IndexSpec(),
|
new IndexSpec(),
|
||||||
|
@ -155,26 +156,6 @@ public class IndexMergeBenchmark
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@TearDown
|
|
||||||
public void tearDown() throws IOException
|
|
||||||
{
|
|
||||||
FileUtils.deleteDirectory(tmpDir);
|
|
||||||
}
|
|
||||||
|
|
||||||
private IncrementalIndex makeIncIndex()
|
|
||||||
{
|
|
||||||
return new IncrementalIndex.Builder()
|
|
||||||
.setIndexSchema(
|
|
||||||
new IncrementalIndexSchema.Builder()
|
|
||||||
.withMetrics(schemaInfo.getAggsArray())
|
|
||||||
.withRollup(rollup)
|
|
||||||
.build()
|
|
||||||
)
|
|
||||||
.setReportParseExceptions(false)
|
|
||||||
.setMaxRowCount(rowsPerSegment)
|
|
||||||
.buildOnheap();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Benchmark
|
@Benchmark
|
||||||
@BenchmarkMode(Mode.AverageTime)
|
@BenchmarkMode(Mode.AverageTime)
|
||||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||||
|
@ -186,7 +167,7 @@ public class IndexMergeBenchmark
|
||||||
try {
|
try {
|
||||||
log.info(tmpFile.getAbsolutePath() + " isFile: " + tmpFile.isFile() + " isDir:" + tmpFile.isDirectory());
|
log.info(tmpFile.getAbsolutePath() + " isFile: " + tmpFile.isFile() + " isDir:" + tmpFile.isDirectory());
|
||||||
|
|
||||||
File mergedFile = INDEX_MERGER_V9.mergeQueryableIndex(
|
File mergedFile = indexMergerV9.mergeQueryableIndex(
|
||||||
indexesToMerge,
|
indexesToMerge,
|
||||||
rollup,
|
rollup,
|
||||||
schemaInfo.getAggsArray(),
|
schemaInfo.getAggsArray(),
|
||||||
|
@ -199,8 +180,46 @@ public class IndexMergeBenchmark
|
||||||
}
|
}
|
||||||
finally {
|
finally {
|
||||||
tmpFile.delete();
|
tmpFile.delete();
|
||||||
|
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@TearDown
|
||||||
|
public void tearDown() throws IOException
|
||||||
|
{
|
||||||
|
FileUtils.deleteDirectory(tmpDir);
|
||||||
|
}
|
||||||
|
|
||||||
|
public enum SegmentWriteOutType
|
||||||
|
{
|
||||||
|
TMP_FILE,
|
||||||
|
OFF_HEAP,
|
||||||
|
ON_HEAP
|
||||||
|
}
|
||||||
|
|
||||||
|
private SegmentWriteOutMediumFactory getSegmentWriteOutMediumFactory(SegmentWriteOutType type)
|
||||||
|
{
|
||||||
|
switch (type) {
|
||||||
|
case TMP_FILE:
|
||||||
|
return TmpFileSegmentWriteOutMediumFactory.instance();
|
||||||
|
case OFF_HEAP:
|
||||||
|
return OffHeapMemorySegmentWriteOutMediumFactory.instance();
|
||||||
|
case ON_HEAP:
|
||||||
|
return OnHeapMemorySegmentWriteOutMediumFactory.instance();
|
||||||
|
}
|
||||||
|
throw new RuntimeException("Could not create SegmentWriteOutMediumFactory of type: " + type);
|
||||||
|
}
|
||||||
|
|
||||||
|
private IncrementalIndex makeIncIndex()
|
||||||
|
{
|
||||||
|
return new IncrementalIndex.Builder()
|
||||||
|
.setIndexSchema(
|
||||||
|
new IncrementalIndexSchema.Builder()
|
||||||
|
.withMetrics(schemaInfo.getAggsArray())
|
||||||
|
.withRollup(rollup)
|
||||||
|
.build()
|
||||||
|
)
|
||||||
|
.setReportParseExceptions(false)
|
||||||
|
.setMaxRowCount(rowsPerSegment)
|
||||||
|
.buildOnheap();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -63,7 +63,7 @@ public class ByteBufferWriter<T> implements Serializer
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public long getSerializedSize() throws IOException
|
public long getSerializedSize()
|
||||||
{
|
{
|
||||||
return headerOut.size() + valueOut.size();
|
return headerOut.size() + valueOut.size();
|
||||||
}
|
}
|
||||||
|
|
|
@ -80,7 +80,7 @@ public class EntireLayoutColumnarDoublesSerializer implements ColumnarDoublesSer
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public long getSerializedSize() throws IOException
|
public long getSerializedSize()
|
||||||
{
|
{
|
||||||
return META_SERDE_HELPER.size(this) + valuesOut.size();
|
return META_SERDE_HELPER.size(this) + valuesOut.size();
|
||||||
}
|
}
|
||||||
|
|
|
@ -81,7 +81,7 @@ public class EntireLayoutColumnarFloatsSerializer implements ColumnarFloatsSeria
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public long getSerializedSize() throws IOException
|
public long getSerializedSize()
|
||||||
{
|
{
|
||||||
return META_SERDE_HELPER.size(this) + valuesOut.size();
|
return META_SERDE_HELPER.size(this) + valuesOut.size();
|
||||||
}
|
}
|
||||||
|
|
|
@ -296,7 +296,7 @@ public class GenericIndexedWriter<T> implements Serializer
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public long getSerializedSize() throws IOException
|
public long getSerializedSize()
|
||||||
{
|
{
|
||||||
if (requireMultipleFiles) {
|
if (requireMultipleFiles) {
|
||||||
// for multi-file version (version 2), getSerializedSize() returns number of bytes in meta file.
|
// for multi-file version (version 2), getSerializedSize() returns number of bytes in meta file.
|
||||||
|
@ -394,7 +394,7 @@ public class GenericIndexedWriter<T> implements Serializer
|
||||||
*
|
*
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
private int bagSizePower() throws IOException
|
private int bagSizePower()
|
||||||
{
|
{
|
||||||
long avgObjectSize = (valuesOut.size() + numWritten - 1) / numWritten;
|
long avgObjectSize = (valuesOut.size() + numWritten - 1) / numWritten;
|
||||||
|
|
||||||
|
@ -421,7 +421,7 @@ public class GenericIndexedWriter<T> implements Serializer
|
||||||
*
|
*
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
private boolean actuallyFits(int powerTwo) throws IOException
|
private boolean actuallyFits(int powerTwo)
|
||||||
{
|
{
|
||||||
long lastValueOffset = 0;
|
long lastValueOffset = 0;
|
||||||
long currentValueOffset = 0;
|
long currentValueOffset = 0;
|
||||||
|
|
|
@ -66,7 +66,7 @@ public class ComplexColumnSerializer<T> implements GenericColumnSerializer<T>
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public long getSerializedSize() throws IOException
|
public long getSerializedSize()
|
||||||
{
|
{
|
||||||
return writer.getSerializedSize();
|
return writer.getSerializedSize();
|
||||||
}
|
}
|
||||||
|
|
|
@ -106,7 +106,7 @@ public class LargeColumnSupportedComplexColumnSerializer<T> implements GenericCo
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public long getSerializedSize() throws IOException
|
public long getSerializedSize()
|
||||||
{
|
{
|
||||||
return writer.getSerializedSize();
|
return writer.getSerializedSize();
|
||||||
}
|
}
|
||||||
|
|
|
@ -117,7 +117,7 @@ public final class MetaSerdeHelper<T>
|
||||||
|
|
||||||
public interface FieldWriter<T>
|
public interface FieldWriter<T>
|
||||||
{
|
{
|
||||||
void writeTo(ByteBuffer buffer, T x) throws IOException;
|
void writeTo(ByteBuffer buffer, T x);
|
||||||
|
|
||||||
int size(T x);
|
int size(T x);
|
||||||
}
|
}
|
||||||
|
@ -125,10 +125,10 @@ public final class MetaSerdeHelper<T>
|
||||||
@FunctionalInterface
|
@FunctionalInterface
|
||||||
public interface IntFieldWriter<T> extends FieldWriter<T>
|
public interface IntFieldWriter<T> extends FieldWriter<T>
|
||||||
{
|
{
|
||||||
int getField(T x) throws IOException;
|
int getField(T x);
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
default void writeTo(ByteBuffer buffer, T x) throws IOException
|
default void writeTo(ByteBuffer buffer, T x)
|
||||||
{
|
{
|
||||||
buffer.putInt(getField(x));
|
buffer.putInt(getField(x));
|
||||||
}
|
}
|
||||||
|
|
|
@ -36,6 +36,7 @@ final class FileWriteOutBytes extends WriteOutBytes
|
||||||
{
|
{
|
||||||
private final File file;
|
private final File file;
|
||||||
private final FileChannel ch;
|
private final FileChannel ch;
|
||||||
|
private long writeOutBytes;
|
||||||
|
|
||||||
/** Purposely big-endian, for {@link #writeInt(int)} implementation */
|
/** Purposely big-endian, for {@link #writeInt(int)} implementation */
|
||||||
private final ByteBuffer buffer = ByteBuffer.allocate(4096); // 4K page sized buffer
|
private final ByteBuffer buffer = ByteBuffer.allocate(4096); // 4K page sized buffer
|
||||||
|
@ -44,6 +45,7 @@ final class FileWriteOutBytes extends WriteOutBytes
|
||||||
{
|
{
|
||||||
this.file = file;
|
this.file = file;
|
||||||
this.ch = ch;
|
this.ch = ch;
|
||||||
|
this.writeOutBytes = 0L;
|
||||||
}
|
}
|
||||||
|
|
||||||
private void flushIfNeeded(int bytesNeeded) throws IOException
|
private void flushIfNeeded(int bytesNeeded) throws IOException
|
||||||
|
@ -66,6 +68,7 @@ final class FileWriteOutBytes extends WriteOutBytes
|
||||||
{
|
{
|
||||||
flushIfNeeded(1);
|
flushIfNeeded(1);
|
||||||
buffer.put((byte) b);
|
buffer.put((byte) b);
|
||||||
|
writeOutBytes++;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -73,6 +76,7 @@ final class FileWriteOutBytes extends WriteOutBytes
|
||||||
{
|
{
|
||||||
flushIfNeeded(Integer.BYTES);
|
flushIfNeeded(Integer.BYTES);
|
||||||
buffer.putInt(v);
|
buffer.putInt(v);
|
||||||
|
writeOutBytes += Integer.BYTES;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -85,6 +89,7 @@ final class FileWriteOutBytes extends WriteOutBytes
|
||||||
try {
|
try {
|
||||||
src.limit(src.position() + buffer.capacity());
|
src.limit(src.position() + buffer.capacity());
|
||||||
buffer.put(src);
|
buffer.put(src);
|
||||||
|
writeOutBytes += buffer.capacity();
|
||||||
flush();
|
flush();
|
||||||
}
|
}
|
||||||
finally {
|
finally {
|
||||||
|
@ -92,7 +97,9 @@ final class FileWriteOutBytes extends WriteOutBytes
|
||||||
src.limit(srcLimit);
|
src.limit(srcLimit);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
int remaining = src.remaining();
|
||||||
buffer.put(src);
|
buffer.put(src);
|
||||||
|
writeOutBytes += remaining;
|
||||||
return len;
|
return len;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -103,10 +110,9 @@ final class FileWriteOutBytes extends WriteOutBytes
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public long size() throws IOException
|
public long size()
|
||||||
{
|
{
|
||||||
flush();
|
return writeOutBytes;
|
||||||
return ch.size();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -45,7 +45,7 @@ public abstract class WriteOutBytes extends OutputStream implements WritableByte
|
||||||
/**
|
/**
|
||||||
* Returns the number of bytes written to this WriteOutBytes so far.
|
* Returns the number of bytes written to this WriteOutBytes so far.
|
||||||
*/
|
*/
|
||||||
public abstract long size() throws IOException;
|
public abstract long size();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Takes all bytes that are written to this WriteOutBytes so far and writes them into the given channel.
|
* Takes all bytes that are written to this WriteOutBytes so far and writes them into the given channel.
|
||||||
|
|
|
@ -20,6 +20,7 @@
|
||||||
package org.apache.druid.segment.writeout;
|
package org.apache.druid.segment.writeout;
|
||||||
|
|
||||||
import org.easymock.EasyMock;
|
import org.easymock.EasyMock;
|
||||||
|
import org.junit.Assert;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
|
@ -36,28 +37,106 @@ public class FileWriteOutBytesTest
|
||||||
@Before
|
@Before
|
||||||
public void setUp()
|
public void setUp()
|
||||||
{
|
{
|
||||||
this.mockFileChannel = EasyMock.mock(FileChannel.class);
|
mockFileChannel = EasyMock.mock(FileChannel.class);
|
||||||
this.fileWriteOutBytes = new FileWriteOutBytes(EasyMock.mock(File.class), mockFileChannel);
|
fileWriteOutBytes = new FileWriteOutBytes(EasyMock.mock(File.class), mockFileChannel);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testWrite4KBInts() throws IOException
|
public void write4KBIntsShouldNotFlush() throws IOException
|
||||||
{
|
{
|
||||||
// Write 4KB of ints and expect the write operation of the file channel will be triggered only once.
|
// Write 4KB of ints and expect the write operation of the file channel will be triggered only once.
|
||||||
EasyMock.expect(this.mockFileChannel.write(EasyMock.anyObject(ByteBuffer.class)))
|
EasyMock.expect(mockFileChannel.write(EasyMock.anyObject(ByteBuffer.class)))
|
||||||
.andAnswer(() -> {
|
.andAnswer(() -> {
|
||||||
ByteBuffer buffer = (ByteBuffer) EasyMock.getCurrentArguments()[0];
|
ByteBuffer buffer = (ByteBuffer) EasyMock.getCurrentArguments()[0];
|
||||||
int remaining = buffer.remaining();
|
int remaining = buffer.remaining();
|
||||||
buffer.position(remaining);
|
buffer.position(remaining);
|
||||||
return remaining;
|
return remaining;
|
||||||
}).times(1);
|
}).times(1);
|
||||||
EasyMock.replay(this.mockFileChannel);
|
EasyMock.replay(mockFileChannel);
|
||||||
final int writeBytes = 4096;
|
final int writeBytes = 4096;
|
||||||
final int numOfInt = writeBytes / Integer.BYTES;
|
final int numOfInt = writeBytes / Integer.BYTES;
|
||||||
for (int i = 0; i < numOfInt; i++) {
|
for (int i = 0; i < numOfInt; i++) {
|
||||||
this.fileWriteOutBytes.writeInt(i);
|
fileWriteOutBytes.writeInt(i);
|
||||||
}
|
}
|
||||||
this.fileWriteOutBytes.flush();
|
// no need to flush up to 4KB
|
||||||
EasyMock.verify(this.mockFileChannel);
|
// the first byte after 4KB will cause a flush
|
||||||
|
fileWriteOutBytes.write(1);
|
||||||
|
EasyMock.verify(mockFileChannel);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void writeShouldIncrementSize() throws IOException
|
||||||
|
{
|
||||||
|
fileWriteOutBytes.write(1);
|
||||||
|
Assert.assertEquals(1, fileWriteOutBytes.size());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void writeIntShouldIncrementSize() throws IOException
|
||||||
|
{
|
||||||
|
fileWriteOutBytes.writeInt(1);
|
||||||
|
Assert.assertEquals(4, fileWriteOutBytes.size());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void writeBufferLargerThanCapacityShouldIncrementSizeCorrectly() throws IOException
|
||||||
|
{
|
||||||
|
EasyMock.expect(mockFileChannel.write(EasyMock.anyObject(ByteBuffer.class)))
|
||||||
|
.andAnswer(() -> {
|
||||||
|
ByteBuffer buffer = (ByteBuffer) EasyMock.getCurrentArguments()[0];
|
||||||
|
int remaining = buffer.remaining();
|
||||||
|
buffer.position(remaining);
|
||||||
|
return remaining;
|
||||||
|
}).times(1);
|
||||||
|
EasyMock.replay(mockFileChannel);
|
||||||
|
ByteBuffer src = ByteBuffer.allocate(4096 + 1);
|
||||||
|
fileWriteOutBytes.write(src);
|
||||||
|
Assert.assertEquals(src.capacity(), fileWriteOutBytes.size());
|
||||||
|
EasyMock.verify(mockFileChannel);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void writeBufferLargerThanCapacityThrowsIOEInTheMiddleShouldIncrementSizeCorrectly() throws IOException
|
||||||
|
{
|
||||||
|
EasyMock.expect(mockFileChannel.write(EasyMock.anyObject(ByteBuffer.class)))
|
||||||
|
.andAnswer(() -> {
|
||||||
|
ByteBuffer buffer = (ByteBuffer) EasyMock.getCurrentArguments()[0];
|
||||||
|
int remaining = buffer.remaining();
|
||||||
|
buffer.position(remaining);
|
||||||
|
return remaining;
|
||||||
|
}).once();
|
||||||
|
EasyMock.expect(mockFileChannel.write(EasyMock.anyObject(ByteBuffer.class)))
|
||||||
|
.andThrow(new IOException())
|
||||||
|
.once();
|
||||||
|
EasyMock.replay(mockFileChannel);
|
||||||
|
ByteBuffer src = ByteBuffer.allocate(4096 * 2 + 1);
|
||||||
|
try {
|
||||||
|
fileWriteOutBytes.write(src);
|
||||||
|
Assert.fail("IOException should have been thrown.");
|
||||||
|
}
|
||||||
|
catch (IOException e) {
|
||||||
|
// The second invocation to flush bytes fails. So the size should count what has already been put successfully
|
||||||
|
Assert.assertEquals(4096 * 2, fileWriteOutBytes.size());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void writeBufferSmallerThanCapacityShouldIncrementSizeCorrectly() throws IOException
|
||||||
|
{
|
||||||
|
ByteBuffer src = ByteBuffer.allocate(4096);
|
||||||
|
fileWriteOutBytes.write(src);
|
||||||
|
Assert.assertEquals(src.capacity(), fileWriteOutBytes.size());
|
||||||
|
}
|
||||||
|
@Test
|
||||||
|
public void sizeDoesNotFlush() throws IOException
|
||||||
|
{
|
||||||
|
EasyMock.expect(mockFileChannel.write(EasyMock.anyObject(ByteBuffer.class)))
|
||||||
|
.andThrow(new AssertionError("file channel should not have been written to."));
|
||||||
|
EasyMock.replay(mockFileChannel);
|
||||||
|
long size = fileWriteOutBytes.size();
|
||||||
|
Assert.assertEquals(0, size);
|
||||||
|
fileWriteOutBytes.writeInt(10);
|
||||||
|
size = fileWriteOutBytes.size();
|
||||||
|
Assert.assertEquals(4, size);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue