Hi Jason,

Thanks for clarifying.

So if we are going to continue supporting the empty group id as before 
(with only an addition of a deprecation warning), and disable 
enable.auto.commit for the new default (null) group id on the client side, 
do we really need to bump up the OffsetCommit version?

You mentioned "If an explicit empty string is configured for the group id, 
then maybe we keep the current behavior for compatibility" which makes 
sense to me, but I find it in conflict with your earlier suggestion "we 
just need to bump the OffsetCommit request API and only accept the offset 
commit for older versions.". Maybe I'm missing something?

Thanks!
--Vahid




From:   Jason Gustafson <ja...@confluent.io>
To:     dev <dev@kafka.apache.org>
Date:   07/23/2018 10:52 PM
Subject:        Re: [DISCUSS] KIP-289: Improve the default group id 
behavior in KafkaConsumer



Hey Vahid,

Thanks for the updates. Just to clarify, I was suggesting that we disable
enable.auto.commit only if no explicit group.id is configured. If an
explicit empty string is configured for the group id, then maybe we keep
the current behavior for compatibility. We can log a warning mentioning 
the
deprecation and we can use the old version of the OffsetCommit API that
allows the empty group id. In a later release, we can drop this support in
the client. Does that seem reasonable?

By the way, instead of using the new ILLEGAL_OFFSET_COMMIT error code,
couldn't we use INVALID_GROUP_ID?

Thanks,
Jason



On Mon, Jul 23, 2018 at 5:14 PM, Stanislav Kozlovski 
<stanis...@confluent.io
> wrote:

> Hey Vahid,
>
> No I don't see an issue with it. I believe it to be the best approach.
>
> Best,
> Stanisav
>
> On Mon, Jul 23, 2018 at 12:41 PM Vahid S Hashemian <
> vahidhashem...@us.ibm.com> wrote:
>
> > Hi Stanislav,
> >
> > Thanks for the feedback.
> > Do you see an issue with using `null` as the default group id (as
> > addressed by Jason in his response)?
> > This default group id would not support offset commits and consumers
> would
> > use `auto.offset.reset` config when there is no current offset.
> >
> > Thanks.
> > --Vahid
> >
> >
> >
> >
> > From:   Stanislav Kozlovski <stanis...@confluent.io>
> > To:     dev@kafka.apache.org
> > Date:   07/20/2018 11:09 AM
> > Subject:        Re: [DISCUSS] KIP-289: Improve the default group id
> > behavior in KafkaConsumer
> >
> >
> >
> > I agree with Jason's notion that
> > >  implicit use of the empty group.id to commit offsets is more likely
> to
> > be causing users unexpected problems than actually providing a useful
> > capability.
> > I was initially confused that this is the behavior when investigating 
a
> > new-ish JIRA issue <
> > 
https://issues.apache.org/jira/browse/KAFKA-6758

