/** * Closes this OutputView, closing the underlying writer. And return number bytes in last * memory segment. */ @Override public int close() throws IOException { if (!writer.isClosed()) { int currentPositionInSegment = getCurrentPositionInSegment(); // write last segment writer.writeBlock(getCurrentSegment()); clear(); writer.getReturnQueue().clear(); this.writer.close(); return currentPositionInSegment; } return -1; }
@Override public void clearAllMemory(List<MemorySegment> target) { if (initialBuildSideChannel != null) { try { this.initialBuildSideWriter.closeAndDelete(); } catch (IOException ioex) { throw new RuntimeException("Error deleting the partition files. Some temporary files might not be removed."); } } super.clearAllMemory(target); }
@Override public MemorySegment nextSegment(MemorySegment current, int posInSegment) throws IOException { if (current != null) { writer.writeBlock(current); } final MemorySegment next = this.writer.getNextReturnedBlock(); this.blockCount++; return next; } }
/** * Spills this partition to disk. This method is invoked once after the initial open() method * * @return Number of memorySegments in the writeBehindBuffers! */ int spillInMemoryPartition(FileIOChannel.ID targetChannel, IOManager ioManager, LinkedBlockingQueue<MemorySegment> writeBehindBuffers) throws IOException { this.initialPartitionBuffersCount = partitionBuffers.length; // for ReOpenableHashMap this.initialBuildSideChannel = targetChannel; initialBuildSideWriter = ioManager.createBlockChannelWriter(targetChannel, writeBehindBuffers); final int numSegments = this.partitionBuffers.length; for (int i = 0; i < numSegments; i++) { initialBuildSideWriter.writeBlock(partitionBuffers[i]); } this.partitionBuffers = null; initialBuildSideWriter.close(); // num partitions are now in the writeBehindBuffers. We propagate this information back return numSegments; }
@Override protected MemorySegment nextSegment(MemorySegment current, int positionInCurrent) throws IOException { final MemorySegment next; if (this.writer == null) { // Must first add current segment: // This may happen when you need to spill: // A partition called nextSegment, can not get memory, need to spill, the result // give itself to the spill, Since it is switching currentSeg, it is necessary // to give the previous currSeg to spill. this.targetList.add(current); next = context.nextSegment(); } else { this.writer.writeBlock(current); try { next = this.writer.getReturnQueue().take(); } catch (InterruptedException iex) { throw new IOException("Hash Join Partition was interrupted while " + "grabbing a new write-behind buffer."); } } this.currentBlockNumber++; return next; }
} else { this.writer.writeBlock(this.getCurrentSegment()); this.numMemorySegmentsInWriter++; this.writer.close(); for (int i = this.numMemorySegmentsInWriter; i > 0; i--) { this.fullSegments.add(this.writer.getNextReturnedBlock()); final BlockChannelReader<MemorySegment> reader = this.ioManager.createBlockChannelReader(this.writer.getChannelID()); this.externalInView = new HeaderlessChannelReaderInputView(reader, this.fullSegments, this.blockCount, this.numBytesInLastSegment, false); return this.externalInView;
this.probeSideChannel.close(); this.buildSideChannel.deleteChannel(); this.probeSideChannel.deleteChannel(); return 0; this.probeSideChannel.close(); spilledPartitions.add(this); return 1;
this.writer.close(); for (int i = this.numMemorySegmentsInWriter; i > 0; i--) { segments.add(this.writer.getNextReturnedBlock()); this.writer.closeAndDelete(); this.writer = null;
currentWriter.writeBlock(fullBuffers.removeFirst()); currentWriter.close(); final BlockChannelReader<MemorySegment> reader = ioManager.createBlockChannelReader(currentWriter.getChannelID());
private void spill() throws IOException { FileIOChannel.ID channel = ioManager.createChannel(); final BlockChannelWriter<MemorySegment> writer = this.ioManager.createBlockChannelWriter(channel); int numRecordBuffers = inMemoryBuffer.getNumRecordBuffers(); ArrayList<MemorySegment> segments = inMemoryBuffer.getRecordBufferSegments(); try { // spill in memory buffer in zero-copy. for (int i = 0; i < numRecordBuffers; i++) { writer.writeBlock(segments.get(i)); } LOG.info("here spill the reset buffer data with {} bytes", writer.getSize()); writer.close(); } catch (IOException e) { writer.closeAndDelete(); throw e; } spillSize += numRecordBuffers * segmentSize; channelIDs.add(new ChannelWithMeta( channel, inMemoryBuffer.getNumRecordBuffers(), inMemoryBuffer.getNumBytesInLastBuffer())); this.numRowsUntilThisChannel.add(numRows); inMemoryBuffer.reset(); }
public FileChannelOutputView(BlockChannelWriter<MemorySegment> writer, MemoryManager memManager, List<MemorySegment> memory, int segmentSize) throws IOException { super(segmentSize, 0); checkNotNull(writer); checkNotNull(memManager); checkNotNull(memory); checkArgument(!writer.isClosed()); this.writer = writer; this.memManager = memManager; this.memory = memory; for (MemorySegment next : memory) { writer.getReturnQueue().add(next); } // move to the first page advance(); }
private void close(boolean delete) throws IOException { try { // send off set last segment, if we have not been closed before MemorySegment current = getCurrentSegment(); if (current != null) { writeSegment(current, getCurrentPositionInSegment()); } clear(); if (delete) { writer.closeAndDelete(); } else { writer.close(); } } finally { memManager.release(memory); } }
private void writeSegment(MemorySegment segment, int writePosition) throws IOException { writer.writeBlock(segment); numBlocksWritten++; bytesInLatestSegment = writePosition; } }
/** * Closes this OutputView, closing the underlying writer and returning all memory segments. * * @return A list containing all memory segments originally supplied to this view. * @throws IOException Thrown, if the underlying writer could not be properly closed. */ public List<MemorySegment> close() throws IOException { // send off set last segment writeSegment(getCurrentSegment(), getCurrentPositionInSegment(), true); clear(); // close the writer and gather all segments final LinkedBlockingQueue<MemorySegment> queue = this.writer.getReturnQueue(); this.writer.close(); // re-collect all memory segments ArrayList<MemorySegment> list = new ArrayList<MemorySegment>(this.numSegments); for (int i = 0; i < this.numSegments; i++) { final MemorySegment m = queue.poll(); if (m == null) { // we get null if the queue is empty. that should not be the case if the reader was properly closed. throw new RuntimeException("ChannelWriterOutputView: MemorySegments have been taken from return queue by different actor."); } list.add(m); } return list; }
protected final MemorySegment nextSegment(MemorySegment current, int posInSegment) throws IOException { if (current != null) { writeSegment(current, posInSegment, false); } final MemorySegment next = this.writer.getNextReturnedBlock(); this.blockCount++; return next; }
this.currentSpilledBuildSide = this.ioManager.createBlockChannelReader(p.getBuildSideChannel().getChannelID()); final ChannelReaderInputView inView = new HeaderlessChannelReaderInputView(currentSpilledBuildSide, memory, p.getBuildSideBlockCount(), p.getLastSegmentLimit(), false); this.currentSpilledProbeSide = this.ioManager.createBlockChannelReader(p.getProbeSideChannel().getChannelID(), returnQueue);
/** * After build phase. * @return build spill return buffer, if have spilled, it returns the current write buffer, * because it was used all the time in build phase, so it can only be returned at this time. */ int finalizeBuildPhase(IOManager ioAccess, FileIOChannel.Enumerator probeChannelEnumerator) throws IOException { this.finalBufferLimit = this.buildSideWriteBuffer.getCurrentPositionInSegment(); this.partitionBuffers = this.buildSideWriteBuffer.close(); if (!isInMemory()) { // close the channel. this.buildSideChannel.close(); this.probeSideBuffer = FileChannelUtil.createOutputView(ioAccess, probeChannelEnumerator.next(), compressionEnable, compressionCodecFactory, compressionBlockSize, memorySegmentSize); return 1; } else { return 0; } }
void finalizeProbePhase(List<LongHashPartition> spilledPartitions) throws IOException { if (isInMemory()) { releaseBuckets(); context.returnAll(partitionBuffers); this.partitionBuffers = null; } else { if (this.probeSideRecordCounter == 0) { // delete the spill files this.probeSideBuffer.close(); this.buildSideChannel.deleteChannel(); this.probeSideBuffer.deleteChannel(); } else { // flush the last probe side buffer and register this partition as pending probeNumBytesInLastSeg = this.probeSideBuffer.close(); spilledPartitions.add(this); } } }
public HeaderlessChannelWriterOutputView(BlockChannelWriter<MemorySegment> writer, List<MemorySegment> memory, int segmentSize) { super(writer, segmentSize, 0); if (writer == null) { throw new NullPointerException(); } this.writer = writer; Preconditions.checkNotNull(memory); // load the segments into the queue final LinkedBlockingQueue<MemorySegment> queue = writer.getReturnQueue(); for (int i = memory.size() - 1; i >= 0; --i) { final MemorySegment seg = memory.get(i); if (seg.size() != segmentSize) { throw new IllegalArgumentException("This segment are not of the specified size."); } queue.add(seg); } // get the first segment try { advance(); } catch (IOException ioex) { throw new RuntimeException(ioex); } }
} else { this.writer.writeBlock(this.getCurrentSegment()); this.numMemorySegmentsInWriter++; this.writer.close(); for (int i = this.numMemorySegmentsInWriter; i > 0; i--) { this.fullSegments.add(this.writer.getNextReturnedBlock()); final BlockChannelReader<MemorySegment> reader = this.ioManager.createBlockChannelReader(this.writer.getChannelID()); this.externalInView = new HeaderlessChannelReaderInputView(reader, this.fullSegments, this.blockCount, this.numBytesInLastSegment, false); return this.externalInView;