Re: zookeeper shows partition in-sync but local disk doesn't have the data dirs

2014-09-10 Thread Steven Wu
this topic foo is created by auto creation at Sep 3. I looked at the old
log files on controller broker 5. it has some errors regarding partition 1
and 2 when the topic is auto created. who is creating/updating IsrLeader
record?

errors regarding partition 1
===
[2014-09-03 21:18:41,164] ERROR Controller 5 epoch 5 encountered error
while changing partition [foo,1]'s state from New to Online since
LeaderAndIsr path alrea
dy exists with value {leader:0,leader_epoch:0,isr:[0,5]} and
controller epoch 5 (state.change.logger)
[2014-09-03 21:18:41,164] ERROR Controller 5 epoch 5 initiated state change
for partition [foo,1] from NewPartition to OnlinePartition failed
(state.change.logg
er)
kafka.common.StateChangeFailedException: encountered error while changing
partition [foo,1]'s state from New to Online since LeaderAndIsr path
already exists with value {leader:0,leader_epoch:0,isr:[0,5]} and
controller epoch 5
at
kafka.controller.PartitionStateMachine.initializeLeaderAndIsrForPartition(PartitionStateMachine.scala:302)
at
kafka.controller.PartitionStateMachine.kafka$controller$PartitionStateMachine$$handleStateChange(PartitionStateMachine.scala:183)
at
kafka.controller.PartitionStateMachine$$anonfun$handleStateChanges$2.apply(PartitionStateMachine.scala:125)
at
kafka.controller.PartitionStateMachine$$anonfun$handleStateChanges$2.apply(PartitionStateMachine.scala:124)
at
scala.collection.immutable.HashSet$HashSet1.foreach(HashSet.scala:130)
at
scala.collection.immutable.HashSet$HashTrieSet.foreach(HashSet.scala:275)
at
kafka.controller.PartitionStateMachine.handleStateChanges(PartitionStateMachine.scala:124)
at
kafka.controller.KafkaController.onNewPartitionCreation(KafkaController.scala:473)
at
kafka.controller.KafkaController.onNewTopicCreation(KafkaController.scala:460)
at
kafka.controller.PartitionStateMachine$TopicChangeListener$$anonfun$handleChildChange$1.apply$mcV$sp(PartitionStateMachine.scala:422)
at
kafka.controller.PartitionStateMachine$TopicChangeListener$$anonfun$handleChildChange$1.apply(PartitionStateMachine.scala:404)
at
kafka.controller.PartitionStateMachine$TopicChangeListener$$anonfun$handleChildChange$1.apply(PartitionStateMachine.scala:404)
at kafka.utils.Utils$.inLock(Utils.scala:538)
at
kafka.controller.PartitionStateMachine$TopicChangeListener.handleChildChange(PartitionStateMachine.scala:403)
at org.I0Itec.zkclient.ZkClient$7.run(ZkClient.java:568)
at org.I0Itec.zkclient.ZkEventThread.run(ZkEventThread.java:71)

errors regarding partition 2
===
[2014-09-03 21:18:41,096] ERROR Controller 5 epoch 5 initiated state change
of replica 1 for partition [foo,2] from NonExistentReplica to NewReplica
failed (state.change.logger)
kafka.common.StateChangeFailedException: Replica 1 for partition [foo,2]
cannot be moved to NewReplicastate as it is being requested to become leader
at
kafka.controller.ReplicaStateMachine.handleStateChange(ReplicaStateMachine.scala:160)
at
kafka.controller.ReplicaStateMachine$$anonfun$handleStateChanges$2.apply(ReplicaStateMachine.scala:96)
at
kafka.controller.ReplicaStateMachine$$anonfun$handleStateChanges$2.apply(ReplicaStateMachine.scala:96)
at
scala.collection.immutable.HashSet$HashSet1.foreach(HashSet.scala:130)
at
scala.collection.immutable.HashSet$HashTrieSet.foreach(HashSet.scala:275)
at
kafka.controller.ReplicaStateMachine.handleStateChanges(ReplicaStateMachine.scala:96)
at
kafka.controller.KafkaController.onNewPartitionCreation(KafkaController.scala:472)
at
kafka.controller.KafkaController.onNewTopicCreation(KafkaController.scala:460)
at
kafka.controller.PartitionStateMachine$TopicChangeListener$$anonfun$handleChildChange$1.apply$mcV$sp(PartitionStateMachine.scala:422)
at
kafka.controller.PartitionStateMachine$TopicChangeListener$$anonfun$handleChildChange$1.apply(PartitionStateMachine.scala:404)
at
kafka.controller.PartitionStateMachine$TopicChangeListener$$anonfun$handleChildChange$1.apply(PartitionStateMachine.scala:404)
at kafka.utils.Utils$.inLock(Utils.scala:538)
at
kafka.controller.PartitionStateMachine$TopicChangeListener.handleChildChange(PartitionStateMachine.scala:403)
at org.I0Itec.zkclient.ZkClient$7.run(ZkClient.java:568)


