The create command tool will only create the path /brokers/topics/uattoqaaa.default, and then controller, once noticing the change in ZK about the added topic, will elect leaders from the assigned replicas written in /brokers/topics/uattoqaaa.default and then create /brokers/topics/uattoqaaa.default/partitions/[partition-id]/state.
If you have created the topic for a long time but /brokers/topics/uattoqaaa.default/partitions/[partition-id]/state is not created by controller, it could because the replicas specified in /brokers/topics/uattoqaaa.default does not exist yet. Could you verify the brokers specified in this path are up and running? Guozhang On Tue, Aug 20, 2013 at 12:14 PM, Yu, Libo <libo...@citi.com> wrote: > One more question: > "The create topic command will not immediately create the path; it will be > created by the controller later during the creation of partitions and > assign replicas to partitions." > > So when will the controller create the path and partitions? When the first > message is published to the topic and partition? > All the partitions are created at once? > > Regards, > > Libo > > > -----Original Message----- > From: Guozhang Wang [mailto:wangg...@gmail.com] > Sent: Tuesday, August 20, 2013 1:48 PM > To: users@kafka.apache.org > Subject: Re: issue with kafka-preferred-replica-election.sh > > Could you check that before you run kafka-preferred-replica-election.sh > the /brokers/topics/uattoqaaa.default/partitions Node has been created or > not? > The create topic command will not immediately create the path; it will be > created by the controller later during the creation of partitions and > assign replicas to partitions. If you start the election command right > after the create topic tool, it might not be created yet. > > Guozhang > > > On Tue, Aug 20, 2013 at 10:24 AM, Yu, Libo <libo...@citi.com> wrote: > > > It is created with 3 partitions. > > > > Regards, > > > > Libo > > > > > > -----Original Message----- > > From: Guozhang Wang [mailto:wangg...@gmail.com] > > Sent: Tuesday, August 20, 2013 12:21 PM > > To: users@kafka.apache.org > > Subject: Re: issue with kafka-preferred-replica-election.sh > > > > Hello Libo, > > > > Before you call the kafka-preferred-replica-election.sh have you > > already created the topic uattoqaaa.default with the number of > partitions? > > > > Guozhang > > > > > > On Tue, Aug 20, 2013 at 8:35 AM, Yu, Libo <libo...@citi.com> wrote: > > > > > Hi, > > > > > > I tried to run kafka-preferred-replica-election.sh on our kafka > cluster. > > > But I got this expection: > > > Failed to start preferred replica election > > > org.I0Itec.zkclient.exception.ZkNoNodeException: > > > org.apache.zookeeper.KeeperException$NoNodeException: > > > KeeperErrorCode = NoNode for > > > /brokers/topics/uattoqaaa.default/partitions > > > > > > I checked zookeeper and there is no > > > /brokers/topics/uattoqaaa.default/partitions. All I found is > > > /brokers/topics/uattoqaaa.default. > > > > > > I believe this is a bug and I am not sure if there is a JIRA for it. > > > Could you confirm? Thanks. > > > > > > Regards, > > > > > > Libo > > > > > > > > > > > > -- > > -- Guozhang > > > > > > -- > -- Guozhang > -- -- Guozhang