Interface SourceFunction<T>

  • Type Parameters:
    T - The type of the elements produced by this source.
    All Superinterfaces:
    org.apache.flink.api.common.functions.Function, Serializable
    All Known Subinterfaces:
    ExternallyInducedSource<T,​CD>, ParallelSourceFunction<OUT>
    All Known Implementing Classes:
    ContinuousFileMonitoringFunction, FileMonitoringFunction, FromElementsFunction, FromIteratorFunction, FromSplittableIteratorFunction, InputFormatSourceFunction, RichParallelSourceFunction, RichSourceFunction, SocketTextStreamFunction, StatefulSequenceSource

    @Internal
    public interface SourceFunction<T>
    extends org.apache.flink.api.common.functions.Function, Serializable
    Deprecated.
    This interface will be removed in future versions. Use the new Source interface instead. NOTE: All sub-tasks from FLINK-28045 must be closed before this API can be completely removed.
    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(org.apache.flink.streaming.api.functions.source.legacy.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.

    CheckpointedFunction Sources

    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);
          }
     }
     

    Timestamps and watermarks:

    Sources may assign timestamps to elements and may manually emit watermarks via the methods SourceFunction.SourceContext.collectWithTimestamp(Object, long) and SourceFunction.SourceContext.emitWatermark(Watermark).

    • Method Detail

      • cancel

        void cancel()
        Deprecated.
        Cancels the source. Most sources will have a while loop inside the 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.