IN - Type of the elements in the input of the sink that are also the elements to be written to its output@Experimental public class FileSink<IN> extends Object implements org.apache.flink.api.connector.sink.Sink<IN,FileSinkCommittable,FileWriterBucketState,Void>
FileSystem files within buckets.
This sink achieves exactly-once semantics for both BATCH and STREAMING.
When creating the sink a basePath must be specified. The base directory contains
one directory for every bucket. The bucket directories themselves contain several part files,
with at least one for each parallel subtask of the sink which is writing data to that bucket.
These part files contain the actual output data.
The sink uses a BucketAssigner to determine in which bucket directory each element should
be written to inside the base directory. The BucketAssigner can, for example, roll on every checkpoint
or use time or a property of the element to determine the bucket directory. The default BucketAssigner is a
DateTimeBucketAssigner which will create one new bucket every hour. You can specify
a custom BucketAssigner using the setBucketAssigner(bucketAssigner) method, after calling
forRowFormat(Path, Encoder) or forBulkFormat(Path, BulkWriter.Factory).
The names of the part files could be defined using OutputFileConfig. This configuration contains
a part prefix and a part suffix that will be used with a random uid assigned to each subtask of the sink and
a rolling counter to determine the file names. For example with a prefix "prefix" and a suffix ".ext", a file named
"prefix-81fc4980-a6af-41c8-9937-9939408a734b-17.ext" contains the data from subtask with uid
81fc4980-a6af-41c8-9937-9939408a734b of the sink and is the 17th part-file created by that subtask.
Part files roll based on the user-specified RollingPolicy. By default, a DefaultRollingPolicy
is used for row-encoded sink output; a OnCheckpointRollingPolicy is used for bulk-encoded sink output.
In some scenarios, the open buckets are required to change based on time. In these cases, the user
can specify a bucketCheckInterval (by default 1m) and the sink will check periodically and roll
the part file if the specified rolling policy says so.
Part files can be in one of three states: in-progress, pending or finished.
The reason for this is how the sink works to provide exactly-once semantics and fault-tolerance. The part
file that is currently being written to is in-progress. Once a part file is closed for writing it
becomes pending. When a checkpoint is successful (for STREAMING) or at the end of the job
(for BATCH) the currently pending files will be moved to finished.
For STREAMING in order to guarantee exactly-once semantics in case of a failure, the sink should roll
back to the state it had when that last successful checkpoint occurred. To this end, when restoring, the restored
files in pending state are transferred into the finished state while any in-progress files
are rolled back, so that they do not contain data that arrived after the checkpoint from which we restore.
| Modifier and Type | Class and Description |
|---|---|
static class |
FileSink.BulkFormatBuilder<IN,T extends FileSink.BulkFormatBuilder<IN,T>>
A builder for configuring the sink for bulk-encoding formats, e.g.
|
static class |
FileSink.DefaultBulkFormatBuilder<IN>
Builder for the vanilla
FileSink using a bulk format. |
static class |
FileSink.DefaultRowFormatBuilder<IN>
Builder for the vanilla
FileSink using a row format. |
static class |
FileSink.RowFormatBuilder<IN,T extends FileSink.RowFormatBuilder<IN,T>>
A builder for configuring the sink for row-wise encoding formats.
|
| Modifier and Type | Method and Description |
|---|---|
Optional<org.apache.flink.api.connector.sink.Committer<FileSinkCommittable>> |
createCommitter() |
Optional<org.apache.flink.api.connector.sink.GlobalCommitter<FileSinkCommittable,Void>> |
createGlobalCommitter() |
org.apache.flink.api.connector.sink.SinkWriter<IN,FileSinkCommittable,FileWriterBucketState> |
createWriter(org.apache.flink.api.connector.sink.Sink.InitContext context,
List<FileWriterBucketState> states) |
static <IN> FileSink.DefaultBulkFormatBuilder<IN> |
forBulkFormat(org.apache.flink.core.fs.Path basePath,
org.apache.flink.api.common.serialization.BulkWriter.Factory<IN> bulkWriterFactory) |
static <IN> FileSink.DefaultRowFormatBuilder<IN> |
forRowFormat(org.apache.flink.core.fs.Path basePath,
org.apache.flink.api.common.serialization.Encoder<IN> encoder) |
Optional<org.apache.flink.core.io.SimpleVersionedSerializer<FileSinkCommittable>> |
getCommittableSerializer() |
Optional<org.apache.flink.core.io.SimpleVersionedSerializer<Void>> |
getGlobalCommittableSerializer() |
Optional<org.apache.flink.core.io.SimpleVersionedSerializer<FileWriterBucketState>> |
getWriterStateSerializer() |
public org.apache.flink.api.connector.sink.SinkWriter<IN,FileSinkCommittable,FileWriterBucketState> createWriter(org.apache.flink.api.connector.sink.Sink.InitContext context, List<FileWriterBucketState> states) throws IOException
createWriter in interface org.apache.flink.api.connector.sink.Sink<IN,FileSinkCommittable,FileWriterBucketState,Void>IOExceptionpublic Optional<org.apache.flink.core.io.SimpleVersionedSerializer<FileWriterBucketState>> getWriterStateSerializer()
getWriterStateSerializer in interface org.apache.flink.api.connector.sink.Sink<IN,FileSinkCommittable,FileWriterBucketState,Void>public Optional<org.apache.flink.api.connector.sink.Committer<FileSinkCommittable>> createCommitter() throws IOException
createCommitter in interface org.apache.flink.api.connector.sink.Sink<IN,FileSinkCommittable,FileWriterBucketState,Void>IOExceptionpublic Optional<org.apache.flink.core.io.SimpleVersionedSerializer<FileSinkCommittable>> getCommittableSerializer()
getCommittableSerializer in interface org.apache.flink.api.connector.sink.Sink<IN,FileSinkCommittable,FileWriterBucketState,Void>public Optional<org.apache.flink.api.connector.sink.GlobalCommitter<FileSinkCommittable,Void>> createGlobalCommitter()
createGlobalCommitter in interface org.apache.flink.api.connector.sink.Sink<IN,FileSinkCommittable,FileWriterBucketState,Void>public Optional<org.apache.flink.core.io.SimpleVersionedSerializer<Void>> getGlobalCommittableSerializer()
getGlobalCommittableSerializer in interface org.apache.flink.api.connector.sink.Sink<IN,FileSinkCommittable,FileWriterBucketState,Void>public static <IN> FileSink.DefaultRowFormatBuilder<IN> forRowFormat(org.apache.flink.core.fs.Path basePath, org.apache.flink.api.common.serialization.Encoder<IN> encoder)
public static <IN> FileSink.DefaultBulkFormatBuilder<IN> forBulkFormat(org.apache.flink.core.fs.Path basePath, org.apache.flink.api.common.serialization.BulkWriter.Factory<IN> bulkWriterFactory)
Copyright © 2014–2020 The Apache Software Foundation. All rights reserved.