Thanks Mathieu.

Is it possible that the background thread within your Streams' consumer
which is responsible for sending the heartbeats got suspended due to a GC?
Note that its heartbeat frequency and the broker's side checking interval
is still defined by the "session.timeout.ms" config, so if there is indeed
a soft failure that suspends all threads then after session.timeout period
the client can still be kicked out. "max.poll.interval.ms" config, on the
other hand, is just for the background thread to check if the user thread
has called "poll()" within that period of time or not, and if not it will
not “cover up” for the user caller thread and stopped periodic heartbeating
so that the broker can finally detect it as failed.


Guozhang


On Mon, Mar 13, 2017 at 7:40 AM, Mathieu Fenniak <
mathieu.fenn...@replicon.com> wrote:

> Hi Guozhang,
>
> Thanks for the response.
>
> I don't believe that my client has soft failed, as my max.poll.interval.ms
> is configured at 1800000 (30 minutes) and the client app shows "Committing
> task StreamTask [N_N]" log messages within the past few (1-3) minutes of
> the failure.
>
> In terms of the "underlying infrastructure issue", I'm not really sure what
> I have in mind.  I'm just assuming there is some reason why the client
> disconnected from the broker, and I'm thinking it's not client-side.  A
> long GC on the broker, rather than the client, might make more sense to
> me...
>
> Mathieu
>
>
> On Fri, Mar 10, 2017 at 6:22 PM, Guozhang Wang <wangg...@gmail.com> wrote:
>
> > Hi Mathieu,
> >
> > Your understanding is correct, just a few remarks:
> >
> > 1. "Marking the coordinator ... dead for group ..." this is because the
> > heartbeat protocol is actually mutual: i.e. from the consumer's point of
> > view, if it has not received a response, it will mark the current
> > remembered coordinator as "dead" and try to re-discovery the coordinator.
> > Note this is not a WARN / ERROR log since it is actually normal in many
> > cases, since as long as it has re-discovered the coordinator, which still
> > acks itself as the member of the group (i.e. its first HB to the newly
> > founded coordinator, which may just be the old one, returns no error
> code)
> > then the consumer assumes that it still owns the assigned partitions and
> > hence continue happy fetching.
> >
> > Only when the coordinator has really kicked itself out of the group, then
> > upon re-connecting it may receive an error code in HB response and hence
> > tries to re-join, and if it happens to piggy-back a commit request
> together
> > you will see the "the group has already rebalanced .." error message.
> >
> > 2. For your case, it is likely that the broker is OK but the client is
> soft
> > failed, like a long GC. try increasing the "max.poll.interval.ms" and
> see
> > if it helps (that controls the maximum period a client can live without
> > calling poll() to send another HB).
> >
> > 3. Re:
> >
> > > A deeper look into the specific state of the client, network, and
> broker
> > at this time interval might suggest an underlying infrastructure issue
> that
> > should be investigated.
> >
> > Could you share what underlying infrastructure issue are you referring
> to?
> >
> > BTW there is some related issue for the commit failed exception that has
> > been fixed in trunk post 0.10.2:
> > https://github.com/apache/kafka/commit/4db048d61206bc6efbd14
> > 3d6293216b7cb4b86c5
> >
> >
> > Guozhang
> >
> >
> > On Fri, Mar 10, 2017 at 1:01 PM, Mathieu Fenniak <
> > mathieu.fenn...@replicon.com> wrote:
> >
> > > Attempting to answer my own questions... but I'd love if someone
> pointed
> > > out any misunderstandings. :-)
> > >
> > > What causes the Kafka client to "Marking the coordinator ... dead for
> > group
> > > > ..."?  Is it untimely heartbeat messages?
> > >
> > >
> > > Small number of causes, but for a running application and a Kafka
> broker
> > > that didn't die, untimely heartbeats are probably occurring.  Maybe an
> > > intermittent network blip, or high load (I/O?) on the network broker.
> > >
> > > Any thoughts on what would cause the commit to fail after a "dead" like
> > > > that, even though it is well within my max.poll.interval.ms?
> > >
> > >
> > > My configuration had default session.timeout.ms (10 seconds) and
> > > heartbeat.interval.ms (3 seconds).  If the intermittent network or
> load
> > > issue disappeared but took 60 seconds, as suggested by my logs, my
> > > consumer's session would have timed out on the server.
> > >
> > > Probably bumping up my session.timeout.ms is the immediate answer (and
> > > possibly heartbeat.interval.ms, just to keep it reasonable).  A deeper
> > > look
> > > into the specific state of the client, network, and broker at this time
> > > interval might suggest an underlying infrastructure issue that should
> be
> > > investigated.
> > >
> > > Is the message "the group has already rebalanced and assigned the
> > > > partitions to another member" really necessarily true?
> > >
> > >
> > > The partitions are revoked from this app's consumer session.  If
> there's
> > no
> > > other consumers, they message's reference to "another member" is a tiny
> > bit
> > > misleading, but the point that they're not assigned to "me" is
> accurate.
> > >
> > > Mathieu
> > >
> > >
> > > On Fri, Mar 10, 2017 at 11:15 AM, Mathieu Fenniak <
> > > mathieu.fenn...@replicon.com> wrote:
> > >
> > > > Hey Kafka Users,
> > > >
> > > > I've been observing a few instances of CommitFailedException (group
> has
> > > > already rebalanced) that seem to happen well-within
> > max.poll.interval.ms
> > > > since the last commit.  In at least one specific case that I've
> looked
> > > at,
> > > > between the last successful commit and the failed commit, there is a
> > > > "Marking the coordinator ... dead for group ..." message.
> > > >
> > > > Log excerpts / sequence, including consumer config dump:
> > > > https://gist.github.com/mfenniak/46113b4e3cbe35cc54ee103cb0515f34
> > This
> > > > is a Kafka Streams application, with Kafka Streams & brokers on
> > 0.10.2.0.
> > > >
> > > > What causes the Kafka client to "Marking the coordinator ... dead for
> > > > group ..."?  Is it untimely heartbeat messages?
> > > >
> > > > Any thoughts on what would cause the commit to fail after a "dead"
> like
> > > > that, even though it is well within my max.poll.interval.ms?
> > > >
> > > > Is the message "the group has already rebalanced and assigned the
> > > > partitions to another member" really necessarily true?  Because in
> this
> > > > specific instance, there was only one application instance running in
> > > this
> > > > consumer group, so I'm a little skeptical.
> > > >
> > > > Thanks all,
> > > >
> > > > Mathieu
> > > >
> > > >
> > >
> >
> >
> >
> > --
> > -- Guozhang
> >
>



-- 
-- Guozhang

Reply via email to