[2014-09-03 21:18:41,112] ERROR Controller 5 epoch 5 encountered error
while changing partition [foo,2]'s state from New to Online since
LeaderAndIsr path already exists with value
{leader:1,leader_epoch:0,isr:[1,0]} and controller epoch 5
(state.change.logger)
[2014-09-03 21:18:41,112] ERROR Controller 5 epoch 5 initiated state change
for partition [foo,2] from NewPartition to OnlinePartition failed
(state.change.logger)
kafka.common.StateChangeFailedException: encountered error while changing

Re: zookeeper shows partition in-sync but local disk doesn't have the data dirs

2014-09-09 Thread Steven Wu
noticed one important thing. topic foo's partition 1 and 2 have empty .log
file on replicas. I suspect replication doesn't create the partition dir on
broker 0 in this case, which then cause the WARN logs.


On Mon, Sep 8, 2014 at 11:11 PM, Steven Wu stevenz...@gmail.com wrote:

 sorry. forgot to mention that I am running 0.8.1.1


 On Mon, Sep 8, 2014 at 9:26 PM, Steven Wu stevenz...@gmail.com wrote:

 did a push in cloud. after new instance for broker 0 comes up, I see a
 lot of WARNs in log file.

 2014-09-09 04:21:09,271 WARN  kafka.utils.Logging$class:83
 [request-expiration-task] [warn] [KafkaApi-0] Fetch request with
 correlation id 51893 from client 1409779957450-6014fc32-0-0 on partition
 [foo,1] failed due to Partition [foo,1] doesn't exist on 0
 2014-09-09 04:21:09,383 WARN  kafka.utils.Logging$class:83
 [kafka-request-handler-5] [warn] [KafkaApi-0] Fetch request with
 correlation id 51894 from client 1409779957450-6014fc32-0-0 on partition
 [foo,2] failed due to Partition [foo,2] doesn't exist on 0

 zookeeper shows it is the leader after this new instance come back.

 {controller_epoch:5,leader:0,version:1,leader_epoch:0,isr:[0,5]}
 {controller_epoch:5,leader:0,version:1,leader_epoch:1,isr:[0,1]}

 but we I check the data dirs. those two partitions aren't there.

 any idea?

 Thanks,
 Steven





Re: zookeeper shows partition in-sync but local disk doesn't have the data dirs

2014-09-09 Thread Jun Rao
Hmm, that seems to indicate that the leader info is not propagated properly
from the controller to broker 0. In the state-change log of broker 0, do
you see anything related to these 2 partitions when broker 0 comes up?

Thanks,

Jun

