Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.


...

Page properties

...


Discussion thread

...

...

...

Jira
serverASF JIRA
serverId5aa69414-a9e9-3523-82ec-879b028fb15b
keyFLINK-10740

...

Release1.12


Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).

...

The SplitEnumerator is similar to the old batch source interface's functionality of creating splits and assigning splits. It runs only once, not in parallel (but could be thought of to parallelize in the future, if necessary).
It might run on the JobManager or in a single task on a TaskManager (see below "Where to run the Enumerator").

Example:

    • In the File Source , the SplitEnumerator lists all files (possibly sub-dividing them into blocks/ranges). 
    • For the Kafka Source, the SplitEnumerator finds all Kafka Partitions that the source should read from.

...

Each source should be able to work as a bounded (batch) and as an unbounded (continuous streaming) source.

The actual decision whether it becomes bounded or unbounded is made in the DataStream API when creating the source stream.The Boundedness is a property that is passed to source when creating the SplitEnumerator. The readers should be agnostic to this distinction, they will simply read the splits assigned to them. Whether a source is bounded or unbounded is passed to the SplitEnumerator upon creation.Boundedness is an intrinsic property to the source instance itself. In most cases, only the SplitEnumerators should know the Boundedness, while the SplitReaders are agnostic.

That way, we can also make the API type safe in the future, when we can explicitly model bounded streams.

Code Block
languagejava
titleDataStream API
linenumberstrue
ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();

FileSource<MyType> theSource = new ParquetFileSource("fs:///path/to/dir", AvroParquet.forSpecific(MyType.class));

DataStream<MyType> stream = env.continuousSource(theSource);

DataStream<MyType> boundedStream// The returned stream will be a DataStream if theSource is unbounded.
// After we add BoundedDataStream which extends DataStream, the returned stream will be a BoundedDataStream.
// This allows users to write programs working in both batch and stream execution mode.
DataStream<MyType> stream = env.boundedSourcesource(theSource);

// this would be an option once Once we add bounded streams to the DataStream API
BoundedDataStream<MyType> batch = env., we will also add the following API.
// The parameter has to be bounded otherwise an exception will be thrown.
BoundedDataStream<MyType> batch = env.boundedSource(theSource);

...

  • Splits are both the type of work assignment and the type of state held by the source. Assigning a split or restoring a split from a checkpoint is the same to the reader.
  • Advancing the reader is a non-blocking call that returns a future.
  • We build higher-level primitives on top of the main interface (see below "High-level Readers")
  • We hide event-time / watermarks in the SourceOutput and pass different source contexts for batch (no watermarks) and streaming (with watermarks).
    The SourceOutput also abstract the per-partition watermark tracking.

The 95653748 SourceReader will run as a PushingAsyncDataInput which works well with the new mailbox threading model in the tasks, similar to the network inputs.

...

The RecordsWithSplitIds returned by the SplitReader will be passed to an 95653748 RecordEmitter one by one. The RecordEmitter is responsible for the following:

  • Convert the raw record type <E> into the eventual record type <T>
  • Provide an event time timestamp for the record that it processes.

Failover

The state of the SplitEnumerator includes the following:

  • The unassigned splits
  • The splits that have been assigned but not successfully checkpointed yet.
    • The assigned but uncheckpointed splits will be associated with each of the checkpoint id they belong to.

The state of the SourceReader includes:

  • The assigned splits
  • The state of the splits (e.g. Kafka offsets, HDFS file offset, etc)

When the SplitEnumerator fails, a full failover will be performed. While it is possible to have a finer grained failover to only restore the state of the SplitEnumerator, we would like to address this in a separate FLIP.

When a SourceReader fails, the failed SourceReader will be restore to its last successful checkpoint. The SplitEnumerator will partially reset its state by adding the assigned-but-uncheckpointed splits back to the SplitEnumerator. In this case, only the failed subtask and its connected nodes will have to reset the states.

Where to run the enumerator

There was a long discussion about where to run the enumerator which we documented in the appendix. The final approach we took was very similar to option 3 with a few differences. The approach is following.

Each SplitEnumerator will be encapsulated in one SourceCoordinator. If there are multiple sources, multiple SourceCoordinator will there be. The SourceCoordinators will run in the JobMaster, but not as part of the ExecutionGraph. In this FLIP, we propose to failover the entire execution graph when the SplitEnumerator fails. A finer grained enumerator failover will be proposed in a later FLIP.

Per Split Event Time

With the introduction of the SourceSplit, we can actually emit per split event time for the users. We plan to propose the solution in a separate FLIP instead of in this FLIP to reduce the complexity.

Event Time Alignment

The event time alignment becomes easier to implement with the generic communication mechanism introduced between SplitEnumerator and SourceReader. In this FLIP we do not include this in the base implementation to reduce the complexity.

Public Interface

The public interface changes introduced by this FLIP consist of three parts.

  • The top level public interfaces.
  • The interfaces introduced as a part of the base implementation of the top level public interfaces.
    • The base implementation provides common functionalities required for most Source implementations. See base implementation for details.
  • The RPC gateway interface change for the generic message passing mechanism.

It is worth noting that while we will try best to maintain stable interfaces, the interfaces introduced as part of the base implementation (e.g. SplitReader) is more likely to change than the top level public interface such as SplitEnumerator / SourceReader. This is primarily because we expect to add more functionality into the base implementation over time.

