@Public public interface ReaderOutput<T> extends SourceOutput<T>
SourceReader
to emit records, and
optionally watermarks, to downstream operators for message processing.
The ReaderOutput
is a SourceOutput
and can be used directly to emit the stream
of events from the source. This is recommended for sources where the SourceReader processes only
a single split, or where NO split-specific characteristics are required (like per-split
watermarks and idleness, split-specific event-time skew handling, etc.). As a special case, this
is true for sources that only support bounded/batch data processing.
For most streaming sources, the SourceReader
should use split-specific outputs, to
allow the processing logic to run per-split watermark generators, idleness detection, etc. To
create a split-specific SourceOutput
use the createOutputForSplit(String)
method, using the Source Split's ID. Make sure to
release the output again once the source has finished processing that split.
Modifier and Type | Method and Description |
---|---|
void |
collect(T record)
Emit a record without a timestamp.
|
void |
collect(T record,
long timestamp)
Emit a record with a timestamp.
|
SourceOutput<T> |
createOutputForSplit(String splitId)
Creates a
SourceOutput for a specific Source Split. |
void |
emitWatermark(Watermark watermark)
Emits the given watermark.
|
void |
markIdle()
Marks this output as idle, meaning that downstream operations do not wait for watermarks from
this output.
|
void |
releaseOutputForSplit(String splitId)
Releases the
SourceOutput created for the split with the given ID. |
markActive
void collect(T record)
Use this method if the source system does not have a notion of records with timestamps.
The events later pass through a TimestampAssigner
, which attaches a timestamp to
the event based on the event's contents. For example a file source with JSON records would
not have a generic timestamp from the file reading and JSON parsing process, and thus use
this method to produce initially a record without a timestamp. The TimestampAssigner
in the next step would be used to extract timestamp from a field of the JSON object.
collect
in interface SourceOutput<T>
record
- the record to emit.void collect(T record, long timestamp)
Use this method if the source system has timestamps attached to records. Typical examples would be Logs, PubSubs, or Message Queues, like Kafka or Kinesis, which store a timestamp with each event.
The events typically still pass through a TimestampAssigner
, which may decide to
either use this source-provided timestamp, or replace it with a timestamp stored within the
event (for example if the event was a JSON object one could configure aTimestampAssigner that
extracts one of the object's fields and uses that as a timestamp).
collect
in interface SourceOutput<T>
record
- the record to emit.timestamp
- the timestamp of the record.void emitWatermark(Watermark watermark)
Emitting a watermark also implicitly marks the stream as active, ending previously marked idleness.
emitWatermark
in interface WatermarkOutput
void markIdle()
An output becomes active again as soon as the next watermark is emitted.
markIdle
in interface WatermarkOutput
SourceOutput<T> createOutputForSplit(String splitId)
SourceOutput
for a specific Source Split. Use these outputs if you want to
run split-local logic, like watermark generation.
If a split-local output was already created for this split-ID, the method will return that instance, so that only one split-local output exists per split-ID.
IMPORTANT: After the split has been finished, it is crucial to release the created output again. Otherwise it will continue to contribute to the watermark generation like a perpetually stalling source split, and may hold back the watermark indefinitely.
releaseOutputForSplit(String)
void releaseOutputForSplit(String splitId)
SourceOutput
created for the split with the given ID.createOutputForSplit(String)
Copyright © 2014–2024 The Apache Software Foundation. All rights reserved.