On Tue, Sep 9, 2014 at 9:41 AM, Steven Wu stevenz...@gmail.com wrote:

 noticed one important thing. topic foo's partition 1 and 2 have empty .log
 file on replicas. I suspect replication doesn't create the partition dir on
 broker 0 in this case, which then cause the WARN logs.


 On Mon, Sep 8, 2014 at 11:11 PM, Steven Wu stevenz...@gmail.com wrote:

  sorry. forgot to mention that I am running 0.8.1.1
 
 
  On Mon, Sep 8, 2014 at 9:26 PM, Steven Wu stevenz...@gmail.com wrote:
 
  did a push in cloud. after new instance for broker 0 comes up, I see a
  lot of WARNs in log file.
 
  2014-09-09 04:21:09,271 WARN  kafka.utils.Logging$class:83
  [request-expiration-task] [warn] [KafkaApi-0] Fetch request with
  correlation id 51893 from client 1409779957450-6014fc32-0-0 on partition
  [foo,1] failed due to Partition [foo,1] doesn't exist on 0
  2014-09-09 04:21:09,383 WARN  kafka.utils.Logging$class:83
  [kafka-request-handler-5] [warn] [KafkaApi-0] Fetch request with
  correlation id 51894 from client 1409779957450-6014fc32-0-0 on partition
  [foo,2] failed due to Partition [foo,2] doesn't exist on 0
 
  zookeeper shows it is the leader after this new instance come back.
 
 
 {controller_epoch:5,leader:0,version:1,leader_epoch:0,isr:[0,5]}
 
 {controller_epoch:5,leader:0,version:1,leader_epoch:1,isr:[0,1]}
 
  but we I check the data dirs. those two partitions aren't there.
 
  any idea?
 
  Thanks,
  Steven
 
 
 



Re: zookeeper shows partition in-sync but local disk doesn't have the data dirs

2014-09-09 Thread Steven Wu
nope. sate-change log files only had some warnings regarding other
partitions. nothing related to these two partitions.

2014-09-09 02:54:30,579 WARN  kafka.utils.Logging$class:83
[kafka-request-handler-1] [warn] Broker 0 received invalid LeaderAndIsr
request with correlation id 497 from controller 5 epoch 5 with an older
leader epoch 8 for partition [bar,1], current leader epoch is 8


On Tue, Sep 9, 2014 at 3:37 PM, Jun Rao jun...@gmail.com wrote:

 Hmm, that seems to indicate that the leader info is not propagated properly
 from the controller to broker 0. In the state-change log of broker 0, do
 you see anything related to these 2 partitions when broker 0 comes up?

 Thanks,

 Jun

 On Tue, Sep 9, 2014 at 9:41 AM, Steven Wu stevenz...@gmail.com wrote:

  noticed one important thing. topic foo's partition 1 and 2 have empty
 .log
  file on replicas. I suspect replication doesn't create the partition dir
 on
  broker 0 in this case, which then cause the WARN logs.
 
 
  On Mon, Sep 8, 2014 at 11:11 PM, Steven Wu stevenz...@gmail.com wrote:
 
   sorry. forgot to mention that I am running 0.8.1.1
  
  
   On Mon, Sep 8, 2014 at 9:26 PM, Steven Wu stevenz...@gmail.com
 wrote:
  
   did a push in cloud. after new instance for broker 0 comes up, I see a
   lot of WARNs in log file.
  
   2014-09-09 04:21:09,271 WARN  kafka.utils.Logging$class:83
   [request-expiration-task] [warn] [KafkaApi-0] Fetch request with
   correlation id 51893 from client 1409779957450-6014fc32-0-0 on
 partition
   [foo,1] failed due to Partition [foo,1] doesn't exist on 0
   2014-09-09 04:21:09,383 WARN  kafka.utils.Logging$class:83
   [kafka-request-handler-5] [warn] [KafkaApi-0] Fetch request with
   correlation id 51894 from client 1409779957450-6014fc32-0-0 on
 partition
   [foo,2] failed due to Partition [foo,2] doesn't exist on 0
  
   zookeeper shows it is the leader after this new instance come back.
  
  
 
 {controller_epoch:5,leader:0,version:1,leader_epoch:0,isr:[0,5]}
  
 
 {controller_epoch:5,leader:0,version:1,leader_epoch:1,isr:[0,1]}
  
   but we I check the data dirs. those two partitions aren't there.
  
   any idea?
  
   Thanks,
   Steven
  
  
  
 



Re: zookeeper shows partition in-sync but local disk doesn't have the data dirs

2014-09-09 Thread Steven Wu
assuming you are talking about controller log on broker 0, there is nothing
there.

$ less kafka-controller
2014-09-09 01:15:02,600 INFO  kafka.utils.Logging$class:68 [main] [info]
[ControllerEpochListener on 0]: Initialized controller epoch to 5 and zk
version 4
2014-09-09 01:15:02,655 INFO  kafka.utils.Logging$class:68 [main] [info]
[Controller 0]: Controller starting up
2014-09-09 01:15:02,692 INFO  kafka.utils.Logging$class:68 [main] [info]
[Controller 0]: Controller startup complete


