public class ResultPartition extends Object implements ResultPartitionWriter, BufferPoolOwner
This class is the runtime part of a logical IntermediateResultPartition
. Essentially,
a result partition is a collection of Buffer
instances. The buffers are organized in one
or more ResultSubpartition
instances, which further partition the data depending on the
number of consuming tasks and the data DistributionPattern
.
Tasks, which consume a result partition have to request one of its subpartitions. The request
happens either remotely (see RemoteInputChannel
) or locally (see LocalInputChannel
)
The life-cycle of each result partition has three (possibly overlapping) phases:
Before a consuming task can request the result, it has to be deployed. The time of deployment depends on the PIPELINED vs. BLOCKING characteristic of the result partition. With pipelined results, receivers are deployed as soon as the first buffer is added to the result partition. With blocking results on the other hand, receivers are deployed after the partition is finished.
Modifier and Type | Field and Description |
---|---|
int |
numTargetKeyGroups |
Constructor and Description |
---|
ResultPartition(String owningTaskName,
TaskActions taskActions,
JobID jobId,
ResultPartitionID partitionId,
ResultPartitionType partitionType,
int numberOfSubpartitions,
int numTargetKeyGroups,
ResultPartitionManager partitionManager,
ResultPartitionConsumableNotifier partitionConsumableNotifier,
IOManager ioManager,
boolean sendScheduleOrUpdateConsumersMessage) |
Modifier and Type | Method and Description |
---|---|
void |
addBufferConsumer(BufferConsumer bufferConsumer,
int subpartitionIndex)
Adds the bufferConsumer to the subpartition with the given index.
|
ResultSubpartitionView |
createSubpartitionView(int index,
BufferAvailabilityListener availabilityListener)
Returns the requested subpartition.
|
void |
destroyBufferPool() |
void |
finish()
Finishes the result partition.
|
void |
flush(int subpartitionIndex)
Manually trigger consumption from enqueued
BufferConsumers in one specified subpartition. |
void |
flushAll()
Manually trigger consumption from enqueued
BufferConsumers in all subpartitions. |
BufferPool |
getBufferPool() |
BufferProvider |
getBufferProvider() |
Throwable |
getFailureCause() |
JobID |
getJobId() |
int |
getNumberOfQueuedBuffers() |
int |
getNumberOfSubpartitions() |
int |
getNumTargetKeyGroups() |
String |
getOwningTaskName() |
ResultPartitionID |
getPartitionId() |
ResultPartitionType |
getPartitionType()
Returns the type of this result partition.
|
boolean |
isReleased()
Whether this partition is released.
|
void |
registerBufferPool(BufferPool bufferPool)
Registers a buffer pool with this result partition.
|
void |
release() |
void |
release(Throwable cause)
Releases the result partition.
|
void |
releaseMemory(int toRelease)
Releases buffers held by this result partition.
|
String |
toString() |
public ResultPartition(String owningTaskName, TaskActions taskActions, JobID jobId, ResultPartitionID partitionId, ResultPartitionType partitionType, int numberOfSubpartitions, int numTargetKeyGroups, ResultPartitionManager partitionManager, ResultPartitionConsumableNotifier partitionConsumableNotifier, IOManager ioManager, boolean sendScheduleOrUpdateConsumersMessage)
public void registerBufferPool(BufferPool bufferPool)
There is one pool for each result partition, which is shared by all its sub partitions.
The pool is registered with the partition *after* it as been constructed in order to conform
to the life-cycle of task registrations in the TaskManager
.
public JobID getJobId()
public String getOwningTaskName()
public ResultPartitionID getPartitionId()
getPartitionId
in interface ResultPartitionWriter
public int getNumberOfSubpartitions()
getNumberOfSubpartitions
in interface ResultPartitionWriter
public BufferProvider getBufferProvider()
getBufferProvider
in interface ResultPartitionWriter
public BufferPool getBufferPool()
public int getNumberOfQueuedBuffers()
public ResultPartitionType getPartitionType()
public void addBufferConsumer(BufferConsumer bufferConsumer, int subpartitionIndex) throws IOException
ResultPartitionWriter
For PIPELINED ResultPartitionType
s,
this will trigger the deployment of consuming tasks after the first buffer has been added.
This method takes the ownership of the passed bufferConsumer
and thus is responsible for releasing
it's resources.
To avoid problems with data re-ordering, before adding new BufferConsumer
the previously added one
the given subpartitionIndex
must be marked as BufferConsumer.isFinished()
.
addBufferConsumer
in interface ResultPartitionWriter
IOException
public void flushAll()
ResultPartitionWriter
BufferConsumers
in all subpartitions.flushAll
in interface ResultPartitionWriter
public void flush(int subpartitionIndex)
ResultPartitionWriter
BufferConsumers
in one specified subpartition.flush
in interface ResultPartitionWriter
public void finish() throws IOException
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.
IOException
public void release()
public void release(Throwable cause)
public void destroyBufferPool()
public ResultSubpartitionView createSubpartitionView(int index, BufferAvailabilityListener availabilityListener) throws IOException
IOException
public Throwable getFailureCause()
public int getNumTargetKeyGroups()
getNumTargetKeyGroups
in interface ResultPartitionWriter
public void releaseMemory(int toRelease) throws IOException
This is a callback from the buffer pool, which is registered for result partitions, which are back pressure-free.
releaseMemory
in interface BufferPoolOwner
IOException
public boolean isReleased()
A partition is released when each subpartition is either consumed and communication is closed by consumer or failed. A partition is also released if task is cancelled.
Copyright © 2014–2020 The Apache Software Foundation. All rights reserved.