You are viewing an old version of this page. View the current version.

Compare with Current View Page History

« Previous Version 16 Next »

Status

Current state:  Under Discussion

Discussion thread:  https://lists.apache.org/thread.html/rf09dfeeaf35da5ee98afe559b5a6e955c9f03ade0262727f6b5c4c1e%40%3Cdev.flink.apache.org%3E

JIRA:

Released: 

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

Motivation

As discussed in FLIP-131, Flink will deprecate the DataSet API in favor of DataStream API and Table API. Users should be able to use DataStream API to write jobs that support both bounded and unbounded execution modes. However Flink does not provide a sink API to guarantee the exactly once semantics in both bounded and unbounded scenarios, which blocks the unification. 

So we want to introduce a new unified sink API which could let the user develop sink once and run it everywhere. Specifically Flink allows the user to 

  1. Choose the different SDK(SQL/Table/DataStream)
  2. Choose the different execution mode(Batch/Stream) according to the scenarios(bounded/unbounded) 

We hope these things(SDK/Execution mode) are transparent to the sink API.

The document includes three parts: The first part describes the semantics the unified sink API should support. According to the first part the second part proposes two versions of unified sink API. In the last part we introduce two open questions related to the API.

Semantics

This section would talk about which semantics the unified sink API would support. This section includes two parts. The first part describes how the sink API supports the exactly once semantics. The second part mainly talks about that drain and snapshot are two general semantics, which are valid for both bounded and unbounded scenarios. 

Consistency

This section tries to figure out the sink developer would be responsible for which part and Flink would be responsible for which part in terms of ensuring consistency. After this we could know what semantics the sink API should have to support the exact once semantics.

From a high-level perspective, we can always abstract a job by reading data from an external system and then writing the processed data to another external system. Sink's responsibility is to write data to external systems. In general, writing data to external systems mainly answers four questions What & How & When & Where to commit the data. Following we would give some explanation about these four questions. After that, we could answer the question asked at the beginning of this part. 

What to Commit. The data generated by the job goes through two stages before it could be committed to the external system. The first stage is the preparation. Normally the data produced by the job could not be written/committed to the external system immediately thanks to some conditions that have not been met. For example, in the `StreamingFileSink` the data is first written to an in-progress-file before it could be committed to the filesystem. The second stage is a commitment. In this stage, the data could be committed to an external system. The data in the second stage is called What to commit.

How to Commit. When data is ready to be committed we need a system-specific way to commit the data to the external. Sometimes the sink needs to commit the data by modifying the file’s name. Sometimes the sink needs to submit some meta infos to let data be visible to the end users.

When to Commit. From the users’ perspective, one of the most important requirements is correctness. Normally the user wants the data generated by the job to be committed to the external system once and only once. If we commit the data at the wrong time there might be duplicated data. For example current Flink through restarting the job when there is failover. This always replays some elements.

Where to Commit. This is about how many resources that we need to guarantee exactly once when time is ready. For example If we assume that commit operation will happen at where the committable data is produced the lifecycle of the component that is responsible for producing the committable data would last until the end of the commit operation.

Actually When & Where are all about how to guarantee the exactly once semantics. Now Flink exposes the internal implementation to the sink developer through two interfaces `CheckpointedFunction` & `CheckpointListener`. All the exactly once sinks are coupled with the internal exactly once implementation through implementing the two interfaces to guarantee the exactly once semantics. 

For supporting the bounded scenario we find that the mechanism of `CheckpointedFunction` & `CheckpointListener` is not inline with the bounded scenario. In a bounded scenario Flink should guarantee the exact once semantics:

  1. Even if there is only one resource.(Where)
  2. Even if there is no normal checkpoint at all.(When)

It means current streaming style sink implementation is not suitable for the bounded scenario. Flink must introduce some new mechanism for the bounded scenario. However, we might not couple the sink API to another new internal mechanism which guarantees consistency in the bounded scenario. These internal mechanisms should be decoupled with the sink’s API.

In summary the sink API should be responsible for producing What to commit & providing How to commit. Flink should be responsible for guaranteeing the exact semantics. Flink could “optimize” When & Where to commit according to the execution mode and these optimizations should be transparent to the sink developer.

Sink API