On Tue, Sep 9, 2014 at 4:06 PM, Jun Rao jun...@gmail.com wrote:

 In the controller log, do you see broker 0 being detected as the new broker
 when broker 0 comes up?

 Thanks,

 Jun

 On Tue, Sep 9, 2014 at 3:51 PM, Steven Wu stevenz...@gmail.com wrote:

  nope. sate-change log files only had some warnings regarding other
  partitions. nothing related to these two partitions.
 
  2014-09-09 02:54:30,579 WARN  kafka.utils.Logging$class:83
  [kafka-request-handler-1] [warn] Broker 0 received invalid LeaderAndIsr
  request with correlation id 497 from controller 5 epoch 5 with an older
  leader epoch 8 for partition [bar,1], current leader epoch is 8
 
 
  On Tue, Sep 9, 2014 at 3:37 PM, Jun Rao jun...@gmail.com wrote:
 
   Hmm, that seems to indicate that the leader info is not propagated
  properly
   from the controller to broker 0. In the state-change log of broker 0,
 do
   you see anything related to these 2 partitions when broker 0 comes up?
  
   Thanks,
  
   Jun
  
   On Tue, Sep 9, 2014 at 9:41 AM, Steven Wu stevenz...@gmail.com
 wrote:
  
noticed one important thing. topic foo's partition 1 and 2 have empty
   .log
file on replicas. I suspect replication doesn't create the partition
  dir
   on
broker 0 in this case, which then cause the WARN logs.
   
   
On Mon, Sep 8, 2014 at 11:11 PM, Steven Wu stevenz...@gmail.com
  wrote:
   
 sorry. forgot to mention that I am running 0.8.1.1


 On Mon, Sep 8, 2014 at 9:26 PM, Steven Wu stevenz...@gmail.com
   wrote:

 did a push in cloud. after new instance for broker 0 comes up, I
  see a
 lot of WARNs in log file.

 2014-09-09 04:21:09,271 WARN  kafka.utils.Logging$class:83
 [request-expiration-task] [warn] [KafkaApi-0] Fetch request with
 correlation id 51893 from client 1409779957450-6014fc32-0-0 on
   partition
 [foo,1] failed due to Partition [foo,1] doesn't exist on 0
 2014-09-09 04:21:09,383 WARN  kafka.utils.Logging$class:83
 [kafka-request-handler-5] [warn] [KafkaApi-0] Fetch request with
 correlation id 51894 from client 1409779957450-6014fc32-0-0 on
   partition
 [foo,2] failed due to Partition [foo,2] doesn't exist on 0

 zookeeper shows it is the leader after this new instance come
 back.


   
  
 
 {controller_epoch:5,leader:0,version:1,leader_epoch:0,isr:[0,5]}

   
  
 
 {controller_epoch:5,leader:0,version:1,leader_epoch:1,isr:[0,1]}

 but we I check the data dirs. those two partitions aren't there.

 any idea?

 Thanks,
 Steven



   
  
 



Re: zookeeper shows partition in-sync but local disk doesn't have the data dirs

2014-09-09 Thread Steven Wu
ah. maybe you mean the controller log on leader/controller broker 5. yes. I
do noticed some errors regarding these two partitions.


