Versions Compared

Key

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

...

  • 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.

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

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
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 fora new reader to read data from the {@link SplitEnumerator} checkpointspits it gets assigned.
	 * The serializerreader isstarts usedfresh forand thedoes resultnot ofhave theany {@link SplitEnumerator#snapshotState()}
	 * methodstate to resume.
	 *
	 * @return@param readerContext The serializer {@link SourceReaderContext context} for the SplitEnumeratorsource checkpointreader.
	 */
	SimpleVersionedSerializer<EnumChkT> getEnumeratorCheckpointSerializer();
}


 @return A new SourceReader.
	 */
	SourceReader<T, SplitT> createReader(SourceReaderContext readerContext);

	/**
	 * Creates a Thenew boundednessSplitEnumerator offor thethis source: "bounded" for the currently available data (batch style),
 * "continuous unbounded" for a continuous streaming style source.
 */
public enum Boundedness {, 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);

	/**
	 * ARestores boundedan sourceenumerator processesfrom the data that is currently available and will end after thata checkpoint.
	 *
	 * <p>When@param aenumContext sourceThe produces{@link aSplitEnumeratorContext boundedcontext} stream,for the runtimerestored may activate additional optimizationssplit enumerator.
	 * 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 {@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();

	/**
	 * GetCreates the splitserializer idfor ofthe this{@link sourceSplitEnumerator} splitcheckpoint.
	 * @returnThe idserializer ofis this source splitused for the result of the {@link SplitEnumerator#snapshotState()}
	 * method.
	 *
	 * @return The serializer for the SplitEnumerator checkpoint.
	 */
	StringSimpleVersionedSerializer<EnumChkT> splitIdgetEnumeratorCheckpointSerializer();
}

...

Code Block
languagejava
titleSourceReader
linenumberstrue



/**
 * The interfaceboundedness forof athe source: reader"bounded" whichfor isthe responsiblecurrently foravailable readingdata the records from(batch style),
 * the"continuous sourceunbounded" splitsfor assigneda continuous bystreaming {@linkstyle SplitEnumerator}source.
 */
public enum Boundedness {

	/**
	 @param* <T>A Thebounded typesource ofprocesses the recorddata emittedthat byis thiscurrently sourceavailable reader.
and *will @paramend <SplitT> The type of the the source splits.
 */
public interface SourceReader<T, SplitT extends SourceSplit> extends Serializable, AutoCloseable {

	/**after that.
	 *
	 * <p>When a source produces a bounded stream, the runtime may activate additional optimizations
	 * Startthat the reader;
	 */
	void start();

	/**
	 * Poll the next available record into the {@link SourceOutput}.are suitable only for bounded input. Incorrectly producing unbounded data when the source
	 *
	 *is <p>Theset implementationto mustproduce makea surebounded thisstream methodwill is non-blocking.
	 *
	 * <p>Although the implementation can emit multiple records into the given SourceOutput,often result in programs that do not output any results
	 * itand ismay recommendedeventually notfail doingdue so.to Instead,runtime emiterrors one(out recordof intomemory theor SourceOutputstorage).
	 * and return a {@link Status#AVAILABLE_NOW} to let the caller thread
	 * know there are more records available/
	BOUNDED,

	/**
	 * A continuous unbounded source continuously processes all data as it comes.
	 *
	 * <p>The @returnsource Themay {@linkrun Status}forever of(until the SourceReader after the method invocation. program is terminated) or might actually end at some point,
	 */
	Status pollNext(SourceOutput<T> sourceOutput) throws Exception;

	/**
	 * Checkpoint on the state of the source.
	 *
	 * @return the state of the source 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.
	 */
	List<SplitT> snapshotState();

