readEnds.remove(i);
}
}
// add the current memorySegment and reset this writer
final MemorySegment current = getCurrentSegment();
current.putInt(0, getCurrentPositionInSegment());
fullBuffers.addLast(current);
// create the reader
final ReadEnd readEnd;
if (numBuffersSpilled == 0 && emptyBuffers.size() >= minBuffersForWriteEnd) {
// read completely from in-memory segments
readEnd = new ReadEnd(fullBuffers.removeFirst(), emptyBuffers, fullBuffers, null, null, segmentSize, 0);
} else {
int toSpill = Math.min(minBuffersForSpilledReadEnd + minBuffersForWriteEnd - emptyBuffers.size(),
fullBuffers.size());
// reader reads also segments on disk
// grab some empty buffers to re-read the first segment
if (toSpill > 0) {
// need to spill to make a buffers available
if (currentWriter == null) {
currentWriter = ioManager.createBlockChannelWriter(channelEnumerator.next(), emptyBuffers);
}
for (int i = 0; i < toSpill; i++) {
currentWriter.writeBlock(fullBuffers.removeFirst());
}
numBuffersSpilled += toSpill;
}
// now close the writer and create the reader
currentWriter.close();
final BlockChannelReader reader = ioManager.createBlockChannelReader(currentWriter.getChannelID());
// gather some memory segments to circulate while reading back the data
final ArrayList<MemorySegment> readSegments = new ArrayList<MemorySegment>();
try {
while (readSegments.size() < minBuffersForSpilledReadEnd) {
readSegments.add(emptyBuffers.take());
}
// read the first segment
MemorySegment firstSeg = readSegments.remove(readSegments.size() - 1);
reader.readBlock(firstSeg);
firstSeg = reader.getReturnQueue().take();
// create the read end reading one less buffer, because the first buffer is already read back
readEnd = new ReadEnd(firstSeg, emptyBuffers, fullBuffers, reader, readSegments, segmentSize,