[2014-09-09 01:10:53,651] ERROR Controller 5 epoch 5 encountered error
while changing partition [foo,2]'s state from New to Online since
LeaderAndIsr path alrea
dy exists with value {leader:0,leader_epoch:1,isr:[0,1]} and
controller epoch 5 (state.change.logger)
[2014-09-09 01:10:53,652] ERROR Controller 5 epoch 5 initiated state change
for partition [foo,2] from NewPartition to OnlinePartition failed
(state.change.logg
er)
kafka.common.StateChangeFailedException: encountered error while changing
partition [foo,2]'s state from New to Online since LeaderAndIsr path
already exists wi
th value {leader:0,leader_epoch:1,isr:[0,1]} and controller epoch 5
at
kafka.controller.PartitionStateMachine.initializeLeaderAndIsrForPartition(PartitionStateMachine.scala:302)
at
kafka.controller.PartitionStateMachine.kafka$controller$PartitionStateMachine$$handleStateChange(PartitionStateMachine.scala:183)
at
kafka.controller.PartitionStateMachine$$anonfun$triggerOnlinePartitionStateChange$3.apply(PartitionStateMachine.scala:99)
at
kafka.controller.PartitionStateMachine$$anonfun$triggerOnlinePartitionStateChange$3.apply(PartitionStateMachine.scala:96)
at
scala.collection.TraversableLike$WithFilter$$anonfun$foreach$1.apply(TraversableLike.scala:743)
at
scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:95)
at
scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:95)
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.foreach(HashMap.scala:95)
at
scala.collection.TraversableLike$WithFilter.foreach(TraversableLike.scala:742)
at
kafka.controller.PartitionStateMachine.triggerOnlinePartitionStateChange(PartitionStateMachine.scala:96)
at
kafka.controller.KafkaController.onBrokerFailure(KafkaController.scala:433)
at
kafka.controller.ReplicaStateMachine$BrokerChangeListener$$anonfun$handleChildChange$1$$anonfun$apply$mcV$sp$1.apply$mcV$sp(ReplicaStateMachine.scala:344)
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)


On Tue, Sep 9, 2014 at 4:08 PM, Steven Wu stevenz...@gmail.com wrote:

 assuming you are talking about controller log on broker 0, there is
 nothing there.

 $ less kafka-controller
 2014-09-09 01:15:02,600 INFO  kafka.utils.Logging$class:68 [main] [info]
 [ControllerEpochListener on 0]: Initialized controller epoch to 5 and zk
 version 4
 2014-09-09 01:15:02,655 INFO  kafka.utils.Logging$class:68 [main] [info]
 [Controller 0]: Controller starting up
 2014-09-09 01:15:02,692 INFO  kafka.utils.Logging$class:68 [main] [info]
 [Controller 0]: Controller startup complete


 On Tue, Sep 9, 2014 at 4:06 PM, Jun Rao jun...@gmail.com wrote:

 In the controller log, do you see broker 0 being detected as the new
 broker
 when broker 0 comes up?

 Thanks,

 Jun

 On Tue, Sep 9, 2014 at 3:51 PM, Steven Wu stevenz...@gmail.com wrote:

  nope. sate-change log files only had some warnings regarding other
  partitions. nothing related to these two partitions.
 
  2014-09-09 02:54:30,579 WARN  kafka.utils.Logging$class:83
  [kafka-request-handler-1] [warn] Broker 0 received invalid LeaderAndIsr
  request with correlation id 497 from controller 5 epoch 5 with an older
  leader epoch 8 for partition [bar,1], current leader epoch is 8
 
 
  On Tue, Sep 9, 2014 at 3:37 PM, Jun Rao jun...@gmail.com wrote:
 
   Hmm, that seems to indicate that the leader info is not propagated
  properly
   from the controller to broker 0. In the state-change log of 

Re: zookeeper shows partition in-sync but local disk doesn't have the data dirs

2014-09-09 Thread Steven Wu
previous email is from state-change.log. also found this WARN in
controller.log

[2014-09-09 01:10:53,669] WARN [Controller 5]: Cannot remove replica 0 from
ISR of partition [cdn_selection_runtime_data,8] since it is not in the ISR.
Leader = 1 ; ISR = List(1) (kafka.controller.KafkaController)


