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

Dmitriy Matveev commented on KAFKA-3410:
----------------------------------------

We have faced the same problem.

it's happened like this, all action i did was in Cloudera Manager:

- 3 days before(it was friday) i did shrink zookeeper cluster and removed two 
nodes. restarted whole cluster of zookeeper.
- kafka stay in state, something like this, - "Outdate configuration need to 
restart"
- today morning(monday) kafka run into trouble - two nodes lost connection to 
zookeeper, and some thing went wrong. Logs one of the nodes:
{code:java}
2018-01-29 03:37:17,480 INFO org.apache.zookeeper.ClientCnxn: Client session 
timed out, have not heard from server in 22837ms for sessionid 
0x3612d6a573501c1, closing socket connection and attempting reco
n nect 
2018-01-29 03:37:17,480 INFO org.apache.zookeeper.ClientCnxn: Client session 
timed out, have not heard from server in 27981ms for sessionid 
0x7612d6a562101dc, closing socket connection and attempting reco
n nect 
2018-01-29 03:37:17,480 INFO org.apache.zookeeper.ClientCnxn: Client session 
timed out, have not heard from server in 22836ms for sessionid 
0x3612d6a573501c2, closing socket connection and attempting reco
n nect 
2018-01-29 03:37:17,480 WARN kafka.server.ReplicaFetcherThread: 
[ReplicaFetcherThread-11-337], Error in fetch 
kafka.server.ReplicaFetcherThread$FetchRequest@1657a3a8. Possible cause: 
java.io.IOException: 
C onnection to 337 was disconnected before the response was read 2018-01-29 
03:37:17,480 WARN kafka.server.ReplicaFetcherThread: 
[ReplicaFetcherThread-3-337], Error in fetch 
kafka.server.ReplicaFetcherThread$FetchRequest@34f57a6c. Possible cause: 
java.io.IOException: C
o nnection to 337 was disconnected before the response was read 
..................................... 
2018-01-29 03:37:17,580 INFO org.I0Itec.zkclient.ZkClient: zookeeper state 
changed (Disconnected) 
2018-01-29 03:37:17,580 INFO 
org.apache.curator.framework.state.ConnectionStateManager: State change: 
SUSPENDED 2018-01-29 03:37:17,580 INFO org.I0Itec.zkclient.ZkClient: zookeeper 
state changed (Disconnected) 
2018-01-29 03:37:17,586 ERROR kafka.server.ReplicaFetcherThread: 
[ReplicaFetcherThread-15-337], Error for partition 
[tv20-long-retention-invalid-topic,23] to broker 
337:org.apache.kafka.common.errors.NotLe aderForPartitionException: This server 
is not the leader for that topic-partition. 
2018-01-29 03:37:17,587 ERROR kafka.server.ReplicaFetcherThread: 
[ReplicaFetcherThread-9-337], Error for partition [trckv20-ch-click,22] to 
broker 337:org.apache.kafka.common.errors.NotLeaderForPartitionEx ception: This 
server is not the leader for that topic-partition. 
.................................
2018-01-29 03:37:17,931 INFO org.apache.zookeeper.ClientCnxn: Opening socket 
connection to server zookeeper-02.cpp.com/192.168.1.101:2181. Will not attempt 
to authenticate using SASL (unknown error) 2018-01-29 03:37:17,931 INFO 
org.apache.zookeeper.ClientCnxn: Socket connection established to 
zookeeper-02.cpp.com/192.168.1.101:2181, initiating session 
2018-01-29 03:37:17,933 INFO org.apache.zookeeper.ClientCnxn: Session 
establishment complete on server zookeeper-02.cpp.com/192.168.1.101:2181, 
sessionid = 0x3612d6a573501c2, negotiated timeout = 30000 2018-01-29 
03:37:17,934 INFO org.I0Itec.zkclient.ZkClient: zookeeper state changed 
(SyncConnected) 
2018-01-29 03:37:18,063 INFO org.apache.zookeeper.ClientCnxn: Opening socket 
connection to server hb03.cpp.com/192.168.0.73:2181. Will not attempt to 
authenticate using SASL (unknown error) 2018-01-29 03:37:18,063 INFO 
org.apache.zookeeper.ClientCnxn: Socket connection established to 
hb03.cpp.com/192.168.0.73:2181, initiating session 
2018-01-29 03:37:18,065 INFO org.apache.zookeeper.ClientCnxn: Unable to 
reconnect to ZooKeeper service, session 0x3612d6a573501c1 has expired, closing 
socket connection 2018-01-29 03:37:18,065 INFO org.I0Itec.zkclient.ZkClient: 
zookeeper state changed (Expired) 
2018-01-29 03:37:18,065 INFO org.apache.zookeeper.ZooKeeper: Initiating client 
connection, 
connectString=hb03.cpp.com:2181,zookeeper-01.cpp.com:2181,zookeeper-02.cpp.com:2181,zookeeper-03.cpp.com:2181,zookeeper-04.cpp.com:2181
 sessionTimeout=10000 watcher=org.I0Itec.zkclient.ZkClient@1364a89 
