This documentation is for an out-of-date version of Apache Flink. We recommend you use the latest stable version.

Streaming File Sink

This connector provides a Sink that writes partitioned files to filesystems supported by the Flink FileSystem abstraction.

Since in streaming the input is potentially infinite, the streaming file sink writes data into buckets. The bucketing behaviour is configurable but a useful default is time-based bucketing where we start writing a new bucket every hour and thus get individual files that each contain a part of the infinite output stream.

Within a bucket, we further split the output into smaller part files based on a rolling policy. This is useful to prevent individual bucket files from getting too big. This is also configurable but the default policy rolls files based on file size and a timeout, i.e if no new data was written to a part file.

The StreamingFileSink supports both row-wise encoding formats and bulk-encoding formats, such as Apache Parquet.

Using Row-encoded Output Formats

The only required configuration are the base path where we want to output our data and an Encoder that is used for serializing records to the OutputStream for each file.

Basic usage thus looks like this:

import org.apache.flink.api.common.serialization.SimpleStringEncoder;
import org.apache.flink.core.fs.Path;
import org.apache.flink.streaming.api.functions.sink.filesystem.StreamingFileSink;

DataStream<String> input = ...;

final StreamingFileSink<String> sink = StreamingFileSink
	.forRowFormat(new Path(outputPath), new SimpleStringEncoder<>("UTF-8"))
	.build();

input.addSink(sink);
import org.apache.flink.api.common.serialization.SimpleStringEncoder
import org.apache.flink.core.fs.Path
import org.apache.flink.streaming.api.functions.sink.filesystem.StreamingFileSink

val input: DataStream[String] = ...

val sink: StreamingFileSink[String] = StreamingFileSink
    .forRowFormat(new Path(outputPath), new SimpleStringEncoder[String]("UTF-8"))
    .build()
    
input.addSink(sink)

This will create a streaming sink that creates hourly buckets and uses a default rolling policy. The default bucket assigner is DateTimeBucketAssigner and the default rolling policy is DefaultRollingPolicy. You can specify a custom BucketAssigner and RollingPolicy on the sink builder. Please check out the JavaDoc for StreamingFileSink for more configuration options and more documentation about the workings and interactions of bucket assigners and rolling policies.

Using Bulk-encoded Output Formats

In the above example we used an Encoder that can encode or serialize each record individually. The streaming file sink also supports bulk-encoded output formats such as Apache Parquet. To use these, instead of StreamingFileSink.forRowFormat() you would use StreamingFileSink.forBulkFormat() and specify a BulkWriter.Factory.

ParquetAvroWriters has static methods for creating a BulkWriter.Factory for various types.

IMPORTANT: Bulk-encoding formats can only be combined with the `OnCheckpointRollingPolicy`, which rolls the in-progress part file on every checkpoint.

Important Considerations for S3

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.

Back to top