Hey Vinay,

Any chance you can run the same test against trunk? I'm guessing this might
be caused by a bug in the 0.9 consumer which basically causes some requests
to fail when a bunch of them are sent to the broker at the same time.

-Jason

On Wed, Apr 27, 2016 at 1:02 PM, vinay sharma <vinsharma.t...@gmail.com>
wrote:

> Hi Jason,
>
> This makes sense.We use 0.9.0.1 and we do have session timeout set a bit
> high but nothing can guarantee that there will be no case when processing
> may not go higher than session timeout. I am trying to test a proactive
> commit approach to handle such cases when processing takes unusually long
> time. To keep consumer's session alive during long processing time i
> proactively commitSync processed records every 15 seconds. Session timeout
> i kept is 30000.
>
> *Problem:-*
> With heart beat interval is 3000 then i expect a hearbeat request to be
> sent on each proactive commit which happens every 15 seconds. In my tests i
> see that this does not happen always. I see a time window which is greater
> than 30 seconds where no hearbeat is sent even thought there were commits
> in this duration. After this window i see a couple of successful heartbeat
> responses till the end of poll but as soon as i poll again and call
> commitSync in next poll i get "ILLEGAL_GENERATION" error. This error always
> happen just after meta refresh or in next poll processing after a meta
> refresh. I am attaching logs where i kept meta refresh interval 40000,
> 90000, 500000.
>
> *Test results *:-
> Test with meta refresh 40000 ms ran around 70 seconds from 1st poll.
> Test with meta refresh 90000 ms ran around 120 seconds from 1st poll.
> Test with meta refresh 500000 ms ran around 564 seconds from 1st poll.
>
> Every test falls in line with above test cases where generation is marked
> dead some time after a meta refresh. Meta refresh before 1st poll does not
> create any issue but the ones after poll and during long processing do.
>
> *Environment:-*
> My setup has 3 brokers 1 zk. Topic has 3 partitions ans has replication
> factor 3. Messages are already published to topic.
>
> *Logic used in test cases :- *
> On each poll I initialize a map with current committed offset position of
> partitions being consumed. I update this map after each record processing
> and use this map to proactively commit every 15 seconds. Map is initialized
> again after a proactive commit.
>
> I am not sure what is wrong here but i do not see any issue in code or
> offset commits going on. Log files and a class with main method are
> attached for your reference.
>
> Regards,
> Vinay Sharma
>
>
>
> On Wed, Apr 27, 2016 at 2:46 PM, Jason Gustafson <ja...@confluent.io>
> wrote:
>
>> Hi Vinay,
>>
>> Answers below:
>>
>> 1)  Is it correct to say that each commitSync will trigger a
>> HeartBeatTask?
>> > If there is no hear beat sent in past since specified heartbeat interval
>> > then i should see a successful heartbeat response or failure message in
>> > logs near to commitSync success log?
>>
>>
>> Not quite. Heartbeats are sent periodically according to the
>> heartbeat.interval.ms configuration. However, since the consumer has no
>> background thread, they can only be sent in API calls such as poll() or
>> commitSync(). So calling commitSync() may or may not result in a heartbeat
>> depending only on whether one is "due."
>>
>> 2) is it correct to say that Meta Data refresh will not act as heartbeat,
>> > will not trigger heartBeatTask and will not reset heartBeatTask?
>>
>>
>> That is correct. Metadata refreshes are not related to heartbeats.
>>
>> 3) Where does a consumer session maintained? Lets say my consumer is
>> > listening to 3 partitions on a 3 broker cluster where each broker is
>> leader
>> > of 1 partition. So will each of the brokers will have a session for my
>> > consumer or is it just 1 session maintained somewhere in common like
>> > zookeeper?
>>
>>
>> One of the brokers serves as the "group coordinator." When the consumer
>> starts up, it sends a GroupCoordinator request to one of the brokers to
>> find out who the coordinator is. Currently, coordinators are chosen from
>> among the leaders of the partitions of the __consumer_offsets topic. This
>> lets us take advantage of the leader election process to also handle
>> coordinator failures. The coordinator of each group maintains state for
>> the
>> group and keeps track of session timeouts.
>>
>> 4) In above setup, during a long processing if I commit a record through
>> > commmitSync which triggers a hear beat request and a successful
>> response is
>> > received for the same then what does this response means? does it mean
>> that
>> > my session with each broker is renewed? or does it mean that just the
>> > leader for partition of committed record knows that my consumer is alive
>> > and consumer's session on other brokers will still timeout?
>>
>>
>> The coordinator is the only broker that is aware of a consumer's session
>> and all offset commits are sent to it. Successful heartbeats mean that the
>> session is still active. Heartbeats are also used to let the consumer
>> discover when a rebalance has begun. If a new member joins the group, then
>> the coordinator returns an error code in the heartbeat responses of the
>> active members to let them know that they need to rejoin the group so that
>> partitions can be rebalanced.
>>
>> I wouldn't get too hung up on commit/heartbeat behavior. The crux of the
>> issue is that you need to call poll() often enough to avoid getting timed
>> out by the coordinator. If you find this happening frequently, you
>> probably
>> need to increase session.timeout.ms. There's not really any downside to
>> doing so other than that hard failures (in which the consumer can't be
>> shutdown cleanly) will take a little longer to detect. Normal shutdown
>> doesn't have this problem. It can be difficult in 0.9 to ensure that
>> poll()
>> is called often enough since you don't have direct control over the amount
>> of data returned in poll(), but we're adding an option (max.poll.records)
>> in 0.10 which hopefully can be set conservatively enough to make this
>> problem go away.
>>
>> -Jason
>>
>> On Wed, Apr 27, 2016 at 7:11 AM, vinay sharma <vinsharma.t...@gmail.com>
>> wrote:
>>
>> > Hey,
>> >
>> > I am working on a simplified test case to check if there is any issue
>> in my
>> > code. Just to make sure that any of my assumptions are not wrong, it
>> will
>> > be great if you can please help me in finding answers to following
>> > queries:-
>> >
>> > 1)  Is it correct to say that each commitSync will trigger a
>> HeartBeatTask?
>> > If there is no hear beat sent in past since specified heartbeat interval
>> > then i should see a successful heartbeat response or failure message in
>> > logs near to commitSync success log?
>> > 2) is it correct to say that Meta Data refresh will not act as
>> heartbeat,
>> > will not trigger heartBeatTask and will not reset heartBeatTask?
>> > 3) Where does a consumer session maintained? Lets say my consumer is
>> > listening to 3 partitions on a 3 broker cluster where each broker is
>> leader
>> > of 1 partition. So will each of the brokers will have a session for my
>> > consumer or is it just 1 session maintained somewhere in common like
>> > zookeeper?
>> > 4) In above setup, during a long processing if I commit a record through
>> > commmitSync which triggers a hear beat request and a successful
>> response is
>> > received for the same then what does this response means? does it mean
>> that
>> > my session with each broker is renewed? or does it mean that just the
>> > leader for partition of committed record knows that my consumer is alive
>> > and consumer's session on other brokers will still timeout?
>> >
>> > Regards,
>> > Vinay Sharma
>> >
>> > On Tue, Apr 26, 2016 at 2:38 PM, Jason Gustafson <ja...@confluent.io>
>> > wrote:
>> >
>> > > Hey Vinay,
>> > >
>> > > Are you saying that heartbeats are not sent while a metadata refresh
>> is
>> > in
>> > > progress? Do you have any logs which show us the apparent problem?
>> > >
>> > > Thanks,
>> > > Jason
>> > >
>> > > On Tue, Apr 26, 2016 at 8:18 AM, vinay sharma <
>> vinsharma.t...@gmail.com>
>> > > wrote:
>> > >
>> > > > Hi Ismael,
>> > > >
>> > > > Treating commitSync as heartbeat will definitely resolve the issue
>> i am
>> > > > facing but the reason behind my issue does not seem to be what
>> > mentioned
>> > > in
>> > > > defect (i.e frequent commitSync requests).
>> > > >
>> > > > I am sending CommitSync periodically only to keep my session alive
>> when
>> > > my
>> > > > consumer is still processing records and is close to session time
>> out
>> > > > (tried 10th / 12th / 15th / 20th second after poll called where
>> session
>> > > > time is 30). I see heartbeat response received in logs along with
>> each
>> > > > commitSync call but this stops after a meta data refresh request is
>> > > issued.
>> > > > I see in logs that commit goes successful but no heartbeat response
>> > > > received message in logs after meta refresh till next poll.
>> > > >
>> > > > Regards,
>> > > > Vinay Sharma
>> > > >
>> > > > On Mon, Apr 25, 2016 at 5:06 PM, Ismael Juma <ism...@juma.me.uk>
>> > wrote:
>> > > >
>> > > > > Hi Vinay,
>> > > > >
>> > > > > This was fixed via
>> https://issues.apache.org/jira/browse/KAFKA-3470
>> > > > (will
>> > > > > be part of 0.10.0.0).
>> > > > >
>> > > > > Ismael
>> > > > >
>> > > > >
>> > > > >
>> > > > > On Mon, Apr 25, 2016 at 1:52 PM, vinay sharma <
>> > > vinsharma.t...@gmail.com>
>> > > > > wrote:
>> > > > >
>> > > > > > Hello,
>> > > > > >
>> > > > > > I am using client API 0.9.0.1 and facing an issue. As per my
>> logs
>> > it
>> > > > > seems
>> > > > > > that on each commitSync(Offsets) a heartbeat request is sent but
>> > > after
>> > > > a
>> > > > > > metada refresh request till next poll(), commits do not send any
>> > > > hearbeat
>> > > > > > request.
>> > > > > >
>> > > > > > KafkaConsumers i create sometimes get session time out due to no
>> > > > hearbeat
>> > > > > > specially during longer processing times. I call
>> > CommitSync(offsets)
>> > > > > after
>> > > > > > regular intervals to keep session alive when processing takes
>> > longer
>> > > > than
>> > > > > > usual. Every thing works fine if commit intervals are very
>> small or
>> > > if
>> > > > i
>> > > > > > commit after each record but if i commit lets say every 12
>> seconds
>> > > and
>> > > > 30
>> > > > > > seconds is session time then i can see consumer getting timed
>> out
>> > > > > > sometimes.
>> > > > > >
>> > > > > > Any help or pointers will be much appreciated. Thanks in
>> advance.
>> > > > > >
>> > > > > > Regards,
>> > > > > > Vinay sharma
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>>
>
>

Reply via email to