Thanks Mickael, +1 from me. Just a couple more comments:
> Yes mapping it to a friendlier error would be nice but I'm not sure if we can do that because UNKNOWN_MEMBER_ID is also returned if the group is dead. I think the logic was chanced recently to return `COORDINATOR_NOT_AVAILABLE` if the group is Dead, so I don't think this is a problem, at least not for new versions. One thing to keep in mind is that the Dead state is just a transient state while we're in the process of unloading the group. It should be rare that we hit it. For old brokers which still return UNKNOWN_MEMBER_ID in this state, I don't think it would be so bad to return a NotEmptyGroup error. Basically we have to throw something in this case, so we may as well throw something which indicates the most likely problem. 4. I assume the default isolation level for listOffsets would be read_uncommitted? 5. I don't feel too strongly about it, but other admin APIs have tended to use "alter" for updates (e.g. `alterConfigs` and `alterReplicaLogDirs`). You might consider `alterConsumerGroupOffsets` over `resetConsumerGroupOffsets`. -Jason On Wed, Aug 7, 2019 at 10:15 AM Mickael Maison <mickael.mai...@gmail.com> wrote: > Hi Jason, > > Thanks for the feedback > > 1. Yes listOffsets() should be able to retrieve earliest, latest and > by-timestamp offsets. > I agree, it's better to avoid exposing magic values. I've updated the > KIP to use an OffsetSpec object as suggested. > > 2. Yes let's expose the leader epoch in ListOffsetsResultInfo. > > 3. Yes mapping it to a friendlier error would be nice but I'm not sure > if we can do that because UNKNOWN_MEMBER_ID is also returned if the > group is dead. > > On Tue, Aug 6, 2019 at 6:38 PM Jason Gustafson <ja...@confluent.io> wrote: > > > > Thanks for the KIP. This makes sense to me. Just a couple small comments: > > > > 1. Can the listOffsets API be used to get the start and end offsets? In > the > > consumer, we use separate APIs for this: `beginningOffsets` and > > `endOffsets` to avoid the need for sentinels. An alternative would be to > > introduce an `OffsetSpec` (or maybe `OffsetQuery`) object to customize > the > > query. For example: > > > > public ListOffsetsResult listOffsets(Map<TopicPartition, OffsetSpec> > > partitionOffsetSpecs) > > > > The benefit is that we can avoid sentinel values and we have an extension > > point for additional query options in the future. What do you think? > > > > 2. The ListOffset response includes the leader epoch corresponding to the > > offset that was found. This is useful for finer-grained reasoning about > the > > log. We expose this in the consumer in the OffsetAndTimestamp object > which > > is returned from `offsetsForTimes`. Does it make sense to add this to > > `ListOffsetsResultInfo` as well? > > > > 3. If the group is still active, the call to reset offsets will fail. > > Currently this would result in an UNKNOWN_MEMBER_ID error. I think it > would > > make sense to map this exception to a friendlier error before raising to > > the user. For example, `NonEmptyGroupException` or something like that. > > > > -Jason > > > > > > > > > > > > On Tue, Aug 6, 2019 at 9:33 AM Mickael Maison <mickael.mai...@gmail.com> > > wrote: > > > > > Hi Colin, > > > > > > Thank you for taking a look! > > > I agree, being able to set consumer group offsets via the AdminClient > > > would be really useful, hence I created this KIP. > > > > > > With the total absence of binding votes, I guessed I needed to make > > > some changes. Do you mean you preferred the previous naming > > > (commitConsumerGroupOffsets) over "resetConsumerGroupOffsets"? > > > > > > Thanks > > > > > > On Mon, Aug 5, 2019 at 8:26 PM Colin McCabe <cmcc...@apache.org> > wrote: > > > > > > > > I think it would be useful to have this in AdminClient. Especially > if > > > we implement KIP-496: Administrative API to delete consumer offsets. > It > > > would be odd to have a way to delete consumer offsets in AdminClient, > but > > > not to create them. What do you think? > > > > > > > > best, > > > > Colin > > > > > > > > > > > > On Sun, Aug 4, 2019, at 09:27, Mickael Maison wrote: > > > > > Hi, > > > > > > > > > > In an attempt to unblock this KIP, I've made some adjustments: > > > > > I've renamed the commitConsumerGroupOffsets() methods to > > > > > resetConsumerGroupOffsets() to reduce confusion. That should better > > > > > highlight the differences with the regular commit() operation from > the > > > > > Consumer API. I've also added some details to the motivation > section. > > > > > > > > > > So we have +5 non binding votes and 0 binding votes > > > > > > > > > > On Mon, Mar 25, 2019 at 1:10 PM Mickael Maison < > > > mickael.mai...@gmail.com> wrote: > > > > > > > > > > > > Bumping this thread once again > > > > > > > > > > > > Ismael, have I answered your questions? > > > > > > While this has received a few non-binding +1s, no committers have > > > > > > voted yet. If you have concerns or questions, please let me know. > > > > > > > > > > > > Thanks > > > > > > > > > > > > On Mon, Feb 11, 2019 at 11:51 AM Mickael Maison > > > > > > <mickael.mai...@gmail.com> wrote: > > > > > > > > > > > > > > Bumping this thread as it's been a couple of weeks. > > > > > > > > > > > > > > On Tue, Jan 22, 2019 at 2:26 PM Mickael Maison < > > > mickael.mai...@gmail.com> wrote: > > > > > > > > > > > > > > > > Thanks Ismael for the feedback. I think your point has 2 > parts: > > > > > > > > - Having the reset functionality in the AdminClient: > > > > > > > > The fact we have a command line tool illustrate that this > > > operation is > > > > > > > > relatively common. I seems valuable to be able to perform > this > > > > > > > > operation directly via a proper API in addition of the CLI > tool. > > > > > > > > > > > > > > > > - Sending an OffsetCommit directly instead of relying on > > > KafkaConsumer: > > > > > > > > The KafkaConsumer requires a lot of stuff to commit offsets. > Its > > > group > > > > > > > > cannot change so you need to start a new Consumer every time, > > > that > > > > > > > > creates new connections and overal sends more requests. Also > > > there are > > > > > > > > already a bunch of AdminClient APIs that have logic very > close > > > to > > > > > > > > what needs to be done to send a commit request, keeping the > code > > > small > > > > > > > > and consistent. > > > > > > > > > > > > > > > > I've updated the KIP with these details and moved the 2nd > part to > > > > > > > > "Proposed changes" as it's more an implementation detail. > > > > > > > > > > > > > > > > I hope this answers your question > > > > > > > > > > > > > > > > On Mon, Jan 21, 2019 at 7:41 PM Ismael Juma < > isma...@gmail.com> > > > wrote: > > > > > > > > > > > > > > > > > > The KIP doesn't discuss the option of using KafkaConsumer > > > directly as far > > > > > > > > > as I can tell. We have tried to avoid having the same > > > functionality in > > > > > > > > > multiple clients so it would be good to explain why this is > > > necessary here > > > > > > > > > (not saying it isn't). > > > > > > > > > > > > > > > > > > Ismael > > > > > > > > > > > > > > > > > > On Mon, Jan 21, 2019, 10:29 AM Mickael Maison < > > > mickael.mai...@gmail.com > > > > > > > > > wrote: > > > > > > > > > > > > > > > > > > > Thanks Ryanne for the feedback, all suggestions sounded > > > good, I've > > > > > > > > > > updated the KIP accordingly. > > > > > > > > > > > > > > > > > > > > On Mon, Jan 21, 2019 at 3:43 PM Ryanne Dolan < > > > ryannedo...@gmail.com> > > > > > > > > > > wrote: > > > > > > > > > > > > > > > > > > > > > > +1 (non-binding) > > > > > > > > > > > > > > > > > > > > > > But I suggest: > > > > > > > > > > > > > > > > > > > > > > - drop "get" from getOffset, getTimestamp. > > > > > > > > > > > > > > > > > > > > > > - add to the motivation section why this is better than > > > constructing a > > > > > > > > > > > KafkaConsumer and using seek(), commit() etc. > > > > > > > > > > > > > > > > > > > > > > - add some rejected alternatives. > > > > > > > > > > > > > > > > > > > > > > Ryanne > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > On Mon, Jan 21, 2019, 7:57 AM Dongjin Lee < > > > dong...@apache.org wrote: > > > > > > > > > > > > > > > > > > > > > > > We have +4 non-binding for this vote. Is there any > > > committer who is > > > > > > > > > > > > interested in this issue? > > > > > > > > > > > > > > > > > > > > > > > > Thanks, > > > > > > > > > > > > Dongjin > > > > > > > > > > > > > > > > > > > > > > > > On Mon, Jan 21, 2019 at 10:33 PM Andrew Schofield < > > > > > > > > > > > > andrew_schofi...@live.com> > > > > > > > > > > > > wrote: > > > > > > > > > > > > > > > > > > > > > > > > > +1 (non-binding). Thanks for the KIP. > > > > > > > > > > > > > > > > > > > > > > > > > > On 21/01/2019, 12:45, "Eno Thereska" < > > > eno.there...@gmail.com> > > > > > > > > > > wrote: > > > > > > > > > > > > > > > > > > > > > > > > > > +1 (non binding). Thanks. > > > > > > > > > > > > > > > > > > > > > > > > > > On Mon, Jan 21, 2019 at 12:30 PM Mickael > Maison < > > > > > > > > > > > > > mickael.mai...@gmail.com> > > > > > > > > > > > > > wrote: > > > > > > > > > > > > > > > > > > > > > > > > > > > Bumping this thread. Considering this KIP is > > > relatively straigh > > > > > > > > > > > > > > forward, can we get some votes or feedback if > > > you think it's > > > > > > > > > > not? > > > > > > > > > > > > > > Thanks > > > > > > > > > > > > > > > > > > > > > > > > > > > > On Tue, Jan 8, 2019 at 5:40 PM Edoardo Comar > < > > > > > > > > > > edoco...@gmail.com> > > > > > > > > > > > > > wrote: > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > +1 (non-binding) > > > > > > > > > > > > > > > Thanks Mickael! > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > On Tue, 8 Jan 2019 at 17:39, Patrik > Kleindl < > > > > > > > > > > pklei...@gmail.com> > > > > > > > > > > > > > wrote: > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > +1 (non-binding) > > > > > > > > > > > > > > > > Thanks, sounds very helpful > > > > > > > > > > > > > > > > Best regards > > > > > > > > > > > > > > > > Patrik > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > Am 08.01.2019 um 18:10 schrieb Mickael > > > Maison < > > > > > > > > > > > > > > mickael.mai...@gmail.com > > > > > > > > > > > > > > > > >: > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > Hi all, > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > I'd like to start the vote on KIP-396: > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > https://eur04.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fpages%2Fviewpage.action%3FpageId%3D97551484&data=02%7C01%7C%7C47c103e1919142c35d7c08d67f9e4c5d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636836715187389495&sdata=ihLaSXvB8C%2BK%2F%2BWjVDqKXgUJoRDmwfIi7FvFLRzmFe4%3D&reserved=0 > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > Thanks > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > -- > > > > > > > > > > > > > > > "When the people fear their government, > there > > > is tyranny; > > > > > > > > > > when > > > > > > > > > > > > the > > > > > > > > > > > > > > > government fears the people, there is > > > liberty." [Thomas > > > > > > > > > > > > Jefferson] > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > -- > > > > > > > > > > > > *Dongjin Lee* > > > > > > > > > > > > > > > > > > > > > > > > *A hitchhiker in the mathematical world.* > > > > > > > > > > > > *github: <http://goog_969573159/> > > > github.com/dongjinleekr > > > > > > > > > > > > <https://github.com/dongjinleekr>linkedin: > > > > > > > > > > kr.linkedin.com/in/dongjinleekr > > > > > > > > > > > > <https://kr.linkedin.com/in/dongjinleekr > >speakerdeck: > > > > > > > > > > > > speakerdeck.com/dongjin > > > > > > > > > > > > <https://speakerdeck.com/dongjin>* > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > >