Open
Description
We are using reactive-kafka (Using com.typesafe.akka:akka-stream-kafka_2.11:0.14) with kafka 10 brokers and are frequently seeing consumers come up (usually immediately following the termination of another consumer in the group) and having no partitions assigned. Logs look like this:
2017-05-11 01:55:49,394 INFO o.a.k.c.consumer.ConsumerConfig - ConsumerConfig values:
auto.commit.interval.ms = 5000
auto.offset.reset = earliest
bootstrap.servers = [192.168.1.1:9093, 192.168.1.2:9093, 192.168.1.3:9093]
check.crcs = true
client.id = consumer-1
connections.max.idle.ms = 540000
enable.auto.commit = false
exclude.internal.topics = true
fetch.max.bytes = 52428800
fetch.max.wait.ms = 500
fetch.min.bytes = 1
group.id = my-group
heartbeat.interval.ms = 3000
interceptor.classes = null
key.deserializer = class org.apache.kafka.common.serialization.ByteArrayDeserializer
max.partition.fetch.bytes = 1048576
max.poll.interval.ms = 300000
max.poll.records = 500
metadata.max.age.ms = 300000
metric.reporters = [com.my.app.adapters.akkalib.metrics.KafkaDropwizardMetricsReporter]
metrics.num.samples = 2
metrics.recording.level = INFO
metrics.sample.window.ms = 30000
partition.assignment.strategy = [class org.apache.kafka.clients.consumer.RangeAssignor]
receive.buffer.bytes = 65536
reconnect.backoff.ms = 50
request.timeout.ms = 305000
retry.backoff.ms = 100
sasl.jaas.config = null
sasl.kerberos.kinit.cmd = /usr/bin/kinit
sasl.kerberos.min.time.before.relogin = 60000
sasl.kerberos.service.name = null
sasl.kerberos.ticket.renew.jitter = 0.05
sasl.kerberos.ticket.renew.window.factor = 0.8
sasl.mechanism = GSSAPI
security.protocol = SSL
send.buffer.bytes = 131072
session.timeout.ms = 10000
ssl.cipher.suites = null
ssl.enabled.protocols = [TLSv1.2, TLSv1.1, TLSv1]
ssl.endpoint.identification.algorithm = null
ssl.key.password = [hidden]
ssl.keymanager.algorithm = SunX509
ssl.keystore.location = /etc/app/server.jks
ssl.keystore.password = [hidden]
ssl.keystore.type = JKS
ssl.protocol = TLS
ssl.provider = null
ssl.secure.random.implementation = null
ssl.trustmanager.algorithm = PKIX
ssl.truststore.location = /etc/ssl/app.truststore.jks
ssl.truststore.password = [hidden]
ssl.truststore.type = JKS
value.deserializer = class org.apache.kafka.common.serialization.ByteArrayDeserializer
2017-05-11 01:55:49,474 INFO o.a.kafka.common.utils.AppInfoParser - Kafka version : 0.10.2.0
2017-05-11 01:55:49,477 INFO o.a.kafka.common.utils.AppInfoParser - Kafka commitId : 576d93a8dc0cf421
2017-05-11 01:55:49,600 WARN o.a.k.c.consumer.ConsumerConfig - The configuration 'metric.tag' was supplied but isn't a known config.
2017-05-11 01:55:49,600 INFO o.a.kafka.common.utils.AppInfoParser - Kafka version : 0.10.2.0
2017-05-11 01:55:49,600 INFO o.a.kafka.common.utils.AppInfoParser - Kafka commitId : 576d93a8dc0cf421
2017-05-11 01:55:51,899 INFO o.a.k.c.c.i.AbstractCoordinator - Discovered coordinator foo.bar.baz:9093 (id: 123 rack: null) for group my-group.
2017-05-11 01:55:51,976 INFO o.a.k.c.c.i.ConsumerCoordinator - Revoking previously assigned partitions [] for group my-group
2017-05-11 01:55:51,978 INFO o.a.k.c.c.i.AbstractCoordinator - (Re-)joining group my-group
2017-05-11 01:55:52,801 WARN akka.kafka.KafkaConsumerActor - Consumer interrupted with WakeupException after timeout. Message: null. Current value of akka.kafka.consumer.wakeup-timeout is 3000 milliseconds
2017-05-11 01:55:52,995 INFO o.a.k.c.c.i.AbstractCoordinator - Successfully joined group my-group with generation 1
2017-05-11 02:52:49,210 INFO c.m.a.GoogleCredentialService - Refreshing Google auth token
2017-05-11 02:52:49,404 INFO c.m.a.GoogleCredentialService - Refreshed Google auth token, scheduling to refresh again in 3420s
Relevant code looks like this:
Consumer.committablePartitionedSource(settings, Subscriptions.topics(config.getTopic())).watchTermination(Keep.both())
.via(killSwitch.flow())
.via(sourceFilter.limiterFlow())
.flatMapMerge(1000, r -> r.second()
.filter(this::filter)
.map(this::parseRecord)
.filter(Optional::isPresent)
.map(Optional::get)
.mapAsync(1, this::sendRequest)
.map(this::handleResponse))
.map(StreamMessage::getKafkaMessage)
.batch(
1000,
first -> ConsumerMessage.emptyCommittableOffsetBatch().updated(first.committableOffset()),
(batch, elem) -> batch.updated(elem.committableOffset()))
.mapAsync(10, ConsumerMessage.Committable::commitJavadsl)
.runWith(Sink.ignore(), materializer);
My apologies for lack of runnable test case, etc. (that will soon follow if need be); just wanted to get this out there in case there is a known issue behind it or something obviously wrong with how we're using the library. Thanks!
Activity
jpeel commentedon Jun 15, 2017
We are seeing this sometimes as well. I think that the wake up time out should be much higher. It currently is 3 seconds, but the default heartbeat in kafka is also 3 seconds. The heartbeat frequency is a factor in how long it takes to rebalance so it makes sense to me that the wake up time out should be significantly higher, perhaps close to the default session timeout of 10 seconds.
romanwozniak commentedon Jun 19, 2017
I'm facing something similar issue in my project:
This happens randomly after couple of hours (or even days) of running.
The reason looks to be this:
(com.typesafe.akka:akka-stream-kafka_2.12:0.15 with kafka brokers 0.10)
rafalbigaj commentedon Jul 13, 2017
+1
I experience exactly the same problem quite randomly after wakeup:
After:
Successfully joined group test-UsageMetricsSpec-group-1 with generation 5
consumer gets stuck forever
wojda commentedon Aug 5, 2017
In my case increasing wakeup timeout didn't help. Consumers get stuck rarely but they still get stuck. It is a big issue because Kafka brokers think partitions are assign to the consumer. The only way to "unclog" the consumer is to trigger a group rebalance. Is the way reactive-kafka uses a wakeup mechanism an issue or vanilla kafka consumer implementation?
anilkumar763 commentedon Aug 7, 2017
I am also facing this issue. Thrice I saw in the past month.
The rebalance happens frequently in my application. But at times, I see 0 partitions assinged (I subsrcibed to 1 partition)
Does it take time to get update that the partition is already revoked ??
Should we poll again , if this arises ??
jpeel commentedon Aug 8, 2017
I am pretty confident that the issue is that the poll initiated by the KafkaConsumerActor is grabbing the metadata and joining the group, but the wakeupexception interrupts this process after the group is joined but before the callbacks (ConsumerRebalanceListener) are called which are used to tell the SingleSourceLogic stuff which topics are subscribed to. The SingleSourceListener makes requests to the KafkaConsumerActor for the topic partitions that it knows about.. but they either aren't the right ones or are empty because the callback was never called. To be clear, the KafkaConsumer has been assigned partitions in this case, but because the SingleSourceLogic doesn't know about them, they are never consumed from.
I'm not fully sure how the code should be changed to deal with this problem, but the workaround we are implementing right now is to set the wake up timeout a little longer (6 seconds or so), set the max wakeup timeouts to 1, and restart the stream whenever the max wakeup timeouts is encountered.
nivox commentedon Oct 27, 2017
We're facing the same issue.
We have a service which sometime stops receiving data right after a WakeupException.
The issue doesn't happen on every WakeupException, but when it happens it's always after a WakeupException.
The explanation of @jpeel fit pretty well with our observation.
We'll try and implement the proposed workaround, but a more structured way of solving the issue (without restarting the stream) would be definitely appreciated.
Has there been any activity on this issue?
nivox commentedon Oct 27, 2017
Ping @patriknw @kciesielski.
Given you're the last to have worked on the code, do you have any insights?
patriknw commentedon Oct 31, 2017
Defaults:
So you mean that increasing the wakeup-timeout to let's say 9s would be different than trying 3 times with 3s?
The reason it's bad to have long timeout is that it's blocking a thread during that time, and that can easily result in starvation and unresponsiveness of other streams running on the same dispatcher. But sure, if that solves the problem please go ahead and create a PR.
Note that the reason for retrying at all, compared to just failing the stream immediately and only rely on stream restart is that then it can hopefully survive short glitches without loosing the state.
nivox commentedon Oct 31, 2017
@patriknw I think the point of @jpeel was that sometimes when the timeout occurs it somehow leaves Kafka consumer in a state where no messages are being received (even though there are messages available) and the stream becomes stuck. The proposed workaround of raising the timeout and reset the stream was to circumvent the problem but indeed it is NOT a solution.
I think that the retry mechanism is the right way to go, as you said it would prevent losing the stream state. Given the thesis of @jpeel on why the streams gets stuck, do you think it is plausible? Do you have any idea on how to avoid the issue while maintaining the retry mechanism?
I'm willing to try and put together a PR, but I don't really know the internals so any guidance would be welcome.
patriknw commentedon Nov 7, 2017
Ok, read his comment again. Is there a way we can reset things completely after a wakeup interupt? So that it is treated as a new consumer in next attempt? Or can we get the assigned partitions with a merhod call after wakeup, since we might have missed the callback. Other ideas?
Topic subscription re-initialized on WakeUpException.
Topic subscriptions re-inizialized on WakeupException.
nivox commentedon Nov 8, 2017
@patriknw @jpeel I've tried to put together an implementation of the first strategy you proposed: reset things completely after a wakeup. commit
The base idea is to just unsubscribe and re-subscribe right away to all the topics. This way the assignment callbacks will be called and the state of the Single/SubSourceLogic should be consistent. Note that I had to introduce another callback just to clean the state of the Single/SubSourceLogic to make things consistent.
While I was working on this it came to me that another strategy we could use is to:
This strategy would still require to keep some book keeping on the KafkaConsumerActor side but would free the Single/SubSourceLogic from any change. This requires that the internal state of the KafkaConsumer is not corrupted after the wakeup, which I think is a reasonable assumption to make.
I'll try to work on this approach in the next few days, in the meantime could someone take a look at the current proposal and give me some feedback?
25 remaining items