Top level public interfaces

  • Source - A factory style class that helps create SplitEnumerator and SourceReader at runtime.
  • SourceSplit - An interface for all the split types.
  • SplitEnumerator - Discover the splits and assign them to the SourceReaders
  • SplitEnumeratorContext - Provide necessary information to the SplitEnumerator to assign splits and send custom events to the the SourceReaders.
  • SplitAssignment - A container class holding the source split assignment for each subtask.
  • SourceReader - Read the records from the splits assigned by the SplitEnumerator.
  • SourceReaderContext - Provide necessary function to the SourceReader to communicate with SplitEnumerator.
  • SourceOutput - A collector style interface to take the records and timestamps emit by the SourceReader.

Source

With the base implementation users writing their own source can just focus on:

  1. Fetch records from external system.
  2. Perform record parsing and conversion.
  3. Extract timestamps and optionally deal with watermarks. A followup FLIP will provide some default behaviors for users to deal with their watermark.

The base implementation can be roughly illustrated below:

Image Added

Some brief explanations:

  1. When a new split is added to the SourceReader by SplitEnumerator, the initial state of that new split is put into a state map maintained by the SourceReaderBase before the split is assigned to a SplitReader.
  2. The records are passed from the the SplitReaders to the RecordEmitter in RecordsBySplitIds. This allows the SplitReader to enqueue records in a batch manner, which benefits performance.
  3. The SourceReaderBase iterates over each records and looks up their corresponding split state. The Record and its corresponding split state is passed to the RecordEmitter.

Note that the abstraction of this base implementation does not specify where the deserialization is performed. Because the RecordEmitter is driven by the main mailbox thread of the task, ideally the deserialization should be done in the split reader so it is more scalable. It is also possible to introduce a deserialization thread pool to do that. However, the detail implementation of deserialization is not the focus of this FLIP and will be covered by followup FLIPs.

The interfaces used by the base implementation is covered in the section of interface for base implementation.

Failover

The state of the SplitEnumerator includes the following:

  • The unassigned splits
  • The splits that have been assigned but not successfully checkpointed yet.
    • The assigned but uncheckpointed splits will be associated with each of the checkpoint id they belong to.

The state of the SourceReader includes:

  • The assigned splits
  • The state of the splits (e.g. Kafka offsets, HDFS file offset, etc)

When the SplitEnumerator fails, a full failover will be performed. While it is possible to have a finer grained failover to only restore the state of the SplitEnumerator, we would like to address this in a separate FLIP.

When a SourceReader fails, the failed SourceReader will be restore to its last successful checkpoint. The SplitEnumerator will partially reset its state by adding the assigned-but-uncheckpointed splits back to the SplitEnumerator. In this case, only the failed subtask and its connected nodes will have to reset the states.

Where to run the enumerator

There was a long discussion about where to run the enumerator which we documented in the appendix. The final approach we took was very similar to option 3 with a few differences. The approach is following.

Each SplitEnumerator will be encapsulated in one SourceCoordinator. If there are multiple sources, multiple SourceCoordinator will there be. The SourceCoordinators will run in the JobMaster, but not as part of the ExecutionGraph. In this FLIP, we propose to failover the entire execution graph when the SplitEnumerator fails. A finer grained enumerator failover will be proposed in a later FLIP.

Per Split Event Time

With the introduction of the SourceSplit, we can actually emit per split event time for the users. We plan to propose the solution in a separate FLIP instead of in this FLIP to reduce the complexity.

Event Time Alignment

The event time alignment becomes easier to implement with the generic communication mechanism introduced between SplitEnumerator and SourceReader. In this FLIP we do not include this in the base implementation to reduce the complexity.

Public Interface

The public interface changes introduced by this FLIP consist of three parts.

  • The top level public interfaces.
  • The interfaces introduced as a part of the base implementation of the top level public interfaces.
    • The base implementation provides common functionalities required for most Source implementations. See base implementation for details.
  • The RPC gateway interface change for the generic message passing mechanism.

It is worth noting that while we will try best to maintain stable interfaces, the interfaces introduced as part of the base implementation (e.g. SplitReader) is more likely to change than the top level public interface such as SplitEnumerator / SourceReader. This is primarily because we expect to add more functionality into the base implementation over time.

Top level public interfaces

  • Source - A factory style class that helps create SplitEnumerator and SourceReader at runtime.
  • SourceSplit - An interface for all the split types.
  • SplitEnumerator - Discover the splits and assign them to the SourceReaders
  • SplitEnumeratorContext - Provide necessary information to the SplitEnumerator to assign splits and send custom events to the the SourceReaders.
  • SplitAssignment - A container class holding the source split assignment for each subtask.
  • SourceReader - Read the records from the splits assigned by the SplitEnumerator.
  • SourceReaderContext - Provide necessary function to the SourceReader to communicate with SplitEnumerator.
  • SourceOutput - A collector style interface to take the records and timestamps emit by the SourceReader.
  • WatermarkOutput - An interface for emitting watermark and indicate idleness of the source.
  • Watermark - A new Watermark class will be created in the package org.apache.flink.api.common.eventtime. This class will eventually replace the existing Watermark in org.apache.flink.streaming.api.watermark. This change allows flink-core to remain independent of other modules. Given that we will eventually put all the watermark generation into the Source, this change will be necessary. Note that this FLIP does not intended to change the existing way that watermark can be overridden in the DataStream after they are emitted by the source.

Source

