You are viewing a plain text version of this content. The canonical link for it is here.
Posted to users@kafka.apache.org by Arjun <ar...@socialtwist.com> on 2014/04/10 16:39:19 UTC

consumer not consuming messages

Hi,

We have 3 node kafka 0.8 setup with zookeepers ensemble. We use high 
level consumer with auto commit offset false. I am facing some peculiar 
problem with kafka. When i send some 10-20 messages or so the consumer 
starts to consume the messages. But if  i send only one message to 
kafka, then even though consumer is active it is not trying to fetch the 
message. There is nothing in logs, just the messages are being fetched 
by the kafka consumer. The messages are there in the Kafka server. Can 
some one let me know where i am doing wrong.


Thanks
Arjun Narasimha Kota

Re: consumer not consuming messages

Posted by Jun Rao <ju...@gmail.com>.
Are you using the high-level consumer? How did you set fetch.wait.max.msand
fetch.min.bytes?

Thanks,

Jun


On Thu, Apr 10, 2014 at 8:13 PM, Arjun <ar...@socialtwist.com> wrote:

> This just dosent happen only when the topic is newly created, happens even
> if topic has lot of messgaes, but all messages are consumed by the
> consumer. Now if you add just one message, consumer will not fetch, if at
> that scenario we add more than 10 messages things work fine. (10 is just a
> arbitary number)
>
> Thanks
> Arjun Narasimha Kota
>
>
> On Friday 11 April 2014 06:23 AM, Arjun Kota wrote:
>
>>
>> The consumer uses do specific topics.
>>
>> On Apr 11, 2014 6:23 AM, "Arjun Kota" <arjun@socialtwist.com <mailto:
>> arjun@socialtwist.com>> wrote:
>>
>>     Yes the message shows up on the server.
>>
>>     On Apr 11, 2014 12:07 AM, "Guozhang Wang" <wangguoz@gmail.com
>>     <ma...@gmail.com>> wrote:
>>
>>         Hi Arjun,
>>
>>         If you only send one message, does that message show up on the
>>         server? Does
>>         you consumer use wildcard topics or specific topics?
>>
>>         Guozhang
>>
>>
>>         On Thu, Apr 10, 2014 at 9:20 AM, Arjun <arjun@socialtwist.com
>>         <ma...@socialtwist.com>> wrote:
>>
>>         > But  we have auto offset reset to smallest not largest, even
>>         then this
>>         > issue arises? If so is there any work around?
>>         >
>>         > Thanks
>>         > Arjun NArasimha Kota
>>         >
>>         >
>>         > On Thursday 10 April 2014 09:39 PM, Guozhang Wang wrote:
>>         >
>>         >> It could be https://issues.apache.org/jira/browse/KAFKA-1006.
>>         >>
>>         >> Guozhang
>>         >>
>>         >>
>>         >> On Thu, Apr 10, 2014 at 8:50 AM, Arjun
>>         <arjun@socialtwist.com <ma...@socialtwist.com>> wrote:
>>         >>
>>         >>  its auto created
>>         >>> but even after topic creation this is the scenario
>>         >>>
>>         >>> Arjun
>>         >>>
>>         >>> On Thursday 10 April 2014 08:41 PM, Guozhang Wang wrote:
>>         >>>
>>         >>>  Hi Arjun,
>>         >>>>
>>         >>>> Did you manually create the topic or use auto.topic.creation?
>>         >>>>
>>         >>>> Guozhang
>>         >>>>
>>         >>>>
>>         >>>> On Thu, Apr 10, 2014 at 7:39 AM, Arjun
>>         <arjun@socialtwist.com <ma...@socialtwist.com>> wrote:
>>         >>>>
>>         >>>>   Hi,
>>         >>>>
>>         >>>>> We have 3 node kafka 0.8 setup with zookeepers ensemble.
>>         We use high
>>         >>>>> level
>>         >>>>> consumer with auto commit offset false. I am facing some
>>         peculiar
>>         >>>>> problem
>>         >>>>> with kafka. When i send some 10-20 messages or so the
>>         consumer starts
>>         >>>>> to
>>         >>>>> consume the messages. But if  i send only one message to
>>         kafka, then
>>         >>>>> even
>>         >>>>> though consumer is active it is not trying to fetch the
>>         message. There
>>         >>>>> is
>>         >>>>> nothing in logs, just the messages are being fetched by
>>         the kafka
>>         >>>>> consumer.
>>         >>>>> The messages are there in the Kafka server. Can some one
>>         let me know
>>         >>>>> where
>>         >>>>> i am doing wrong.
>>         >>>>>
>>         >>>>>
>>         >>>>> Thanks
>>         >>>>> Arjun Narasimha Kota
>>         >>>>>
>>         >>>>>
>>         >>>>>
>>         >>>>
>>         >>
>>         >
>>
>>
>>         --
>>         -- Guozhang
>>
>>
>

Re: consumer not consuming messages

Posted by Arjun <ar...@socialtwist.com>.
This just dosent happen only when the topic is newly created, happens 
even if topic has lot of messgaes, but all messages are consumed by the 
consumer. Now if you add just one message, consumer will not fetch, if 
at that scenario we add more than 10 messages things work fine. (10 is 
just a arbitary number)

Thanks
Arjun Narasimha Kota

On Friday 11 April 2014 06:23 AM, Arjun Kota wrote:
>
> The consumer uses do specific topics.
>
> On Apr 11, 2014 6:23 AM, "Arjun Kota" <arjun@socialtwist.com 
> <ma...@socialtwist.com>> wrote:
>
>     Yes the message shows up on the server.
>
>     On Apr 11, 2014 12:07 AM, "Guozhang Wang" <wangguoz@gmail.com
>     <ma...@gmail.com>> wrote:
>
>         Hi Arjun,
>
>         If you only send one message, does that message show up on the
>         server? Does
>         you consumer use wildcard topics or specific topics?
>
>         Guozhang
>
>
>         On Thu, Apr 10, 2014 at 9:20 AM, Arjun <arjun@socialtwist.com
>         <ma...@socialtwist.com>> wrote:
>
>         > But  we have auto offset reset to smallest not largest, even
>         then this
>         > issue arises? If so is there any work around?
>         >
>         > Thanks
>         > Arjun NArasimha Kota
>         >
>         >
>         > On Thursday 10 April 2014 09:39 PM, Guozhang Wang wrote:
>         >
>         >> It could be https://issues.apache.org/jira/browse/KAFKA-1006.
>         >>
>         >> Guozhang
>         >>
>         >>
>         >> On Thu, Apr 10, 2014 at 8:50 AM, Arjun
>         <arjun@socialtwist.com <ma...@socialtwist.com>> wrote:
>         >>
>         >>  its auto created
>         >>> but even after topic creation this is the scenario
>         >>>
>         >>> Arjun
>         >>>
>         >>> On Thursday 10 April 2014 08:41 PM, Guozhang Wang wrote:
>         >>>
>         >>>  Hi Arjun,
>         >>>>
>         >>>> Did you manually create the topic or use auto.topic.creation?
>         >>>>
>         >>>> Guozhang
>         >>>>
>         >>>>
>         >>>> On Thu, Apr 10, 2014 at 7:39 AM, Arjun
>         <arjun@socialtwist.com <ma...@socialtwist.com>> wrote:
>         >>>>
>         >>>>   Hi,
>         >>>>
>         >>>>> We have 3 node kafka 0.8 setup with zookeepers ensemble.
>         We use high
>         >>>>> level
>         >>>>> consumer with auto commit offset false. I am facing some
>         peculiar
>         >>>>> problem
>         >>>>> with kafka. When i send some 10-20 messages or so the
>         consumer starts
>         >>>>> to
>         >>>>> consume the messages. But if  i send only one message to
>         kafka, then
>         >>>>> even
>         >>>>> though consumer is active it is not trying to fetch the
>         message. There
>         >>>>> is
>         >>>>> nothing in logs, just the messages are being fetched by
>         the kafka
>         >>>>> consumer.
>         >>>>> The messages are there in the Kafka server. Can some one
>         let me know
>         >>>>> where
>         >>>>> i am doing wrong.
>         >>>>>
>         >>>>>
>         >>>>> Thanks
>         >>>>> Arjun Narasimha Kota
>         >>>>>
>         >>>>>
>         >>>>>
>         >>>>
>         >>
>         >
>
>
>         --
>         -- Guozhang
>


Re: consumer not consuming messages

Posted by Arjun <ar...@socialtwist.com>.
i see this in the consumer logs
[kafka.consumer.ConsumerFetcherManager] 
[ConsumerFetcherManager-1397188062631] Adding fetcher for partition 
[taf.referral.emails.service,11], initOffset 250 to broker 1 with 
fetcherId 0

but no data and i get this warning

[ConsumerFetcherThread-group1_ip-10-91-35-43-1397188061429-b5ff1205-0-1] 
[kafka.consumer.ConsumerFetcherThread] 
[ConsumerFetcherThread-group1_ip-10-91-35-43-1397188061429-b5ff1205-0-1], Error 
in fetch Name: FetchRequest; Version: 0; CorrelationId: 73; ClientId: 
group1-ConsumerFetcherThread-group1_ip-10-91-35-43-1397188061429-b5ff1205-0-1; 
ReplicaId: -1; MaxWait: 180000 ms; MinBytes: 1 bytes; RequestInfo: 
[taf.referral.emails.service,0] -> PartitionFetchInfo(253,10485760)
java.net.SocketTimeoutException
     at 
sun.nio.ch.SocketAdaptor$SocketInputStream.read(SocketAdaptor.java:201)
     at sun.nio.ch.ChannelInputStream.read(ChannelInputStream.java:86)
     at 
java.nio.channels.Channels$ReadableByteChannelImpl.read(Channels.java:221)
     at kafka.utils.Utils$.read(Utils.scala:395)
     at 
kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:54)
     at kafka.network.Receive$class.readCompletely(Transmission.scala:56)
     at 
kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)
     at kafka.network.BlockingChannel.receive(BlockingChannel.scala:100)
     at kafka.consumer.SimpleConsumer.liftedTree1$1(SimpleConsumer.scala:81)
     at 
kafka.consumer.SimpleConsumer.kafka$consumer$SimpleConsumer$$sendRequest(SimpleConsumer.scala:71)
     at 
kafka.consumer.SimpleConsumer$$anonfun$fetch$1$$anonfun$apply$mcV$sp$1.apply$mcV$sp(SimpleConsumer.scala:110)
     at 
kafka.consumer.SimpleConsumer$$anonfun$fetch$1$$anonfun$apply$mcV$sp$1.apply(SimpleConsumer.scala:110)
     at 
kafka.consumer.SimpleConsumer$$anonfun$fetch$1$$anonfun$apply$mcV$sp$1.apply(SimpleConsumer.scala:110)
     at kafka.metrics.KafkaTimer.time(KafkaTimer.scala:33)
     at 
kafka.consumer.SimpleConsumer$$anonfun$fetch$1.apply$mcV$sp(SimpleConsumer.scala:109)
     at 
kafka.consumer.SimpleConsumer$$anonfun$fetch$1.apply(SimpleConsumer.scala:109)
     at 
kafka.consumer.SimpleConsumer$$anonfun$fetch$1.apply(SimpleConsumer.scala:109)
     at kafka.metrics.KafkaTimer.time(KafkaTimer.scala:33)
     at kafka.consumer.SimpleConsumer.fetch(SimpleConsumer.scala:108)
     at 
kafka.server.AbstractFetcherThread.processFetchRequest(AbstractFetcherThread.scala:94)
     at 
kafka.server.AbstractFetcherThread.doWork(AbstractFetcherThread.scala:86)
     at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:51)


Thanks
Arjun Narasimha Kota



On Friday 11 April 2014 09:51 AM, Arjun wrote:
> I hope this one would give u  a better idea.
>
> bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker --group 
> group1 --zkconnect zkhost:port --topic testtopic
> Group           Topic                          Pid Offset          
> logSize         Lag             Owner
> group1          testtopic    0   253             253 0               
> group1_ip-xx-1397188061429-b5ff1205-0
> group1          testtopic    1   267             267 0               
> group1_ip-xx-1397188061429-b5ff1205-0
> group1          testtopic    2   254             254 0               
> group1_ip-xx-1397188061429-b5ff1205-0
> group1          testtopic    3   265             265 0               
> group1_ip-xx-1397188061429-b5ff1205-0
> group1          testtopic    4   261             261 0               
> group1_ip-xx-1397188061429-b5ff1205-1
> group1          testtopic    5   294             294 0               
> group1_ip-xx-1397188061429-b5ff1205-1
> group1          testtopic    6   248             248 0               
> group1_ip-xx-1397188061429-b5ff1205-1
> group1          testtopic    7   271             271 0               
> group1_ip-xx-1397188061429-b5ff1205-1
> group1          testtopic    8   240             240 0               
> group1_ip-xx-1397188061429-b5ff1205-2
> group1          testtopic    9   261             261 0               
> group1_ip-xx-1397188061429-b5ff1205-2
> group1          testtopic    10  290             290 0               
> group1_ip-xx-1397188061429-b5ff1205-2
> group1          testtopic    11  250             251 1               
> group1_ip-xx-1397188061429-b5ff1205-2
>
> If you see the output, in the last line the lag is 1 for that 
> partition. I just send one message. This topic is not new as you see 
> there are lot of messages which have accumlated from yesterday. This 
> one message will not be consumed by consumer what so ever. But if i 
> send some 10 messages then all the messages are consumed.
>
> Please let me know if i have to change any consumer properties.
>
> My consumer properties are :
> "fetch.wait.max.ms"="180000"
> "fetch.min.bytes" = "1"
> "auto.offset.reset" = "smallest"
> "auto.commit.enable"=  "false"
> "fetch.message.max.bytes" = "1048576"
>
> Thanks
> Arjun Narasimha Kota
> On Friday 11 April 2014 06:23 AM, Arjun Kota wrote:
>>
>> The consumer uses do specific topics.
>>
>> On Apr 11, 2014 6:23 AM, "Arjun Kota" <arjun@socialtwist.com 
>> <ma...@socialtwist.com>> wrote:
>>
>>     Yes the message shows up on the server.
>>
>>     On Apr 11, 2014 12:07 AM, "Guozhang Wang" <wangguoz@gmail.com
>>     <ma...@gmail.com>> wrote:
>>
>>         Hi Arjun,
>>
>>         If you only send one message, does that message show up on
>>         the server? Does
>>         you consumer use wildcard topics or specific topics?
>>
>>         Guozhang
>>
>>
>>         On Thu, Apr 10, 2014 at 9:20 AM, Arjun <arjun@socialtwist.com
>>         <ma...@socialtwist.com>> wrote:
>>
>>         > But  we have auto offset reset to smallest not largest,
>>         even then this
>>         > issue arises? If so is there any work around?
>>         >
>>         > Thanks
>>         > Arjun NArasimha Kota
>>         >
>>         >
>>         > On Thursday 10 April 2014 09:39 PM, Guozhang Wang wrote:
>>         >
>>         >> It could be https://issues.apache.org/jira/browse/KAFKA-1006.
>>         >>
>>         >> Guozhang
>>         >>
>>         >>
>>         >> On Thu, Apr 10, 2014 at 8:50 AM, Arjun
>>         <arjun@socialtwist.com <ma...@socialtwist.com>> wrote:
>>         >>
>>         >>  its auto created
>>         >>> but even after topic creation this is the scenario
>>         >>>
>>         >>> Arjun
>>         >>>
>>         >>> On Thursday 10 April 2014 08:41 PM, Guozhang Wang wrote:
>>         >>>
>>         >>>  Hi Arjun,
>>         >>>>
>>         >>>> Did you manually create the topic or use
>>         auto.topic.creation?
>>         >>>>
>>         >>>> Guozhang
>>         >>>>
>>         >>>>
>>         >>>> On Thu, Apr 10, 2014 at 7:39 AM, Arjun
>>         <arjun@socialtwist.com <ma...@socialtwist.com>> wrote:
>>         >>>>
>>         >>>>   Hi,
>>         >>>>
>>         >>>>> We have 3 node kafka 0.8 setup with zookeepers
>>         ensemble. We use high
>>         >>>>> level
>>         >>>>> consumer with auto commit offset false. I am facing
>>         some peculiar
>>         >>>>> problem
>>         >>>>> with kafka. When i send some 10-20 messages or so the
>>         consumer starts
>>         >>>>> to
>>         >>>>> consume the messages. But if  i send only one message
>>         to kafka, then
>>         >>>>> even
>>         >>>>> though consumer is active it is not trying to fetch the
>>         message. There
>>         >>>>> is
>>         >>>>> nothing in logs, just the messages are being fetched by
>>         the kafka
>>         >>>>> consumer.
>>         >>>>> The messages are there in the Kafka server. Can some
>>         one let me know
>>         >>>>> where
>>         >>>>> i am doing wrong.
>>         >>>>>
>>         >>>>>
>>         >>>>> Thanks
>>         >>>>> Arjun Narasimha Kota
>>         >>>>>
>>         >>>>>
>>         >>>>>
>>         >>>>
>>         >>
>>         >
>>
>>
>>         --
>>         -- Guozhang
>>
>


Re: consumer not consuming messages

Posted by Arjun Kota <ar...@socialtwist.com>.
Yup will try that
On Apr 12, 2014 8:42 AM, "Jun Rao" <ju...@gmail.com> wrote:

