title: “Streaming File Sink” nav-title: Streaming File Sink nav-parent_id: connectors nav-pos: 5

This connector provides a Sink that writes partitioned files to filesystems supported by the [Flink FileSystem abstraction]({{ site.baseurl}}/ops/filesystems.html).

Important Note: 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.

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]({{ site.javadocs_baseurl }}/api/java/org/apache/flink/api/common/serialization/Encoder.html) that is used for serializing records to the OutputStream for each file.

Basic usage thus looks like this:

DataStream input = ...;

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

input.addSink(sink);

{% endhighlight %}

val input: DataStream[String] = ...

val sink: StreamingFileSink[String] = StreamingFileSink .forRowFormat(new Path(outputPath), new SimpleStringEncoderString) .build()

input.addSink(sink)

{% endhighlight %}

This will create a streaming sink that creates hourly buckets and uses a default rolling policy. The default bucket assigner is [DateTimeBucketAssigner]({{ site.javadocs_baseurl }}/api/java/org/apache/flink/streaming/api/functions/sink/filesystem/bucketassigners/DateTimeBucketAssigner.html) and the default rolling policy is [DefaultRollingPolicy]({{ site.javadocs_baseurl }}/api/java/org/apache/flink/streaming/api/functions/sink/filesystem/rollingpolicies/DefaultRollingPolicy.html). You can specify a custom [BucketAssigner]({{ site.javadocs_baseurl }}/api/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketAssigner.html) and [RollingPolicy]({{ site.javadocs_baseurl }}/api/java/org/apache/flink/streaming/api/functions/sink/filesystem/RollingPolicy.html) on the sink builder. Please check out the JavaDoc for [StreamingFileSink]({{ site.javadocs_baseurl }}/api/java/org/apache/flink/streaming/api/functions/sink/filesystem/StreamingFileSink.html) 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]({{ site.javadocs_baseurl }}/api/java/org/apache/flink/formats/parquet/avro/ParquetAvroWriters.html) has static methods for creating a BulkWriter.Factory for various types.

{% top %}