You are viewing a plain text version of this content. The canonical link for it is here.
Posted to users@kafka.apache.org by Stevo Slavić <ss...@gmail.com> on 2015/07/24 19:11:28 UTC

New consumer - offset one gets in poll is not offset one is supposed to commit

Hello Apache Kafka community,

Say there is only one topic with single partition and a single message on
it.
Result of calling a poll with new consumer will return ConsumerRecord for
that message and it will have offset of 0.

After processing message, current KafkaConsumer implementation expects one
to commit not offset 0 as processed, but to commit offset 1 - next
offset/position one would like to consume.

Does this sound strange to you as well?

Wondering couldn't this offset+1 handling for next position to read been
done in one place, in KafkaConsumer implementation or broker or whatever,
instead of every user of KafkaConsumer having to do it.

Kind regards,
Stevo Slavic.

Re: New consumer - offset one gets in poll is not offset one is supposed to commit

Posted by Jay Kreps <ja...@confluent.io>.
It seems less weird if you think of the offset as the position of the
consumer, i.e. it is "on" record 5. In some sense the consumer is actually
in between records, i.e. if it has processed 4 and not processed 5 do you
think about your position as being on 4 or on 5? Well not on 4 because it
already processed 4, and not on 5 because it hasn't got that yet, it really
is in between. But in english it seems most natural to think of it as being
"on" record 5 in the same sense you'd tell your boss you're working on the
thing you're starting not the thing you most recently completed.

I do agree the advantage of your preferred approach is that the commit
offset is the last record you read. The disadvantage is that the initial
position is now -1, which is also weird.

For log compacted topics this still works fine. Your position remains 6
even if there is no longer a message with offset 6, a fetch request at
position 6 returns messages starting with the first offset after 6. This
would be necessary regardless of the offset scheme.

-Jay

On Tue, Jul 28, 2015 at 6:31 PM, tao xiao <xi...@gmail.com> wrote:

> Correct me if I m wrong. If compaction is used +1 to indicate next offset
> is no longer valid. For the compacted section the offset is not increasing
> sequentially. i think you need to call the next offset of the last
> processed record to figure out what the next offset will be
>
> On Wed, 29 Jul 2015 at 06:16 Stevo Slavić <ss...@gmail.com> wrote:
>
> > Hello Jason,
> >
> > Thanks for reply!
> >
> > About your proposal, in general case it might be helpful. In my case it
> > will not help much - I'm allowing each ConsumerRecord or subset of
> > ConsumerRecords to be processed and ACKed independently and out of HLC
> > process/thread (not to block partition), and then committing largest
> > consecutive ACKed processed offset (+1) since current last committed
> offset
> > per partition.
> >
> > Kind regards,
> > Stevo Slavic.
> >
> > On Mon, Jul 27, 2015 at 6:52 PM, Jason Gustafson <ja...@confluent.io>
> > wrote:
> >
> > > Hey Stevo,
> > >
> > > I agree that it's a little unintuitive that what you are committing is
> > the
> > > next offset that should be read from and not the one that has already
> > been
> > > read. We're probably constrained in that we already have a consumer
> which
> > > implements this behavior. Would it help if we added a method on
> > > ConsumerRecords to get the next offset (e.g. nextOffset(partition))?
> > >
> > > Thanks,
> > > Jason
> > >
> > > On Fri, Jul 24, 2015 at 10:11 AM, Stevo Slavić <ss...@gmail.com>
> > wrote:
> > >
> > > > Hello Apache Kafka community,
> > > >
> > > > Say there is only one topic with single partition and a single
> message
> > on
> > > > it.
> > > > Result of calling a poll with new consumer will return ConsumerRecord
> > for
> > > > that message and it will have offset of 0.
> > > >
> > > > After processing message, current KafkaConsumer implementation
> expects
> > > one
> > > > to commit not offset 0 as processed, but to commit offset 1 - next
> > > > offset/position one would like to consume.
> > > >
> > > > Does this sound strange to you as well?
> > > >
> > > > Wondering couldn't this offset+1 handling for next position to read
> > been
> > > > done in one place, in KafkaConsumer implementation or broker or
> > whatever,
> > > > instead of every user of KafkaConsumer having to do it.
> > > >
> > > > Kind regards,
> > > > Stevo Slavic.
> > > >
> > >
> >
>

Re: New consumer - offset one gets in poll is not offset one is supposed to commit

Posted by tao xiao <xi...@gmail.com>.
Correct me if I m wrong. If compaction is used +1 to indicate next offset
is no longer valid. For the compacted section the offset is not increasing
sequentially. i think you need to call the next offset of the last
processed record to figure out what the next offset will be

On Wed, 29 Jul 2015 at 06:16 Stevo Slavić <ss...@gmail.com> wrote:

