public int getNumberOfQueuedBuffers() { int totalBuffers = 0; for (ResultSubpartition subpartition : subpartitions) { totalBuffers += subpartition.getNumberOfQueuedBuffers(); } return totalBuffers; }
/** * Releases buffers held by this result partition. * * <p>This is a callback from the buffer pool, which is registered for result partitions, which * are back pressure-free. */ @Override public void releaseMemory(int toRelease) throws IOException { checkArgument(toRelease > 0); for (ResultSubpartition subpartition : subpartitions) { toRelease -= subpartition.releaseMemory(); // Only release as much memory as needed if (toRelease <= 0) { break; } } }
/** * Finishes the result partition. * * <p>After this operation, it is not possible to add further data to the result partition. * * <p>For BLOCKING results, this will trigger the deployment of consuming tasks. */ public void finish() throws IOException { boolean success = false; try { checkInProduceState(); for (ResultSubpartition subpartition : subpartitions) { synchronized (subpartition) { subpartition.finish(); } } success = true; } finally { if (success) { isFinished = true; notifyPipelinedConsumers(); } } }
/** Releases the result partition. */ public void release(Throwable cause) { if (isReleased.compareAndSet(false, true)) { LOG.debug("{}: Releasing {}.", owningTaskName, this); // Set the error cause if (cause != null) { this.cause = cause; } // Release all subpartitions for (ResultSubpartition subpartition : subpartitions) { try { synchronized (subpartition) { subpartition.release(); } } // Catch this in order to ensure that release is called on all subpartitions catch (Throwable t) { LOG.error("Error during release of result subpartition: " + t.getMessage(), t); } } } }
/** * Adds a buffer to the subpartition with the given index. * * <p>For PIPELINED results, this will trigger the deployment of consuming tasks after the first * buffer has been added. */ public void add(Buffer buffer, int subpartitionIndex) throws IOException { boolean success = false; try { checkInProduceState(); final ResultSubpartition subpartition = subpartitions[subpartitionIndex]; synchronized (subpartition) { success = subpartition.add(buffer); // Update statistics totalNumberOfBuffers++; totalNumberOfBytes += buffer.getSize(); } } finally { if (success) { notifyPipelinedConsumers(); } else { buffer.recycle(); } } }
@Override public Throwable getFailureCause() { return parent.getFailureCause(); }
@Override public boolean isReleased() { return parent.isReleased() || isReleased; }
@Override public void notifySubpartitionConsumed() throws IOException { parent.onConsumedSubpartition(); }