On Tue, Sep 9, 2014 at 4:14 PM, Steven Wu stevenz...@gmail.com wrote:

 ah. maybe you mean the controller log on leader/controller broker 5. yes.
 I do noticed some errors regarding these two partitions.


 [2014-09-09 01:10:53,651] ERROR Controller 5 epoch 5 encountered error
 while changing partition [foo,2]'s state from New to Online since
 LeaderAndIsr path alrea
 dy exists with value {leader:0,leader_epoch:1,isr:[0,1]} and
 controller epoch 5 (state.change.logger)
 [2014-09-09 01:10:53,652] ERROR Controller 5 epoch 5 initiated state
 change for partition [foo,2] from NewPartition to OnlinePartition failed
 (state.change.logg
 er)
 kafka.common.StateChangeFailedException: encountered error while changing
 partition [foo,2]'s state from New to Online since LeaderAndIsr path
 already exists wi
 th value {leader:0,leader_epoch:1,isr:[0,1]} and controller epoch 5
 at
 kafka.controller.PartitionStateMachine.initializeLeaderAndIsrForPartition(PartitionStateMachine.scala:302)
 at
 kafka.controller.PartitionStateMachine.kafka$controller$PartitionStateMachine$$handleStateChange(PartitionStateMachine.scala:183)
 at
 kafka.controller.PartitionStateMachine$$anonfun$triggerOnlinePartitionStateChange$3.apply(PartitionStateMachine.scala:99)
 at
 kafka.controller.PartitionStateMachine$$anonfun$triggerOnlinePartitionStateChange$3.apply(PartitionStateMachine.scala:96)
 at
 scala.collection.TraversableLike$WithFilter$$anonfun$foreach$1.apply(TraversableLike.scala:743)
 at
 scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:95)
 at
 scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:95)
 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.foreach(HashMap.scala:95)
 at
 scala.collection.TraversableLike$WithFilter.foreach(TraversableLike.scala:742)
 at
 kafka.controller.PartitionStateMachine.triggerOnlinePartitionStateChange(PartitionStateMachine.scala:96)
 at
 kafka.controller.KafkaController.onBrokerFailure(KafkaController.scala:433)
 at
 kafka.controller.ReplicaStateMachine$BrokerChangeListener$$anonfun$handleChildChange$1$$anonfun$apply$mcV$sp$1.apply$mcV$sp(ReplicaStateMachine.scala:344)
 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)


 On Tue, Sep 9, 2014 at 4:08 PM, Steven Wu stevenz...@gmail.com wrote:

 assuming you are talking about controller log on broker 0, there is
 nothing there.

 $ less kafka-controller
 2014-09-09 01:15:02,600 INFO  kafka.utils.Logging$class:68 [main] [info]
 [ControllerEpochListener on 0]: Initialized controller epoch to 5 and zk
 version 4
 2014-09-09 01:15:02,655 INFO  kafka.utils.Logging$class:68 [main] [info]
 [Controller 0]: Controller starting up
 2014-09-09 01:15:02,692 INFO  kafka.utils.Logging$class:68 [main] [info]
 [Controller 0]: Controller startup complete


 On Tue, Sep 9, 2014 at 4:06 PM, Jun Rao jun...@gmail.com wrote:

 In the controller log, do you see broker 0 being detected as the new
 broker
 when broker 0 comes up?

 Thanks,

 Jun

 On Tue, Sep 9, 2014 at 3:51 PM, Steven Wu stevenz...@gmail.com wrote:

  nope. sate-change log files only had some warnings regarding other
  partitions. nothing related to these two partitions.
 
  2014-09-09 02:54:30,579 WARN  kafka.utils.Logging$class:83
  

zookeeper shows partition in-sync but local disk doesn't have the data dirs

2014-09-08 Thread Steven Wu
did a push in cloud. after new instance for broker 0 comes up, I see a lot
of WARNs in log file.

2014-09-09 04:21:09,271 WARN  kafka.utils.Logging$class:83
[request-expiration-task] [warn] [KafkaApi-0] Fetch request with
correlation id 51893 from client 1409779957450-6014fc32-0-0 on partition
[foo,1] failed due to Partition [foo,1] doesn't exist on 0
2014-09-09 04:21:09,383 WARN  kafka.utils.Logging$class:83
[kafka-request-handler-5] [warn] [KafkaApi-0] Fetch request with
correlation id 51894 from client 1409779957450-6014fc32-0-0 on partition
[foo,2] failed due to Partition [foo,2] doesn't exist on 0

zookeeper shows it is the leader after this new instance come back.

{controller_epoch:5,leader:0,version:1,leader_epoch:0,isr:[0,5]}
{controller_epoch:5,leader:0,version:1,leader_epoch:1,isr:[0,1]}

but we I check the data dirs. those two partitions aren't there.

any idea?

Thanks,
Steven