Code Block
languagejava
titleSource
linenumberstrue
/**
 * The interface for Source. It acts like a factory class that helps construct
 * the {@link SplitEnumerator} and {@link SourceReader} and corresponding
 * serializers.
 *
 * @param <T>        The type of records produced by the source.
 * @param <SplitT>   The type of splits handled by the source.
 * @param <EnumChkT> The type of the enumerator checkpoints.
 */
public interface Source<T, SplitT extends SourceSplit, EnumChkT> extends Serializable {

	/**
	 * Get the boundedness of this source.
	 * 
	 * @return the boundedness of this source.
	 */
	Boundedness getBoundedness();

	/**
	 * Creates a new reader to read data from the spits it gets assigned.
	 * The reader starts fresh and does not have any state to resume.
	 *
	 * @param readerContext The {@link SourceReaderContext context} for the source reader.
	 * @return A new SourceReader.
	 */
	SourceReader<T, SplitT> createReader(SourceReaderContext readerContext);

	/**
	 * Creates a new SplitEnumerator for this source, starting a new input.
	 *
	 * @param enumContext The {@link SplitEnumeratorContext context} for the split enumerator.
	 * @return A new SplitEnumerator.
	 */
	SplitEnumerator<SplitT, EnumChkT> createEnumerator(SplitEnumeratorContext<SplitT> enumContext);

	/**
	 * Restores an enumerator from a checkpoint.
	 *
	 * @param enumContext The {@link SplitEnumeratorContext context} for the restored split enumerator.
	 * @param checkpoint The checkpoint to restore the SplitEnumerator from.
	 * @return A SplitEnumerator restored from the given checkpoint.
	 */
	SplitEnumerator<SplitT, EnumChkT> restoreEnumerator(
			SplitEnumeratorContext<SplitT> enumContext,
			EnumChkT checkpoint) throws IOException;

	// ------------------------------------------------------------------------
	//  serializers for the metadata
	// ------------------------------------------------------------------------

	/**
	 * Creates a serializer for the source splits. Splits are serialized when sending them
	 * from enumerator to reader, and when checkpointing the reader's current state.
	 *
	 * @return The serializer for the split type.
	 */
	SimpleVersionedSerializer<SplitT> getSplitSerializer();

	/**
	 * Creates the serializer for the {@link SplitEnumerator} checkpoint.
	 * The serializer is used for the result of the {@link SplitEnumerator#snapshotState()}
	 * method.
	 *
	 * @return The serializer for the SplitEnumerator checkpoint.
	 */
	SimpleVersionedSerializer<EnumChkT> getEnumeratorCheckpointSerializer();
}


/**
 * The boundedness of the source: "bounded" for the currently available data (batch style),
 * "continuous unbounded" for a continuous streaming style source.
 */
public enum Boundedness {

	/**
	 * A bounded source processes the data that is currently available and will end after that.
	 *
	 * <p>When a source produces a bounded stream, the runtime may activate additional optimizations
	 * that are suitable only for bounded input. Incorrectly producing unbounded data when the source
	 * is set to produce a bounded stream will often result in programs that do not output any results
	 * and may eventually fail due to runtime errors (out of memory or storage).
	 */
	BOUNDED,

	/**
	 * A continuous unbounded source continuously processes all data as it comes.
	 *
	 * <p>The source may run forever (until the program is terminated) or might actually end at some point,
	 * based on some source-specific conditions. Because that is not transparent to the runtime,
	 * the runtime will use an execution mode for continuous unbounded streams whenever this mode
	 * is chosen.
	 */
	CONTINUOUS_UNBOUNDED
}

SourceSplit

Code Block
languagejava
titleSourceSplit
linenumberstrue
/**
 * An interface for all the Split types to implement.
 */
public interface SourceSplit {

	/**
	 * Get the split id of this source split.
	 * @return id of this source split.
	 */
	String splitId();
}

Anchor
SourceReader
SourceReader
SourceReader

Code Block
languagejava
titleSourceReader
linenumberstrue
/**
 * The interface for a source reader which is responsible for reading the records from
 * the source splits assigned by {@link SplitEnumerator}.
 *
 * @param <T> The type of the record emitted by this source reader.
 * @param <SplitT> The type of the the source splits.
 */
public interface SourceReader<T, SplitT extends SourceSplit> extends Serializable, AutoCloseable {

	/**
	 * Start the reader;
	 */
	void start();

	/**
	 * Poll the next available record into the {@link SourceOutput}.
	 *
	 * <p>The implementation must make sure this method is non-blocking.
	 *
	 * <p>Although the implementation can emit multiple records into the given SourceOutput,
	 * it is recommended not doing so. Instead, emit one record into the SourceOutput
	 * and return a {@link Status#AVAILABLE_NOW} to let the caller thread
	 * know there are more records available.
	 *
	 * @return The {@link Status} of the SourceReader after the method invocation.
	 */
	Status pollNext(SourceOutput<T> sourceOutput) throws Exception;

	/**
	 * Checkpoint on the state of the source.
	 *
	 * @return the state of the source.
	 */
	List<SplitT> snapshotState();

	/**
	 * @return a future that will be completed once there is a record available to poll.
	 */
	CompletableFuture<Void> isAvailable();
Code Block
languagejava
titleSource
linenumberstrue
/**
 * The interface for Source. It acts like a factory class that helps construct
 * the {@link SplitEnumerator} and {@link SourceReader} and corresponding
 * serializers.
 *
 * @param <T>        The type of records produced by the source.
 * @param <SplitT>   The type of splits handled by the source.
 * @param <EnumChkT> The type of the enumerator checkpoints.
 */
public interface Source<T, SplitT extends SourceSplit, EnumChkT> extends Serializable {

