Stanislav Kozlovski created KAFKA-7984:
------------------------------------------

             Summary: Do not rebuild leader epochs on segments that do not 
support it
                 Key: KAFKA-7984
                 URL: https://issues.apache.org/jira/browse/KAFKA-7984
             Project: Kafka
          Issue Type: Bug
            Reporter: Stanislav Kozlovski
            Assignee: Stanislav Kozlovski


h3. Preface

https://issues.apache.org/jira/browse/KAFKA-7897 (logs would store some leader 
epochs even if they did not support them - this is essentially a regression 
from https://issues.apache.org/jira/browse/KAFKA-7415)
https://issues.apache.org/jira/browse/KAFKA-7959

If users are running Kafka with 
https://issues.apache.org/jira/browse/KAFKA-7415 merged in, chances are they 
have sparsely-populated leader epoch cache files.
KAFKA-7897's implementation unintentionally handled the case of deletes those 
leader epoch cache files for versions 2.1+. For versions below, KAFKA-7959 
fixes that.

In any case, as it currently stands, a broker started up with a message format 
of `0.10.0` will have those leader epoch cache files deleted.


h3. Problem

We have logic [that rebuilds these leader epoch cache 
files|https://github.com/apache/kafka/blob/217f45ed554b34d5221e1dd3db76e4be892661cf/core/src/main/scala/kafka/log/Log.scala#L614]
 when recovering segments that do not have a clean shutdown file. It goes over 
the record batches and rebuilds the leader epoch.
KAFKA-7959's implementation guards against this by checking that the 
log.message.format supports it, *but* that issue is only merged for versions 
*below 2.1*.

Moreover, the case where `log.message.format >= 0.11` *is not handled*. If a 
broker has the following log segment file:
{code:java}
offset 0, format v2, epoch 1
offset 1, format v2, epoch 1
offset 2, format v1, no epoch
offset 3, format v1, no epoch
{code}
and gets upgraded to a new log message format that supports it, the rebuild of 
any logs that had an unclean shutdown will populate the leader epoch cache 
again, potentially resulting in the issue described in KAFKA-7897

One potential simple way to solve this is to clear the accumulated leader epoch 
cache when encountering a batch with no epoch upon segment rebuilding.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to