2018-01-29 03:37:18,066 INFO 
kafka.controller.KafkaController$SessionExpirationListener: 
[SessionExpirationListener on 338], ZK expired; shut down all controller 
components and try to re-elect 
2018-01-29 03:37:18,066 INFO org.apache.zookeeper.ClientCnxn: EventThread shut 
down 
2018-01-29 03:37:18,066 INFO kafka.controller.PartitionStateMachine: [Partition 
state machine on Controller 338]: Stopped partition state machine 
2018-01-29 03:37:18,067 INFO kafka.controller.ReplicaStateMachine: [Replica 
state machine on controller 338]: Stopped replica state machine 
2018-01-29 03:37:18,067 INFO kafka.controller.KafkaController: [Controller 
338]: Broker 338 resigned as the controller 
2018-01-29 03:37:18,067 INFO org.apache.zookeeper.ClientCnxn: Opening socket 
connection to server hb03.cpp.com/192.168.0.73:2181. Will not attempt to 
authenticate using SASL (unknown error) 
2018-01-29 03:37:18,090 INFO org.apache.zookeeper.ClientCnxn: Socket connection 
established to hb03.cpp.com/192.168.0.73:2181, initiating session 
2018-01-29 03:37:18,152 INFO org.apache.zookeeper.ClientCnxn: Session 
establishment complete on server hb03.cpp.com/192.168.0.73:2181, sessionid = 
0x7612d6a562113a6, negotiated timeout = 10000 
2018-01-29 03:37:18,152 INFO org.I0Itec.zkclient.ZkClient: zookeeper state 
changed (SyncConnected) 
2018-01-29 03:37:18,153 INFO kafka.server.KafkaHealthcheck: re-registering 
broker info in ZK for broker 338 
2018-01-29 03:37:18,153 INFO kafka.utils.ZKCheckedEphemeral: Creating 
/brokers/ids/338 (is it secure? false) 
2018-01-29 03:37:18,162 INFO org.apache.zookeeper.ClientCnxn: Opening socket 
connection to server zookeeper-01.cpp.com/192.168.1.100:2181. Will not attempt 
to authenticate using SASL (unknown error) 
2018-01-29 03:37:18,222 ERROR kafka.server.ReplicaFetcherThread: 
[ReplicaFetcherThread-6-337], Error for partition 
[tv20-long-retention-topic,18] to broker 
337:org.apache.kafka.common.errors.NotLeaderForP
artitionException: This server is not the leader for that topic-partition. 
.......................... 

2018-01-29 03:37:18,260 INFO kafka.utils.ZKCheckedEphemeral: Result of znode 
creation is: OK 
2018-01-29 03:37:18,260 INFO kafka.utils.ZkUtils: Registered broker 338 at path 
/brokers/ids/338 with addresses: PLAINTEXT -> 
EndPoint(historical03.cpp.com,9092,PLAINTEXT) 
2018-01-29 03:37:18,260 INFO kafka.server.KafkaHealthcheck: done re-registering 
broker 
2018-01-29 03:37:18,260 INFO kafka.server.KafkaHealthcheck: Subscribing to 
/brokers/topics path to watch for new topics 
2018-01-29 03:37:18,260 ERROR kafka.server.ReplicaFetcherThread: 
[ReplicaFetcherThread-0-337], Error for partition [trckv20-lead-processing,53] 
to broker 337:org.apache.kafka.common.errors.NotLeaderForPar
t itionException: This server is not the leader for that topic-partition. 
.............and it's ends with ...... 