	/**
	 * Checks whether the source supports the given boundednessAdds a list of splits for this reader to read.
	 *
	 * <p>Some@param sourcessplits mightThe onlysplits supportassigned eitherby continuous unbounded streams, orthe split enumerator.
	 * bounded streams.
	 /
	void addSplits(List<SplitT> splits);

	/**
	 * @param boundedness The boundedness to check.Handle a source event sent by the {@link SplitEnumerator}
	 *
	 @return* <code>true</code>@param ifsourceEvent the givenevent boundednesssent isby supported, <code>false</code> otherwisethe {@link SplitEnumerator}.
	 */
	booleanvoid supportsBoundednesshandleSourceEvents(BoundednessSourceEvent boundednesssourceEvent);

	/**
	 * The Createsstatus aof newthis reader to read data from the spits it gets assigned.
	 .
	 */
	enum Status {
		/** The next record is available right now. */
		AVAILABLE_NOW,
		/** The readernext startsrecord freshwill andbe doesavailable not have any state to resume.
	 *
	 * @param readerContext The {@link SourceReaderContext context} for the source reader.
	 * @return A new SourceReader.
	 */
	SourceReader<T, SplitT> createReader(SourceReaderContext readerContext);

	/**
	 * Creates a new SplitEnumerator for this source, starting a new input.
	 *
     * @param boundedness The boundedness of this source.
	 * @param enumContext The {@link SplitEnumeratorContext context} for the split enumerator.
	 * @return A new SplitEnumerator.
	 */
	SplitEnumerator<SplitT, EnumChkT> createEnumerator(Boundedness boundedness, SplitEnumeratorContext<SplitT> enumContext);later. */
		AVAILABLE_LATER,
		/** The source reader has completed all the reading work. */
		FINISHED
	}
}

SourceOutput

Code Block
languagejava
titleSourceOutput
linenumberstrue
public interface SourceOutput<E> extends WatermarkOutput {

	void emitRecord(E record);

	void emitRecord(E record, long timestamp);
}

WatermarkOutput

Code Block
languagejava
titleWatermarkOutput
linenumberstrue
/**
 * An output for watermarks. The output accepts watermarks and idleness (inactivity) status.
 */
@Public
public interface WatermarkOutput {

	/**
	 * RestoresEmits anthe enumerator from a checkpointgiven watermark.
	 *
	 * @param<p>Emitting enumContexta Thewatermark {@linkalso SplitEnumeratorContextimplicitly context}marks forthe thestream restored split enumerator.as <i>active</i>, ending
	 * @parampreviously checkpoint The checkpoint to restore the SplitEnumerator from.
	 * @return A SplitEnumerator restored from the given checkpoint.
	 */
	SplitEnumerator<SplitT, EnumChkT> restoreEnumerator(
			SplitEnumeratorContext<SplitT> enumContext,
			EnumChkT checkpoint) throws IOException;

	// ------------------------------------------------------------------------
	//  serializers for the metadata
	// ------------------------------------------------------------------------

	/**
	 * Creates a serializer for the source splits. Splits are serialized when sending them
	 * from enumerator to reader, and when checkpointing the reader's current state.
	 *
	 * @return The serializer for the split type.
	 */
	SimpleVersionedSerializer<SplitT> getSplitSerializer();

	/**
	 * Creates the serializer for the {@link SplitEnumerator} checkpoint.
	 * The serializer is used for the result of the {@link SplitEnumerator#snapshotState()}
	 * method.
	 *
	 * @return The serializer for the SplitEnumerator checkpoint.
	 */
	SimpleVersionedSerializer<EnumChkT> getEnumeratorCheckpointSerializer();
}


/**
 * The boundedness of the source: "bounded" for the currently available data (batch style),
 * "continuous unbounded" for a continuous streaming style source.
 */
public enum Boundedness {

	/**
	 * A bounded source processes the data that is currently available and will end after that.
	 *
	 * <p>When a source produces a bounded stream, the runtime may activate additional optimizations
	 * that are suitable only for bounded input. Incorrectly producing unbounded data when the source
	 * is set to produce a bounded stream will often result in programs that do not output any results
	 * and may eventually fail due to runtime errors (out of memory or storage).
	 */
	BOUNDED,

	/**
	 * A continuous unbounded source continuously processes all data as it comes.
	 *
	 * <p>The source may run forever (until the program is terminated) or might actually end at some point,
	 * based on some source-specific conditions. Because that is not transparent to the runtime,
	 * the runtime will use an execution mode for continuous unbounded streams whenever this mode
	 * is chosen.
	 */
	CONTINUOUS_UNBOUNDED
}

SourceSplit

marked idleness.
	 */
	void emitWatermark(Watermark watermark);

	/**
	 * Marks this output as idle, meaning that downstream operations do not
	 * wait for watermarks from this output.
	 *
	 * <p>An output becomes active again as soon as the next watermark is emitted.
	 */
	void markIdle();
}

Watermark

