Data Sinks #
This page describes Flink’s Data Sink API and the concepts and architecture behind it. Read this, if you are interested in how data sinks in Flink work, or if you want to implement a new Data Sink.
If you are looking for pre-defined sink connectors, please check the Connector Docs.
The Data Sink API #
This section describes the major interfaces of the new Sink API introduced in FLIP-191 and FLIP-372, and provides tips to the developers on the Sink development.
Sink #
The Sink API is a factory style interface to create the SinkWriter to write the data.
The Sink implementations should be serializable as the Sink instances are serialized and uploaded to the Flink cluster at runtime.
Use the Sink #
We can add a Sink
to DataStream
by calling DataStream.sinkTo(Sink)
method. For example,
final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
Source mySource = new MySource(...);
DataStream<Integer> stream = env.fromSource(
mySource,
WatermarkStrategy.noWatermarks(),
"MySourceName");
Sink mySink = new MySink(...);
stream.sinkTo(mySink);
...
SinkWriter #
The core SinkWriter API is responsible for writing data to downstream system.
The SinkWriter
API only has three methods:
- write(InputT element, Context context): Adds an element to the writer.
- flush(boolean endOfInput): Called on checkpoint or end of the input, setting this flag causes the writer to flush all pending data for
at-least-once
. To archiveexactly-once
semantic, the writer should implement the SupportsCommitter interface. - writeWatermark(Watermark watermark): Adds a watermark to the writer.
Please check the Java doc of the class for more details.
Advanced Sink API #
SupportsWriterState #
The SupportsWriterState interface is used to indicate that the sink supports writer state, which means that the sink can be recovered from a failure.
The SupportsWriterState
interface requires the SinkWriter
to implement the {{ gh_link file=“flink-core/src/main/java/org/apache/flink/api/connector/sink2/StatefulSinkWriter.java” name=“StatefulSinkWriter” >}} interface.
SupportsCommitter #
The SupportsCommitter interface is used to indicate that the sink supports exactly-once semantics using a two-phase commit protocol.
The Sink
consists of a CommittingSinkWriter
that performs the precommits and a Committer
that actually commits the data. To facilitate the separation, the CommittingSinkWriter
creates committables
on checkpoint or end of input and the sends it to the Committer
.
The Sink
needs to be serializable. All configuration should be validated eagerly. The respective sink writers and committers are transient and will only be created in the subtasks on the TaskManagers.
Custom sink topology #
For advanced developers, they may want to specify their own sink operator topology(A structure composed of a series of operators), such as collecting committables
to one subtask and processing them together, or performing operations such as merging small files after Committer
. Flink provides the following interfaces to allow expert users to customize the sink operator topology.
SupportsPreWriteTopology #
SupportsPreWriteTopology
interface Allows expert users to implement a custom operator topology before SinkWriter
, which can be used to process or redistribute the input data. For example, sending data of the same partition to the same SinkWriter of Kafka or Iceberg.
The following figure shows the operator topology of using SupportsPreWriteTopology :

In the figure above, user add a PrePartition
and PostPartition
operator in the SupportsPreWriteTopology
topology, and redistribute the input data to the SinkWriter
.
SupportsPreCommitTopology #
SupportsPreCommitTopology
interface Allows expert users to implement a custom operator topology after SinkWriter
and before Committer
, which can be used to process or redistribute the commit messages.
The following figure shows the operator topology of using SupportsPreCommitTopology :

In the figure above, user add a CollectCommit
operator in the SupportsPreCommitTopology
topology, and collect all the commit messages from the SinkWriter
to one subtask, then send to the Committer
to process them centrally, this can reduce the number of interactions with the server.
Please note that the parallelism has only been modified here for display purposes. In fact, the parallelism can be set by user.
SupportsPostCommitTopology #
SupportsPostCommitTopology
interface Allows expert users to implement a custom operator topology after Committer
.
The following figure shows the operator topology of using SupportsPostCommitTopology :

In the figure above, users add a MergeFile
operator in the SupportsPostCommitTopology
topology, the MergeFile
operator can merge small files into larger files to speed up the reading of file system.