> Hello Jason,
>
> Thanks for reply!
>
> About your proposal, in general case it might be helpful. In my case it
> will not help much - I'm allowing each ConsumerRecord or subset of
> ConsumerRecords to be processed and ACKed independently and out of HLC
> process/thread (not to block partition), and then committing largest
> consecutive ACKed processed offset (+1) since current last committed offset
> per partition.
>
> Kind regards,
> Stevo Slavic.
>
> On Mon, Jul 27, 2015 at 6:52 PM, Jason Gustafson <ja...@confluent.io>
> wrote:
>
> > Hey Stevo,
> >
> > I agree that it's a little unintuitive that what you are committing is
> the
> > next offset that should be read from and not the one that has already
> been
> > read. We're probably constrained in that we already have a consumer which
> > implements this behavior. Would it help if we added a method on
> > ConsumerRecords to get the next offset (e.g. nextOffset(partition))?
> >
> > Thanks,
> > Jason
> >
> > On Fri, Jul 24, 2015 at 10:11 AM, Stevo Slavić <ss...@gmail.com>
> wrote:
> >
> > > Hello Apache Kafka community,
> > >
> > > Say there is only one topic with single partition and a single message
> on
> > > it.
> > > Result of calling a poll with new consumer will return ConsumerRecord
> for
> > > that message and it will have offset of 0.
> > >
> > > After processing message, current KafkaConsumer implementation expects
> > one
> > > to commit not offset 0 as processed, but to commit offset 1 - next
> > > offset/position one would like to consume.
> > >
> > > Does this sound strange to you as well?
> > >
> > > Wondering couldn't this offset+1 handling for next position to read
> been
> > > done in one place, in KafkaConsumer implementation or broker or
> whatever,
> > > instead of every user of KafkaConsumer having to do it.
> > >
> > > Kind regards,
> > > Stevo Slavic.
> > >
> >
>

Re: New consumer - offset one gets in poll is not offset one is supposed to commit

Posted by Stevo Slavić <ss...@gmail.com>.
Hello Jason,

Thanks for reply!

About your proposal, in general case it might be helpful. In my case it
will not help much - I'm allowing each ConsumerRecord or subset of
ConsumerRecords to be processed and ACKed independently and out of HLC
process/thread (not to block partition), and then committing largest
consecutive ACKed processed offset (+1) since current last committed offset
per partition.

Kind regards,
Stevo Slavic.

On Mon, Jul 27, 2015 at 6:52 PM, Jason Gustafson <ja...@confluent.io> wrote:

> Hey Stevo,
>
> I agree that it's a little unintuitive that what you are committing is the
> next offset that should be read from and not the one that has already been
> read. We're probably constrained in that we already have a consumer which
> implements this behavior. Would it help if we added a method on
> ConsumerRecords to get the next offset (e.g. nextOffset(partition))?
>
> Thanks,
> Jason
>
> On Fri, Jul 24, 2015 at 10:11 AM, Stevo Slavić <ss...@gmail.com> wrote:
>
> > Hello Apache Kafka community,
> >
> > Say there is only one topic with single partition and a single message on
> > it.
> > Result of calling a poll with new consumer will return ConsumerRecord for
> > that message and it will have offset of 0.
> >
> > After processing message, current KafkaConsumer implementation expects
> one
> > to commit not offset 0 as processed, but to commit offset 1 - next
> > offset/position one would like to consume.
> >
> > Does this sound strange to you as well?
> >
> > Wondering couldn't this offset+1 handling for next position to read been
> > done in one place, in KafkaConsumer implementation or broker or whatever,
> > instead of every user of KafkaConsumer having to do it.
> >
> > Kind regards,
> > Stevo Slavic.
> >
>

Re: New consumer - offset one gets in poll is not offset one is supposed to commit

Posted by Jason Gustafson <ja...@confluent.io>.
Hey Stevo,

I agree that it's a little unintuitive that what you are committing is the
next offset that should be read from and not the one that has already been
read. We're probably constrained in that we already have a consumer which
implements this behavior. Would it help if we added a method on
ConsumerRecords to get the next offset (e.g. nextOffset(partition))?

Thanks,
Jason

On Fri, Jul 24, 2015 at 10:11 AM, Stevo Slavić <ss...@gmail.com> wrote:

> Hello Apache Kafka community,
>
> Say there is only one topic with single partition and a single message on
> it.
> Result of calling a poll with new consumer will return ConsumerRecord for
> that message and it will have offset of 0.
>
> After processing message, current KafkaConsumer implementation expects one
> to commit not offset 0 as processed, but to commit offset 1 - next
> offset/position one would like to consume.
>
> Does this sound strange to you as well?
>
> Wondering couldn't this offset+1 handling for next position to read been
> done in one place, in KafkaConsumer implementation or broker or whatever,
> instead of every user of KafkaConsumer having to do it.
>
> Kind regards,
> Stevo Slavic.
>