Code Block
languagejava
titleWatermark
linenumberstrue
/**
 * Watermarks are the progress indicators in the data streams. A watermark signifies
 * that no events with a timestamp smaller or equal to the watermark's time will occur after the
 * water. A watermark with timestamp <i>T</i> indicates that the stream's event time has progressed
 * to time <i>T</i>.
 *
 * <p>Watermarks are created at the sources and propagate through the streams and operators.
 *
 * <p>In some cases a watermark is only a heuristic, meaning some events with a lower timestamp
 * may still follow. In that case, it is up to the logic of the operators to decide what to do
 * with the "late events". Operators can for example ignore these late events, route them to a
 * different stream, or send update to their previously emitted results.
 *
 * <p>When a source reaches the end of the input, it emits a final watermark with timestamp
 * {@code Long.MAX_VALUE}, indicating the "end of time".
 *
 * <p>Note: A stream's time starts with a watermark of {@code Long.MIN_VALUE}. That means that all records
 * in the stream with a timestamp of {@code Long.MIN_VALUE} are immediately late.
 */
@Public
public final class Watermark implements Serializable {

	private static final long serialVersionUID = 1L;

	/** Thread local formatter for stringifying the timestamps. */
	private static final ThreadLocal<SimpleDateFormat> TS_FORMATTER = ThreadLocal.withInitial(
		() -> new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS"));

	// ------------------------------------------------------------------------

	/** The watermark that signifies end-of-event-time. */
	public static final Watermark MAX_WATERMARK = new Watermark(Long.MAX_VALUE);

	// ------------------------------------------------------------------------

	/** The timestamp of the watermark in milliseconds. */
	private final long timestamp;

	/**
	 * Creates a new watermark with the given timestamp in milliseconds.
	 */
	public Watermark(long timestamp) {
		this.timestamp = timestamp;
	}

	/**
	 * Returns the timestamp associated with this Watermark.
	 */
	public long getTimestamp() {
		return timestamp;
	}
Code Block
languagejava
titleSourceSplit
linenumberstrue
/**
 * An interface for all the Split types to implement.
 */
public interface SourceSplit {

	/**
	 * GetFormats the split idtimestamp of this source split watermark, assuming it is a millisecond timestamp.
	 * @returnThe idreturned offormat this source splitis "yyyy-MM-dd HH:mm:ss.SSS".
	 */
	String splitId( */
	public String getFormattedTimestamp() {
		return TS_FORMATTER.get().format(new Date(timestamp));
}

...

	}

	// ------------------------------------------------------------------------

	@Override
	public boolean equals(Object o) {
		return this == o ||
				o != null &&
				o.getClass() == Watermark.class &&
				((Watermark) o).timestamp == this.timestamp;
	}

	@Override
	public int hashCode() {
		return Long.hashCode(timestamp);
	}

	@Override
	public String toString() {
		return "Watermark @ " + timestamp + " (" + getFormattedTimestamp() + ')';
	}
}

SourceReaderContext

Code Block
languagejava
titleSourceReaderContext
linenumberstrue
/**
 * A context for the source reader. It allows the source reader to get the context information and
 * allows the SourceReader to send source event to its split enumerator.
 */
public interface SourceReaderContext {

	/**
	 * Returns the metric group for this parallel subtask.
	 *
	 * @return metric group for this parallel subtask.
	 */
	MetricGroup getMetricGroup();

	/**
	 * Send a source event to the corresponding SplitEnumerator
Code Block
languagejava
titleSourceReader
linenumberstrue
/**
 * The interface for a source reader which is responsible for reading the records from
 * the source splits assigned by {@link SplitEnumerator}.
 *
 * @param <T> The type of the record emitted by this source reader.
 * @param <SplitT> The type of the the source splits.
 */
public interface SourceReader<T, SplitT extends SourceSplit> extends Serializable, AutoCloseable {

	/**
	 * Start the reader;
	 */
	void start();

	/**
	 * Poll the next available record into the {@link SourceOutput}.
	 *
	 * <p>The implementation must make sure this method is non-blocking.
	 *
	 * <p>Although the implementation can emit multiple records into the given SourceOutput,
	 * it is recommended not doing so. Instead, emit one record into the SourceOutput
	 * and return a {@link Status#AVAILABLE_NOW} to let the caller thread
	 * know there are more records available.
	 *
	 * @return@param event The {@linksource Status}event of the SourceReader after the method invocationto send.
	 */
	Statusvoid pollNextsendEventToEnumerator(SourceOutput<T> sourceOutput) throws Exception;

	SourceEvent event);
}

SplitEnumerator

Code Block
languagejava
titleSplitEnumerator
linenumberstrue
/**
	 * Checkpoint on the state ofA interface of a split enumerator responsible for the source.followings:
	 *
	 *1. @returndiscover the statesplits offor the source.
	 */
	List<SplitT> snapshotState();

	/**
	 * @return a future that will be completed once there is a record available to poll.
	 */
	CompletableFuture<Void> isAvailable();{@link SourceReader} to read.
 * 2. assign the splits to the source reader.
 */
public interface SplitEnumerator<SplitT extends SourceSplit, CheckpointT> extends AutoCloseable {

	/**
	 * AddsStart athe list of splits for this reader to readsplit enumerator.
	 *
	 * @param<p>The splitsdefault Thebehavior splits assigned by the split enumeratordoes nothing.
	 */
	void addSplitsstart(List<SplitT> splits);

	/**
	 * HandleHandles athe source event sent byfrom the {@link SplitEnumerator}source reader.
	 *
	 * @param subtaskId the subtask id of the source reader who sent the source event.
	 * @param sourceEvent the source event sent byfrom the {@linksource SplitEnumerator}reader.
	 */
	void handleSourceEventshandleSourceEvent(int subtaskId, SourceEvent sourceEvent);

	/**
	 * The status of this reader.
	 */
	enum Status {
		/** The next record is available right now. */
		AVAILABLE_NOW,
		/** The next record will be available later. */
		AVAILABLE_LATER,
		/** The source reader has completed all the reading work. */
		FINISHED
	}
}

SourceOutput

Code Block
languagejava
titleSourceOutput
linenumberstrue
public interface SourceOutput<E> extends WatermarkOutput {

	void emitRecord(E record);

	void emitRecord(E record, long timestamp);
}

SourceReaderContext

Code Block
languagejava
titleSourceReaderContext
linenumberstrue
/**
 * A context for the source reader. It allows the source reader to get the context information and
 * allows the SourceReader to send source event to its split enumerator.
 */
public interface SourceReaderContext {Add a split back to the split enumerator. It will only happen when a {@link SourceReader} fails
	 * and there are splits assigned to it after the last successful checkpoint.
	 *
	 * @param splits The split to add back to the enumerator for reassignment.
	 * @param subtaskId The id of the subtask to which the returned splits belong.
	 */
	void addSplitsBack(List<SplitT> splits, int subtaskId);

	/**
	 * Add a new source reader with the given subtask ID.
	 *
	 * @param subtaskId the subtask ID of the new source reader.
	 */
	void addReader(int subtaskId);

	/**
	 * ReturnsCheckpoints the metricstate group forof this parallelsplit subtaskenumerator.
	 *
	 * @return metric group for this parallel subtask.
	 */
	MetricGroupCheckpointT getMetricGroupsnapshotState();

	/**
	 * Send a source event to the corresponding SplitEnumerator.
	 *
	 * Called to close the enumerator, in case it holds on to any resources, like threads or
	 * @param event The source event to sendnetwork connections.
	 */
	@Override
	void sendEventToEnumeratorclose(SourceEvent event)) throws IOException;
}

...

SplitEnumeratorContext

Code Block
languagejava
titleSplitEnumeratorSplitEnumeratorContext
linenumberstrue
/**
 * A context class interfacefor ofthe a split enumerator responsible for the followings{@link SplitEnumerator}. This class serves the following purposes:
 * 1. discover Host information necessary for the splits for the {@link SourceReader} to read SplitEnumerator to make split assignment decisions.
 * 2. Accept and track the split assignment from the enumerator.
 * 23. assignProvide thea splitsmanaged tothreading themodel sourceso reader.
 */
public interface SplitEnumerator<SplitT extends SourceSplit, CheckpointT> extends AutoCloseable {

