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

James Cheng commented on KAFKA-3410:
------------------------------------

[~junrao], [~tcrayford-heroku], can I get your opinions on the priority of this 
bug? It seems like a high priority issue to me. I believe that this bug means 
replacing a disk on a single broker has the potential to take down an entire 
Kafka cluster.  That's what happened to [~maysamyabandeh] in [this 
comment|l#comment-15362701].

[~gwenshap] said that the correct use of min.isr will avoid you getting into 
this scenario. But the correct use of min.isr requires participation from both 
of the broker-side and the producer side. On the broker-side, your topic(s) 
need to be configured with min.isr. On the producer-side, all producers to 
those topics need to be using acks=all (or acks=-1)

Since it is dependent on the client-side, that means that the operator of the 
Kafka cluster currently has no way to avoid hitting this scenario. Or rather, 
the only way to avoid this scenario seems to be enabling unclean leader 
election. I'd rather not do that, if possible.

> Unclean leader election and "Halting because log truncation is not allowed"
> ---------------------------------------------------------------------------
>
>                 Key: KAFKA-3410
>                 URL: https://issues.apache.org/jira/browse/KAFKA-3410
>             Project: Kafka
>          Issue Type: Bug
>            Reporter: James Cheng
>
> I ran into a scenario where one of my brokers would continually shutdown, 
> with the error message:
> [2016-02-25 00:29:39,236] FATAL [ReplicaFetcherThread-0-1], Halting because 
> log truncation is not allowed for topic test, Current leader 1's latest 
> offset 0 is less than replica 2's latest offset 151 
> (kafka.server.ReplicaFetcherThread)
> I managed to reproduce it with the following scenario:
> 1. Start broker1, with unclean.leader.election.enable=false
> 2. Start broker2, with unclean.leader.election.enable=false
> 3. Create topic, single partition, with replication-factor 2.
> 4. Write data to the topic.
> 5. At this point, both brokers are in the ISR. Broker1 is the partition 
> leader.
> 6. Ctrl-Z on broker2. (Simulates a GC pause or a slow network) Broker2 gets 
> dropped out of ISR. Broker1 is still the leader. I can still write data to 
> the partition.
> 7. Shutdown Broker1. Hard or controlled, doesn't matter.
> 8. rm -rf the log directory of broker1. (This simulates a disk replacement or 
> full hardware replacement)
> 9. Resume broker2. It attempts to connect to broker1, but doesn't succeed 
> because broker1 is down. At this point, the partition is offline. Can't write 
> to it.
> 10. Resume broker1. Broker1 resumes leadership of the topic. Broker2 attempts 
> to join ISR, and immediately halts with the error message:
> [2016-02-25 00:29:39,236] FATAL [ReplicaFetcherThread-0-1], Halting because 
> log truncation is not allowed for topic test, Current leader 1's latest 
> offset 0 is less than replica 2's latest offset 151 
> (kafka.server.ReplicaFetcherThread)
> I am able to recover by setting unclean.leader.election.enable=true on my 
> brokers.
> I'm trying to understand a couple things:
> * In step 10, why is broker1 allowed to resume leadership even though it has 
> no data?
> * In step 10, why is it necessary to stop the entire broker due to one 
> partition that is in this state? Wouldn't it be possible for the broker to 
> continue to serve traffic for all the other topics, and just mark this one as 
> unavailable?
> * Would it make sense to allow an operator to manually specify which broker 
> they want to become the new master? This would give me more control over how 
> much data loss I am willing to handle. In this case, I would want broker2 to 
> become the new master. Or, is that possible and I just don't know how to do 
> it?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to