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

2019-03-07 Thread Stanislav Kozlovski (JIRA)


[ 
https://issues.apache.org/jira/browse/KAFKA-8036?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16786939#comment-16786939
 ] 

Stanislav Kozlovski edited comment on KAFKA-8036 at 3/7/19 4:23 PM:


I was wrong. After writing a test to verify the behavior 
([https://github.com/stanislavkozlovski/kafka/commit/b93a866e2318c47ff7538ddecd2fa38f29abd188|https://github.com/stanislavkozlovski/kafka/commit/b93a866e2318c47ff7538ddecd2fa38f29abd188),]),
 I found that KAFKA-7897 fixed this issue across all versions it affected.

Resolving this...


was (Author: enether):
I was wrong. After writing a test to verify the behavior 
([https://github.com/stanislavkozlovski/kafka/commit/b93a866e2318c47ff7538ddecd2fa38f29abd188|https://github.com/stanislavkozlovski/kafka/commit/b93a866e2318c47ff7538ddecd2fa38f29abd188),]),
 I found that KAFKA-7897 fixed this issue across all versions it affected.

> 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.1.1, 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.
> h2.  
> h2. Exception and logs
>  I reproduced this locally by running two brokers. The steps to reproduce: 
> {code:java}
> Create partition replicated across two brokers (A, B) with leader A
> Move partition leadership to B
> Alter log dirs on A
> Move partition leadership back to A{code}
> This results in a log directory structure on broker B similar to this:
> {code:java}
> ├── new_dir
> │   ├── cleaner-offset-checkpoint
> │   ├── log-start-offset-checkpoint
> │   ├── meta.properties
> │   ├── recovery-point-offset-checkpoint
> │   ├── replication-offset-checkpoint
> │   └── test_log_dir-0
> │   ├── .index
> │   ├── .log
> │   ├── .timeindex
> │   └── leader-epoch-checkpoint
> └── old_dir
>   ├── cleaner-offset-checkpoint
>   ├── log-start-offset-checkpoint
>   ├── meta.properties
>   ├── recovery-point-offset-checkpoint
>   ├── replication-offset-checkpoint
>   └── test_log_dir-0.32e77c96939140f9a56a49b75ad8ec8d-delete
> ├── .index
> ├── .log
> ├── .timeindex
> ├── 0009.snapshot
> └── leader-epoch-checkpoint
> {code}
>  
>  
> {code:java}
> [2019-03-04 15:36:56,854] INFO [Partition test_log_dir-0 broker=0] 
> test_log_dir-0 starts at Leader Epoch 3 from offset 9. Previous Leader Epoch 
> was: 2 (kafka.cluster.Partition) [2019-03-04 15:36:56,855] WARN 
> [LeaderEpochCache test_log_dir-0] New epoch entry EpochEntry(epoch=3, 
> startOffset=9) caused truncation of conflicting entries 
> ListBuffer(EpochEntry(epoch=1, startOffset=9)). Cache now contains 2 entries. 
> (kafka.server.epoch.LeaderEpochFileCache) [2019-03-04 15:36:56,857] ERROR 
> Error while writing to checkpoint file 
> /logs/old_dir/test_log_dir-0/leader-epoch-checkpoint 
> (kafka.server.LogDirFailureChannel) java.io.FileNotFoundException: 
> /logs/old_dir/test_log_dir-0/leader-epoch-checkpoint.tmp (No such file or 
> directory) at java.base/java.io.FileOutputStream.open0(Native Method) at 
> java.base/java.io.FileOutputStream.open(FileOutputStream.java:299) at 
> java.base/java.io.FileOutputStream.(FileO

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

2019-03-07 Thread Stanislav Kozlovski (JIRA)


[ 
https://issues.apache.org/jira/browse/KAFKA-8036?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16786939#comment-16786939
 ] 

Stanislav Kozlovski edited comment on KAFKA-8036 at 3/7/19 4:23 PM:


I was wrong. After writing a test to verify the behavior 
([https://github.com/stanislavkozlovski/kafka/commit/b93a866e2318c47ff7538ddecd2fa38f29abd188|https://github.com/stanislavkozlovski/kafka/commit/b93a866e2318c47ff7538ddecd2fa38f29abd188),]),
 I found that KAFKA-7897 fixed this issue across all versions it affected.


was (Author: enether):
I was wrong. After writing a test to verify the behavior 
([https://github.com/stanislavkozlovski/kafka/commit/b93a866e2318c47ff7538ddecd2fa38f29abd188),]
 I found that KAFKA-7897 fixed this issue across all versions it affected.

> 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.1.1, 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.
> h2.  
> h2. Exception and logs
>  I reproduced this locally by running two brokers. The steps to reproduce: 
> {code:java}
> Create partition replicated across two brokers (A, B) with leader A
> Move partition leadership to B
> Alter log dirs on A
> Move partition leadership back to A{code}
> This results in a log directory structure on broker B similar to this:
> {code:java}
> ├── new_dir
> │   ├── cleaner-offset-checkpoint
> │   ├── log-start-offset-checkpoint
> │   ├── meta.properties
> │   ├── recovery-point-offset-checkpoint
> │   ├── replication-offset-checkpoint
> │   └── test_log_dir-0
> │   ├── .index
> │   ├── .log
> │   ├── .timeindex
> │   └── leader-epoch-checkpoint
> └── old_dir
>   ├── cleaner-offset-checkpoint
>   ├── log-start-offset-checkpoint
>   ├── meta.properties
>   ├── recovery-point-offset-checkpoint
>   ├── replication-offset-checkpoint
>   └── test_log_dir-0.32e77c96939140f9a56a49b75ad8ec8d-delete
> ├── .index
> ├── .log
> ├── .timeindex
> ├── 0009.snapshot
> └── leader-epoch-checkpoint
> {code}
>  
>  
> {code:java}
> [2019-03-04 15:36:56,854] INFO [Partition test_log_dir-0 broker=0] 
> test_log_dir-0 starts at Leader Epoch 3 from offset 9. Previous Leader Epoch 
> was: 2 (kafka.cluster.Partition) [2019-03-04 15:36:56,855] WARN 
> [LeaderEpochCache test_log_dir-0] New epoch entry EpochEntry(epoch=3, 
> startOffset=9) caused truncation of conflicting entries 
> ListBuffer(EpochEntry(epoch=1, startOffset=9)). Cache now contains 2 entries. 
> (kafka.server.epoch.LeaderEpochFileCache) [2019-03-04 15:36:56,857] ERROR 
> Error while writing to checkpoint file 
> /logs/old_dir/test_log_dir-0/leader-epoch-checkpoint 
> (kafka.server.LogDirFailureChannel) java.io.FileNotFoundException: 
> /logs/old_dir/test_log_dir-0/leader-epoch-checkpoint.tmp (No such file or 
> directory) at java.base/java.io.FileOutputStream.open0(Native Method) at 
> java.base/java.io.FileOutputStream.open(FileOutputStream.java:299) at 
> java.base/java.io.FileOutputStream.(FileOutputStream.java:238) at 
> java.base/java.io.FileOutputStream.(FileOutputStream.java:188) at 
> kafka.server.che