OutputT
- Type of records output by this source.CheckpointMarkT
- Type of checkpoint marks used by the readers of this source.public abstract class UnboundedSource<OutputT,CheckpointMarkT extends UnboundedSource.CheckpointMark> extends Source<OutputT>
Source
that reads an unbounded amount of input and, because of that, supports some
additional operations such as checkpointing, watermarks, and record ids.
See Window
and Trigger
for more information on timestamps and
watermarks.
Modifier and Type | Class and Description |
---|---|
static interface |
UnboundedSource.CheckpointMark
A marker representing the progress and state of an
UnboundedSource.UnboundedReader . |
static class |
UnboundedSource.UnboundedReader<OutputT>
A
Reader that reads an unbounded amount of input. |
Source.Reader<T>
Constructor and Description |
---|
UnboundedSource() |
Modifier and Type | Method and Description |
---|---|
abstract UnboundedSource.UnboundedReader<OutputT> |
createReader(PipelineOptions options,
CheckpointMarkT checkpointMark)
Create a new
UnboundedSource.UnboundedReader to read from this source, resuming from the given
checkpoint if present. |
abstract Coder<CheckpointMarkT> |
getCheckpointMarkCoder()
Returns a
Coder for encoding and decoding the checkpoints for this source. |
boolean |
requiresDeduping()
Returns whether this source requires explicit deduping.
|
abstract java.util.List<? extends UnboundedSource<OutputT,CheckpointMarkT>> |
split(int desiredNumSplits,
PipelineOptions options)
Returns a list of
UnboundedSource objects representing the instances of this source
that should be used when executing the workflow. |
getDefaultOutputCoder, getOutputCoder, populateDisplayData, validate
public abstract java.util.List<? extends UnboundedSource<OutputT,CheckpointMarkT>> split(int desiredNumSplits, PipelineOptions options) throws java.lang.Exception
UnboundedSource
objects representing the instances of this source
that should be used when executing the workflow. Each split should return a separate partition
of the input data.
For example, for a source reading from a growing directory of files, each split could correspond to a prefix of file names.
Some sources are not splittable, such as reading from a single TCP stream. In that case, only a single split should be returned.
Some data sources automatically partition their data among readers. For these types of
inputs, n
identical replicas of the top-level source can be returned.
The size of the returned list should be as close to desiredNumSplits
as possible,
but does not have to match exactly. A low number of splits will limit the amount of parallelism
in the source.
java.lang.Exception
public abstract UnboundedSource.UnboundedReader<OutputT> createReader(PipelineOptions options, CheckpointMarkT checkpointMark) throws java.io.IOException
UnboundedSource.UnboundedReader
to read from this source, resuming from the given
checkpoint if present.java.io.IOException
public abstract Coder<CheckpointMarkT> getCheckpointMarkCoder()
Coder
for encoding and decoding the checkpoints for this source.public boolean requiresDeduping()
This is needed if the underlying data source can return the same record multiple times, such a queuing system with a pull-ack model. Sources where the records read are uniquely identified by the persisted state in the CheckpointMark do not need this.
Generally, if UnboundedSource.CheckpointMark.finalizeCheckpoint()
is overridden, this method should
return true. Checkpoint finalization is best-effort, and readers can be resumed from a
checkpoint that has not been finalized.