	CONTINUOUS_UNBOUNDED
}

SourceSplit

Code Block
languagejava
titleSourceSplit
linenumberstrue
/**
	 * @returnAn ainterface futurefor thatall willthe beSplit completedtypes once there is a record available to pollto implement.
	 */
	CompletableFuture<Void> isAvailable();public interface SourceSplit {

	/**
	 * Get Addsthe asplit listid of splits for this readersource to readsplit.
	 *
	 *@return @paramid splitsof Thethis splits assigned by the split enumeratorsource split.
	 */
	voidString addSplitssplitId(List<SplitT> splits);
}

Anchor
SourceReader
SourceReader
SourceReader

Code Block
languagejava
titleSourceReader
linenumberstrue

	/**
	 * The interface Handlefor a source eventreader sentwhich byis the {@link SplitEnumerator}
	 *
	responsible for reading the records from
 * @paramthe sourceEventsource thesplits event sentassigned by the {@link SplitEnumerator}.
	 */
	void handleSourceEvents(SourceEvent sourceEvent);

	/**
	 ** @param <T> The statustype of the record emitted by this source reader.
	 */
	enum Status {
		/**@param <SplitT> The nexttype recordof isthe availablethe rightsource nowsplits.
 */
		AVAILABLE_NOW,
		/** The next record will be available later. */
		AVAILABLE_LATER,
		/** The source reader has completed all the reading work.public interface SourceReader<T, SplitT extends SourceSplit> extends Serializable, AutoCloseable {

	/**
	 * Start the reader;
	 */
		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 {

	/**
	 * Returns the metric group for this parallel subtaskvoid 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 metric group for this parallel subtask The {@link Status} of the SourceReader after the method invocation.
	 */
	MetricGroupStatus getMetricGrouppollNext(SourceOutput<T> sourceOutput) throws Exception;

	/**
	 * SendCheckpoint aon sourcethe eventstate toof the corresponding SplitEnumeratorsource.
	 *
	 * @param@return eventthe Thestate sourceof eventthe to sendsource.
	 */
	voidList<SplitT> sendEventToEnumeratorsnapshotState(SourceEvent event);
}

SplitEnumerator

Code Block
languagejava
titleSplitEnumerator
linenumberstrue
/**
 * A interface of a split enumerator responsible for the followings:
 * 1. discover the

	/**
	 * @return a future that will be completed once there is a record available to poll.
	 */
	CompletableFuture<Void> isAvailable();

	/**
	 * Adds a list of splits for the {@link SourceReader}this reader to read.
	 * 2. assign the splits to
	 * @param splits The splits assigned by the sourcesplit readerenumerator.
	 */
public interface SplitEnumerator<SplitT extends SourceSplit, CheckpointT> extends AutoCloseable {	void addSplits(List<SplitT> splits);

	/**
	 * StartHandle a source event sent by the split enumerator.{@link SplitEnumerator}
	 *
	 * <p>The default behavior does nothing @param sourceEvent the event sent by the {@link SplitEnumerator}.
	 */
	void starthandleSourceEvents(SourceEvent sourceEvent);

	/**
	 * HandlesThe thestatus sourceof event from the source this reader.
	 */
	enum Status {
		/** @paramThe subtaskIdnext therecord subtaskis idavailable ofright the source reader who sent the source event.
	 * @param sourceEvent the source event from the source reader.
	now. */
		AVAILABLE_NOW,
		/** The next record will be available later. */
	void handleSourceEvent(int subtaskId, SourceEvent sourceEvent);

		AVAILABLE_LATER,
		/**
	 *The Addsource areader splithas backcompleted toall the splitreading 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);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 {

	/**
	 * AddReturns athe newmetric sourcegroup readerfor withthis the givenparallel subtask ID.
	 *
	 * @param@return subtaskIdmetric thegroup subtaskfor IDthis of the new source readerparallel subtask.
	 */
	voidMetricGroup addReadergetMetricGroup(int subtaskId);

	/**
	 * CheckpointsSend a thesource stateevent ofto thisthe splitcorresponding enumeratorSplitEnumerator.
	 */
	CheckpointT snapshotState();

