> On Jan. 23, 2015, 4:47 p.m., Neha Narkhede wrote:
> > core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala, line 259
> > <https://reviews.apache.org/r/29831/diff/6/?file=829769#file829769line259>
> >
> >     => 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?

There are currently four types of delete:
bin/kafka-consumer-groups.sh --zookeeper 192.168.50.11:2181 --delete --group g1 
--group g5
bin/kafka-consumer-groups.sh --zookeeper 192.168.50.11:2181 --delete --group g3 
--group g4 --topic t2
bin/kafka-consumer-groups.sh --zookeeper 192.168.50.11:2181 --delete --topic t1
bin/kafka-consumer-groups.sh --zookeeper 192.168.50.11:2181 --delete --topic t3 
--force-delete

--force-delete only applies to topic-wide delete. My concern was that when you 
do a topic-wide delete, you can potentially impact many consumer groups. So by 
default, topic-wide delete first checks if the topic still exists. My reasoning 
was that if a topic still exists during a topic-wide offset delete, it probably 
wasn't intentional. It overrides the default behavior by ignoring the 
topic-existance check.

One complication of the topic existance check is the following scenario:
1. We delete topic t.
2. t gets recreated due to some producers still producing events to t.
3. We try to do a topic-wide offset delete on t.
4. The check will prevent the offset delete from happening.

--force-delete attempts to address that scenario.

I agree that it is not safe to delete offsets while the group is active, and 
none of the 4 deletes currently check for this. While it is documented, it 
makes more sense to push this into the code.


- Onur


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


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