Consumer
A consumer subscribes to Kafka topics and passes the messages into an Akka Stream.
The underlying implementation is using the KafkaConsumer
, see Kafka API for a description of consumer groups, offsets, and other details.
Choosing a consumer
Alpakka Kafka offers a large variety of consumers that connect to Kafka and stream data. The tables below may help you to find the consumer best suited for your use-case.
Consumers
These factory methods are part of the Consumer APIConsumer API.
Offsets handling | Partition aware | Subscription | Shared consumer | Factory method | Stream element type |
---|---|---|---|---|---|
No (auto commit can be enabled) | No | Topic or Partition | No | plainSource |
ConsumerRecord |
No (auto commit can be enabled) | No | Partition | Yes | plainExternalSource |
ConsumerRecord |
Explicit committing | No | Topic or Partition | No | committableSource |
CommittableMessage |
Explicit committing | No | Partition | Yes | committableExternalSource |
CommittableMessage |
Explicit committing with metadata | No | Topic or Partition | No | commitWithMetadataSource |
CommittableMessage |
Explicit committing (with metadata) | No | Topic or Partition | No | sourceWithOffsetContext |
ConsumerRecord |
Offset committed per element | No | Topic or Partition | No | atMostOnceSource |
ConsumerRecord |
No (auto commit can be enabled) | Yes | Topic or Partition | No | plainPartitionedSource |
(TopicPartition, Source[ConsumerRecord, ..]) |
External to Kafka | Yes | Topic or Partition | No | plainPartitionedManualOffsetSource |
(TopicPartition, Source[ConsumerRecord, ..]) |
Explicit committing | Yes | Topic or Partition | No | committablePartitionedSource |
(TopicPartition, Source[CommittableMessage, ..]) |
Explicit committing with metadata | Yes | Topic or Partition | No | commitWithMetadataPartitionedSource |
(TopicPartition, Source[CommittableMessage, ..]) |
Transactional consumers
These factory methods are part of the Transactional APITransactional API. For details see Transactions.
Offsets handling | Partition aware | Shared consumer | Factory method | Stream element type |
---|---|---|---|---|
Transactional | No | No | Transactional.source |
TransactionalMessage |
Transactional | No | No | Transactional.sourceWithOffsetContext |
ConsumerRecord |
Settings
When creating a consumer stream you need to pass in ConsumerSettings
(API) that define things like:
- de-serializers for the keys and values
- bootstrap servers of the Kafka cluster
- group id for the consumer, note that offsets are always committed for a given consumer group
- Kafka consumer tuning parameters
- Scala
-
val config = system.settings.config.getConfig("akka.kafka.consumer") val consumerSettings = ConsumerSettings(config, new StringDeserializer, new ByteArrayDeserializer) .withBootstrapServers(bootstrapServers) .withGroupId("group1") .withProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest")
- Java
-
final Config config = system.settings().config().getConfig("akka.kafka.consumer"); final ConsumerSettings<String, byte[]> consumerSettings = ConsumerSettings.create(config, new StringDeserializer(), new ByteArrayDeserializer()) .withBootstrapServers("localhost:9092") .withGroupId("group1") .withProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
In addition to programmatic construction of the ConsumerSettings
(API) it can also be created from configuration (application.conf
).
When creating ConsumerSettings
with the ActorSystem
(API) settings it uses the config section akka.kafka.consumer
. The format of these settings files are described in the Typesafe Config Documentation.
# Properties for akka.kafka.ConsumerSettings can be
# defined in this section or a configuration section with
# the same layout.
akka.kafka.consumer {
# Tuning property of scheduled polls.
# Controls the interval from one scheduled poll to the next.
poll-interval = 50ms
# Tuning property of the `KafkaConsumer.poll` parameter.
# Note that non-zero value means that the thread that
# is executing the stage will be blocked. See also the `wakup-timeout` setting below.
poll-timeout = 50ms
# The stage will delay stopping the internal actor to allow processing of
# messages already in the stream (required for successful committing).
# Prefer use of `DrainingControl` over a large stop-timeout.
stop-timeout = 30s
# Duration to wait for `KafkaConsumer.close` to finish.
close-timeout = 20s
# If offset commit requests are not completed within this timeout
# the returned Future is completed `CommitTimeoutException`.
# The `Transactional.source` waits this ammount of time for the producer to mark messages as not
# being in flight anymore as well as waiting for messages to drain, when rebalance is triggered.
commit-timeout = 15s
# If commits take longer than this time a warning is logged
commit-time-warning = 1s
# Not used anymore (since 1.0-RC1)
# wakeup-timeout = 3s
# Not used anymore (since 1.0-RC1)
# max-wakeups = 10
# If set to a finite duration, the consumer will re-send the last committed offsets periodically
# for all assigned partitions. See https://issues.apache.org/jira/browse/KAFKA-4682.
commit-refresh-interval = infinite
# Not used anymore (since 1.0-RC1)
# wakeup-debug = true
# Fully qualified config path which holds the dispatcher configuration
# to be used by the KafkaConsumerActor. Some blocking may occur.
use-dispatcher = "akka.kafka.default-dispatcher"
# Properties defined by org.apache.kafka.clients.consumer.ConsumerConfig
# can be defined in this configuration section.
kafka-clients {
# Disable auto-commit by default
enable.auto.commit = false
}
# Time to wait for pending requests when a partition is closed
wait-close-partition = 500ms
# Limits the query to Kafka for a topic's position
position-timeout = 5s
# When using `AssignmentOffsetsForTimes` subscriptions: timeout for the
# call to Kafka's API
offset-for-times-timeout = 5s
# Timeout for akka.kafka.Metadata requests
# This value is used instead of Kafka's default from `default.api.timeout.ms`
# which is 1 minute.
metadata-request-timeout = 5s
# Interval for checking that transaction was completed before closing the consumer.
# Used in the transactional flow for exactly-once-semantics processing.
eos-draining-check-interval = 30ms
# Issue warnings when a call to a partition assignment handler method takes
# longer than this.
partition-handler-warning = 5s
# Settings for checking the connection to the Kafka broker. Connection checking uses `listTopics` requests with the timeout
# configured by `consumer.metadata-request-timeout`
connection-checker {
#Flag to turn on connection checker
enable = false
# Amount of attempts to be performed after a first connection failure occurs
# Required, non-negative integer
max-retries = 3
# Interval for the connection check. Used as the base for exponential retry.
check-interval = 15s
# Check interval multiplier for backoff interval
# Required, positive number
backoff-factor = 2.0
}
}
ConsumerSettings
(API) can also be created from any other Config
section with the same layout as above.
The Kafka documentation Consumer Configs lists the settings, their defaults and importance. More detailed explanations are given in the KafkaConsumer API and constants are defined in ConsumerConfig API.
Offset Storage external to Kafka
The Kafka read offset can either be stored in Kafka (see below), or at a data store of your choice.
Consumer.plainSource
(Consumer APIConsumer API) and Consumer.plainPartitionedManualOffsetSource
can be used to emit ConsumerRecord
(Kafka API) elements as received from the underlying KafkaConsumer
. They do not have support for committing offsets to Kafka. When using these Sources, either store an offset externally, or use auto-commit (note that auto-commit is disabled by default).
- Scala
-
consumerSettings .withProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true") .withProperty(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG, "5000")
- Java
-
consumerSettings .withProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true") .withProperty(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG, "5000");
The consumer application doesn’t need to use Kafka’s built-in offset storage, it can store offsets in a store of its own choosing. The primary use case for this is allowing the application to store both the offset and the results of the consumption in the same system in a way that both the results and offsets are stored atomically. This is not always possible, but when it is it will make the consumption fully atomic and give “exactly once” semantics that are stronger than the “at-least-once” semantics you get with Kafka’s offset commit functionality.
- Scala
-
val db = new OffsetStore val control = db.loadOffset().map { fromOffset => Consumer .plainSource( consumerSettings, Subscriptions.assignmentWithOffset( new TopicPartition(topic, /* partition = */ 0) -> fromOffset ) ) .mapAsync(1)(db.businessLogicAndStoreOffset) .toMat(Sink.seq)(Keep.both) .mapMaterializedValue(DrainingControl.apply) .run() } class OffsetStore { def businessLogicAndStoreOffset(record: ConsumerRecord[String, String]): Future[Done] = // ... def loadOffset(): Future[Long] = // ... }
- Java
-
final OffsetStorage db = new OffsetStorage(); CompletionStage<Consumer.Control> controlCompletionStage = db.loadOffset() .thenApply( fromOffset -> Consumer.plainSource( consumerSettings, Subscriptions.assignmentWithOffset( new TopicPartition(topic, partition0), fromOffset)) .mapAsync(1, db::businessLogicAndStoreOffset) .to(Sink.ignore()) .run(materializer)); class OffsetStorage { public CompletionStage<Done> businessLogicAndStoreOffset( ConsumerRecord<String, String> record) { // ... } public CompletionStage<Long> loadOffset() { // ... } }
For Consumer.plainSource
the Subscriptions.assignmentWithOffset
specifies the starting point (offset) for a given consumer group id, topic and partition. The group id is defined in the ConsumerSettings
.
Alternatively, with Consumer.plainPartitionedManualOffsetSource
(Consumer APIConsumer API), only the consumer group id and the topic are required on creation. The starting point is fetched by calling the getOffsetsOnAssign
function passed in by the user. This function should return a Map
of TopicPartition
(API) to Long
, with the Long
representing the starting point. If a consumer is assigned a partition that is not included in the Map
that results from getOffsetsOnAssign
, the default starting position will be used, according to the consumer configuration value auto.offset.reset
. Also note that Consumer.plainPartitionedManualOffsetSource
emits tuples of assigned topic-partition and a corresponding source, as in Source per partition.
Offset Storage in Kafka - committing
The Consumer.committableSource
(Consumer APIConsumer API) makes it possible to commit offset positions to Kafka. Compared to auto-commit this gives exact control of when a message is considered consumed.
This is useful when “at-least-once” delivery is desired, as each message will likely be delivered one time, but in failure cases could be received more than once.
- Scala
-
val control = Consumer .committableSource(consumerSettings, Subscriptions.topics(topic)) .mapAsync(10) { msg => business(msg.record.key, msg.record.value).map(_ => msg.committableOffset) } .via(Committer.flow(committerDefaults.withMaxBatch(1))) .toMat(Sink.seq)(Keep.both) .mapMaterializedValue(DrainingControl.apply) .run() def business(key: String, value: Array[Byte]): Future[Done] = // ???
- Java
-
CompletionStage<String> business(String key, String value) { // .... } Consumer.DrainingControl<Done> control = Consumer.committableSource(consumerSettings, Subscriptions.topics(topic)) .mapAsync( 1, msg -> business(msg.record().key(), msg.record().value()) .thenApply(done -> msg.committableOffset())) .toMat(Committer.sink(committerSettings.withMaxBatch(1)), Keep.both()) .mapMaterializedValue(Consumer::createDrainingControl) .run(materializer);
Committing the offset for each message (withMaxBatch(1)
) as illustrated above is rather slow. It is recommended to batch the commits for better throughput, with the trade-off that more messages may be re-delivered in case of failures.
Committer sink
You can use a pre-defined Committer.sink
to perform commits in batches:
- Scala
-
val committerSettings = CommitterSettings(system) val control: DrainingControl[Done] = Consumer .committableSource(consumerSettings, Subscriptions.topics(topic)) .mapAsync(1) { msg => business(msg.record.key, msg.record.value) .map(_ => msg.committableOffset) } .toMat(Committer.sink(committerSettings))(Keep.both) .mapMaterializedValue(DrainingControl.apply) .run()
- Java
-
CommitterSettings committerSettings = CommitterSettings.create(config); Consumer.DrainingControl<Done> control = Consumer.committableSource(consumerSettings, Subscriptions.topics(topic)) .mapAsync( 1, msg -> business(msg.record().key(), msg.record().value()) .<ConsumerMessage.Committable>thenApply(done -> msg.committableOffset())) .toMat(Committer.sink(committerSettings), Keep.both()) .mapMaterializedValue(Consumer::createDrainingControl) .run(materializer);
When creating a Committer.sink
you need to pass in CommitterSettings
(API). These may be created by passing the actor system to read the defaults from the config section akka.kafka.committer
, or by passing a Config
(API) instance with the same structure.
- Table
-
Setting Description Default Value maxBatch maximum number of messages to commit at once 1000 maxInterval maximum interval between commits 10 seconds parallelism parallelsim for async committing 1 - reference.conf
-
# Properties for akka.kafka.CommitterSettings can be # defined in this section or a configuration section with # the same layout. akka.kafka.committer { # Maximum number of messages in a single commit batch max-batch = 1000 # Maximum interval between commits max-interval = 10s # Parallelsim for async committing parallelism = 1 }
The bigger the values are, the less load you put on Kafka and the smaller are chances that committing offsets will become a bottleneck. However, increasing these values also means that in case of a failure you will have to re-process more messages.
If you consume from a topic with low activity, and possibly no messages arrive for more than 24 hours, consider enabling periodical commit refresh (akka.kafka.consumer.commit-refresh-interval
configuration parameters), otherwise offsets might expire in the Kafka storage.
Committer variants
These factory methods are part of the Committer APICommitter API.
Factory method | Stream element type | Emits |
---|---|---|
sink |
Committable |
N/A |
sinkWithOffsetContext |
Any (CommittableOffset in context) |
N/A |
flow |
Committable |
Done |
batchFlow |
Committable |
CommittableOffsetBatch |
flowWithOffsetContext |
Any (CommittableOffset in context) |
NotUsed (CommittableOffsetBatch in context) |
Commit with meta-data
The Consumer.commitWithMetadataSource
allows you to add metadata to the committed offset based on the last consumed record.
Note that the first offset provided to the consumer during a partition assignment will not contain metadata. This offset can get committed due to a periodic commit refresh (akka.kafka.consumer.commit-refresh-interval
configuration parmeters) and the commit will not contain metadata.
- Scala
-
def metadataFromRecord(record: ConsumerRecord[String, String]): String = record.timestamp().toString val control = Consumer .commitWithMetadataSource(consumerSettings, Subscriptions.topics(topic), metadataFromRecord) .mapAsync(1) { msg => business(msg.record.key, msg.record.value) .map(_ => msg.committableOffset) } .toMat(Committer.sink(committerDefaults))(Keep.both) .mapMaterializedValue(DrainingControl.apply) .run()
- Java
-
Consumer.DrainingControl<Done> control = Consumer.commitWithMetadataSource( consumerSettings, Subscriptions.topics(topic), (record) -> Long.toString(record.timestamp())) .mapAsync( 1, msg -> business(msg.record().key(), msg.record().value()) .thenApply(done -> msg.committableOffset())) .toMat(Committer.sink(committerSettings), Keep.both()) .mapMaterializedValue(Consumer::createDrainingControl) .run(materializer);
If you commit the offset before processing the message you get “at-most-once” delivery semantics, this is provided by Consumer.atMostOnceSource
. However, atMostOnceSource
commits the offset for each message and that is rather slow, batching of commits is recommended.
- Scala
-
val control: DrainingControl[immutable.Seq[Done]] = Consumer .atMostOnceSource(consumerSettings, Subscriptions.topics(topic)) .mapAsync(1)(record => business(record.key, record.value())) .toMat(Sink.seq)(Keep.both) .mapMaterializedValue(DrainingControl.apply) .run() def business(key: String, value: String): Future[Done] = // ???
- Java
-
Consumer.Control control = Consumer.atMostOnceSource(consumerSettings, Subscriptions.topics(topic)) .mapAsync(10, record -> business(record.key(), record.value())) .to(Sink.foreach(it -> System.out.println("Done with " + it))) .run(materializer); CompletionStage<String> business(String key, String value) { // .... }
Maintaining at-least-once delivery semantics requires care, many risks and solutions are covered in At-Least-Once Delivery.
Connecting Producer and Consumer
For cases when you need to read messages from one topic, transform or enrich them, and then write to another topic you can use Consumer.committableSource
and connect it to a Producer.committableSink
. The committableSink
will commit the offset back to the consumer when it has successfully published the message.
The committableSink
accepts implementations ProducerMessage.Envelope
(API) that contain the offset to commit the consumption of the originating message (of type ConsumerMessage.Committable
(API)). See Producing messages about different implementations of Envelope
supported.
Note that there is a risk that something fails after publishing but before committing, so committableSink
has “at-least-once” delivery semantics.
- Scala
-
val control = Consumer .committableSource(consumerSettings, Subscriptions.topics(topic1, topic2)) .map { msg => ProducerMessage.single( new ProducerRecord(targetTopic, msg.record.key, msg.record.value), msg.committableOffset ) } .toMat(Producer.committableSink(producerSettings))(Keep.both) .mapMaterializedValue(DrainingControl.apply) .run()
- Java
-
Consumer.DrainingControl<Done> control = Consumer.committableSource(consumerSettings, Subscriptions.topics(topic1, topic2)) .map( msg -> ProducerMessage.<String, String, ConsumerMessage.Committable>single( new ProducerRecord<>(targetTopic, msg.record().key(), msg.record().value()), msg.committableOffset())) .toMat(Producer.committableSink(producerSettings), Keep.both()) .mapMaterializedValue(Consumer::createDrainingControl) .run(materializer);
As Producer.committableSink
’s committing of messages one-by-one is rather slow, prefer a flow together with batching of commits with Committer.sink
.
- Scala
-
val control = Consumer .committableSource(consumerSettings, Subscriptions.topics(topic)) .map { msg => ProducerMessage.single( new ProducerRecord(targetTopic, msg.record.key, msg.record.value), passThrough = msg.committableOffset ) } .via(Producer.flexiFlow(producerSettings)) .map(_.passThrough) .toMat(Committer.sink(committerSettings))(Keep.both) .mapMaterializedValue(DrainingControl.apply) .run()
- Java
-
Consumer.DrainingControl<Done> control = Consumer.committableSource(consumerSettings, Subscriptions.topics(topic)) .map( msg -> ProducerMessage.single( new ProducerRecord<>(targetTopic, msg.record().key(), msg.record().value()), msg.committableOffset() // the passThrough )) .via(Producer.flexiFlow(producerSettings)) .map(m -> m.passThrough()) .toMat(Committer.sink(committerSettings), Keep.both()) .mapMaterializedValue(Consumer::createDrainingControl) .run(materializer);
There is a risk that something fails after publishing, but before committing, so committableSink
has “at-least-once” delivery semantics.
To get delivery guarantees, please read about transactions.
Source per partition
Consumer.plainPartitionedSource
(Consumer APIConsumer API) , Consumer.committablePartitionedSource
, and Consumer.commitWithMetadataPartitionedSource
support tracking the automatic partition assignment from Kafka. When a topic-partition is assigned to a consumer, this source will emit a tuple with the assigned topic-partition and a corresponding source. When a topic-partition is revoked, the corresponding source completes.
- Scala
-
val control = Consumer .committablePartitionedSource(consumerSettings, Subscriptions.topics(topic)) .flatMapMerge(maxPartitions, _._2) .via(businessFlow) .map(_.committableOffset) .toMat(Committer.sink(committerDefaults))(Keep.both) .mapMaterializedValue(DrainingControl.apply) .run()
- Java
-
Consumer.DrainingControl<Done> control = Consumer.committablePartitionedSource(consumerSettings, Subscriptions.topics(topic)) .flatMapMerge(maxPartitions, Pair::second) .via(business()) .map(msg -> msg.committableOffset()) .toMat(Committer.sink(committerSettings), Keep.both()) .mapMaterializedValue(Consumer::createDrainingControl) .run(materializer);
Separate streams per partition:
- Scala
-
val control = Consumer .committablePartitionedSource(consumerSettings, Subscriptions.topics(topic)) .mapAsyncUnordered(maxPartitions) { case (topicPartition, source) => source .via(businessFlow) .map(_.committableOffset) .runWith(Committer.sink(comitterSettings)) } .toMat(Sink.ignore)(Keep.both) .mapMaterializedValue(DrainingControl.apply) .run()
- Java
-
Consumer.DrainingControl<Done> control = Consumer.committablePartitionedSource(consumerSettings, Subscriptions.topics(topic)) .mapAsyncUnordered( maxPartitions, pair -> { Source<ConsumerMessage.CommittableMessage<String, String>, NotUsed> source = pair.second(); return source .via(business()) .map(message -> message.committableOffset()) .runWith(Committer.sink(committerSettings), materializer); }) .toMat(Sink.ignore(), Keep.both()) .mapMaterializedValue(Consumer::createDrainingControl) .run(materializer);
Join flows based on automatically assigned partitions:
- Scala
-
type Msg = CommittableMessage[String, Array[Byte]] def zipper(left: Source[Msg, _], right: Source[Msg, _]): Source[(Msg, Msg), NotUsed] = ??? Consumer .committablePartitionedSource(consumerSettings, Subscriptions.topics("topic1")) .map { case (topicPartition, source) => // get corresponding partition from other topic val otherTopicPartition = new TopicPartition("otherTopic", topicPartition.partition()) val otherSource = Consumer.committableSource(consumerSettings, Subscriptions.assignment(otherTopicPartition)) zipper(source, otherSource) } .flatMapMerge(maxPartitions, identity) .via(businessFlow) //build commit offsets .batch( max = 20, seed = { case (left, right) => ( CommittableOffsetBatch(left.committableOffset), CommittableOffsetBatch(right.committableOffset) ) } )( aggregate = { case ((batchL, batchR), (l, r)) => batchL.updated(l.committableOffset) batchR.updated(r.committableOffset) (batchL, batchR) } ) .mapAsync(1) { case (l, r) => l.commitScaladsl().map(_ => r) } .mapAsync(1)(_.commitScaladsl()) .runWith(Sink.ignore)
Sharing the KafkaConsumer instance
If you have many streams it can be more efficient to share the underlying KafkaConsumer
(Kafka API) instance. It is shared by creating a KafkaConsumerActor
(API). You need to create the actor and stop it by sending KafkaConsumerActor.Stop
when it is not needed any longer. You pass the ActorRef
as a parameter to the Consumer
(Consumer APIConsumer API) factory methods.
- Scala
-
//Consumer is represented by actor val consumer: ActorRef = system.actorOf(KafkaConsumerActor.props(consumerSettings)) //Manually assign topic partition to it val (controlPartition1, result1) = Consumer .plainExternalSource[String, Array[Byte]]( consumer, Subscriptions.assignment(new TopicPartition(topic, partition1)) ) .via(businessFlow) .toMat(Sink.seq)(Keep.both) .run() //Manually assign another topic partition val (controlPartition2, result2) = Consumer .plainExternalSource[String, Array[Byte]]( consumer, Subscriptions.assignment(new TopicPartition(topic, partition2)) ) .via(businessFlow) .toMat(Sink.seq)(Keep.both) .run() // .... controlPartition1.shutdown() controlPartition2.shutdown() consumer ! KafkaConsumerActor.Stop
- Java
-
// Consumer is represented by actor ActorRef consumer = system.actorOf((KafkaConsumerActor.props(consumerSettings))); // Manually assign topic partition to it Consumer.Control controlPartition1 = Consumer.plainExternalSource( consumer, Subscriptions.assignment(new TopicPartition(topic, partition0))) .via(business()) .to(Sink.ignore()) .run(materializer); // Manually assign another topic partition Consumer.Control controlPartition2 = Consumer.plainExternalSource( consumer, Subscriptions.assignment(new TopicPartition(topic, partition1))) .via(business()) .to(Sink.ignore()) .run(materializer); consumer.tell(KafkaConsumerActor.stop(), self);
Accessing KafkaConsumer metrics
You can access the underlying consumer metrics via the materialized Control
instance:
- Scala
-
val control: Consumer.Control = Consumer .plainSource(consumerSettings, Subscriptions.assignment(new TopicPartition(topic, partition))) .via(businessFlow) .to(Sink.ignore) .run() val metrics: Future[Map[MetricName, Metric]] = control.metrics metrics.foreach(map => println(s"metrics: ${map.mkString("\n")}"))
- Java
-
// run the stream to obtain the materialized Control value Consumer.DrainingControl<Done> control = Consumer.plainSource( consumerSettings, Subscriptions.assignment(new TopicPartition(topic, 0))) .via(business()) .toMat(Sink.ignore(), Keep.both()) .mapMaterializedValue(Consumer::createDrainingControl) .run(materializer); CompletionStage<Map<MetricName, Metric>> metrics = control.getMetrics(); metrics.thenAccept(map -> System.out.println("Metrics: " + map));
Accessing KafkaConsumer metadata
Accessing of Kafka consumer metadata is possible as described in Consumer Metadata.
Listening for rebalance events
You may set up an rebalance event listener actor that will be notified when your consumer will be assigned or revoked from consuming from specific topic partitions. Two kinds of messages will be sent to this listener actor
akka.kafka.TopicPartitionsAssigned
andakka.kafka.TopicPartitionsRevoked
, like this:
- Scala
-
import akka.kafka.{TopicPartitionsAssigned, TopicPartitionsRevoked} class RebalanceListener extends Actor with ActorLogging { def receive: Receive = { case TopicPartitionsAssigned(subscription, topicPartitions) => log.info("Assigned: {}", topicPartitions) case TopicPartitionsRevoked(subscription, topicPartitions) => log.info("Revoked: {}", topicPartitions) } } val rebalanceListener = system.actorOf(Props(new RebalanceListener)) val subscription = Subscriptions .topics(topic) // additionally, pass the actor reference: .withRebalanceListener(rebalanceListener) // use the subscription as usual: Consumer .plainSource(consumerSettings, subscription)
- Java
-
static class RebalanceListener extends AbstractLoggingActor { @Override public Receive createReceive() { return receiveBuilder() .match( TopicPartitionsAssigned.class, assigned -> { log().info("Assigned: {}", assigned); }) .match( TopicPartitionsRevoked.class, revoked -> { log().info("Revoked: {}", revoked); }) .build(); } } ActorRef rebalanceListener = system.actorOf(Props.create(RebalanceListener.class)); Subscription subscription = Subscriptions.topics(topic) // additionally, pass the actor reference: .withRebalanceListener(rebalanceListener); // use the subscription as usual: Consumer.DrainingControl<List<ConsumerRecord<String, String>>> control = Consumer.plainSource(consumerSettings, subscription) .toMat(Sink.seq(), Keep.both()) .mapMaterializedValue(Consumer::createDrainingControl) .run(materializer);
Controlled shutdown
The Source
created with Consumer.plainSource
and similar methods materializes to a Consumer.Control
(APIAPI) instance. This can be used to stop the stream in a controlled manner.
When using external offset storage, a call to Consumer.Control.shutdown()
suffices to complete the Source
, which starts the completion of the stream.
- Scala
-
val (consumerControl, streamComplete) = Consumer .plainSource(consumerSettings, Subscriptions.assignmentWithOffset( new TopicPartition(topic, 0) -> offset )) .via(businessFlow) .toMat(Sink.ignore)(Keep.both) .run() consumerControl.shutdown()
- Java
-
final OffsetStorage db = new OffsetStorage(); CompletionStage<Consumer.DrainingControl<Done>> control = db.loadOffset() .thenApply( fromOffset -> Consumer.plainSource( consumerSettings, Subscriptions.assignmentWithOffset( new TopicPartition(topic, 0), fromOffset)) .mapAsync( 10, record -> business(record.key(), record.value()) .thenApply(res -> db.storeProcessedOffset(record.offset()))) .toMat(Sink.ignore(), Keep.both()) .mapMaterializedValue(Consumer::createDrainingControl) .run(materializer)); // Shutdown the consumer when desired control.thenAccept(c -> c.drainAndShutdown(executor));
When you are using offset storage in Kafka, the shutdown process involves several steps:
Consumer.Control.stop()
to stop producing messages from theSource
. This does not stop the underlying Kafka Consumer.- Wait for the stream to complete, so that a commit request has been made for all offsets of all processed messages (via
Committer.sink/flow
,commitScaladsl()
orcommitJavadsl()
). Consumer.Control.shutdown()
to wait for all outstanding commit requests to finish and stop the Kafka Consumer.
Draining control
To manage this shutdown process, use the Consumer.DrainingControl
(APIAPI) by combining the Consumer.Control
with the sink’s materialized completion future in mapMaterializedValue
. That control offers the method drainAndShutdown
which implements the process descibed above.
Note: The ConsummerSettings
stop-timeout
delays stopping the Kafka Consumer and the stream, but when using drainAndShutdown
that delay is not required and can be set to zero (as below).
- Scala
-
val drainingControl = Consumer .committableSource(consumerSettings.withStopTimeout(Duration.Zero), Subscriptions.topics(topic)) .mapAsync(1) { msg => business(msg.record).map(_ => msg.committableOffset) } .toMat(Committer.sink(committerSettings))(Keep.both) .mapMaterializedValue(DrainingControl.apply) .run() val streamComplete = drainingControl.drainAndShutdown()
- Java
-
final Executor ec = Executors.newCachedThreadPool(); Consumer.DrainingControl<Done> control = Consumer.committableSource( consumerSettings.withStopTimeout(Duration.ZERO), Subscriptions.topics(topic)) .mapAsync( 1, msg -> business(msg.record().key(), msg.record().value()) .thenApply(done -> msg.committableOffset())) .toMat(Committer.sink(committerSettings.withMaxBatch(1)), Keep.both()) .mapMaterializedValue(Consumer::createDrainingControl) .run(materializer); control.drainAndShutdown(ec);