You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "Stanislav Kozlovski (JIRA)" <ji...@apache.org> on 2019/03/04 16:06:00 UTC

[jira] [Updated] (KAFKA-8036) Log dir reassignment on followers fails with FileNotFoundException for the leader epoch cache on leader election

     [ https://issues.apache.org/jira/browse/KAFKA-8036?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Stanislav Kozlovski updated KAFKA-8036:
---------------------------------------
    Description: 
When changing a partition's log directories for a follower broker, we move all the data related to that partition to the other log dir (as per [KIP-113|https://cwiki.apache.org/confluence/display/KAFKA/KIP-113:+Support+replicas+movement+between+log+directories]). On a successful move, we rename the original directory by adding a suffix consisting of an UUID and `-delete`. (e.g `test_log_dir` would be renamed to `test_log_dir-0.32e77c96939140f9a56a49b75ad8ec8d-delete`)

We copy every log file and [initialize a new leader epoch file cache|https://github.com/apache/kafka/blob/0d56f1413557adabc736cae2dffcdc56a620403e/core/src/main/scala/kafka/log/Log.scala#L768]. The problem is that we do not update the associated `Replica` class' leader epoch cache - it still points to the old `LeaderEpochFileCache` instance.
This results in a FileNotFound exception when the broker is [elected as a leader for the partition|[https://github.com/apache/kafka/blob/255f4a6effdc71c273691859cd26c4138acad778/core/src/main/scala/kafka/cluster/Partition.scala#L312]]. This has the unintended side effect of marking the log directory as offline, resulting in all partitions from that log directory becoming unavailable for the specific broker.

  was:
When changing a partition's log directories for a follower broker, we move all the data related to that partition to the other log dir (as per [KIP-113|https://cwiki.apache.org/confluence/display/KAFKA/KIP-113:+Support+replicas+movement+between+log+directories]). On a successful move, we rename the original directory by adding a suffix consisting of an UUID and `-delete`. (e.g `test_log_dir` would be renamed to `test_log_dir-0.32e77c96939140f9a56a49b75ad8ec8d-delete`)

We copy every log file and [initialize a new leader epoch file cache|https://github.com/apache/kafka/blob/0d56f1413557adabc736cae2dffcdc56a620403e/core/src/main/scala/kafka/log/Log.scala#L768]. The problem is that we do not update the associated `Replica` class' leader epoch cache - it still points to the old `LeaderEpochFileCache` instance.
This results in a FileNotFound exception when the broker is [elected as a leader for the partition|[https://github.com/apache/kafka/blob/255f4a6effdc71c273691859cd26c4138acad778/core/src/main/scala/kafka/cluster/Partition.scala#L312].] This has the unintended side effect of marking the log directory as offline, resulting in all partitions from that log directory becoming unavailable for the specific broker.


> Log dir reassignment on followers fails with FileNotFoundException for the leader epoch cache on leader election
> ----------------------------------------------------------------------------------------------------------------
>
>                 Key: KAFKA-8036
>                 URL: https://issues.apache.org/jira/browse/KAFKA-8036
>             Project: Kafka
>          Issue Type: Improvement
>    Affects Versions: 1.0.2, 1.1.0, 2.0.1
>            Reporter: Stanislav Kozlovski
>            Assignee: Stanislav Kozlovski
>            Priority: Major
>
> When changing a partition's log directories for a follower broker, we move all the data related to that partition to the other log dir (as per [KIP-113|https://cwiki.apache.org/confluence/display/KAFKA/KIP-113:+Support+replicas+movement+between+log+directories]). On a successful move, we rename the original directory by adding a suffix consisting of an UUID and `-delete`. (e.g `test_log_dir` would be renamed to `test_log_dir-0.32e77c96939140f9a56a49b75ad8ec8d-delete`)
> We copy every log file and [initialize a new leader epoch file cache|https://github.com/apache/kafka/blob/0d56f1413557adabc736cae2dffcdc56a620403e/core/src/main/scala/kafka/log/Log.scala#L768]. The problem is that we do not update the associated `Replica` class' leader epoch cache - it still points to the old `LeaderEpochFileCache` instance.
> This results in a FileNotFound exception when the broker is [elected as a leader for the partition|[https://github.com/apache/kafka/blob/255f4a6effdc71c273691859cd26c4138acad778/core/src/main/scala/kafka/cluster/Partition.scala#L312]]. This has the unintended side effect of marking the log directory as offline, resulting in all partitions from that log directory becoming unavailable for the specific broker.



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