T
- The type of the elements produced by this source.@Public public interface SourceFunction<T> extends Function, Serializable
run(org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext<T>)
method is called with
a SourceFunction.SourceContext
that can be used for emitting elements. The run method can run for as
long as necessary. The source must, however, react to an invocation of cancel()
by
breaking out of its main loop.
Sources that also implement the CheckpointedFunction
interface must ensure that state
checkpointing, updating of internal state and emission of elements are not done concurrently.
This is achieved by using the provided checkpointing lock object to protect update of state and
emission of elements in a synchronized block.
This is the basic pattern one should follow when implementing a checkpointed source:
public class ExampleCountSource implements SourceFunction<Long>, CheckpointedFunction {
private long count = 0L;
private volatile boolean isRunning = true;
private transient ListState<Long> checkpointedCount;
public void run(SourceContext<T> ctx) {
while (isRunning && count < 1000) {
// this synchronized block ensures that state checkpointing,
// internal state updates and emission of elements are an atomic operation
synchronized (ctx.getCheckpointLock()) {
ctx.collect(count);
count++;
}
}
}
public void cancel() {
isRunning = false;
}
public void initializeState(FunctionInitializationContext context) {
this.checkpointedCount = context
.getOperatorStateStore()
.getListState(new ListStateDescriptor<>("count", Long.class));
if (context.isRestored()) {
for (Long count : this.checkpointedCount.get()) {
this.count += count;
}
}
}
public void snapshotState(FunctionSnapshotContext context) {
this.checkpointedCount.clear();
this.checkpointedCount.add(count);
}
}
Sources may assign timestamps to elements and may manually emit watermarks via the methods
SourceFunction.SourceContext.collectWithTimestamp(Object, long)
and SourceFunction.SourceContext.emitWatermark(Watermark)
.
Modifier and Type | Interface and Description |
---|---|
static interface |
SourceFunction.SourceContext<T>
Interface that source functions use to emit elements, and possibly watermarks.
|
Modifier and Type | Method and Description |
---|---|
void |
cancel()
Cancels the source.
|
void |
run(SourceFunction.SourceContext<T> ctx)
Starts the source.
|
void run(SourceFunction.SourceContext<T> ctx) throws Exception
SourceFunction.SourceContext
to emit elements. Sources
that checkpoint their state for fault tolerance should use the SourceFunction.SourceContext.getCheckpointLock()
checkpoint lock} to ensure consistency between the
bookkeeping and emitting the elements.
Sources that implement CheckpointedFunction
must lock on the SourceFunction.SourceContext.getCheckpointLock()
checkpoint lock} checkpoint lock (using a synchronized
block) before updating internal state and emitting elements, to make both an atomic
operation.
Refer to the top-level class docs
for an example.
ctx
- The context to emit elements to and for accessing locks.Exception
void cancel()
run(SourceContext)
method. The implementation needs to ensure that the source will break
out of that loop after this method is called.
A typical pattern is to have an "volatile boolean isRunning"
flag that is set to
false
in this method. That flag is checked in the loop condition.
In case of an ungraceful shutdown (cancellation of the source operator, possibly for
failover), the thread that calls run(SourceContext)
will also be interrupted
) by the Flink runtime, in order to speed up the cancellation
(to ensure threads exit blocking methods fast, like I/O, blocking queues, etc.). The
interruption happens strictly after this method has been called, so any interruption handler
can rely on the fact that this method has completed (for example to ignore exceptions that
happen after cancellation).
During graceful shutdown (for example stopping a job with a savepoint), the program must
cleanly exit the run(SourceContext)
method soon after this method was called. The
Flink runtime will NOT interrupt the source thread during graceful shutdown. Source
implementors must ensure that no thread interruption happens on any thread that emits records
through the SourceContext
from the run(SourceContext)
method; otherwise the
clean shutdown may fail when threads are interrupted while processing the final records.
Because the SourceFunction
cannot easily differentiate whether the shutdown should
be graceful or ungraceful, we recommend that implementors refrain from interrupting any
threads that interact with the SourceContext
at all. You can rely on the Flink
runtime to interrupt the source thread in case of ungraceful cancellation. Any additionally
spawned threads that directly emit records through the SourceContext
should use a
shutdown method that does not rely on thread interruption.
Copyright © 2014–2022 The Apache Software Foundation. All rights reserved.