Hi Jorge,

Thanks for the useful KIP.

I have a question regarding the proposed "plan" option.
The "current offset" and "lag" values of a topic partition are meaningful 
within a consumer group. In other words, different consumer groups could 
have different values for these properties of each topic partition.
I don't see that reflected in the discussion around the "plan" option. 
Unless we are assuming a "--group" option is also provided by user (which 
is not clear from the KIP if that is the case).

Also, I was wondering if you can provide at least one full command example 
for each of the "plan", "execute", and "export" options. They would 
definitely help in understanding some of the details.

Sorry for the delayed question/suggestion. I hope they make sense.

Thanks.
--Vahid



From:   Jorge Esteban Quilcate Otoya <quilcate.jo...@gmail.com>
To:     dev@kafka.apache.org
Date:   02/24/2017 09:51 AM
Subject:        Re: KIP-122: Add a tool to Reset Consumer Group Offsets



Great! KIP updated.



El vie., 24 feb. 2017 a las 18:22, Matthias J. Sax 
(<matth...@confluent.io>)
escribió:

> I like this!
>
> --by-duration and --shift-by
>
>
> -Matthias
>
> On 2/24/17 12:57 AM, Jorge Esteban Quilcate Otoya wrote:
> > Renaming to --by-duration LGTM
> >
> > Not sure about changing it to --shift-by-duration because we could end 
up
> > with the same redundancy as before with reset: --reset-offsets
> > --reset-to-*.
> >
> > Maybe changing --shift-offset-by to --shift-by 'n' could make it
> consistent
> > enough?
> >
> >
> > El vie., 24 feb. 2017 a las 6:39, Matthias J. Sax (<
> matth...@confluent.io>)
> > escribió:
> >
> >> I just read the update KIP once more.
> >>
> >> I would suggest to rename --to-duration to --by-duration
> >>
> >> Or as a second idea, rename --to-duration to --shift-by-duration and 
at
> >> the same time rename --shift-offset-by to --shift-by-offset
> >>
> >> Not sure what the best option is, but naming would be more consistent
> IMHO.
> >>
> >>
> >>
> >> -Matthias
> >>
> >> On 2/23/17 4:42 PM, Jorge Esteban Quilcate Otoya wrote:
> >>> Hi All,
> >>>
> >>> If there are no more concerns, I'd like to start vote for this KIP.
> >>>
> >>> Thanks!
> >>> Jorge.
> >>>
> >>> El jue., 23 feb. 2017 a las 22:50, Jorge Esteban Quilcate Otoya (<
> >>> quilcate.jo...@gmail.com>) escribió:
> >>>
> >>>> Oh ok :)
> >>>>
> >>>> So, we can keep `--topic t1:1,2,3`
> >>>>
> >>>> I think with this one we have most of the feedback applied. I will
> >> update
> >>>> the KIP with this change.
> >>>>
> >>>> El jue., 23 feb. 2017 a las 22:38, Matthias J. Sax (<
> >> matth...@confluent.io>)
> >>>> escribió:
> >>>>
> >>>> Sounds reasonable.
> >>>>
> >>>> If we have multiple --topic arguments, it does also not matter if 
we
> use
> >>>> t1:1,2 or t2=1,2
> >>>>
> >>>> I just suggested '=' because I wanted use ':' to chain multiple
> topics.
> >>>>
> >>>>
> >>>> -Matthias
> >>>>
> >>>> On 2/23/17 10:49 AM, Jorge Esteban Quilcate Otoya wrote:
> >>>>> Yeap, `--topic t1=1,2`LGTM
> >>>>>
> >>>>> Don't have idea neither about getting rid of repeated --topic, but
> >>>> --group
> >>>>> is also repeated in the case of deletion, so it could be ok to 
have
> >>>>> repeated --topic arguments.
> >>>>>
> >>>>> El jue., 23 feb. 2017 a las 19:14, Matthias J. Sax (<
> >>>> matth...@confluent.io>)
> >>>>> escribió:
> >>>>>
> >>>>>> So you suggest to merge "scope options" --topics, --topic, and
> >>>>>> --partitions into a single option? Sound good to me.
> >>>>>>
> >>>>>> I like the compact way to express it, ie,
> topicname:list-of-partitions
> >>>>>> with "all partitions" if not partitions are specified. It's quite
> >>>>>> intuitive to use.
> >>>>>>
> >>>>>> Just wondering, if we could get rid of the repeated --topic 
option;
> >> it's
> >>>>>> somewhat verbose. Have no good idea though who to improve it.
> >>>>>>
> >>>>>> If you concatenate multiple topic, we need one more character 
that
> is
> >>>>>> not allowed in topic names to separate the topics:
> >>>>>>
> >>>>>>> invalidChars = {'/', '\\', ',', '\u0000', ':', '"', '\'', ';', 
'*',
> >>>>>> '?', ' ', '\t', '\r', '\n', '='};
> >>>>>>
> >>>>>> maybe
> >>>>>>
> >>>>>> --topics t1=1,2,3:t2:t3=3
> >>>>>>
> >>>>>> use '=' to specify partitions (instead of ':' as you proposed) 
and
> ':'
> >>>>>> to separate topics? All other characters seem to be worse to use 
to
> >> me.
> >>>>>> But maybe you have a better idea.
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> -Matthias
> >>>>>>
> >>>>>>
> >>>>>> On 2/23/17 3:15 AM, Jorge Esteban Quilcate Otoya wrote:
> >>>>>>> @Matthias about the point 9:
> >>>>>>>
> >>>>>>> What about keeping only the --topic option, and support this
> format:
> >>>>>>>
> >>>>>>> `--topic t1:0,1,2 --topic t2 --topic t3:2`
> >>>>>>>
> >>>>>>> In this case topics t1, t2, and t3 will be selected: topic t1 
with
> >>>>>>> partitions 0,1 and 2; topic t2 with all its partitions; and 
topic
> t3,
> >>>>>> with
> >>>>>>> only partition 2.
> >>>>>>>
> >>>>>>> Jorge.
> >>>>>>>
> >>>>>>> El mar., 21 feb. 2017 a las 11:11, Jorge Esteban Quilcate Otoya 
(<
> >>>>>>> quilcate.jo...@gmail.com>) escribió:
> >>>>>>>
> >>>>>>>> Thanks for the feedback Matthias.
> >>>>>>>>
> >>>>>>>> * 1. You're right. I'll reorder the scenarios.
> >>>>>>>>
> >>>>>>>> * 2. Agree. I'll update the KIP.
> >>>>>>>>
> >>>>>>>> * 3. I like it, updating to `reset-offsets`
> >>>>>>>>
> >>>>>>>> * 4. Agree, removing the `reset-` part
> >>>>>>>>
> >>>>>>>> * 5. Yes, 1.e option without --execute or --export will print 
out
> >>>>>> current
> >>>>>>>> offset, and the new offset, that will be the same. The use-case 
of
> >>>> this
> >>>>>>>> option is to use it in combination with --export mostly and 
have a
> >>>>>> current
> >>>>>>>> 'checkpoint' to reset later. I will add to the KIP how the 
output
> >>>> should
> >>>>>>>> looks like.
> >>>>>>>>
> >>>>>>>> * 6. Considering 4., I will update it to `--to-offset`
> >>>>>>>>
> >>>>>>>> * 7. I like the idea to unify these options (plus, minus).
> >>>>>>>> `shift-offsets-by` is a good option, but I will like some more
> >>>> feedback
> >>>>>>>> here about the name. I will update the KIP in the meantime.
> >>>>>>>>
> >>>>>>>> * 8. Yes, discussed in 9.
> >>>>>>>>
> >>>>>>>> * 9. Agree. I'll love some feedback here. `topic` is already 
used
> by
> >>>>>>>> `delete`, and we can add `--all-topics` to consider all
> >>>>>> topics/partitions
> >>>>>>>> assigned to a group. How could we define specific
> topics/partitions?
> >>>>>>>>
> >>>>>>>> * 10. Haven't thought about it, but make sense.
> >>>>>>>> <topic>,<partition>,<offset> would be enough.
> >>>>>>>>
> >>>>>>>> * 11. Agree. Solved with 10.
> >>>>>>>>
> >>>>>>>> Also, I have a couple of changes to mention:
> >>>>>>>>
> >>>>>>>> 1. I have add a reference to the branch where I'm working on 
this
> >> KIP.
> >>>>>>>>
> >>>>>>>> 2. About the period scenario `--to-period`. I will change it to
> >>>>>>>> `--to-duration` given that duration (
> >>>>>>>> 
https://docs.oracle.com/javase/8/docs/api/java/time/Duration.html
> )
> >>>>>>>> follows this format: 'PnDTnHnMnS' and does not consider 
daylight
> >>>> saving
> >>>>>>>> efects.
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> El mar., 21 feb. 2017 a las 2:47, Matthias J. Sax (<
> >>>>>> matth...@confluent.io>)
> >>>>>>>> escribió:
> >>>>>>>>
> >>>>>>>> Hi,
> >>>>>>>>
> >>>>>>>> thanks for updating the KIP. Couple of follow up comments:
> >>>>>>>>
> >>>>>>>> * Nit: Why is "Reset to Earliest" and "Reset to Latest" a 
"reset
> by
> >>>>>>>> time" option -- IMHO it belongs to "reset by position"?
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> * Nit: Description of "Reset to Earliest"
> >>>>>>>>
> >>>>>>>>> using Kafka Consumer's `auto.offset.reset` to `earliest`
> >>>>>>>>
> >>>>>>>> I think this is strictly speaking not correct (as
> auto.offset.reset
> >>>> only
> >>>>>>>> triggered if no valid offset is found, but this tool explicitly
> >>>> modified
> >>>>>>>> committed offset), and should be phrased as
> >>>>>>>>
> >>>>>>>>> using Kafka Consumer's #seekToBeginning()
> >>>>>>>>
> >>>>>>>> -> similar issue for description of "Reset to Latest"
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> * Main option: rename to --reset-offsets (plural instead of
> >> singular)
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> * Scenario Options: I would remove "reset" from all options,
> because
> >>>> the
> >>>>>>>> main argument "--reset-offset" says already what to do:
> >>>>>>>>
> >>>>>>>>> bin/kafka-consumer-groups.sh --reset-offset 
--reset-to-datetime
> XXX
> >>>>>>>>
> >>>>>>>> better (IMHO):
> >>>>>>>>
> >>>>>>>>> bin/kafka-consumer-groups.sh --reset-offsets --to-datetime XXX
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> * Option 1.e ("print and export current offset") is not 
intuitive
> to
> >>>> use
> >>>>>>>> IMHO. The main option is "--reset-offset" but nothing happens 
if
> no
> >>>>>>>> scenario is specified. It is also not specified, what the 
output
> >>>> should
> >>>>>>>> look like?
> >>>>>>>>
> >>>>>>>> Furthermore, --describe should actually show currently 
committed
> >>>> offset
> >>>>>>>> for a group. So it seems to be redundant to have the same 
option
> in
> >>>>>>>> --reset-offsets
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> * Option 2.a: I would rename to "--reset-to-offset" (or
> considering
> >>>> the
> >>>>>>>> comment above to "--to-offset")
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> * Option 2.b and 2.c: I would unify to "--shift-offsets-by" (or
> >>>> similar)
> >>>>>>>> and accept positive/negative values
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> * About Scope "all": maybe it's better to have an option
> >>>> "--all-topics"
> >>>>>>>> (or similar). IMHO explicit arguments are preferable over 
implicit
> >>>>>>>> setting to guard again accidental miss use of the tool.
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> * Scope: I also think, that "--topic" (singular) and "--topics"
> >>>> (plural)
> >>>>>>>> are too similar and easy to use in a wrong way (ie, mix up) --
> maybe
> >>>> we
> >>>>>>>> can have two options that are easier to distinguish.
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> * I still think that JSON is not the best format (it's too
> >>>> verbose/hard
> >>>>>>>> to write for humans from scratch). A simple CSV format with
> implicit
> >>>>>>>> schema (topic,partition,offset) would be sufficient.
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> * Why does the JSON contain "group_id" field -- there is 
parameter
> >>>>>>>> "--group" to specify the group ID. Would one overwrite the 
other
> >> (what
> >>>>>>>> order) or would there be an error if "--group" is used in
> >> combination
> >>>>>>>> with "--reset-from-file"?
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> -Matthias
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> On 2/17/17 6:43 AM, Jorge Esteban Quilcate Otoya wrote:
> >>>>>>>>> Hi,
> >>>>>>>>>
> >>>>>>>>> according to the feedback, I've updated the KIP:
> >>>>>>>>>
> >>>>>>>>> - We have added and ordered the scenarios, scopes and 
executions
> of
> >>>> the
> >>>>>>>>> Reset Offset tool.
> >>>>>>>>> - Consider it as an extension to the current
> `ConsumerGroupCommand`
> >>>>>> tool
> >>>>>>>>> - Execution will be possible without generating JSON files.
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>
> >>>>
> >>
> 
https://cwiki.apache.org/confluence/display/KAFKA/KIP-122%3A+Add+Reset+Consumer+Group+Offsets+tooling

