This connector provides a Sink that writes partitioned files to filesystems
supported by the Flink FileSystem
abstraction.
The streaming file sink writes incoming data into buckets. Given that the incoming streams can be unbounded, data in each bucket are organized into part files of finite size. The bucketing behaviour is fully configurable with a default time-based bucketing where we start writing a new bucket every hour. This means that each resulting bucket will contain files with records received during 1 hour intervals from the stream.
Data within the bucket directories are split into part files. Each bucket will contain at least one part file for each subtask of the sink that has received data for that bucket. Additional part files will be created according to the configurable rolling policy. The default policy rolls part files based on size, a timeout that specifies the maximum duration for which a file can be open, and a maximum inactivity timeout after which the file is closed.
The StreamingFileSink
supports both row-wise and bulk encoding formats, such as Apache Parquet.
These two variants come with their respective builders that can be created with the following static methods:
StreamingFileSink.forRowFormat(basePath, rowEncoder)
StreamingFileSink.forBulkFormat(basePath, bulkWriterFactory)
When creating either a row or a bulk encoded sink we have to specify the base path where the buckets will be stored and the encoding logic for our data.
Please check out the JavaDoc for StreamingFileSink for all the configuration options and more documentation about the implementation of the different data formats.
Row-encoded formats need to specify an Encoder that is used for serializing individual rows to the OutputStream
of the in-progress part files.
In addition to the bucket assigner the RowFormatBuilder allows the user to specify:
Basic usage for writing String elements thus looks like this:
This example creates a simple sink that assigns records to the default one hour time buckets. It also specifies a rolling policy that rolls the in-progress part file on either of the following 3 conditions:
Bulk-encoded sinks are created similarly to the row-encoded ones but here instead of
specifying an Encoder
we have to specify BulkWriter.Factory.
The BulkWriter
logic defines how new elements added, flushed and how the bulk of records
are finalized for further encoding purposes.
Flink comes with four built-in BulkWriter factories:
Flink contains built in convenience methods for creating Parquet writer factories for Avro data. These methods and their associated documentation can be found in the ParquetAvroWriters class.
For writing to other Parquet compatible data formats, users need to create the ParquetWriterFactory with a custom implementation of the ParquetBuilder interface.
To use the Parquet bulk encoder in your application you need to add the following dependency:
A StreamingFileSink that writes Avro data to Parquet format can be created like this:
Flink also provides built-in support for writing data into Avro files. A list of convenience methods to create Avro writer factories and their associated documentation can be found in the AvroWriters class.
To use the Avro writers in your application you need to add the following dependency:
A StreamingFileSink that writes data to Avro files can be created like this:
For creating customized Avro writers, e.g. enabling compression, users need to create the AvroWriterFactory
with a custom implementation of the AvroBuilder interface:
To enable the data to be bulk encoded in ORC format, Flink offers OrcBulkWriterFactory which takes a concrete implementation of Vectorizer.
Like any other columnar format that encodes data in bulk fashion, Flink’s OrcBulkWriter
writes the input elements in batches. It uses
ORC’s VectorizedRowBatch
to achieve this.
Since the input element has to be transformed to a VectorizedRowBatch
, users have to extend the abstract Vectorizer
class and override the vectorize(T element, VectorizedRowBatch batch)
method. As you can see, the method provides an
instance of VectorizedRowBatch
to be used directly by the users so users just have to write the logic to transform the
input element
to ColumnVectors
and set them in the provided VectorizedRowBatch
instance.
For example, if the input element is of type Person
which looks like:
Then a child implementation to convert the element of type Person
and set them in the VectorizedRowBatch
can be like:
To use the ORC bulk encoder in an application, users need to add the following dependency:
And then a StreamingFileSink
that writes data in ORC format can be created like this:
OrcBulkWriterFactory can also take Hadoop Configuration
and Properties
so that a custom Hadoop configuration and ORC
writer properties can be provided.
The complete list of ORC writer properties can be found here.
Users who want to add user metadata to the ORC files can do so by calling addUserMetadata(...)
inside the overriding
vectorize(...)
method.
To use the SequenceFile bulk encoder in your application you need to add the following dependency:
A simple SequenceFile writer can be created like this:
The SequenceFileWriterFactory supports additional constructor parameters to specify compression settings.
The bucketing logic defines how the data will be structured into subdirectories inside the base output directory.
Both row and bulk formats (see File Formats) use the DateTimeBucketAssigner as the default assigner.
By default the DateTimeBucketAssigner
creates hourly buckets based on the system default timezone
with the following format: yyyy-MM-dd--HH
. Both the date format (i.e. bucket size) and timezone can be
configured manually.
We can specify a custom BucketAssigner by calling .withBucketAssigner(assigner)
on the format builders.
Flink comes with two built in BucketAssigners:
The RollingPolicy defines when a given in-progress part file will be closed and moved to the pending and later to finished state. Part files in the “finished” state are the ones that are ready for viewing and are guaranteed to contain valid data that will not be reverted in case of failure. The Rolling Policy in combination with the checkpointing interval (pending files become finished on the next checkpoint) control how quickly part files become available for downstream readers and also the size and number of these parts.
Flink comes with two built-in RollingPolicies:
In order to use the output of the StreamingFileSink
in downstream systems, we need to understand the naming and lifecycle of the output files produced.
Part files can be in one of three states:
Only finished files are safe to read by downstream systems as those are guaranteed to not be modified later.
Each writer subtask will have a single in-progress part file at any given time for every active bucket, but there can be several pending and finished files.
Part file example
To better understand the lifecycle of these files let’s look at a simple example with 2 sink subtasks:
└── 2019-08-25--12
├── part-0-0.inprogress.bd053eb0-5ecf-4c85-8433-9eff486ac334
└── part-1-0.inprogress.ea65a428-a1d0-4a0b-bbc5-7a436a75e575
When the part file part-1-0
is rolled (let’s say it becomes too large), it becomes pending but it is not renamed. The sink then opens a new part file: part-1-1
:
└── 2019-08-25--12
├── part-0-0.inprogress.bd053eb0-5ecf-4c85-8433-9eff486ac334
├── part-1-0.inprogress.ea65a428-a1d0-4a0b-bbc5-7a436a75e575
└── part-1-1.inprogress.bc279efe-b16f-47d8-b828-00ef6e2fbd11
As part-1-0
is now pending completion, after the next successful checkpoint, it is finalized:
└── 2019-08-25--12
├── part-0-0.inprogress.bd053eb0-5ecf-4c85-8433-9eff486ac334
├── part-1-0
└── part-1-1.inprogress.bc279efe-b16f-47d8-b828-00ef6e2fbd11
New buckets are created as dictated by the bucketing policy, and this doesn’t affect currently in-progress files:
└── 2019-08-25--12
├── part-0-0.inprogress.bd053eb0-5ecf-4c85-8433-9eff486ac334
├── part-1-0
└── part-1-1.inprogress.bc279efe-b16f-47d8-b828-00ef6e2fbd11
└── 2019-08-25--13
└── part-0-2.inprogress.2b475fec-1482-4dea-9946-eb4353b475f1
Old buckets can still receive new records as the bucketing policy is evaluated on a per-record basis.
Finished files can be distinguished from the in-progress ones by their naming scheme only.
By default, the file naming strategy is as follows:
part-<subtaskIndex>-<partFileIndex>.inprogress.uid
part-<subtaskIndex>-<partFileIndex>
Flink allows the user to specify a prefix and/or a suffix for his/her part files.
This can be done using an OutputFileConfig
.
For example for a prefix “prefix” and a suffix “.ext” the sink will create the following files:
└── 2019-08-25--12
├── prefix-0-0.ext
├── prefix-0-1.ext.inprogress.bd053eb0-5ecf-4c85-8433-9eff486ac334
├── prefix-1-0.ext
└── prefix-1-1.ext.inprogress.bc279efe-b16f-47d8-b828-00ef6e2fbd11
The user can specify an OutputFileConfig
in the following way:
Important Note 1: When using Hadoop < 2.7, please use
the OnCheckpointRollingPolicy
which rolls part files on every checkpoint. The reason is that if part files “traverse”
the checkpoint interval, then, upon recovery from a failure the StreamingFileSink
may use the truncate()
method of the
filesystem to discard uncommitted data from the in-progress file. This method is not supported by pre-2.7 Hadoop versions
and Flink will throw an exception.
Important Note 2: Given that Flink sinks and UDFs in general do not differentiate between normal job termination (e.g. finite input stream) and termination due to failure, upon normal termination of a job, the last in-progress files will not be transitioned to the “finished” state.
Important Note 3: Flink and the StreamingFileSink
never overwrites committed data.
Given this, when trying to restore from an old checkpoint/savepoint which assumes an in-progress file which was committed
by subsequent successful checkpoints, Flink will refuse to resume and it will throw an exception as it cannot locate the
in-progress file.
Important Note 4: Currently, the StreamingFileSink
only supports three filesystems:
HDFS, S3, and Local. Flink will throw an exception when using an unsupported filesystem at runtime.
Important Note 1: For S3, the StreamingFileSink
supports only the Hadoop-based FileSystem implementation, not
the implementation based on Presto. In case your job uses the
StreamingFileSink
to write to S3 but you want to use the Presto-based one for checkpointing,
it is advised to use explicitly “s3a://” (for Hadoop) as the scheme for the target path of
the sink and “s3p://” for checkpointing (for Presto). Using “s3://” for both the sink
and checkpointing may lead to unpredictable behavior, as both implementations “listen” to that scheme.
Important Note 2: To guarantee exactly-once semantics while
being efficient, the StreamingFileSink
uses the Multi-part Upload
feature of S3 (MPU from now on). This feature allows to upload files in independent chunks (thus the “multi-part”)
which can be combined into the original file when all the parts of the MPU are successfully uploaded.
For inactive MPUs, S3 supports a bucket lifecycle rule that the user can use to abort multipart uploads
that don’t complete within a specified number of days after being initiated. This implies that if you set this rule
aggressively and take a savepoint with some part-files being not fully uploaded, their associated MPUs may time-out
before the job is restarted. This will result in your job not being able to restore from that savepoint as the
pending part-files are no longer there and Flink will fail with an exception as it tries to fetch them and fails.