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. However, these are
only interpreted if the streaming program runs on TimeCharacteristic.EventTime
. On other
time characteristics (TimeCharacteristic.IngestionTime
and TimeCharacteristic.ProcessingTime
), the watermarks from the source function are ignored.
TimeCharacteristic
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
emit elements.
Sources that implement CheckpointedFunction
must lock on the checkpoint
lock (using a synchronized block) before updating internal state and emitting elements, to
make both an atomic operation:
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);
}
}
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.
When a source is canceled, the executing thread will also be interrupted (via Thread.interrupt()
). The interruption happens strictly after this method has been called, so
any interruption handler can rely on the fact that this method has completed. It is good
practice to make any flags altered by this method "volatile", in order to guarantee the
visibility of the effects of this method to any interruption handler.
Copyright © 2014–2021 The Apache Software Foundation. All rights reserved.