T
- The type of records produced by this data source@Internal public abstract class FlinkKafkaConsumerBase<T> extends RichParallelSourceFunction<T> implements CheckpointListener, ResultTypeQueryable<T>, CheckpointedFunction
The Kafka version specific behavior is defined mainly in the specific subclasses of the
AbstractFetcher
.
SourceFunction.SourceContext<T>
Modifier and Type | Field and Description |
---|---|
protected KeyedDeserializationSchema<T> |
deserializer
The schema to convert between Kafka's byte messages, and Flink's objects.
|
static String |
KEY_DISABLE_METRICS
Boolean configuration key to disable metrics tracking.
|
static String |
KEY_PARTITION_DISCOVERY_INTERVAL_MILLIS
Configuration key to define the consumer's partition discovery interval, in milliseconds.
|
protected static org.slf4j.Logger |
LOG |
static int |
MAX_NUM_PENDING_CHECKPOINTS
The maximum number of pending non-committed checkpoints to track, to avoid memory leaks.
|
static long |
PARTITION_DISCOVERY_DISABLED
The default interval to execute partition discovery,
in milliseconds (
Long.MIN_VALUE , i.e. |
Constructor and Description |
---|
FlinkKafkaConsumerBase(List<String> topics,
Pattern topicPattern,
KeyedDeserializationSchema<T> deserializer,
long discoveryIntervalMillis,
boolean useMetrics)
Base constructor.
|
Modifier and Type | Method and Description |
---|---|
FlinkKafkaConsumerBase<T> |
assignTimestampsAndWatermarks(AssignerWithPeriodicWatermarks<T> assigner)
Specifies an
AssignerWithPunctuatedWatermarks to emit watermarks in a punctuated manner. |
FlinkKafkaConsumerBase<T> |
assignTimestampsAndWatermarks(AssignerWithPunctuatedWatermarks<T> assigner)
Specifies an
AssignerWithPunctuatedWatermarks to emit watermarks in a punctuated manner. |
void |
cancel()
Cancels the source.
|
void |
close()
Tear-down method for the user code.
|
protected abstract AbstractFetcher<T,?> |
createFetcher(SourceFunction.SourceContext<T> sourceContext,
Map<KafkaTopicPartition,Long> subscribedPartitionsToStartOffsets,
SerializedValue<AssignerWithPeriodicWatermarks<T>> watermarksPeriodic,
SerializedValue<AssignerWithPunctuatedWatermarks<T>> watermarksPunctuated,
StreamingRuntimeContext runtimeContext,
OffsetCommitMode offsetCommitMode,
MetricGroup kafkaMetricGroup,
boolean useMetrics)
Creates the fetcher that connect to the Kafka brokers, pulls data, deserialized the
data, and emits it into the data streams.
|
protected abstract AbstractPartitionDiscoverer |
createPartitionDiscoverer(KafkaTopicsDescriptor topicsDescriptor,
int indexOfThisSubtask,
int numParallelSubtasks)
Creates the partition discoverer that is used to find new partitions for this subtask.
|
protected abstract Map<KafkaTopicPartition,Long> |
fetchOffsetsWithTimestamp(Collection<KafkaTopicPartition> partitions,
long timestamp) |
protected abstract boolean |
getIsAutoCommitEnabled() |
TypeInformation<T> |
getProducedType()
Gets the data type (as a
TypeInformation ) produced by this function or input format. |
void |
initializeState(FunctionInitializationContext context)
This method is called when the parallel function instance is created during distributed
execution.
|
void |
notifyCheckpointComplete(long checkpointId)
This method is called as a notification once a distributed checkpoint has been completed.
|
void |
open(Configuration configuration)
Initialization method for the function.
|
void |
run(SourceFunction.SourceContext<T> sourceContext)
Starts the source.
|
FlinkKafkaConsumerBase<T> |
setCommitOffsetsOnCheckpoints(boolean commitOnCheckpoints)
Specifies whether or not the consumer should commit offsets back to Kafka on checkpoints.
|
FlinkKafkaConsumerBase<T> |
setStartFromEarliest()
Specifies the consumer to start reading from the earliest offset for all partitions.
|
FlinkKafkaConsumerBase<T> |
setStartFromGroupOffsets()
Specifies the consumer to start reading from any committed group offsets found
in Zookeeper / Kafka brokers.
|
FlinkKafkaConsumerBase<T> |
setStartFromLatest()
Specifies the consumer to start reading from the latest offset for all partitions.
|
FlinkKafkaConsumerBase<T> |
setStartFromSpecificOffsets(Map<KafkaTopicPartition,Long> specificStartupOffsets)
Specifies the consumer to start reading partitions from specific offsets, set independently for each partition.
|
protected FlinkKafkaConsumerBase<T> |
setStartFromTimestamp(long startupOffsetsTimestamp)
Specifies the consumer to start reading partitions from a specified timestamp.
|
void |
snapshotState(FunctionSnapshotContext context)
This method is called when a snapshot for a checkpoint is requested.
|
getIterationRuntimeContext, getRuntimeContext, setRuntimeContext
protected static final org.slf4j.Logger LOG
public static final int MAX_NUM_PENDING_CHECKPOINTS
public static final long PARTITION_DISCOVERY_DISABLED
Long.MIN_VALUE
, i.e. disabled by default).public static final String KEY_DISABLE_METRICS
public static final String KEY_PARTITION_DISCOVERY_INTERVAL_MILLIS
protected final KeyedDeserializationSchema<T> deserializer
public FlinkKafkaConsumerBase(List<String> topics, Pattern topicPattern, KeyedDeserializationSchema<T> deserializer, long discoveryIntervalMillis, boolean useMetrics)
topics
- fixed list of topics to subscribe to (null, if using topic pattern)topicPattern
- the topic pattern to subscribe to (null, if using fixed topics)deserializer
- The deserializer to turn raw byte messages into Java/Scala objects.discoveryIntervalMillis
- the topic / partition discovery interval, in
milliseconds (0 if discovery is disabled).public FlinkKafkaConsumerBase<T> assignTimestampsAndWatermarks(AssignerWithPunctuatedWatermarks<T> assigner)
AssignerWithPunctuatedWatermarks
to emit watermarks in a punctuated manner.
The watermark extractor will run per Kafka partition, watermarks will be merged across partitions
in the same way as in the Flink runtime, when streams are merged.
When a subtask of a FlinkKafkaConsumer source reads multiple Kafka partitions, the streams from the partitions are unioned in a "first come first serve" fashion. Per-partition characteristics are usually lost that way. For example, if the timestamps are strictly ascending per Kafka partition, they will not be strictly ascending in the resulting Flink DataStream, if the parallel source subtask reads more that one partition.
Running timestamp extractors / watermark generators directly inside the Kafka source, per Kafka partition, allows users to let them exploit the per-partition characteristics.
Note: One can use either an AssignerWithPunctuatedWatermarks
or an
AssignerWithPeriodicWatermarks
, not both at the same time.
assigner
- The timestamp assigner / watermark generator to use.public FlinkKafkaConsumerBase<T> assignTimestampsAndWatermarks(AssignerWithPeriodicWatermarks<T> assigner)
AssignerWithPunctuatedWatermarks
to emit watermarks in a punctuated manner.
The watermark extractor will run per Kafka partition, watermarks will be merged across partitions
in the same way as in the Flink runtime, when streams are merged.
When a subtask of a FlinkKafkaConsumer source reads multiple Kafka partitions, the streams from the partitions are unioned in a "first come first serve" fashion. Per-partition characteristics are usually lost that way. For example, if the timestamps are strictly ascending per Kafka partition, they will not be strictly ascending in the resulting Flink DataStream, if the parallel source subtask reads more that one partition.
Running timestamp extractors / watermark generators directly inside the Kafka source, per Kafka partition, allows users to let them exploit the per-partition characteristics.
Note: One can use either an AssignerWithPunctuatedWatermarks
or an
AssignerWithPeriodicWatermarks
, not both at the same time.
assigner
- The timestamp assigner / watermark generator to use.public FlinkKafkaConsumerBase<T> setCommitOffsetsOnCheckpoints(boolean commitOnCheckpoints)
This setting will only have effect if checkpointing is enabled for the job. If checkpointing isn't enabled, only the "auto.commit.enable" (for 0.8) / "enable.auto.commit" (for 0.9+) property settings will be
public FlinkKafkaConsumerBase<T> setStartFromEarliest()
This method does not affect where partitions are read from when the consumer is restored from a checkpoint or savepoint. When the consumer is restored from a checkpoint or savepoint, only the offsets in the restored state will be used.
public FlinkKafkaConsumerBase<T> setStartFromLatest()
This method does not affect where partitions are read from when the consumer is restored from a checkpoint or savepoint. When the consumer is restored from a checkpoint or savepoint, only the offsets in the restored state will be used.
protected FlinkKafkaConsumerBase<T> setStartFromTimestamp(long startupOffsetsTimestamp)
The consumer will look up the earliest offset whose timestamp is greater than or equal to the specific timestamp from Kafka. If there's no such offset, the consumer will use the latest offset to read data from kafka.
This method does not affect where partitions are read from when the consumer is restored from a checkpoint or savepoint. When the consumer is restored from a checkpoint or savepoint, only the offsets in the restored state will be used.
startupOffsetsTimestamp
- timestamp for the startup offsets, as milliseconds from epoch.public FlinkKafkaConsumerBase<T> setStartFromGroupOffsets()
This method does not affect where partitions are read from when the consumer is restored from a checkpoint or savepoint. When the consumer is restored from a checkpoint or savepoint, only the offsets in the restored state will be used.
public FlinkKafkaConsumerBase<T> setStartFromSpecificOffsets(Map<KafkaTopicPartition,Long> specificStartupOffsets)
If the provided map of offsets contains entries whose KafkaTopicPartition
is not subscribed by the
consumer, the entry will be ignored. If the consumer subscribes to a partition that does not exist in the provided
map of offsets, the consumer will fallback to the default group offset behaviour (see
setStartFromGroupOffsets()
) for that particular partition.
If the specified offset for a partition is invalid, or the behaviour for that partition is defaulted to group offsets but still no group offset could be found for it, then the "auto.offset.reset" behaviour set in the configuration properties will be used for the partition
This method does not affect where partitions are read from when the consumer is restored from a checkpoint or savepoint. When the consumer is restored from a checkpoint or savepoint, only the offsets in the restored state will be used.
public void open(Configuration configuration) throws Exception
RichFunction
The configuration object passed to the function can be used for configuration and initialization. The configuration contains all parameters that were configured on the function in the program composition.
public class MyMapper extends FilterFunction<String> {
private String searchString;
public void open(Configuration parameters) {
this.searchString = parameters.getString("foo");
}
public boolean filter(String value) {
return value.equals(searchString);
}
}
By default, this method does nothing.
open
in interface RichFunction
open
in class AbstractRichFunction
configuration
- The configuration containing the parameters attached to the contract.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.Configuration
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 final void initializeState(FunctionInitializationContext context) throws Exception
CheckpointedFunction
initializeState
in interface CheckpointedFunction
context
- the context for initializing the operatorException
public final 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
public final void notifyCheckpointComplete(long checkpointId) throws Exception
CheckpointListener
notifyCheckpointComplete
in interface CheckpointListener
checkpointId
- The ID of the checkpoint that has been completed.Exception
protected abstract AbstractFetcher<T,?> createFetcher(SourceFunction.SourceContext<T> sourceContext, Map<KafkaTopicPartition,Long> subscribedPartitionsToStartOffsets, SerializedValue<AssignerWithPeriodicWatermarks<T>> watermarksPeriodic, SerializedValue<AssignerWithPunctuatedWatermarks<T>> watermarksPunctuated, StreamingRuntimeContext runtimeContext, OffsetCommitMode offsetCommitMode, MetricGroup kafkaMetricGroup, boolean useMetrics) throws Exception
sourceContext
- The source context to emit data to.subscribedPartitionsToStartOffsets
- The set of partitions that this subtask should handle, with their start offsets.watermarksPeriodic
- Optional, a serialized timestamp extractor / periodic watermark generator.watermarksPunctuated
- Optional, a serialized timestamp extractor / punctuated watermark generator.runtimeContext
- The task's runtime context.Exception
- The method should forward exceptionsprotected abstract AbstractPartitionDiscoverer createPartitionDiscoverer(KafkaTopicsDescriptor topicsDescriptor, int indexOfThisSubtask, int numParallelSubtasks)
topicsDescriptor
- Descriptor that describes whether we are discovering partitions for fixed topics or a topic pattern.indexOfThisSubtask
- The index of this consumer subtask.numParallelSubtasks
- The total number of parallel consumer subtasks.protected abstract boolean getIsAutoCommitEnabled()
protected abstract Map<KafkaTopicPartition,Long> fetchOffsetsWithTimestamp(Collection<KafkaTopicPartition> partitions, long timestamp)
public TypeInformation<T> getProducedType()
ResultTypeQueryable
TypeInformation
) produced by this function or input format.getProducedType
in interface ResultTypeQueryable<T>
Copyright © 2014–2019 The Apache Software Foundation. All rights reserved.