SplitT
- the type of the splits.@Public public interface SplitEnumeratorContext<SplitT extends SourceSplit>
SplitEnumerator
. This class serves the following purposes:
Modifier and Type | Method and Description |
---|---|
default void |
assignSplit(SplitT split,
int subtask)
Assigns a single split.
|
void |
assignSplits(SplitsAssignment<SplitT> newSplitAssignments)
Assign the splits.
|
<T> void |
callAsync(Callable<T> callable,
BiConsumer<T,Throwable> handler)
Invoke the callable and handover the return value to the handler which will be executed by
the source coordinator.
|
<T> void |
callAsync(Callable<T> callable,
BiConsumer<T,Throwable> handler,
long initialDelayMillis,
long periodMillis)
Invoke the given callable periodically and handover the return value to the handler which
will be executed by the source coordinator.
|
int |
currentParallelism()
Get the current parallelism of this Source.
|
SplitEnumeratorMetricGroup |
metricGroup() |
Map<Integer,ReaderInfo> |
registeredReaders()
Get the currently registered readers.
|
default Map<Integer,Map<Integer,ReaderInfo>> |
registeredReadersOfAttempts()
Get the currently registered readers of all the subtask attempts.
|
void |
runInCoordinatorThread(Runnable runnable)
Invoke the given runnable in the source coordinator thread.
|
default void |
sendEventToSourceReader(int subtaskId,
int attemptNumber,
SourceEvent event)
Send a source event to a source reader.
|
void |
sendEventToSourceReader(int subtaskId,
SourceEvent event)
Send a source event to a source reader.
|
default void |
setIsProcessingBacklog(boolean isProcessingBacklog)
Reports to JM whether this source is currently processing backlog.
|
void |
signalNoMoreSplits(int subtask)
Signals a subtask that it will not receive any further split.
|
SplitEnumeratorMetricGroup metricGroup()
void sendEventToSourceReader(int subtaskId, SourceEvent event)
subtaskId
- the subtask id of the source reader to send this event to.event
- the source event to send.default void sendEventToSourceReader(int subtaskId, int attemptNumber, SourceEvent event)
sendEventToSourceReader(int, SourceEvent)
but it is
aware of the subtask execution attempt to send this event to.
The SplitEnumerator
must invoke this method instead of sendEventToSourceReader(int, SourceEvent)
if it is used in cases that a subtask can have
multiple concurrent execution attempts, e.g. if speculative execution is enabled. Otherwise
an error will be thrown when the split enumerator tries to send a custom source event.
subtaskId
- the subtask id of the source reader to send this event to.attemptNumber
- the attempt number of the source reader to send this event to.event
- the source event to send.int currentParallelism()
Map<Integer,ReaderInfo> registeredReaders()
Note that if a subtask has multiple concurrent attempts, the map will contain the earliest
attempt of that subtask. This is for compatibility purpose. It's recommended to use registeredReadersOfAttempts()
instead.
default Map<Integer,Map<Integer,ReaderInfo>> registeredReadersOfAttempts()
void assignSplits(SplitsAssignment<SplitT> newSplitAssignments)
newSplitAssignments
- the new split assignments to add.default void assignSplit(SplitT split, int subtask)
When assigning multiple splits, it is more efficient to assign all of them in a single
call to the assignSplits(SplitsAssignment)
method.
split
- The new splitsubtask
- The index of the operator's parallel subtask that shall receive the split.void signalNoMoreSplits(int subtask)
subtask
- The index of the operator's parallel subtask that shall be signaled it will
not receive any further split.<T> void callAsync(Callable<T> callable, BiConsumer<T,Throwable> handler)
Callable
s may be executed in a thread pool concurrently.
It is important to make sure that the callable does not modify any shared state,
especially the states that will be a part of the SplitEnumerator.snapshotState(long)
.
Otherwise, there might be unexpected behavior.
Note that an exception thrown from the handler would result in failing the job.
callable
- a callable to call.handler
- a handler that handles the return value of or the exception thrown from the
callable.<T> void callAsync(Callable<T> callable, BiConsumer<T,Throwable> handler, long initialDelayMillis, long periodMillis)
Callable
s may be executed in a thread pool concurrently.
It is important to make sure that the callable does not modify any shared state,
especially the states that will be a part of the SplitEnumerator.snapshotState(long)
.
Otherwise, there might be unexpected behavior.
Note that an exception thrown from the handler would result in failing the job.
callable
- the callable to call.handler
- a handler that handles the return value of or the exception thrown from the
callable.initialDelayMillis
- the initial delay of calling the callable, in milliseconds.periodMillis
- the period between two invocations of the callable, in milliseconds.void runInCoordinatorThread(Runnable runnable)
This can be useful when the enumerator needs to execute some action (like assignSplits) triggered by some external events. E.g., Watermark from another source advanced and this source now be able to assign splits to awaiting readers. The trigger can be initiated from the coordinator thread of the other source. Instead of using lock for thread safety, this API allows to run such externally triggered action in the coordinator thread. Hence, we can ensure all enumerator actions are serialized in the single coordinator thread.
It is important that the runnable does not block.
runnable
- a runnable to execute@PublicEvolving default void setIsProcessingBacklog(boolean isProcessingBacklog)
When source is processing backlog, it means the records being emitted by this source is already stale and there is no processing latency requirement for these records. This allows downstream operators to optimize throughput instead of reducing latency for intermediate results.
If no API has been explicitly invoked to specify the backlog status of a source, the source is considered to have isProcessingBacklog=false by default.
Copyright © 2014–2024 The Apache Software Foundation. All rights reserved.