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

Kurt Harriger commented on KAFKA-2300:
--------------------------------------

We eventually resolved the problem after hours with much less success than 
desired. 

Reviewing the code it appeared that the controller would not be able to send 
state to any brokers and would need to be restarted.  

We were hesitant to bring this node down entirely as it was the only ISR for 
many partitions and since the controller was unable to send messages to other 
brokers we were unable to reassign any partitions.  

We determined it would be possible to force a controller re-election without 
fully shutting down the broker by deleting the /controller node in zookeeper.  
Our hope was that once a new controller was elected broker 1 would be able to 
rejoin the cluster and we would then be able to restart broker 2 after 
replication was up to date.

Once we deleted the /controller node from zookeeper, election did occur as 
expected and broker 3 became the new leader and broker 1 was able to join the 
cluster.  However we started lots of unknown errors in the logs from broker 2. 
...
[2015-06-24 18:14:23,274] ERROR [ReplicaFetcherThread-0-2], Error for partition 
[prod-eclipselinkchannel,5] to broker 2:class kafka.common.UnknownException 
(kafka.server.ReplicaFetcherThread)
[2015-06-24 18:14:23,274] ERROR [ReplicaFetcherThread-0-2], Error for partition 
[birdseed-metadata-stream,18] to broker 2:class kafka.common.UnknownException 
(kafka.server.ReplicaFetcherThread)
[2015-06-24 18:14:23,274] ERROR [ReplicaFetcherThread-0-2], Error for partition 
[trial-server-start,7] to broker 2:class kafka.common.UnknownException 
(kafka.server.ReplicaFetcherThread)
... 
We looked through the code a bit but were not easily able to identify the root 
cause of this exception.  More logging may have been helpful here.  We knew 
that broker 2 was needed to be restarted so we did that now.  Once node 2 came 
back online the error messages changed and the logs seemed to indicate data was 
lost: 
...
[2015-06-24 18:43:51,136] ERROR [Replica Manager on Broker 3]: Error when 
processing fetch request for partition [feedback-events,10] offset 1757 from 
follower with correlation id 10539. Possible cause: Request for offset 1757 but 
we only have log segments in the range 0 to 0. (kafka.server.ReplicaManager)
[2015-06-24 18:43:51,136] ERROR [Replica Manager on Broker 3]: Error when 
processing fetch request for partition [prod-beacon,10] offset 70616524 from 
follower with correlation id 10539. Possible cause: Request for offset 70616524 
but we only have log segments in the range 0 to 1246. 
(kafka.server.ReplicaManager)
...
The logs were still noisy and replication did not appear to be working so we 
decided to restart broker 3.  Following this we saw large number of files being 
deleted. 
[2015-06-24 19:34:11,237] INFO Deleting index 
/home/kafka/shared/logs/alm-start-spans-2/00000000000420766992.index.deleted 
(kafka.log.OffsetIndex)
[2015-06-24 19:34:11,248] INFO Deleting index 
/home/kafka/shared/logs/alm-spans-2/00000000000420150674.index.deleted 
(kafka.log.OffsetIndex)
[2015-06-24 19:34:11,269] INFO Deleting index 
/home/kafka/shared/logs/alm-start-spans-2/00000000000426783743.index.deleted 
(kafka.log.OffsetIndex)
[2015-06-24 19:34:11,270] INFO Deleting index 
/home/kafka/shared/logs/beacon-spans-11/00000000000389662179.index.deleted 
(kafka.log.OffsetIndex)
[2015-06-24 19:34:11,275] INFO Deleting index 
/home/kafka/shared/logs/prod-server-start-11/00000000000415746006.index.deleted 
(kafka.log.OffsetIndex)

After this the logs settled down and the cluster began replicating and was 
otherwise healthy.  We know we lost some partitions that were on broker 1. I'm 
not entirely sure if the above indicated errors indicated that topics were 
partitions were lost entirely or just replicas of said partitions, but we 
definitely lost some partitions.  

We also mirror the data into another data center where we do the actual data 
analytics, so I was able to compare message counts by dumping all the data and 
grouping by date to see that about 2/3rds of the data on at least one topic was 
lost.  This topic had 2 of 12 located on broker 1 which went unavailable after 
broker 2 (the controller) entered a bad state since it was the only ISR.  We 
expected we would probably lose these partitions, but this ratio suggests that 
much more data was lost.  Given that we mirror the data we aren't particularly 
concerned with historical data loss, but I thought I would mention it.  



