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);
}
}
TimeCharacteristic.EventTime
. On other time characteristics
(TimeCharacteristic.IngestionTime
and TimeCharacteristic.ProcessingTime
),
the watermarks from the source function are ignored.
StoppableFunction
interface. "Stopping" a function, in contrast to "canceling" means a graceful exit that leaves the
state and the emitted elements in a consistent state.
When a source is stopped, the executing thread is not interrupted, but expected to leave the
run(SourceContext)
method in reasonable time on its own, preserving the atomicity
of state updates and element emission.
StoppableFunction
,
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–2020 The Apache Software Foundation. All rights reserved.