Class InputGate
- All Implemented Interfaces:
AutoCloseable,AvailabilityProvider,ChannelStateHolder,PullingAsyncDataInput<BufferOrEvent>
- Direct Known Subclasses:
IndexedInputGate,UnionInputGate
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.
-
Nested Class Summary
Nested ClassesModifier and TypeClassDescriptionprotected static classSimple pojo for INPUT, DATA and moreAvailable.Nested classes/interfaces inherited from interface org.apache.flink.runtime.io.AvailabilityProvider
AvailabilityProvider.AvailabilityHelperNested classes/interfaces inherited from interface org.apache.flink.runtime.io.PullingAsyncDataInput
PullingAsyncDataInput.EndOfDataStatus -
Field Summary
FieldsModifier and TypeFieldDescriptionprotected final AvailabilityProvider.AvailabilityHelperprotected final AvailabilityProvider.AvailabilityHelperFields inherited from interface org.apache.flink.runtime.io.AvailabilityProvider
AVAILABLE -
Constructor Summary
Constructors -
Method Summary
Modifier and TypeMethodDescriptionabstract voidacknowledgeAllRecordsProcessed(InputChannelInfo channelInfo) abstract voidabstract InputChannelgetChannel(int channelIndex) Returns the channel of this gate.Returns the channel infos of this gate.abstract Optional<BufferOrEvent>getNext()Blocking call waiting for nextBufferOrEvent.abstract intNotifies when a priority event has been enqueued.abstract CompletableFuture<Void>abstract booleanabstract Optional<BufferOrEvent>pollNext()Poll theBufferOrEvent.abstract voidabstract voidresumeConsumption(InputChannelInfo channelInfo) abstract voidabstract voidsendTaskEvent(TaskEvent event) voidsetChannelStateWriter(ChannelStateWriter channelStateWriter) Injects theChannelStateWriter.abstract voidsetup()Setup gate, potentially heavy-weight, blocking operation comparing to just creation.Methods inherited from class java.lang.Object
clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, waitMethods inherited from interface java.lang.AutoCloseable
closeMethods inherited from interface org.apache.flink.runtime.io.AvailabilityProvider
isApproximatelyAvailable, isAvailableMethods inherited from interface org.apache.flink.runtime.io.PullingAsyncDataInput
hasReceivedEndOfData
-
Field Details
-
availabilityHelper
-
priorityAvailabilityHelper
-
-
Constructor Details
-
InputGate
public InputGate()
-
-
Method Details
-
setChannelStateWriter
Description copied from interface:ChannelStateHolderInjects theChannelStateWriter. Must only be called once.- Specified by:
setChannelStateWriterin interfaceChannelStateHolder
-
getNumberOfInputChannels
public abstract int getNumberOfInputChannels() -
isFinished
public abstract boolean isFinished()- Specified by:
isFinishedin interfacePullingAsyncDataInput<BufferOrEvent>- Returns:
- true if is finished and for example end of input was reached, false otherwise.
-
getNext
Blocking call waiting for nextBufferOrEvent.Note: It should be guaranteed that the previous returned buffer has been recycled before getting next one.
- Returns:
Optional.empty()ifisFinished()returns true.- Throws:
IOExceptionInterruptedException
-
pollNext
Poll theBufferOrEvent.Note: It should be guaranteed that the previous returned buffer has been recycled before polling next one.
- Specified by:
pollNextin interfacePullingAsyncDataInput<BufferOrEvent>- Returns:
Optional.empty()if there is no data to return or ifisFinished()returns true.- Throws:
IOExceptionInterruptedException
-
sendTaskEvent
- Throws:
IOException
-
getAvailableFuture
- Specified by:
getAvailableFuturein interfaceAvailabilityProvider- Returns:
- a future that is completed if there are more records available. If there are more
records available immediately,
AvailabilityProvider.AVAILABLEshould be returned. Previously returned not completed futures should become completed once there are more records available.
-
resumeGateConsumption
- Throws:
IOException
-
resumeConsumption
- Throws:
IOException
-
acknowledgeAllRecordsProcessed
public abstract void acknowledgeAllRecordsProcessed(InputChannelInfo channelInfo) throws IOException - Throws:
IOException
-
getChannel
Returns the channel of this gate. -
getChannelInfos
Returns the channel infos of this gate. -
getPriorityEventAvailableFuture
Notifies when a priority event has been enqueued. If this future is queried from task thread, it is guaranteed that a priority event is available and retrieved throughgetNext(). -
setup
Setup gate, potentially heavy-weight, blocking operation comparing to just creation.- Throws:
IOException
-
requestPartitions
- Throws:
IOException
-
getStateConsumedFuture
-
finishReadRecoveredState
- Throws:
IOException
-