> Error in controller log when broker tries to rejoin cluster
> -----------------------------------------------------------
>
>                 Key: KAFKA-2300
>                 URL: https://issues.apache.org/jira/browse/KAFKA-2300
>             Project: Kafka
>          Issue Type: Bug
>    Affects Versions: 0.8.2.1
>            Reporter: Johnny Brown
>
> Hello Kafka folks,
> We are having an issue where a broker attempts to join the cluster after 
> being restarted, but is never added to the ISR for its assigned partitions. 
> This is a three-node cluster, and the controller is broker 2.
> When broker 1 starts, we see the following message in broker 2's 
> controller.log.
> {{
> [2015-06-23 13:57:16,535] ERROR [BrokerChangeListener on Controller 2]: Error 
> while handling broker changes 
> (kafka.controller.ReplicaStateMachine$BrokerChangeListener)
> java.lang.IllegalStateException: Controller to broker state change requests 
> batch is not empty while creating a new one. Some UpdateMetadata state 
> changes Map(2 -> Map([prod-sver-end,1] -> 
> (LeaderAndIsrInfo:(Leader:-2,ISR:1,LeaderEpoch:0,ControllerEpoch:165),ReplicationFactor:1),AllReplicas:1)),
>  1 -> Map([prod-sver-end,1] -> 
> (LeaderAndIsrInfo:(Leader:-2,ISR:1,LeaderEpoch:0,ControllerEpoch:165),ReplicationFactor:1),AllReplicas:1)),
>  3 -> Map([prod-sver-end,1] -> 
> (LeaderAndIsrInfo:(Leader:-2,ISR:1,LeaderEpoch:0,ControllerEpoch:165),ReplicationFactor:1),AllReplicas:1)))
>  might be lost 
>   at 
> kafka.controller.ControllerBrokerRequestBatch.newBatch(ControllerChannelManager.scala:202)
>   at 
> kafka.controller.KafkaController.sendUpdateMetadataRequest(KafkaController.scala:974)
>   at 
> kafka.controller.KafkaController.onBrokerStartup(KafkaController.scala:399)
>   at 
> kafka.controller.ReplicaStateMachine$BrokerChangeListener$$anonfun$handleChildChange$1$$anonfun$apply$mcV$sp$1.apply$mcV$sp(ReplicaStateMachine.scala:371)
>   at 
> kafka.controller.ReplicaStateMachine$BrokerChangeListener$$anonfun$handleChildChange$1$$anonfun$apply$mcV$sp$1.apply(ReplicaStateMachine.scala:359)
>   at 
> kafka.controller.ReplicaStateMachine$BrokerChangeListener$$anonfun$handleChildChange$1$$anonfun$apply$mcV$sp$1.apply(ReplicaStateMachine.scala:359)
>   at kafka.metrics.KafkaTimer.time(KafkaTimer.scala:33)
>   at 
> kafka.controller.ReplicaStateMachine$BrokerChangeListener$$anonfun$handleChildChange$1.apply$mcV$sp(ReplicaStateMachine.scala:358)
>   at 
> kafka.controller.ReplicaStateMachine$BrokerChangeListener$$anonfun$handleChildChange$1.apply(ReplicaStateMachine.scala:357)
>   at 
> kafka.controller.ReplicaStateMachine$BrokerChangeListener$$anonfun$handleChildChange$1.apply(ReplicaStateMachine.scala:357)
>   at kafka.utils.Utils$.inLock(Utils.scala:535)
>   at 
> kafka.controller.ReplicaStateMachine$BrokerChangeListener.handleChildChange(ReplicaStateMachine.scala:356)
>   at org.I0Itec.zkclient.ZkClient$7.run(ZkClient.java:568)
>   at org.I0Itec.zkclient.ZkEventThread.run(ZkEventThread.java:71)
> }}
> {{prod-sver-end}} is a topic we previously deleted. It seems some remnant of 
> it persists in the controller's memory, causing an exception which interrupts 
> the state change triggered by the broker startup.
> Has anyone seen something like this? Any idea what's happening here? Any 
> information would be greatly appreciated.
> Thanks,
> Johnny



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

Reply via email to