Streaming File Sink

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

Since in streaming the input is potentially infinite, the streaming file sink writes datainto buckets. The bucketing behaviour is configurable but a useful default is time-basedbucketing where we start writing a new bucket every hour and thus getindividual 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 arolling policy. This is useful to prevent individual bucket files from gettingtoo big. This is also configurable but the default policy rolls files based onfile size and a timeout, i.e if no new data was written to a part file.

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

Using Row-encoded Output Formats

The only required configuration are the base path where we want to output ourdata and anEncoderthat is used for serializing records to the OutputStream for each file.

Basic usage thus looks like this:

  1. import org.apache.flink.api.common.serialization.SimpleStringEncoder;
  2. import org.apache.flink.core.fs.Path;
  3. import org.apache.flink.streaming.api.functions.sink.filesystem.StreamingFileSink;
  4. DataStream<String> input = ...;
  5. final StreamingFileSink<String> sink = StreamingFileSink
  6. .forRowFormat(new Path(outputPath), new SimpleStringEncoder<>("UTF-8"))
  7. .build();
  8. input.addSink(sink);
  1. import org.apache.flink.api.common.serialization.SimpleStringEncoder
  2. import org.apache.flink.core.fs.Path
  3. import org.apache.flink.streaming.api.functions.sink.filesystem.StreamingFileSink
  4. val input: DataStream[String] = ...
  5. val sink: StreamingFileSink[String] = StreamingFileSink
  6. .forRowFormat(new Path(outputPath), new SimpleStringEncoder[String]("UTF-8"))
  7. .build()
  8. input.addSink(sink)

This will create a streaming sink that creates hourly buckets and uses adefault rolling policy. The default bucket assigner isDateTimeBucketAssignerand the default rolling policy isDefaultRollingPolicy.You can specify a customBucketAssignerandRollingPolicyon the sink builder. Please check out the JavaDoc forStreamingFileSinkfor more configuration options and more documentation about the workings andinteractions of bucket assigners and rolling policies.

Using Bulk-encoded Output Formats

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

ParquetAvroWritershas 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, notthe 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 ofthe sink and “s3p://” for checkpointing (for Presto). Using “s3://” for both the sinkand checkpointing may lead to unpredictable behavior, as both implementations “listen” to that scheme.

Important Note 2: To guarantee exactly-once semantics whilebeing efficient, the StreamingFileSink uses the Multi-part Uploadfeature 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 thepending part-files are no longer there and Flink will fail with an exception as it tries to fetch them and fails.