Interface SplitAssigner
-
- All Superinterfaces:
java.lang.AutoCloseable
,java.io.Closeable
- All Known Implementing Classes:
DefaultSplitAssigner
public interface SplitAssigner extends java.io.Closeable
SplitAssigner interface is extracted out as a separate component so that we can plug in different split assignment strategy for different requirements. E.g.- Simple assigner with no ordering guarantee or locality aware optimization.
- Locality aware assigner that prefer splits that are local.
- Snapshot aware assigner that assign splits based on the order they are committed.
- Event time alignment assigner that assign splits satisfying certain time ordering within a single source or across sources.
Assigner implementation needs to be thread safe. Enumerator call the assigner APIs mostly from the coordinator thread. But enumerator may call the
pendingSplitCount()
from the I/O threads.
-
-
Method Summary
All Methods Instance Methods Abstract Methods Default Methods Modifier and Type Method Description default void
close()
Some assigners may need to perform certain actions when their corresponding enumerators are closedGetSplitResult
getNext(java.lang.String hostname)
Request a new split from the assigner when enumerator trying to assign splits to awaiting readers.java.util.concurrent.CompletableFuture<java.lang.Void>
isAvailable()
Enumerator can get a notification via CompletableFuture when the assigner has more splits available later.default void
onCompletedSplits(java.util.Collection<java.lang.String> completedSplitIds)
Some assigner (like event time alignment) may rack in-progress splits to advance watermark upon completed splitsvoid
onDiscoveredSplits(java.util.Collection<IcebergSourceSplit> splits)
Add new splits discovered by enumeratorvoid
onUnassignedSplits(java.util.Collection<IcebergSourceSplit> splits)
Forward addSplitsBack event (for failed reader) to assignerlong
pendingRecords()
Return the number of pending records, which can act as a measure of the source lag.int
pendingSplitCount()
Return the number of pending splits that haven't been assigned yet.default void
start()
Some assigners may need to start background threads or perform other activity such as registering as listeners to updates from other event sources e.g., watermark tracker.java.util.Collection<IcebergSourceSplitState>
state()
Get assigner state for checkpointing.
-
-
-
Method Detail
-
start
default void start()
Some assigners may need to start background threads or perform other activity such as registering as listeners to updates from other event sources e.g., watermark tracker.
-
close
default void close()
Some assigners may need to perform certain actions when their corresponding enumerators are closed- Specified by:
close
in interfacejava.lang.AutoCloseable
- Specified by:
close
in interfacejava.io.Closeable
-
getNext
GetSplitResult getNext(@Nullable java.lang.String hostname)
Request a new split from the assigner when enumerator trying to assign splits to awaiting readers.If enumerator wasn't able to assign the split (e.g., reader disconnected), enumerator should call
onUnassignedSplits(java.util.Collection<org.apache.iceberg.flink.source.split.IcebergSourceSplit>)
to return the split.
-
onDiscoveredSplits
void onDiscoveredSplits(java.util.Collection<IcebergSourceSplit> splits)
Add new splits discovered by enumerator
-
onUnassignedSplits
void onUnassignedSplits(java.util.Collection<IcebergSourceSplit> splits)
Forward addSplitsBack event (for failed reader) to assigner
-
onCompletedSplits
default void onCompletedSplits(java.util.Collection<java.lang.String> completedSplitIds)
Some assigner (like event time alignment) may rack in-progress splits to advance watermark upon completed splits
-
state
java.util.Collection<IcebergSourceSplitState> state()
Get assigner state for checkpointing. This is a super-set API that works for all currently imagined assigners.
-
isAvailable
java.util.concurrent.CompletableFuture<java.lang.Void> isAvailable()
Enumerator can get a notification via CompletableFuture when the assigner has more splits available later. Enumerator should schedule assignment in the thenAccept action of the future.Assigner will return the same future if this method is called again before the previous future is completed.
The future can be completed from other thread, e.g. the coordinator thread from another thread for event time alignment.
If enumerator need to trigger action upon the future completion, it may want to run it in the coordinator thread using
SplitEnumeratorContext.runInCoordinatorThread(Runnable)
.
-
pendingSplitCount
int pendingSplitCount()
Return the number of pending splits that haven't been assigned yet.The enumerator can poll this API to publish a metric on the number of pending splits.
The enumerator can also use this information to throttle split discovery for streaming read. If there are already many pending splits tracked by the assigner, it is undesirable to discover more splits and track them in the assigner. That will increase the memory footprint and enumerator checkpoint size.
Throttling works better together with
ScanContext.maxPlanningSnapshotCount()
. Otherwise, the next split discovery after throttling will just discover all non-enumerated snapshots and splits, which defeats the purpose of throttling.
-
pendingRecords
long pendingRecords()
Return the number of pending records, which can act as a measure of the source lag. This value could be an estimation if the exact number of records cannot be accurately computed.
-
-