You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by "Neha Narkhede (JIRA)" <ji...@apache.org> on 2015/01/16 03:22:34 UTC

[jira] [Comment Edited] (KAFKA-1476) Get a list of consumer groups

    [ https://issues.apache.org/jira/browse/KAFKA-1476?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14279715#comment-14279715 ] 

Neha Narkhede edited comment on KAFKA-1476 at 1/16/15 2:21 AM:
---------------------------------------------------------------

Thanks for attaching the output, [~onurkaraman]. Here are some review comments-

1. The error stack trace for describing groups that don't exist is pretty ugly, so let's remove that and output a message that states the group doesn't exist
{code}
vagrant@worker1:/opt/kafka$ bin/kafka-consumer-groups.sh --zookeeper 192.168.50.11:2181 --describe --group g3
Error while executing consumer group command org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode for /consumers/g3/owners
org.I0Itec.zkclient.exception.ZkNoNodeException: org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode for /consumers/g3/owners
	at org.I0Itec.zkclient.exception.ZkException.create(ZkException.java:47)
	at org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:685)
	at org.I0Itec.zkclient.ZkClient.getChildren(ZkClient.java:413)
	at org.I0Itec.zkclient.ZkClient.getChildren(ZkClient.java:409)
	at kafka.utils.ZkUtils$.getTopicsByConsumerGroup(ZkUtils.scala:758)
	at kafka.admin.ConsumerGroupCommand$.describe(ConsumerGroupCommand.scala:81)
	at kafka.admin.ConsumerGroupCommand$.main(ConsumerGroupCommand.scala:56)
	at kafka.admin.ConsumerGroupCommand.main(ConsumerGroupCommand.scala)
Caused by: org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode for /consumers/g3/owners
	at org.apache.zookeeper.KeeperException.create(KeeperException.java:111)
	at org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
	at org.apache.zookeeper.ZooKeeper.getChildren(ZooKeeper.java:1472)
	at org.apache.zookeeper.ZooKeeper.getChildren(ZooKeeper.java:1500)
	at org.I0Itec.zkclient.ZkConnection.getChildren(ZkConnection.java:99)
	at org.I0Itec.zkclient.ZkClient$2.call(ZkClient.java:416)
	at org.I0Itec.zkclient.ZkClient$2.call(ZkClient.java:413)
	at org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:675)
	... 6 more
{code}
2. I'm not sure why the header fields are not comma separated but the data is. It will be best to stick to one separator only. So if you pick comma as the separator, then the output should change to
{code}
GROUP, TOPIC, PID, CURRENT, OFFSET, LOG SIZE, LAG, OWNER
g2, t2, 0, 1, 1, 0, none
{code}
3. For deleting only a specific topic's information, it is sufficient to rely on the user specifying the --topic along with --delete. The --delete-with-topic option seems unnecessary. 


was (Author: nehanarkhede):
Thanks for attaching the output, [~onurkaraman]. Here are some review comments-

1. The error stack trace for describing groups that don't exist is pretty ugly, so let's remove that and output a message that states the group doesn't exist
{code}
vagrant@worker1:/opt/kafka$ bin/kafka-consumer-groups.sh --zookeeper 192.168.50.11:2181 --describe --group g3
Error while executing consumer group command org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode for /consumers/g3/owners
org.I0Itec.zkclient.exception.ZkNoNodeException: org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode for /consumers/g3/owners
	at org.I0Itec.zkclient.exception.ZkException.create(ZkException.java:47)
	at org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:685)
	at org.I0Itec.zkclient.ZkClient.getChildren(ZkClient.java:413)
	at org.I0Itec.zkclient.ZkClient.getChildren(ZkClient.java:409)
	at kafka.utils.ZkUtils$.getTopicsByConsumerGroup(ZkUtils.scala:758)
	at kafka.admin.ConsumerGroupCommand$.describe(ConsumerGroupCommand.scala:81)
	at kafka.admin.ConsumerGroupCommand$.main(ConsumerGroupCommand.scala:56)
	at kafka.admin.ConsumerGroupCommand.main(ConsumerGroupCommand.scala)
Caused by: org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode for /consumers/g3/owners
	at org.apache.zookeeper.KeeperException.create(KeeperException.java:111)
	at org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
	at org.apache.zookeeper.ZooKeeper.getChildren(ZooKeeper.java:1472)
	at org.apache.zookeeper.ZooKeeper.getChildren(ZooKeeper.java:1500)
	at org.I0Itec.zkclient.ZkConnection.getChildren(ZkConnection.java:99)
	at org.I0Itec.zkclient.ZkClient$2.call(ZkClient.java:416)
	at org.I0Itec.zkclient.ZkClient$2.call(ZkClient.java:413)
	at org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:675)
	... 6 more
{code}
2. I'm not sure why the header fields are not comma separated but the data is. It will be best to stick to one separator only. So if you pick , as the separator, then the output should change to
{code}
GROUP, TOPIC, PID, CURRENT, OFFSET, LOG SIZE, LAG, OWNER
g2, t2, 0, 1, 1, 0, none
{code}
3. For deleting only a specific topic's information, it is sufficient to rely on the user specifying the --topic along with --delete. The --delete-with-topic option seems unnecessary. 

> Get a list of consumer groups
> -----------------------------
>
>                 Key: KAFKA-1476
>                 URL: https://issues.apache.org/jira/browse/KAFKA-1476
>             Project: Kafka
>          Issue Type: Wish
>          Components: tools
>    Affects Versions: 0.8.1.1
>            Reporter: Ryan Williams
>            Assignee: Balaji Seshadri
>              Labels: newbie
>             Fix For: 0.9.0
>
>         Attachments: ConsumerCommand.scala, KAFKA-1476-LIST-GROUPS.patch, KAFKA-1476-RENAME.patch, KAFKA-1476-REVIEW-COMMENTS.patch, KAFKA-1476.patch, KAFKA-1476.patch, KAFKA-1476.patch, KAFKA-1476.patch, KAFKA-1476_2014-11-10_11:58:26.patch, KAFKA-1476_2014-11-10_12:04:01.patch, KAFKA-1476_2014-11-10_12:06:35.patch, KAFKA-1476_2014-12-05_12:00:12.patch, KAFKA-1476_2015-01-12_16:22:26.patch, KAFKA-1476_2015-01-12_16:31:20.patch, KAFKA-1476_2015-01-13_10:36:18.patch, KAFKA-1476_2015-01-15_14:30:04.patch, sample-kafka-consumer-groups-sh-output.txt
>
>
> It would be useful to have a way to get a list of consumer groups currently active via some tool/script that ships with kafka. This would be helpful so that the system tools can be explored more easily.
> For example, when running the ConsumerOffsetChecker, it requires a group option
> bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker --topic test --group ?
> But, when just getting started with kafka, using the console producer and consumer, it is not clear what value to use for the group option.  If a list of consumer groups could be listed, then it would be clear what value to use.
> Background:
> http://mail-archives.apache.org/mod_mbox/kafka-users/201405.mbox/%3cCAOq_b1w=SLZe5jRNAkxvaK0Gu9cTDKPaZaK1G4DygVqZbSgUyg@mail.gmail.com%3e



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)