Hi Jason, Stanislav,

Thanks for your feedback. I'll update the KIP later today based on your 
suggestion.
I'll send a note once the update is complete.

Thanks.
--Vahid



From:   Stanislav Kozlovski <stanis...@confluent.io>
To:     dev@kafka.apache.org
Date:   08/03/2018 02:34 AM
Subject:        Re: [DISCUSS] KIP-289: Improve the default group id 
behavior in KafkaConsumer



Hi,

I also believe the minimal change might be better. In my opinion, the less
logic we can stuff in the broker - the better. The use ofempty group.id is
not ideal and should be discouraged, but I don't inherently believe strict
validation in the broker is useful, since it doesn't break anything
internal (except maybe not being able to set ACLs).
Having sufficient warnings or even restriction on the clients I feel might
be a good enough compromise.

That being said, I very much look forward to voting for this KIP.

Thanks,
Stanislav

On Fri, Aug 3, 2018 at 7:51 AM Jason Gustafson <ja...@confluent.io> wrote:

> Hey Vahid,
>
> I think we're in agreement. Perhaps since none of us feel too strongly, 
we
> should go for the more minimal change? I'll vote for the KIP either way 
as
> long as we can change the default group.id.
>
> Thanks,
> Jason
>
> On Thu, Aug 2, 2018 at 11:55 AM, Vahid S Hashemian <
> vahidhashem...@us.ibm.com> wrote:
>
> > Hi Jason, Colin,
> >
> > Thanks for your feedback.
> >
> > If we are going to keep supporting the empty group id like before then 
it
> > makes sense to do what you suggested.
> >
> > Though, IMHO if using the empty group id for committing offsets is a 
bad
> > practice we should stop supporting it at some point; and it would be
> > better to do it sooner before potentially more users adopt that.
> > I understand that changing the default would reduce the chance of 
someone
> > using it, but it doesn't stop them. Note that we did not receive any
> > objections to dropping the support even right-away (a good sign).
> > We could keep supporting the old API version in case someone must use
> this
> > group id like before. But at least we won't expose new users to this 
bad
> > practice.
> >
> > I, too, do now have a strong opinion about this. So if the current KIP
> > should not move forward as is, I'm okay with just changing the default
> and
> > deprecating the use on the client, as you suggested.
> >
> > Thanks.
> > --Vahid
> >
> >
> >
> > From:   Colin McCabe <cmcc...@apache.org>
> > To:     dev@kafka.apache.org
> > Date:   08/02/2018 11:23 AM
> > Subject:        Re: [DISCUSS] KIP-289: Improve the default group id
> > behavior in KafkaConsumer
> >
> >
> >
> > Thanks, Jason.  I don't have a very strong opinion on this.  But like 
you
> > said, if we skip bumping the RPC versions, this would be a smaller
> change,
> > which might be good.
> >
> > best,
> > Colin
> >
> >
> > On Wed, Aug 1, 2018, at 17:43, Jason Gustafson wrote:
> > > Hey Vahid,
> > >
> > > I talked with Colin offline. I think specifically he felt the 
version
> > bump
> > > on the broker was overkill since the broker still has to support the
> > empty
> > > group id for older versions. I had thought that eventually we would 
be
> > able
> > > to remove those old versions, but it's true that this may not happen
> > until
> > > indefinitely far in the future. I think the main improvement here is
> > > changing the default group.id to null instead of "". I could go 
either
> > way
> > > on whether bumping the protocol is useful. I do think it is helpful
> > though
> > > to signal clearly that it its use is deprecated and discouraged,
> > especially
> > > in light of the ACL problem. I guess we could just deprecate the use 
on
> > the
> > > client. What do you think?
> > >
> > > Thanks,
> > > Jason
> > >
> > > On Wed, Aug 1, 2018 at 3:19 PM, Vahid S Hashemian
> > <vahidhashem...@us.ibm.com
> > > > wrote:
> > >
> > > > Thanks Jason for responding to Colin's concerns.
> > > >
> > > > If there are no other comment / feedback / objection I'll start a
> vote
> > > > soon.
> > > >
> > > > Thanks.
> > > > --Vahid
> > > >
> > > >
> > > >
> > > > From:   Jason Gustafson <ja...@confluent.io>
> > > > To:     dev <dev@kafka.apache.org>
> > > > Date:   07/27/2018 10:38 AM
> > > > Subject:        Re: [DISCUSS] KIP-289: Improve the default group 
id
> > > > behavior in KafkaConsumer
> > > >
> > > >
> > > >
> > > > Hey Colin,
> > > >
> > > > The problem is both that the empty group id is the default value 
and
> > that
> > > > it is actually accepted by the broker for offset commits. Combine
> that
> > > > with
> > > > the fact that auto commit is enabled by default and you users get
> > > > surprising behavior. If you look at a random Kafka cluster, you'll
> > > > probably
> > > > find a bunch of inadvertent offset commits for the empty group id. 
I
> > was
> > > > hoping we could distinguish between users who are using the empty
> > group id
> > > > as an accident of the default configuration and those who use it
> > > > intentionally. By default, there will be no group id and the 
consumer
> > will
> > > > not commit offsets. If a user has actually intentionally used the
> > empty
> > > > group id, however, it will continue to work. I actually think 
there
> > are
> > > > probably very few people doing this (maybe even no one), but I
> thought
> > we
> > > > might err on the side of compatibility.
> > > >
> > > > The big incompatible change here is having brokers reject using
> > > > assign(...)
> > > > > with empty / null group.id.
> > > >
> > > >
> > > > This is not correct. In the proposal, the broker will only reject 
the
> > > > empty
> > > > group id for the new version of OffsetCommit. Older clients, which
> > cannot
> > > > be changed, will continue to work because the old versions of the
> > > > OffsetCommit API still accept the empty group id. The null group 
id
> is
> > > > different from the empty group id: it is not allowed in any 
version
> of
> > the
> > > > API. It is basically a way to indicate that the consumer has no
> > dependence
> > > > on the coordinator at all, which we actually have a surprising 
number
> > of
> > > > use cases for. Furthermore, if a user has an actual need for the
> empty
> > > > group id, it will still be allowed. We are just deprecating it.
> > > >
> > > > -Jason
> > > >
> > > > On Fri, Jul 27, 2018 at 9:56 AM, Colin McCabe <cmcc...@apache.org>
> > wrote:
> > > >
> > > > > Sorry if this is a silly question, but what's the rationale for
> > > > switching
> > > > > to using null for the default group id, rather than the empty
> > string?
> > > > > Continuing to use the empty string seems like less churn.  And
> after
> > > > all,
> > > > > we're not using the empty string group name for anything else.
> > > > >
> > > > > The big incompatible change here is having brokers reject using
> > > > > assign(...) with empty / null group.id.  If I understand
> correctly,
> > the
> > > > > KIP proposes that this change be made on the brokers on the next
> > > > > incompatible Kafka release.  But that has nothing to do with 
client
> > > > > versions.  Why not just have a broker config which controls 
this?
> > Maybe
> > > > "
> > > > > allow.assign.empty.group.id", or something like that.  At first,
> the
> > > > > default will be true, and then eventually we can flip it over to
> > false.
> > > > >
> > > > > It seems like the main rationale for tying this behavior to the
> > Kafka
> > > > > client version is to force people to stop using the empty group 
id
> > so
> > > > that
> > > > > they can upgrade their clients.  But it's also possible that 
people
> > will
> > > > > stop upgrading their Kafka clients instead.  That would be 
pretty
> > > > negative
> > > > > since  they'd miss out on any efficiency and feature 
improvements
> in
> > the
> > > > > new clients and eventually have to do more protocol downgrading,
> > etc.
> > > > >
> > > > > best,
> > > > > Colin
> > > > >
> > > > >
> > > > > On Thu, Jul 26, 2018, at 11:50, Vahid S Hashemian wrote:
> > > > > > Hi Jason,
> > > > > >
> > > > > > That makes sense.
> > > > > > I have updated the KIP based on the recent feedback.
> > > > > >
> > > > > > Thanks!
> > > > > > --Vahid
> > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > > From:   Jason Gustafson <ja...@confluent.io>
> > > > > > To:     dev <dev@kafka.apache.org>
> > > > > > Date:   07/25/2018 02:23 PM
> > > > > > Subject:        Re: [DISCUSS] KIP-289: Improve the default 
group
> > id
> > > > > > behavior in KafkaConsumer
> > > > > >
> > > > > >
> > > > > >
> > > > > > Hi Vahid,
> > > > > >
> > > > > > I was thinking we'd only use the old API version if we had to.
> > That
> > > > is,
> > > > > > only if the user has explicitly configured "" as the group.id.
> > > > > Otherwise,
> > > > > > we'd just use the new one. Another option is to just drop 
support
> > in
> > > > the
> > > > > > client for the empty group id, but usually we allow a 
deprecation
> > > > period
> > > > > > for changes like this.
> > > > > >
> > > > > > -Jason
> > > > > >
> > > > > > On Wed, Jul 25, 2018 at 12:49 PM, Vahid S Hashemian <
> > > > > > vahidhashem...@us.ibm.com> wrote:
> > > > > >
> > > > > > > Hi Jason,
> > > > > > >
> > > > > > > Thanks for additional clarification.
> > > > > > >
> > > > > > > So the next version of the OffsetCommit API will return an
> > > > > > > INVALID_GROUP_ID error for empty group ids; but on the 
client
> > side
> > > > we
> > > > > > call
> > > > > > > the older version of the client until the next major 
release.
> > > > > > > The table below should summarize this.
> > > > > > >
> > > > > > > +-----------------------------------------------------+
> > > > > > >                   |                 Client (group.id="") |
> > > > > > > +-----------------------------------------------------+
> > > > > > >                   | pre-2.1 |           2.1          |
>  3.0
> > |
> > > > > > >
> > > > > > +-----+-----------+---------+------------------------+------
> > > > > ------------+
> > > > > > > |     | V5 (cur.) | works   | works                  | works 
|
> > > > > > > + API
> > > > > >
> > +-----------+---------+------------------------+------------------+
> > > > > > > |     | V6        | N/A     | N/A (calls V5/warning) |
> > > > > INVALID_GROUP_ID
> > > > > > |
> > > > > > >
> > > > > > +-----+-----------+---------+------------------------+------
> > > > > ------------+
> > > > > > >
> > > > > > > Assumptions:
> > > > > > > * 2.1: The target release version for this KIP
> > > > > > > * 3.0: The next major release
> > > > > > >
> > > > > > > Please advise if you see an issue; otherwise, I'll update 
the
> > KIP
> > > > > > > accordingly.
> > > > > > >
> > > > > > > Thanks!
> > > > > > > --Vahid
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > From:   Jason Gustafson <ja...@confluent.io>
> > > > > > > To:     dev <dev@kafka.apache.org>
> > > > > > > Date:   07/25/2018 12:08 AM
> > > > > > > Subject:        ***UNCHECKED*** Re: [DISCUSS] KIP-289: 
Improve
> > the
> > > > > > default
> > > > > > > group id        behavior in KafkaConsumer
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > Hey Vahid,
> > > > > > >
> > > > > > > Sorry for the confusion. I think we all agree that going
> > forward, we
> > > > > > > shouldn't support the empty group id, so the question is 
just
> > around
> > > > > > > compatibility. I think we have to bump the OffsetCommit API
> > version
> > > > so
> > > > > > > that
> > > > > > > old clients which are unknowingly depending on the default
> empty
> > > > group
> > > > > > id
> > > > > > > will continue to work with new brokers. For new versions of 
the
> > > > > client,
> > > > > > we
> > > > > > > can either drop support for the empty group id immediately 
or
> we
> > can
> > > > > > give
> > > > > > > users a grace period. I was thinking we would do the latter. 
We
> > can
> > > > > > change
> > > > > > > the default group.id, but in the case that a user has
> explicitly
> > > > > > > configured
> > > > > > > the empty group, then we can just use an old version of the
> > > > > OffsetCommit
> > > > > > > API which still supports it. In a future release, we can 
drop
> > this
> > > > > > support
> > > > > > > and only use the latest OffsetCommit version. Does that make
> > sense?
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Jason
> > > > > > >
> > > > > > >
> > > > > > > On Tue, Jul 24, 2018 at 12:36 PM, Vahid S Hashemian <
> > > > > > > vahidhashem...@us.ibm.com> wrote:
> > > > > > >
> > > > > > > > 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
> > > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > >
> > > >
> > > >
> > > >
> > > >
> > > >
> >
> >
> >
> >
> >
> >
>


-- 
Best,
Stanislav




Reply via email to