You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by "Lucas Bradstreet (Jira)" <ji...@apache.org> on 2020/08/25 17:02:00 UTC

[jira] [Created] (KAFKA-10432) LeaderEpochCache is incorrectly recovered on segment recovery for epoch 0

Lucas Bradstreet created KAFKA-10432:
----------------------------------------

             Summary: LeaderEpochCache is incorrectly recovered on segment recovery for epoch 0
                 Key: KAFKA-10432
                 URL: https://issues.apache.org/jira/browse/KAFKA-10432
             Project: Kafka
          Issue Type: Bug
    Affects Versions: 2.6.0, 2.5.0, 2.4.0, 2.3.0
            Reporter: Lucas Bradstreet


I added some functionality to the system tests to compare epoch cache lineages ([https://github.com/apache/kafka/pull/9213]), and I found a bug in leader epoch cache recovery.

The test hard kills a broker and the cache hasn't been flushed yet, and then it starts up and goes through log recovery. After recovery there is divergence in the epoch caches for epoch 0:
{noformat}
AssertionError: leader epochs for output-topic-1 didn't match
 [{0: 9393L, 2: 9441L, 4: 42656L},
 {0: 0L, 2: 9441L, 4: 42656L}, 
 {0: 0L, 2: 9441L, 4: 42656L}]                                                                                                                                                                
{noformat}
The cache is supposed to include the offset for epoch 0 but in recovery it skips it [https://github.com/apache/kafka/blob/487b3682ebe0eefde3445b37ee72956451a9d15e/core/src/main/scala/kafka/log/LogSegment.scala#L364] due to [https://github.com/apache/kafka/commit/d152989f26f51b9004b881397db818ad6eaf0392]. Then it stamps the epoch with a later offset when fetching from the leader.

I'm not sure why the recovery code includes the condition `batch.partitionLeaderEpoch > 0`. I discussed this with Jason Gustafson and he believes it may have been intended to avoid assigning negative epochs but is not sure why it was added. None of the tests fail with this check removed.
{noformat}
          leaderEpochCache.foreach { cache =>
            if (batch.partitionLeaderEpoch > 0 && cache.latestEpoch.forall(batch.partitionLeaderEpoch > _))
              cache.assign(batch.partitionLeaderEpoch, batch.baseOffset)
          }
{noformat}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)