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
emit
elements.
Sources that implement CheckpointedFunction
must lock on the checkpoint lock (using a synchronized block) before updating internal
state and emitting elements, to make both an atomic operation:
public class ExampleCountSource implements SourceFunction<Long>, CheckpointedFunction {
private long count = 0L;
private volatile boolean isRunning = true;
private transient ListState<Long> checkpointedCount;
public void run(SourceContext<T> ctx) {
while (isRunning && count < 1000) {
// this synchronized block ensures that state checkpointing,
// internal state updates and emission of elements are an atomic operation
synchronized (ctx.getCheckpointLock()) {
ctx.collect(count);
count++;
}
}
}
public void cancel() {
isRunning = false;
}
public void initializeState(FunctionInitializationContext context) {
this.checkpointedCount = context
.getOperatorStateStore()
.getListState(new ListStateDescriptor<>("count", Long.class));
if (context.isRestored()) {
for (Long count : this.checkpointedCount.get()) {
this.count = count;
}
}
}
public void snapshotState(FunctionSnapshotContext context) {
this.checkpointedCount.clear();
this.checkpointedCount.add(count);
}
}
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.
When a source is canceled, the executing thread will also be interrupted
(via Thread.interrupt()
). The interruption happens strictly after this
method has been called, so any interruption handler can rely on the fact that
this method has completed. It is good practice to make any flags altered by
this method "volatile", in order to guarantee the visibility of the effects of
this method to any interruption handler.
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
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
protected KinesisDataFetcher<T> createFetcher(List<String> streams, SourceFunction.SourceContext<T> sourceContext, RuntimeContext runtimeContext, Properties configProps, KinesisDeserializationSchema<T> deserializationSchema)
Copyright © 2014–2020 The Apache Software Foundation. All rights reserved.