> >>>>>>>>>
> >>>>>>>>> Looking forward to your feedback!
> >>>>>>>>>
> >>>>>>>>> Jorge.
> >>>>>>>>>
> >>>>>>>>> El mié., 8 feb. 2017 a las 23:23, Jorge Esteban Quilcate Otoya 
(<
> >>>>>>>>> quilcate.jo...@gmail.com>) escribió:
> >>>>>>>>>
> >>>>>>>>>> Great. I think I got the idea. What about this options:
> >>>>>>>>>>
> >>>>>>>>>> Scenarios:
> >>>>>>>>>>
> >>>>>>>>>> 1. Current status
> >>>>>>>>>>
> >>>>>>>>>> ´kafka-consumer-groups.sh --reset-offset --group cg1´
> >>>>>>>>>>
> >>>>>>>>>> 2. To Datetime
> >>>>>>>>>>
> >>>>>>>>>> ´kafka-consumer-groups.sh --reset-offset --group cg1
> >>>>>> --reset-to-datetime
> >>>>>>>>>> 2017-01-01T00:00:00.000´
> >>>>>>>>>>
> >>>>>>>>>> 3. To Period
> >>>>>>>>>>
> >>>>>>>>>> ´kafka-consumer-groups.sh --reset-offset --group cg1
> >>>> --reset-to-period
> >>>>>>>> P2D´
> >>>>>>>>>>
> >>>>>>>>>> 4. To Earliest
> >>>>>>>>>>
> >>>>>>>>>> ´kafka-consumer-groups.sh --reset-offset --group cg1
> >>>>>>>> --reset-to-earliest´
> >>>>>>>>>>
> >>>>>>>>>> 5. To Latest
> >>>>>>>>>>
> >>>>>>>>>> ´kafka-consumer-groups.sh --reset-offset --group cg1
> >>>>>> --reset-to-latest´
> >>>>>>>>>>
> >>>>>>>>>> 6. Minus 'n' offsets
> >>>>>>>>>>
> >>>>>>>>>> ´kafka-consumer-groups.sh --reset-offset --group cg1
> --reset-minus
> >>>> n´
> >>>>>>>>>>
> >>>>>>>>>> 7. Plus 'n' offsets
> >>>>>>>>>>
> >>>>>>>>>> ´kafka-consumer-groups.sh --reset-offset --group cg1
> --reset-plus
> >> n´
> >>>>>>>>>>
> >>>>>>>>>> 8. To specific offset
> >>>>>>>>>>
> >>>>>>>>>> ´kafka-consumer-groups.sh --reset-offset --group cg1 
--reset-to
> x´
> >>>>>>>>>>
> >>>>>>>>>> Scopes:
> >>>>>>>>>>
> >>>>>>>>>> a. All topics used by Consumer Group
> >>>>>>>>>>
> >>>>>>>>>> Don't specify --topics
> >>>>>>>>>>
> >>>>>>>>>> b. Specific List of Topics
> >>>>>>>>>>
> >>>>>>>>>> Add list of values in --topics t1,t2,tn
> >>>>>>>>>>
> >>>>>>>>>> c. One Topic, all Partitions
> >>>>>>>>>>
> >>>>>>>>>> Add one topic and no partitions values: --topic t1
> >>>>>>>>>>
> >>>>>>>>>> d. One Topic, List of Partitions
> >>>>>>>>>>
> >>>>>>>>>> Add one topic and partitions values: --topic t1 --partitions
> 0,1,2
> >>>>>>>>>>
> >>>>>>>>>> About Reset Plan (JSON file):
> >>>>>>>>>>
> >>>>>>>>>> I think is still valid to have the option to persist reset
> >>>>>> configuration
> >>>>>>>>>> as a file, but I agree to give the option to run the tool
> without
> >>>>>> going
> >>>>>>>>>> down to the JSON file.
> >>>>>>>>>>
> >>>>>>>>>> Execution options:
> >>>>>>>>>>
> >>>>>>>>>> 1. Without execution argument (No args):
> >>>>>>>>>>
> >>>>>>>>>> Print out results (reset plan)
> >>>>>>>>>>
> >>>>>>>>>> 2. With --execute argument:
> >>>>>>>>>>
> >>>>>>>>>> Run reset process
> >>>>>>>>>>
> >>>>>>>>>> 3. With --output argument:
> >>>>>>>>>>
> >>>>>>>>>> Save result in a JSON format.
> >>>>>>>>>>
> >>>>>>>>>> 4. Only with --execute option and --reset-file (path to JSON)
> >>>>>>>>>>
> >>>>>>>>>> Reset based on file
> >>>>>>>>>>
> >>>>>>>>>> 4. Only with --verify option and --reset-file (path to JSON)
> >>>>>>>>>>
> >>>>>>>>>> Verify file values with current offsets
> >>>>>>>>>>
> >>>>>>>>>> I think we can remove --generate-and-execute because is a bit
> >>>> clumsy.
> >>>>>>>>>>
> >>>>>>>>>> With this options we will be able to execute with manual JSON
> >>>>>>>>>> configuration.
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> El mié., 8 feb. 2017 a las 22:43, Ben Stopford (<
> b...@confluent.io
> >>> )
> >>>>>>>>>> escribió:
> >>>>>>>>>>
> >>>>>>>>>> Yes - using a tool like this to skip a set of consumer groups
> >> over a
> >>>>>>>>>> corrupt/bad message is definitely appealing.
> >>>>>>>>>>
> >>>>>>>>>> B
> >>>>>>>>>>
> >>>>>>>>>> On Wed, Feb 8, 2017 at 9:37 PM Gwen Shapira 
<g...@confluent.io>
> >>>>>> wrote:
> >>>>>>>>>>
> >>>>>>>>>>> I like the --reset-to-earliest and --reset-to-latest. In
> general,
> >>>>>>>>>>> since the JSON route is the most challenging for users, we 
want
> >> to
> >>>>>>>>>>> provide a lot of ways to do useful things without going 
there.
> >>>>>>>>>>>
> >>>>>>>>>>> Two things that can help:
> >>>>>>>>>>>
> >>>>>>>>>>> 1. A lot of times, users want to skip few messages that 
cause
> >>>> issues
> >>>>>>>>>>> and continue. maybe just specifying the topic, partition and
> >> delta
> >>>>>>>>>>> will be better than having to find the offset and write a 
JSON
> >> and
> >>>>>>>>>>> validate the JSON etc.
> >>>>>>>>>>>
> >>>>>>>>>>> 2. Thinking if there are other common use-cases that we can
> make
> >>>> easy
> >>>>>>>>>>> rather than just one generic but not very usable method.
> >>>>>>>>>>>
> >>>>>>>>>>> Gwen
> >>>>>>>>>>>
> >>>>>>>>>>> On Wed, Feb 8, 2017 at 3:25 AM, Jorge Esteban Quilcate Otoya
> >>>>>>>>>>> <quilcate.jo...@gmail.com> wrote:
> >>>>>>>>>>>> Thanks for the feedback!
> >>>>>>>>>>>>
> >>>>>>>>>>>> @Onur, @Gwen:
> >>>>>>>>>>>>
> >>>>>>>>>>>> Agree. Actually at the first draft I considered to have it
> >> inside
> >>>>>>>>>>>> ´kafka-consumer-groups.sh´, but I decide to propose it as a
> >>>>>> standalone
> >>>>>>>>>>> tool
> >>>>>>>>>>>> to describe it clearly and focus it on reset functionality.
> >>>>>>>>>>>>
> >>>>>>>>>>>> But now that you mentioned, it does make sense to have it 
in
> >>>>>>>>>>>> ´kafka-consumer-groups.sh´. How would be a consistent way 
to
> >>>>>> introduce
> >>>>>>>>>>> it?
> >>>>>>>>>>>>
> >>>>>>>>>>>> Maybe something like this:
> >>>>>>>>>>>>
> >>>>>>>>>>>> ´kafka-consumer-groups.sh --reset-offset --generate --group
> cg1
> >>>>>>>>>> --topics
> >>>>>>>>>>> t1
> >>>>>>>>>>>> --reset-from 2017-01-01T00:00:00.000 --output plan.json´
> >>>>>>>>>>>>
> >>>>>>>>>>>> ´kafka-consumer-groups.sh --reset-offset --verify
> >>>> --reset-json-file
> >>>>>>>>>>>> plan.json´
> >>>>>>>>>>>>
> >>>>>>>>>>>> ´kafka-consumer-groups.sh --reset-offset --execute
> >>>> --reset-json-file
> >>>>>>>>>>>> plan.json´
> >>>>>>>>>>>>
> >>>>>>>>>>>> ´kafka-consumer-groups.sh --reset-offset
> --generate-and-execute
> >>>>>>>> --group
> >>>>>>>>>>> cg1
> >>>>>>>>>>>> --topics t1 --reset-from 2017-01-01T00:00:00.000´
> >>>>>>>>>>>>
> >>>>>>>>>>>> @Gwen:
> >>>>>>>>>>>>
> >>>>>>>>>>>>> It looks exactly like the replica assignment tool
> >>>>>>>>>>>>
> >>>>>>>>>>>> It was influenced by ;-) I use the generate-verify-execute
> >> process
> >>>>>>>> here
> >>>>>>>>>>> to
> >>>>>>>>>>>> make sure user will be aware of the result of this 
operation.
> At
> >>>> the
> >>>>>>>>>>>> beginning we considered only add a couple of options to
> Consumer
> >>>>>> Group
> >>>>>>>>>>>> Command:
> >>>>>>>>>>>>
> >>>>>>>>>>>> --rewind-to-timestamp and --rewind-to-period
> >>>>>>>>>>>>
> >>>>>>>>>>>> @Onur:
> >>>>>>>>>>>>
> >>>>>>>>>>>>> You can actually get away with overriding while members of
> the
> >>>>>> group
> >>>>>>>>>>> are live
> >>>>>>>>>>>> with method 2 by using group information from
> >>>> DescribeGroupsRequest.
> >>>>>>>>>>>>
> >>>>>>>>>>>> This means that we need to have Consumer Group stopped 
before
> >>>>>>>> executing
> >>>>>>>>>>> and
> >>>>>>>>>>>> start a new consumer internally to do this? Therefore, we
> won't
> >> be
> >>>>>>>> able
> >>>>>>>>>>> to
> >>>>>>>>>>>> consider executing reset when ConsumerGroup is active? 
(trying
> >> to
> >>>>>>>>>> relate
> >>>>>>>>>>> it
> >>>>>>>>>>>> with @Dong 5th question)
> >>>>>>>>>>>>
> >>>>>>>>>>>> @Dong:
> >>>>>>>>>>>>
> >>>>>>>>>>>>> Should we allow user to use wildcard to reset offset of 
all
> >>>> groups
> >>>>>>>>>> for a
> >>>>>>>>>>>> given topic as well?
> >>>>>>>>>>>>
> >>>>>>>>>>>> I haven't thought about this scenario. Could be 
interesting.
> >>>>>> Following
> >>>>>>>>>>> the
> >>>>>>>>>>>> recommendation to add it into Consumer Group Command, in 
this
> >> case
> >>>>>>>>>> Group
> >>>>>>>>>>>> argument will be optional if there are only 1 topic. I 
think
> for
> >>>>>>>>>> multiple
> >>>>>>>>>>>> topic won't be that useful.
> >>>>>>>>>>>>
> >>>>>>>>>>>>> Should we allow user to specify timestamp per topic 
partition
> >> in
> >>>>>> the
> >>>>>>>>>>> json
> >>>>>>>>>>>> file as well?
> >>>>>>>>>>>>
> >>>>>>>>>>>> Don't think this could be a valid from the tool, but if 
Reset
> >> Plan
> >>>>>> is
> >>>>>>>>>>>> generated, and user want to set the offset for a specific
> >>>> partition
> >>>>>> to
> >>>>>>>>>>>> other offset (eventually based on another timestamp), and
> >> execute
> >>>>>> it,
> >>>>>>>>>> it
> >>>>>>>>>>>> will be up to her/him.
> >>>>>>>>>>>>
> >>>>>>>>>>>>> Should the script take some credential file to make sure 
that
> >>>> this
> >>>>>>>>>>>> operation is authenticated given the potential impact of 
this
> >>>>>>>>>> operation?
> >>>>>>>>>>>>
> >>>>>>>>>>>> Haven't tried to secure brokers yet, but the tool should
> support
> >>>>>>>>>>>> authorization if it's enabled in the broker.
> >>>>>>>>>>>>
> >>>>>>>>>>>>> Should we provide constant to reset committed offset to
> >>>>>>>>>> earliest/latest
> >>>>>>>>>>>> offset of a partition, e.g. -1 indicates earliest offset 
and
> -2
> >>>>>>>>>> indicates
> >>>>>>>>>>>> latest offset.
> >>>>>>>>>>>>
> >>>>>>>>>>>> I will go for something like ´--reset-to-earliest´ and
> >>>>>>>>>>> ´--reset-to-latest´
> >>>>>>>>>>>>
> >>>>>>>>>>>>> Should we allow dynamic change of the comitted offset when
> >>>> consumer
> >>>>>>>>>> are
> >>>>>>>>>>>> running, such that consumer will seek to the newly 
committed
> >>>> offset
> >>>>>>>> and
> >>>>>>>>>>>> start consuming from there?
> >>>>>>>>>>>>
> >>>>>>>>>>>> Not sure about this. I will recommend to keep it simple and
> ask
> >>>> user
> >>>>>>>> to
> >>>>>>>>>>>> stop consumers first. But I would considered it if the
> >> trade-offs
> >>>>>> are
> >>>>>>>>>>>> clear.
> >>>>>>>>>>>>
> >>>>>>>>>>>> @Matthias
> >>>>>>>>>>>>
> >>>>>>>>>>>> Added :). And thanks a lot for your help to define this 
KIP!
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> El mié., 8 feb. 2017 a las 7:47, Gwen Shapira (<
> >> g...@confluent.io
> >>>>> )
> >>>>>>>>>>>> escribió:
> >>>>>>>>>>>>
> >>>>>>>>>>>>> As long as the CLI is a bit consistent? Like, not just
> adding 3
> >>>>>>>>>>>>> arguments and a JSON parser to the existing tool, right?
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> On Tue, Feb 7, 2017 at 10:29 PM, Onur Karaman
> >>>>>>>>>>>>> <onurkaraman.apa...@gmail.com> wrote:
> >>>>>>>>>>>>>> I think it makes sense to just add the feature to
> >>>>>>>>>>>>> kafka-consumer-groups.sh
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> On Tue, Feb 7, 2017 at 10:24 PM, Gwen Shapira <
> >>>> g...@confluent.io>
> >>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Thanks for the KIP. I'm super happy about adding the
> >>>> capability.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> I hate the interface, though. It looks exactly like the
> >> replica
> >>>>>>>>>>>>>>> assignment tool. A tool everyone loves so much that 
there
> are
> >>>>>>>>>>> multiple
> >>>>>>>>>>>>>>> projects, open and closed, that try to fix it.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Can we swap it with something that looks a bit more like
> the
> >>>>>>>>>> consumer
> >>>>>>>>>>>>>>> group tool? or the kafka streams reset tool? Consistency 
is
> >>>>>> helpful
> >>>>>>>>>>> in
> >>>>>>>>>>>>>>> such cases. I spent some time learning existing tools 
and
> >>>>>> learning
> >>>>>>>>>>> yet
> >>>>>>>>>>>>>>> another one is a deterrent.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Gwen
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> On Tue, Feb 7, 2017 at 6:43 PM, Jorge Esteban Quilcate
> Otoya
> >>>>>>>>>>>>>>> <quilcate.jo...@gmail.com> wrote:
> >>>>>>>>>>>>>>>> Hi all,
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> I would like to propose a KIP to Add a tool to Reset
> >> Consumer
> >>>>>>>>>> Group
> >>>>>>>>>>>>>>> Offsets.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >>>>>>>>>>>>>>> 122%3A+Add+a+tool+to+Reset+Consumer+Group+Offsets
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Please, take a look at the proposal and share your
> feedback.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>> Jorge.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> --
> >>>>>>>>>>>>>>> Gwen Shapira
> >>>>>>>>>>>>>>> Product Manager | Confluent
> >>>>>>>>>>>>>>> 650.450.2760 <(650)%20450-2760> <(650)%20450-2760>
> <(650)%20450-2760>
> >> <(650)%20450-2760>
> >>>> <(650)%20450-2760>
> >>>>>> <(650)%20450-2760>
> >>>>>>>> <(650)%20450-2760>
> >>>>>>>>>> <(650)%20450-2760> | @gwenshap
> >>>>>>>>>>>>>>> Follow us: Twitter | blog
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> --
> >>>>>>>>>>>>> Gwen Shapira
> >>>>>>>>>>>>> Product Manager | Confluent
> >>>>>>>>>>>>> 650.450.2760 <(650)%20450-2760> <(650)%20450-2760>
> <(650)%20450-2760>
> >> <(650)%20450-2760>
> >>>> <(650)%20450-2760>
> >>>>>> <(650)%20450-2760>
> >>>>>>>> <(650)%20450-2760>
> >>>>>>>>>> <(650)%20450-2760> | @gwenshap
> >>>>>>>>>>>>> Follow us: Twitter | blog
> >>>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> --
> >>>>>>>>>>> Gwen Shapira
> >>>>>>>>>>> Product Manager | Confluent
> >>>>>>>>>>> 650.450.2760 <(650)%20450-2760> <(650)%20450-2760>
> <(650)%20450-2760>
> >> <(650)%20450-2760>
> >>>> <(650)%20450-2760>
> >>>>>> <(650)%20450-2760> <(650)%20450-2760>
> >>>>>>>> | @gwenshap
> >>>>>>>>>>> Follow us: Twitter | blog
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>>
> >>>
> >>
> >>
> >
>
>




Reply via email to