You are viewing a plain text version of this content. The canonical link for it is here.
Posted to users@kafka.apache.org by Jun Rao <ju...@confluent.io> on 2015/03/03 17:00:49 UTC

Re: broker restart problems

Do you see broker 1 being deleted in the controller log?

Thanks,

Jun

On Fri, Feb 27, 2015 at 5:25 PM, ZhuGe <tc...@outlook.com> wrote:

> Thanks for the reply.I confirmed that broker 1 is registered in the zk.
>
>
> > Date: Fri, 27 Feb 2015 09:27:52 -0800
> > Subject: Re: broker restart problems
> > From: jun@confluent.io
> > To: users@kafka.apache.org
> >
> > The log seems to suggest that broker 1 is offline. Is broker 1 registered
> > properly in ZK? You can find this out by reading the broker registration
> > path (
> >
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+data+structures+in+Zookeeper
> )
> > from ZK.
> >
> > Thanks,
> >
> > Jun
> >
> > On Thu, Feb 26, 2015 at 10:31 PM, ZhuGe <tc...@outlook.com> wrote:
> >
> > > Sorry, i found the controller log. it shows some log like this:
> > > [2015-02-16 11:10:03,094] DEBUG [Controller 0]: Removing replica 1 from
> > > ISR 2,0 for partition [wx_rtdc_PageViewData_bolt1,4].
> > > (kafka.controller.KafkaController)[2015-02-16 11:10:03,096] WARN
> > > [Controller 0]: Cannot remove replica 1 from ISR of partition
> > > [wx_rtdc_PageViewData_bolt1,4] since it is not in the ISR. Leader = 2
> ; ISR
> > > = List(2, 0) (kafka.controller.KafkaController)[2015-02-16
> 11:10:03,096]
> > > DEBUG [Controller 0]: Removing replica 1 from ISR 0,1,2 for partition
> > > [wx_rtdc_DownLoadData_bolt2,9].
> > > (kafka.controller.KafkaController)[2015-02-16 11:10:03,099] INFO
> > > [Controller 0]: New leader and ISR for partition
> > > [wx_rtdc_DownLoadData_bolt2,9] is
> {"leader":0,"leader_epoch":1,"isr":[0,2]}
> > > (kafka.controller.KafkaController)[2015-02-16 11:10:03,100] DEBUG
> > > [Controller 0]: Removing replica 1 from ISR 2,0,1 for partition
> > > [wx_rtdc_EventTrackData_bolt5,2].
> > > (kafka.controller.KafkaController)[2015-02-16 11:10:03,103] INFO
> > > [Controller 0]: New leader and ISR for partition
> > > [wx_rtdc_EventTrackData_bolt5,2] is
> > > {"leader":2,"leader_epoch":1,"isr":[2,0]}
> (kafka.controller.KafkaController)
> > > And after that, there are a lot WARN message like:[2015-02-16
> > > 11:10:03,220] WARN [Channel manager on controller 0]: Not sending
> request
> > > Name: StopReplicaRequest; Version: 0; CorrelationId: 188; ClientId: ;
> > > DeletePartitions: false; ControllerId: 0; ControllerEpoch: 36;
> Partitions:
> > > [wx_rtdc_DownLoadData_bolt1,8] to broker 1, since it is offline.
> > > (kafka.controller.ControllerChannelManager)[2015-02-16 11:10:03,221]
> WARN
> > > [Channel manager on controller 0]: Not sending request Name:
> > > StopReplicaRequest; Version: 0; CorrelationId: 188; ClientId: ;
> > > DeletePartitions: false; ControllerId: 0; ControllerEpoch: 36;
> Partitions:
> > > [wx_rtdc_StartStat_bolt3,0] to broker 1, since it is offline.
> > > (kafka.controller.ControllerChannelManager)[2015-02-16 11:10:03,221]
> WARN
> > > [Channel manager on controller 0]: Not sending request Name:
> > > StopReplicaRequest; Version: 0; CorrelationId: 188; ClientId: ;
> > > DeletePartitions: false; ControllerId: 0; ControllerEpoch: 36;
> Partitions:
> > > [wx_rtdc_PageViewData_bolt3,5] to broker 1, since it is offline.
> > > (kafka.controller.ControllerChannelManager)
> > > And when the broker 1 stat up, it log the message:
> > > [2015-02-16 11:11:04,960] INFO [BrokerChangeListener on Controller 0]:
> > > Broker change listener fired for path /brokers/ids with children 2,1,0
> > > (kafka.controller.ReplicaStateMachine$BrokerChangeListener)[2015-02-16
> > > 11:11:04,969] INFO [BrokerChangeListener on Controller 0]: Newly added
> > > brokers: 1, deleted brokers: , all live brokers: 2,1,0
> > > (kafka.controller.ReplicaStateMachine$BrokerChangeListener)[2015-02-16
> > > 11:11:04,969] DEBUG [Channel manager on controller 0]: Controller 0
> trying
> > > to connect to broker 1
> > > (kafka.controller.ControllerChannelManager)[2015-02-16 11:11:04,970]
> INFO
> > > [Controller-0-to-broker-1-send-thread], Controller 0 connected to
> > > id:1,host:kslave2,port:9092 for sending state change requests
> > > (kafka.controller.RequestSendThread)[2015-02-16 11:11:04,970] INFO
> > > [Controller 0]: New broker startup callback for 1
> > > (kafka.controller.KafkaController)[2015-02-16 11:11:04,970] INFO
> > > [Controller-0-to-broker-1-send-thread], Starting
> > > (kafka.controller.RequestSendThread)[2015-02-16 11:11:04,978] ERROR
> > > [BrokerChangeListener on Controller 0]: Error while handling broker
> changes
> > >
> (kafka.controller.ReplicaStateMachine$BrokerChangeListener)java.util.NoSuchElementException:
> > > key not found: [cjy_kafkawebservice_AirTicketHbaseData_New,0]        at
> > > scala.collection.MapLike$class.default(MapLike.scala:225)        at
> > > scala.collection.mutable.HashMap.default(HashMap.scala:45)        at
> > > scala.collection.MapLike$class.apply(MapLike.scala:135)        at
> > > scala.collection.mutable.HashMap.apply(HashMap.scala:45)        at
> > >
> kafka.controller.ControllerBrokerRequestBatch.updateMetadataRequestMapFor$1(ControllerChannelManager.scala:242)
> > >       at
> > >
> kafka.controller.ControllerBrokerRequestBatch$$anonfun$addUpdateMetadataRequestForBrokers$1.apply(ControllerChannelManager.scala:268)
> > >       at
> > >
> kafka.controller.ControllerBrokerRequestBatch$$anonfun$addUpdateMetadataRequestForBrokers$1.apply(ControllerChannelManager.scala:268)
> > >       at
> > >
> scala.collection.mutable.HashMap$$anon$1$$anonfun$foreach$2.apply(HashMap.scala:99)
> > >       at
> > >
> scala.collection.mutable.HashMap$$anon$1$$anonfun$foreach$2.apply(HashMap.scala:99)
> > >       at scala.collection.Iterator$class.foreach(Iterator.scala:772)
> > > at
> scala.collection.mutable.HashTable$$anon$1.foreach(HashTable.scala:157)
> > >       at
> > >
> scala.collection.mutable.HashTable$class.foreachEntry(HashTable.scala:190)
> > >       at
> scala.collection.mutable.HashMap.foreachEntry(HashMap.scala:45)
> > >     at
> scala.collection.mutable.HashMap$$anon$1.foreach(HashMap.scala:99)
> > >       at
> > >
> kafka.controller.ControllerBrokerRequestBatch.addUpdateMetadataRequestForBrokers(ControllerChannelManager.scala:268)
> > >       at
> > >
> kafka.controller.KafkaController.sendUpdateMetadataRequest(KafkaController.scala:940)
> > >       at
> > >
> kafka.controller.KafkaController.onBrokerStartup(KafkaController.scala:386)
> > >       at
> > >
> kafka.controller.ReplicaStateMachine$BrokerChangeListener$$anonfun$handleChildChange$1$$anonfun$apply$mcV$sp$1.apply$mcV$sp(ReplicaStateMachine.scala:342)
> > >       at
> > >
> kafka.controller.ReplicaStateMachine$BrokerChangeListener$$anonfun$handleChildChange$1$$anonfun$apply$mcV$sp$1.apply(ReplicaStateMachine.scala:330)
> > >       at
> > >
> kafka.controller.ReplicaStateMachine$BrokerChangeListener$$anonfun$handleChildChange$1$$anonfun$apply$mcV$sp$1.apply(ReplicaStateMachine.scala:330)
> > >       at kafka.metrics.KafkaTimer.time(KafkaTimer.scala:33)        at
> > >
> kafka.controller.ReplicaStateMachine$BrokerChangeListener$$anonfun$handleChildChange$1.apply$mcV$sp(ReplicaStateMachine.scala:329)
> > >       at
> > >
> kafka.controller.ReplicaStateMachine$BrokerChangeListener$$anonfun$handleChildChange$1.apply(ReplicaStateMachine.scala:328)
> > >       at
> > >
> kafka.controller.ReplicaStateMachine$BrokerChangeListener$$anonfun$handleChildChange$1.apply(ReplicaStateMachine.scala:328)
> > >       at kafka.utils.Utils$.inLock(Utils.scala:538)        at
> > >
> kafka.controller.ReplicaStateMachine$BrokerChangeListener.handleChildChange(ReplicaStateMachine.scala:327)
> > >       at org.I0Itec.zkclient.ZkClient$7.run(ZkClient.java:568)
> at
> > > org.I0Itec.zkclient.ZkEventThread.run(ZkEventThread.java:71)
> > >
> > > Thank you very much
> > > > From: tczg@outlook.com
> > > > To: users@kafka.apache.org
> > > > Subject: RE: broker restart problems
> > > > Date: Fri, 27 Feb 2015 02:14:28 +0000
> > > >
> > > > The version of kafka is 0.8.1.1.
> > > > The controller log shows the controller starting up complete with no
> > > warn or error message. And we could not find the state-change log
> belong to
> > > the day we restart the broker. I found some error message in  broker
> 0's
> > > log when i restart the broker 2. I am not sure whether it is related
> or not.
> > > > [2015-02-16 11:10:11,869] INFO Partition
> > > [wx_rtdc_EventTrackData_bolt2,8] on broker 0: Shrinking ISR for
> partition
> > > [wx_rtdc_EventTrackData_bolt2,8] from 0,2,1 to 0,2
> > > (kafka.cluster.Partition)[2015-02-16 11:10:11,878] ERROR Conditional
> update
> > > of path /brokers/topics/wx_rtdc_EventTrackData_bolt2/partitions/8/state
> > > with data
> > >
> {"controller_epoch":36,"leader":0,"version":1,"leader_epoch":0,"isr":[0,2]}
> > > and expected version 0 failed due to
> > > org.apache.zookeeper.KeeperException$BadVersionException:
> KeeperErrorCode =
> > > BadVersion for
> > > /brokers/topics/wx_rtdc_EventTrackData_bolt2/partitions/8/state
> > > (kafka.utils.ZkUtils$)[2015-02-16 11:10:11,878] INFO Partition
> > > [wx_rtdc_EventTrackData_bolt2,8] on broker 0: Cached zkVersion [0] not
> > > equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition)
> > > > Thanks> Date: Thu, 26 Feb 2015 13:11:24 -0800
> > > > > Subject: Re: broker restart problems
> > > > > From: jun@confluent.io
> > > > > To: users@kafka.apache.org
> > > > >
> > > > > Right, you need to look into why the restarted broker is not
> sync-ed
> > > up.
> > > > > Any error in the controller and state-change log? Also, what
> version of
> > > > > Kafka are you on?
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Jun
> > > > >
> > > > > On Wed, Feb 25, 2015 at 5:46 PM, ZhuGe <tc...@outlook.com> wrote:
> > > > >
> > > > > > we did not have this setting in the property file, so it should
> be
> > > false.
> > > > > > BTW, it this command means periodically invoking 'preferred
> replica
> > > leader
> > > > > > election tool'?and how should i solve the "out of syn" problem
> of the
> > > > > > broker?
> > > > > > > Date: Wed, 25 Feb 2015 16:09:42 -0800
> > > > > > > Subject: Re: broker restart problems
> > > > > > > From: kzakee1@netzero.net
> > > > > > > To: users@kafka.apache.org
> > > > > > >
> > > > > > > Do you have the property auto.leader.rebalance.enable=true set
> in
> > > > > > brokers?
> > > > > > >
> > > > > > > Thanks
> > > > > > > -Zakee
> > > > > > >
> > > > > > > On Tue, Feb 24, 2015 at 11:47 PM, ZhuGe <tc...@outlook.com>
> wrote:
> > > > > > >
> > > > > > > > Hi all:We have a cluster of 3 brokers(id : 0,1,2). We
> > > restart(simply
> > > > > > use
> > > > > > > > stop.sh and start.sh in bin directory) broker 1. The broker
> > > started
> > > > > > > > successfully. However, all the partitions' leader moved to
> other
> > > > > > brokers
> > > > > > > > and no data were written into broker 2. This is the status
> of one
> > > > > > > > topic:Topic:wx_rtdc_flumesinks        PartitionCount:12
> > > > > > > >  ReplicationFactor:3     Configs:        Topic:
> > > wx_rtdc_flumesinks
> > > > > > > >  Partition: 0    Leader: 2       Replicas: 1,2,0 Isr: 2,0
> > > Topic:
> > > > > > > > wx_rtdc_flumesinks       Partition: 1    Leader: 2
> > >  Replicas:
> > > > > > 2,0,1
> > > > > > > > Isr: 2,0        Topic: wx_rtdc_flumesinks       Partition: 2
> > > > > > Leader: 0
> > > > > > > >      Replicas: 0,1,2 Isr: 0,2        Topic:
> wx_rtdc_flumesinks
> > > > > > > >  Partition: 3    Leader: 0       Replicas: 1,0,2 Isr: 0,2
> > > Topic:
> > > > > > > > wx_rtdc_flumesinks       Partition: 4    Leader: 2
> > >  Replicas:
> > > > > > 2,1,0
> > > > > > > > Isr: 2,0        Topic: wx_rtdc_flumesinks       Partition: 5
> > > > > > Leader: 0
> > > > > > > >      Replicas: 0,2,1 Isr: 0,2        Topic:
> wx_rtdc_flumesinks
> > > > > > > >  Partition: 6    Leader: 2       Replicas: 1,2,0 Isr: 2,0
> > > Topic:
> > > > > > > > wx_rtdc_flumesinks       Partition: 7    Leader: 2
> > >  Replicas:
> > > > > > 2,0,1
> > > > > > > > Isr: 2,0        Topic: wx_rtdc_flumesinks       Partition: 8
> > > > > > Leader: 0
> > > > > > > >      Replicas: 0,1,2 Isr: 0,2        Topic:
> wx_rtdc_flumesinks
> > > > > > > >  Partition: 9    Leader: 0       Replicas: 1,0,2 Isr: 0,2
> > > Topic:
> > > > > > > > wx_rtdc_flumesinks       Partition: 10   Leader: 2
> > >  Replicas:
> > > > > > 2,1,0
> > > > > > > > Isr: 2,0        Topic: wx_rtdc_flumesinks       Partition: 11
> > > > > >  Leader: 0
> > > > > > > >      Replicas: 0,2,1 Isr: 0,2
> > > > > > > > It seems the broker is out of synchronize with other brokers.
> > > and
> > > > > > nothing
> > > > > > > > changed after i run preferred replication leader election
> tool.
> > > i
> > > > > > think it
> > > > > > > > is because the preferred replication is not in Isr, which is
> > > described
> > > > > > in
> > > > > > > > the wiki of replication tool.
> > > > > > > > I want to know how to synchronize the replications of 3
> brokers
> > > so that
> > > > > > > > the broker 1 could work properly.any help would be
> appreciated.
> > > > > > > > ____________________________________________________________
> > > > > > > > High School Yearbooks
> > > > > > > > View Class Yearbooks Online Free. Reminisce & Buy a Reprint
> > > Today!
> > > > > > > >
> > > > > >
> > >
> http://thirdpartyoffers.netzero.net/TGL3255/54ed7fb69616f7fb61578mp03duc
> > > > > >
> > > > > >
> > > >
> > >
> > >
>
>