public abstract class InputGate extends Object implements PullingAsyncDataInput<BufferOrEvent>, AutoCloseable, ChannelStateHolder
Each intermediate result is partitioned over its producing parallel subtasks; each of these partitions is furthermore partitioned into one or more subpartitions.
As an example, consider a map-reduce program, where the map operator produces data and the reduce operator consumes the produced data.
+-----+ +---------------------+ +--------+
| Map | = produce => | Intermediate Result | <= consume = | Reduce |
+-----+ +---------------------+ +--------+
When deploying such a program in parallel, the intermediate result will be partitioned over its producing parallel subtasks; each of these partitions is furthermore partitioned into one or more subpartitions.
Intermediate result
+-----------------------------------------+
| +----------------+ | +-----------------------+
+-------+ | +-------------+ +=> | Subpartition 1 | | <=======+=== | Input Gate | Reduce 1 |
| Map 1 | ==> | | Partition 1 | =| +----------------+ | | +-----------------------+
+-------+ | +-------------+ +=> | Subpartition 2 | | <==+ |
| +----------------+ | | | Subpartition request
| | | |
| +----------------+ | | |
+-------+ | +-------------+ +=> | Subpartition 1 | | <==+====+
| Map 2 | ==> | | Partition 2 | =| +----------------+ | | +-----------------------+
+-------+ | +-------------+ +=> | Subpartition 2 | | <==+======== | Input Gate | Reduce 2 |
| +----------------+ | +-----------------------+
+-----------------------------------------+
In the above example, two map subtasks produce the intermediate result in parallel, resulting in two partitions (Partition 1 and 2). Each of these partitions is further partitioned into two subpartitions -- one for each parallel reduce subtask. As shown in the Figure, each reduce task will have an input gate attached to it. This will provide its input, which will consist of one subpartition from each partition of the intermediate result.
Modifier and Type | Class and Description |
---|---|
protected static class |
InputGate.InputWithData<INPUT,DATA>
Simple pojo for INPUT, DATA and moreAvailable.
|
PullingAsyncDataInput.EndOfDataStatus
AvailabilityProvider.AvailabilityHelper
Modifier and Type | Field and Description |
---|---|
protected AvailabilityProvider.AvailabilityHelper |
availabilityHelper |
protected AvailabilityProvider.AvailabilityHelper |
priorityAvailabilityHelper |
AVAILABLE
Constructor and Description |
---|
InputGate() |
Modifier and Type | Method and Description |
---|---|
abstract void |
acknowledgeAllRecordsProcessed(InputChannelInfo channelInfo) |
abstract void |
finishReadRecoveredState() |
CompletableFuture<?> |
getAvailableFuture() |
abstract InputChannel |
getChannel(int channelIndex)
Returns the channel of this gate.
|
List<InputChannelInfo> |
getChannelInfos()
Returns the channel infos of this gate.
|
abstract Optional<BufferOrEvent> |
getNext()
Blocking call waiting for next
BufferOrEvent . |
abstract int |
getNumberOfInputChannels() |
CompletableFuture<?> |
getPriorityEventAvailableFuture()
Notifies when a priority event has been enqueued.
|
abstract CompletableFuture<Void> |
getStateConsumedFuture() |
abstract boolean |
isFinished() |
abstract Optional<BufferOrEvent> |
pollNext()
Poll the
BufferOrEvent . |
abstract void |
requestPartitions() |
abstract void |
resumeConsumption(InputChannelInfo channelInfo) |
abstract void |
sendTaskEvent(TaskEvent event) |
void |
setChannelStateWriter(ChannelStateWriter channelStateWriter)
Injects the
ChannelStateWriter . |
abstract void |
setup()
Setup gate, potentially heavy-weight, blocking operation comparing to just creation.
|
clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
hasReceivedEndOfData
and, isApproximatelyAvailable, isAvailable, or
close
protected final AvailabilityProvider.AvailabilityHelper availabilityHelper
protected final AvailabilityProvider.AvailabilityHelper priorityAvailabilityHelper
public void setChannelStateWriter(ChannelStateWriter channelStateWriter)
ChannelStateHolder
ChannelStateWriter
. Must only be called once.setChannelStateWriter
in interface ChannelStateHolder
public abstract int getNumberOfInputChannels()
public abstract boolean isFinished()
isFinished
in interface PullingAsyncDataInput<BufferOrEvent>
public abstract Optional<BufferOrEvent> getNext() throws IOException, InterruptedException
BufferOrEvent
.
Note: It should be guaranteed that the previous returned buffer has been recycled before getting next one.
Optional.empty()
if isFinished()
returns true.IOException
InterruptedException
public abstract Optional<BufferOrEvent> pollNext() throws IOException, InterruptedException
BufferOrEvent
.
Note: It should be guaranteed that the previous returned buffer has been recycled before polling next one.
pollNext
in interface PullingAsyncDataInput<BufferOrEvent>
Optional.empty()
if there is no data to return or if isFinished()
returns true.IOException
InterruptedException
public abstract void sendTaskEvent(TaskEvent event) throws IOException
IOException
public CompletableFuture<?> getAvailableFuture()
getAvailableFuture
in interface AvailabilityProvider
AvailabilityProvider.AVAILABLE
should be returned. Previously returned
not completed futures should become completed once there are more records available.public abstract void resumeConsumption(InputChannelInfo channelInfo) throws IOException
IOException
public abstract void acknowledgeAllRecordsProcessed(InputChannelInfo channelInfo) throws IOException
IOException
public abstract InputChannel getChannel(int channelIndex)
public List<InputChannelInfo> getChannelInfos()
public CompletableFuture<?> getPriorityEventAvailableFuture()
getNext()
.public abstract void setup() throws IOException
IOException
public abstract void requestPartitions() throws IOException
IOException
public abstract CompletableFuture<Void> getStateConsumedFuture()
public abstract void finishReadRecoveredState() throws IOException
IOException
Copyright © 2014–2024 The Apache Software Foundation. All rights reserved.