Interface BulkFormat<T,SplitT extends FileSourceSplit>
- All Superinterfaces:
org.apache.flink.api.java.typeutils.ResultTypeQueryable<T>,Serializable
- All Known Implementing Classes:
DeserializationSchemaAdapter,LimitableBulkFormat,StreamFormatAdapter
BulkFormat reads and decodes batches of records at a time. Examples of bulk formats
are formats like ORC or Parquet.
The outer 'BulkFormat' class acts mainly as a configuration holder and factory for the
reader. The actual reading is done by the BulkFormat.Reader, which is created in the
createReader(Configuration, FileSourceSplit) method. If a bulk reader is
created based on a checkpoint during checkpointed streaming execution, then the reader is
re-created in the restoreReader(Configuration, FileSourceSplit) method.
Splitting
File splitting means dividing a file into multiple regions that can be read independently.
Whether a format supports splitting is indicated via the isSplittable() method.
Splitting has the potential to increase parallelism and performance, but poses additional constraints on the format readers: Readers need to be able to find a consistent starting point within the file near the offset where the split starts, (like the next record delimiter, or a block start or a sync marker). This is not necessarily possible for all formats, which is why splitting is optional.
Checkpointing
The bulk reader returns an iterator per batch that it reads. The iterator produces records together with a position. That position is stored in the checkpointed state atomically with the processing of the record. That means it must be the position from where the reading can be resumed AFTER the record was processed; the position hence points effectively to the record AFTER the current record.
The simplest way to return this position information is to store no offset and simply store an incrementing count of records to skip after recovery. Given the above contract, the fist record would be returned with a records-to-skip count of one, the second one with a record count of two, etc.
Formats that have the ability to efficiently seek to a record (or to every n-th record) can use the position field to seek to a record directly and avoid having to read and discard many records on recovery.
Note on this design: Why do we not make the position point to the current record and always skip one record after recovery (the just processed record)? We need to be able to support formats where skipping records (even one) is not an option. For example formats that execute (pushed down) filters may want to avoid a skip-record-count all together, so that they don't skip the wrong records when the filter gets updated around a checkpoint/savepoint.
Serializable
Like many other API classes in Flink, the outer class is serializable to support sending instances to distributed workers for parallel execution. This is purely short-term serialization for RPC and no instance of this will be long-term persisted in a serialized form.
Record Batching
Internally in the file source, the readers pass batches of records from the reading threads (that perform the typically blocking I/O operations) to the async mailbox threads that do the streaming and batch data processing. Passing records in batches (rather than one-at-a-time) much reduce the thread-to-thread handover overhead.
For the BulkFormat, one batch (as returned by BulkFormat.Reader.readBatch())
is handed over as one.
-
Nested Class Summary
Nested ClassesModifier and TypeInterfaceDescriptionstatic interfaceThe actual reader that reads the batches of records.static interfaceAn iterator over records with their position in the file. -
Method Summary
Modifier and TypeMethodDescriptioncreateReader(org.apache.flink.configuration.Configuration config, SplitT split) Creates a new reader that reads from thesplit's pathstarting at thesplit's offsetand readslengthbytes after the offset.org.apache.flink.api.common.typeinfo.TypeInformation<T>Gets the type produced by this format.booleanChecks whether this format is splittable.restoreReader(org.apache.flink.configuration.Configuration config, SplitT split) Creates a new reader that reads fromsplit.path()starting atoffsetand reads untillengthbytes after the offset.
-
Method Details
-
createReader
BulkFormat.Reader<T> createReader(org.apache.flink.configuration.Configuration config, SplitT split) throws IOException Creates a new reader that reads from thesplit's pathstarting at thesplit's offsetand readslengthbytes after the offset.- Throws:
IOException
-
restoreReader
BulkFormat.Reader<T> restoreReader(org.apache.flink.configuration.Configuration config, SplitT split) throws IOException Creates a new reader that reads fromsplit.path()starting atoffsetand reads untillengthbytes after the offset. A number ofrecordsToSkiprecords should be read and discarded after the offset. This is typically part of restoring a reader to a checkpointed position.- Throws:
IOException
-
isSplittable
boolean isSplittable()Checks whether this format is splittable. Splittable formats allow Flink to create multiple splits per file, so that Flink can read multiple regions of the file concurrently.See
top-level JavaDocs(section "Splitting") for details. -
getProducedType
org.apache.flink.api.common.typeinfo.TypeInformation<T> getProducedType()Gets the type produced by this format. This type will be the type produced by the file source as a whole.- Specified by:
getProducedTypein interfaceorg.apache.flink.api.java.typeutils.ResultTypeQueryable<T>
-