T
- the type of data emitted@PublicEvolving public class FlinkKinesisConsumer<T> extends RichParallelSourceFunction<T> implements ResultTypeQueryable<T>, CheckpointedFunction
To leverage Flink's checkpointing mechanics for exactly-once streaming processing guarantees, the Flink Kinesis consumer is implemented with the AWS Java SDK, instead of the officially recommended AWS Kinesis Client Library, for low-level control on the management of stream state. The Flink Kinesis Connector also supports setting the initial starting points of Kinesis streams, namely TRIM_HORIZON and LATEST.
Kinesis and the Flink consumer support dynamic re-sharding and shard IDs, while sequential, cannot be assumed to be consecutive. There is no perfect generic default assignment function. Default shard to subtask assignment, which is based on hash code, may result in skew, with some subtasks having many shards assigned and others none.
It is recommended to monitor the shard distribution and adjust assignment appropriately. A
custom assigner implementation can be set via setShardAssigner(KinesisShardAssigner)
to
optimize the hash function or use static overrides to limit skew.
In order for the consumer to emit watermarks, a timestamp assigner needs to be set via setPeriodicWatermarkAssigner(AssignerWithPeriodicWatermarks)
and the auto watermark emit
interval configured via ExecutionConfig.setAutoWatermarkInterval(long)
.
Watermarks can only advance when all shards of a subtask continuously deliver records. To
avoid an inactive or closed shard to block the watermark progress, the idle timeout should be
configured via configuration property ConsumerConfigConstants.SHARD_IDLE_INTERVAL_MILLIS
.
By default, shards won't be considered idle and watermark calculation will wait for newer records
to arrive from all shards.
Note that re-sharding of the Kinesis stream while an application (that relies on the Kinesis records for watermarking) is running can lead to incorrect late events. This depends on how shards are assigned to subtasks and applies regardless of whether watermarks are generated in the source or a downstream operator.
SourceFunction.SourceContext<T>
Constructor and Description |
---|
FlinkKinesisConsumer(List<String> streams,
KinesisDeserializationSchema<T> deserializer,
Properties configProps)
Creates a new Flink Kinesis Consumer.
|
FlinkKinesisConsumer(String stream,
DeserializationSchema<T> deserializer,
Properties configProps)
Creates a new Flink Kinesis Consumer.
|
FlinkKinesisConsumer(String stream,
KinesisDeserializationSchema<T> deserializer,
Properties configProps)
Creates a new Flink Kinesis Consumer.
|
Modifier and Type | Method and Description |
---|---|
void |
cancel()
Cancels the source.
|
void |
close()
Tear-down method for the user code.
|
protected KinesisDataFetcher<T> |
createFetcher(List<String> streams,
SourceFunction.SourceContext<T> sourceContext,
RuntimeContext runtimeContext,
Properties configProps,
KinesisDeserializationSchema<T> deserializationSchema)
This method is exposed for tests that need to mock the KinesisDataFetcher in the consumer.
|
AssignerWithPeriodicWatermarks<T> |
getPeriodicWatermarkAssigner() |
TypeInformation<T> |
getProducedType()
Gets the data type (as a
TypeInformation ) produced by this function or input format. |
KinesisShardAssigner |
getShardAssigner() |
WatermarkTracker |
getWatermarkTracker() |
void |
initializeState(FunctionInitializationContext context)
This method is called when the parallel function instance is created during distributed
execution.
|
void |
run(SourceFunction.SourceContext<T> sourceContext)
Starts the source.
|
void |
setPeriodicWatermarkAssigner(AssignerWithPeriodicWatermarks<T> periodicWatermarkAssigner)
Set the assigner that will extract the timestamp from
T and calculate the watermark. |
void |
setShardAssigner(KinesisShardAssigner shardAssigner)
Provide a custom assigner to influence how shards are distributed over subtasks.
|
void |
setWatermarkTracker(WatermarkTracker watermarkTracker)
Set the global watermark tracker.
|
void |
snapshotState(FunctionSnapshotContext context)
This method is called when a snapshot for a checkpoint is requested.
|
getIterationRuntimeContext, getRuntimeContext, open, setRuntimeContext
public FlinkKinesisConsumer(String stream, DeserializationSchema<T> deserializer, Properties configProps)
The AWS credentials to be used, AWS region of the Kinesis streams, initial position to
start streaming from are configured with a Properties
instance.
stream
- The single AWS Kinesis stream to read from.deserializer
- The deserializer used to convert raw bytes of Kinesis records to Java
objects (without key).configProps
- The properties used to configure AWS credentials, AWS region, and initial
starting position.public FlinkKinesisConsumer(String stream, KinesisDeserializationSchema<T> deserializer, Properties configProps)
The AWS credentials to be used, AWS region of the Kinesis streams, initial position to
start streaming from are configured with a Properties
instance.
stream
- The single AWS Kinesis stream to read from.deserializer
- The keyed deserializer used to convert raw bytes of Kinesis records to
Java objects.configProps
- The properties used to configure AWS credentials, AWS region, and initial
starting position.public FlinkKinesisConsumer(List<String> streams, KinesisDeserializationSchema<T> deserializer, Properties configProps)
The AWS credentials to be used, AWS region of the Kinesis streams, initial position to
start streaming from are configured with a Properties
instance.
streams
- The AWS Kinesis streams to read from.deserializer
- The keyed deserializer used to convert raw bytes of Kinesis records to
Java objects.configProps
- The properties used to configure AWS credentials, AWS region, and initial
starting position.public KinesisShardAssigner getShardAssigner()
public void setShardAssigner(KinesisShardAssigner shardAssigner)
shardAssigner
- shard assignerpublic AssignerWithPeriodicWatermarks<T> getPeriodicWatermarkAssigner()
public void setPeriodicWatermarkAssigner(AssignerWithPeriodicWatermarks<T> periodicWatermarkAssigner)
T
and calculate the watermark.periodicWatermarkAssigner
- periodic watermark assignerpublic WatermarkTracker getWatermarkTracker()
public void setWatermarkTracker(WatermarkTracker watermarkTracker)
watermarkTracker
- public void run(SourceFunction.SourceContext<T> sourceContext) throws Exception
SourceFunction
SourceFunction.SourceContext
to emit elements. Sources
that checkpoint their state for fault tolerance should use the SourceFunction.SourceContext.getCheckpointLock()
checkpoint lock} to ensure consistency between the
bookkeeping and emitting the elements.
Sources that implement CheckpointedFunction
must lock on the SourceFunction.SourceContext.getCheckpointLock()
checkpoint lock} checkpoint lock (using a synchronized
block) before updating internal state and emitting elements, to make both an atomic
operation.
Refer to the top-level class docs
for an example.
run
in interface SourceFunction<T>
sourceContext
- The context to emit elements to and for accessing locks.Exception
public void cancel()
SourceFunction
SourceFunction.run(SourceContext)
method. The implementation needs to ensure that the source will break
out of that loop after this method is called.
A typical pattern is to have an "volatile boolean isRunning"
flag that is set to
false
in this method. That flag is checked in the loop condition.
In case of an ungraceful shutdown (cancellation of the source operator, possibly for
failover), the thread that calls SourceFunction.run(SourceContext)
will also be interrupted
) by the Flink runtime, in order to speed up the cancellation
(to ensure threads exit blocking methods fast, like I/O, blocking queues, etc.). The
interruption happens strictly after this method has been called, so any interruption handler
can rely on the fact that this method has completed (for example to ignore exceptions that
happen after cancellation).
During graceful shutdown (for example stopping a job with a savepoint), the program must
cleanly exit the SourceFunction.run(SourceContext)
method soon after this method was called. The
Flink runtime will NOT interrupt the source thread during graceful shutdown. Source
implementors must ensure that no thread interruption happens on any thread that emits records
through the SourceContext
from the SourceFunction.run(SourceContext)
method; otherwise the
clean shutdown may fail when threads are interrupted while processing the final records.
Because the SourceFunction
cannot easily differentiate whether the shutdown should
be graceful or ungraceful, we recommend that implementors refrain from interrupting any
threads that interact with the SourceContext
at all. You can rely on the Flink
runtime to interrupt the source thread in case of ungraceful cancellation. Any additionally
spawned threads that directly emit records through the SourceContext
should use a
shutdown method that does not rely on thread interruption.
cancel
in interface SourceFunction<T>
public void close() throws Exception
RichFunction
This method can be used for clean up work.
close
in interface RichFunction
close
in class AbstractRichFunction
Exception
- Implementations may forward exceptions, which are caught by the runtime.
When the runtime catches an exception, it aborts the task and lets the fail-over logic
decide whether to retry the task execution.public TypeInformation<T> getProducedType()
ResultTypeQueryable
TypeInformation
) produced by this function or input format.getProducedType
in interface ResultTypeQueryable<T>
public void initializeState(FunctionInitializationContext context) throws Exception
CheckpointedFunction
initializeState
in interface CheckpointedFunction
context
- the context for initializing the operatorException
- Thrown, if state could not be created ot restored.public void snapshotState(FunctionSnapshotContext context) throws Exception
CheckpointedFunction
FunctionInitializationContext
when the Function was initialized, or offered now by FunctionSnapshotContext
itself.snapshotState
in interface CheckpointedFunction
context
- the context for drawing a snapshot of the operatorException
- Thrown, if state could not be created ot restored.protected KinesisDataFetcher<T> createFetcher(List<String> streams, SourceFunction.SourceContext<T> sourceContext, RuntimeContext runtimeContext, Properties configProps, KinesisDeserializationSchema<T> deserializationSchema)
Copyright © 2014–2024 The Apache Software Foundation. All rights reserved.