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

sandeep gupta commented on KAFKA-3410:
--------------------------------------

I also encountered the same issue. Is there any solution for this. I don't want 
to lose any data.
In our kafka based network, there are 4 kafka brokers and 3 zookeepers running 
as docker containers, we have 3 channels and one orderer system channel 
testchainid in our network. Everything was working fine till sunday. However 
after that we saw errors in ordering service during invokations. After that we 
restarted our zookeepers services and then restarted kafka0, kafka1, kafka2 and 
kafka3 in the same order maintaining 10 secs gap after every kafka restart. 
This process we used to do (roughly every 3 weeks) whenever we faced such 
issue. 
However this time when we did the same process, kafka2 and kafka1 got shut down 
after restart and when we checked the logs we found this error *FATAL 
[ReplicaFetcher replicaId=2, leaderId=0, fetcherId=0] Exiting because log 
truncation is not allowed for partition testchainid-0, current leader's latest 
offset 96672 is less than replica's latest offset 96674 
(kafka.server.ReplicaFetcherThread)* on broker2 and same error we found on 
broker1 as well. so basically we have two channels on broker0 as leader - ort 
and testchainid and rest of the channels are present on other brokers. 
Also when we stopped kafka0 broker and then restart kafka1, kafka2 and kafka3, 
then kafka1 and kafka2 didn't get shut down. So the problem is as soon as i 
restart kafka0 broker, then the brokers kafka1 and kafka2 get shut down 
immediately.
Now with kafka0 stopped and rest of the brokers kafka1, kafka2, and kafka3 
running, though I am able to invoke on rest of the other channels but I am able 
to see error in orderer logs for the ort channel *[orderer/consensus/kafka] 
processMessagesToBlocks -> ERRO 3a0123d [channel: ort] Error during 
consumption: kafka: error while consuming ort/0: kafka server: In the middle of 
a leadership election, there is currently no leader for this partition and 
hence it is unavailable for writes.*
 

> 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
>          Components: replication
>            Reporter: James Cheng
>            Priority: Major
>              Labels: reliability
>
> 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
(v7.6.3#76005)

Reply via email to