Base interface for all stream data sources in Flink. The contract of a stream source
is the following: When the source should start emitting elements, the
#run method
is called with a
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.
CheckpointedFunction Sources
Sources that also implement the
org.apache.flink.streaming.api.checkpoint.CheckpointedFunctioninterface 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, CheckpointedFunction }
}
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);
}
}
}
Timestamps and watermarks:
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.
Gracefully Stopping Functions
Functions may additionally implement the
org.apache.flink.api.common.functions.StoppableFunctioninterface. "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.