Jun Thanks. To clarify, do you mean that clients will have cached broker
lists or some other data that will make them ignore the new brokers?

Like so

topic-1 replication factor 3, on broker-ids 1,2,3
all brokers 1,2,3 die, and are never coming back.
delete all kafka data in zookeeper.
boot 4,5,6, create new topic called topic-1 repl factor 3, brokers 4,5,6

clients will/will not start sending to topic-1 on 4,5,6?



On Sun, Mar 24, 2013 at 4:01 PM, Jun Rao <jun...@gmail.com> wrote:

> If you bring up 3 new brokers with different broker ids, you won't be able
> to use them on existing topics until after you have run the partition
> reassignment tool.
>
> Thanks,
>
> Jun
>
> On Fri, Mar 22, 2013 at 9:23 PM, Scott Clasen <sc...@heroku.com> wrote:
>
> > Thanks!
> >
> >  Would there be any difference if I instead  deleted all the Kafka data
> > from zookeeper and booted 3 instances  with different broker id? clients
> > with cached broker id lists or any other issue?
> >
> > Sent from my iPhone
> >
> > On Mar 22, 2013, at 9:15 PM, Jun Rao <jun...@gmail.com> wrote:
> >
> > > In scenario 2, you can bring up 3 new brokers with the same broker id.
> > You
> > > won't get the data back. However, new data can be published to and
> > consumed
> > > from the new brokers.
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > > On Fri, Mar 22, 2013 at 2:17 PM, Scott Clasen <sc...@heroku.com>
> wrote:
> > >
> > >> Thanks Neha-
> > >>
> > >> To Clarify...
> > >>
> > >> *In scenario => 1 will the new broker get all messages on the other
> > brokers
> > >> replicated to it?
> > >>
> > >> *In Scenario 2 => it is clear that the data is gone, but I still need
> > >> producers to be able to send and consumers to receive on the same
> > topic. In
> > >> my testing today I was unable to do that as I kept getting errors...so
> > if i
> > >> was doing the correct steps it seems there is a bug here, basically
> the
> > >> "second-cluster-topic" topic is unusable after all 3 brokers crash,
> and
> > 3
> > >> more are booted to replace them.  Something not quite correct in
> > zookeeper?
> > >>
> > >> Like so
> > >>
> > >> ./bin/kafka-reassign-partitions.sh --zookeeper ... --path-to-json-file
> > >> reassign.json
> > >>
> > >> kafka.common.LeaderNotAvailableException: Leader not available for
> topic
> > >> second-cluster-topic partition 0
> > >> at kafka.admin.AdminUtils$$anonfun$3.apply(AdminUtils.scala:120)
> > >> at kafka.admin.AdminUtils$$anonfun$3.apply(AdminUtils.scala:103)
> > >> at
> > >>
> > >>
> >
> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:206)
> > >> at
> > >>
> > >>
> >
> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:206)
> > >> at
> > >>
> > >>
> >
> scala.collection.LinearSeqOptimized$class.foreach(LinearSeqOptimized.scala:61)
> > >> at scala.collection.immutable.List.foreach(List.scala:45)
> > >> at
> scala.collection.TraversableLike$class.map(TraversableLike.scala:206)
> > >> at scala.collection.immutable.List.map(List.scala:45)
> > >> at
> > >>
> > >>
> >
> kafka.admin.AdminUtils$.kafka$admin$AdminUtils$$fetchTopicMetadataFromZk(AdminUtils.scala:103)
> > >> at
> kafka.admin.AdminUtils$.fetchTopicMetadataFromZk(AdminUtils.scala:92)
> > >> at kafka.admin.ListTopicCommand$.showTopic(ListTopicCommand.scala:80)
> > >> at
> > >>
> > >>
> >
> kafka.admin.ListTopicCommand$$anonfun$main$2.apply(ListTopicCommand.scala:66)
> > >> at
> > >>
> > >>
> >
> kafka.admin.ListTopicCommand$$anonfun$main$2.apply(ListTopicCommand.scala:65)
> > >> at
> > >>
> > >>
> >
> scala.collection.LinearSeqOptimized$class.foreach(LinearSeqOptimized.scala:61)
> > >> at scala.collection.immutable.List.foreach(List.scala:45)
> > >> at kafka.admin.ListTopicCommand$.main(ListTopicCommand.scala:65)
> > >> at kafka.admin.ListTopicCommand.main(ListTopicCommand.scala)
> > >> Caused by: kafka.common.LeaderNotAvailableException: No leader exists
> > for
> > >> partition 0
> > >> at kafka.admin.AdminUtils$$anonfun$3.apply(AdminUtils.scala:117)
> > >> ... 16 more
> > >> topic: second-cluster-topic
> > >>
> > >> ./bin/kafka-preferred-replica-election.sh  --zookeeper...
> > >> --path-to-json-file elect.json
> > >>
> > >>
> > >> ....[2013-03-22 10:24:20,706] INFO Created preferred replica election
> > path
> > >> with { "partitions":[ { "partition":0, "topic":"first-cluster-topic"
> },
> > {
> > >> "partition":0, "topic":"second-cluster-topic" } ], "version":1 }
> > >> (kafka.admin.PreferredReplicaLeaderElectionCommand$)
> > >>
> > >> ./bin/kafka-list-topic.sh  --zookeeper ... --topic
> second-cluster-topic
> > >>
> > >> 2013-03-22 10:24:30,869] ERROR Error while fetching metadata for
> > partition
> > >> [second-cluster-topic,0] (kafka.admin.AdminUtils$)
> > >> kafka.common.LeaderNotAvailableException: Leader not available for
> topic
> > >> second-cluster-topic partition 0
> > >> at kafka.admin.AdminUtils$$anonfun$3.apply(AdminUtils.scala:120)
> > >> at kafka.admin.AdminUtils$$anonfun$3.apply(AdminUtils.scala:103)
> > >> at
> > >>
> > >>
> >
> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:206)
> > >> at
> > >>
> > >>
> >
> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:206)
> > >> at
> > >>
> > >>
> >
> scala.collection.LinearSeqOptimized$class.foreach(LinearSeqOptimized.scala:61)
> > >> at scala.collection.immutable.List.foreach(List.scala:45)
> > >> at
> scala.collection.TraversableLike$class.map(TraversableLike.scala:206)
> > >> at scala.collection.immutable.List.map(List.scala:45)
> > >> at
> > >>
> > >>
> >
> kafka.admin.AdminUtils$.kafka$admin$AdminUtils$$fetchTopicMetadataFromZk(AdminUtils.scala:103)
> > >> at
> kafka.admin.AdminUtils$.fetchTopicMetadataFromZk(AdminUtils.scala:92)
> > >> at kafka.admin.ListTopicCommand$.showTopic(ListTopicCommand.scala:80)
> > >> at
> > >>
> > >>
> >
> kafka.admin.ListTopicCommand$$anonfun$main$2.apply(ListTopicCommand.scala:66)
> > >> at
> > >>
> > >>
> >
> kafka.admin.ListTopicCommand$$anonfun$main$2.apply(ListTopicCommand.scala:65)
> > >> at
> > >>
> > >>
> >
> scala.collection.LinearSeqOptimized$class.foreach(LinearSeqOptimized.scala:61)
> > >> at scala.collection.immutable.List.foreach(List.scala:45)
> > >> at kafka.admin.ListTopicCommand$.main(ListTopicCommand.scala:65)
> > >> at kafka.admin.ListTopicCommand.main(ListTopicCommand.scala)
> > >> Caused by: kafka.common.LeaderNotAvailableException: No leader exists
> > for
> > >> partition 0
> > >> at kafka.admin.AdminUtils$$anonfun$3.apply(AdminUtils.scala:117)
> > >> ... 16 more
> > >>
> > >>
> > >>
> > >>
> > >>
> > >> On Fri, Mar 22, 2013 at 1:12 PM, Neha Narkhede <
> neha.narkh...@gmail.com
> > >>> wrote:
> > >>
> > >>> * Scenario 1:  BrokerID 1,2,3   Broker 2 dies.
> > >>>
> > >>> Here, you can use reassign partitions tool and for all partitions
> that
> > >>> had a replica on broker 2, move it to broker 4
> > >>>
> > >>> * Scenario 2: BrokerID 1,2,3 Catastrophic failure 1,2,3 die but ZK
> > still
> > >>> there.
> > >>>
> > >>> There is no way to recover any data here since there is nothing
> > >>> available to consume data from.
> > >>>
> > >>> Thanks,
> > >>> Neha
> > >>>
> > >>> On Fri, Mar 22, 2013 at 10:46 AM, Scott Clasen <sc...@heroku.com>
> > wrote:
> > >>>> What would the recommended practice be for the following scenarios?
> > >>>>
> > >>>> Running on EC2, ephemperal disks only for kafka.
> > >>>>
> > >>>> There are 3 kafka servers. The broker ids are always increasing. If
> a
> > >>>> broker dies its never coming back.
> > >>>>
> > >>>> All topics have a replication factor of 3.
> > >>>>
> > >>>> * Scenario 1:  BrokerID 1,2,3   Broker 2 dies.
> > >>>>
> > >>>> Recover by:
> > >>>>
> > >>>> Boot another: BrokerID 4
> > >>>> ?? run bin/kafka-reassign-partitions.sh   for any topic+partition
> and
> > >>>> replace brokerid 2 with brokerid 4
> > >>>> ?? anything else to do to cause messages to be replicated to 4??
> > >>>>
> > >>>> NOTE: This appears to work but not positive 4 got messages
> replicated
> > >> to
> > >>> it.
> > >>>>
> > >>>> * Scenario 2: BrokerID 1,2,3 Catastrophic failure 1,2,3 die but ZK
> > >> still
> > >>>> there.
> > >>>>
> > >>>> Messages obviously lost.
> > >>>> Recover to a functional state by:
> > >>>>
> > >>>> Boot 3 more: 4,5 6
> > >>>> ?? run bin/kafka-reassign-partitions.sh  for all topics/partitions,
> > >> swap
> > >>>> 1,2,3 for 4,5,6?
> > >>>> ?? rin bin/kafka-preferred-replica-election.sh for all
> > >> topics/partitions
> > >>>> ?? anything else to do to allow producers to start sending
> > >> successfully??
> > >>>>
> > >>>>
> > >>>> NOTE: I had some trouble with scenario 2. Will try to reproduce and
> > >> open
> > >>> a
> > >>>> ticket, if in fact my procedures for scenario 2 are correct, and I
> > >> still
> > >>>> cant get to a good state.
> > >>
> >
>

Reply via email to