This connector provides access to event streams served by Apache Kafka.
Flink provides special Kafka Connectors for reading and writing data from/to Kafka topics. The Flink Kafka Consumer integrates with Flink’s checkpointing mechanism to provide exactly-once processing semantics. To achieve that, Flink does not purely rely on Kafka’s consumer group offset tracking, but tracks and checkpoints these offsets internally as well.
Please pick a package (maven artifact id) and class name for your use-case and environment.
For most users, the FlinkKafkaConsumer08
(part of flink-connector-kafka
) is appropriate.
Maven Dependency | Supported since | Consumer and Producer Class name |
Kafka version | Notes |
---|---|---|---|---|
flink-connector-kafka-0.8_2.11 | 1.0.0 | FlinkKafkaConsumer08 FlinkKafkaProducer08 |
0.8.x | Uses the SimpleConsumer API of Kafka internally. Offsets are committed to ZK by Flink. |
flink-connector-kafka-0.9_2.11 | 1.0.0 | FlinkKafkaConsumer09 FlinkKafkaProducer09 |
0.9.x | Uses the new Consumer API Kafka. |
flink-connector-kafka-0.10_2.11 | 1.2.0 | FlinkKafkaConsumer010 FlinkKafkaProducer010 |
0.10.x | This connector supports Kafka messages with timestamps both for producing and consuming. |
flink-connector-kafka-0.11_2.11 | 1.4.0 | FlinkKafkaConsumer011 FlinkKafkaProducer011 |
0.11.x | Since 0.11.x Kafka does not support scala 2.10. This connector supports Kafka transactional messaging to provide exactly once semantic for the producer. |
flink-connector-kafka_2.11 | 1.7.0 | FlinkKafkaConsumer FlinkKafkaProducer |
>= 1.0.0 |
This universal Kafka connector attempts to track the latest version of the Kafka client.
The version of the client it uses may change between Flink releases.
Modern Kafka clients are backwards compatible with broker versions 0.10.0 or later.
However for Kafka 0.11.x and 0.10.x versions, we recommend using dedicated
flink-connector-kafka-0.11_2.11 and link-connector-kafka-0.10_2.11 respectively.
Attention: as of Flink 1.7 the universal Kafka connector is considered to be
in a BETA status and might not be as stable as the 0.11 connector.
In case of problems with the universal connector, you can try to use flink-connector-kafka-0.11_2.11
which should be compatible with all of the Kafka versions starting from 0.11.
|
Then, import the connector in your maven project:
Note that the streaming connectors are currently not part of the binary distribution. See how to link with them for cluster execution here.
advertised.host.name
setting in the config/server.properties
file must be set to the machine’s IP address.Starting with Flink 1.7, there is a new universal Kafka connector that does not track a specific Kafka major version. Rather, it tracks the latest version of Kafka at the time of the Flink release.
If your Kafka broker version is 1.0.0 or newer, you should use this Kafka connector. If you use an older version of Kafka (0.11, 0.10, 0.9, or 0.8), you should use the connector corresponding to the broker version.
The universal Kafka connector is compatible with older and newer Kafka brokers through the compatibility guarantees of the Kafka client API and broker. It is compatible with broker versions 0.11.0 or newer, depending on the features used. For details on Kafka compatibility, please refer to the Kafka documentation.
To use the universal Kafka connector add a dependency to it:
Then instantiate the new source (FlinkKafkaConsumer
) and sink (FlinkKafkaProducer
).
The API is backward compatible with the Kafka 0.11 connector,
except of dropping specific Kafka version from the module and class names.
Flink’s Kafka consumer is called FlinkKafkaConsumer08
(or 09 for Kafka 0.9.0.x versions, etc.
or just FlinkKafkaConsumer
for Kafka >= 1.0.0 versions). It provides access to one or more Kafka topics.
The constructor accepts the following arguments:
Example:
DeserializationSchema
The Flink Kafka Consumer needs to know how to turn the binary data in Kafka into Java/Scala objects. The
DeserializationSchema
allows users to specify such a schema. The T deserialize(byte[] message)
method gets called for each Kafka message, passing the value from Kafka.
It is usually helpful to start from the AbstractDeserializationSchema
, which takes care of describing the
produced Java/Scala type to Flink’s type system. Users that implement a vanilla DeserializationSchema
need
to implement the getProducedType(...)
method themselves.
For accessing both the key and value of the Kafka message, the KeyedDeserializationSchema
has
the following deserialize method ` T deserialize(byte[] messageKey, byte[] message, String topic, int partition, long offset)`.
For convenience, Flink provides the following schemas:
TypeInformationSerializationSchema
(and TypeInformationKeyValueSerializationSchema
) which creates
a schema based on a Flink’s TypeInformation
. This is useful if the data is both written and read by Flink.
This schema is a performant Flink-specific alternative to other generic serialization approaches.
JsonDeserializationSchema
(and JSONKeyValueDeserializationSchema
) which turns the serialized JSON
into an ObjectNode object, from which fields can be accessed using objectNode.get(“field”).as(Int/String/…)().
The KeyValue objectNode contains a “key” and “value” field which contain all fields, as well as
an optional “metadata” field that exposes the offset/partition/topic for this message.
AvroDeserializationSchema
which reads data serialized with Avro format using a statically provided schema. It can
infer the schema from Avro generated classes (AvroDeserializationSchema.forSpecific(...)
) or it can work with GenericRecords
with a manually provided schema (with AvroDeserializationSchema.forGeneric(...)
). This deserialization schema expects that
the serialized records DO NOT contain embedded schema.
ConfluentRegistryAvroDeserializationSchema.forGeneric(...)
or ConfluentRegistryAvroDeserializationSchema.forSpecific(...)
).
To use this deserialization schema one has to add the following additional dependency:
When encountering a corrupted message that cannot be deserialized for any reason, there
are two options - either throwing an exception from the deserialize(...)
method
which will cause the job to fail and be restarted, or returning null
to allow
the Flink Kafka consumer to silently skip the corrupted message. Note that
due to the consumer’s fault tolerance (see below sections for more details),
failing the job on the corrupted message will let the consumer attempt
to deserialize the message again. Therefore, if deserialization still fails, the
consumer will fall into a non-stop restart and fail loop on that corrupted
message.
The Flink Kafka Consumer allows configuring how the start position for Kafka partitions are determined.
Example:
All versions of the Flink Kafka Consumer have the above explicit configuration methods for start position.
setStartFromGroupOffsets
(default behaviour): Start reading partitions from
the consumer group’s (group.id
setting in the consumer properties) committed
offsets in Kafka brokers (or Zookeeper for Kafka 0.8). If offsets could not be
found for a partition, the auto.offset.reset
setting in the properties will be used.setStartFromEarliest()
/ setStartFromLatest()
: Start from the earliest / latest
record. Under these modes, committed offsets in Kafka will be ignored and
not used as starting positions.setStartFromTimestamp(long)
: Start from the specified timestamp. For each partition, the record
whose timestamp is larger than or equal to the specified timestamp will be used as the start position.
If a partition’s latest record is earlier than the timestamp, the partition will simply be read
from the latest record. Under this mode, committed offsets in Kafka will be ignored and not used as
starting positions.You can also specify the exact offsets the consumer should start from for each partition:
The above example configures the consumer to start from the specified offsets for
partitions 0, 1, and 2 of topic myTopic
. The offset values should be the
next record that the consumer should read for each partition. Note that
if the consumer needs to read a partition which does not have a specified
offset within the provided offsets map, it will fallback to the default
group offsets behaviour (i.e. setStartFromGroupOffsets()
) for that
particular partition.
Note that these start position configuration methods do not affect the start position when the job is automatically restored from a failure or manually restored using a savepoint. On restore, the start position of each Kafka partition is determined by the offsets stored in the savepoint or checkpoint (please see the next section for information about checkpointing to enable fault tolerance for the consumer).
With Flink’s checkpointing enabled, the Flink Kafka Consumer will consume records from a topic and periodically checkpoint all its Kafka offsets, together with the state of other operations, in a consistent manner. In case of a job failure, Flink will restore the streaming program to the state of the latest checkpoint and re-consume the records from Kafka, starting from the offsets that were stored in the checkpoint.
The interval of drawing checkpoints therefore defines how much the program may have to go back at most, in case of a failure.
To use fault tolerant Kafka Consumers, checkpointing of the topology needs to be enabled at the execution environment:
Also note that Flink can only restart the topology if enough processing slots are available to restart the topology. So if the topology fails due to loss of a TaskManager, there must still be enough slots available afterwards. Flink on YARN supports automatic restart of lost YARN containers.
If checkpointing is not enabled, the Kafka consumer will periodically commit the offsets to Zookeeper.
The Flink Kafka Consumer supports discovering dynamically created Kafka partitions, and consumes them with exactly-once guarantees. All partitions discovered after the initial retrieval of partition metadata (i.e., when the job starts running) will be consumed from the earliest possible offset.
By default, partition discovery is disabled. To enable it, set a non-negative value
for flink.partition-discovery.interval-millis
in the provided properties config,
representing the discovery interval in milliseconds.
Limitation When the consumer is restored from a savepoint from Flink versions prior to Flink 1.3.x, partition discovery cannot be enabled on the restore run. If enabled, the restore would fail with an exception. In this case, in order to use partition discovery, please first take a savepoint in Flink 1.3.x and then restore again from that.
At a higher-level, the Flink Kafka Consumer is also capable of discovering topics, based on pattern matching on the topic names using regular expressions. See the below for an example:
In the above example, all topics with names that match the specified regular expression
(starting with test-topic-
and ending with a single digit) will be subscribed by the consumer
when the job starts running.
To allow the consumer to discover dynamically created topics after the job started running,
set a non-negative value for flink.partition-discovery.interval-millis
. This allows
the consumer to discover partitions of new topics with names that also match the specified
pattern.
The Flink Kafka Consumer allows configuring the behaviour of how offsets are committed back to Kafka brokers (or Zookeeper in 0.8). Note that the Flink Kafka Consumer does not rely on the committed offsets for fault tolerance guarantees. The committed offsets are only a means to expose the consumer’s progress for monitoring purposes.
The way to configure offset commit behaviour is different, depending on whether or not checkpointing is enabled for the job.
Checkpointing disabled: if checkpointing is disabled, the Flink Kafka
Consumer relies on the automatic periodic offset committing capability
of the internally used Kafka clients. Therefore, to disable or enable offset
committing, simply set the enable.auto.commit
(or auto.commit.enable
for Kafka 0.8) / auto.commit.interval.ms
keys to appropriate values
in the provided Properties
configuration.
Checkpointing enabled: if checkpointing is enabled, the Flink Kafka
Consumer will commit the offsets stored in the checkpointed states when
the checkpoints are completed. This ensures that the committed offsets
in Kafka brokers is consistent with the offsets in the checkpointed states.
Users can choose to disable or enable offset committing by calling the
setCommitOffsetsOnCheckpoints(boolean)
method on the consumer (by default,
the behaviour is true
).
Note that in this scenario, the automatic periodic offset committing
settings in Properties
is completely ignored.
In many scenarios, the timestamp of a record is embedded (explicitly or implicitly) in the record itself.
In addition, the user may want to emit watermarks either periodically, or in an irregular fashion, e.g. based on
special records in the Kafka stream that contain the current event-time watermark. For these cases, the Flink Kafka
Consumer allows the specification of an AssignerWithPeriodicWatermarks
or an AssignerWithPunctuatedWatermarks
.
You can specify your custom timestamp extractor/watermark emitter as described here, or use one from the predefined ones. After doing so, you can pass it to your consumer in the following way:
Internally, an instance of the assigner is executed per Kafka partition.
When such an assigner is specified, for each record read from Kafka, the
extractTimestamp(T element, long previousElementTimestamp)
is called to assign a timestamp to the record and
the Watermark getCurrentWatermark()
(for periodic) or the
Watermark checkAndGetNextWatermark(T lastElement, long extractedTimestamp)
(for punctuated) is called to determine
if a new watermark should be emitted and with which timestamp.
Note: If a watermark assigner depends on records read from Kafka to advance its watermarks (which is commonly the case), all topics and partitions need to have a continuous stream of records. Otherwise, the watermarks of the whole application cannot advance and all time-based operations, such as time windows or functions with timers, cannot make progress. A single idle Kafka partition causes this behavior. A Flink improvement is planned to prevent this from happening (see FLINK-5479: Per-partition watermarks in FlinkKafkaConsumer should consider idle partitions). In the meanwhile, a possible workaround is to send heartbeat messages to all consumed partitions that advance the watermarks of idle partitions.
Flink’s Kafka Producer is called FlinkKafkaProducer011
(or 010
for Kafka 0.10.0.x versions, etc. or just FlinkKafkaProducer
for Kafka >= 1.0.0 versions).
It allows writing a stream of records to one or more Kafka topics.
Example:
The above examples demonstrate the basic usage of creating a Flink Kafka Producer to write streams to a single Kafka target topic. For more advanced usages, there are other constructor variants that allow providing the following:
KafkaProducer
.
Please refer to the Apache Kafka documentation for
details on how to configure Kafka Producers.FlinkKafkaPartitioner
to the
constructor. This partitioner will be called for each record in the stream
to determine which exact partition of the target topic the record should be sent to.
Please see Kafka Producer Partitioning Scheme for more details.KeyedSerializationSchema
,
which allows serializing the key and value separately. It also allows to override the target topic,
so that one producer instance can send data to multiple topics.By default, if a custom partitioner is not specified for the Flink Kafka Producer, the producer will use
a FlinkFixedPartitioner
that maps each Flink Kafka Producer parallel subtask to a single Kafka partition
(i.e., all records received by a sink subtask will end up in the same Kafka partition).
A custom partitioner can be implemented by extending the FlinkKafkaPartitioner
class. All
Kafka versions’ constructors allow providing a custom partitioner when instantiating the producer.
Note that the partitioner implementation must be serializable, as they will be transferred across Flink nodes.
Also, keep in mind that any state in the partitioner will be lost on job failures since the partitioner
is not part of the producer’s checkpointed state.
It is also possible to completely avoid using and kind of partitioner, and simply let Kafka partition
the written records by their attached key (as determined for each record using the provided serialization schema).
To do this, provide a null
custom partitioner when instantiating the producer. It is important
to provide null
as the custom partitioner; as explained above, if a custom partitioner is not specified
the FlinkFixedPartitioner
is used instead.
Before 0.9 Kafka did not provide any mechanisms to guarantee at-least-once or exactly-once semantics.
With Flink’s checkpointing enabled, the FlinkKafkaProducer09
and FlinkKafkaProducer010
can provide at-least-once delivery guarantees.
Besides enabling Flink’s checkpointing, you should also configure the setter
methods setLogFailuresOnly(boolean)
and setFlushOnCheckpoint(boolean)
appropriately.
setLogFailuresOnly(boolean)
: by default, this is set to false
.
Enabling this will let the producer only log failures
instead of catching and rethrowing them. This essentially accounts the record
to have succeeded, even if it was never written to the target Kafka topic. This
must be disabled for at-least-once.setFlushOnCheckpoint(boolean)
: by default, this is set to true
.
With this enabled, Flink’s checkpoints will wait for any
on-the-fly records at the time of the checkpoint to be acknowledged by Kafka before
succeeding the checkpoint. This ensures that all records before the checkpoint have
been written to Kafka. This must be enabled for at-least-once.In conclusion, the Kafka producer by default has at-least-once guarantees for versions
0.9 and 0.10, with setLogFailureOnly
set to false
and setFlushOnCheckpoint
set
to true
.
Note: By default, the number of retries is set to “0”. This means that when setLogFailuresOnly
is set to false
,
the producer fails immediately on errors, including leader changes. The value is set to “0” by default to avoid
duplicate messages in the target topic that are caused by retries. For most production environments with frequent broker changes,
we recommend setting the number of retries to a higher value.
Note: There is currently no transactional producer for Kafka, so Flink can not guarantee exactly-once delivery into a Kafka topic.
With Flink’s checkpointing enabled, the FlinkKafkaProducer011
(FlinkKafkaProducer
for Kafka >= 1.0.0 versions) can provide
exactly-once delivery guarantees.
Besides enabling Flink’s checkpointing, you can also choose three different modes of operating
chosen by passing appropriate semantic
parameter to the FlinkKafkaProducer011
(FlinkKafkaProducer
for Kafka >= 1.0.0 versions):
Semantic.NONE
: Flink will not guarantee anything. Produced records can be lost or they can
be duplicated.Semantic.AT_LEAST_ONCE
(default setting): similar to setFlushOnCheckpoint(true)
in
FlinkKafkaProducer010
. This guarantees that no records will be lost (although they can be duplicated).Semantic.EXACTLY_ONCE
: uses Kafka transactions to provide exactly-once semantic. Whenever you write
to Kafka using transactions, do not forget about setting desired isolation.level
(read_committed
or read_uncommitted
- the latter one is the default value) for any application consuming records
from Kafka.Semantic.EXACTLY_ONCE
mode relies on the ability to commit transactions
that were started before taking a checkpoint, after recovering from the said checkpoint. If the time
between Flink application crash and completed restart is larger then Kafka’s transaction timeout
there will be data loss (Kafka will automatically abort transactions that exceeded timeout time).
Having this in mind, please configure your transaction timeout appropriately to your expected down
times.
Kafka brokers by default have transaction.max.timeout.ms
set to 15 minutes. This property will
not allow to set transaction timeouts for the producers larger then it’s value.
FlinkKafkaProducer011
by default sets the transaction.timeout.ms
property in producer config to
1 hour, thus transaction.max.timeout.ms
should be increased before using the
Semantic.EXACTLY_ONCE
mode.
In read_committed
mode of KafkaConsumer
, any transactions that were not finished
(neither aborted nor completed) will block all reads from the given Kafka topic past any
un-finished transaction. In other words after following sequence of events:
transaction1
and written some records using ittransaction2
and written some further records using ittransaction2
Even if records from transaction2
are already committed, they will not be visible to
the consumers until transaction1
is committed or aborted. This has two implications:
Note: Semantic.EXACTLY_ONCE
mode uses a fixed size pool of KafkaProducers
per each FlinkKafkaProducer011
instance. One of each of those producers is used per one
checkpoint. If the number of concurrent checkpoints exceeds the pool size, FlinkKafkaProducer011
will throw an exception and will fail the whole application. Please configure max pool size and max
number of concurrent checkpoints accordingly.
Note: Semantic.EXACTLY_ONCE
takes all possible measures to not leave any lingering transactions
that would block the consumers from reading from Kafka topic more then it is necessary. However in the
event of failure of Flink application before first checkpoint, after restarting such application there
is no information in the system about previous pool sizes. Thus it is unsafe to scale down Flink
application before first checkpoint completes, by factor larger then FlinkKafkaProducer011.SAFE_SCALE_DOWN_FACTOR
.
Since Apache Kafka 0.10+, Kafka’s messages can carry timestamps, indicating the time the event has occurred (see “event time” in Apache Flink) or the time when the message has been written to the Kafka broker.
The FlinkKafkaConsumer010
will emit records with the timestamp attached, if the time characteristic in Flink is
set to TimeCharacteristic.EventTime
(StreamExecutionEnvironment.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
).
The Kafka consumer does not emit watermarks. To emit watermarks, the same mechanisms as described above in
“Kafka Consumers and Timestamp Extraction/Watermark Emission” using the assignTimestampsAndWatermarks
method are applicable.
There is no need to define a timestamp extractor when using the timestamps from Kafka. The previousElementTimestamp
argument of
the extractTimestamp()
method contains the timestamp carried by the Kafka message.
A timestamp extractor for a Kafka consumer would look like this:
The FlinkKafkaProducer010
only emits the record timestamp, if setWriteTimestampToKafka(true)
is set.
Flink’s Kafka connectors provide some metrics through Flink’s metrics system to analyze the behavior of the connector. The producers export Kafka’s internal metrics through Flink’s metric system for all supported versions. The consumers export all metrics starting from Kafka version 0.9. The Kafka documentation lists all exported metrics in its documentation.
In addition to these metrics, all consumers expose the current-offsets
and committed-offsets
for each topic partition.
The current-offsets
refers to the current offset in the partition. This refers to the offset of the last element that
we retrieved and emitted successfully. The committed-offsets
is the last committed offset.
The Kafka Consumers in Flink commit the offsets back to Zookeeper (Kafka 0.8) or the Kafka brokers (Kafka 0.9+). If checkpointing is disabled, offsets are committed periodically. With checkpointing, the commit happens once all operators in the streaming topology have confirmed that they’ve created a checkpoint of their state. This provides users with at-least-once semantics for the offsets committed to Zookeeper or the broker. For offsets checkpointed to Flink, the system provides exactly once guarantees.
The offsets committed to ZK or the broker can also be used to track the read progress of the Kafka consumer. The difference between the committed offset and the most recent offset in each partition is called the consumer lag. If the Flink topology is consuming the data slower from the topic than new data is added, the lag will increase and the consumer will fall behind. For large production deployments we recommend monitoring that metric to avoid increasing latency.
Flink provides first-class support through the Kafka connector to authenticate to a Kafka installation
configured for Kerberos. Simply configure Flink in flink-conf.yaml
to enable Kerberos authentication for Kafka like so:
security.kerberos.login.use-ticket-cache
: By default, this is true
and Flink will attempt to use Kerberos credentials in ticket caches managed by kinit
.
Note that when using the Kafka connector in Flink jobs deployed on YARN, Kerberos authorization using ticket caches will not work.
This is also the case when deploying using Mesos, as authorization using ticket cache is not supported for Mesos deployments.security.kerberos.login.keytab
and security.kerberos.login.principal
: To use Kerberos keytabs instead, set values for both of these properties.KafkaClient
to security.kerberos.login.contexts
: This tells Flink to provide the configured Kerberos credentials to the Kafka login context to be used for Kafka authentication.Once Kerberos-based Flink security is enabled, you can authenticate to Kafka with either the Flink Kafka Consumer or Producer by simply including the following two settings in the provided properties configuration that is passed to the internal Kafka client:
security.protocol
to SASL_PLAINTEXT
(default NONE
): The protocol used to communicate to Kafka brokers.
When using standalone Flink deployment, you can also use SASL_SSL
; please see how to configure the Kafka client for SSL here.sasl.kerberos.service.name
to kafka
(default kafka
): The value for this should match the sasl.kerberos.service.name
used for Kafka broker configurations.
A mismatch in service name between client and server configuration will cause the authentication to fail.For more information on Flink configuration for Kerberos security, please see here. You can also find here further details on how Flink internally setups Kerberos-based security.
Depending on your Kafka configuration, even after Kafka acknowledges writes you can still experience data loss. In particular keep in mind about the following properties in Kafka config:
acks
log.flush.interval.messages
log.flush.interval.ms
log.flush.*
Default values for the above options can easily lead to data loss. Please refer to the Kafka documentation for more explanation.
One possible cause of this error is when a new leader election is taking place,
for example after or during restarting a Kafka broker.
This is a retriable exception, so Flink job should be able to restart and resume normal operation.
It also can be circumvented by changing retries
property in the producer settings.
However this might cause reordering of messages,
which in turn if undesired can be circumvented by setting max.in.flight.requests.per.connection
to 1.