|
|
-
no partition leader yet for partition 0 (0.8.0)
Chris Curtin 2012-11-27, 21:11
Hi,
I noticed several errors when writing to a topic with 5 partitions. It looks like the data was written to all 3 brokers, but I get the following errors:
9961 [main] DEBUG kafka.producer.BrokerPartitionInfo - Metadata for topic partition [test1, 0] is errornous: [PartitionMetadata(0,None,WrappedArray(),WrappedArray(),5)] kafka.common.LeaderNotAvailableException at sun.reflect.GeneratedConstructorAccessor1.newInstance(Unknown Source) at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:27) <snip>
9962 [main] DEBUG kafka.producer.async.DefaultEventHandler - Getting the number of broker partitions registered for topic: test1 9963 [main] DEBUG kafka.producer.BrokerPartitionInfo - Getting broker partition info for topic test1 9963 [main] DEBUG kafka.producer.BrokerPartitionInfo - Topic test1 partition 0 does not have a leader yet 9963 [main] DEBUG kafka.producer.BrokerPartitionInfo - Topic test1 partition 1 has leader 7 9963 [main] DEBUG kafka.producer.BrokerPartitionInfo - Topic test1 partition 2 has leader 8 9963 [main] DEBUG kafka.producer.BrokerPartitionInfo - Topic test1 partition 3 has leader 9 9963 [main] DEBUG kafka.producer.BrokerPartitionInfo - Topic test1 partition 4 has leader 1 9963 [main] DEBUG kafka.producer.async.DefaultEventHandler - Broker partitions registered for topic: test1 are 0,1,2,3,4
This happens a lot as I write data to the Broker.
Topic was created with:
-bash-3.2$ ./kafka-create-topic.sh --topic test1 --partition 5 --replica 3 --zookeeper localhost:2181
Doing a list of topics shows an empty list for that partition:
[2012-11-27 16:03:35,604] INFO Session establishment complete on server localhost/127.0.0.1:2181, sessionid = 0x23b4218eccd000b, negotiated timeout = 30000 (org.apache.zookeeper.ClientCnxn) [2012-11-27 16:03:35,607] INFO zookeeper state changed (SyncConnected) (org.I0Itec.zkclient.ZkClient) topic: test1 PartitionMetadata(0,None,List(),List(),5) PartitionMetadata(1,Some(id:7,creatorId:10.121.31.57-1354023708335,host:10.121.31.57,port:9092),List(id:7,creatorId:10.121.31.57-1354023708335,host:10.121.31.57,port:9092, id:8,creatorId:10.121.31.57-1354023708340,host:10.121.31.57,port:9093, id:9,creatorId:10.121.31.57-1354023944130,host:10.121.31.57,port:9094),ArrayBuffer(id:7,creatorId:10.121.31.57-1354023708335,host:10.121.31.57,port:9092, id:8,creatorId:10.121.31.57-1354023708340,host:10.121.31.57,port:9093, id:9,creatorId:10.121.31.57-1354023944130,host:10.121.31.57,port:9094),0) PartitionMetadata(2,Some(id:8,creatorId:10.121.31.57-1354023708340,host:10.121.31.57,port:9093),List(id:8,creatorId:10.121.31.57-1354023708340,host:10.121.31.57,port:9093, id:9,creatorId:10.121.31.57-1354023944130,host:10.121.31.57,port:9094, id:1,creatorId:10.121.31.55-1354023701351,host:10.121.31.55,port:9092),ArrayBuffer(id:8,creatorId:10.121.31.57-1354023708340,host:10.121.31.57,port:9093, id:9,creatorId:10.121.31.57-1354023944130,host:10.121.31.57,port:9094, id:1,creatorId:10.121.31.55-1354023701351,host:10.121.31.55,port:9092),0) PartitionMetadata(3,Some(id:9,creatorId:10.121.31.57-1354023944130,host:10.121.31.57,port:9094),List(id:9,creatorId:10.121.31.57-1354023944130,host:10.121.31.57,port:9094, id:1,creatorId:10.121.31.55-1354023701351,host:10.121.31.55,port:9092, id:2,creatorId:10.121.31.55-1354023701344,host:10.121.31.55,port:9093),ArrayBuffer(id:9,creatorId:10.121.31.57-1354023944130,host:10.121.31.57,port:9094, id:1,creatorId:10.121.31.55-1354023701351,host:10.121.31.55,port:9092, id:2,creatorId:10.121.31.55-1354023701344,host:10.121.31.55,port:9093),0) PartitionMetadata(4,Some(id:1,creatorId:10.121.31.55-1354023701351,host:10.121.31.55,port:9092),List(id:1,creatorId:10.121.31.55-1354023701351,host:10.121.31.55,port:9092, id:2,creatorId:10.121.31.55-1354023701344,host:10.121.31.55,port:9093, id:3,creatorId:10.121.31.55-1354023701345,host:10.121.31.55,port:9094),ArrayBuffer(id:1,creatorId:10.121.31.55-1354023701351,host:10.121.31.55,port:9092, id:2,creatorId:10.121.31.55-1354023701344,host:10.121.31.55,port:9093, id:3,creatorId:10.121.31.55-1354023701345,host:10.121.31.55,port:9094),0) [2012-11-27 16:03:36,005] INFO Terminate ZkClient event thread. (org.I0Itec.zkclient.ZkEventThread)
My partitioner logic is doing a simple modulo on the # of partitions passed:
return (int) (organizationId % a_numPartitions);
Did I miss a step setting up the topics?
Thanks,
Chris
-
Re: no partition leader yet for partition 0 (0.8.0)
Jun Rao 2012-11-28, 05:13
Is a broker down in your test? If so, you could see LeaderNotAvailableException in the producer. The producer is trying to refresh the metadata and the leader may not have been elected yet. You shouldn't see it often though.
Thanks,
Jun
On Tue, Nov 27, 2012 at 1:11 PM, Chris Curtin <[EMAIL PROTECTED]>wrote:
> Hi, > > I noticed several errors when writing to a topic with 5 partitions. It > looks like the data was written to all 3 brokers, but I get the following > errors: > > 9961 [main] DEBUG kafka.producer.BrokerPartitionInfo - Metadata for topic > partition [test1, 0] is errornous: > [PartitionMetadata(0,None,WrappedArray(),WrappedArray(),5)] > kafka.common.LeaderNotAvailableException > at sun.reflect.GeneratedConstructorAccessor1.newInstance(Unknown Source) > at > > sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:27) > <snip> > > 9962 [main] DEBUG kafka.producer.async.DefaultEventHandler - Getting the > number of broker partitions registered for topic: test1 > 9963 [main] DEBUG kafka.producer.BrokerPartitionInfo - Getting broker > partition info for topic test1 > 9963 [main] DEBUG kafka.producer.BrokerPartitionInfo - Topic test1 > partition 0 does not have a leader yet > 9963 [main] DEBUG kafka.producer.BrokerPartitionInfo - Topic test1 > partition 1 has leader 7 > 9963 [main] DEBUG kafka.producer.BrokerPartitionInfo - Topic test1 > partition 2 has leader 8 > 9963 [main] DEBUG kafka.producer.BrokerPartitionInfo - Topic test1 > partition 3 has leader 9 > 9963 [main] DEBUG kafka.producer.BrokerPartitionInfo - Topic test1 > partition 4 has leader 1 > 9963 [main] DEBUG kafka.producer.async.DefaultEventHandler - Broker > partitions registered for topic: test1 are 0,1,2,3,4 > > This happens a lot as I write data to the Broker. > > Topic was created with: > > -bash-3.2$ ./kafka-create-topic.sh --topic test1 --partition 5 --replica 3 > --zookeeper localhost:2181 > > Doing a list of topics shows an empty list for that partition: > > [2012-11-27 16:03:35,604] INFO Session establishment complete on server > localhost/127.0.0.1:2181, sessionid = 0x23b4218eccd000b, negotiated > timeout > = 30000 (org.apache.zookeeper.ClientCnxn) > [2012-11-27 16:03:35,607] INFO zookeeper state changed (SyncConnected) > (org.I0Itec.zkclient.ZkClient) > topic: test1 > PartitionMetadata(0,None,List(),List(),5) > > PartitionMetadata(1,Some(id:7,creatorId:10.121.31.57-1354023708335,host:10.121.31.57,port:9092),List(id:7,creatorId:10.121.31.57-1354023708335,host:10.121.31.57,port:9092, > id:8,creatorId:10.121.31.57-1354023708340,host:10.121.31.57,port:9093, > > id:9,creatorId:10.121.31.57-1354023944130,host:10.121.31.57,port:9094),ArrayBuffer(id:7,creatorId:10.121.31.57-1354023708335,host:10.121.31.57,port:9092, > id:8,creatorId:10.121.31.57-1354023708340,host:10.121.31.57,port:9093, > id:9,creatorId:10.121.31.57-1354023944130,host:10.121.31.57,port:9094),0) > > PartitionMetadata(2,Some(id:8,creatorId:10.121.31.57-1354023708340,host:10.121.31.57,port:9093),List(id:8,creatorId:10.121.31.57-1354023708340,host:10.121.31.57,port:9093, > id:9,creatorId:10.121.31.57-1354023944130,host:10.121.31.57,port:9094, > > id:1,creatorId:10.121.31.55-1354023701351,host:10.121.31.55,port:9092),ArrayBuffer(id:8,creatorId:10.121.31.57-1354023708340,host:10.121.31.57,port:9093, > id:9,creatorId:10.121.31.57-1354023944130,host:10.121.31.57,port:9094, > id:1,creatorId:10.121.31.55-1354023701351,host:10.121.31.55,port:9092),0) > > PartitionMetadata(3,Some(id:9,creatorId:10.121.31.57-1354023944130,host:10.121.31.57,port:9094),List(id:9,creatorId:10.121.31.57-1354023944130,host:10.121.31.57,port:9094, > id:1,creatorId:10.121.31.55-1354023701351,host:10.121.31.55,port:9092, > > id:2,creatorId:10.121.31.55-1354023701344,host:10.121.31.55,port:9093),ArrayBuffer(id:9,creatorId:10.121.31.57-1354023944130,host:10.121.31.57,port:9094, > id:1,creatorId:10.121.31.55-1354023701351,host:10.121.31.55,port:9092, > id:2,creatorId:10.121.31.55-1354023701344,host:10.121.31.55,port:9093),0)
-
Re: no partition leader yet for partition 0 (0.8.0)
Chris Curtin 2013-03-04, 19:21
I was able to reproduce today after upgrading to HEAD (as of 3/4/2013) JIRA: https://issues.apache.org/jira/browse/KAFKA-784On Mon, Dec 3, 2012 at 11:09 AM, Chris Curtin <[EMAIL PROTECTED]>wrote: > Hi Jun, > > Couldn't make either happen again with a clean start (removed all Kafka > and Zookeeper configuration and data files). > > Thanks, > > Chris > > > On Wed, Nov 28, 2012 at 12:12 PM, Chris Curtin <[EMAIL PROTECTED]>wrote: > >> Hi Jun, >> >> Sorry, neither the missing 0 leader or all those WARN messages have >> been reproducible. Tried several times this morning. >> >> I'll be starting from a green-field cluster again this afternoon so I'll >> keep an eye out for it happening again. >> >> Thanks, >> >> Chris >> >> >> On Wed, Nov 28, 2012 at 12:08 PM, Jun Rao <[EMAIL PROTECTED]> wrote: >> >>> Chris, >>> >>> Not sure what happened to the WARN logging that you saw. Is that easily >>> reproducible? As for log4j, you just need to change log4j.properties. You >>> can find out on the web how to configure a rolling log file. >>> >>> Thanks, >>> >>> Jun >>> >>> On Wed, Nov 28, 2012 at 5:10 AM, Chris Curtin <[EMAIL PROTECTED] >>> >wrote: >>> >>> > Hi Jun, >>> > >>> > No, all 9 brokers are up and when I look at the files in >>> /opt/kafka-[]-logs >>> > there is data for partition 0 of that topic on 3 different brokers. >>> > >>> > After confirming this was still happening this morning, I bounced all >>> the >>> > brokers and on restart one of them took over primary on partition 0. No >>> > more errors after reboot. >>> > >>> > However, I now have a different problem. To see if the issue was >>> creating a >>> > new topic with all the brokers live, I created a new topic using the >>> same >>> > command line as below. The list_topics show it was created with >>> primaries >>> > on all partitions. However on one of machines (with 3 brokers running >>> (1,2& >>> > 3) ) I keep getting the following warning: >>> > >>> > [2012-11-28 07:56:46,014] WARN [ReplicaFetcherThread-9-0-on-broker-1], >>> > error for test2 2 to broker 9 (kafka.server.ReplicaFetcherThread) >>> > kafka.common.UnknownTopicOrPartitionException >>> > at >>> sun.reflect.GeneratedConstructorAccessor1.newInstance(Unknown >>> > Source) >>> > at >>> > >>> > >>> sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:27) >>> > at >>> java.lang.reflect.Constructor.newInstance(Constructor.java:513) >>> > at java.lang.Class.newInstance0(Class.java:355) >>> > at java.lang.Class.newInstance(Class.java:308) >>> > at >>> kafka.common.ErrorMapping$.exceptionFor(ErrorMapping.scala:70) >>> > at >>> > >>> > >>> kafka.server.AbstractFetcherThread$$anonfun$doWork$5$$anonfun$apply$3.apply(AbstractFetcherThread.scala:131) >>> > at >>> > >>> > >>> kafka.server.AbstractFetcherThread$$anonfun$doWork$5$$anonfun$apply$3.apply(AbstractFetcherThread.scala:131) >>> > at kafka.utils.Logging$class.warn(Logging.scala:88) >>> > at >>> kafka.utils.ShutdownableThread.warn(ShutdownableThread.scala:23) >>> > at >>> > >>> > >>> kafka.server.AbstractFetcherThread$$anonfun$doWork$5.apply(AbstractFetcherThread.scala:130) >>> > at >>> > >>> > >>> kafka.server.AbstractFetcherThread$$anonfun$doWork$5.apply(AbstractFetcherThread.scala:106) >>> > at scala.collection.immutable.Map$Map2.foreach(Map.scala:127) >>> > at >>> > >>> kafka.server.AbstractFetcherThread.doWork(AbstractFetcherThread.scala:106) >>> > at >>> kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:50) >>> > [2012-11-28 07:56:46,289] WARN [ReplicaFetcherThread-8-0-on-broker-1], >>> > error for test2 1 to broker 8 (kafka.server.ReplicaFetcherThread) >>> > kafka.common.UnknownTopicOrPartitionException >>> > at >>> sun.reflect.GeneratedConstructorAccessor1.newInstance(Unknown >>> > Source) >>> > at >>> > >>> > >>> sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:27)
|
|