	/**
	 * Start the split enumerator.
	 *
	 * <p>The default behavior does nothing.
	 */
	void startthe split enumerators do not need to create their
 *    own internal threads.
 *
 * @param <SplitT> the type of the splits.
 */
public interface SplitEnumeratorContext<SplitT extends SourceSplit> {

	MetricGroup metricGroup();

	/**
	 * Handles the source event from the source reader Send a source event to a source reader. The source reader is identified by its subtask id.
	 *
	 * @param subtaskId the subtask id of the source reader whoto sentsend thethis sourceevent eventto.
	 * @param sourceEventevent the source event from the source readerto send.
	 * @return a completable future which will be completed when the event is successfully sent.
	 */
	void handleSourceEventsendEventToSourceReader(int subtaskId, SourceEvent sourceEventevent);

	/**
	 * Get the number of subtasks.
	 *
	 * Add a split back to the split enumerator. It will only happen when a {@link SourceReader} fails
	 * and there are splits assigned to it after the last successful checkpoint@return the number of subtasks.
	 */
	int numSubtasks();

	/**
	 * Get the currently registered readers. The mapping is from subtask id to the reader info.
	 *
	 * @return the currently registered readers.
	 */
	Map<Integer, ReaderInfo> registeredReaders();

	/**
	 @param splits The split to add back to the enumerator for reassignment.* Assign the splits.
	 *
	 * @param subtaskId The id of newSplitAssignments the subtasknew to which the returned splits belongsplit assignments to add.
	 */
	void addSplitsBackassignSplits(List<SplitT> splits, int subtaskIdSplitsAssignment<SplitT> newSplitAssignments);

	/**
	 * Add a new source reader with the given subtask ID Invoke the callable and handover the return value to the handler which will be executed
	 * by the source coordinator.
	 *
	 * @param<p>It is subtaskIdimportant theto subtaskmake IDsure ofthat the callable newshould sourcenot reader.modify
	 */
	void addReader(int subtaskId);

	/**
	 * Checkpoints any shared state. Otherwise the statethere ofmight thisbe splitunexpected enumeratorbehavior.
	 */
	CheckpointT snapshotState();

	/**
	 * Called @param callable a callable to close the enumerator, in case it holds on to any resources, like threads or
	 * network connections call.
	 * @param handler a handler that handles the return value of or the exception thrown from the callable.
	 */
	@Override
	<T> void close() throws IOException;
}

SplitEnumeratorContext

callAsync(Callable<T> callable, BiConsumer<T, Throwable> handler);

	/**
	 * Invoke the callable and handover the return value to the handler which will be executed
	 * by the source coordinator.
	 *
	 * <p>It is important to make sure that the callable should not modify
	 * any shared state. Otherwise the there might be unexpected behavior.
	 *
	 * @param callable the callable to call.
	 * @param handler a handler that handles the return value of or the exception thrown from the callable.
	 * @param initialDelay the initial delay of calling the callable.
	 * @param period the period between two invocations of the callable.
	 */
	<T> void callAsync(Callable<T> callable,
					   BiConsumer<T, Throwable> handler,
					   long initialDelay,
					   long period);
}

SplitAssignment

Code Block
languagejava
titleSplitAssignment
linenumberstrue
/**
 * A class containing the splits assignment to the source readers.
 *
 * <p>The assignment is always incremental. In another word, splits in the assignment are simply
 * added to the existing assignment.
 */
public class SplitsAssignment<SplitT extends SourceSplit> {
	private final Map<Integer, List<SplitT>> assignment;

