Do you see the following warn messages on the producer ?

Fetching topic metadata with correlation id %d for topics [%s] from broker
[%s] failed

fetching topic metadata for topics [%s] from broker [%s] failed


On the server side -

1. could you enable DEBUG on kafka.server.KafkaApis and see if you get the
following message -

    Error while fetching topic metadata for topic %s due to %s

2. Do you see the following ERROR message on the server -

"Error while fetching metadata for partition %s"





On Wed, Oct 9, 2013 at 10:57 PM, Shafaq <s.abdullah...@gmail.com> wrote:

> I did some more debugging and found the corresponding debug message on the
> broker for topic (417topic1381383668416) in the producer:
>
> Seems like broker is removing fetcher for partition as producer throws no
> metadata for topic.
>
>
> [2013-10-09 22:41:09,529] INFO [Log Manager on Broker 2] Created log for
> partition [417topic1381383668416,3] in /tmp/kafka-logs-3.
> (kafka.log.LogManager)
> [2013-10-09 22:41:09,532] WARN No previously checkpointed highwatermark
> value found for topic 417topic1381383668416 partition 3. Returning 0 as the
> highwatermark (kafka.server.HighwaterMarkCheckpoint)
> [2013-10-09 22:41:09,533] INFO [ReplicaFetcherManager on broker 2] Removing
> fetcher for partition [417topic1381383668416,2]
> (kafka.server.ReplicaFetcherManager)
> [2013-10-09 22:41:09,534] INFO [Kafka Log on Broker 2], Completed load of
> log 417topic1381383668416-2 with log end offset 0 (kafka.log.Log)
> [2013-10-09 22:41:09,534] INFO [Log Manager on Broker 2] Created log for
> partition [417topic1381383668416,2] in /tmp/kafka-logs-3.
> (kafka.log.LogManager)
> [2013-10-09 22:41:09,537] WARN No previously checkpointed highwatermark
> value found for topic 417topic1381383668416 partition 2. Returning 0 as the
> highwatermark (kafka.server.HighwaterMarkCheckpoint)
> [2013-10-09 22:41:09,538] INFO [ReplicaFetcherManager on broker 2] Removing
> fetcher for partition [417topic1381383668416,4]
> (kafka.server.ReplicaFetcherManager)
> [2013-10-09 22:41:09,539] INFO [Kafka Log on Broker 2], Completed load of
> log 417topic1381383668416-4 with log end offset 0 (kafka.log.Log)
> [2013-10-09 22:41:09,539] INFO [Log Manager on Broker 2] Created log for
> partition [417topic1381383668416,4] in /tmp/kafka-logs-3.
> (kafka.log.LogManager)
> [2013-10-09 22:41:09,542] WARN No previously checkpointed highwatermark
> value found for topic 417topic1381383668416 partition 4. Returning 0 as the
> highwatermark (kafka.server.HighwaterMarkCheckpoint)
> [2013-10-09 22:41:09,543] INFO [ReplicaFetcherManager on broker 2] Removing
> fetcher for partition [417topic1381383668416,0]
> (kafka.server.ReplicaFetcherManager)
> [2013-10-09 22:41:09,544] INFO [Kafka Log on Broker 2], Completed load of
> log 417topic1381383668416-0 with log end offset 0 (kafka.log.Log)
> [2013-10-09 22:41:09,545] INFO [Log Manager on Broker 2] Created log for
> partition [417topic1381383668416,0] in /tmp/kafka-logs-3.
> (kafka.log.LogManager)
> [2013-10-09 22:41:09,547] WARN No previously checkpointed highwatermark
> value found for topic 417topic1381383668416 partition 0. Returning 0 as the
> highwatermark (kafka.server.HighwaterMarkCheckpoint)
> [2013-10-09 22:41:09,548] INFO [ReplicaFetcherManager on broker 2] Removing
> fetcher for partition [417topic1381383668416,1]
> (kafka.server.ReplicaFetcherManager)
> [2013-10-09 22:41:09,549] INFO [Kafka Log on Broker 2], Completed load of
> log 417topic1381383668416-1 with log end offset 0 (kafka.log.Log)
> [2013-10-09 22:41:09,549] INFO [Log Manager on Broker 2] Created log for
> partition [417topic1381383668416,1] in /tmp/kafka-logs-3.
> (kafka.log.LogManager)
> [2013-10-09 22:41:09,551] WARN No previously checkpointed highwatermark
> value found for topic 417topic1381383668416 partition 1. Returning 0 as the
> highwatermark (kafka.server.HighwaterMarkCheckpoint)
> [2013-10-09 22:41:09,552] INFO [Replica Manager on Broker 2]: Handled
> leader and isr request
>
> Name:LeaderAndIsrRequest;Version:0;Controller:2;ControllerEpoch:2;CorrelationId:18;ClientId:id_2-host_stg-analytics-002.int.funzio.com-port_9092;PartitionState:(417topic1381383668416,3)
> ->
>
> (LeaderAndIsrInfo:(Leader:2,ISR:2,LeaderEpoch:0,ControllerEpoch:2),ReplicationFactor:1),AllReplicas:2),(417topic1381383668416,2)
> ->
>
> (LeaderAndIsrInfo:(Leader:2,ISR:2,LeaderEpoch:0,ControllerEpoch:2),ReplicationFactor:1),AllReplicas:2),(417topic1381383668416,4)
> ->
>
> (LeaderAndIsrInfo:(Leader:2,ISR:2,LeaderEpoch:0,ControllerEpoch:2),ReplicationFactor:1),AllReplicas:2),(417topic1381383668416,0)
> ->
>
> (LeaderAndIsrInfo:(Leader:2,ISR:2,LeaderEpoch:0,ControllerEpoch:2),ReplicationFactor:1),AllReplicas:2),(417topic1381383668416,1)
> ->
>
> (LeaderAndIsrInfo:(Leader:2,ISR:2,LeaderEpoch:0,ControllerEpoch:2),ReplicationFactor:1),AllReplicas:2);Leaders:id:2,host:
> stg-analytics-002.int.funzio.com,port:9092 (kafka.server.ReplicaManager)
> java.io.IOException: No such file or directory
>
>
>
> On Wed, Oct 9, 2013 at 10:25 PM, Shafaq <s.abdullah...@gmail.com> wrote:
>
> > I could not see such messages (Error while fetching metadata [%s] for
> > topic [%s]: %s ")
> >
> > In the perf test,  I create new topic for sending the JSON blob in a new
> > producer thread.
> >
> > The error happened when I increased no. of files (mapping to producer
> > threads). There are already topics of previous performance test present
> in
> > the kafka cluster having 0.8 head version.
> >
> >
> > Regards,
> > S.Abdullah
> >
> >
> >
> > On Wed, Oct 9, 2013 at 6:08 PM, Neha Narkhede <neha.narkh...@gmail.com
> >wrote:
> >
> >> Do you see WARN messages like -
> >>
> >> "Error while fetching metadata [%s] for topic [%s]: %s "
> >> OR
> >> "Error while fetching metadata %s for topic partition [%s,%d]: [%s]"
> >>
> >>
> >> On Wed, Oct 9, 2013 at 5:49 PM, Shafaq <s.abdullah...@gmail.com> wrote:
> >>
> >> > yes I'm using 0.8 head
> >> >
> >> >
> >> > roducer starting with option to multi-thread
> >> > ----compressed---true
> >> > Exception in thread "pool-1-thread-1" kafka.common.KafkaException:
> >> Failed
> >> > to fetch topic metadata for topic: 225topic1381362148396
> >> >     at
> >> >
> >> >
> >>
> kafka.producer.BrokerPartitionInfo.getBrokerPartitionInfo(BrokerPartitionInfo.scala:53)
> >> >     at
> >> >
> >> >
> >>
> kafka.producer.async.DefaultEventHandler.kafka$producer$async$DefaultEventHandler$$getPartitionListForTopic(DefaultEventHandler.scala:185)
> >> >     at
> >> >
> >> >
> >>
> kafka.producer.async.DefaultEventHandler$$anonfun$partitionAndCollate$1.apply(DefaultEventHandler.scala:149)
> >> >     at
> >> >
> >> >
> >>
> kafka.producer.async.DefaultEventHandler$$anonfun$partitionAndCollate$1.apply(DefaultEventHandler.scala:148)
> >> >     at
> >> >
> >> >
> >>
> scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:57)
> >> >     at
> >> scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:43)
> >> >     at
> >> >
> >> >
> >>
> kafka.producer.async.DefaultEventHandler.partitionAndCollate(DefaultEventHandler.scala:148)
> >> >     at
> >> >
> >> >
> >>
> kafka.producer.async.DefaultEventHandler.dispatchSerializedData(DefaultEventHandler.scala:94)
> >> >     at
> >> >
> >> >
> >>
> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:72)
> >> >     at kafka.producer.Producer.send(Producer.scala:74)
> >> >     at kafka.javaapi.producer.Producer.send(Producer.scala:32)
> >> >     at
> >> >
> >> >
> >>
> gree.relayx.producer.distributed.GreeKafkaProducerDistributed.run(GreeKafkaProducerDistributed.java:124)
> >> >     at
> >> >
> >> >
> >>
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> >> >     at
> >> >
> >> >
> >>
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> >> >     at java.lang.Thread.run(Thread.java:724)
> >> >
> >> >
> >> > ----Processing file multithreaded
> >> > mode----/mnt/prod_logs/relay_data_node-analytics-001_1380412084.gz
> >>  with
> >> > compression enabled = true
> >> > Exception in thread "pool-1-thread-2" kafka.common.KafkaException:
> >> Failed
> >> > to fetch topic metadata for topic: 166topic1381362148385
> >> >     at
> >> >
> >> >
> >>
> kafka.producer.BrokerPartitionInfo.getBrokerPartitionInfo(BrokerPartitionInfo.scala:53)
> >> >     at
> >> >
> >> >
> >>
> kafka.producer.async.DefaultEventHandler.kafka$producer$async$DefaultEventHandler$$getPartitionListForTopic(DefaultEventHandler.scala:185)
> >> >     at
> >> >
> >> >
> >>
> kafka.producer.async.DefaultEventHandler$$anonfun$partitionAndCollate$1.apply(DefaultEventHandler.scala:149)
> >> >     at
> >> >
> >> >
> >>
> kafka.producer.async.DefaultEventHandler$$anonfun$partitionAndCollate$1.apply(DefaultEventHandler.scala:148)
> >> >     at
> >> >
> >> >
> >>
> scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:57)
> >> >     at
> >> scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:43)
> >> >     at
> >> >
> >> >
> >>
> kafka.producer.async.DefaultEventHandler.partitionAndCollate(DefaultEventHandler.scala:148)
> >> >     at
> >> >
> >> >
> >>
> kafka.producer.async.DefaultEventHandler.dispatchSerializedData(DefaultEventHandler.scala:94)
> >> >     at
> >> >
> >> >
> >>
> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:72)
> >> >     at kafka.producer.Producer.send(Producer.scala:74)
> >> >     at kafka.javaapi.producer.Producer.send(Producer.scala:32)
> >> >     at
> >> >
> >> >
> >>
> gree.relayx.producer.distributed.GreeKafkaProducerDistributed.run(GreeKafkaProducerDistributed.java:124)
> >> >     at
> >> >
> >> >
> >>
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> >> >     at
> >> >
> >> >
> >>
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> >> >     at java.lang.Thread.run(Thread.java:724)
> >> >
> >> >
> >> > On Wed, Oct 9, 2013 at 5:37 PM, Neha Narkhede <
> neha.narkh...@gmail.com
> >> > >wrote:
> >> >
> >> > > Are you using 0.8 HEAD ?
> >> > >
> >> > > Can you send around the full stack trace since one of the common
> >> reasons
> >> > > for failed topic metadata requests are socket timeouts.
> >> > >
> >> > > Thanks,
> >> > > Neha
> >> > >
> >> > >
> >> > > On Wed, Oct 9, 2013 at 4:30 PM, Shafaq <s.abdullah...@gmail.com>
> >> wrote:
> >> > >
> >> > > > I run into following error while doing some performance testing in
> >> > Kafka
> >> > > > with producers running in multiple threads.
> >> > > >
> >> > > > I can see the topic under /broker/topics in zooKeeper
> >> > > >
> >> > > > I believe the producer tries to get the metadata info from
> >> Zookeeper.
> >> > > >
> >> > > > I have tried to restart the 2-node Kafka broker cluster but of no
> >> use.
> >> > > >
> >> > > > Any idea what can cause this to happen?
> >> > > > I'm using kafka 0.8 version
> >> > > >
> >> > > >
> >> > >
> >> >
> >>
> ---------------------------------------------------------------------------------------
> >> > > >
> >> > > > Exception in thread "pool-1-thread-1" kafka.common.KafkaException:
> >> > Failed
> >> > > > to fetch topic metadata for topic: 37topic1381360774212
> >> > > >     at
> >> > > >
> >> > > >
> >> > >
> >> >
> >>
> kafka.producer.BrokerPartitionInfo.getBrokerPartitionInfo(BrokerPartitionInfo.scala:53)
> >> > > >     at
> >> > > >
> >> > > >
> >> > >
> >> >
> >>
> kafka.producer.async.DefaultEventHandler.kafka$producer$async$DefaultEventHandler$$getPartitionListForTopic(DefaultEventHandler.scala:185)
> >> > > > --
> >> > > > Kind Regards,
> >> > > > Shafaq
> >> > > >
> >> > >
> >> >
> >> >
> >> >
> >> > --
> >> > Kind Regards,
> >> > Shafaq
> >> >
> >>
> >
> >
> >
> > --
> > Kind Regards,
> > Shafaq
> >
> >
>
>
> --
> Kind Regards,
> Shafaq
>

Reply via email to