it is pretty random

On Wed, Jul 15, 2015 at 4:22 PM, Jiangjie Qin <j...@linkedin.com.invalid>
wrote:

> I’m not sure if it is related to running in cloud. Do you see this
> disconnection issue always happening on committing offsets or it happens
> randomly?
>
> Jiangjie (becket) qin
>
> On 7/15/15, 12:53 PM, "Vadim Bobrov" <vadimbob...@gmail.com> wrote:
>
> >there are lots of files under logs directory of the broker, just in case I
> >checked all modified around the time of error and found nothing unusual
> >both client and broker are 0.8.2.1
> >
> >could it have something to do with running it in the cloud? we are on
> >Linode and I remember having random disconnections problem with MySQL on
> >other nodes that since gone forever
> >
> >On Wed, Jul 15, 2015 at 3:43 PM, Jiangjie Qin <j...@linkedin.com.invalid>
> >wrote:
> >
> >> Is there anything on the broker log?
> >> Is it possible that your client and broker are not running on the same
> >> version?
> >>
> >> Jiangjie (Becket) Qin
> >>
> >> On 7/15/15, 11:40 AM, "Vadim Bobrov" <vadimbob...@gmail.com> wrote:
> >>
> >> >caught it, thanks for help!
> >> >any ideas what to do?
> >> >
> >> >TRACE 2015-07-15 18:37:58,070 [chaos-akka.actor.jms-dispatcher-1019 ]
> >> >kafka.network.BoundedByteBufferSend - 113 bytes written.
> >> >ERROR 2015-07-15 18:37:58,078 [chaos-akka.actor.jms-dispatcher-1019 ]
> >> >kafka.consumer.ZookeeperConsumerConnector -
> >> >[chaos-test_ds2.outsmartinc.com-1436984542831-4d399e71], Error while
> >> >committing offsets.
> >> >java.io.EOFException: Received -1 when reading from channel, socket has
> >> >likely been closed.
> >> >        at kafka.utils.Utils$.read(Utils.scala:381)
> >> >        at
> >>
> >>>kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive
> >>>.s
> >> >cala:54)
> >> >        at
> >> >kafka.network.Receive$class.readCompletely(Transmission.scala:56)
> >> >        at
> >>
> >>>kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferR
> >>>ec
> >> >eive.scala:29)
> >> >        at
> >> >kafka.network.BlockingChannel.receive(BlockingChannel.scala:111)
> >> >        at
> >>
> >>>kafka.consumer.ZookeeperConsumerConnector.liftedTree2$1(ZookeeperConsume
> >>>rC
> >> >onnector.scala:313)
> >> >        at
> >>
> >>>kafka.consumer.ZookeeperConsumerConnector.commitOffsets(ZookeeperConsume
> >>>rC
> >> >onnector.scala:310)
> >> >        at
> >>
> >>>kafka.javaapi.consumer.ZookeeperConsumerConnector.commitOffsets(Zookeepe
> >>>rC
> >> >onsumerConnector.scala:111)
> >> >        at
> >>
> >>>com.os.messaging.kafka.KafkaMessageService$class.acknowledge(KafkaMessag
> >>>eS
> >> >ervice.scala:55)
> >> >        at
> >>com.os.actor.Main$$anonfun$2$$anon$2.acknowledge(Main.scala:84)
> >> >        at
> >>
> >>>com.os.actor.acquisition.MessageListenerActor$$anonfun$8.applyOrElse(Mes
> >>>sa
> >> >geListenerActor.scala:180)
> >> >        at
> >>
> >>>com.os.actor.acquisition.MessageListenerActor$$anonfun$8.applyOrElse(Mes
> >>>sa
> >> >geListenerActor.scala:164)
> >> >        at
> >>scala.PartialFunction$OrElse.apply(PartialFunction.scala:162)
> >> >        at akka.actor.FSM$class.processEvent(FSM.scala:607)
> >> >        at
> >>
> >>>com.os.actor.acquisition.MessageListenerActor.processEvent(MessageListen
> >>>er
> >> >Actor.scala:32)
> >> >        at
> >>akka.actor.FSM$class.akka$actor$FSM$$processMsg(FSM.scala:598)
> >> >        at akka.actor.FSM$$anonfun$receive$1.applyOrElse(FSM.scala:592)
> >> >        at akka.actor.Actor$class.aroundReceive(Actor.scala:465)
> >> >        at
> >>
> >>>com.os.actor.acquisition.MessageListenerActor.aroundReceive(MessageListe
> >>>ne
> >> >rActor.scala:32)
> >> >        at akka.actor.ActorCell.receiveMessage(ActorCell.scala:516)
> >> >        at akka.actor.ActorCell.invoke(ActorCell.scala:487)
> >> >        at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:254)
> >> >        at akka.dispatch.Mailbox.run(Mailbox.scala:221)
> >> >        at akka.dispatch.Mailbox.exec(Mailbox.scala:231)
> >> >        at
> >> >scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
> >> >        at
> >>
> >>>scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.ja
> >>>va
> >> >:1339)
> >> >        at
> >>
> >>>scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
> >> >        at
> >>
> >>>scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.
> >>>ja
> >> >va:107)
> >> >
> >> >
> >> >On Wed, Jul 15, 2015 at 1:36 PM, Vadim Bobrov <vadimbob...@gmail.com>
> >> >wrote:
> >> >
> >> >> thanks Joel and Jiangjie,
> >> >> I have figured it out. In addition to my log4j 2 config file I also
> >> >>needed
> >> >> a log4j 1 config file, then it works. Let me trace what happens when
> >>the
> >> >> offsets are not committed and report back
> >> >>
> >> >> On Wed, Jul 15, 2015 at 1:33 PM, Joel Koshy <jjkosh...@gmail.com>
> >> wrote:
> >> >>
> >> >>> - You can also change the log4j level dynamically via the
> >> >>>   kafka.Log4jController mbean.
> >> >>> - You can also look at offset commit request metrics (mbeans) on the
> >> >>>   broker (just to check if _any_ offset commits are coming through
> >> >>>   during the period you see no moving offsets).
> >> >>> - The alternative is to just consume the offsets topic.
> >> >>>
> >> >>> On Wed, Jul 15, 2015 at 05:30:17PM +0000, Jiangjie Qin wrote:
> >> >>> > I am not sure how your project was setup. But I think it depends
> >>on
> >> >>>what
> >> >>> > log4j property file you specified when you started your
> >>application.
> >> >>>Can
> >> >>> > you check if you have log4j appender defined and the loggers are
> >> >>> directed
> >> >>> > to the correct appender?
> >> >>> >
> >> >>> > Thanks,
> >> >>> >
> >> >>> > Jiangjie (Becket) Qin
> >> >>> >
> >> >>> > On 7/15/15, 8:10 AM, "Vadim Bobrov" <vadimbob...@gmail.com>
> wrote:
> >> >>> >
> >> >>> > >Thanks Jiangjie,
> >> >>> > >
> >> >>> > >unfortunately turning trace level on does not seem to work (any
> >>log
> >> >>> level
> >> >>> > >actually) I am using log4j2 (through slf4j) and despite including
> >> >>> log4j1
> >> >>> > >bridge and these lines:
> >> >>> > >
> >> >>> > ><Logger name="org.apache.kafka" level="trace"/>
> >> >>> > ><Logger name="kafka" level="trace"/>
> >> >>> > >
> >> >>> > >in my conf file I could not squeeze out any logging from kafka.
> >> >>>Logging
> >> >>> > >for
> >> >>> > >all other libs (like zookeeper e.g.) work perfectly. Am I doing
> >> >>> something
> >> >>> > >wrong?
> >> >>> > >
> >> >>> > >
> >> >>> > >On Tue, Jul 14, 2015 at 6:55 PM, Jiangjie Qin
> >> >>> <j...@linkedin.com.invalid>
> >> >>> > >wrote:
> >> >>> > >
> >> >>> > >> Hi Vadim,
> >> >>> > >>
> >> >>> > >> Can you turn on trace level logging on your consumer and search
> >> >>>for
> >> >>> > >> "offset commit response² in the log?
> >> >>> > >> Also maybe take a look at the log to see if there is any
> >>exception
> >> >>> > >>thrown.
> >> >>> > >>
> >> >>> > >> Thanks,
> >> >>> > >>
> >> >>> > >> Jiangjie (Becket) Qin
> >> >>> > >>
> >> >>> > >> On 7/14/15, 11:06 AM, "Vadim Bobrov" <vadimbob...@gmail.com>
> >> >>>wrote:
> >> >>> > >>
> >> >>> > >> >just caught this error again. I issue commitOffsets - no error
> >> >>>but
> >> >>> no
> >> >>> > >> >committng offsets either. __consumer_offsets watching shows no
> >> >>>new
> >> >>> > >> >messages
> >> >>> > >> >either. Then in a few minutes I issue commitOffsets again -
> >>all
> >> >>> > >>committed.
> >> >>> > >> >Unless I am doing something terribly wrong this is very
> >> >>>unreliable
> >> >>> > >> >
> >> >>> > >> >On Tue, Jul 14, 2015 at 1:49 PM, Joel Koshy
> >><jjkosh...@gmail.com
> >> >
> >> >>> > >>wrote:
> >> >>> > >> >
> >> >>> > >> >> Actually, how are you committing offsets? Are you using the
> >>old
> >> >>> > >> >> (zookeeperconsumerconnector) or new KafkaConsumer?
> >> >>> > >> >>
> >> >>> > >> >> It is true that the current APIs don't return any result,
> >>but
> >> >>>it
> >> >>> > >>would
> >> >>> > >> >> help to check if anything is getting into the offsets topic
> >>-
> >> >>> unless
> >> >>> > >> >> you are seeing errors in the logs, the offset commit should
> >> >>> succeed
> >> >>> > >> >> (if you are indeed explicitly committing offsets).
> >> >>> > >> >>
> >> >>> > >> >> Thanks,
> >> >>> > >> >>
> >> >>> > >> >> Joel
> >> >>> > >> >>
> >> >>> > >> >> On Tue, Jul 14, 2015 at 12:19:01PM -0400, Vadim Bobrov
> >>wrote:
> >> >>> > >> >> > Thanks, Joel, I will but regardless of my findings the
> >>basic
> >> >>> > >>problem
> >> >>> > >> >>will
> >> >>> > >> >> > still be there: there is no guarantee that the offsets
> >>will
> >> >>>be
> >> >>> > >> >>committed
> >> >>> > >> >> > after commitOffsets. Because commitOffsets does not return
> >> >>>its
> >> >>> exit
> >> >>> > >> >> status,
> >> >>> > >> >> > nor does it block as I understand until offsets are
> >> >>>committed.
> >> >>> In
> >> >>> > >> >>other
> >> >>> > >> >> > words, there is no way to know that it has, in fact,
> >>commited
> >> >>> the
> >> >>> > >> >>offsets
> >> >>> > >> >> >
> >> >>> > >> >> > or am I missing something? And then another question - why
> >> >>>does
> >> >>> it
> >> >>> > >> >>seem
> >> >>> > >> >> to
> >> >>> > >> >> > depend on the number of consumed messages?
> >> >>> > >> >> >
> >> >>> > >> >> > On Tue, Jul 14, 2015 at 11:36 AM, Joel Koshy <
> >> >>> jjkosh...@gmail.com>
> >> >>> > >> >> wrote:
> >> >>> > >> >> >
> >> >>> > >> >> > > Can you take a look at the kafka commit rate mbean on
> >>your
> >> >>> > >>consumer?
> >> >>> > >> >> > > Also, can you consume the offsets topic while you are
> >> >>> committing
> >> >>> > >> >> > > offsets and see if/what offsets are getting committed?
> >> >>> > >> >> > > (
> >> >>> http://www.slideshare.net/jjkoshy/offset-management-in-kafka/32)
> >> >>> > >> >> > >
> >> >>> > >> >> > > Thanks,
> >> >>> > >> >> > >
> >> >>> > >> >> > > Joel
> >> >>> > >> >> > >
> >> >>> > >> >> > > On Tue, Jul 14, 2015 at 11:12:03AM -0400, Vadim Bobrov
> >> >>>wrote:
> >> >>> > >> >> > > > I am trying to replace ActiveMQ with Kafka in our
> >> >>> environment
> >> >>> > >> >> however I
> >> >>> > >> >> > > > have encountered a strange problem that basically
> >> >>>prevents
> >> >>> from
> >> >>> > >> >>using
> >> >>> > >> >> > > Kafka
> >> >>> > >> >> > > > in production. The problem is that sometimes the
> >>offsets
> >> >>>are
> >> >>> > >>not
> >> >>> > >> >> > > committed.
> >> >>> > >> >> > > >
> >> >>> > >> >> > > > I am using Kafka 0.8.2.1, offset storage = kafka, high
> >> >>>level
> >> >>> > >> >> consumer,
> >> >>> > >> >> > > > auto-commit = off. Every N messages I issue
> >> >>>commitOffsets().
> >> >>> > >>Now
> >> >>> > >> >> here is
> >> >>> > >> >> > > > the problem - if N is below a certain number (180 000
> >>for
> >> >>> me)
> >> >>> > >>it
> >> >>> > >> >> works
> >> >>> > >> >> > > and
> >> >>> > >> >> > > > the offset is moving. If N is 180 000 or more the
> >>offset
> >> >>>is
> >> >>> not
> >> >>> > >> >> updated
> >> >>> > >> >> > > > after commitOffsets
> >> >>> > >> >> > > >
> >> >>> > >> >> > > > I am looking at offsets using kafka-run-class.sh
> >> >>> > >> >> > > > kafka.tools.ConsumerOffsetChecker
> >> >>> > >> >> > > > Any help?
> >> >>> > >> >> > >
> >> >>> > >> >> > >
> >> >>> > >> >>
> >> >>> > >> >>
> >> >>> > >>
> >> >>> > >>
> >> >>> >
> >> >>>
> >> >>> --
> >> >>> Joel
> >> >>>
> >> >>
> >> >>
> >>
> >>
>
>

Reply via email to