public class HsResultPartition extends ResultPartition
HsResultPartition
appends records and events to HsMemoryDataManager
, the shuffle
data maybe spilled to disk according to the HsSpillingStrategy
, and the downstream can
consume data from memory or disk.AvailabilityProvider.AvailabilityHelper
Modifier and Type | Field and Description |
---|---|
static int |
BROADCAST_CHANNEL |
static String |
DATA_FILE_SUFFIX |
static String |
INDEX_FILE_SUFFIX |
bufferCompressor, bufferPool, LOG, numBuffersOut, numBytesOut, numSubpartitions, partitionId, partitionManager, partitionType, resultPartitionBytes
AVAILABLE
Constructor and Description |
---|
HsResultPartition(String owningTaskName,
int partitionIndex,
ResultPartitionID partitionId,
ResultPartitionType partitionType,
int numSubpartitions,
int numTargetKeyGroups,
BatchShuffleReadBufferPool readBufferPool,
ScheduledExecutorService readIOExecutor,
ResultPartitionManager partitionManager,
String dataFileBashPath,
int networkBufferSize,
HybridShuffleConfiguration hybridShuffleConfiguration,
BufferCompressor bufferCompressor,
boolean isBroadcastOnly,
SupplierWithException<BufferPool,IOException> bufferPoolFactory) |
Modifier and Type | Method and Description |
---|---|
void |
abortCheckpoint(long checkpointId,
CheckpointException cause)
Abort the checkpoint.
|
void |
alignedBarrierTimeout(long checkpointId)
Timeout the aligned barrier to unaligned barrier.
|
void |
broadcastEvent(AbstractEvent event,
boolean isPriorityEvent)
Writes the given
AbstractEvent to all channels. |
void |
broadcastRecord(ByteBuffer record)
Writes the given serialized record to all subpartitions.
|
void |
close()
Closes the partition writer which releases the allocated resource, for example the buffer
pool.
|
ResultSubpartitionView |
createSubpartitionView(int subpartitionId,
BufferAvailabilityListener availabilityListener)
Returns a reader for the subpartition with the given index.
|
void |
emitRecord(ByteBuffer record,
int targetSubpartition)
Writes the given serialized record to the target subpartition.
|
void |
finish()
Finishes the result partition.
|
void |
flush(int subpartitionIndex)
Manually trigger the consumption of data from the given subpartitions.
|
void |
flushAll()
Manually trigger the consumption of data from all subpartitions.
|
int |
getNumberOfQueuedBuffers()
Returns the total number of queued buffers of all subpartitions.
|
int |
getNumberOfQueuedBuffers(int targetSubpartition)
Returns the number of queued buffers of the given target subpartition.
|
long |
getSizeOfQueuedBuffersUnsafe()
Returns the total size in bytes of queued buffers of all subpartitions.
|
void |
notifyEndOfData(StopMode mode)
Notifies the downstream tasks that this
ResultPartitionWriter have emitted all the
user records. |
protected void |
releaseInternal()
Releases all produced data including both those stored in memory and persisted on disk.
|
void |
setMetricGroup(TaskIOMetricGroup metrics)
Sets the metric group for the
ResultPartitionWriter . |
protected void |
setupInternal()
Do the subclass's own setup operation.
|
canBeCompressed, checkInProduceState, fail, getAllDataProcessedFuture, getAvailableFuture, getBufferPool, getFailureCause, getNumberOfSubpartitions, getNumTargetKeyGroups, getOwningTaskName, getPartitionId, getPartitionIndex, getPartitionManager, getPartitionType, isFinished, isReleased, onSubpartitionAllDataProcessed, release, release, setMaxOverdraftBuffersPerGate, setup, toString
clone, equals, finalize, getClass, hashCode, notify, notifyAll, wait, wait, wait
and, isApproximatelyAvailable, isAvailable, or
public static final String DATA_FILE_SUFFIX
public static final String INDEX_FILE_SUFFIX
public static final int BROADCAST_CHANNEL
public HsResultPartition(String owningTaskName, int partitionIndex, ResultPartitionID partitionId, ResultPartitionType partitionType, int numSubpartitions, int numTargetKeyGroups, BatchShuffleReadBufferPool readBufferPool, ScheduledExecutorService readIOExecutor, ResultPartitionManager partitionManager, String dataFileBashPath, int networkBufferSize, HybridShuffleConfiguration hybridShuffleConfiguration, @Nullable BufferCompressor bufferCompressor, boolean isBroadcastOnly, SupplierWithException<BufferPool,IOException> bufferPoolFactory)
protected void setupInternal() throws IOException
ResultPartition
setupInternal
in class ResultPartition
IOException
public void setMetricGroup(TaskIOMetricGroup metrics)
ResultPartitionWriter
ResultPartitionWriter
.setMetricGroup
in interface ResultPartitionWriter
setMetricGroup
in class ResultPartition
public void emitRecord(ByteBuffer record, int targetSubpartition) throws IOException
ResultPartitionWriter
IOException
public void broadcastRecord(ByteBuffer record) throws IOException
ResultPartitionWriter
IOException
public void broadcastEvent(AbstractEvent event, boolean isPriorityEvent) throws IOException
ResultPartitionWriter
AbstractEvent
to all channels.IOException
public ResultSubpartitionView createSubpartitionView(int subpartitionId, BufferAvailabilityListener availabilityListener) throws IOException
ResultPartitionWriter
IOException
public void alignedBarrierTimeout(long checkpointId) throws IOException
ResultPartitionWriter
IOException
public void abortCheckpoint(long checkpointId, CheckpointException cause)
ResultPartitionWriter
public void flushAll()
ResultPartitionWriter
public void flush(int subpartitionIndex)
ResultPartitionWriter
public void finish() throws IOException
ResultPartition
After this operation, it is not possible to add further data to the result partition.
For BLOCKING results, this will trigger the deployment of consuming tasks.
finish
in interface ResultPartitionWriter
finish
in class ResultPartition
IOException
public void close()
ResultPartitionWriter
close
in interface AutoCloseable
close
in interface ResultPartitionWriter
close
in class ResultPartition
protected void releaseInternal()
ResultPartition
releaseInternal
in class ResultPartition
public int getNumberOfQueuedBuffers()
ResultPartition
getNumberOfQueuedBuffers
in class ResultPartition
public long getSizeOfQueuedBuffersUnsafe()
ResultPartition
getSizeOfQueuedBuffersUnsafe
in class ResultPartition
public int getNumberOfQueuedBuffers(int targetSubpartition)
ResultPartition
getNumberOfQueuedBuffers
in class ResultPartition
public void notifyEndOfData(StopMode mode) throws IOException
ResultPartitionWriter
ResultPartitionWriter
have emitted all the
user records.notifyEndOfData
in interface ResultPartitionWriter
notifyEndOfData
in class ResultPartition
mode
- tells if we should flush all records or not (it is false in case of
stop-with-savepoint (--no-drain))IOException
Copyright © 2014–2024 The Apache Software Foundation. All rights reserved.