-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/29831/#review69400
-----------------------------------------------------------



core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala
<https://reviews.apache.org/r/29831/#comment114100>

    As I described in my other comment, the only time safe enough to delete 
consumer group information is if there are no active consumers in the group. If 
not, then we should error out from --delete and give a WARNING to the user. 
    
    The question is if --force-delete makes sense or not since that would mean 
deleting offsets and group information for active consumers, which is very 
disruptive.



core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala
<https://reviews.apache.org/r/29831/#comment114099>

    How is this tool going to behave if the consumer's offset information is 
stored in kafka, not zookeeper?
    
    The assumption of the user would be to handle that case transparently as 
well.



core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala
<https://reviews.apache.org/r/29831/#comment114098>

    change to --delete



core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala
<https://reviews.apache.org/r/29831/#comment114097>

    => If set along with --delete
    
    I'm not sure if I fully understood the purpose of force-delete. Basically, 
the only time safe for deleting a consumer group's offset information is if 
there are no live consumers in that group anymore. 
    
    If so, --force-delete would mean deleting even if that is not true. This is 
pretty disruptive and I can't think of any case where this action will be 
useful. 
    
    Thoughts?


- Neha Narkhede


On Jan. 22, 2015, 10:32 a.m., Onur Karaman wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/29831/
> -----------------------------------------------------------
> 
> (Updated Jan. 22, 2015, 10:32 a.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1476
>     https://issues.apache.org/jira/browse/KAFKA-1476
> 
> 
> Repository: kafka
> 
> 
> Description
> -------
> 
> Merged in work for KAFKA-1476 and sub-task KAFKA-1826
> 
> 
> Diffs
> -----
> 
>   bin/kafka-consumer-groups.sh PRE-CREATION 
>   core/src/main/scala/kafka/admin/AdminUtils.scala 
> 28b12c7b89a56c113b665fbde1b95f873f8624a3 
>   core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala PRE-CREATION 
>   core/src/main/scala/kafka/utils/ZkUtils.scala 
> c14bd455b6642f5e6eb254670bef9f57ae41d6cb 
>   core/src/test/scala/unit/kafka/admin/DeleteConsumerGroupTest.scala 
> PRE-CREATION 
>   core/src/test/scala/unit/kafka/utils/TestUtils.scala 
> ac15d34425795d5be20c51b01fa1108bdcd66583 
> 
> Diff: https://reviews.apache.org/r/29831/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Onur Karaman
> 
>

Reply via email to