	/**
	 *@param Calledevent toThe closesource theevent enumerator, in case it holds on to any resources, like threads or
	 * network connections.to send.
	 */
	@Override
	void closesendEventToEnumerator() throws IOExceptionSourceEvent event);
}

...

SplitEnumerator

Code Block
languagejava
titleSplitEnumeratorContextSplitEnumerator
linenumberstrue
/**
 * A contextinterface classof fora thesplit {@link SplitEnumerator}. This class servesenumerator responsible for the following purposesfollowings:
 * 1. Hostdiscover informationthe necessarysplits for the SplitEnumerator{@link SourceReader} to make split assignment decisionsread.
 * 2. Accept and trackassign the splitsplits assignmentto fromthe thesource enumeratorreader.
 */
public 3.interface ProvideSplitEnumerator<SplitT aextends managedSourceSplit, threadingCheckpointT> modelextends so the split enumerators do not need to create their
AutoCloseable {

	/**
	 * Start the split own internal threadsenumerator.
	 *
	 * @param<p>The <SplitT>default thebehavior type of the splitsdoes nothing.
	 */
public interface SplitEnumeratorContext<SplitT extends SourceSplit> {

	MetricGroup metricGroup	void start();

	/**
	 * SendHandles athe source event tofrom athe source reader. The source reader is identified by its subtask id.
	 *
	 * @param subtaskId the subtask id of the source reader towho sent sendthe thissource event to.
	 * @param eventsourceEvent the source event to send.
	 * @return a completable future which will be completed whenfrom the event is successfully sentsource reader.
	 */
	void sendEventToSourceReaderhandleSourceEvent(int subtaskId, SourceEvent eventsourceEvent);

	/**
	 * GetAdd a split back to the numbersplit of subtasksenumerator.
	 *
	It *will @returnonly thehappen numberwhen of subtasks.
	 */
	int numSubtasks();

	/**a {@link SourceReader} fails
	 * Getand thethere currentlyare registeredsplits readers.assigned Theto mappingit is from subtask idafter the last successful checkpoint.
	 *
	 * @param splits The split to add back to the enumerator readerfor inforeassignment.
	 *
	 * @return @param subtaskId The id of the subtask to which the currentlyreturned registeredsplits readersbelong.
	 */
	Map<Integervoid addSplitsBack(List<SplitT> splits, ReaderInfo>int registeredReaders(subtaskId);

	/**
	 * Assign Add a new source reader with the splitsgiven subtask ID.
	 *
	 * @param newSplitAssignmentssubtaskId the subtask newID of splitthe assignmentsnew tosource addreader.
	 */
	void assignSplitsaddReader(SplitsAssignment<SplitT>int newSplitAssignmentssubtaskId);

	/**
	 * InvokeCheckpoints the callablestate andof handoverthis the return value to the handler which will be executedsplit enumerator.
	 * by the source coordinator.
	 /
	CheckpointT snapshotState();

	/**
	 * <p>It is importantCalled to makeclose surethe thatenumerator, thein callablecase shouldit notholds modify
	on *to any sharedresources, state.like Otherwisethreads theor
	 there* might be unexpected behaviornetwork connections.
	 */
	@Override
	void *close() throws IOException;
}

SplitEnumeratorContext

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@param callable a callable to call.
	 * @param handler a handler that handles the return value of or the exception thrown from the callableenumerator.
	 */
	<T> void callAsync(Callable<T> callable, BiConsumer<T, Throwable> handler 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();

	/**
	 * 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.

...

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

	/**
	 * 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.

  • 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 A RecordsWithSplitIds that contains the fetched Idsrecords grouped ofby 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 A 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>
	 *     <li>
	 * are following:
	 *
	 * <ul>
	 * 	<li>The method maybe interrupted in the middle. In that case, the same set of records will be passed        Extract timestamp from the passed in element and emit the timestamp along with the record.
	 *     </li>
	 * 	to the record emitter again later. The implementation needs to make sure it reades    <li>
	 *         Emit watermarks for the source.
	 * 	<li>     </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;
}

...