> > > about
> > the same topic.
> > So, +1 to deprecating "" as a group.id
> >
> > The question after that becomes what the *default* value should be -
> > should
> > we:
> > a) treat an unconfigured group.id consumer as a sort of intermittent
> > consumer where you don't store offsets at all (thereby making the user
> > explicitly sign up for them)
> > b) have a default value which makes use of them? I sort of like the
> > former.
> >
> > @Dhruvil, thinking about it at a high-level - yes. I can't think of a
> > situation where it makes sense to name something an empty string as 
far
> as
> > I'm aware - to me it seems like potential for confusion
> >
> >
> > On Fri, Jul 20, 2018 at 10:22 AM Rajini Sivaram 
<rajinisiva...@gmail.com
> >
> > wrote:
> >
> > > +1 to deprecate use of "" as group.id since it is odd to have a
> resource
> > > name that you cannot set ACLs for. Agree, we have to support older
> > clients
> > > though.
> > >
> > > Thanks,
> > >
> > > Rajini
> > >
> > > On Fri, Jul 20, 2018 at 5:25 PM, Jason Gustafson 
<ja...@confluent.io>
> > > wrote:
> > >
> > > > Hi Vahid,
> > > >
> > > > Sorry for getting to this so late. I think there are two things 
here:
> > > >
> > > > 1. The use of "" as a groupId has always been a dubious practice 
at
> > best.
> > > > We definitely ought to deprecate its use in the client. Perhaps in
> the
> > > next
> > > > major release, we can remove support completely. However, since 
older
> > > > clients depend on it, we may have to continue letting the broker
> > support
> > > it
> > > > to some extent. Perhaps we just need to bump the OffsetCommit 
request
> > API
> > > > and only accept the offset commit for older versions. You probably
> > have
> > > to
> > > > do this anyway if you want to introduce the new error code since 
old
> > > > clients will not expect it.
> > > >
> > > > 2. There should be a way for the consumer to indicate that it has 
no
> > > group
> > > > id and will not commit offsets. This is an explicit instruction 
that
> > the
> > > > consumer should not bother with coordinator lookup and such. We
> > currently
> > > > have some brittle logic in place to let users avoid the 
coordinator
> > > lookup,
> > > > but it is a bit error-prone. I was hoping that we could change the
> > > default
> > > > value of group.id to be null so that the user had to take an
> explicit
> > > > action to opt into coordinator management (groups or offsets).
> > However,
> > > it
> > > > is true that some users may be unknowingly depending on offset
> storage
> > if
> > > > they are using both the default group.id and the default
> > > > enable.auto.commit. Perhaps one option is to disable
> > enable.auto.commit
> > > > automatically if no group.id is specified? I am not sure if there
> are
> > > any
> > > > drawbacks, but my feeling is that implicit use of the empty 
group.id
> > to
> > > > commit offsets is more likely to be causing users unexpected 
problems
> > > than
> > > > actually providing a useful capability.
> > > >
> > > > Thoughts?
> > > >
> > > > Thanks,
> > > > Jason
> > > >
> > > >
> > > >
> > > >
> > > > On Mon, May 28, 2018 at 9:50 AM, Vahid S Hashemian <
> > > > vahidhashem...@us.ibm.com> wrote:
> > > >
> > > > > Hi Viktor,
> > > > >
> > > > > Thanks for sharing your opinion.
> > > > > So you're in favor of disallowing the empty ("") group id
> altogether
> > > > (even
> > > > > for fetching).
> > > > > Given that ideally no one should be using the empty group id (at
> > least
> > > in
> > > > > a production setting) I think the impact would be minimal in 
either
> > > case.
> > > > >
> > > > > But as you said, let's hear what others think and I'd be happy 
to
> > > modify
> > > > > the KIP if needed.
> > > > >
> > > > > Regards.
> > > > > --Vahid
> > > > >
> > > > >
> > > > >
> > > > >
> > > > > From:   Viktor Somogyi <viktorsomo...@gmail.com>
> > > > > To:     dev <dev@kafka.apache.org>
> > > > > Date:   05/28/2018 05:18 AM
> > > > > Subject:        Re: [DISCUSS] KIP-289: Improve the default group 
id
> > > > > behavior in KafkaConsumer
> > > > >
> > > > >
> > > > >
> > > > > Hi Vahid,
> > > > >
> > > > > (with the argument that using the default group id for offset
> commit
> > > > > should not be the user's intention in practice).
> > > > >
> > > > > Yea, so in my opinion too this use case doesn't seem too 
practical.
> > > Also
> > > > I
> > > > > think breaking the offset commit is not smaller from this
> > perspective
> > > > than
> > > > > breaking fetch and offset fetch. If we suppose that someone uses
> the
> > > > > default group id and we break the offset commit then that might 
be
> > > harder
> > > > > to detect than breaking the whole thing altogether. (If we think
> > about
> > > an
> > > > > upgrade situation.)
> > > > > So since we think it is not a practical use case, I think it 
would
> > be
> > > > > better to break altogether but ofc that's just my 2 cents :). 
Let's
> > > > gather
> > > > > other's input as well.
> > > > >
> > > > > Cheers,
> > > > > Viktor
> > > > >
> > > > > On Fri, May 25, 2018 at 5:43 PM, Vahid S Hashemian <
> > > > > vahidhashem...@us.ibm.com> wrote:
> > > > >
> > > > > > Hi Victor,
> > > > > >
> > > > > > Thanks for reviewing the KIP.
> > > > > >
> > > > > > Yes, to minimize the backward compatibility impact, there 
would
> be
> > no
> > > > > harm
> > > > > > in letting a stand-alone consumer consume messages under a ""
> > group
> > > id
> > > > > (as
> > > > > > long as there is no offset commit).
> > > > > > It would have to knowingly seek to an offset or rely on the
> > > > > > auto.offset.reset config for the starting offset.
> > > > > > This way the existing functionality would be preserved for the
> > most
> > > > part
> > > > > > (with the argument that using the default group id for offset
> > commit
> > > > > > should not be the user's intention in practice).
> > > > > >
> > > > > > Does it seem reasonable?
> > > > > >
> > > > > > Thanks.
> > > > > > --Vahid
> > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > > From:   Viktor Somogyi <viktorsomo...@gmail.com>
> > > > > > To:     dev <dev@kafka.apache.org>
> > > > > > Date:   05/25/2018 04:56 AM
> > > > > > Subject:        Re: [DISCUSS] KIP-289: Improve the default 
group
> > id
> > > > > > behavior in KafkaConsumer
> > > > > >
> > > > > >
> > > > > >
> > > > > > Hi Vahid,
> > > > > >
> > > > > > When reading your KIP I coldn't fully understand why did you
> > decide
> > > at
> > > > > > failing with "offset_commit" in case #2? Can't we fail with an
> > empty
> > > > > group
> > > > > > id even in "fetch" or "fetch_offset"? What was the reason for
> > > deciding
> > > > > to
> > > > > > fail at "offset_commit"? Was it because of upgrade 
compatibility
> > > > > reasons?
> > > > > >
> > > > > > Thanks,
> > > > > > Viktor
> > > > > >
> > > > > > On Thu, May 24, 2018 at 12:06 AM, Ted Yu <yuzhih...@gmail.com>
> > > wrote:
> > > > > >
> > > > > > > Looks good to me.
> > > > > > > -------- Original message --------From: Vahid S Hashemian <
> > > > > > > vahidhashem...@us.ibm.com> Date: 5/23/18  11:19 AM
> (GMT-08:00)
> > > To:
> > > > > > > dev@kafka.apache.org Subject: Re: [DISCUSS] KIP-289: Improve
> the
> > > > > default
> > > > > > > group id behavior in KafkaConsumer
> > > > > > > Hi Ted,
> > > > > > >
> > > > > > > Thanks for reviewing the KIP. I updated the KIP and 
introduced
> > an
> > > > > error
> > > > > > > code for the scenario described.
> > > > > > >
> > > > > > > --Vahid
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > From:   Ted Yu <yuzhih...@gmail.com>
> > > > > > > To:     dev@kafka.apache.org
> > > > > > > Date:   04/27/2018 04:31 PM
> > > > > > > Subject:        Re: [DISCUSS] KIP-289: Improve the default
> group
> > id
> > > > > > > behavior in KafkaConsumer
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > bq. If they attempt an offset commit they will receive an
> error.
> > > > > > >
> > > > > > > Can you outline what specific error would be encountered ?
> > > > > > >
> > > > > > > Thanks
> > > > > > >
> > > > > > > On Fri, Apr 27, 2018 at 2:17 PM, Vahid S Hashemian <
> > > > > > > vahidhashem...@us.ibm.com> wrote:
> > > > > > >
> > > > > > > > Hi all,
> > > > > > > >
> > > > > > > > I have drafted a proposal for improving the behavior of
> > > > > KafkaConsumer
> > > > > > > when
> > > > > > > > using the default group id:
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > 
https://cwiki.apache.org/confluence/display/KAFKA/KIP-

> >
> > > > >
> > > > > >
> > > > > > >
> > > > > > > > 289%3A+Improve+the+default+group+id+behavior+in+
> KafkaConsumer
> > > > > > > > The proposal based on the issue and suggestion reported in
> > > > > KAFKA-6774.
> > > > > > > >
> > > > > > > > Your feedback is welcome!
> > > > > > > >
> > > > > > > > Thanks.
> > > > > > > > --Vahid
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > >
> > > > >
> > > > >
> > > > >
> > > > >
> > > >
> > >
> >
> >
> > --
> > Best,
> > Stanislav
> >
> >
> >
> >
> >
>
> --
> Best,
> Stanislav
>




Reply via email to