Sure. Thanks Neha.  Created Issue:
https://issues.apache.org/jira/browse/KAFKA-1041




On Wed, Sep 4, 2013 at 10:32 AM, Neha Narkhede <neha.narkh...@gmail.com>wrote:

> Ideally if the producer runs into any error, it should close the previous
> socket and open a new one. Seems like that is not happening here. I will
> take a closer look at this today. Do you mind filing a bug?
>
> Thanks,
> Neha
> On Sep 4, 2013 7:23 AM, "Rajasekar Elango" <rela...@salesforce.com> wrote:
>
> > I can easily reproduce this with console producer,  If I run console
> > producer with right hostname and if broker is not running, the console
> > producer will exit after three tries. But If I run console producer with
> > unresolvable broker, it throws below exception and continues to wait for
> > user input, every time I enter new message, it opens socket and file
> handle
> > count keeps increasing..
> >
> > Here is Exception in producer
> >
> > ERROR fetching topic metadata for topics [Set(test-1378245487417)] from
> > broker [ArrayBuffer(id:0,host:localhost1,port:6667)] failed
> > (kafka.utils.Utils$)
> > kafka.common.KafkaException: fetching topic metadata for topics
> > [Set(test-1378245487417)] from broker
> > [ArrayBuffer(id:0,host:localhost1,port:6667)] failed
> >         at
> > kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:51)
> >         at
> >
> kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)
> >         at
> >
> >
> kafka.producer.async.DefaultEventHandler$$anonfun$handle$2.apply$mcV$sp(DefaultEventHandler.scala:79)
> >         at kafka.utils.Utils$.swallow(Utils.scala:186)
> >         at kafka.utils.Logging$class.swallowError(Logging.scala:105)
> >         at kafka.utils.Utils$.swallowError(Utils.scala:45)
> >         at
> >
> >
> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:79)
> >         at
> >
> >
> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
> >         at
> >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
> >         at
> >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
> >         at scala.collection.immutable.Stream.foreach(Stream.scala:526)
> >         at
> >
> >
> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
> >         at
> > kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
> > Caused by: java.nio.channels.UnresolvedAddressException
> >         at sun.nio.ch.Net.checkAddress(Net.java:30)
> >         at
> sun.nio.ch.SocketChannelImpl.connect(SocketChannelImpl.java:487)
> >         at
> kafka.network.BlockingChannel.connect(BlockingChannel.scala:59)
> >         at kafka.producer.SyncProducer.connect(SyncProducer.scala:151)
> >         at
> > kafka.producer.SyncProducer.getOrMakeConnection(SyncProducer.scala:166)
> >         at
> >
> >
> kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:73)
> >         at kafka.producer.SyncProducer.send(SyncProducer.scala:117)
> >         at
> > kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:37)
> >         ... 12 more
> >
> >
> >
> > On Tue, Sep 3, 2013 at 9:29 PM, Neha Narkhede <neha.narkh...@gmail.com
> > >wrote:
> >
> > > Interesting. What errors/exceptions do you see in the producer logs?
> > >
> > > Thanks,
> > > Neha
> > >
> > >
> > > On Tue, Sep 3, 2013 at 3:28 PM, Rajasekar Elango <
> rela...@salesforce.com
> > > >wrote:
> > >
> > > > We found a issue that if broker host is un resolvable, the number of
> > file
> > > > handle keep increasing for every message we produce and eventually it
> > > uses
> > > > up all available files handles in operating system. If broker itself
> is
> > > not
> > > > running and broker host name is resolvable, open file handles count
> > stays
> > > > flat.
> > > >
> > > > lsof output shows number of these open file handles continue to grow
> > for
> > > > every message we produce.
> > > >
> > > >  java      19631    relango   81u     sock                0,6
>  0t0
> > > >  196966526 can't identify protocol
> > > >
> > > > Is this a bug is producer API..? What is best way to self protect our
> > > self
> > > > ?
> > > >
> > > > --
> > > > Thanks,
> > > > Raja.
> > > >
> > >
> >
> >
> >
> > --
> > Thanks,
> > Raja.
> >
>



-- 
Thanks,
Raja.

Reply via email to