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 StatefulSink<IN,FileWriterBucketState>, TwoPhaseCommittingSink<IN,FileSinkCommittable>, StatefulSink.WithCompatibleState, WithPreCommitTopology<IN,FileSinkCommittable>, SupportsConcurrentExecutionAttempts
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.
FileSink also support compacting small files to accelerate the access speed of the resulted
files. Compaction could be enabled via enableCompact
. Once enabled, the compaction could
only be disabled via calling disableCompact
explicitly, otherwise there might be data
loss.
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.
|
StatefulSink.StatefulSinkWriter<InputT,WriterStateT>, StatefulSink.WithCompatibleState
TwoPhaseCommittingSink.PrecommittingSinkWriter<InputT,CommT>
Sink.InitContext
public FileWriter<IN> createWriter(Sink.InitContext context) throws IOException
StatefulSink
StatefulSink.StatefulSinkWriter
.createWriter
in interface Sink<IN>
createWriter
in interface StatefulSink<IN,FileWriterBucketState>
createWriter
in interface TwoPhaseCommittingSink<IN,FileSinkCommittable>
context
- the runtime context.IOException
- for any failure during creation.public FileWriter<IN> restoreWriter(Sink.InitContext context, Collection<FileWriterBucketState> recoveredState) throws IOException
StatefulSink
StatefulSink.StatefulSinkWriter
from a recovered state.restoreWriter
in interface StatefulSink<IN,FileWriterBucketState>
context
- the runtime context.IOException
- for any failure during creation.public SimpleVersionedSerializer<FileWriterBucketState> getWriterStateSerializer()
StatefulSink
StatefulSink.StatefulSinkWriter.snapshotState(long)
properly. The respective state is used in #restoreWriter(InitContext, Collection)
on recovery.getWriterStateSerializer
in interface StatefulSink<IN,FileWriterBucketState>
public Committer<FileSinkCommittable> createCommitter() throws IOException
TwoPhaseCommittingSink
Committer
that permanently makes the previously written data visible
through Committer.commit(Collection)
.createCommitter
in interface TwoPhaseCommittingSink<IN,FileSinkCommittable>
IOException
- for any failure during creation.public SimpleVersionedSerializer<FileSinkCommittable> getCommittableSerializer()
TwoPhaseCommittingSink
getCommittableSerializer
in interface TwoPhaseCommittingSink<IN,FileSinkCommittable>
public Collection<String> getCompatibleWriterStateNames()
StatefulSink.WithCompatibleState
FileSink
can resume from the state of an old StreamingFileSink
as a
drop-in replacement when resuming from a checkpoint/savepoint.getCompatibleWriterStateNames
in interface StatefulSink.WithCompatibleState
public static <IN> FileSink.DefaultRowFormatBuilder<IN> forRowFormat(Path basePath, Encoder<IN> encoder)
public static <IN> FileSink.DefaultBulkFormatBuilder<IN> forBulkFormat(Path basePath, BulkWriter.Factory<IN> bulkWriterFactory)
public DataStream<CommittableMessage<FileSinkCommittable>> addPreCommitTopology(DataStream<CommittableMessage<FileSinkCommittable>> committableStream)
WithPreCommitTopology
CommittableMessage
s appropriately.addPreCommitTopology
in interface WithPreCommitTopology<IN,FileSinkCommittable>
committableStream
- the stream of committables.Committer
.Copyright © 2014–2024 The Apache Software Foundation. All rights reserved.