From above analysis we could know that there are four semantics the sink API should support.

  1. What to commit
  2. How to commit
  3. SnapshotState
  4. Drain(Flush)

Following we propose two versions of the sink APIs. There is no difference on how to express the four semantics between the two versions: 

  1. `Sink` is responsible for creating the `Writer` and `Committer`
  2. The `Writer` is responsible for writing data and handling any potential tmp area used to write yet unstaged data, e.g. in-progress files. As soon as some data is ready to commit, they (or metadata pointing to where the actual data is staged) are shipped to an operator who knows when to commit them.
  3. The `Writer` also responds to the progress notification: Snapshot and Flush.
  4. The `Committer` knows how to commit the data to the destination storage system.

The only difference between the two versions is how to expose the state to the user. The cons & pros are following. 

First Alternative

  1. PROS
    1. This alternative has the advantage of not exposing anything about internal state handling to the user. We require the sink to provide us with the state to be persisted and it is up to the underlying runtime to handle them. For example, instead of using union state for the shared state, we could leverage a potential operator coordinator.
    2. We do not expose complex classes such as StateDescriptors, TypeSerializer... (only a simplified version: SimpleVersionedSerializer)
    3. The intent of the #snapshotState is cleaner. The signature tells the user already that it expects/allows some state out of this method.
  2. CONS
    1. It exposes more methods
    2. We get the state to snapshot via a return value from a method, the state must fit into memory. Moreover we must put the whole state into the state backend on each checkpoint instead of putting only the updated values during elements processing
Sink
/**
* The sink interface acts like a factory to create the {@link Writer} and {@link Committer}.
*
* @param <IN>      The type of the sink's input
* @param <CommT>   The type of the committable data
* @param <WriterS> The type of the writer's state
* @param <SharedS> The type of the writer's shared state
*/
public interface Sink<IN, CommT, WriterS, SharedS> extends Serializable {

	/**
	* Create a {@link Writer}.
	* @param context the runtime context
	* @return A new sink writer.
	*/
	Writer<IN, CommT, WriterS, SharedS> createWriter(InitContext context);

	/**
	* Restore a {@link Writer} from the state.
	* @param context the runtime context
	* @param state the previous writers' state
	* @param share the previous writers' shared state
	* @return A sink writer
	*/
	Writer<IN, CommT, WriterS, SharedS> restoreWriter(InitContext context, List<WriterS> state, List<SharedS> share);

	/**
	* Create a {@link Committer}.
	* @return A new sink committer
	*/
	Committer<CommT> createCommitter();

	/**
	* @return a committable serializer
	*/
	SimpleVersionedSerializer<CommT> getCommittableSerializer();

	/**
	* @return a writer's state serializer
	*/
	SimpleVersionedSerializer<WriterS> getStateSerializer();

	/**
	* @return a writer's shared state serializer
	*/
	SimpleVersionedSerializer<SharedS> getSharedStateSerializer();

	/**
	* Providing the runtime information.
	*/
	interface InitContext {

		int getSubtaskId();

		int getAttemptID();

		MetricGroup metricGroup();
	}
}


Writer
/**
* The Writer is responsible for writing data and handling any potential tmp area used to write yet un-staged data, e.g. in-progress files. 
* As soon as some data is ready to commit, they (or metadata pointing to where the actual data is staged) are shipped to an operator who knows when to commit them.
*
* @param <IN> 			The type of the writer's input
* @param <CommT> 		The type of the committable data
* @param <StateT> 		The type of the writer's state
* @param <SharedStateT> The type of the writer's shared state
*/
public interface Writer<IN, CommT, StateT, SharedStateT> {

	/**
	* Add an element to the writer.
	* @param element The input record.
	* @param ctx     The additional information about the input record
	* @param output  The committable data could be shipped to the committer by this.
	* @throws Exception
	*/
	void write(IN element, Context ctx, WriterOutput<CommT> output) throws Exception;

	/**
	* Snapshot the writer's state.
	* @param output  The committable data could be shipped to the committer by this.
	* @return The writer's state.
	*/
	List<StateT> snapshotState(WriterOutput<CommT> output) throws Exception;

	/**
	* Snapshot the writer's shared state.
	* @return The writer's shared state.
	*/
	SharedStateT snapshotSharedState() throws Exception;