	public SplitsAssignment(Map<Integer, List<SplitT>> assignment) {
		this.assignment = assignment;
	}
Code Block
languagejava
titleSplitEnumeratorContext
linenumberstrue
/**
 * A context class for the {@link SplitEnumerator}. This class serves the following purposes:
 * 1. Host information necessary for the SplitEnumerator to make split assignment decisions.
 * 2. Accept and track the split assignment from the enumerator.
 * 3. Provide a managed threading model so the split enumerators do not need to create their
 *    own internal threads.
 *
 * @param <SplitT> the type of the splits.
 */
public interface SplitEnumeratorContext<SplitT extends SourceSplit> {

	MetricGroup metricGroup();

	/**
	 * Send a source event to a source reader. The source reader is identified by its subtask id.
	 *
	 * @param subtaskId the subtask id of the source reader to send this event to.
	 * @param event the source event to send.
	 * @return a completable future which will be completed when the event is successfully sent.
	 */
	void sendEventToSourceReader(int subtaskId, SourceEvent event);

	/**
	 * Get the number of subtasks @return A mapping from subtask ID to their split assignment.
	 */
	public *Map<Integer, @returnList<SplitT>> the number of subtasks.
	 */
	int numSubtasksassignment() {
		return assignment;
	}

	@Override
	public String toString() {
		return assignment.toString();

		}
}

SourceEvent

Code Block
languagejava
titleSourceEvent
linenumberstrue
/**
	 * GetAn theinterface currentlyfor registeredthe readers.events Thepassed mappingbetween isthe fromSourceReaders subtask id to the reader infoand Enumerators.
	 */
	public *interface @returnSourceEvent theextends currently registered readers.
	 */
	Map<Integer, ReaderInfo> registeredReaders();

	/**
	 * Assign the splits.
	 *
	 * @param newSplitAssignments the new split assignments to add.
	 */
	void assignSplits(SplitsAssignment<SplitT> newSplitAssignments);

	/**
	 * Invoke the callable and handover the return value to the handler which will be executed
	 * by the source coordinator.
	 *
	 * <p>It is important to make sure that the callable should not modify
	 * any shared state. Otherwise the there might be unexpected behavior.
	 *
	 * @param callable a callable to call.
	 * @param handler a handler that handles the return value of or the exception thrown from the callable.
	 */
	<T> void callAsync(Callable<T> callable, BiConsumer<T, Throwable> handler);

	/**
	 * Invoke the callable and handover the return value to the handler which will be executed
	 * by the source coordinator.
	 *
	 * <p>It is important to make sure that the callable should not modify
	 * any shared state. Otherwise the there might be unexpected behavior.
	 *
	 * @param callable the callable to call.
	 * @param handler a handler that handles the return value of or the exception thrown from the callable.
	 * @param initialDelay the initial delay of calling the callable.
	 * @param period the period between two invocations of the callable.
	 */
	<T> void callAsync(Callable<T> callable,
					   BiConsumer<T, Throwable> handler,
					   long initialDelay,
					   long period);
}

SplitAssignment

Code Block
languagejava
titleSplitAssignment
linenumberstrue
/**
 * A class containing the splits assignment to the source readers.
 *
 * <p>The assignment is always incremental. In another word, splits in the assignment are simply
 * added to the existing assignment.
 */
public class SplitsAssignment<SplitT extends SourceSplit> {
	private final Map<Integer, List<SplitT>> assignment;

	public SplitsAssignment(Map<Integer, List<SplitT>> assignment) {
		this.assignment = assignment;
	}

	/**
	 * @return A mapping from subtask ID to their split assignment.
	 */
	public Map<Integer, List<SplitT>> assignment() {
		return assignment;
	}

	@Override
	public String toString() {
		return assignment.toString();
	}
}

SourceEvent

Code Block
languagejava
titleSourceEvent
linenumberstrue
/**
 * An interface for the events passed between the SourceReaders and Enumerators.
 */
public interface SourceEvent extends Serializable {}

StreamExecutionEnvironment

Code Block
languagejava
titleStreamExecutionEnvironment
linenumberstrue
public class StreamExecutionEnvironment {
...
    public <T> DataStream<T> continuousSource(Source<T, ?, ?> source) {...}

    public <T> DataStream<T> continuousSource(Source<T, ?, ?> source, TypeInformation<T> type) {...}

    public <T> DataStream<T> boundedSource(Source<T, ?, ?> source) {...}

    public <T> DataStream<T> boundedSource(Source<T, ?, ?> source, TypeInformation<T> type) {...}
...
}

Public interface from base Source implementation

The following interfaces are high level interfaces that are introduced by the base implementation of Source.

...

Serializable {}

StreamExecutionEnvironment

Code Block
languagejava
titleStreamExecutionEnvironment
linenumberstrue
public class StreamExecutionEnvironment {
...
    public <T> DataStream<T> continuousSource(Source<T, ?, ?> source) {...}

    public <T> DataStream<T> continuousSource(Source<T, ?, ?> source, TypeInformation<T> type) {...}