> Console consumer also uses the high level consumer. Could you try setting
> fetch.wait.max.ms to 100ms?
>
> Thanks,
>
> Jun
>
>
> On Fri, Apr 11, 2014 at 9:56 AM, Arjun Kota <ar...@socialtwist.com> wrote:
>
> > Console consumer works fine. Its the high level java consumer which is
> > giving this problem.
> >
> > Thanks
> > Arjun narasimha kota
> > On Apr 11, 2014 8:42 PM, "Jun Rao" <ju...@gmail.com> wrote:
> >
> > > We may have a bug that doesn't observe etch.min.bytes accurately. So a
> > > lower fetch.wait.max.ms will improve consumer latency.
> > >
> > > Could you run a console consumer and see if you have the same issue?
> That
> > > will tell us if this is a server side issue or an issue just in your
> > > consumer.
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > >
> > > On Thu, Apr 10, 2014 at 10:28 PM, Arjun <ar...@socialtwist.com> wrote:
> > >
> > > > i changed the time to 60 seconds even now i see the same result. The
> > > > Consumer is not consuming the messages.
> > > >
> > > > Thanks
> > > > Arjun Narasimha Kota
> > > >
> > > >
> > > > On Friday 11 April 2014 10:36 AM, Arjun wrote:
> > > >
> > > >> yup i will change the value and recheck. Thanks for the help.
> > > >>
> > > >> thanks
> > > >> Arjun Narasimha Kota
> > > >>
> > > >> On Friday 11 April 2014 10:28 AM, Guozhang Wang wrote:
> > > >>
> > > >>> What I tried to say is that it may be caused by your
> > > >>> "fetch.wait.max.ms"="180000"
> > > >>> too large. Try a small value and see if that helps.
> > > >>>
> > > >>>
> > > >>> On Thu, Apr 10, 2014 at 9:44 PM, Arjun <ar...@socialtwist.com>
> > wrote:
> > > >>>
> > > >>>  Hi,
> > > >>>>
> > > >>>> I could not see any out of memory exceptions in the broker logs.
> One
> > > >>>> thing
> > > >>>> i can see is i  may have configured consumer poorly. If its not
> too
> > > >>>> much to
> > > >>>> ask can u let me know the changes i have to do for over coming
> this
> > > >>>> problem.
> > > >>>>
> > > >>>> Thanks
> > > >>>> Arjun Narasimha Kota
> > > >>>>
> > > >>>>
> > > >>>> On Friday 11 April 2014 10:04 AM, Guozhang Wang wrote:
> > > >>>>
> > > >>>>  Hi Ajrun,
> > > >>>>>
> > > >>>>> It seems to be the cause:
> > > >>>>>
> > > >>>>> https://issues.apache.org/jira/browse/KAFKA-1016
> > > >>>>>
> > > >>>>> Guozhang
> > > >>>>>
> > > >>>>>
> > > >>>>>
> > > >>>>> On Thu, Apr 10, 2014 at 9:21 PM, Arjun <ar...@socialtwist.com>
> > > wrote:
> > > >>>>>
> > > >>>>>   I hope this one would give u  a better idea.
> > > >>>>>
> > > >>>>>> bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker --group
> > > >>>>>> group1
> > > >>>>>> --zkconnect zkhost:port --topic testtopic
> > > >>>>>> Group           Topic                          Pid Offset
> logSize
> > > >>>>>> Lag             Owner
> > > >>>>>> group1          testtopic    0   253             253 0
> > > >>>>>> group1_ip-xx-1397188061429-b5ff1205-0
> > > >>>>>> group1          testtopic    1   267             267 0
> > > >>>>>> group1_ip-xx-1397188061429-b5ff1205-0
> > > >>>>>> group1          testtopic    2   254             254 0
> > > >>>>>> group1_ip-xx-1397188061429-b5ff1205-0
> > > >>>>>> group1          testtopic    3   265             265 0
> > > >>>>>> group1_ip-xx-1397188061429-b5ff1205-0
> > > >>>>>> group1          testtopic    4   261             261 0
> > > >>>>>> group1_ip-xx-1397188061429-b5ff1205-1
> > > >>>>>> group1          testtopic    5   294             294 0
> > > >>>>>> group1_ip-xx-1397188061429-b5ff1205-1
> > > >>>>>> group1          testtopic    6   248             248 0
> > > >>>>>> group1_ip-xx-1397188061429-b5ff1205-1
> > > >>>>>> group1          testtopic    7   271             271 0
> > > >>>>>> group1_ip-xx-1397188061429-b5ff1205-1
> > > >>>>>> group1          testtopic    8   240             240 0
> > > >>>>>> group1_ip-xx-1397188061429-b5ff1205-2
> > > >>>>>> group1          testtopic    9   261             261 0
> > > >>>>>> group1_ip-xx-1397188061429-b5ff1205-2
> > > >>>>>> group1          testtopic    10  290             290 0
> > > >>>>>> group1_ip-xx-1397188061429-b5ff1205-2
> > > >>>>>> group1          testtopic    11  250             251 1
> > > >>>>>> group1_ip-xx-1397188061429-b5ff1205-2
> > > >>>>>>
> > > >>>>>> If you see the output, in the last line the lag is 1 for that
> > > >>>>>> partition.
> > > >>>>>> I
> > > >>>>>> just send one message. This topic is not new as you see there
> are
> > > lot
> > > >>>>>> of
> > > >>>>>> messages which have accumlated from yesterday. This one message
> > will
> > > >>>>>> not
> > > >>>>>> be
> > > >>>>>> consumed by consumer what so ever. But if i send some 10
> messages
> > > then
> > > >>>>>> all
> > > >>>>>> the messages are consumed.
> > > >>>>>>
> > > >>>>>> Please let me know if i have to change any consumer properties.
> > > >>>>>>
> > > >>>>>> My consumer properties are :
> > > >>>>>> "fetch.wait.max.ms"="180000"
> > > >>>>>> "fetch.min.bytes" = "1"
> > > >>>>>> "auto.offset.reset" = "smallest"
> > > >>>>>> "auto.commit.enable"=  "false"
> > > >>>>>> "fetch.message.max.bytes" = "1048576"
> > > >>>>>>
> > > >>>>>>
> > > >>>>>> Thanks
> > > >>>>>> Arjun Narasimha Kota
> > > >>>>>> On Friday 11 April 2014 06:23 AM, Arjun Kota wrote:
> > > >>>>>>
> > > >>>>>>   The consumer uses do specific topics.
> > > >>>>>>
> > > >>>>>>> On Apr 11, 2014 6:23 AM, "Arjun Kota" <arjun@socialtwist.com
> > > <mailto:
> > > >>>>>>> arjun@socialtwist.com>> wrote:
> > > >>>>>>>
> > > >>>>>>>       Yes the message shows up on the server.
> > > >>>>>>>
> > > >>>>>>>       On Apr 11, 2014 12:07 AM, "Guozhang Wang" <
> > > wangguoz@gmail.com
> > > >>>>>>>       <ma...@gmail.com>> wrote:
> > > >>>>>>>
> > > >>>>>>>           Hi Arjun,
> > > >>>>>>>
> > > >>>>>>>           If you only send one message, does that message show
> up
> > > on
> > > >>>>>>> the
> > > >>>>>>>           server? Does
> > > >>>>>>>           you consumer use wildcard topics or specific topics?
> > > >>>>>>>
> > > >>>>>>>           Guozhang
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>>           On Thu, Apr 10, 2014 at 9:20 AM, Arjun <
> > > >>>>>>> arjun@socialtwist.com
> > > >>>>>>>           <ma...@socialtwist.com>> wrote:
> > > >>>>>>>
> > > >>>>>>>           > But  we have auto offset reset to smallest not
> > largest,
> > > >>>>>>> even
> > > >>>>>>>           then this
> > > >>>>>>>           > issue arises? If so is there any work around?
> > > >>>>>>>           >
> > > >>>>>>>           > Thanks
> > > >>>>>>>           > Arjun NArasimha Kota
> > > >>>>>>>           >
> > > >>>>>>>           >
> > > >>>>>>>           > On Thursday 10 April 2014 09:39 PM, Guozhang Wang
> > > wrote:
> > > >>>>>>>           >
> > > >>>>>>>           >> It could be https://issues.apache.org/
> > > >>>>>>> jira/browse/KAFKA-1006.
> > > >>>>>>>           >>
> > > >>>>>>>           >> Guozhang
> > > >>>>>>>           >>
> > > >>>>>>>           >>
> > > >>>>>>>           >> On Thu, Apr 10, 2014 at 8:50 AM, Arjun
> > > >>>>>>>           <arjun@socialtwist.com <mailto:arjun@socialtwist.com
> >>
> > > >>>>>>> wrote:
> > > >>>>>>>           >>
> > > >>>>>>>           >>  its auto created
> > > >>>>>>>           >>> but even after topic creation this is the
> scenario
> > > >>>>>>>           >>>
> > > >>>>>>>           >>> Arjun
> > > >>>>>>>           >>>
> > > >>>>>>>           >>> On Thursday 10 April 2014 08:41 PM, Guozhang Wang
> > > >>>>>>> wrote:
> > > >>>>>>>           >>>
> > > >>>>>>>           >>>  Hi Arjun,
> > > >>>>>>>           >>>>
> > > >>>>>>>           >>>> Did you manually create the topic or use
> > > >>>>>>> auto.topic.creation?
> > > >>>>>>>           >>>>
> > > >>>>>>>           >>>> Guozhang
> > > >>>>>>>           >>>>
> > > >>>>>>>           >>>>
> > > >>>>>>>           >>>> On Thu, Apr 10, 2014 at 7:39 AM, Arjun
> > > >>>>>>>           <arjun@socialtwist.com <mailto:arjun@socialtwist.com
> >>
> > > >>>>>>> wrote:
> > > >>>>>>>           >>>>
> > > >>>>>>>           >>>>   Hi,
> > > >>>>>>>           >>>>
> > > >>>>>>>           >>>>> We have 3 node kafka 0.8 setup with zookeepers
> > > >>>>>>> ensemble.
> > > >>>>>>>           We use high
> > > >>>>>>>           >>>>> level
> > > >>>>>>>           >>>>> consumer with auto commit offset false. I am
> > facing
> > > >>>>>>> some
> > > >>>>>>>           peculiar
> > > >>>>>>>           >>>>> problem
> > > >>>>>>>           >>>>> with kafka. When i send some 10-20 messages or
> so
> > > the
> > > >>>>>>>           consumer starts
> > > >>>>>>>           >>>>> to
> > > >>>>>>>           >>>>> consume the messages. But if  i send only one
> > > >>>>>>> message to
> > > >>>>>>>           kafka, then
> > > >>>>>>>           >>>>> even
> > > >>>>>>>           >>>>> though consumer is active it is not trying to
> > fetch
> > > >>>>>>> the
> > > >>>>>>>           message. There
> > > >>>>>>>           >>>>> is
> > > >>>>>>>           >>>>> nothing in logs, just the messages are being
> > > fetched
> > > >>>>>>> by
> > > >>>>>>>           the kafka
> > > >>>>>>>           >>>>> consumer.
> > > >>>>>>>           >>>>> The messages are there in the Kafka server. Can
> > > some
> > > >>>>>>> one
> > > >>>>>>>           let me know
> > > >>>>>>>           >>>>> where
> > > >>>>>>>           >>>>> i am doing wrong.
> > > >>>>>>>           >>>>>
> > > >>>>>>>           >>>>>
> > > >>>>>>>           >>>>> Thanks
> > > >>>>>>>           >>>>> Arjun Narasimha Kota
> > > >>>>>>>           >>>>>
> > > >>>>>>>           >>>>>
> > > >>>>>>>           >>>>>
> > > >>>>>>>           >>>>
> > > >>>>>>>           >>
> > > >>>>>>>           >
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>>           --
> > > >>>>>>>           -- Guozhang
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>>
> > > >>>
> > > >>
> > > >
> > >
> >
>

Re: consumer not consuming messages

Posted by Jun Rao <ju...@gmail.com>.
"Rebalancing attempt failed" indicates the rebalancing failed. I added some
notes in the last item in
https://cwiki.apache.org/confluence/display/KAFKA/FAQ#FAQ-Myconsumerseemstohavestopped,why
?

Thanks,

Jun


On Fri, Apr 11, 2014 at 11:23 PM, Arjun <ar...@socialtwist.com> wrote:

> Even after changing the  fetch wait max ms the same thing is repeting just
> that some partitions have the owners now, i mean
>
>
> bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker --group group1
> --zkconnect zkhost:zkport --topic testtopic
>
> Group           Topic                          Pid Offset logSize
> Lag             Owner
> group1          testtopic    0   253             253 0
> group1_xxxx-1397216047177-6f419d28-0
> group1          testtopic    1   268             268 0
> group1_xxxx-1397216047177-6f419d28-1
> group1          testtopic    2   258             258 0
> group1_xxxx-1397216047177-6f419d28-2
> group1          testtopic    3   265             265 0               none
> group1          testtopic    4   262             262 0               none
> group1          testtopic    5   296             296 0               none
> group1          testtopic    6   249             249 0
> group1_xxxx1-1397219018735-a2cf60df-0
> group1          testtopic    7   272             272 0
> group1_xxxx1-1397219018735-a2cf60df-1
> group1          testtopic    8   242             242 0
> group1_xxxx1-1397219018735-a2cf60df-2
> group1          testtopic    9   263             263 0
> group1_xxxx1-1397279906839-12366659-0
> group1          testtopic    10  294             294 0
> group1_xxxx1-1397279906839-12366659-1
> group1          testtopic    11  254             254 0
> group1_xxxx1-1397279906839-12366659-2
>
> and i dont see any exception after
>
> "end rebalancing consumer group1_ip-XXXX-82095a28 try #9"
>
> i just see the below log
>
> 12 Apr 2014 00:18:58,987 INFO  [Thread-321] [kafka.consumer.ZookeeperConsumerConnector]
> [group1_ip-xxxx-82095a28], Rebalancing attempt failed. Clearing the cache
> before the next rebalancing operation is triggered
> 12 Apr 2014 00:18:58,987 INFO  [Thread-321] [kafka.consumer.ConsumerFetcherManager]
> [ConsumerFetcherManager-1397279912725] Stopping leader finder thread
> 12 Apr 2014 00:18:58,987 INFO  [Thread-321] [kafka.consumer.ConsumerFetcherManager]
> [ConsumerFetcherManager-1397279912725] Stopping all fetchers
> 12 Apr 2014 00:18:58,987 INFO  [Thread-321] [kafka.consumer.ConsumerFetcherManager]
> [ConsumerFetcherManager-1397279912725] All connections stopped
> 12 Apr 2014 00:18:58,987 INFO  [Thread-321] [kafka.consumer.ZookeeperConsumerConnector]
> [group1_xxxx-82095a28], Cleared all relevant queues for this fetcher
> 12 Apr 2014 00:18:58,988 DEBUG [Thread-321] [kafka.consumer.ConsumerIterator]
> Clearing the current data chunk for this consumer iterator
> 12 Apr 2014 00:18:58,988 DEBUG [Thread-321] [kafka.consumer.ConsumerIterator]
> Clearing the current data chunk for this consumer iterator
> 12 Apr 2014 00:18:58,988 DEBUG [Thread-321] [kafka.consumer.ConsumerIterator]
> Clearing the current data chunk for this consumer iterator
> 12 Apr 2014 00:18:58,988 INFO  [Thread-321] [kafka.consumer.ZookeeperConsumerConnector]
> [group1_xxxx-82095a28], Cleared the data chunks in all the consumer message
> iterators
> 12 Apr 2014 00:18:58,988 INFO  [Thread-321] [kafka.consumer.ZookeeperConsumerConnector]
> [group1_xxxx-82095a28], Committing all offsets after clearing the fetcher
> queues
>
> after this kafka consumer will not log a thing.Not even the exception.(I
> have put my consumer log level to debug)
>
>
>
> Thanks
> Arjun Narasimha Kota
>
>
>
> On Saturday 12 April 2014 08:41 AM, Jun Rao wrote:
>
>> Console consumer also uses the high level consumer. Could you try setting
>> fetch.wait.max.ms to 100ms?
>>
>> Thanks,
>>
>> Jun
>>
>>
>> On Fri, Apr 11, 2014 at 9:56 AM, Arjun Kota <ar...@socialtwist.com>
>> wrote:
>>
>>  Console consumer works fine. Its the high level java consumer which is
>>> giving this problem.
>>>
>>> Thanks
>>> Arjun narasimha kota
>>> On Apr 11, 2014 8:42 PM, "Jun Rao" <ju...@gmail.com> wrote:
>>>
>>>  We may have a bug that doesn't observe etch.min.bytes accurately. So a
>>>> lower fetch.wait.max.ms will improve consumer latency.
>>>>
>>>> Could you run a console consumer and see if you have the same issue?
>>>> That
>>>> will tell us if this is a server side issue or an issue just in your
>>>> consumer.
>>>>
>>>> Thanks,
>>>>
>>>> Jun
>>>>
>>>>
>>>> On Thu, Apr 10, 2014 at 10:28 PM, Arjun <ar...@socialtwist.com> wrote:
>>>>
>>>>  i changed the time to 60 seconds even now i see the same result. The
>>>>> Consumer is not consuming the messages.
>>>>>
>>>>> Thanks
>>>>> Arjun Narasimha Kota
>>>>>
>>>>>
>>>>> On Friday 11 April 2014 10:36 AM, Arjun wrote:
>>>>>
>>>>>  yup i will change the value and recheck. Thanks for the help.
>>>>>>
>>>>>> thanks
>>>>>> Arjun Narasimha Kota
>>>>>>
>>>>>> On Friday 11 April 2014 10:28 AM, Guozhang Wang wrote:
>>>>>>
>>>>>>  What I tried to say is that it may be caused by your
>>>>>>> "fetch.wait.max.ms"="180000"
>>>>>>> too large. Try a small value and see if that helps.
>>>>>>>
>>>>>>>
>>>>>>> On Thu, Apr 10, 2014 at 9:44 PM, Arjun <ar...@socialtwist.com>
>>>>>>>
>>>>>> wrote:
>>>
>>>>   Hi,
>>>>>>>
>>>>>>>> I could not see any out of memory exceptions in the broker logs. One
>>>>>>>> thing
>>>>>>>> i can see is i  may have configured consumer poorly. If its not too
>>>>>>>> much to
>>>>>>>> ask can u let me know the changes i have to do for over coming this
>>>>>>>> problem.
>>>>>>>>
>>>>>>>> Thanks
>>>>>>>> Arjun Narasimha Kota
>>>>>>>>
>>>>>>>>
>>>>>>>> On Friday 11 April 2014 10:04 AM, Guozhang Wang wrote:
>>>>>>>>
>>>>>>>>   Hi Ajrun,
>>>>>>>>
>>>>>>>>> It seems to be the cause:
>>>>>>>>>
>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-1016
>>>>>>>>>
>>>>>>>>> Guozhang
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Thu, Apr 10, 2014 at 9:21 PM, Arjun <ar...@socialtwist.com>
>>>>>>>>>
>>>>>>>> wrote:
>>>>
>>>>>    I hope this one would give u  a better idea.
>>>>>>>>>
>>>>>>>>>  bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker --group
>>>>>>>>>> group1
>>>>>>>>>> --zkconnect zkhost:port --topic testtopic
>>>>>>>>>> Group           Topic                          Pid Offset logSize
>>>>>>>>>> Lag             Owner
>>>>>>>>>> group1          testtopic    0   253             253 0
>>>>>>>>>> group1_ip-xx-1397188061429-b5ff1205-0
>>>>>>>>>> group1          testtopic    1   267             267 0
>>>>>>>>>> group1_ip-xx-1397188061429-b5ff1205-0
>>>>>>>>>> group1          testtopic    2   254             254 0
>>>>>>>>>> group1_ip-xx-1397188061429-b5ff1205-0
>>>>>>>>>> group1          testtopic    3   265             265 0
>>>>>>>>>> group1_ip-xx-1397188061429-b5ff1205-0
>>>>>>>>>> group1          testtopic    4   261             261 0
>>>>>>>>>> group1_ip-xx-1397188061429-b5ff1205-1
>>>>>>>>>> group1          testtopic    5   294             294 0
>>>>>>>>>> group1_ip-xx-1397188061429-b5ff1205-1
>>>>>>>>>> group1          testtopic    6   248             248 0
>>>>>>>>>> group1_ip-xx-1397188061429-b5ff1205-1
>>>>>>>>>> group1          testtopic    7   271             271 0
>>>>>>>>>> group1_ip-xx-1397188061429-b5ff1205-1
>>>>>>>>>> group1          testtopic    8   240             240 0
>>>>>>>>>> group1_ip-xx-1397188061429-b5ff1205-2
>>>>>>>>>> group1          testtopic    9   261             261 0
>>>>>>>>>> group1_ip-xx-1397188061429-b5ff1205-2
>>>>>>>>>> group1          testtopic    10  290             290 0
>>>>>>>>>> group1_ip-xx-1397188061429-b5ff1205-2
>>>>>>>>>> group1          testtopic    11  250             251 1
>>>>>>>>>> group1_ip-xx-1397188061429-b5ff1205-2
>>>>>>>>>>
>>>>>>>>>> If you see the output, in the last line the lag is 1 for that
>>>>>>>>>> partition.
>>>>>>>>>> I
>>>>>>>>>> just send one message. This topic is not new as you see there are
>>>>>>>>>>
>>>>>>>>> lot
>>>>
>>>>> of
>>>>>>>>>> messages which have accumlated from yesterday. This one message
>>>>>>>>>>
>>>>>>>>> will
>>>
>>>> not
>>>>>>>>>> be
>>>>>>>>>> consumed by consumer what so ever. But if i send some 10 messages
>>>>>>>>>>
>>>>>>>>> then
>>>>
>>>>> all
>>>>>>>>>> the messages are consumed.
>>>>>>>>>>
>>>>>>>>>> Please let me know if i have to change any consumer properties.
>>>>>>>>>>
>>>>>>>>>> My consumer properties are :
>>>>>>>>>> "fetch.wait.max.ms"="180000"
>>>>>>>>>> "fetch.min.bytes" = "1"
>>>>>>>>>> "auto.offset.reset" = "smallest"
>>>>>>>>>> "auto.commit.enable"=  "false"
>>>>>>>>>> "fetch.message.max.bytes" = "1048576"
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Thanks
>>>>>>>>>> Arjun Narasimha Kota
>>>>>>>>>> On Friday 11 April 2014 06:23 AM, Arjun Kota wrote:
>>>>>>>>>>
>>>>>>>>>>    The consumer uses do specific topics.
>>>>>>>>>>
>>>>>>>>>>  On Apr 11, 2014 6:23 AM, "Arjun Kota" <arjun@socialtwist.com
>>>>>>>>>>>
>>>>>>>>>> <mailto:
>>>>
>>>>> arjun@socialtwist.com>> wrote:
>>>>>>>>>>>
>>>>>>>>>>>        Yes the message shows up on the server.
>>>>>>>>>>>
>>>>>>>>>>>        On Apr 11, 2014 12:07 AM, "Guozhang Wang" <
>>>>>>>>>>>
>>>>>>>>>> wangguoz@gmail.com
>>>>
>>>>>        <ma...@gmail.com>> wrote:
>>>>>>>>>>>
>>>>>>>>>>>            Hi Arjun,
>>>>>>>>>>>
>>>>>>>>>>>            If you only send one message, does that message show
>>>>>>>>>>> up
>>>>>>>>>>>
>>>>>>>>>> on
>>>>
>>>>> the
>>>>>>>>>>>            server? Does
>>>>>>>>>>>            you consumer use wildcard topics or specific topics?
>>>>>>>>>>>
>>>>>>>>>>>            Guozhang
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>            On Thu, Apr 10, 2014 at 9:20 AM, Arjun <
>>>>>>>>>>> arjun@socialtwist.com
>>>>>>>>>>>            <ma...@socialtwist.com>> wrote:
>>>>>>>>>>>
>>>>>>>>>>>            > But  we have auto offset reset to smallest not
>>>>>>>>>>>
>>>>>>>>>> largest,
>>>
>>>> even
>>>>>>>>>>>            then this
>>>>>>>>>>>            > issue arises? If so is there any work around?
>>>>>>>>>>>            >
>>>>>>>>>>>            > Thanks
>>>>>>>>>>>            > Arjun NArasimha Kota
>>>>>>>>>>>            >
>>>>>>>>>>>            >
>>>>>>>>>>>            > On Thursday 10 April 2014 09:39 PM, Guozhang Wang
>>>>>>>>>>>
>>>>>>>>>> wrote:
>>>>
>>>>>            >
>>>>>>>>>>>            >> It could be https://issues.apache.org/
>>>>>>>>>>> jira/browse/KAFKA-1006.
>>>>>>>>>>>            >>
>>>>>>>>>>>            >> Guozhang
>>>>>>>>>>>            >>
>>>>>>>>>>>            >>
>>>>>>>>>>>            >> On Thu, Apr 10, 2014 at 8:50 AM, Arjun
>>>>>>>>>>>            <arjun@socialtwist.com <ma...@socialtwist.com>
>>>>>>>>>>> >
>>>>>>>>>>> wrote:
>>>>>>>>>>>            >>
>>>>>>>>>>>            >>  its auto created
>>>>>>>>>>>            >>> but even after topic creation this is the scenario
>>>>>>>>>>>            >>>
>>>>>>>>>>>            >>> Arjun
>>>>>>>>>>>            >>>
>>>>>>>>>>>            >>> On Thursday 10 April 2014 08:41 PM, Guozhang Wang
>>>>>>>>>>> wrote:
>>>>>>>>>>>            >>>
>>>>>>>>>>>            >>>  Hi Arjun,
>>>>>>>>>>>            >>>>
>>>>>>>>>>>            >>>> Did you manually create the topic or use
>>>>>>>>>>> auto.topic.creation?
>>>>>>>>>>>            >>>>
>>>>>>>>>>>            >>>> Guozhang
>>>>>>>>>>>            >>>>
>>>>>>>>>>>            >>>>
>>>>>>>>>>>            >>>> On Thu, Apr 10, 2014 at 7:39 AM, Arjun
>>>>>>>>>>>            <arjun@socialtwist.com <ma...@socialtwist.com>
>>>>>>>>>>> >
>>>>>>>>>>> wrote:
>>>>>>>>>>>            >>>>
>>>>>>>>>>>            >>>>   Hi,
>>>>>>>>>>>            >>>>
>>>>>>>>>>>            >>>>> We have 3 node kafka 0.8 setup with zookeepers
>>>>>>>>>>> ensemble.
>>>>>>>>>>>            We use high
>>>>>>>>>>>            >>>>> level
>>>>>>>>>>>            >>>>> consumer with auto commit offset false. I am
>>>>>>>>>>>
>>>>>>>>>> facing
>>>
>>>> some
>>>>>>>>>>>            peculiar
>>>>>>>>>>>            >>>>> problem
>>>>>>>>>>>            >>>>> with kafka. When i send some 10-20 messages or
>>>>>>>>>>> so
>>>>>>>>>>>
>>>>>>>>>> the
>>>>
>>>>>            consumer starts
>>>>>>>>>>>            >>>>> to
>>>>>>>>>>>            >>>>> consume the messages. But if  i send only one
>>>>>>>>>>> message to
>>>>>>>>>>>            kafka, then
>>>>>>>>>>>            >>>>> even
>>>>>>>>>>>            >>>>> though consumer is active it is not trying to
>>>>>>>>>>>
>>>>>>>>>> fetch
>>>
>>>> the
>>>>>>>>>>>            message. There
>>>>>>>>>>>            >>>>> is
>>>>>>>>>>>            >>>>> nothing in logs, just the messages are being
>>>>>>>>>>>
>>>>>>>>>> fetched
>>>>
>>>>> by
>>>>>>>>>>>            the kafka
>>>>>>>>>>>            >>>>> consumer.
>>>>>>>>>>>            >>>>> The messages are there in the Kafka server. Can
>>>>>>>>>>>
>>>>>>>>>> some
>>>>
>>>>> one
>>>>>>>>>>>            let me know
>>>>>>>>>>>            >>>>> where
>>>>>>>>>>>            >>>>> i am doing wrong.
>>>>>>>>>>>            >>>>>
>>>>>>>>>>>            >>>>>
>>>>>>>>>>>            >>>>> Thanks
>>>>>>>>>>>            >>>>> Arjun Narasimha Kota
>>>>>>>>>>>            >>>>>
>>>>>>>>>>>            >>>>>
>>>>>>>>>>>            >>>>>
>>>>>>>>>>>            >>>>
>>>>>>>>>>>            >>
>>>>>>>>>>>            >
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>            --
>>>>>>>>>>>            -- Guozhang
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>