	/**
	* Flush all the data which is un-staged to the committer.
	* @param output
	*/
	void flush(WriterOutput<CommT> output) throws Exception;

	interface Context {

		long currentProcessingTime();

		long currentWatermark();

		Long timestamp();
	}
}


Committer
/**
* This interface knows how to commit the data to the external system.
* 
* @param <CommT> The type of the committable data.
*/
public interface Committer<CommT> {
	void commit(CommT committable) throws Exception;
}


WriterOutput
/**
* The {@link Writer} uses this interface to send the committable data to the operator who knows when to commit to the external system.
*
* @param <CommT> The type of the committable data.
*/
public interface WriterOutput<CommT> {

	/**
	* Send the committable data to the operator who knows when to commit.
	*
	* @param committable The data that is ready for committing.
	*/
	void sendToCommit(CommT committable) throws IOException;
}


Open Questions

There are still two open questions related to the unified sink API

How does the sink API support to write to the Hive?  

In general HiveSink needs three steps before committing the data to the Hive:

  1. The first step is writing the data to the FileSystem. 
  2. The second step is computing which directories could be committed 
  3. The third step is commit the directories the HMS

One of the special requirements of Hive is that the data partitioning key of the first two steps might be different.  For example the first step needs partition by the order.id and the second step needs to partition by the order.created_at. It is because it would introduce data skew if we use the same key to partition. The unified sink API uses the `Writer` to produce the committable data. It implies that there would be only one partition key. So this api does not meet the above scenario directly.

Is the sink an operator or a topology?

The scenario could be more complicated. For example, some users want to merge the files in one bucket before committing to the HMS. Where to place this logic? Do we want to put all these logic into one operator or a sink topology?

Compatibility, Deprecation, and Migration Plan

  • We does not change the current streaming and batch style sink API. So there is no compatibility issue to the existing sink implementation.
  • In the long run we might need to deprecate the old streaming and batch style sink API. 
  • At first we plan to migrate the StreamingFileSink to this new api.

Rejected Alternatives

The only difference between the rejected version and accepted version is how to expose the state to the user. The accepted version could give the framework a greater opportunity to optimize the state.

Sink
/**
* The sink interface acts like a factory to create the {@link Writer} and {@link Committer}.
*
* @param <T>       The type of the sink's input.
* @param <CommT>   The type of the committable data.
*/
public interface Sink<T, CommT> extends Serializable {

    /**
	* Create a {@link Writer}.
	* @param context the runtime context
	* @return A new sink writer.
	*/
	Writer<T, CommT> createWriter(InitialContext context) throws Exception;

	/**
	* Create a {@link Committer}.
	* @return A new sink committer
	*/
	Committer<CommT> createCommitter();

	/**
	* @return a committable serializer
	*/
	SimpleVersionedSerializer<CommT> getCommittableSerializer();

    /**
	* Providing the runtime information.
	*/
	interface InitialContext {

		boolean isRestored();

		int getSubtaskIndex();

		OperatorStateStore getOperatorStateStore();

		MetricGroup metricGroup();

	}
}
Writer
/**
* The Writer is responsible for writing data and handling any potential tmp area used to write yet un-staged data, e.g. in-progress files.
* As soon as some data is ready to commit, they (or metadata pointing to where the actual data is staged) are shipped to an operator who knows when to commit them.
*
* @param <T> 		The type of writer's input
* @param <CommT> 	The type of the committable data.
*/
public interface Writer<T, CommT> extends AutoCloseable {

	/**
	* Add an element to the writer.
	* @param t          The input record
	* @param context    The additional information about the input record
	* @param output     The committable data could be shipped to the committer by this
	* @throws Exception
	*/
	void write(T t, Context context, WriterOutput<CommT> output) throws Exception;

	/**
	* Snapshot the state of the writer.
	* @param output The committable data could be shipped to the committer by this.
	*/
	void snapshotState(WriterOutput<CommT> output) throws Exception;

	/**
	* Flush all the data which is un-staged to the committer.
	*/
	void flush(WriterOutput<CommT> output) throws IOException;

    interface Context {

		long currentProcessingTime();

		long currentWatermark();
		
		Long timestamp();
	}
}


  • No labels