    public <T> DataStream<T> boundedSource(Source<T, ?, ?> source) {...}

    public <T> DataStream<T> boundedSource(Source<T, ?, ?> source, TypeInformation<T> type) {...}
...
}

Anchor
BaseImplInterface
BaseImplInterface
Public interface from base Source implementation

The following interfaces are high level interfaces that are introduced by the base implementation of Source.

  • SourceReaderBase - The base implementation for SourceReader. It uses the following interfaces.
  • SplitReader - The stateless and thread-less high level reader which is only responsible for reading raw records of type <E> from the assigned splits.
  • SplitChange - The split change to the split reader. Right now there is only one subclass which is SplitAddition.
  • RecordsWithSplitIds - A container class holding the raw records of type <E> read by SplitReader. It allows the SplitReader to fetch and pass the records in batch.
  • RecordEmitter - A class that takes the raw records of type <E> returned by the SplitReader, convert them into the final record type <T> and emit them into the SourceOutput.

SourceReaderBase

Code Block
languagejava
titleSplitReader
linenumberstrue
/**
 * An abstract implementation of {@link SourceReader} which provides some sychronization between
 * the mail box main thread and the SourceReader internal threads. This class allows user to have
 * a SourceReader implementation by just providing the following:
 * <ul>
 *     <li>A {@link SplitReader}.</li>
 *     <li>A {@link RecordEmitter}</li>
 *     <li>The logic to clean up a split state after it is finished.</li>
 *     <li>The logic to get the state from a {@link SourceSplit}.</li>
 *     <li>The logic to restore a {@link SourceSplit} from its state.</li>
 * </ul>
 *
 * @param <E> The rich element type that contains information for split state update or timestamp extraction.
 * @param <T> The final element type to emit.
 * @param <SplitT> the immutable split type.
 * @param <SplitStateT> the mutable type of split state.
 */
public abstract class SourceReaderBase<E, T, SplitT extends SourceSplit, SplitStateT>
		implements SourceReader<T, SplitT> {
	
	// -------------------- Abstract method to allow different implementations ------------------
	/**
	 * Handles the finished splits to clean the state if needed.
	 */
	protected abstract void onSplitFinished(Collection<String> finishedSplitIds);

	/**
	 * When new splits are added to the reader. The initialize the state of the new splits.
	 *
	 * @param split a newly added split.
	 */
	protected abstract SplitStateT initializedState(SplitT split);

	/**
	 * Convert a mutable SplitStateT to immutable SplitT.
	 *
	 * @param splitState splitState.
	 * @return an immutable Split state.
	 */
	protected abstract SplitT toSplitType(String splitId, SplitStateT splitState);
}

...

SplitReader

Code Block
languagejava
titleSplitReader
linenumberstrue
/**
 * An interface used to read from splits. The implementation could either read from a single split or from
 * multiple splits.
 *
 * @param <E> the element type.
 * @param <SplitT> the split type.
 */
public interface SplitReader<E, SplitT extends SourceSplit> {

	/**
	 * Fetch elements into the blocking queue for the given splits. The fetch call could be blocking
	 * but it should get unblocked when {@link #wakeUp()} is invoked. In that case, the implementation
	 * may either decide to return without throwing an exception, or it can just throw an interrupted
	 * exception. In either case, this method should be reentrant, meaning that the next fetch call
	 * should just resume from where the last fetch call was waken up or interrupted.
	 *
	 * @return the Ids of * @return A RecordsWithSplitIds that contains the fetched records grouped by the finishedsplit splitsids.
	 *
	 * @throws InterruptedException when interrupted
	 */
	RecordsWithSplitIds<E> fetch() throws InterruptedException;

	/**
	 * Handle the split changes. This call should be non-blocking.
	 *
	 * @param splitsChanges a queue with split changes that has not been handled by this SplitReader.
	 */
	void handleSplitsChanges(Queue<SplitsChange<SplitT>> splitsChanges);

	/**
	 * Wake up the split reader in case the fetcher thread is blocking in
	 * {@link #fetch()}.
	 */
	void wakeUp();
}

...

Code Block
languagejava
titleRecordEmitter
linenumberstrue
/**
 * Emit a record to the downstream.
 *
 * @param <E> the type of the record emitted by the {@link SplitReader}
 * @param <T> the type of records that are eventually emitted to the {@link SourceOutput}.
 * @param <SplitStateT> the mutable type of split state.
 */
public interface RecordEmitter<E, T, SplitStateT> {

	/**
	 * Process and emit the records to the {@link SourceOutput}. A few recommendations to the implementation typical implementation will do the
	 * followings:
	 *
	 * <ul>
	 *     <li>
	 *         Convert the element emitted by the {@link SplitReader} to the target type taken by the
	 *         {@link SourceOutput}.
	 *     </li>
	 * are following:    <li>
	 *
	 * <ul>
	 * 	<li>The method maybe interrupted in the middle. In that case, the same set of records will be passed
	 * 	to the record emitter again later. The implementation needs to make sure it reades
	 * 	<li>         Extract timestamp from the passed in element and emit the timestamp along with the record.
	 *     </li>
	 *     <li>
	 *         Emit watermarks for the source.
	 *     </li>
	 * </ul>
	 *
	 * @param element The intermediate element read by the SplitReader.
	 * @param output The output to which the final records are emit to.
	 * @param splitState The state of the split where the given element was from.
	 */
	void emitRecord(E element, SourceOutput<T> output, SplitStateT splitState) throws Exception;
}

...