Re: consumer not consuming messages

Posted by Arjun <ar...@socialtwist.com>.
Even after changing the  fetch wait max ms the same thing is repeting 
just that some partitions have the owners now, i mean


bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker --group group1 
--zkconnect zkhost:zkport --topic testtopic
Group           Topic                          Pid Offset 
logSize         Lag             Owner
group1          testtopic    0   253             253 0               
group1_xxxx-1397216047177-6f419d28-0
group1          testtopic    1   268             268 0               
group1_xxxx-1397216047177-6f419d28-1
group1          testtopic    2   258             258 0               
group1_xxxx-1397216047177-6f419d28-2
group1          testtopic    3   265             265 0               none
group1          testtopic    4   262             262 0               none
group1          testtopic    5   296             296 0               none
group1          testtopic    6   249             249 0               
group1_xxxx1-1397219018735-a2cf60df-0
group1          testtopic    7   272             272 0               
group1_xxxx1-1397219018735-a2cf60df-1
group1          testtopic    8   242             242 0               
group1_xxxx1-1397219018735-a2cf60df-2
group1          testtopic    9   263             263 0               
group1_xxxx1-1397279906839-12366659-0
group1          testtopic    10  294             294 0               
group1_xxxx1-1397279906839-12366659-1
group1          testtopic    11  254             254 0               
group1_xxxx1-1397279906839-12366659-2

and i dont see any exception after

"end rebalancing consumer group1_ip-XXXX-82095a28 try #9"

i just see the below log

12 Apr 2014 00:18:58,987 INFO  [Thread-321] 
[kafka.consumer.ZookeeperConsumerConnector] [group1_ip-xxxx-82095a28], 
Rebalancing attempt failed. Clearing the cache before the next 
rebalancing operation is triggered
12 Apr 2014 00:18:58,987 INFO  [Thread-321] 
[kafka.consumer.ConsumerFetcherManager] 
[ConsumerFetcherManager-1397279912725] Stopping leader finder thread
12 Apr 2014 00:18:58,987 INFO  [Thread-321] 
[kafka.consumer.ConsumerFetcherManager] 
[ConsumerFetcherManager-1397279912725] Stopping all fetchers
12 Apr 2014 00:18:58,987 INFO  [Thread-321] 
[kafka.consumer.ConsumerFetcherManager] 
[ConsumerFetcherManager-1397279912725] All connections stopped
12 Apr 2014 00:18:58,987 INFO  [Thread-321] 
[kafka.consumer.ZookeeperConsumerConnector] [group1_xxxx-82095a28], 
Cleared all relevant queues for this fetcher
12 Apr 2014 00:18:58,988 DEBUG [Thread-321] 
[kafka.consumer.ConsumerIterator] Clearing the current data chunk for 
this consumer iterator
12 Apr 2014 00:18:58,988 DEBUG [Thread-321] 
[kafka.consumer.ConsumerIterator] Clearing the current data chunk for 
this consumer iterator
12 Apr 2014 00:18:58,988 DEBUG [Thread-321] 
[kafka.consumer.ConsumerIterator] Clearing the current data chunk for 
this consumer iterator
12 Apr 2014 00:18:58,988 INFO  [Thread-321] 
[kafka.consumer.ZookeeperConsumerConnector] [group1_xxxx-82095a28], 
Cleared the data chunks in all the consumer message iterators
12 Apr 2014 00:18:58,988 INFO  [Thread-321] 
[kafka.consumer.ZookeeperConsumerConnector] [group1_xxxx-82095a28], 
Committing all offsets after clearing the fetcher queues

after this kafka consumer will not log a thing.Not even the exception.(I 
have put my consumer log level to debug)



Thanks
Arjun Narasimha Kota


On Saturday 12 April 2014 08:41 AM, Jun Rao wrote:
> Console consumer also uses the high level consumer. Could you try setting
> fetch.wait.max.ms to 100ms?
>
> Thanks,
>
> Jun
>
>
> On Fri, Apr 11, 2014 at 9:56 AM, Arjun Kota <ar...@socialtwist.com> wrote:
>
>> Console consumer works fine. Its the high level java consumer which is
>> giving this problem.
>>
>> Thanks
>> Arjun narasimha kota
>> On Apr 11, 2014 8:42 PM, "Jun Rao" <ju...@gmail.com> wrote:
>>
>>> We may have a bug that doesn't observe etch.min.bytes accurately. So a
>>> lower fetch.wait.max.ms will improve consumer latency.
>>>
>>> Could you run a console consumer and see if you have the same issue? That
>>> will tell us if this is a server side issue or an issue just in your
>>> consumer.
>>>
>>> Thanks,
>>>
>>> Jun
>>>
>>>
>>> On Thu, Apr 10, 2014 at 10:28 PM, Arjun <ar...@socialtwist.com> wrote:
>>>
>>>> i changed the time to 60 seconds even now i see the same result. The
>>>> Consumer is not consuming the messages.
>>>>
>>>> Thanks
>>>> Arjun Narasimha Kota
>>>>
>>>>
>>>> On Friday 11 April 2014 10:36 AM, Arjun wrote:
>>>>
>>>>> yup i will change the value and recheck. Thanks for the help.
>>>>>
>>>>> thanks
>>>>> Arjun Narasimha Kota
>>>>>
>>>>> On Friday 11 April 2014 10:28 AM, Guozhang Wang wrote:
>>>>>
>>>>>> What I tried to say is that it may be caused by your
>>>>>> "fetch.wait.max.ms"="180000"
>>>>>> too large. Try a small value and see if that helps.
>>>>>>
>>>>>>
>>>>>> On Thu, Apr 10, 2014 at 9:44 PM, Arjun <ar...@socialtwist.com>
>> wrote:
>>>>>>   Hi,
>>>>>>> I could not see any out of memory exceptions in the broker logs. One
>>>>>>> thing
>>>>>>> i can see is i  may have configured consumer poorly. If its not too
>>>>>>> much to
>>>>>>> ask can u let me know the changes i have to do for over coming this
>>>>>>> problem.
>>>>>>>
>>>>>>> Thanks
>>>>>>> Arjun Narasimha Kota
>>>>>>>
>>>>>>>
>>>>>>> On Friday 11 April 2014 10:04 AM, Guozhang Wang wrote:
>>>>>>>
>>>>>>>   Hi Ajrun,
>>>>>>>> It seems to be the cause:
>>>>>>>>
>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-1016
>>>>>>>>
>>>>>>>> Guozhang
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> On Thu, Apr 10, 2014 at 9:21 PM, Arjun <ar...@socialtwist.com>
>>> wrote:
>>>>>>>>    I hope this one would give u  a better idea.
>>>>>>>>
>>>>>>>>> bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker --group
>>>>>>>>> group1
>>>>>>>>> --zkconnect zkhost:port --topic testtopic
>>>>>>>>> Group           Topic                          Pid Offset logSize
>>>>>>>>> Lag             Owner
>>>>>>>>> group1          testtopic    0   253             253 0
>>>>>>>>> group1_ip-xx-1397188061429-b5ff1205-0
>>>>>>>>> group1          testtopic    1   267             267 0
>>>>>>>>> group1_ip-xx-1397188061429-b5ff1205-0
>>>>>>>>> group1          testtopic    2   254             254 0
>>>>>>>>> group1_ip-xx-1397188061429-b5ff1205-0
>>>>>>>>> group1          testtopic    3   265             265 0
>>>>>>>>> group1_ip-xx-1397188061429-b5ff1205-0
>>>>>>>>> group1          testtopic    4   261             261 0
>>>>>>>>> group1_ip-xx-1397188061429-b5ff1205-1
>>>>>>>>> group1          testtopic    5   294             294 0
>>>>>>>>> group1_ip-xx-1397188061429-b5ff1205-1
>>>>>>>>> group1          testtopic    6   248             248 0
>>>>>>>>> group1_ip-xx-1397188061429-b5ff1205-1
>>>>>>>>> group1          testtopic    7   271             271 0
>>>>>>>>> group1_ip-xx-1397188061429-b5ff1205-1
>>>>>>>>> group1          testtopic    8   240             240 0
>>>>>>>>> group1_ip-xx-1397188061429-b5ff1205-2
>>>>>>>>> group1          testtopic    9   261             261 0
>>>>>>>>> group1_ip-xx-1397188061429-b5ff1205-2
>>>>>>>>> group1          testtopic    10  290             290 0
>>>>>>>>> group1_ip-xx-1397188061429-b5ff1205-2
>>>>>>>>> group1          testtopic    11  250             251 1
>>>>>>>>> group1_ip-xx-1397188061429-b5ff1205-2
>>>>>>>>>
>>>>>>>>> If you see the output, in the last line the lag is 1 for that
>>>>>>>>> partition.
>>>>>>>>> I
>>>>>>>>> just send one message. This topic is not new as you see there are
>>> lot
>>>>>>>>> of
>>>>>>>>> messages which have accumlated from yesterday. This one message
>> will
>>>>>>>>> not
>>>>>>>>> be
>>>>>>>>> consumed by consumer what so ever. But if i send some 10 messages
>>> then
>>>>>>>>> all
>>>>>>>>> the messages are consumed.
>>>>>>>>>
>>>>>>>>> Please let me know if i have to change any consumer properties.
>>>>>>>>>
>>>>>>>>> My consumer properties are :
>>>>>>>>> "fetch.wait.max.ms"="180000"
>>>>>>>>> "fetch.min.bytes" = "1"
>>>>>>>>> "auto.offset.reset" = "smallest"
>>>>>>>>> "auto.commit.enable"=  "false"
>>>>>>>>> "fetch.message.max.bytes" = "1048576"
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Thanks
>>>>>>>>> Arjun Narasimha Kota
>>>>>>>>> On Friday 11 April 2014 06:23 AM, Arjun Kota wrote:
>>>>>>>>>
>>>>>>>>>    The consumer uses do specific topics.
>>>>>>>>>
>>>>>>>>>> On Apr 11, 2014 6:23 AM, "Arjun Kota" <arjun@socialtwist.com
>>> <mailto:
>>>>>>>>>> arjun@socialtwist.com>> wrote:
>>>>>>>>>>
>>>>>>>>>>        Yes the message shows up on the server.
>>>>>>>>>>
>>>>>>>>>>        On Apr 11, 2014 12:07 AM, "Guozhang Wang" <
>>> wangguoz@gmail.com
>>>>>>>>>>        <ma...@gmail.com>> wrote:
>>>>>>>>>>
>>>>>>>>>>            Hi Arjun,
>>>>>>>>>>
>>>>>>>>>>            If you only send one message, does that message show up
>>> on
>>>>>>>>>> the
>>>>>>>>>>            server? Does
>>>>>>>>>>            you consumer use wildcard topics or specific topics?
>>>>>>>>>>
>>>>>>>>>>            Guozhang
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>            On Thu, Apr 10, 2014 at 9:20 AM, Arjun <
>>>>>>>>>> arjun@socialtwist.com
>>>>>>>>>>            <ma...@socialtwist.com>> wrote:
>>>>>>>>>>
>>>>>>>>>>            > But  we have auto offset reset to smallest not
>> largest,
>>>>>>>>>> even
>>>>>>>>>>            then this
>>>>>>>>>>            > issue arises? If so is there any work around?
>>>>>>>>>>            >
>>>>>>>>>>            > Thanks
>>>>>>>>>>            > Arjun NArasimha Kota
>>>>>>>>>>            >
>>>>>>>>>>            >
>>>>>>>>>>            > On Thursday 10 April 2014 09:39 PM, Guozhang Wang
>>> wrote:
>>>>>>>>>>            >
>>>>>>>>>>            >> It could be https://issues.apache.org/
>>>>>>>>>> jira/browse/KAFKA-1006.
>>>>>>>>>>            >>
>>>>>>>>>>            >> Guozhang
>>>>>>>>>>            >>
>>>>>>>>>>            >>
>>>>>>>>>>            >> On Thu, Apr 10, 2014 at 8:50 AM, Arjun
>>>>>>>>>>            <arjun@socialtwist.com <ma...@socialtwist.com>>
>>>>>>>>>> wrote:
>>>>>>>>>>            >>
>>>>>>>>>>            >>  its auto created
>>>>>>>>>>            >>> but even after topic creation this is the scenario
>>>>>>>>>>            >>>
>>>>>>>>>>            >>> Arjun
>>>>>>>>>>            >>>
>>>>>>>>>>            >>> On Thursday 10 April 2014 08:41 PM, Guozhang Wang
>>>>>>>>>> wrote:
>>>>>>>>>>            >>>
>>>>>>>>>>            >>>  Hi Arjun,
>>>>>>>>>>            >>>>
>>>>>>>>>>            >>>> Did you manually create the topic or use
>>>>>>>>>> auto.topic.creation?
>>>>>>>>>>            >>>>
>>>>>>>>>>            >>>> Guozhang
>>>>>>>>>>            >>>>
>>>>>>>>>>            >>>>
>>>>>>>>>>            >>>> On Thu, Apr 10, 2014 at 7:39 AM, Arjun
>>>>>>>>>>            <arjun@socialtwist.com <ma...@socialtwist.com>>
>>>>>>>>>> wrote:
>>>>>>>>>>            >>>>
>>>>>>>>>>            >>>>   Hi,
>>>>>>>>>>            >>>>
>>>>>>>>>>            >>>>> We have 3 node kafka 0.8 setup with zookeepers
>>>>>>>>>> ensemble.
>>>>>>>>>>            We use high
>>>>>>>>>>            >>>>> level
>>>>>>>>>>            >>>>> consumer with auto commit offset false. I am
>> facing
>>>>>>>>>> some
>>>>>>>>>>            peculiar
>>>>>>>>>>            >>>>> problem
>>>>>>>>>>            >>>>> with kafka. When i send some 10-20 messages or so
>>> the
>>>>>>>>>>            consumer starts
>>>>>>>>>>            >>>>> to
>>>>>>>>>>            >>>>> consume the messages. But if  i send only one
>>>>>>>>>> message to
>>>>>>>>>>            kafka, then
>>>>>>>>>>            >>>>> even
>>>>>>>>>>            >>>>> though consumer is active it is not trying to
>> fetch
>>>>>>>>>> the
>>>>>>>>>>            message. There
>>>>>>>>>>            >>>>> is
>>>>>>>>>>            >>>>> nothing in logs, just the messages are being
>>> fetched
>>>>>>>>>> by
>>>>>>>>>>            the kafka
>>>>>>>>>>            >>>>> consumer.
>>>>>>>>>>            >>>>> The messages are there in the Kafka server. Can
>>> some
>>>>>>>>>> one
>>>>>>>>>>            let me know
>>>>>>>>>>            >>>>> where
>>>>>>>>>>            >>>>> i am doing wrong.
>>>>>>>>>>            >>>>>
>>>>>>>>>>            >>>>>
>>>>>>>>>>            >>>>> Thanks
>>>>>>>>>>            >>>>> Arjun Narasimha Kota
>>>>>>>>>>            >>>>>
>>>>>>>>>>            >>>>>
>>>>>>>>>>            >>>>>
>>>>>>>>>>            >>>>
>>>>>>>>>>            >>
>>>>>>>>>>            >
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>            --
>>>>>>>>>>            -- Guozhang
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>


Re: consumer not consuming messages

Posted by Jun Rao <ju...@gmail.com>.
Console consumer also uses the high level consumer. Could you try setting
fetch.wait.max.ms to 100ms?

Thanks,

Jun


On Fri, Apr 11, 2014 at 9:56 AM, Arjun Kota <ar...@socialtwist.com> wrote:

> Console consumer works fine. Its the high level java consumer which is
> giving this problem.
>
> Thanks
> Arjun narasimha kota
> On Apr 11, 2014 8:42 PM, "Jun Rao" <ju...@gmail.com> wrote:
>
> > We may have a bug that doesn't observe etch.min.bytes accurately. So a
> > lower fetch.wait.max.ms will improve consumer latency.
> >
> > Could you run a console consumer and see if you have the same issue? That
> > will tell us if this is a server side issue or an issue just in your
> > consumer.
> >
> > Thanks,
> >
> > Jun
> >
> >
> > On Thu, Apr 10, 2014 at 10:28 PM, Arjun <ar...@socialtwist.com> wrote:
> >
> > > i changed the time to 60 seconds even now i see the same result. The
> > > Consumer is not consuming the messages.
> > >
> > > Thanks
> > > Arjun Narasimha Kota
> > >
> > >
> > > On Friday 11 April 2014 10:36 AM, Arjun wrote:
> > >
> > >> yup i will change the value and recheck. Thanks for the help.
> > >>
> > >> thanks
> > >> Arjun Narasimha Kota
> > >>
> > >> On Friday 11 April 2014 10:28 AM, Guozhang Wang wrote:
> > >>
> > >>> What I tried to say is that it may be caused by your
> > >>> "fetch.wait.max.ms"="180000"
> > >>> too large. Try a small value and see if that helps.
> > >>>
> > >>>
> > >>> On Thu, Apr 10, 2014 at 9:44 PM, Arjun <ar...@socialtwist.com>
> wrote:
> > >>>
> > >>>  Hi,
> > >>>>
> > >>>> I could not see any out of memory exceptions in the broker logs. One
> > >>>> thing
> > >>>> i can see is i  may have configured consumer poorly. If its not too
> > >>>> much to
> > >>>> ask can u let me know the changes i have to do for over coming this
> > >>>> problem.
> > >>>>
> > >>>> Thanks
> > >>>> Arjun Narasimha Kota
> > >>>>
> > >>>>
> > >>>> On Friday 11 April 2014 10:04 AM, Guozhang Wang wrote:
> > >>>>
> > >>>>  Hi Ajrun,
> > >>>>>
> > >>>>> It seems to be the cause:
> > >>>>>
> > >>>>> https://issues.apache.org/jira/browse/KAFKA-1016
> > >>>>>
> > >>>>> Guozhang
> > >>>>>
> > >>>>>
> > >>>>>
> > >>>>> On Thu, Apr 10, 2014 at 9:21 PM, Arjun <ar...@socialtwist.com>
> > wrote:
> > >>>>>
> > >>>>>   I hope this one would give u  a better idea.
> > >>>>>
> > >>>>>> bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker --group
> > >>>>>> group1
> > >>>>>> --zkconnect zkhost:port --topic testtopic
> > >>>>>> Group           Topic                          Pid Offset logSize
> > >>>>>> Lag             Owner
> > >>>>>> group1          testtopic    0   253             253 0
> > >>>>>> group1_ip-xx-1397188061429-b5ff1205-0
> > >>>>>> group1          testtopic    1   267             267 0
> > >>>>>> group1_ip-xx-1397188061429-b5ff1205-0
> > >>>>>> group1          testtopic    2   254             254 0
> > >>>>>> group1_ip-xx-1397188061429-b5ff1205-0
> > >>>>>> group1          testtopic    3   265             265 0
> > >>>>>> group1_ip-xx-1397188061429-b5ff1205-0
> > >>>>>> group1          testtopic    4   261             261 0
> > >>>>>> group1_ip-xx-1397188061429-b5ff1205-1
> > >>>>>> group1          testtopic    5   294             294 0
> > >>>>>> group1_ip-xx-1397188061429-b5ff1205-1
> > >>>>>> group1          testtopic    6   248             248 0
> > >>>>>> group1_ip-xx-1397188061429-b5ff1205-1
> > >>>>>> group1          testtopic    7   271             271 0
> > >>>>>> group1_ip-xx-1397188061429-b5ff1205-1
> > >>>>>> group1          testtopic    8   240             240 0
> > >>>>>> group1_ip-xx-1397188061429-b5ff1205-2
> > >>>>>> group1          testtopic    9   261             261 0
> > >>>>>> group1_ip-xx-1397188061429-b5ff1205-2
> > >>>>>> group1          testtopic    10  290             290 0
> > >>>>>> group1_ip-xx-1397188061429-b5ff1205-2
> > >>>>>> group1          testtopic    11  250             251 1
> > >>>>>> group1_ip-xx-1397188061429-b5ff1205-2
> > >>>>>>
> > >>>>>> If you see the output, in the last line the lag is 1 for that
> > >>>>>> partition.
> > >>>>>> I
> > >>>>>> just send one message. This topic is not new as you see there are
> > lot
> > >>>>>> of
> > >>>>>> messages which have accumlated from yesterday. This one message
> will
> > >>>>>> not
> > >>>>>> be
> > >>>>>> consumed by consumer what so ever. But if i send some 10 messages
> > then
> > >>>>>> all
> > >>>>>> the messages are consumed.
> > >>>>>>
> > >>>>>> Please let me know if i have to change any consumer properties.
> > >>>>>>
> > >>>>>> My consumer properties are :
> > >>>>>> "fetch.wait.max.ms"="180000"
> > >>>>>> "fetch.min.bytes" = "1"
> > >>>>>> "auto.offset.reset" = "smallest"
> > >>>>>> "auto.commit.enable"=  "false"
> > >>>>>> "fetch.message.max.bytes" = "1048576"
> > >>>>>>
> > >>>>>>
> > >>>>>> Thanks
> > >>>>>> Arjun Narasimha Kota
> > >>>>>> On Friday 11 April 2014 06:23 AM, Arjun Kota wrote:
> > >>>>>>
> > >>>>>>   The consumer uses do specific topics.
> > >>>>>>
> > >>>>>>> On Apr 11, 2014 6:23 AM, "Arjun Kota" <arjun@socialtwist.com
> > <mailto:
> > >>>>>>> arjun@socialtwist.com>> wrote:
> > >>>>>>>
> > >>>>>>>       Yes the message shows up on the server.
> > >>>>>>>
> > >>>>>>>       On Apr 11, 2014 12:07 AM, "Guozhang Wang" <
> > wangguoz@gmail.com
> > >>>>>>>       <ma...@gmail.com>> wrote:
> > >>>>>>>
> > >>>>>>>           Hi Arjun,
> > >>>>>>>
> > >>>>>>>           If you only send one message, does that message show up
> > on
> > >>>>>>> the
> > >>>>>>>           server? Does
> > >>>>>>>           you consumer use wildcard topics or specific topics?
> > >>>>>>>
> > >>>>>>>           Guozhang
> > >>>>>>>
> > >>>>>>>
> > >>>>>>>           On Thu, Apr 10, 2014 at 9:20 AM, Arjun <
> > >>>>>>> arjun@socialtwist.com
> > >>>>>>>           <ma...@socialtwist.com>> wrote:
> > >>>>>>>
> > >>>>>>>           > But  we have auto offset reset to smallest not
> largest,
> > >>>>>>> even
> > >>>>>>>           then this
> > >>>>>>>           > issue arises? If so is there any work around?
> > >>>>>>>           >
> > >>>>>>>           > Thanks
> > >>>>>>>           > Arjun NArasimha Kota
> > >>>>>>>           >
> > >>>>>>>           >
> > >>>>>>>           > On Thursday 10 April 2014 09:39 PM, Guozhang Wang
> > wrote:
> > >>>>>>>           >
> > >>>>>>>           >> It could be https://issues.apache.org/
> > >>>>>>> jira/browse/KAFKA-1006.
> > >>>>>>>           >>
> > >>>>>>>           >> Guozhang
> > >>>>>>>           >>
> > >>>>>>>           >>
> > >>>>>>>           >> On Thu, Apr 10, 2014 at 8:50 AM, Arjun
> > >>>>>>>           <arjun@socialtwist.com <ma...@socialtwist.com>>
> > >>>>>>> wrote:
> > >>>>>>>           >>
> > >>>>>>>           >>  its auto created
> > >>>>>>>           >>> but even after topic creation this is the scenario
> > >>>>>>>           >>>
> > >>>>>>>           >>> Arjun
> > >>>>>>>           >>>
> > >>>>>>>           >>> On Thursday 10 April 2014 08:41 PM, Guozhang Wang
> > >>>>>>> wrote:
> > >>>>>>>           >>>
> > >>>>>>>           >>>  Hi Arjun,
> > >>>>>>>           >>>>
> > >>>>>>>           >>>> Did you manually create the topic or use
> > >>>>>>> auto.topic.creation?
> > >>>>>>>           >>>>
> > >>>>>>>           >>>> Guozhang
> > >>>>>>>           >>>>
> > >>>>>>>           >>>>
> > >>>>>>>           >>>> On Thu, Apr 10, 2014 at 7:39 AM, Arjun
> > >>>>>>>           <arjun@socialtwist.com <ma...@socialtwist.com>>
> > >>>>>>> wrote:
> > >>>>>>>           >>>>
> > >>>>>>>           >>>>   Hi,
> > >>>>>>>           >>>>
> > >>>>>>>           >>>>> We have 3 node kafka 0.8 setup with zookeepers
> > >>>>>>> ensemble.
> > >>>>>>>           We use high
> > >>>>>>>           >>>>> level
> > >>>>>>>           >>>>> consumer with auto commit offset false. I am
> facing
> > >>>>>>> some
> > >>>>>>>           peculiar
> > >>>>>>>           >>>>> problem
> > >>>>>>>           >>>>> with kafka. When i send some 10-20 messages or so
> > the
> > >>>>>>>           consumer starts
> > >>>>>>>           >>>>> to
> > >>>>>>>           >>>>> consume the messages. But if  i send only one
> > >>>>>>> message to
> > >>>>>>>           kafka, then
> > >>>>>>>           >>>>> even
> > >>>>>>>           >>>>> though consumer is active it is not trying to
> fetch
> > >>>>>>> the
> > >>>>>>>           message. There
> > >>>>>>>           >>>>> is
> > >>>>>>>           >>>>> nothing in logs, just the messages are being
> > fetched
> > >>>>>>> by
> > >>>>>>>           the kafka
> > >>>>>>>           >>>>> consumer.
> > >>>>>>>           >>>>> The messages are there in the Kafka server. Can
> > some
> > >>>>>>> one
> > >>>>>>>           let me know
> > >>>>>>>           >>>>> where
> > >>>>>>>           >>>>> i am doing wrong.
> > >>>>>>>           >>>>>
> > >>>>>>>           >>>>>
> > >>>>>>>           >>>>> Thanks
> > >>>>>>>           >>>>> Arjun Narasimha Kota
> > >>>>>>>           >>>>>
> > >>>>>>>           >>>>>
> > >>>>>>>           >>>>>
> > >>>>>>>           >>>>
> > >>>>>>>           >>
> > >>>>>>>           >
> > >>>>>>>
> > >>>>>>>
> > >>>>>>>           --
> > >>>>>>>           -- Guozhang
> > >>>>>>>
> > >>>>>>>
> > >>>>>>>
> > >>>>>>>
> > >>>
> > >>
> > >
> >
>

RE: consumer not consuming messages

Posted by Arjun Kota <ar...@socialtwist.com>.
Yup i am, if i get any message only then i commit the offset, if not i am
not commiting.

Thanks
Arjun narasimha kota
On Apr 11, 2014 10:40 PM, "Seshadri, Balaji" <Ba...@dish.com>
wrote:

> Are you committing offsets manually after you consume as you mentioned
> earlier that "auto.commit.offset" is false.
>
> -----Original Message-----
> From: Arjun Kota [mailto:arjun@socialtwist.com]
> Sent: Friday, April 11, 2014 10:56 AM
> To: users@kafka.apache.org
> Subject: Re: consumer not consuming messages
>
> Console consumer works fine. Its the high level java consumer which is
> giving this problem.
>
> Thanks
> Arjun narasimha kota
> On Apr 11, 2014 8:42 PM, "Jun Rao" <ju...@gmail.com> wrote:
>
> > We may have a bug that doesn't observe etch.min.bytes accurately. So a
> > lower fetch.wait.max.ms will improve consumer latency.
> >
> > Could you run a console consumer and see if you have the same issue?
> > That will tell us if this is a server side issue or an issue just in
> > your consumer.
> >
> > Thanks,
> >
> > Jun
> >
> >
> > On Thu, Apr 10, 2014 at 10:28 PM, Arjun <ar...@socialtwist.com> wrote:
> >
> > > i changed the time to 60 seconds even now i see the same result. The
> > > Consumer is not consuming the messages.
> > >
> > > Thanks
> > > Arjun Narasimha Kota
> > >
> > >
> > > On Friday 11 April 2014 10:36 AM, Arjun wrote:
> > >
> > >> yup i will change the value and recheck. Thanks for the help.
> > >>
> > >> thanks
> > >> Arjun Narasimha Kota
> > >>
> > >> On Friday 11 April 2014 10:28 AM, Guozhang Wang wrote:
> > >>
> > >>> What I tried to say is that it may be caused by your
> > >>> "fetch.wait.max.ms"="180000"
> > >>> too large. Try a small value and see if that helps.
> > >>>
> > >>>
> > >>> On Thu, Apr 10, 2014 at 9:44 PM, Arjun <ar...@socialtwist.com>
> wrote:
> > >>>
> > >>>  Hi,
> > >>>>
> > >>>> I could not see any out of memory exceptions in the broker logs.
> > >>>> One thing i can see is i  may have configured consumer poorly. If
> > >>>> its not too much to ask can u let me know the changes i have to
> > >>>> do for over coming this problem.
> > >>>>
> > >>>> Thanks
> > >>>> Arjun Narasimha Kota
> > >>>>
> > >>>>
> > >>>> On Friday 11 April 2014 10:04 AM, Guozhang Wang wrote:
> > >>>>
> > >>>>  Hi Ajrun,
> > >>>>>
> > >>>>> It seems to be the cause:
> > >>>>>
> > >>>>> https://issues.apache.org/jira/browse/KAFKA-1016
> > >>>>>
> > >>>>> Guozhang
> > >>>>>
> > >>>>>
> > >>>>>
> > >>>>> On Thu, Apr 10, 2014 at 9:21 PM, Arjun <ar...@socialtwist.com>
> > wrote:
> > >>>>>
> > >>>>>   I hope this one would give u  a better idea.
> > >>>>>
> > >>>>>> bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker
> > >>>>>> --group
> > >>>>>> group1
> > >>>>>> --zkconnect zkhost:port --topic testtopic
> > >>>>>> Group           Topic                          Pid Offset logSize
> > >>>>>> Lag             Owner
> > >>>>>> group1          testtopic    0   253             253 0
> > >>>>>> group1_ip-xx-1397188061429-b5ff1205-0
> > >>>>>> group1          testtopic    1   267             267 0
> > >>>>>> group1_ip-xx-1397188061429-b5ff1205-0
> > >>>>>> group1          testtopic    2   254             254 0
> > >>>>>> group1_ip-xx-1397188061429-b5ff1205-0
> > >>>>>> group1          testtopic    3   265             265 0
> > >>>>>> group1_ip-xx-1397188061429-b5ff1205-0
> > >>>>>> group1          testtopic    4   261             261 0
> > >>>>>> group1_ip-xx-1397188061429-b5ff1205-1
> > >>>>>> group1          testtopic    5   294             294 0
> > >>>>>> group1_ip-xx-1397188061429-b5ff1205-1
> > >>>>>> group1          testtopic    6   248             248 0
> > >>>>>> group1_ip-xx-1397188061429-b5ff1205-1
> > >>>>>> group1          testtopic    7   271             271 0
> > >>>>>> group1_ip-xx-1397188061429-b5ff1205-1
> > >>>>>> group1          testtopic    8   240             240 0
> > >>>>>> group1_ip-xx-1397188061429-b5ff1205-2
> > >>>>>> group1          testtopic    9   261             261 0
> > >>>>>> group1_ip-xx-1397188061429-b5ff1205-2
> > >>>>>> group1          testtopic    10  290             290 0
> > >>>>>> group1_ip-xx-1397188061429-b5ff1205-2
> > >>>>>> group1          testtopic    11  250             251 1
> > >>>>>> group1_ip-xx-1397188061429-b5ff1205-2
> > >>>>>>
> > >>>>>> If you see the output, in the last line the lag is 1 for that
> > >>>>>> partition.
> > >>>>>> I
> > >>>>>> just send one message. This topic is not new as you see there
> > >>>>>> are
> > lot
> > >>>>>> of
> > >>>>>> messages which have accumlated from yesterday. This one message
> > >>>>>> will not be consumed by consumer what so ever. But if i send
> > >>>>>> some 10 messages
> > then
> > >>>>>> all
> > >>>>>> the messages are consumed.
> > >>>>>>
> > >>>>>> Please let me know if i have to change any consumer properties.
> > >>>>>>
> > >>>>>> My consumer properties are :
> > >>>>>> "fetch.wait.max.ms"="180000"
> > >>>>>> "fetch.min.bytes" = "1"
> > >>>>>> "auto.offset.reset" = "smallest"
> > >>>>>> "auto.commit.enable"=  "false"
> > >>>>>> "fetch.message.max.bytes" = "1048576"
> > >>>>>>
> > >>>>>>
> > >>>>>> Thanks
> > >>>>>> Arjun Narasimha Kota
> > >>>>>> On Friday 11 April 2014 06:23 AM, Arjun Kota wrote:
> > >>>>>>
> > >>>>>>   The consumer uses do specific topics.
> > >>>>>>
> > >>>>>>> On Apr 11, 2014 6:23 AM, "Arjun Kota" <arjun@socialtwist.com
> > <mailto:
> > >>>>>>> arjun@socialtwist.com>> wrote:
> > >>>>>>>
> > >>>>>>>       Yes the message shows up on the server.
> > >>>>>>>
> > >>>>>>>       On Apr 11, 2014 12:07 AM, "Guozhang Wang" <
> > wangguoz@gmail.com
> > >>>>>>>       <ma...@gmail.com>> wrote:
> > >>>>>>>
> > >>>>>>>           Hi Arjun,
> > >>>>>>>
> > >>>>>>>           If you only send one message, does that message show
> > >>>>>>> up
> > on
> > >>>>>>> the
> > >>>>>>>           server? Does
> > >>>>>>>           you consumer use wildcard topics or specific topics?
> > >>>>>>>
> > >>>>>>>           Guozhang
> > >>>>>>>
> > >>>>>>>
> > >>>>>>>           On Thu, Apr 10, 2014 at 9:20 AM, Arjun <
> > >>>>>>> arjun@socialtwist.com
> > >>>>>>>           <ma...@socialtwist.com>> wrote:
> > >>>>>>>
> > >>>>>>>           > But  we have auto offset reset to smallest not
> > >>>>>>> largest, even
> > >>>>>>>           then this
> > >>>>>>>           > issue arises? If so is there any work around?
> > >>>>>>>           >
> > >>>>>>>           > Thanks
> > >>>>>>>           > Arjun NArasimha Kota
> > >>>>>>>           >
> > >>>>>>>           >
> > >>>>>>>           > On Thursday 10 April 2014 09:39 PM, Guozhang Wang
> > wrote:
> > >>>>>>>           >
> > >>>>>>>           >> It could be https://issues.apache.org/
> > >>>>>>> jira/browse/KAFKA-1006.
> > >>>>>>>           >>
> > >>>>>>>           >> Guozhang
> > >>>>>>>           >>
> > >>>>>>>           >>
> > >>>>>>>           >> On Thu, Apr 10, 2014 at 8:50 AM, Arjun
> > >>>>>>>           <arjun@socialtwist.com
> > >>>>>>> <ma...@socialtwist.com>>
> > >>>>>>> wrote:
> > >>>>>>>           >>
> > >>>>>>>           >>  its auto created
> > >>>>>>>           >>> but even after topic creation this is the scenario
> > >>>>>>>           >>>
> > >>>>>>>           >>> Arjun
> > >>>>>>>           >>>
> > >>>>>>>           >>> On Thursday 10 April 2014 08:41 PM, Guozhang
> > >>>>>>> Wang
> > >>>>>>> wrote:
> > >>>>>>>           >>>
> > >>>>>>>           >>>  Hi Arjun,
> > >>>>>>>           >>>>
> > >>>>>>>           >>>> Did you manually create the topic or use
> > >>>>>>> auto.topic.creation?
> > >>>>>>>           >>>>
> > >>>>>>>           >>>> Guozhang
> > >>>>>>>           >>>>
> > >>>>>>>           >>>>
> > >>>>>>>           >>>> On Thu, Apr 10, 2014 at 7:39 AM, Arjun
> > >>>>>>>           <arjun@socialtwist.com
> > >>>>>>> <ma...@socialtwist.com>>
> > >>>>>>> wrote:
> > >>>>>>>           >>>>
> > >>>>>>>           >>>>   Hi,
> > >>>>>>>           >>>>
> > >>>>>>>           >>>>> We have 3 node kafka 0.8 setup with zookeepers
> > >>>>>>> ensemble.
> > >>>>>>>           We use high
> > >>>>>>>           >>>>> level
> > >>>>>>>           >>>>> consumer with auto commit offset false. I am
> > >>>>>>> facing some
> > >>>>>>>           peculiar
> > >>>>>>>           >>>>> problem
> > >>>>>>>           >>>>> with kafka. When i send some 10-20 messages or
> > >>>>>>> so
> > the
> > >>>>>>>           consumer starts
> > >>>>>>>           >>>>> to
> > >>>>>>>           >>>>> consume the messages. But if  i send only one
> > >>>>>>> message to
> > >>>>>>>           kafka, then
> > >>>>>>>           >>>>> even
> > >>>>>>>           >>>>> though consumer is active it is not trying to
> > >>>>>>> fetch the
> > >>>>>>>           message. There
> > >>>>>>>           >>>>> is
> > >>>>>>>           >>>>> nothing in logs, just the messages are being
> > fetched
> > >>>>>>> by
> > >>>>>>>           the kafka
> > >>>>>>>           >>>>> consumer.
> > >>>>>>>           >>>>> The messages are there in the Kafka server.
> > >>>>>>> Can
> > some
> > >>>>>>> one
> > >>>>>>>           let me know
> > >>>>>>>           >>>>> where
> > >>>>>>>           >>>>> i am doing wrong.
> > >>>>>>>           >>>>>
> > >>>>>>>           >>>>>
> > >>>>>>>           >>>>> Thanks
> > >>>>>>>           >>>>> Arjun Narasimha Kota
> > >>>>>>>           >>>>>
> > >>>>>>>           >>>>>
> > >>>>>>>           >>>>>
> > >>>>>>>           >>>>
> > >>>>>>>           >>
> > >>>>>>>           >
> > >>>>>>>
> > >>>>>>>
> > >>>>>>>           --
> > >>>>>>>           -- Guozhang
> > >>>>>>>
> > >>>>>>>
> > >>>>>>>
> > >>>>>>>
> > >>>
> > >>
> > >
> >
>

RE: consumer not consuming messages

Posted by "Seshadri, Balaji" <Ba...@dish.com>.
Are you committing offsets manually after you consume as you mentioned earlier that "auto.commit.offset" is false.

-----Original Message-----
From: Arjun Kota [mailto:arjun@socialtwist.com] 
Sent: Friday, April 11, 2014 10:56 AM
To: users@kafka.apache.org
Subject: Re: consumer not consuming messages

Console consumer works fine. Its the high level java consumer which is giving this problem.

Thanks
Arjun narasimha kota
On Apr 11, 2014 8:42 PM, "Jun Rao" <ju...@gmail.com> wrote:

> We may have a bug that doesn't observe etch.min.bytes accurately. So a 
> lower fetch.wait.max.ms will improve consumer latency.
>
> Could you run a console consumer and see if you have the same issue? 
> That will tell us if this is a server side issue or an issue just in 
> your consumer.
>
> Thanks,
>
> Jun
>
>
> On Thu, Apr 10, 2014 at 10:28 PM, Arjun <ar...@socialtwist.com> wrote:
>
> > i changed the time to 60 seconds even now i see the same result. The 
> > Consumer is not consuming the messages.
> >
> > Thanks
> > Arjun Narasimha Kota
> >
> >
> > On Friday 11 April 2014 10:36 AM, Arjun wrote:
> >
> >> yup i will change the value and recheck. Thanks for the help.
> >>
> >> thanks
> >> Arjun Narasimha Kota
> >>
> >> On Friday 11 April 2014 10:28 AM, Guozhang Wang wrote:
> >>
> >>> What I tried to say is that it may be caused by your 
> >>> "fetch.wait.max.ms"="180000"
> >>> too large. Try a small value and see if that helps.
> >>>
> >>>
> >>> On Thu, Apr 10, 2014 at 9:44 PM, Arjun <ar...@socialtwist.com> wrote:
> >>>
> >>>  Hi,
> >>>>
> >>>> I could not see any out of memory exceptions in the broker logs. 
> >>>> One thing i can see is i  may have configured consumer poorly. If 
> >>>> its not too much to ask can u let me know the changes i have to 
> >>>> do for over coming this problem.
> >>>>
> >>>> Thanks
> >>>> Arjun Narasimha Kota
> >>>>
> >>>>
> >>>> On Friday 11 April 2014 10:04 AM, Guozhang Wang wrote:
> >>>>
> >>>>  Hi Ajrun,
> >>>>>
> >>>>> It seems to be the cause:
> >>>>>
> >>>>> https://issues.apache.org/jira/browse/KAFKA-1016
> >>>>>
> >>>>> Guozhang
> >>>>>
> >>>>>
> >>>>>
> >>>>> On Thu, Apr 10, 2014 at 9:21 PM, Arjun <ar...@socialtwist.com>
> wrote:
> >>>>>
> >>>>>   I hope this one would give u  a better idea.
> >>>>>
> >>>>>> bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker 
> >>>>>> --group
> >>>>>> group1
> >>>>>> --zkconnect zkhost:port --topic testtopic
> >>>>>> Group           Topic                          Pid Offset logSize
> >>>>>> Lag             Owner
> >>>>>> group1          testtopic    0   253             253 0
> >>>>>> group1_ip-xx-1397188061429-b5ff1205-0
> >>>>>> group1          testtopic    1   267             267 0
> >>>>>> group1_ip-xx-1397188061429-b5ff1205-0
> >>>>>> group1          testtopic    2   254             254 0
> >>>>>> group1_ip-xx-1397188061429-b5ff1205-0
> >>>>>> group1          testtopic    3   265             265 0
> >>>>>> group1_ip-xx-1397188061429-b5ff1205-0
> >>>>>> group1          testtopic    4   261             261 0
> >>>>>> group1_ip-xx-1397188061429-b5ff1205-1
> >>>>>> group1          testtopic    5   294             294 0
> >>>>>> group1_ip-xx-1397188061429-b5ff1205-1
> >>>>>> group1          testtopic    6   248             248 0
> >>>>>> group1_ip-xx-1397188061429-b5ff1205-1
> >>>>>> group1          testtopic    7   271             271 0
> >>>>>> group1_ip-xx-1397188061429-b5ff1205-1
> >>>>>> group1          testtopic    8   240             240 0
> >>>>>> group1_ip-xx-1397188061429-b5ff1205-2
> >>>>>> group1          testtopic    9   261             261 0
> >>>>>> group1_ip-xx-1397188061429-b5ff1205-2
> >>>>>> group1          testtopic    10  290             290 0
> >>>>>> group1_ip-xx-1397188061429-b5ff1205-2
> >>>>>> group1          testtopic    11  250             251 1
> >>>>>> group1_ip-xx-1397188061429-b5ff1205-2
> >>>>>>
> >>>>>> If you see the output, in the last line the lag is 1 for that 
> >>>>>> partition.
> >>>>>> I
> >>>>>> just send one message. This topic is not new as you see there 
> >>>>>> are
> lot
> >>>>>> of
> >>>>>> messages which have accumlated from yesterday. This one message 
> >>>>>> will not be consumed by consumer what so ever. But if i send 
> >>>>>> some 10 messages
> then
> >>>>>> all
> >>>>>> the messages are consumed.
> >>>>>>
> >>>>>> Please let me know if i have to change any consumer properties.
> >>>>>>
> >>>>>> My consumer properties are :
> >>>>>> "fetch.wait.max.ms"="180000"
> >>>>>> "fetch.min.bytes" = "1"
> >>>>>> "auto.offset.reset" = "smallest"
> >>>>>> "auto.commit.enable"=  "false"
> >>>>>> "fetch.message.max.bytes" = "1048576"
> >>>>>>
> >>>>>>
> >>>>>> Thanks
> >>>>>> Arjun Narasimha Kota
> >>>>>> On Friday 11 April 2014 06:23 AM, Arjun Kota wrote:
> >>>>>>
> >>>>>>   The consumer uses do specific topics.
> >>>>>>
> >>>>>>> On Apr 11, 2014 6:23 AM, "Arjun Kota" <arjun@socialtwist.com
> <mailto:
> >>>>>>> arjun@socialtwist.com>> wrote:
> >>>>>>>
> >>>>>>>       Yes the message shows up on the server.
> >>>>>>>
> >>>>>>>       On Apr 11, 2014 12:07 AM, "Guozhang Wang" <
> wangguoz@gmail.com
> >>>>>>>       <ma...@gmail.com>> wrote:
> >>>>>>>
> >>>>>>>           Hi Arjun,
> >>>>>>>
> >>>>>>>           If you only send one message, does that message show 
> >>>>>>> up
> on
> >>>>>>> the
> >>>>>>>           server? Does
> >>>>>>>           you consumer use wildcard topics or specific topics?
> >>>>>>>
> >>>>>>>           Guozhang
> >>>>>>>
> >>>>>>>
> >>>>>>>           On Thu, Apr 10, 2014 at 9:20 AM, Arjun < 
> >>>>>>> arjun@socialtwist.com
> >>>>>>>           <ma...@socialtwist.com>> wrote:
> >>>>>>>
> >>>>>>>           > But  we have auto offset reset to smallest not 
> >>>>>>> largest, even
> >>>>>>>           then this
> >>>>>>>           > issue arises? If so is there any work around?
> >>>>>>>           >
> >>>>>>>           > Thanks
> >>>>>>>           > Arjun NArasimha Kota
> >>>>>>>           >
> >>>>>>>           >
> >>>>>>>           > On Thursday 10 April 2014 09:39 PM, Guozhang Wang
> wrote:
> >>>>>>>           >
> >>>>>>>           >> It could be https://issues.apache.org/ 
> >>>>>>> jira/browse/KAFKA-1006.
> >>>>>>>           >>
> >>>>>>>           >> Guozhang
> >>>>>>>           >>
> >>>>>>>           >>
> >>>>>>>           >> On Thu, Apr 10, 2014 at 8:50 AM, Arjun
> >>>>>>>           <arjun@socialtwist.com 
> >>>>>>> <ma...@socialtwist.com>>
> >>>>>>> wrote:
> >>>>>>>           >>
> >>>>>>>           >>  its auto created
> >>>>>>>           >>> but even after topic creation this is the scenario
> >>>>>>>           >>>
> >>>>>>>           >>> Arjun
> >>>>>>>           >>>
> >>>>>>>           >>> On Thursday 10 April 2014 08:41 PM, Guozhang 
> >>>>>>> Wang
> >>>>>>> wrote:
> >>>>>>>           >>>
> >>>>>>>           >>>  Hi Arjun,
> >>>>>>>           >>>>
> >>>>>>>           >>>> Did you manually create the topic or use 
> >>>>>>> auto.topic.creation?
> >>>>>>>           >>>>
> >>>>>>>           >>>> Guozhang
> >>>>>>>           >>>>
> >>>>>>>           >>>>
> >>>>>>>           >>>> On Thu, Apr 10, 2014 at 7:39 AM, Arjun
> >>>>>>>           <arjun@socialtwist.com 
> >>>>>>> <ma...@socialtwist.com>>
> >>>>>>> wrote:
> >>>>>>>           >>>>
> >>>>>>>           >>>>   Hi,
> >>>>>>>           >>>>
> >>>>>>>           >>>>> We have 3 node kafka 0.8 setup with zookeepers 
> >>>>>>> ensemble.
> >>>>>>>           We use high
> >>>>>>>           >>>>> level
> >>>>>>>           >>>>> consumer with auto commit offset false. I am 
> >>>>>>> facing some
> >>>>>>>           peculiar
> >>>>>>>           >>>>> problem
> >>>>>>>           >>>>> with kafka. When i send some 10-20 messages or 
> >>>>>>> so
> the
> >>>>>>>           consumer starts
> >>>>>>>           >>>>> to
> >>>>>>>           >>>>> consume the messages. But if  i send only one 
> >>>>>>> message to
> >>>>>>>           kafka, then
> >>>>>>>           >>>>> even
> >>>>>>>           >>>>> though consumer is active it is not trying to 
> >>>>>>> fetch the
> >>>>>>>           message. There
> >>>>>>>           >>>>> is
> >>>>>>>           >>>>> nothing in logs, just the messages are being
> fetched
> >>>>>>> by
> >>>>>>>           the kafka
> >>>>>>>           >>>>> consumer.
> >>>>>>>           >>>>> The messages are there in the Kafka server. 
> >>>>>>> Can
> some
> >>>>>>> one
> >>>>>>>           let me know
> >>>>>>>           >>>>> where
> >>>>>>>           >>>>> i am doing wrong.
> >>>>>>>           >>>>>
> >>>>>>>           >>>>>
> >>>>>>>           >>>>> Thanks
> >>>>>>>           >>>>> Arjun Narasimha Kota
> >>>>>>>           >>>>>
> >>>>>>>           >>>>>
> >>>>>>>           >>>>>
> >>>>>>>           >>>>
> >>>>>>>           >>
> >>>>>>>           >
> >>>>>>>
> >>>>>>>
> >>>>>>>           --
> >>>>>>>           -- Guozhang
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>
> >>
> >
>

Re: consumer not consuming messages

Posted by Arjun Kota <ar...@socialtwist.com>.
Console consumer works fine. Its the high level java consumer which is
giving this problem.

Thanks
Arjun narasimha kota
On Apr 11, 2014 8:42 PM, "Jun Rao" <ju...@gmail.com> wrote:

> We may have a bug that doesn't observe etch.min.bytes accurately. So a
> lower fetch.wait.max.ms will improve consumer latency.
>
> Could you run a console consumer and see if you have the same issue? That
> will tell us if this is a server side issue or an issue just in your
> consumer.
>
> Thanks,
>
> Jun
>
>
> On Thu, Apr 10, 2014 at 10:28 PM, Arjun <ar...@socialtwist.com> wrote:
>
> > i changed the time to 60 seconds even now i see the same result. The
> > Consumer is not consuming the messages.
> >
> > Thanks
> > Arjun Narasimha Kota
> >
> >
> > On Friday 11 April 2014 10:36 AM, Arjun wrote:
> >
> >> yup i will change the value and recheck. Thanks for the help.
> >>
> >> thanks
> >> Arjun Narasimha Kota
> >>
> >> On Friday 11 April 2014 10:28 AM, Guozhang Wang wrote:
> >>
> >>> What I tried to say is that it may be caused by your
> >>> "fetch.wait.max.ms"="180000"
> >>> too large. Try a small value and see if that helps.
> >>>
> >>>
> >>> On Thu, Apr 10, 2014 at 9:44 PM, Arjun <ar...@socialtwist.com> wrote:
> >>>
> >>>  Hi,
> >>>>
> >>>> I could not see any out of memory exceptions in the broker logs. One
> >>>> thing
> >>>> i can see is i  may have configured consumer poorly. If its not too
> >>>> much to
> >>>> ask can u let me know the changes i have to do for over coming this
> >>>> problem.
> >>>>
> >>>> Thanks
> >>>> Arjun Narasimha Kota
> >>>>
> >>>>
> >>>> On Friday 11 April 2014 10:04 AM, Guozhang Wang wrote:
> >>>>
> >>>>  Hi Ajrun,
> >>>>>
> >>>>> It seems to be the cause:
> >>>>>
> >>>>> https://issues.apache.org/jira/browse/KAFKA-1016
> >>>>>
> >>>>> Guozhang
> >>>>>
> >>>>>
> >>>>>
> >>>>> On Thu, Apr 10, 2014 at 9:21 PM, Arjun <ar...@socialtwist.com>
> wrote:
> >>>>>
> >>>>>   I hope this one would give u  a better idea.
> >>>>>
> >>>>>> bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker --group
> >>>>>> group1
> >>>>>> --zkconnect zkhost:port --topic testtopic
> >>>>>> Group           Topic                          Pid Offset logSize
> >>>>>> Lag             Owner
> >>>>>> group1          testtopic    0   253             253 0
> >>>>>> group1_ip-xx-1397188061429-b5ff1205-0
> >>>>>> group1          testtopic    1   267             267 0
> >>>>>> group1_ip-xx-1397188061429-b5ff1205-0
> >>>>>> group1          testtopic    2   254             254 0
> >>>>>> group1_ip-xx-1397188061429-b5ff1205-0
> >>>>>> group1          testtopic    3   265             265 0
> >>>>>> group1_ip-xx-1397188061429-b5ff1205-0
> >>>>>> group1          testtopic    4   261             261 0
> >>>>>> group1_ip-xx-1397188061429-b5ff1205-1
> >>>>>> group1          testtopic    5   294             294 0
> >>>>>> group1_ip-xx-1397188061429-b5ff1205-1
> >>>>>> group1          testtopic    6   248             248 0
> >>>>>> group1_ip-xx-1397188061429-b5ff1205-1
> >>>>>> group1          testtopic    7   271             271 0
> >>>>>> group1_ip-xx-1397188061429-b5ff1205-1
> >>>>>> group1          testtopic    8   240             240 0
> >>>>>> group1_ip-xx-1397188061429-b5ff1205-2
> >>>>>> group1          testtopic    9   261             261 0
> >>>>>> group1_ip-xx-1397188061429-b5ff1205-2
> >>>>>> group1          testtopic    10  290             290 0
> >>>>>> group1_ip-xx-1397188061429-b5ff1205-2
> >>>>>> group1          testtopic    11  250             251 1
> >>>>>> group1_ip-xx-1397188061429-b5ff1205-2
> >>>>>>
> >>>>>> If you see the output, in the last line the lag is 1 for that
> >>>>>> partition.
> >>>>>> I
> >>>>>> just send one message. This topic is not new as you see there are
> lot
> >>>>>> of
> >>>>>> messages which have accumlated from yesterday. This one message will
> >>>>>> not
> >>>>>> be
> >>>>>> consumed by consumer what so ever. But if i send some 10 messages
> then
> >>>>>> all
> >>>>>> the messages are consumed.
> >>>>>>
> >>>>>> Please let me know if i have to change any consumer properties.
> >>>>>>
> >>>>>> My consumer properties are :
> >>>>>> "fetch.wait.max.ms"="180000"
> >>>>>> "fetch.min.bytes" = "1"
> >>>>>> "auto.offset.reset" = "smallest"
> >>>>>> "auto.commit.enable"=  "false"
> >>>>>> "fetch.message.max.bytes" = "1048576"
> >>>>>>
> >>>>>>
> >>>>>> Thanks
> >>>>>> Arjun Narasimha Kota
> >>>>>> On Friday 11 April 2014 06:23 AM, Arjun Kota wrote:
> >>>>>>
> >>>>>>   The consumer uses do specific topics.
> >>>>>>
> >>>>>>> On Apr 11, 2014 6:23 AM, "Arjun Kota" <arjun@socialtwist.com
> <mailto:
> >>>>>>> arjun@socialtwist.com>> wrote:
> >>>>>>>
> >>>>>>>       Yes the message shows up on the server.
> >>>>>>>
> >>>>>>>       On Apr 11, 2014 12:07 AM, "Guozhang Wang" <
> wangguoz@gmail.com
> >>>>>>>       <ma...@gmail.com>> wrote:
> >>>>>>>
> >>>>>>>           Hi Arjun,
> >>>>>>>
> >>>>>>>           If you only send one message, does that message show up
> on
> >>>>>>> the
> >>>>>>>           server? Does
> >>>>>>>           you consumer use wildcard topics or specific topics?
> >>>>>>>
> >>>>>>>           Guozhang
> >>>>>>>
> >>>>>>>
> >>>>>>>           On Thu, Apr 10, 2014 at 9:20 AM, Arjun <
> >>>>>>> arjun@socialtwist.com
> >>>>>>>           <ma...@socialtwist.com>> wrote:
> >>>>>>>
> >>>>>>>           > But  we have auto offset reset to smallest not largest,
> >>>>>>> even
> >>>>>>>           then this
> >>>>>>>           > issue arises? If so is there any work around?
> >>>>>>>           >
> >>>>>>>           > Thanks
> >>>>>>>           > Arjun NArasimha Kota
> >>>>>>>           >
> >>>>>>>           >
> >>>>>>>           > On Thursday 10 April 2014 09:39 PM, Guozhang Wang
> wrote:
> >>>>>>>           >
> >>>>>>>           >> It could be https://issues.apache.org/
> >>>>>>> jira/browse/KAFKA-1006.
> >>>>>>>           >>
> >>>>>>>           >> Guozhang
> >>>>>>>           >>
> >>>>>>>           >>
> >>>>>>>           >> On Thu, Apr 10, 2014 at 8:50 AM, Arjun
> >>>>>>>           <arjun@socialtwist.com <ma...@socialtwist.com>>
> >>>>>>> wrote:
> >>>>>>>           >>
> >>>>>>>           >>  its auto created
> >>>>>>>           >>> but even after topic creation this is the scenario
> >>>>>>>           >>>
> >>>>>>>           >>> Arjun
> >>>>>>>           >>>
> >>>>>>>           >>> On Thursday 10 April 2014 08:41 PM, Guozhang Wang
> >>>>>>> wrote:
> >>>>>>>           >>>
> >>>>>>>           >>>  Hi Arjun,
> >>>>>>>           >>>>
> >>>>>>>           >>>> Did you manually create the topic or use
> >>>>>>> auto.topic.creation?
> >>>>>>>           >>>>
> >>>>>>>           >>>> Guozhang
> >>>>>>>           >>>>
> >>>>>>>           >>>>
> >>>>>>>           >>>> On Thu, Apr 10, 2014 at 7:39 AM, Arjun
> >>>>>>>           <arjun@socialtwist.com <ma...@socialtwist.com>>
> >>>>>>> wrote:
> >>>>>>>           >>>>
> >>>>>>>           >>>>   Hi,
> >>>>>>>           >>>>
> >>>>>>>           >>>>> We have 3 node kafka 0.8 setup with zookeepers
> >>>>>>> ensemble.
> >>>>>>>           We use high
> >>>>>>>           >>>>> level
> >>>>>>>           >>>>> consumer with auto commit offset false. I am facing
> >>>>>>> some
> >>>>>>>           peculiar
> >>>>>>>           >>>>> problem
> >>>>>>>           >>>>> with kafka. When i send some 10-20 messages or so
> the
> >>>>>>>           consumer starts
> >>>>>>>           >>>>> to
> >>>>>>>           >>>>> consume the messages. But if  i send only one
> >>>>>>> message to
> >>>>>>>           kafka, then
> >>>>>>>           >>>>> even
> >>>>>>>           >>>>> though consumer is active it is not trying to fetch
> >>>>>>> the
> >>>>>>>           message. There
> >>>>>>>           >>>>> is
> >>>>>>>           >>>>> nothing in logs, just the messages are being
> fetched
> >>>>>>> by
> >>>>>>>           the kafka
> >>>>>>>           >>>>> consumer.
> >>>>>>>           >>>>> The messages are there in the Kafka server. Can
> some
> >>>>>>> one
> >>>>>>>           let me know
> >>>>>>>           >>>>> where
> >>>>>>>           >>>>> i am doing wrong.
> >>>>>>>           >>>>>
> >>>>>>>           >>>>>
> >>>>>>>           >>>>> Thanks
> >>>>>>>           >>>>> Arjun Narasimha Kota
> >>>>>>>           >>>>>
> >>>>>>>           >>>>>
> >>>>>>>           >>>>>
> >>>>>>>           >>>>
> >>>>>>>           >>
> >>>>>>>           >
> >>>>>>>
> >>>>>>>
> >>>>>>>           --
> >>>>>>>           -- Guozhang
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>
> >>
> >
>

Re: consumer not consuming messages

Posted by Jun Rao <ju...@gmail.com>.
We may have a bug that doesn't observe etch.min.bytes accurately. So a
lower fetch.wait.max.ms will improve consumer latency.

Could you run a console consumer and see if you have the same issue? That
will tell us if this is a server side issue or an issue just in your
consumer.

Thanks,

Jun


On Thu, Apr 10, 2014 at 10:28 PM, Arjun <ar...@socialtwist.com> wrote:

> i changed the time to 60 seconds even now i see the same result. The
> Consumer is not consuming the messages.
>
> Thanks
> Arjun Narasimha Kota
>
>
> On Friday 11 April 2014 10:36 AM, Arjun wrote:
>
>> yup i will change the value and recheck. Thanks for the help.
>>
>> thanks
>> Arjun Narasimha Kota
>>
>> On Friday 11 April 2014 10:28 AM, Guozhang Wang wrote:
>>
>>> What I tried to say is that it may be caused by your
>>> "fetch.wait.max.ms"="180000"
>>> too large. Try a small value and see if that helps.
>>>
>>>
>>> On Thu, Apr 10, 2014 at 9:44 PM, Arjun <ar...@socialtwist.com> wrote:
>>>
>>>  Hi,
>>>>
>>>> I could not see any out of memory exceptions in the broker logs. One
>>>> thing
>>>> i can see is i  may have configured consumer poorly. If its not too
>>>> much to
>>>> ask can u let me know the changes i have to do for over coming this
>>>> problem.
>>>>
>>>> Thanks
>>>> Arjun Narasimha Kota
>>>>
>>>>
>>>> On Friday 11 April 2014 10:04 AM, Guozhang Wang wrote:
>>>>
>>>>  Hi Ajrun,
>>>>>
>>>>> It seems to be the cause:
>>>>>
>>>>> https://issues.apache.org/jira/browse/KAFKA-1016
>>>>>
>>>>> Guozhang
>>>>>
>>>>>
>>>>>
>>>>> On Thu, Apr 10, 2014 at 9:21 PM, Arjun <ar...@socialtwist.com> wrote:
>>>>>
>>>>>   I hope this one would give u  a better idea.
>>>>>
>>>>>> bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker --group
>>>>>> group1
>>>>>> --zkconnect zkhost:port --topic testtopic
>>>>>> Group           Topic                          Pid Offset logSize
>>>>>> Lag             Owner
>>>>>> group1          testtopic    0   253             253 0
>>>>>> group1_ip-xx-1397188061429-b5ff1205-0
>>>>>> group1          testtopic    1   267             267 0
>>>>>> group1_ip-xx-1397188061429-b5ff1205-0
>>>>>> group1          testtopic    2   254             254 0
>>>>>> group1_ip-xx-1397188061429-b5ff1205-0
>>>>>> group1          testtopic    3   265             265 0
>>>>>> group1_ip-xx-1397188061429-b5ff1205-0
>>>>>> group1          testtopic    4   261             261 0
>>>>>> group1_ip-xx-1397188061429-b5ff1205-1
>>>>>> group1          testtopic    5   294             294 0
>>>>>> group1_ip-xx-1397188061429-b5ff1205-1
>>>>>> group1          testtopic    6   248             248 0
>>>>>> group1_ip-xx-1397188061429-b5ff1205-1
>>>>>> group1          testtopic    7   271             271 0
>>>>>> group1_ip-xx-1397188061429-b5ff1205-1
>>>>>> group1          testtopic    8   240             240 0
>>>>>> group1_ip-xx-1397188061429-b5ff1205-2
>>>>>> group1          testtopic    9   261             261 0
>>>>>> group1_ip-xx-1397188061429-b5ff1205-2
>>>>>> group1          testtopic    10  290             290 0
>>>>>> group1_ip-xx-1397188061429-b5ff1205-2
>>>>>> group1          testtopic    11  250             251 1
>>>>>> group1_ip-xx-1397188061429-b5ff1205-2
>>>>>>
>>>>>> If you see the output, in the last line the lag is 1 for that
>>>>>> partition.
>>>>>> I
>>>>>> just send one message. This topic is not new as you see there are lot
>>>>>> of
>>>>>> messages which have accumlated from yesterday. This one message will
>>>>>> not
>>>>>> be
>>>>>> consumed by consumer what so ever. But if i send some 10 messages then
>>>>>> all
>>>>>> the messages are consumed.
>>>>>>
>>>>>> Please let me know if i have to change any consumer properties.
>>>>>>
>>>>>> My consumer properties are :
>>>>>> "fetch.wait.max.ms"="180000"
>>>>>> "fetch.min.bytes" = "1"
>>>>>> "auto.offset.reset" = "smallest"
>>>>>> "auto.commit.enable"=  "false"
>>>>>> "fetch.message.max.bytes" = "1048576"
>>>>>>
>>>>>>
>>>>>> Thanks
>>>>>> Arjun Narasimha Kota
>>>>>> On Friday 11 April 2014 06:23 AM, Arjun Kota wrote:
>>>>>>
>>>>>>   The consumer uses do specific topics.
>>>>>>
>>>>>>> On Apr 11, 2014 6:23 AM, "Arjun Kota" <arjun@socialtwist.com<mailto:
>>>>>>> arjun@socialtwist.com>> wrote:
>>>>>>>
>>>>>>>       Yes the message shows up on the server.
>>>>>>>
>>>>>>>       On Apr 11, 2014 12:07 AM, "Guozhang Wang" <wangguoz@gmail.com
>>>>>>>       <ma...@gmail.com>> wrote:
>>>>>>>
>>>>>>>           Hi Arjun,
>>>>>>>
>>>>>>>           If you only send one message, does that message show up on
>>>>>>> the
>>>>>>>           server? Does
>>>>>>>           you consumer use wildcard topics or specific topics?
>>>>>>>
>>>>>>>           Guozhang
>>>>>>>
>>>>>>>
>>>>>>>           On Thu, Apr 10, 2014 at 9:20 AM, Arjun <
>>>>>>> arjun@socialtwist.com
>>>>>>>           <ma...@socialtwist.com>> wrote:
>>>>>>>
>>>>>>>           > But  we have auto offset reset to smallest not largest,
>>>>>>> even
>>>>>>>           then this
>>>>>>>           > issue arises? If so is there any work around?
>>>>>>>           >
>>>>>>>           > Thanks
>>>>>>>           > Arjun NArasimha Kota
>>>>>>>           >
>>>>>>>           >
>>>>>>>           > On Thursday 10 April 2014 09:39 PM, Guozhang Wang wrote:
>>>>>>>           >
>>>>>>>           >> It could be https://issues.apache.org/
>>>>>>> jira/browse/KAFKA-1006.
>>>>>>>           >>
>>>>>>>           >> Guozhang
>>>>>>>           >>
>>>>>>>           >>
>>>>>>>           >> On Thu, Apr 10, 2014 at 8:50 AM, Arjun
>>>>>>>           <arjun@socialtwist.com <ma...@socialtwist.com>>
>>>>>>> wrote:
>>>>>>>           >>
>>>>>>>           >>  its auto created
>>>>>>>           >>> but even after topic creation this is the scenario
>>>>>>>           >>>
>>>>>>>           >>> Arjun
>>>>>>>           >>>
>>>>>>>           >>> On Thursday 10 April 2014 08:41 PM, Guozhang Wang
>>>>>>> wrote:
>>>>>>>           >>>
>>>>>>>           >>>  Hi Arjun,
>>>>>>>           >>>>
>>>>>>>           >>>> Did you manually create the topic or use
>>>>>>> auto.topic.creation?
>>>>>>>           >>>>
>>>>>>>           >>>> Guozhang
>>>>>>>           >>>>
>>>>>>>           >>>>
>>>>>>>           >>>> On Thu, Apr 10, 2014 at 7:39 AM, Arjun
>>>>>>>           <arjun@socialtwist.com <ma...@socialtwist.com>>
>>>>>>> wrote:
>>>>>>>           >>>>
>>>>>>>           >>>>   Hi,
>>>>>>>           >>>>
>>>>>>>           >>>>> We have 3 node kafka 0.8 setup with zookeepers
>>>>>>> ensemble.
>>>>>>>           We use high
>>>>>>>           >>>>> level
>>>>>>>           >>>>> consumer with auto commit offset false. I am facing
>>>>>>> some
>>>>>>>           peculiar
>>>>>>>           >>>>> problem
>>>>>>>           >>>>> with kafka. When i send some 10-20 messages or so the
>>>>>>>           consumer starts
>>>>>>>           >>>>> to
>>>>>>>           >>>>> consume the messages. But if  i send only one
>>>>>>> message to
>>>>>>>           kafka, then
>>>>>>>           >>>>> even
>>>>>>>           >>>>> though consumer is active it is not trying to fetch
>>>>>>> the
>>>>>>>           message. There
>>>>>>>           >>>>> is
>>>>>>>           >>>>> nothing in logs, just the messages are being fetched
>>>>>>> by
>>>>>>>           the kafka
>>>>>>>           >>>>> consumer.
>>>>>>>           >>>>> The messages are there in the Kafka server. Can some
>>>>>>> one
>>>>>>>           let me know
>>>>>>>           >>>>> where
>>>>>>>           >>>>> i am doing wrong.
>>>>>>>           >>>>>
>>>>>>>           >>>>>
>>>>>>>           >>>>> Thanks
>>>>>>>           >>>>> Arjun Narasimha Kota
>>>>>>>           >>>>>
>>>>>>>           >>>>>
>>>>>>>           >>>>>
>>>>>>>           >>>>
>>>>>>>           >>
>>>>>>>           >
>>>>>>>
>>>>>>>
>>>>>>>           --
>>>>>>>           -- Guozhang
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>
>>
>

Re: consumer not consuming messages

Posted by Arjun <ar...@socialtwist.com>.
i changed the time to 60 seconds even now i see the same result. The 
Consumer is not consuming the messages.

Thanks
Arjun Narasimha Kota

On Friday 11 April 2014 10:36 AM, Arjun wrote:
> yup i will change the value and recheck. Thanks for the help.
>
> thanks
> Arjun Narasimha Kota
>
> On Friday 11 April 2014 10:28 AM, Guozhang Wang wrote:
>> What I tried to say is that it may be caused by your
>> "fetch.wait.max.ms"="180000"
>> too large. Try a small value and see if that helps.
>>
>>
>> On Thu, Apr 10, 2014 at 9:44 PM, Arjun <ar...@socialtwist.com> wrote:
>>
>>> Hi,
>>>
>>> I could not see any out of memory exceptions in the broker logs. One 
>>> thing
>>> i can see is i  may have configured consumer poorly. If its not too 
>>> much to
>>> ask can u let me know the changes i have to do for over coming this 
>>> problem.
>>>
>>> Thanks
>>> Arjun Narasimha Kota
>>>
>>>
>>> On Friday 11 April 2014 10:04 AM, Guozhang Wang wrote:
>>>
>>>> Hi Ajrun,
>>>>
>>>> It seems to be the cause:
>>>>
>>>> https://issues.apache.org/jira/browse/KAFKA-1016
>>>>
>>>> Guozhang
>>>>
>>>>
>>>>
>>>> On Thu, Apr 10, 2014 at 9:21 PM, Arjun <ar...@socialtwist.com> wrote:
>>>>
>>>>   I hope this one would give u  a better idea.
>>>>> bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker --group 
>>>>> group1
>>>>> --zkconnect zkhost:port --topic testtopic
>>>>> Group           Topic                          Pid Offset logSize
>>>>> Lag             Owner
>>>>> group1          testtopic    0   253             253 0
>>>>> group1_ip-xx-1397188061429-b5ff1205-0
>>>>> group1          testtopic    1   267             267 0
>>>>> group1_ip-xx-1397188061429-b5ff1205-0
>>>>> group1          testtopic    2   254             254 0
>>>>> group1_ip-xx-1397188061429-b5ff1205-0
>>>>> group1          testtopic    3   265             265 0
>>>>> group1_ip-xx-1397188061429-b5ff1205-0
>>>>> group1          testtopic    4   261             261 0
>>>>> group1_ip-xx-1397188061429-b5ff1205-1
>>>>> group1          testtopic    5   294             294 0
>>>>> group1_ip-xx-1397188061429-b5ff1205-1
>>>>> group1          testtopic    6   248             248 0
>>>>> group1_ip-xx-1397188061429-b5ff1205-1
>>>>> group1          testtopic    7   271             271 0
>>>>> group1_ip-xx-1397188061429-b5ff1205-1
>>>>> group1          testtopic    8   240             240 0
>>>>> group1_ip-xx-1397188061429-b5ff1205-2
>>>>> group1          testtopic    9   261             261 0
>>>>> group1_ip-xx-1397188061429-b5ff1205-2
>>>>> group1          testtopic    10  290             290 0
>>>>> group1_ip-xx-1397188061429-b5ff1205-2
>>>>> group1          testtopic    11  250             251 1
>>>>> group1_ip-xx-1397188061429-b5ff1205-2
>>>>>
>>>>> If you see the output, in the last line the lag is 1 for that 
>>>>> partition.
>>>>> I
>>>>> just send one message. This topic is not new as you see there are 
>>>>> lot of
>>>>> messages which have accumlated from yesterday. This one message 
>>>>> will not
>>>>> be
>>>>> consumed by consumer what so ever. But if i send some 10 messages 
>>>>> then
>>>>> all
>>>>> the messages are consumed.
>>>>>
>>>>> Please let me know if i have to change any consumer properties.
>>>>>
>>>>> My consumer properties are :
>>>>> "fetch.wait.max.ms"="180000"
>>>>> "fetch.min.bytes" = "1"
>>>>> "auto.offset.reset" = "smallest"
>>>>> "auto.commit.enable"=  "false"
>>>>> "fetch.message.max.bytes" = "1048576"
>>>>>
>>>>>
>>>>> Thanks
>>>>> Arjun Narasimha Kota
>>>>> On Friday 11 April 2014 06:23 AM, Arjun Kota wrote:
>>>>>
>>>>>   The consumer uses do specific topics.
>>>>>> On Apr 11, 2014 6:23 AM, "Arjun Kota" <arjun@socialtwist.com 
>>>>>> <mailto:
>>>>>> arjun@socialtwist.com>> wrote:
>>>>>>
>>>>>>       Yes the message shows up on the server.
>>>>>>
>>>>>>       On Apr 11, 2014 12:07 AM, "Guozhang Wang" <wangguoz@gmail.com
>>>>>>       <ma...@gmail.com>> wrote:
>>>>>>
>>>>>>           Hi Arjun,
>>>>>>
>>>>>>           If you only send one message, does that message show up 
>>>>>> on the
>>>>>>           server? Does
>>>>>>           you consumer use wildcard topics or specific topics?
>>>>>>
>>>>>>           Guozhang
>>>>>>
>>>>>>
>>>>>>           On Thu, Apr 10, 2014 at 9:20 AM, Arjun 
>>>>>> <arjun@socialtwist.com
>>>>>>           <ma...@socialtwist.com>> wrote:
>>>>>>
>>>>>>           > But  we have auto offset reset to smallest not 
>>>>>> largest, even
>>>>>>           then this
>>>>>>           > issue arises? If so is there any work around?
>>>>>>           >
>>>>>>           > Thanks
>>>>>>           > Arjun NArasimha Kota
>>>>>>           >
>>>>>>           >
>>>>>>           > On Thursday 10 April 2014 09:39 PM, Guozhang Wang wrote:
>>>>>>           >
>>>>>>           >> It could be https://issues.apache.org/
>>>>>> jira/browse/KAFKA-1006.
>>>>>>           >>
>>>>>>           >> Guozhang
>>>>>>           >>
>>>>>>           >>
>>>>>>           >> On Thu, Apr 10, 2014 at 8:50 AM, Arjun
>>>>>>           <arjun@socialtwist.com <ma...@socialtwist.com>> 
>>>>>> wrote:
>>>>>>           >>
>>>>>>           >>  its auto created
>>>>>>           >>> but even after topic creation this is the scenario
>>>>>>           >>>
>>>>>>           >>> Arjun
>>>>>>           >>>
>>>>>>           >>> On Thursday 10 April 2014 08:41 PM, Guozhang Wang 
>>>>>> wrote:
>>>>>>           >>>
>>>>>>           >>>  Hi Arjun,
>>>>>>           >>>>
>>>>>>           >>>> Did you manually create the topic or use
>>>>>> auto.topic.creation?
>>>>>>           >>>>
>>>>>>           >>>> Guozhang
>>>>>>           >>>>
>>>>>>           >>>>
>>>>>>           >>>> On Thu, Apr 10, 2014 at 7:39 AM, Arjun
>>>>>>           <arjun@socialtwist.com <ma...@socialtwist.com>> 
>>>>>> wrote:
>>>>>>           >>>>
>>>>>>           >>>>   Hi,
>>>>>>           >>>>
>>>>>>           >>>>> We have 3 node kafka 0.8 setup with zookeepers 
>>>>>> ensemble.
>>>>>>           We use high
>>>>>>           >>>>> level
>>>>>>           >>>>> consumer with auto commit offset false. I am 
>>>>>> facing some
>>>>>>           peculiar
>>>>>>           >>>>> problem
>>>>>>           >>>>> with kafka. When i send some 10-20 messages or so 
>>>>>> the
>>>>>>           consumer starts
>>>>>>           >>>>> to
>>>>>>           >>>>> consume the messages. But if  i send only one 
>>>>>> message to
>>>>>>           kafka, then
>>>>>>           >>>>> even
>>>>>>           >>>>> though consumer is active it is not trying to 
>>>>>> fetch the
>>>>>>           message. There
>>>>>>           >>>>> is
>>>>>>           >>>>> nothing in logs, just the messages are being 
>>>>>> fetched by
>>>>>>           the kafka
>>>>>>           >>>>> consumer.
>>>>>>           >>>>> The messages are there in the Kafka server. Can 
>>>>>> some one
>>>>>>           let me know
>>>>>>           >>>>> where
>>>>>>           >>>>> i am doing wrong.
>>>>>>           >>>>>
>>>>>>           >>>>>
>>>>>>           >>>>> Thanks
>>>>>>           >>>>> Arjun Narasimha Kota
>>>>>>           >>>>>
>>>>>>           >>>>>
>>>>>>           >>>>>
>>>>>>           >>>>
>>>>>>           >>
>>>>>>           >
>>>>>>
>>>>>>
>>>>>>           --
>>>>>>           -- Guozhang
>>>>>>
>>>>>>
>>>>>>
>>
>


Re: consumer not consuming messages

Posted by Arjun <ar...@socialtwist.com>.
yup i will change the value and recheck. Thanks for the help.

thanks
Arjun Narasimha Kota

On Friday 11 April 2014 10:28 AM, Guozhang Wang wrote:
> What I tried to say is that it may be caused by your
> "fetch.wait.max.ms"="180000"
> too large. Try a small value and see if that helps.
>
>
> On Thu, Apr 10, 2014 at 9:44 PM, Arjun <ar...@socialtwist.com> wrote:
>
>> Hi,
>>
>> I could not see any out of memory exceptions in the broker logs. One thing
>> i can see is i  may have configured consumer poorly. If its not too much to
>> ask can u let me know the changes i have to do for over coming this problem.
>>
>> Thanks
>> Arjun Narasimha Kota
>>
>>
>> On Friday 11 April 2014 10:04 AM, Guozhang Wang wrote:
>>
>>> Hi Ajrun,
>>>
>>> It seems to be the cause:
>>>
>>> https://issues.apache.org/jira/browse/KAFKA-1016
>>>
>>> Guozhang
>>>
>>>
>>>
>>> On Thu, Apr 10, 2014 at 9:21 PM, Arjun <ar...@socialtwist.com> wrote:
>>>
>>>   I hope this one would give u  a better idea.
>>>> bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker --group group1
>>>> --zkconnect zkhost:port --topic testtopic
>>>> Group           Topic                          Pid Offset logSize
>>>> Lag             Owner
>>>> group1          testtopic    0   253             253 0
>>>> group1_ip-xx-1397188061429-b5ff1205-0
>>>> group1          testtopic    1   267             267 0
>>>> group1_ip-xx-1397188061429-b5ff1205-0
>>>> group1          testtopic    2   254             254 0
>>>> group1_ip-xx-1397188061429-b5ff1205-0
>>>> group1          testtopic    3   265             265 0
>>>> group1_ip-xx-1397188061429-b5ff1205-0
>>>> group1          testtopic    4   261             261 0
>>>> group1_ip-xx-1397188061429-b5ff1205-1
>>>> group1          testtopic    5   294             294 0
>>>> group1_ip-xx-1397188061429-b5ff1205-1
>>>> group1          testtopic    6   248             248 0
>>>> group1_ip-xx-1397188061429-b5ff1205-1
>>>> group1          testtopic    7   271             271 0
>>>> group1_ip-xx-1397188061429-b5ff1205-1
>>>> group1          testtopic    8   240             240 0
>>>> group1_ip-xx-1397188061429-b5ff1205-2
>>>> group1          testtopic    9   261             261 0
>>>> group1_ip-xx-1397188061429-b5ff1205-2
>>>> group1          testtopic    10  290             290 0
>>>> group1_ip-xx-1397188061429-b5ff1205-2
>>>> group1          testtopic    11  250             251 1
>>>> group1_ip-xx-1397188061429-b5ff1205-2
>>>>
>>>> If you see the output, in the last line the lag is 1 for that partition.
>>>> I
>>>> just send one message. This topic is not new as you see there are lot of
>>>> messages which have accumlated from yesterday. This one message will not
>>>> be
>>>> consumed by consumer what so ever. But if i send some 10 messages then
>>>> all
>>>> the messages are consumed.
>>>>
>>>> Please let me know if i have to change any consumer properties.
>>>>
>>>> My consumer properties are :
>>>> "fetch.wait.max.ms"="180000"
>>>> "fetch.min.bytes" = "1"
>>>> "auto.offset.reset" = "smallest"
>>>> "auto.commit.enable"=  "false"
>>>> "fetch.message.max.bytes" = "1048576"
>>>>
>>>>
>>>> Thanks
>>>> Arjun Narasimha Kota
>>>> On Friday 11 April 2014 06:23 AM, Arjun Kota wrote:
>>>>
>>>>   The consumer uses do specific topics.
>>>>> On Apr 11, 2014 6:23 AM, "Arjun Kota" <arjun@socialtwist.com <mailto:
>>>>> arjun@socialtwist.com>> wrote:
>>>>>
>>>>>       Yes the message shows up on the server.
>>>>>
>>>>>       On Apr 11, 2014 12:07 AM, "Guozhang Wang" <wangguoz@gmail.com
>>>>>       <ma...@gmail.com>> wrote:
>>>>>
>>>>>           Hi Arjun,
>>>>>
>>>>>           If you only send one message, does that message show up on the
>>>>>           server? Does
>>>>>           you consumer use wildcard topics or specific topics?
>>>>>
>>>>>           Guozhang
>>>>>
>>>>>
>>>>>           On Thu, Apr 10, 2014 at 9:20 AM, Arjun <arjun@socialtwist.com
>>>>>           <ma...@socialtwist.com>> wrote:
>>>>>
>>>>>           > But  we have auto offset reset to smallest not largest, even
>>>>>           then this
>>>>>           > issue arises? If so is there any work around?
>>>>>           >
>>>>>           > Thanks
>>>>>           > Arjun NArasimha Kota
>>>>>           >
>>>>>           >
>>>>>           > On Thursday 10 April 2014 09:39 PM, Guozhang Wang wrote:
>>>>>           >
>>>>>           >> It could be https://issues.apache.org/
>>>>> jira/browse/KAFKA-1006.
>>>>>           >>
>>>>>           >> Guozhang
>>>>>           >>
>>>>>           >>
>>>>>           >> On Thu, Apr 10, 2014 at 8:50 AM, Arjun
>>>>>           <arjun@socialtwist.com <ma...@socialtwist.com>> wrote:
>>>>>           >>
>>>>>           >>  its auto created
>>>>>           >>> but even after topic creation this is the scenario
>>>>>           >>>
>>>>>           >>> Arjun
>>>>>           >>>
>>>>>           >>> On Thursday 10 April 2014 08:41 PM, Guozhang Wang wrote:
>>>>>           >>>
>>>>>           >>>  Hi Arjun,
>>>>>           >>>>
>>>>>           >>>> Did you manually create the topic or use
>>>>> auto.topic.creation?
>>>>>           >>>>
>>>>>           >>>> Guozhang
>>>>>           >>>>
>>>>>           >>>>
>>>>>           >>>> On Thu, Apr 10, 2014 at 7:39 AM, Arjun
>>>>>           <arjun@socialtwist.com <ma...@socialtwist.com>> wrote:
>>>>>           >>>>
>>>>>           >>>>   Hi,
>>>>>           >>>>
>>>>>           >>>>> We have 3 node kafka 0.8 setup with zookeepers ensemble.
>>>>>           We use high
>>>>>           >>>>> level
>>>>>           >>>>> consumer with auto commit offset false. I am facing some
>>>>>           peculiar
>>>>>           >>>>> problem
>>>>>           >>>>> with kafka. When i send some 10-20 messages or so the
>>>>>           consumer starts
>>>>>           >>>>> to
>>>>>           >>>>> consume the messages. But if  i send only one message to
>>>>>           kafka, then
>>>>>           >>>>> even
>>>>>           >>>>> though consumer is active it is not trying to fetch the
>>>>>           message. There
>>>>>           >>>>> is
>>>>>           >>>>> nothing in logs, just the messages are being fetched by
>>>>>           the kafka
>>>>>           >>>>> consumer.
>>>>>           >>>>> The messages are there in the Kafka server. Can some one
>>>>>           let me know
>>>>>           >>>>> where
>>>>>           >>>>> i am doing wrong.
>>>>>           >>>>>
>>>>>           >>>>>
>>>>>           >>>>> Thanks
>>>>>           >>>>> Arjun Narasimha Kota
>>>>>           >>>>>
>>>>>           >>>>>
>>>>>           >>>>>
>>>>>           >>>>
>>>>>           >>
>>>>>           >
>>>>>
>>>>>
>>>>>           --
>>>>>           -- Guozhang
>>>>>
>>>>>
>>>>>
>


Re: consumer not consuming messages

Posted by Guozhang Wang <wa...@gmail.com>.
What I tried to say is that it may be caused by your
"fetch.wait.max.ms"="180000"
too large. Try a small value and see if that helps.


On Thu, Apr 10, 2014 at 9:44 PM, Arjun <ar...@socialtwist.com> wrote:

> Hi,
>
> I could not see any out of memory exceptions in the broker logs. One thing
> i can see is i  may have configured consumer poorly. If its not too much to
> ask can u let me know the changes i have to do for over coming this problem.
>
> Thanks
> Arjun Narasimha Kota
>
>
> On Friday 11 April 2014 10:04 AM, Guozhang Wang wrote:
>
>> Hi Ajrun,
>>
>> It seems to be the cause:
>>
>> https://issues.apache.org/jira/browse/KAFKA-1016
>>
>> Guozhang
>>
>>
>>
>> On Thu, Apr 10, 2014 at 9:21 PM, Arjun <ar...@socialtwist.com> wrote:
>>
>>  I hope this one would give u  a better idea.
>>>
>>> bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker --group group1
>>> --zkconnect zkhost:port --topic testtopic
>>> Group           Topic                          Pid Offset logSize
>>> Lag             Owner
>>> group1          testtopic    0   253             253 0
>>> group1_ip-xx-1397188061429-b5ff1205-0
>>> group1          testtopic    1   267             267 0
>>> group1_ip-xx-1397188061429-b5ff1205-0
>>> group1          testtopic    2   254             254 0
>>> group1_ip-xx-1397188061429-b5ff1205-0
>>> group1          testtopic    3   265             265 0
>>> group1_ip-xx-1397188061429-b5ff1205-0
>>> group1          testtopic    4   261             261 0
>>> group1_ip-xx-1397188061429-b5ff1205-1
>>> group1          testtopic    5   294             294 0
>>> group1_ip-xx-1397188061429-b5ff1205-1
>>> group1          testtopic    6   248             248 0
>>> group1_ip-xx-1397188061429-b5ff1205-1
>>> group1          testtopic    7   271             271 0
>>> group1_ip-xx-1397188061429-b5ff1205-1
>>> group1          testtopic    8   240             240 0
>>> group1_ip-xx-1397188061429-b5ff1205-2
>>> group1          testtopic    9   261             261 0
>>> group1_ip-xx-1397188061429-b5ff1205-2
>>> group1          testtopic    10  290             290 0
>>> group1_ip-xx-1397188061429-b5ff1205-2
>>> group1          testtopic    11  250             251 1
>>> group1_ip-xx-1397188061429-b5ff1205-2
>>>
>>> If you see the output, in the last line the lag is 1 for that partition.
>>> I
>>> just send one message. This topic is not new as you see there are lot of
>>> messages which have accumlated from yesterday. This one message will not
>>> be
>>> consumed by consumer what so ever. But if i send some 10 messages then
>>> all
>>> the messages are consumed.
>>>
>>> Please let me know if i have to change any consumer properties.
>>>
>>> My consumer properties are :
>>> "fetch.wait.max.ms"="180000"
>>> "fetch.min.bytes" = "1"
>>> "auto.offset.reset" = "smallest"
>>> "auto.commit.enable"=  "false"
>>> "fetch.message.max.bytes" = "1048576"
>>>
>>>
>>> Thanks
>>> Arjun Narasimha Kota
>>> On Friday 11 April 2014 06:23 AM, Arjun Kota wrote:
>>>
>>>  The consumer uses do specific topics.
>>>>
>>>> On Apr 11, 2014 6:23 AM, "Arjun Kota" <arjun@socialtwist.com <mailto:
>>>> arjun@socialtwist.com>> wrote:
>>>>
>>>>      Yes the message shows up on the server.
>>>>
>>>>      On Apr 11, 2014 12:07 AM, "Guozhang Wang" <wangguoz@gmail.com
>>>>      <ma...@gmail.com>> wrote:
>>>>
>>>>          Hi Arjun,
>>>>
>>>>          If you only send one message, does that message show up on the
>>>>          server? Does
>>>>          you consumer use wildcard topics or specific topics?
>>>>
>>>>          Guozhang
>>>>
>>>>
>>>>          On Thu, Apr 10, 2014 at 9:20 AM, Arjun <arjun@socialtwist.com
>>>>          <ma...@socialtwist.com>> wrote:
>>>>
>>>>          > But  we have auto offset reset to smallest not largest, even
>>>>          then this
>>>>          > issue arises? If so is there any work around?
>>>>          >
>>>>          > Thanks
>>>>          > Arjun NArasimha Kota
>>>>          >
>>>>          >
>>>>          > On Thursday 10 April 2014 09:39 PM, Guozhang Wang wrote:
>>>>          >
>>>>          >> It could be https://issues.apache.org/
>>>> jira/browse/KAFKA-1006.
>>>>          >>
>>>>          >> Guozhang
>>>>          >>
>>>>          >>
>>>>          >> On Thu, Apr 10, 2014 at 8:50 AM, Arjun
>>>>          <arjun@socialtwist.com <ma...@socialtwist.com>> wrote:
>>>>          >>
>>>>          >>  its auto created
>>>>          >>> but even after topic creation this is the scenario
>>>>          >>>
>>>>          >>> Arjun
>>>>          >>>
>>>>          >>> On Thursday 10 April 2014 08:41 PM, Guozhang Wang wrote:
>>>>          >>>
>>>>          >>>  Hi Arjun,
>>>>          >>>>
>>>>          >>>> Did you manually create the topic or use
>>>> auto.topic.creation?
>>>>          >>>>
>>>>          >>>> Guozhang
>>>>          >>>>
>>>>          >>>>
>>>>          >>>> On Thu, Apr 10, 2014 at 7:39 AM, Arjun
>>>>          <arjun@socialtwist.com <ma...@socialtwist.com>> wrote:
>>>>          >>>>
>>>>          >>>>   Hi,
>>>>          >>>>
>>>>          >>>>> We have 3 node kafka 0.8 setup with zookeepers ensemble.
>>>>          We use high
>>>>          >>>>> level
>>>>          >>>>> consumer with auto commit offset false. I am facing some
>>>>          peculiar
>>>>          >>>>> problem
>>>>          >>>>> with kafka. When i send some 10-20 messages or so the
>>>>          consumer starts
>>>>          >>>>> to
>>>>          >>>>> consume the messages. But if  i send only one message to
>>>>          kafka, then
>>>>          >>>>> even
>>>>          >>>>> though consumer is active it is not trying to fetch the
>>>>          message. There
>>>>          >>>>> is
>>>>          >>>>> nothing in logs, just the messages are being fetched by
>>>>          the kafka
>>>>          >>>>> consumer.
>>>>          >>>>> The messages are there in the Kafka server. Can some one
>>>>          let me know
>>>>          >>>>> where
>>>>          >>>>> i am doing wrong.
>>>>          >>>>>
>>>>          >>>>>
>>>>          >>>>> Thanks
>>>>          >>>>> Arjun Narasimha Kota
>>>>          >>>>>
>>>>          >>>>>
>>>>          >>>>>
>>>>          >>>>
>>>>          >>
>>>>          >
>>>>
>>>>
>>>>          --
>>>>          -- Guozhang
>>>>
>>>>
>>>>
>>
>


-- 
-- Guozhang

Re: consumer not consuming messages

Posted by Arjun <ar...@socialtwist.com>.
Hi,

 From my understanding, the fetch wait max time is the maximum time the 
consumer waits if there are no messages in the broker. If there are 
messages in the broker, it just gets all the messages from the broker.Is 
my understanding wrong?

thanks
Arjun Narasimha Kota

On Friday 11 April 2014 10:14 AM, Arjun wrote:
> Hi,
>
> I could not see any out of memory exceptions in the broker logs. One 
> thing i can see is i  may have configured consumer poorly. If its not 
> too much to ask can u let me know the changes i have to do for over 
> coming this problem.
>
> Thanks
> Arjun Narasimha Kota
>
> On Friday 11 April 2014 10:04 AM, Guozhang Wang wrote:
>> Hi Ajrun,
>>
>> It seems to be the cause:
>>
>> https://issues.apache.org/jira/browse/KAFKA-1016
>>
>> Guozhang
>>
>>
>>
>> On Thu, Apr 10, 2014 at 9:21 PM, Arjun <ar...@socialtwist.com> wrote:
>>
>>> I hope this one would give u  a better idea.
>>>
>>> bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker --group group1
>>> --zkconnect zkhost:port --topic testtopic
>>> Group           Topic                          Pid Offset logSize
>>> Lag             Owner
>>> group1          testtopic    0   253             253 0
>>> group1_ip-xx-1397188061429-b5ff1205-0
>>> group1          testtopic    1   267             267 0
>>> group1_ip-xx-1397188061429-b5ff1205-0
>>> group1          testtopic    2   254             254 0
>>> group1_ip-xx-1397188061429-b5ff1205-0
>>> group1          testtopic    3   265             265 0
>>> group1_ip-xx-1397188061429-b5ff1205-0
>>> group1          testtopic    4   261             261 0
>>> group1_ip-xx-1397188061429-b5ff1205-1
>>> group1          testtopic    5   294             294 0
>>> group1_ip-xx-1397188061429-b5ff1205-1
>>> group1          testtopic    6   248             248 0
>>> group1_ip-xx-1397188061429-b5ff1205-1
>>> group1          testtopic    7   271             271 0
>>> group1_ip-xx-1397188061429-b5ff1205-1
>>> group1          testtopic    8   240             240 0
>>> group1_ip-xx-1397188061429-b5ff1205-2
>>> group1          testtopic    9   261             261 0
>>> group1_ip-xx-1397188061429-b5ff1205-2
>>> group1          testtopic    10  290             290 0
>>> group1_ip-xx-1397188061429-b5ff1205-2
>>> group1          testtopic    11  250             251 1
>>> group1_ip-xx-1397188061429-b5ff1205-2
>>>
>>> If you see the output, in the last line the lag is 1 for that 
>>> partition. I
>>> just send one message. This topic is not new as you see there are 
>>> lot of
>>> messages which have accumlated from yesterday. This one message will 
>>> not be
>>> consumed by consumer what so ever. But if i send some 10 messages 
>>> then all
>>> the messages are consumed.
>>>
>>> Please let me know if i have to change any consumer properties.
>>>
>>> My consumer properties are :
>>> "fetch.wait.max.ms"="180000"
>>> "fetch.min.bytes" = "1"
>>> "auto.offset.reset" = "smallest"
>>> "auto.commit.enable"=  "false"
>>> "fetch.message.max.bytes" = "1048576"
>>>
>>>
>>> Thanks
>>> Arjun Narasimha Kota
>>> On Friday 11 April 2014 06:23 AM, Arjun Kota wrote:
>>>
>>>> The consumer uses do specific topics.
>>>>
>>>> On Apr 11, 2014 6:23 AM, "Arjun Kota" <arjun@socialtwist.com <mailto:
>>>> arjun@socialtwist.com>> wrote:
>>>>
>>>>      Yes the message shows up on the server.
>>>>
>>>>      On Apr 11, 2014 12:07 AM, "Guozhang Wang" <wangguoz@gmail.com
>>>>      <ma...@gmail.com>> wrote:
>>>>
>>>>          Hi Arjun,
>>>>
>>>>          If you only send one message, does that message show up on 
>>>> the
>>>>          server? Does
>>>>          you consumer use wildcard topics or specific topics?
>>>>
>>>>          Guozhang
>>>>
>>>>
>>>>          On Thu, Apr 10, 2014 at 9:20 AM, Arjun <arjun@socialtwist.com
>>>>          <ma...@socialtwist.com>> wrote:
>>>>
>>>>          > But  we have auto offset reset to smallest not largest, 
>>>> even
>>>>          then this
>>>>          > issue arises? If so is there any work around?
>>>>          >
>>>>          > Thanks
>>>>          > Arjun NArasimha Kota
>>>>          >
>>>>          >
>>>>          > On Thursday 10 April 2014 09:39 PM, Guozhang Wang wrote:
>>>>          >
>>>>          >> It could be 
>>>> https://issues.apache.org/jira/browse/KAFKA-1006.
>>>>          >>
>>>>          >> Guozhang
>>>>          >>
>>>>          >>
>>>>          >> On Thu, Apr 10, 2014 at 8:50 AM, Arjun
>>>>          <arjun@socialtwist.com <ma...@socialtwist.com>> wrote:
>>>>          >>
>>>>          >>  its auto created
>>>>          >>> but even after topic creation this is the scenario
>>>>          >>>
>>>>          >>> Arjun
>>>>          >>>
>>>>          >>> On Thursday 10 April 2014 08:41 PM, Guozhang Wang wrote:
>>>>          >>>
>>>>          >>>  Hi Arjun,
>>>>          >>>>
>>>>          >>>> Did you manually create the topic or use 
>>>> auto.topic.creation?
>>>>          >>>>
>>>>          >>>> Guozhang
>>>>          >>>>
>>>>          >>>>
>>>>          >>>> On Thu, Apr 10, 2014 at 7:39 AM, Arjun
>>>>          <arjun@socialtwist.com <ma...@socialtwist.com>> wrote:
>>>>          >>>>
>>>>          >>>>   Hi,
>>>>          >>>>
>>>>          >>>>> We have 3 node kafka 0.8 setup with zookeepers 
>>>> ensemble.
>>>>          We use high
>>>>          >>>>> level
>>>>          >>>>> consumer with auto commit offset false. I am facing 
>>>> some
>>>>          peculiar
>>>>          >>>>> problem
>>>>          >>>>> with kafka. When i send some 10-20 messages or so the
>>>>          consumer starts
>>>>          >>>>> to
>>>>          >>>>> consume the messages. But if i send only one message to
>>>>          kafka, then
>>>>          >>>>> even
>>>>          >>>>> though consumer is active it is not trying to fetch the
>>>>          message. There
>>>>          >>>>> is
>>>>          >>>>> nothing in logs, just the messages are being fetched by
>>>>          the kafka
>>>>          >>>>> consumer.
>>>>          >>>>> The messages are there in the Kafka server. Can some 
>>>> one
>>>>          let me know
>>>>          >>>>> where
>>>>          >>>>> i am doing wrong.
>>>>          >>>>>
>>>>          >>>>>
>>>>          >>>>> Thanks
>>>>          >>>>> Arjun Narasimha Kota
>>>>          >>>>>
>>>>          >>>>>
>>>>          >>>>>
>>>>          >>>>
>>>>          >>
>>>>          >
>>>>
>>>>
>>>>          --
>>>>          -- Guozhang
>>>>
>>>>
>>
>


Re: consumer not consuming messages

Posted by Arjun <ar...@socialtwist.com>.
Hi,

I could not see any out of memory exceptions in the broker logs. One 
thing i can see is i  may have configured consumer poorly. If its not 
too much to ask can u let me know the changes i have to do for over 
coming this problem.

Thanks
Arjun Narasimha Kota

On Friday 11 April 2014 10:04 AM, Guozhang Wang wrote:
> Hi Ajrun,
>
> It seems to be the cause:
>
> https://issues.apache.org/jira/browse/KAFKA-1016
>
> Guozhang
>
>
>
> On Thu, Apr 10, 2014 at 9:21 PM, Arjun <ar...@socialtwist.com> wrote:
>
>> I hope this one would give u  a better idea.
>>
>> bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker --group group1
>> --zkconnect zkhost:port --topic testtopic
>> Group           Topic                          Pid Offset logSize
>> Lag             Owner
>> group1          testtopic    0   253             253 0
>> group1_ip-xx-1397188061429-b5ff1205-0
>> group1          testtopic    1   267             267 0
>> group1_ip-xx-1397188061429-b5ff1205-0
>> group1          testtopic    2   254             254 0
>> group1_ip-xx-1397188061429-b5ff1205-0
>> group1          testtopic    3   265             265 0
>> group1_ip-xx-1397188061429-b5ff1205-0
>> group1          testtopic    4   261             261 0
>> group1_ip-xx-1397188061429-b5ff1205-1
>> group1          testtopic    5   294             294 0
>> group1_ip-xx-1397188061429-b5ff1205-1
>> group1          testtopic    6   248             248 0
>> group1_ip-xx-1397188061429-b5ff1205-1
>> group1          testtopic    7   271             271 0
>> group1_ip-xx-1397188061429-b5ff1205-1
>> group1          testtopic    8   240             240 0
>> group1_ip-xx-1397188061429-b5ff1205-2
>> group1          testtopic    9   261             261 0
>> group1_ip-xx-1397188061429-b5ff1205-2
>> group1          testtopic    10  290             290 0
>> group1_ip-xx-1397188061429-b5ff1205-2
>> group1          testtopic    11  250             251 1
>> group1_ip-xx-1397188061429-b5ff1205-2
>>
>> If you see the output, in the last line the lag is 1 for that partition. I
>> just send one message. This topic is not new as you see there are lot of
>> messages which have accumlated from yesterday. This one message will not be
>> consumed by consumer what so ever. But if i send some 10 messages then all
>> the messages are consumed.
>>
>> Please let me know if i have to change any consumer properties.
>>
>> My consumer properties are :
>> "fetch.wait.max.ms"="180000"
>> "fetch.min.bytes" = "1"
>> "auto.offset.reset" = "smallest"
>> "auto.commit.enable"=  "false"
>> "fetch.message.max.bytes" = "1048576"
>>
>>
>> Thanks
>> Arjun Narasimha Kota
>> On Friday 11 April 2014 06:23 AM, Arjun Kota wrote:
>>
>>> The consumer uses do specific topics.
>>>
>>> On Apr 11, 2014 6:23 AM, "Arjun Kota" <arjun@socialtwist.com <mailto:
>>> arjun@socialtwist.com>> wrote:
>>>
>>>      Yes the message shows up on the server.
>>>
>>>      On Apr 11, 2014 12:07 AM, "Guozhang Wang" <wangguoz@gmail.com
>>>      <ma...@gmail.com>> wrote:
>>>
>>>          Hi Arjun,
>>>
>>>          If you only send one message, does that message show up on the
>>>          server? Does
>>>          you consumer use wildcard topics or specific topics?
>>>
>>>          Guozhang
>>>
>>>
>>>          On Thu, Apr 10, 2014 at 9:20 AM, Arjun <arjun@socialtwist.com
>>>          <ma...@socialtwist.com>> wrote:
>>>
>>>          > But  we have auto offset reset to smallest not largest, even
>>>          then this
>>>          > issue arises? If so is there any work around?
>>>          >
>>>          > Thanks
>>>          > Arjun NArasimha Kota
>>>          >
>>>          >
>>>          > On Thursday 10 April 2014 09:39 PM, Guozhang Wang wrote:
>>>          >
>>>          >> It could be https://issues.apache.org/jira/browse/KAFKA-1006.
>>>          >>
>>>          >> Guozhang
>>>          >>
>>>          >>
>>>          >> On Thu, Apr 10, 2014 at 8:50 AM, Arjun
>>>          <arjun@socialtwist.com <ma...@socialtwist.com>> wrote:
>>>          >>
>>>          >>  its auto created
>>>          >>> but even after topic creation this is the scenario
>>>          >>>
>>>          >>> Arjun
>>>          >>>
>>>          >>> On Thursday 10 April 2014 08:41 PM, Guozhang Wang wrote:
>>>          >>>
>>>          >>>  Hi Arjun,
>>>          >>>>
>>>          >>>> Did you manually create the topic or use auto.topic.creation?
>>>          >>>>
>>>          >>>> Guozhang
>>>          >>>>
>>>          >>>>
>>>          >>>> On Thu, Apr 10, 2014 at 7:39 AM, Arjun
>>>          <arjun@socialtwist.com <ma...@socialtwist.com>> wrote:
>>>          >>>>
>>>          >>>>   Hi,
>>>          >>>>
>>>          >>>>> We have 3 node kafka 0.8 setup with zookeepers ensemble.
>>>          We use high
>>>          >>>>> level
>>>          >>>>> consumer with auto commit offset false. I am facing some
>>>          peculiar
>>>          >>>>> problem
>>>          >>>>> with kafka. When i send some 10-20 messages or so the
>>>          consumer starts
>>>          >>>>> to
>>>          >>>>> consume the messages. But if  i send only one message to
>>>          kafka, then
>>>          >>>>> even
>>>          >>>>> though consumer is active it is not trying to fetch the
>>>          message. There
>>>          >>>>> is
>>>          >>>>> nothing in logs, just the messages are being fetched by
>>>          the kafka
>>>          >>>>> consumer.
>>>          >>>>> The messages are there in the Kafka server. Can some one
>>>          let me know
>>>          >>>>> where
>>>          >>>>> i am doing wrong.
>>>          >>>>>
>>>          >>>>>
>>>          >>>>> Thanks
>>>          >>>>> Arjun Narasimha Kota
>>>          >>>>>
>>>          >>>>>
>>>          >>>>>
>>>          >>>>
>>>          >>
>>>          >
>>>
>>>
>>>          --
>>>          -- Guozhang
>>>
>>>
>


Re: consumer not consuming messages

Posted by Guozhang Wang <wa...@gmail.com>.
Hi Ajrun,

It seems to be the cause:

https://issues.apache.org/jira/browse/KAFKA-1016

Guozhang



On Thu, Apr 10, 2014 at 9:21 PM, Arjun <ar...@socialtwist.com> wrote:

