@PublicEvolving public class FlinkKafkaConsumer<T> extends FlinkKafkaConsumerBase<T>
The Flink Kafka Consumer participates in checkpointing and guarantees that no data is lost during a failure, and that the computation processes elements "exactly once". (Note: These guarantees naturally assume that Kafka itself does not loose any data.)
Please note that Flink snapshots the offsets internally as part of its distributed checkpoints. The offsets committed to Kafka are only to bring the outside view of progress in sync with Flink's view of the progress. That way, monitoring and other jobs can get a view of how far the Flink Kafka consumer has consumed a topic.
Please refer to Kafka's documentation for the available configuration properties: http://kafka.apache.org/documentation.html#newconsumerconfigs
SourceFunction.SourceContext<T>
Modifier and Type | Field and Description |
---|---|
static long |
DEFAULT_POLL_TIMEOUT
From Kafka's Javadoc: The time, in milliseconds, spent waiting in poll if data is not
available.
|
static String |
KEY_POLL_TIMEOUT
Configuration key to change the polling timeout.
|
protected long |
pollTimeout
From Kafka's Javadoc: The time, in milliseconds, spent waiting in poll if data is not
available.
|
protected Properties |
properties
User-supplied properties for Kafka.
|
deserializer, KEY_DISABLE_METRICS, KEY_PARTITION_DISCOVERY_INTERVAL_MILLIS, LOG, MAX_NUM_PENDING_CHECKPOINTS, PARTITION_DISCOVERY_DISABLED
Constructor and Description |
---|
FlinkKafkaConsumer(List<String> topics,
DeserializationSchema<T> deserializer,
Properties props)
Creates a new Kafka streaming source consumer.
|
FlinkKafkaConsumer(List<String> topics,
KafkaDeserializationSchema<T> deserializer,
Properties props)
Creates a new Kafka streaming source consumer.
|
FlinkKafkaConsumer(Pattern subscriptionPattern,
DeserializationSchema<T> valueDeserializer,
Properties props)
Creates a new Kafka streaming source consumer.
|
FlinkKafkaConsumer(Pattern subscriptionPattern,
KafkaDeserializationSchema<T> deserializer,
Properties props)
Creates a new Kafka streaming source consumer.
|
FlinkKafkaConsumer(String topic,
DeserializationSchema<T> valueDeserializer,
Properties props)
Creates a new Kafka streaming source consumer.
|
FlinkKafkaConsumer(String topic,
KafkaDeserializationSchema<T> deserializer,
Properties props)
Creates a new Kafka streaming source consumer.
|
Modifier and Type | Method and Description |
---|---|
protected AbstractFetcher<T,?> |
createFetcher(SourceFunction.SourceContext<T> sourceContext,
Map<KafkaTopicPartition,Long> assignedPartitionsWithInitialOffsets,
SerializedValue<WatermarkStrategy<T>> watermarkStrategy,
StreamingRuntimeContext runtimeContext,
OffsetCommitMode offsetCommitMode,
MetricGroup consumerMetricGroup,
boolean useMetrics)
Creates the fetcher that connect to the Kafka brokers, pulls data, deserialized the data, and
emits it into the data streams.
|
protected AbstractPartitionDiscoverer |
createPartitionDiscoverer(KafkaTopicsDescriptor topicsDescriptor,
int indexOfThisSubtask,
int numParallelSubtasks)
Creates the partition discoverer that is used to find new partitions for this subtask.
|
protected Map<KafkaTopicPartition,Long> |
fetchOffsetsWithTimestamp(Collection<KafkaTopicPartition> partitions,
long timestamp) |
protected boolean |
getIsAutoCommitEnabled() |
adjustAutoCommitConfig, assignTimestampsAndWatermarks, assignTimestampsAndWatermarks, assignTimestampsAndWatermarks, cancel, close, disableFilterRestoredPartitionsWithSubscribedTopics, getEnableCommitOnCheckpoints, getProducedType, initializeState, notifyCheckpointAborted, notifyCheckpointComplete, open, run, setCommitOffsetsOnCheckpoints, setStartFromEarliest, setStartFromGroupOffsets, setStartFromLatest, setStartFromSpecificOffsets, setStartFromTimestamp, snapshotState
getIterationRuntimeContext, getRuntimeContext, setRuntimeContext
public static final String KEY_POLL_TIMEOUT
public static final long DEFAULT_POLL_TIMEOUT
protected final Properties properties
protected final long pollTimeout
public FlinkKafkaConsumer(String topic, DeserializationSchema<T> valueDeserializer, Properties props)
topic
- The name of the topic that should be consumed.valueDeserializer
- The de-/serializer used to convert between Kafka's byte messages and
Flink's objects.props
- public FlinkKafkaConsumer(String topic, KafkaDeserializationSchema<T> deserializer, Properties props)
This constructor allows passing a KafkaDeserializationSchema
for reading key/value
pairs, offsets, and topic names from Kafka.
topic
- The name of the topic that should be consumed.deserializer
- The keyed de-/serializer used to convert between Kafka's byte messages
and Flink's objects.props
- public FlinkKafkaConsumer(List<String> topics, DeserializationSchema<T> deserializer, Properties props)
This constructor allows passing multiple topics to the consumer.
topics
- The Kafka topics to read from.deserializer
- The de-/serializer used to convert between Kafka's byte messages and
Flink's objects.props
- public FlinkKafkaConsumer(List<String> topics, KafkaDeserializationSchema<T> deserializer, Properties props)
This constructor allows passing multiple topics and a key/value deserialization schema.
topics
- The Kafka topics to read from.deserializer
- The keyed de-/serializer used to convert between Kafka's byte messages
and Flink's objects.props
- public FlinkKafkaConsumer(Pattern subscriptionPattern, DeserializationSchema<T> valueDeserializer, Properties props)
If partition discovery is enabled (by setting a non-negative value for FlinkKafkaConsumerBase.KEY_PARTITION_DISCOVERY_INTERVAL_MILLIS
in the properties), topics with
names matching the pattern will also be subscribed to as they are created on the fly.
subscriptionPattern
- The regular expression for a pattern of topic names to subscribe
to.valueDeserializer
- The de-/serializer used to convert between Kafka's byte messages and
Flink's objects.props
- public FlinkKafkaConsumer(Pattern subscriptionPattern, KafkaDeserializationSchema<T> deserializer, Properties props)
If partition discovery is enabled (by setting a non-negative value for FlinkKafkaConsumerBase.KEY_PARTITION_DISCOVERY_INTERVAL_MILLIS
in the properties), topics with
names matching the pattern will also be subscribed to as they are created on the fly.
This constructor allows passing a KafkaDeserializationSchema
for reading key/value
pairs, offsets, and topic names from Kafka.
subscriptionPattern
- The regular expression for a pattern of topic names to subscribe
to.deserializer
- The keyed de-/serializer used to convert between Kafka's byte messages
and Flink's objects.props
- protected AbstractFetcher<T,?> createFetcher(SourceFunction.SourceContext<T> sourceContext, Map<KafkaTopicPartition,Long> assignedPartitionsWithInitialOffsets, SerializedValue<WatermarkStrategy<T>> watermarkStrategy, StreamingRuntimeContext runtimeContext, OffsetCommitMode offsetCommitMode, MetricGroup consumerMetricGroup, boolean useMetrics) throws Exception
FlinkKafkaConsumerBase
createFetcher
in class FlinkKafkaConsumerBase<T>
sourceContext
- The source context to emit data to.assignedPartitionsWithInitialOffsets
- The set of partitions that this subtask should
handle, with their start offsets.watermarkStrategy
- Optional, a serialized WatermarkStrategy.runtimeContext
- The task's runtime context.Exception
- The method should forward exceptionsprotected AbstractPartitionDiscoverer createPartitionDiscoverer(KafkaTopicsDescriptor topicsDescriptor, int indexOfThisSubtask, int numParallelSubtasks)
FlinkKafkaConsumerBase
createPartitionDiscoverer
in class FlinkKafkaConsumerBase<T>
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 Map<KafkaTopicPartition,Long> fetchOffsetsWithTimestamp(Collection<KafkaTopicPartition> partitions, long timestamp)
fetchOffsetsWithTimestamp
in class FlinkKafkaConsumerBase<T>
protected boolean getIsAutoCommitEnabled()
getIsAutoCommitEnabled
in class FlinkKafkaConsumerBase<T>
Copyright © 2014–2021 The Apache Software Foundation. All rights reserved.