[ https://issues.apache.org/jira/browse/KAFKA-9212?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16990283#comment-16990283 ]
ASF GitHub Bot commented on KAFKA-9212: --------------------------------------- hachikuji commented on pull request #7795: KAFKA-9212; Update LeaderAndIsr state in controller context after reassignment URL: https://github.com/apache/kafka/pull/7795 KIP-320 improved fetch semantics by adding leader epoch validation. This relies on reliable propagation of leader epoch information from the controller. Unfortunately, we have encountered a bug during partition reassignment in which the leader epoch in the controller context does not get properly updated. This causes UpdateMetadata requests to be sent with stale epoch information which results in the metadata caches on the brokers falling out of sync. This bug has existed for a long time, but it is only a problem due to the new epoch validation done by the client. Because the client includes the stale leader epoch in its requests, the leader rejects them, but the stale metadata cache on the brokers prevents the consumer from getting the latest epoch. Hence the consumer cannot make progress while a reassignment is ongoing. Although it is straightforward to fix this problem in the controller for the new releases (which is what this patch does), it is not so easy to fix older brokers which means new clients could still encounter brokers with this bug. To address this problem, this patch also modifies the client to treat the leader epoch returned from the Metadata response as "unreliable" if it comes from an older version of the protocol. The client in this case we discard the returned epoch and it won't be included in any requests. Also, note that the correct epoch is still forwarded to replicas correctly in the LeaderAndIsr request, so this bug does not affect replication. ### Committer Checklist (excluded from commit message) - [ ] Verify design and implementation - [ ] Verify test coverage and CI build status - [ ] Verify documentation (including upgrade notes) ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org > Keep receiving FENCED_LEADER_EPOCH while sending ListOffsetRequest > ------------------------------------------------------------------ > > Key: KAFKA-9212 > URL: https://issues.apache.org/jira/browse/KAFKA-9212 > Project: Kafka > Issue Type: Bug > Components: consumer, offset manager > Affects Versions: 2.3.0 > Environment: Linux > Reporter: Yannick > Assignee: Jason Gustafson > Priority: Critical > > When running Kafka connect s3 sink connector ( confluent 5.3.0), after one > broker got restarted (leaderEpoch updated at this point), the connect worker > crashed with the following error : > [2019-11-19 16:20:30,097] ERROR [Worker clientId=connect-1, > groupId=connect-ls] Uncaught exception in herder work thread, exiting: > (org.apache.kafka.connect.runtime.distributed.DistributedHerder:253) > org.apache.kafka.common.errors.TimeoutException: Failed to get offsets by > times in 30003ms > > After investigation, it seems it's because it got fenced when sending > ListOffsetRequest in loop and then got timed out , as follows : > [2019-11-19 16:20:30,020] DEBUG [Consumer clientId=consumer-3, > groupId=connect-ls] Sending ListOffsetRequest (type=ListOffsetRequest, > replicaId=-1, partitionTimestamps={connect_ls_config-0={timestamp: -1, > maxNumOffsets: 1, currentLeaderEpoch: Optional[1]}}, > isolationLevel=READ_UNCOMMITTED) to broker kafka6.fra2.internal:9092 (id: 4 > rack: null) (org.apache.kafka.clients.consumer.internals.Fetcher:905) > [2019-11-19 16:20:30,044] DEBUG [Consumer clientId=consumer-3, > groupId=connect-ls] Attempt to fetch offsets for partition > connect_ls_config-0 failed due to FENCED_LEADER_EPOCH, retrying. > (org.apache.kafka.clients.consumer.internals.Fetcher:985) > > The above happens multiple times until timeout. > > According to the debugs, the consumer always get a leaderEpoch of 1 for this > topic when starting up : > > [2019-11-19 13:27:30,802] DEBUG [Consumer clientId=consumer-3, > groupId=connect-ls] Updating last seen epoch from null to 1 for partition > connect_ls_config-0 (org.apache.kafka.clients.Metadata:178) > > > But according to our brokers log, the leaderEpoch should be 2, as follows : > > [2019-11-18 14:19:28,988] INFO [Partition connect_ls_config-0 broker=4] > connect_ls_config-0 starts at Leader Epoch 2 from offset 22. Previous Leader > Epoch was: 1 (kafka.cluster.Partition) > > > This make impossible to restart the worker as it will always get fenced and > then finally timeout. > > It is also impossible to consume with a 2.3 kafka-console-consumer as > follows : > > kafka-console-consumer --bootstrap-server BOOTSTRAPSERVER:9092 --topic > connect_ls_config --from-beginning > > the above will just hang forever ( which is not expected cause there is > data) and we can see those debug messages : > [2019-11-19 22:17:59,124] DEBUG [Consumer clientId=consumer-1, > groupId=console-consumer-3844] Attempt to fetch offsets for partition > connect_ls_config-0 failed due to FENCED_LEADER_EPOCH, retrying. > (org.apache.kafka.clients.consumer.internals.Fetcher) > > > Interesting fact, if we do subscribe the same way with kafkacat (1.5.0) we > can consume without problem ( must be the way kafkacat is consuming ignoring > FENCED_LEADER_EPOCH): > > kafkacat -b BOOTSTRAPSERVER:9092 -t connect_ls_config -o beginning > > -- This message was sent by Atlassian Jira (v8.3.4#803005)