2018-01-29 03:38:13,597 INFO kafka.server.ReplicaFetcherManager: 
[ReplicaFetcherManager on broker 338] Removed fetcher for partitions 
2018-01-29 03:38:13,661 INFO kafka.server.ReplicaFetcherManager: 
[ReplicaFetcherManager on broker 338] Added fetcher for partitions List() 
2018-01-29 03:38:17,254 FATAL kafka.server.ReplicaFetcherThread: 
[ReplicaFetcherThread-8-339], Halting because log truncation is not allowed for 
topic trckv20-ch-click, Current leader 339's latest offset 
4494396139 is less than replica 338's latest offset 4494403425 
2018-01-29 03:38:17,254 FATAL kafka.server.ReplicaFetcherThread: 
[ReplicaFetcherThread-12-339], Halting because log truncation is not allowed 
for topic trckv20-ch-click, Current leader 339's latest offset
4494474772 is less than replica 338's latest offset 4494481669 
2018-01-29 03:38:17,254 FATAL kafka.server.ReplicaFetcherThread: 
[ReplicaFetcherThread-10-339], Halting because log truncation is not allowed 
for topic trckv20-ch-click, Current leader 339's latest offset
4494433300 is less than replica 338's latest offset 4494439854 
2018-01-29 03:38:17,254 FATAL kafka.server.ReplicaFetcherThread: 
[ReplicaFetcherThread-2-339], Halting because log truncation is not allowed for 
topic trckv20-ch-click, Current leader 339's latest offset 
4494387381 is less than replica 338's latest offset 4494394866 
2018-01-29 03:38:17,254 FATAL kafka.server.ReplicaFetcherThread: 
[ReplicaFetcherThread-6-339], Halting because log truncation is not allowed for 
topic trckv20-ch-click, Current leader 339's latest offset 
4494466871 is less than replica 338's latest offset 4494472899 
2018-01-29 03:38:17,254 FATAL kafka.server.ReplicaFetcherThread: 
[ReplicaFetcherThread-4-339], Halting because log truncation is not allowed for 
topic trckv20-ch-click, Current leader 339's latest offset 
4494407851 is less than replica 338's latest offset 4494413095 
2018-01-29 03:38:17,254 FATAL kafka.server.ReplicaFetcherThread: 
[ReplicaFetcherThread-14-339], Halting because log truncation is not allowed 
for topic __consumer_offsets, Current leader 339's latest offs
et 168707094 is less than replica 338's latest offset 168707288 
2018-01-29 03:38:17,254 FATAL kafka.server.ReplicaFetcherThread: 
[ReplicaFetcherThread-5-339], Halting because log truncation is not allowed for 
topic tv20-long-retention-topic, Current leader 339's lates
t offset 18615 is less than replica 338's latest offset 18616 
2018-01-29 03:38:17,254 FATAL kafka.server.ReplicaFetcherThread: 
[ReplicaFetcherThread-0-339], Halting because log truncation is not allowed for 
topic trckv20-ch-click, Current leader 339's latest offset 
4494445097 is less than replica 338's latest offset 4494450915{code}
- only one node kept working
- consumers sucked with this message

 
{code:java}
msg="Error: kafka server: Request was for a consumer group that is not 
coordinated by this broker  

our consumer wrote on golang, and we use sarama-cluster goclient{code}
 
Return kafka to normal state helped `unclean.leader.election.enable=true`, but 
i had make 3 restaring of kafka cluster

- set `unclean.leader.election.enable=true` and restart, wait until kafka goes 
normal, check logs and dashboard in cloudera.
- set `unclean.leader.election.enable=false` and restart, wait untill kafka 
goes normal, restarted consumers. but something went wrong with consumers. 

They was could not obtain consumer group. And one of the nodes of kafka, i 
think, stucked - 2 two nodes writing to log, but this one was silent, and this 
one was an "Active controller"

the same massage in log of consumer:
{code:java}
"msg="Error: kafka server: Request was for a consumer group that is not 
coordinated by this broker"

{code}
- I have restarted "silent" node, and after this all was back to normal state, 
another node become a "Active controller"

Sorry if this a big amount of logs.

> 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
>            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