> I hope this one would give u  a better idea.
>
> bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker --group group1
> --zkconnect zkhost:port --topic testtopic
> Group           Topic                          Pid Offset logSize
> Lag             Owner
> group1          testtopic    0   253             253 0
> group1_ip-xx-1397188061429-b5ff1205-0
> group1          testtopic    1   267             267 0
> group1_ip-xx-1397188061429-b5ff1205-0
> group1          testtopic    2   254             254 0
> group1_ip-xx-1397188061429-b5ff1205-0
> group1          testtopic    3   265             265 0
> group1_ip-xx-1397188061429-b5ff1205-0
> group1          testtopic    4   261             261 0
> group1_ip-xx-1397188061429-b5ff1205-1
> group1          testtopic    5   294             294 0
> group1_ip-xx-1397188061429-b5ff1205-1
> group1          testtopic    6   248             248 0
> group1_ip-xx-1397188061429-b5ff1205-1
> group1          testtopic    7   271             271 0
> group1_ip-xx-1397188061429-b5ff1205-1
> group1          testtopic    8   240             240 0
> group1_ip-xx-1397188061429-b5ff1205-2
> group1          testtopic    9   261             261 0
> group1_ip-xx-1397188061429-b5ff1205-2
> group1          testtopic    10  290             290 0
> group1_ip-xx-1397188061429-b5ff1205-2
> group1          testtopic    11  250             251 1
> group1_ip-xx-1397188061429-b5ff1205-2
>
> If you see the output, in the last line the lag is 1 for that partition. I
> just send one message. This topic is not new as you see there are lot of
> messages which have accumlated from yesterday. This one message will not be
> consumed by consumer what so ever. But if i send some 10 messages then all
> the messages are consumed.
>
> Please let me know if i have to change any consumer properties.
>
> My consumer properties are :
> "fetch.wait.max.ms"="180000"
> "fetch.min.bytes" = "1"
> "auto.offset.reset" = "smallest"
> "auto.commit.enable"=  "false"
> "fetch.message.max.bytes" = "1048576"
>
>
> Thanks
> Arjun Narasimha Kota
> On Friday 11 April 2014 06:23 AM, Arjun Kota wrote:
>
>>
>> The consumer uses do specific topics.
>>
>> On Apr 11, 2014 6:23 AM, "Arjun Kota" <arjun@socialtwist.com <mailto:
>> arjun@socialtwist.com>> wrote:
>>
>>     Yes the message shows up on the server.
>>
>>     On Apr 11, 2014 12:07 AM, "Guozhang Wang" <wangguoz@gmail.com
>>     <ma...@gmail.com>> wrote:
>>
>>         Hi Arjun,
>>
>>         If you only send one message, does that message show up on the
>>         server? Does
>>         you consumer use wildcard topics or specific topics?
>>
>>         Guozhang
>>
>>
>>         On Thu, Apr 10, 2014 at 9:20 AM, Arjun <arjun@socialtwist.com
>>         <ma...@socialtwist.com>> wrote:
>>
>>         > But  we have auto offset reset to smallest not largest, even
>>         then this
>>         > issue arises? If so is there any work around?
>>         >
>>         > Thanks
>>         > Arjun NArasimha Kota
>>         >
>>         >
>>         > On Thursday 10 April 2014 09:39 PM, Guozhang Wang wrote:
>>         >
>>         >> It could be https://issues.apache.org/jira/browse/KAFKA-1006.
>>         >>
>>         >> Guozhang
>>         >>
>>         >>
>>         >> On Thu, Apr 10, 2014 at 8:50 AM, Arjun
>>         <arjun@socialtwist.com <ma...@socialtwist.com>> wrote:
>>         >>
>>         >>  its auto created
>>         >>> but even after topic creation this is the scenario
>>         >>>
>>         >>> Arjun
>>         >>>
>>         >>> On Thursday 10 April 2014 08:41 PM, Guozhang Wang wrote:
>>         >>>
>>         >>>  Hi Arjun,
>>         >>>>
>>         >>>> Did you manually create the topic or use auto.topic.creation?
>>         >>>>
>>         >>>> Guozhang
>>         >>>>
>>         >>>>
>>         >>>> On Thu, Apr 10, 2014 at 7:39 AM, Arjun
>>         <arjun@socialtwist.com <ma...@socialtwist.com>> wrote:
>>         >>>>
>>         >>>>   Hi,
>>         >>>>
>>         >>>>> We have 3 node kafka 0.8 setup with zookeepers ensemble.
>>         We use high
>>         >>>>> level
>>         >>>>> consumer with auto commit offset false. I am facing some
>>         peculiar
>>         >>>>> problem
>>         >>>>> with kafka. When i send some 10-20 messages or so the
>>         consumer starts
>>         >>>>> to
>>         >>>>> consume the messages. But if  i send only one message to
>>         kafka, then
>>         >>>>> even
>>         >>>>> though consumer is active it is not trying to fetch the
>>         message. There
>>         >>>>> is
>>         >>>>> nothing in logs, just the messages are being fetched by
>>         the kafka
>>         >>>>> consumer.
>>         >>>>> The messages are there in the Kafka server. Can some one
>>         let me know
>>         >>>>> where
>>         >>>>> i am doing wrong.
>>         >>>>>
>>         >>>>>
>>         >>>>> Thanks
>>         >>>>> Arjun Narasimha Kota
>>         >>>>>
>>         >>>>>
>>         >>>>>
>>         >>>>
>>         >>
>>         >
>>
>>
>>         --
>>         -- Guozhang
>>
>>
>


-- 
-- Guozhang

Re: consumer not consuming messages

Posted by Arjun <ar...@socialtwist.com>.
I hope this one would give u  a better idea.

bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker --group group1 
--zkconnect zkhost:port --topic testtopic
Group           Topic                          Pid Offset 
logSize         Lag             Owner
group1          testtopic    0   253             253 0               
group1_ip-xx-1397188061429-b5ff1205-0
group1          testtopic    1   267             267 0               
group1_ip-xx-1397188061429-b5ff1205-0
group1          testtopic    2   254             254 0               
group1_ip-xx-1397188061429-b5ff1205-0
group1          testtopic    3   265             265 0               
group1_ip-xx-1397188061429-b5ff1205-0
group1          testtopic    4   261             261 0               
group1_ip-xx-1397188061429-b5ff1205-1
group1          testtopic    5   294             294 0               
group1_ip-xx-1397188061429-b5ff1205-1
group1          testtopic    6   248             248 0               
group1_ip-xx-1397188061429-b5ff1205-1
group1          testtopic    7   271             271 0               
group1_ip-xx-1397188061429-b5ff1205-1
group1          testtopic    8   240             240 0               
group1_ip-xx-1397188061429-b5ff1205-2
group1          testtopic    9   261             261 0               
group1_ip-xx-1397188061429-b5ff1205-2
group1          testtopic    10  290             290 0               
group1_ip-xx-1397188061429-b5ff1205-2
group1          testtopic    11  250             251 1               
group1_ip-xx-1397188061429-b5ff1205-2

If you see the output, in the last line the lag is 1 for that partition. 
I just send one message. This topic is not new as you see there are lot 
of messages which have accumlated from yesterday. This one message will 
not be consumed by consumer what so ever. But if i send some 10 messages 
then all the messages are consumed.

Please let me know if i have to change any consumer properties.

My consumer properties are :
"fetch.wait.max.ms"="180000"
"fetch.min.bytes" = "1"
"auto.offset.reset" = "smallest"
"auto.commit.enable"=  "false"
"fetch.message.max.bytes" = "1048576"

Thanks
Arjun Narasimha Kota
On Friday 11 April 2014 06:23 AM, Arjun Kota wrote:
>
> The consumer uses do specific topics.
>
> On Apr 11, 2014 6:23 AM, "Arjun Kota" <arjun@socialtwist.com 
> <ma...@socialtwist.com>> wrote:
>
>     Yes the message shows up on the server.
>
>     On Apr 11, 2014 12:07 AM, "Guozhang Wang" <wangguoz@gmail.com
>     <ma...@gmail.com>> wrote:
>
>         Hi Arjun,
>
>         If you only send one message, does that message show up on the
>         server? Does
>         you consumer use wildcard topics or specific topics?
>
>         Guozhang
>
>
>         On Thu, Apr 10, 2014 at 9:20 AM, Arjun <arjun@socialtwist.com
>         <ma...@socialtwist.com>> wrote:
>
>         > But  we have auto offset reset to smallest not largest, even
>         then this
>         > issue arises? If so is there any work around?
>         >
>         > Thanks
>         > Arjun NArasimha Kota
>         >
>         >
>         > On Thursday 10 April 2014 09:39 PM, Guozhang Wang wrote:
>         >
>         >> It could be https://issues.apache.org/jira/browse/KAFKA-1006.
>         >>
>         >> Guozhang
>         >>
>         >>
>         >> On Thu, Apr 10, 2014 at 8:50 AM, Arjun
>         <arjun@socialtwist.com <ma...@socialtwist.com>> wrote:
>         >>
>         >>  its auto created
>         >>> but even after topic creation this is the scenario
>         >>>
>         >>> Arjun
>         >>>
>         >>> On Thursday 10 April 2014 08:41 PM, Guozhang Wang wrote:
>         >>>
>         >>>  Hi Arjun,
>         >>>>
>         >>>> Did you manually create the topic or use auto.topic.creation?
>         >>>>
>         >>>> Guozhang
>         >>>>
>         >>>>
>         >>>> On Thu, Apr 10, 2014 at 7:39 AM, Arjun
>         <arjun@socialtwist.com <ma...@socialtwist.com>> wrote:
>         >>>>
>         >>>>   Hi,
>         >>>>
>         >>>>> We have 3 node kafka 0.8 setup with zookeepers ensemble.
>         We use high
>         >>>>> level
>         >>>>> consumer with auto commit offset false. I am facing some
>         peculiar
>         >>>>> problem
>         >>>>> with kafka. When i send some 10-20 messages or so the
>         consumer starts
>         >>>>> to
>         >>>>> consume the messages. But if  i send only one message to
>         kafka, then
>         >>>>> even
>         >>>>> though consumer is active it is not trying to fetch the
>         message. There
>         >>>>> is
>         >>>>> nothing in logs, just the messages are being fetched by
>         the kafka
>         >>>>> consumer.
>         >>>>> The messages are there in the Kafka server. Can some one
>         let me know
>         >>>>> where
>         >>>>> i am doing wrong.
>         >>>>>
>         >>>>>
>         >>>>> Thanks
>         >>>>> Arjun Narasimha Kota
>         >>>>>
>         >>>>>
>         >>>>>
>         >>>>
>         >>
>         >
>
>
>         --
>         -- Guozhang
>


Re: consumer not consuming messages

Posted by Arjun Kota <ar...@socialtwist.com>.
The consumer uses do specific topics.
On Apr 11, 2014 6:23 AM, "Arjun Kota" <ar...@socialtwist.com> wrote:

> Yes the message shows up on the server.
> On Apr 11, 2014 12:07 AM, "Guozhang Wang" <wa...@gmail.com> wrote:
>
>> Hi Arjun,
>>
>> If you only send one message, does that message show up on the server?
>> Does
>> you consumer use wildcard topics or specific topics?
>>
>> Guozhang
>>
>>
>> On Thu, Apr 10, 2014 at 9:20 AM, Arjun <ar...@socialtwist.com> wrote:
>>
>> > But  we have auto offset reset to smallest not largest, even then this
>> > issue arises? If so is there any work around?
>> >
>> > Thanks
>> > Arjun NArasimha Kota
>> >
>> >
>> > On Thursday 10 April 2014 09:39 PM, Guozhang Wang wrote:
>> >
>> >> It could be https://issues.apache.org/jira/browse/KAFKA-1006.
>> >>
>> >> Guozhang
>> >>
>> >>
>> >> On Thu, Apr 10, 2014 at 8:50 AM, Arjun <ar...@socialtwist.com> wrote:
>> >>
>> >>  its auto created
>> >>> but even after topic creation this is the scenario
>> >>>
>> >>> Arjun
>> >>>
>> >>> On Thursday 10 April 2014 08:41 PM, Guozhang Wang wrote:
>> >>>
>> >>>  Hi Arjun,
>> >>>>
>> >>>> Did you manually create the topic or use auto.topic.creation?
>> >>>>
>> >>>> Guozhang
>> >>>>
>> >>>>
>> >>>> On Thu, Apr 10, 2014 at 7:39 AM, Arjun <ar...@socialtwist.com>
>> wrote:
>> >>>>
>> >>>>   Hi,
>> >>>>
>> >>>>> We have 3 node kafka 0.8 setup with zookeepers ensemble. We use high
>> >>>>> level
>> >>>>> consumer with auto commit offset false. I am facing some peculiar
>> >>>>> problem
>> >>>>> with kafka. When i send some 10-20 messages or so the consumer
>> starts
>> >>>>> to
>> >>>>> consume the messages. But if  i send only one message to kafka, then
>> >>>>> even
>> >>>>> though consumer is active it is not trying to fetch the message.
>> There
>> >>>>> is
>> >>>>> nothing in logs, just the messages are being fetched by the kafka
>> >>>>> consumer.
>> >>>>> The messages are there in the Kafka server. Can some one let me know
>> >>>>> where
>> >>>>> i am doing wrong.
>> >>>>>
>> >>>>>
>> >>>>> Thanks
>> >>>>> Arjun Narasimha Kota
>> >>>>>
>> >>>>>
>> >>>>>
>> >>>>
>> >>
>> >
>>
>>
>> --
>> -- Guozhang
>>
>

Re: consumer not consuming messages

Posted by Arjun Kota <ar...@socialtwist.com>.
Yes the message shows up on the server.
On Apr 11, 2014 12:07 AM, "Guozhang Wang" <wa...@gmail.com> wrote:

> Hi Arjun,
>
> If you only send one message, does that message show up on the server? Does
> you consumer use wildcard topics or specific topics?
>
> Guozhang
>
>
> On Thu, Apr 10, 2014 at 9:20 AM, Arjun <ar...@socialtwist.com> wrote:
>
> > But  we have auto offset reset to smallest not largest, even then this
> > issue arises? If so is there any work around?
> >
> > Thanks
> > Arjun NArasimha Kota
> >
> >
> > On Thursday 10 April 2014 09:39 PM, Guozhang Wang wrote:
> >
> >> It could be https://issues.apache.org/jira/browse/KAFKA-1006.
> >>
> >> Guozhang
> >>
> >>
> >> On Thu, Apr 10, 2014 at 8:50 AM, Arjun <ar...@socialtwist.com> wrote:
> >>
> >>  its auto created
> >>> but even after topic creation this is the scenario
> >>>
> >>> Arjun
> >>>
> >>> On Thursday 10 April 2014 08:41 PM, Guozhang Wang wrote:
> >>>
> >>>  Hi Arjun,
> >>>>
> >>>> Did you manually create the topic or use auto.topic.creation?
> >>>>
> >>>> Guozhang
> >>>>
> >>>>
> >>>> On Thu, Apr 10, 2014 at 7:39 AM, Arjun <ar...@socialtwist.com> wrote:
> >>>>
> >>>>   Hi,
> >>>>
> >>>>> We have 3 node kafka 0.8 setup with zookeepers ensemble. We use high
> >>>>> level
> >>>>> consumer with auto commit offset false. I am facing some peculiar
> >>>>> problem
> >>>>> with kafka. When i send some 10-20 messages or so the consumer starts
> >>>>> to
> >>>>> consume the messages. But if  i send only one message to kafka, then
> >>>>> even
> >>>>> though consumer is active it is not trying to fetch the message.
> There
> >>>>> is
> >>>>> nothing in logs, just the messages are being fetched by the kafka
> >>>>> consumer.
> >>>>> The messages are there in the Kafka server. Can some one let me know
> >>>>> where
> >>>>> i am doing wrong.
> >>>>>
> >>>>>
> >>>>> Thanks
> >>>>> Arjun Narasimha Kota
> >>>>>
> >>>>>
> >>>>>
> >>>>
> >>
> >
>
>
> --
> -- Guozhang
>

Re: consumer not consuming messages

Posted by Guozhang Wang <wa...@gmail.com>.
Hi Arjun,

If you only send one message, does that message show up on the server? Does
you consumer use wildcard topics or specific topics?

Guozhang


On Thu, Apr 10, 2014 at 9:20 AM, Arjun <ar...@socialtwist.com> wrote:

> But  we have auto offset reset to smallest not largest, even then this
> issue arises? If so is there any work around?
>
> Thanks
> Arjun NArasimha Kota
>
>
> On Thursday 10 April 2014 09:39 PM, Guozhang Wang wrote:
>
>> It could be https://issues.apache.org/jira/browse/KAFKA-1006.
>>
>> Guozhang
>>
>>
>> On Thu, Apr 10, 2014 at 8:50 AM, Arjun <ar...@socialtwist.com> wrote:
>>
>>  its auto created
>>> but even after topic creation this is the scenario
>>>
>>> Arjun
>>>
>>> On Thursday 10 April 2014 08:41 PM, Guozhang Wang wrote:
>>>
>>>  Hi Arjun,
>>>>
>>>> Did you manually create the topic or use auto.topic.creation?
>>>>
>>>> Guozhang
>>>>
>>>>
>>>> On Thu, Apr 10, 2014 at 7:39 AM, Arjun <ar...@socialtwist.com> wrote:
>>>>
>>>>   Hi,
>>>>
>>>>> We have 3 node kafka 0.8 setup with zookeepers ensemble. We use high
>>>>> level
>>>>> consumer with auto commit offset false. I am facing some peculiar
>>>>> problem
>>>>> with kafka. When i send some 10-20 messages or so the consumer starts
>>>>> to
>>>>> consume the messages. But if  i send only one message to kafka, then
>>>>> even
>>>>> though consumer is active it is not trying to fetch the message. There
>>>>> is
>>>>> nothing in logs, just the messages are being fetched by the kafka
>>>>> consumer.
>>>>> The messages are there in the Kafka server. Can some one let me know
>>>>> where
>>>>> i am doing wrong.
>>>>>
>>>>>
>>>>> Thanks
>>>>> Arjun Narasimha Kota
>>>>>
>>>>>
>>>>>
>>>>
>>
>


-- 
-- Guozhang

Re: consumer not consuming messages

Posted by Arjun <ar...@socialtwist.com>.
But  we have auto offset reset to smallest not largest, even then this 
issue arises? If so is there any work around?

Thanks
Arjun NArasimha Kota

On Thursday 10 April 2014 09:39 PM, Guozhang Wang wrote:
> It could be https://issues.apache.org/jira/browse/KAFKA-1006.
>
> Guozhang
>
>
> On Thu, Apr 10, 2014 at 8:50 AM, Arjun <ar...@socialtwist.com> wrote:
>
>> its auto created
>> but even after topic creation this is the scenario
>>
>> Arjun
>>
>> On Thursday 10 April 2014 08:41 PM, Guozhang Wang wrote:
>>
>>> Hi Arjun,
>>>
>>> Did you manually create the topic or use auto.topic.creation?
>>>
>>> Guozhang
>>>
>>>
>>> On Thu, Apr 10, 2014 at 7:39 AM, Arjun <ar...@socialtwist.com> wrote:
>>>
>>>   Hi,
>>>> We have 3 node kafka 0.8 setup with zookeepers ensemble. We use high
>>>> level
>>>> consumer with auto commit offset false. I am facing some peculiar problem
>>>> with kafka. When i send some 10-20 messages or so the consumer starts to
>>>> consume the messages. But if  i send only one message to kafka, then even
>>>> though consumer is active it is not trying to fetch the message. There is
>>>> nothing in logs, just the messages are being fetched by the kafka
>>>> consumer.
>>>> The messages are there in the Kafka server. Can some one let me know
>>>> where
>>>> i am doing wrong.
>>>>
>>>>
>>>> Thanks
>>>> Arjun Narasimha Kota
>>>>
>>>>
>>>
>


Re: consumer not consuming messages

Posted by Guozhang Wang <wa...@gmail.com>.
It could be https://issues.apache.org/jira/browse/KAFKA-1006.

Guozhang


On Thu, Apr 10, 2014 at 8:50 AM, Arjun <ar...@socialtwist.com> wrote:

> its auto created
> but even after topic creation this is the scenario
>
> Arjun
>
> On Thursday 10 April 2014 08:41 PM, Guozhang Wang wrote:
>
>> Hi Arjun,
>>
>> Did you manually create the topic or use auto.topic.creation?
>>
>> Guozhang
>>
>>
>> On Thu, Apr 10, 2014 at 7:39 AM, Arjun <ar...@socialtwist.com> wrote:
>>
>>  Hi,
>>>
>>> We have 3 node kafka 0.8 setup with zookeepers ensemble. We use high
>>> level
>>> consumer with auto commit offset false. I am facing some peculiar problem
>>> with kafka. When i send some 10-20 messages or so the consumer starts to
>>> consume the messages. But if  i send only one message to kafka, then even
>>> though consumer is active it is not trying to fetch the message. There is
>>> nothing in logs, just the messages are being fetched by the kafka
>>> consumer.
>>> The messages are there in the Kafka server. Can some one let me know
>>> where
>>> i am doing wrong.
>>>
>>>
>>> Thanks
>>> Arjun Narasimha Kota
>>>
>>>
>>
>>
>


-- 
-- Guozhang

Re: consumer not consuming messages

Posted by Arjun <ar...@socialtwist.com>.
its auto created
but even after topic creation this is the scenario

Arjun
On Thursday 10 April 2014 08:41 PM, Guozhang Wang wrote:
> Hi Arjun,
>
> Did you manually create the topic or use auto.topic.creation?
>
> Guozhang
>
>
> On Thu, Apr 10, 2014 at 7:39 AM, Arjun <ar...@socialtwist.com> wrote:
>
>> Hi,
>>
>> We have 3 node kafka 0.8 setup with zookeepers ensemble. We use high level
>> consumer with auto commit offset false. I am facing some peculiar problem
>> with kafka. When i send some 10-20 messages or so the consumer starts to
>> consume the messages. But if  i send only one message to kafka, then even
>> though consumer is active it is not trying to fetch the message. There is
>> nothing in logs, just the messages are being fetched by the kafka consumer.
>> The messages are there in the Kafka server. Can some one let me know where
>> i am doing wrong.
>>
>>
>> Thanks
>> Arjun Narasimha Kota
>>
>
>


Re: consumer not consuming messages

Posted by Guozhang Wang <wa...@gmail.com>.
Hi Arjun,

Did you manually create the topic or use auto.topic.creation?

Guozhang


On Thu, Apr 10, 2014 at 7:39 AM, Arjun <ar...@socialtwist.com> wrote:

> Hi,
>
> We have 3 node kafka 0.8 setup with zookeepers ensemble. We use high level
> consumer with auto commit offset false. I am facing some peculiar problem
> with kafka. When i send some 10-20 messages or so the consumer starts to
> consume the messages. But if  i send only one message to kafka, then even
> though consumer is active it is not trying to fetch the message. There is
> nothing in logs, just the messages are being fetched by the kafka consumer.
> The messages are there in the Kafka server. Can some one let me know where
> i am doing wrong.
>
>
> Thanks
> Arjun Narasimha Kota
>



-- 
-- Guozhang