[jira] [Commented] (KAFKA-7141) kafka-consumer-group doesn't describe existing group
[ https://issues.apache.org/jira/browse/KAFKA-7141?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16550903#comment-16550903 ] ASF GitHub Bot commented on KAFKA-7141: --- hachikuji closed pull request #5356: KAFKA-7141: ConsumerGroupCommand should describe group assignment eve… URL: https://github.com/apache/kafka/pull/5356 This is a PR merged from a forked repository. As GitHub hides the original diff on merge, it is displayed below for the sake of provenance: As this is a foreign pull request (from a fork), the diff is supplied below (as it won't show otherwise due to GitHub magic): diff --git a/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala b/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala index 48c2cffb5d3..1d61720bfa9 100755 --- a/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala +++ b/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala @@ -27,7 +27,6 @@ import kafka.utils._ import org.apache.kafka.clients.{CommonClientConfigs, admin} import org.apache.kafka.clients.admin._ import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer, OffsetAndMetadata} -import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.serialization.StringDeserializer import org.apache.kafka.common.utils.Utils import org.apache.kafka.common.{KafkaException, Node, TopicPartition} @@ -35,7 +34,6 @@ import org.apache.kafka.common.{KafkaException, Node, TopicPartition} import scala.collection.JavaConverters._ import scala.collection.mutable.ListBuffer import scala.collection.{Seq, Set} -import scala.concurrent.ExecutionException import scala.util.{Failure, Success, Try} object ConsumerGroupCommand extends Logging { @@ -340,20 +338,19 @@ object ConsumerGroupCommand extends Logging { val state = consumerGroup.state val committedOffsets = getCommittedOffsets(groupId).asScala.toMap var assignedTopicPartitions = ListBuffer[TopicPartition]() - val rowsWithConsumer = if (committedOffsets.isEmpty) List[PartitionAssignmentState]() else consumerGroup.members.asScala.filter(!_.assignment.topicPartitions.isEmpty).toSeq -.sortWith(_.assignment.topicPartitions.size > _.assignment.topicPartitions.size) -.flatMap { consumerSummary => - val topicPartitions = consumerSummary.assignment.topicPartitions.asScala - assignedTopicPartitions = assignedTopicPartitions ++ topicPartitions - val partitionOffsets = consumerSummary.assignment.topicPartitions.asScala -.map { topicPartition => - topicPartition -> committedOffsets.get(topicPartition).map(_.offset) -}.toMap + val rowsWithConsumer = consumerGroup.members.asScala.filter(!_.assignment.topicPartitions.isEmpty).toSeq +.sortWith(_.assignment.topicPartitions.size > _.assignment.topicPartitions.size).flatMap { consumerSummary => +val topicPartitions = consumerSummary.assignment.topicPartitions.asScala +assignedTopicPartitions = assignedTopicPartitions ++ topicPartitions +val partitionOffsets = consumerSummary.assignment.topicPartitions.asScala + .map { topicPartition => +topicPartition -> committedOffsets.get(topicPartition).map(_.offset) + }.toMap collectConsumerAssignment(groupId, Option(consumerGroup.coordinator), topicPartitions.toList, partitionOffsets, Some(s"${consumerSummary.consumerId}"), Some(s"${consumerSummary.host}"), Some(s"${consumerSummary.clientId}")) -} + } val rowsWithoutConsumer = committedOffsets.filterKeys(!assignedTopicPartitions.contains(_)).flatMap { case (topicPartition, offset) => diff --git a/core/src/test/scala/unit/kafka/admin/ConsumerGroupCommandTest.scala b/core/src/test/scala/unit/kafka/admin/ConsumerGroupCommandTest.scala index cf00e93558c..51082effdbd 100644 --- a/core/src/test/scala/unit/kafka/admin/ConsumerGroupCommandTest.scala +++ b/core/src/test/scala/unit/kafka/admin/ConsumerGroupCommandTest.scala @@ -92,8 +92,9 @@ class ConsumerGroupCommandTest extends KafkaServerTestHarness { def addConsumerGroupExecutor(numConsumers: Int, topic: String = topic, group: String = group, - strategy: String = classOf[RangeAssignor].getName): ConsumerGroupExecutor = { -val executor = new ConsumerGroupExecutor(brokerList, numConsumers, group, topic, strategy) + strategy: String = classOf[RangeAssignor].getName, + customPropsOpt: Option[Properties] = None): ConsumerGroupExecutor = { +val executor = new ConsumerGroupExecutor(brokerList, numConsumers, group, topic, strategy, customPropsOpt) addExecutor(executor) executor } @@ -114,9 +115,10 @@ class ConsumerGroupCommandTest extends
[jira] [Commented] (KAFKA-7141) kafka-consumer-group doesn't describe existing group
[ https://issues.apache.org/jira/browse/KAFKA-7141?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16539619#comment-16539619 ] huxihx commented on KAFKA-7141: --- [~wushujames] Agreed. This makes no sense that whether committing offsets impact the showing of group assignment, as shown below: {code:java} val rowsWithConsumer = if (offsets.isEmpty) // If offsets is empty, we give up the chance to get the group assignment at all. List[PartitionAssignmentState]() else { consumers.filter(_.assignment.nonEmpty).sortWith(_.assignment.size > _.assignment.size).flatMap { consumerSummary => val topicPartitions = consumerSummary.assignment assignedTopicPartitions = assignedTopicPartitions ++ consumerSummary.assignment val partitionOffsets: Map[TopicPartition, Option[Long]] = consumerSummary.assignment.map { topicPartition => new TopicPartition(topicPartition.topic, topicPartition.partition) -> offsets.get(topicPartition) }.toMap collectConsumerAssignment(group, Some(consumerGroupSummary.coordinator), topicPartitions, partitionOffsets, Some(s"${consumerSummary.consumerId}"), Some(s"${consumerSummary.host}"), Some(s"${consumerSummary.clientId}")) } } {code} > kafka-consumer-group doesn't describe existing group > > > Key: KAFKA-7141 > URL: https://issues.apache.org/jira/browse/KAFKA-7141 > Project: Kafka > Issue Type: Bug > Components: admin >Affects Versions: 0.11.0.0, 1.0.1 >Reporter: Bohdana Panchenko >Assignee: huxihx >Priority: Major > > I am running two consumers: akka-stream-kafka consumer with standard config > section as described in the > [https://doc.akka.io/docs/akka-stream-kafka/current/consumer.html] and > kafka-console-consumer. > akka-stream-kafka consumer configuration looks like this > {color:#33}_akka.kafka.consumer{_{color} > {color:#33} _kafka-clients{_{color} > {color:#33} _group.id = "myakkastreamkafka-1"_{color} > {color:#33} _enable.auto.commit = false_{color} > } > {color:#33} }{color} > > I am able to see the both groups with the command > > *kafka-consumer-groups --bootstrap-server 127.0.0.1:9092 --list* > _Note: This will not show information about old Zookeeper-based consumers._ > > _myakkastreamkafka-1_ > _console-consumer-57171_ > {color:#33}I am able to view details about the console consumer > group{color} > *kafka-consumer-groups --describe --bootstrap-server 127.0.0.1:9092 --group > console-consumer-57171* > _{color:#205081}Note: This will not show information about old > Zookeeper-based consumers.{color}_ > _{color:#205081}TOPIC PARTITION CURRENT-OFFSET LOG-END-OFFSET LAG CONSUMER-ID > HOST CLIENT-ID{color}_ > _{color:#205081}STREAM-TEST 0 0 0 0 > consumer-1-6b928e07-196a-4322-9928-068681617878 /172.19.0.4 consumer-1{color}_ > {color:#33}But the command to describe my akka stream consumer gives me > empty output:{color} > *kafka-consumer-groups --describe --bootstrap-server 127.0.0.1:9092 --group > myakkastreamkafka-1* > {color:#205081}_Note: This will not show information about old > Zookeeper-based consumers._{color} > {color:#205081}_TOPIC PARTITION CURRENT-OFFSET LOG-END-OFFSET LAG CONSUMER-ID > HOST CLIENT-ID_{color} > > {color:#33}That is strange. Can you please check the issue?{color} -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (KAFKA-7141) kafka-consumer-group doesn't describe existing group
[ https://issues.apache.org/jira/browse/KAFKA-7141?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16539616#comment-16539616 ] ASF GitHub Bot commented on KAFKA-7141: --- huxihx opened a new pull request #5356: KAFKA-7141: ConsumerGroupCommand should describe group assignment eve… URL: https://github.com/apache/kafka/pull/5356 …n with no offsets committed. https://issues.apache.org/jira/browse/KAFKA-7141 Currently, if a consumer group never commits offsets, ConsumerGroupCommand cannot describe it at all even if the member assignment is valid. Instead, the tool should be able to describe the group information showing empty current_offset and LAG. *More detailed description of your change, if necessary. The PR title and PR message become the squashed commit message, so use a separate comment to ping reviewers.* *Summary of testing strategy (including rationale) for the feature or bug fix. Unit and/or integration tests are expected for any behaviour change and system tests should be considered for larger changes.* ### Committer Checklist (excluded from commit message) - [ ] Verify design and implementation - [ ] Verify test coverage and CI build status - [ ] Verify documentation (including upgrade notes) This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org > kafka-consumer-group doesn't describe existing group > > > Key: KAFKA-7141 > URL: https://issues.apache.org/jira/browse/KAFKA-7141 > Project: Kafka > Issue Type: Bug > Components: admin >Affects Versions: 0.11.0.0, 1.0.1 >Reporter: Bohdana Panchenko >Assignee: huxihx >Priority: Major > > I am running two consumers: akka-stream-kafka consumer with standard config > section as described in the > [https://doc.akka.io/docs/akka-stream-kafka/current/consumer.html] and > kafka-console-consumer. > akka-stream-kafka consumer configuration looks like this > {color:#33}_akka.kafka.consumer{_{color} > {color:#33} _kafka-clients{_{color} > {color:#33} _group.id = "myakkastreamkafka-1"_{color} > {color:#33} _enable.auto.commit = false_{color} > } > {color:#33} }{color} > > I am able to see the both groups with the command > > *kafka-consumer-groups --bootstrap-server 127.0.0.1:9092 --list* > _Note: This will not show information about old Zookeeper-based consumers._ > > _myakkastreamkafka-1_ > _console-consumer-57171_ > {color:#33}I am able to view details about the console consumer > group{color} > *kafka-consumer-groups --describe --bootstrap-server 127.0.0.1:9092 --group > console-consumer-57171* > _{color:#205081}Note: This will not show information about old > Zookeeper-based consumers.{color}_ > _{color:#205081}TOPIC PARTITION CURRENT-OFFSET LOG-END-OFFSET LAG CONSUMER-ID > HOST CLIENT-ID{color}_ > _{color:#205081}STREAM-TEST 0 0 0 0 > consumer-1-6b928e07-196a-4322-9928-068681617878 /172.19.0.4 consumer-1{color}_ > {color:#33}But the command to describe my akka stream consumer gives me > empty output:{color} > *kafka-consumer-groups --describe --bootstrap-server 127.0.0.1:9092 --group > myakkastreamkafka-1* > {color:#205081}_Note: This will not show information about old > Zookeeper-based consumers._{color} > {color:#205081}_TOPIC PARTITION CURRENT-OFFSET LOG-END-OFFSET LAG CONSUMER-ID > HOST CLIENT-ID_{color} > > {color:#33}That is strange. Can you please check the issue?{color} -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (KAFKA-7141) kafka-consumer-group doesn't describe existing group
[ https://issues.apache.org/jira/browse/KAFKA-7141?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16539373#comment-16539373 ] James Cheng commented on KAFKA-7141: I think it's a little odd that kafka-consumer-groups doesn't show partition assignment at all, when there are no offsets. Currently, if there are 2 partitions (partitions 1 and 2) * A) Active consumer, no committed offsets on either of them means that you see nothing. No group assignment, no partitions. * B) Active consumer, committed offsets on 1, no committed offsets on 2, means that you will see rows for both of them, but the CURRENT-OFFSET field for partition 2 will have a hyphen in it. * C) Active consumer, Committed offsets on both 1 and 2, means you will see all the data * D) No active consumer, committed offsets on both 1 and 2, means you will see the rows, but CONSUMER-ID/HOST/CLIENT-ID will have hyphens. This Jira is talking about "A". I would have expected that "A" would display similar to "B". That you would see partition assignments, but "-" wherever there are missing offsets. > kafka-consumer-group doesn't describe existing group > > > Key: KAFKA-7141 > URL: https://issues.apache.org/jira/browse/KAFKA-7141 > Project: Kafka > Issue Type: Bug > Components: admin >Affects Versions: 0.11.0.0, 1.0.1 >Reporter: Bohdana Panchenko >Priority: Major > > I am running two consumers: akka-stream-kafka consumer with standard config > section as described in the > [https://doc.akka.io/docs/akka-stream-kafka/current/consumer.html] and > kafka-console-consumer. > akka-stream-kafka consumer configuration looks like this > {color:#33}_akka.kafka.consumer{_{color} > {color:#33} _kafka-clients{_{color} > {color:#33} _group.id = "myakkastreamkafka-1"_{color} > {color:#33} _enable.auto.commit = false_{color} > } > {color:#33} }{color} > > I am able to see the both groups with the command > > *kafka-consumer-groups --bootstrap-server 127.0.0.1:9092 --list* > _Note: This will not show information about old Zookeeper-based consumers._ > > _myakkastreamkafka-1_ > _console-consumer-57171_ > {color:#33}I am able to view details about the console consumer > group{color} > *kafka-consumer-groups --describe --bootstrap-server 127.0.0.1:9092 --group > console-consumer-57171* > _{color:#205081}Note: This will not show information about old > Zookeeper-based consumers.{color}_ > _{color:#205081}TOPIC PARTITION CURRENT-OFFSET LOG-END-OFFSET LAG CONSUMER-ID > HOST CLIENT-ID{color}_ > _{color:#205081}STREAM-TEST 0 0 0 0 > consumer-1-6b928e07-196a-4322-9928-068681617878 /172.19.0.4 consumer-1{color}_ > {color:#33}But the command to describe my akka stream consumer gives me > empty output:{color} > *kafka-consumer-groups --describe --bootstrap-server 127.0.0.1:9092 --group > myakkastreamkafka-1* > {color:#205081}_Note: This will not show information about old > Zookeeper-based consumers._{color} > {color:#205081}_TOPIC PARTITION CURRENT-OFFSET LOG-END-OFFSET LAG CONSUMER-ID > HOST CLIENT-ID_{color} > > {color:#33}That is strange. Can you please check the issue?{color} -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (KAFKA-7141) kafka-consumer-group doesn't describe existing group
[ https://issues.apache.org/jira/browse/KAFKA-7141?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16539282#comment-16539282 ] Vahid Hashemian commented on KAFKA-7141: [~kioria] thanks for the detailed info. What you're seeing makes sense. While there is no offset commit within a group there is nothing to register (an offset commit indicates successful consumption). Even with disabling auto offset commit, in normal scenarios offsets should be committed regularly when the processing of consumed records is complete. I don't think there is anything to fix here. I'll close the Jira for now. If you still see an issue please reopen and provide additional info about it. Thanks! > kafka-consumer-group doesn't describe existing group > > > Key: KAFKA-7141 > URL: https://issues.apache.org/jira/browse/KAFKA-7141 > Project: Kafka > Issue Type: Bug > Components: admin >Affects Versions: 0.11.0.0, 1.0.1 >Reporter: Bohdana Panchenko >Priority: Major > > I am running two consumers: akka-stream-kafka consumer with standard config > section as described in the > [https://doc.akka.io/docs/akka-stream-kafka/current/consumer.html] and > kafka-console-consumer. > akka-stream-kafka consumer configuration looks like this > {color:#33}_akka.kafka.consumer{_{color} > {color:#33} _kafka-clients{_{color} > {color:#33} _group.id = "myakkastreamkafka-1"_{color} > {color:#33} _enable.auto.commit = false_{color} > } > {color:#33} }{color} > > I am able to see the both groups with the command > > *kafka-consumer-groups --bootstrap-server 127.0.0.1:9092 --list* > _Note: This will not show information about old Zookeeper-based consumers._ > > _myakkastreamkafka-1_ > _console-consumer-57171_ > {color:#33}I am able to view details about the console consumer > group{color} > *kafka-consumer-groups --describe --bootstrap-server 127.0.0.1:9092 --group > console-consumer-57171* > _{color:#205081}Note: This will not show information about old > Zookeeper-based consumers.{color}_ > _{color:#205081}TOPIC PARTITION CURRENT-OFFSET LOG-END-OFFSET LAG CONSUMER-ID > HOST CLIENT-ID{color}_ > _{color:#205081}STREAM-TEST 0 0 0 0 > consumer-1-6b928e07-196a-4322-9928-068681617878 /172.19.0.4 consumer-1{color}_ > {color:#33}But the command to describe my akka stream consumer gives me > empty output:{color} > *kafka-consumer-groups --describe --bootstrap-server 127.0.0.1:9092 --group > myakkastreamkafka-1* > {color:#205081}_Note: This will not show information about old > Zookeeper-based consumers._{color} > {color:#205081}_TOPIC PARTITION CURRENT-OFFSET LOG-END-OFFSET LAG CONSUMER-ID > HOST CLIENT-ID_{color} > > {color:#33}That is strange. Can you please check the issue?{color} -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (KAFKA-7141) kafka-consumer-group doesn't describe existing group
[ https://issues.apache.org/jira/browse/KAFKA-7141?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16538861#comment-16538861 ] Bohdana Panchenko commented on KAFKA-7141: -- [~vahid], i have updates from my side. First: i checked setting {color:#d04437}enable.auto.commit = true. {color} {color:#d04437}{color:#33}After that change, +even with the empty topic partition, I got the desired output for the describe command.+ I{color}{color}{color:#d04437}{color:#33}s that so by design?{color}{color} *$* *kafka-consumer-groups --describe --bootstrap-server 127.0.0.1:9092 --group myakkastreamkafka-1* _Note: This will not show information about old Zookeeper-based consumers._ _TOPIC PARTITION CURRENT-OFFSET LOG-END-OFFSET LAG CONSUMER-ID HOST CLIENT-ID_ _STREAM-TEST 0 0 0 0 consumer-1-7f68330a-9c2f-4694-9978-0378368e6000 /172.19.0.8 consumer-1_ Second _:_ That made me thinking I am doing something wrong while using consumer's commit API for manual offset management. I set the {color:#d04437}enable.auto.commit = false.{color} I have double-checked and verified that i was not committing __ offsets properly. I fixed that and the result is: +before manually committing offsets - no output+ Is that by design? *$ kafka-consumer-groups --describe --bootstrap-server 127.0.0.1:9092 --group myakkastreamkafka-1* _Note: This will not show information about old Zookeeper-based consumers._ _TOPIC PARTITION CURRENT-OFFSET LOG-END-OFFSET LAG CONSUMER-ID HOST CLIENT-ID_ +after correctly manually committing offsets - desired output+ __ *$ kafka-consumer-groups --describe --bootstrap-server 127.0.0.1:9092 --group myakkastreamkafka-1* _Note: This will not show information about old Zookeeper-based consumers._ _TOPIC PARTITION CURRENT-OFFSET LOG-END-OFFSET LAG CONSUMER-ID HOST CLIENT-ID_ _STREAM-TEST 0 5 5 0 consumer-1-868599a6-272a-4929-9b28-b67de153fab4 /172.17.0.1 consumer-1_ +So to summarize: is it so, that for the group to be properly described, it is required either: using automatic offset management or wait for the first offset commit from the consumer for that group?+ +Thank you in advance.+ __ > kafka-consumer-group doesn't describe existing group > > > Key: KAFKA-7141 > URL: https://issues.apache.org/jira/browse/KAFKA-7141 > Project: Kafka > Issue Type: Bug > Components: admin >Affects Versions: 0.11.0.0, 1.0.1 >Reporter: Bohdana Panchenko >Priority: Major > > I am running two consumers: akka-stream-kafka consumer with standard config > section as described in the > [https://doc.akka.io/docs/akka-stream-kafka/current/consumer.html] and > kafka-console-consumer. > akka-stream-kafka consumer configuration looks like this > {color:#33}_akka.kafka.consumer{_{color} > {color:#33} _kafka-clients{_{color} > {color:#33} _group.id = "myakkastreamkafka-1"_{color} > {color:#33} _enable.auto.commit = false_{color} > } > {color:#33} }{color} > > I am able to see the both groups with the command > > *kafka-consumer-groups --bootstrap-server 127.0.0.1:9092 --list* > _Note: This will not show information about old Zookeeper-based consumers._ > > _myakkastreamkafka-1_ > _console-consumer-57171_ > {color:#33}I am able to view details about the console consumer > group{color} > *kafka-consumer-groups --describe --bootstrap-server 127.0.0.1:9092 --group > console-consumer-57171* > _{color:#205081}Note: This will not show information about old > Zookeeper-based consumers.{color}_ > _{color:#205081}TOPIC PARTITION CURRENT-OFFSET LOG-END-OFFSET LAG CONSUMER-ID > HOST CLIENT-ID{color}_ > _{color:#205081}STREAM-TEST 0 0 0 0 > consumer-1-6b928e07-196a-4322-9928-068681617878 /172.19.0.4 consumer-1{color}_ > {color:#33}But the command to describe my akka stream consumer gives me > empty output:{color} > *kafka-consumer-groups --describe --bootstrap-server 127.0.0.1:9092 --group > myakkastreamkafka-1* > {color:#205081}_Note: This will not show information about old > Zookeeper-based consumers._{color} > {color:#205081}_TOPIC PARTITION CURRENT-OFFSET LOG-END-OFFSET LAG CONSUMER-ID > HOST CLIENT-ID_{color} > > {color:#33}That is strange. Can you please check the issue?{color} -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (KAFKA-7141) kafka-consumer-group doesn't describe existing group
[ https://issues.apache.org/jira/browse/KAFKA-7141?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16538400#comment-16538400 ] Bohdana Panchenko commented on KAFKA-7141: -- Steps to reproduce the issue: I am using Windows and Docker Client for Windows. Kafka broker is running as a part of the Landoop distribution [https://github.com/Landoop/fast-data-dev.] Setup looks like this h4. Kafka 1.0.1-L0 @ Landoop's Apache Kafka Distribution 1× Broker, 1× Schema Registry, 1× Connect Distributed Worker, 1× REST Proxy, 1× Zookeeper My Alpakka Kafka connector client and my scala producer are running in other docker containers - all are in the custom bridge network on my local dev machine. I can see my topic in the landoop topic ui and all the messages submitted by my scala producer. My alpakka consumer processes them happily and delegates offsets committing to the alpakka library ( {color:#d04437}_enable.auto.commit = false and https://doc.akka.io/docs/akka-stream-kafka/current/consumer.html#offset-storage-in-kafka-committing_){color}. 1) start fast-data-dev broker, any producer, alpakka consumer - all in separate docker containers. Start console consumer from the kafka command line. 2) Send message, verify it has been received from both consumers. 3) Describe both consumers - and compare the output of the describe command for the console consumer with the output for the alpakka consumer. What do you mean by " after manually committing offsets"? I am using _enable.auto.commit = false -_ is that not enough? > kafka-consumer-group doesn't describe existing group > > > Key: KAFKA-7141 > URL: https://issues.apache.org/jira/browse/KAFKA-7141 > Project: Kafka > Issue Type: Bug > Components: admin >Affects Versions: 0.11.0.0, 1.0.1 >Reporter: Bohdana Panchenko >Priority: Major > > I am running two consumers: akka-stream-kafka consumer with standard config > section as described in the > [https://doc.akka.io/docs/akka-stream-kafka/current/consumer.html] and > kafka-console-consumer. > akka-stream-kafka consumer configuration looks like this > {color:#33}_akka.kafka.consumer{_{color} > {color:#33} _kafka-clients{_{color} > {color:#33} _group.id = "myakkastreamkafka-1"_{color} > {color:#33} _enable.auto.commit = false_{color} > } > {color:#33} }{color} > > I am able to see the both groups with the command > > *kafka-consumer-groups --bootstrap-server 127.0.0.1:9092 --list* > _Note: This will not show information about old Zookeeper-based consumers._ > > _myakkastreamkafka-1_ > _console-consumer-57171_ > {color:#33}I am able to view details about the console consumer > group{color} > *kafka-consumer-groups --describe --bootstrap-server 127.0.0.1:9092 --group > console-consumer-57171* > _{color:#205081}Note: This will not show information about old > Zookeeper-based consumers.{color}_ > _{color:#205081}TOPIC PARTITION CURRENT-OFFSET LOG-END-OFFSET LAG CONSUMER-ID > HOST CLIENT-ID{color}_ > _{color:#205081}STREAM-TEST 0 0 0 0 > consumer-1-6b928e07-196a-4322-9928-068681617878 /172.19.0.4 consumer-1{color}_ > {color:#33}But the command to describe my akka stream consumer gives me > empty output:{color} > *kafka-consumer-groups --describe --bootstrap-server 127.0.0.1:9092 --group > myakkastreamkafka-1* > {color:#205081}_Note: This will not show information about old > Zookeeper-based consumers._{color} > {color:#205081}_TOPIC PARTITION CURRENT-OFFSET LOG-END-OFFSET LAG CONSUMER-ID > HOST CLIENT-ID_{color} > > {color:#33}That is strange. Can you please check the issue?{color} -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (KAFKA-7141) kafka-consumer-group doesn't describe existing group
[ https://issues.apache.org/jira/browse/KAFKA-7141?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16538255#comment-16538255 ] Bohdana Panchenko commented on KAFKA-7141: -- [~vahid], let me check and i will be back > kafka-consumer-group doesn't describe existing group > > > Key: KAFKA-7141 > URL: https://issues.apache.org/jira/browse/KAFKA-7141 > Project: Kafka > Issue Type: Bug > Components: admin >Affects Versions: 0.11.0.0, 1.0.1 >Reporter: Bohdana Panchenko >Priority: Major > > I am running two consumers: akka-stream-kafka consumer with standard config > section as described in the > [https://doc.akka.io/docs/akka-stream-kafka/current/consumer.html] and > kafka-console-consumer. > akka-stream-kafka consumer configuration looks like this > {color:#33}_akka.kafka.consumer{_{color} > {color:#33} _kafka-clients{_{color} > {color:#33} _group.id = "myakkastreamkafka-1"_{color} > {color:#33} _enable.auto.commit = false_{color} > } > {color:#33} }{color} > > I am able to see the both groups with the command > > *kafka-consumer-groups --bootstrap-server 127.0.0.1:9092 --list* > _Note: This will not show information about old Zookeeper-based consumers._ > > _myakkastreamkafka-1_ > _console-consumer-57171_ > {color:#33}I am able to view details about the console consumer > group{color} > *kafka-consumer-groups --describe --bootstrap-server 127.0.0.1:9092 --group > console-consumer-57171* > _{color:#205081}Note: This will not show information about old > Zookeeper-based consumers.{color}_ > _{color:#205081}TOPIC PARTITION CURRENT-OFFSET LOG-END-OFFSET LAG CONSUMER-ID > HOST CLIENT-ID{color}_ > _{color:#205081}STREAM-TEST 0 0 0 0 > consumer-1-6b928e07-196a-4322-9928-068681617878 /172.19.0.4 consumer-1{color}_ > {color:#33}But the command to describe my akka stream consumer gives me > empty output:{color} > *kafka-consumer-groups --describe --bootstrap-server 127.0.0.1:9092 --group > myakkastreamkafka-1* > {color:#205081}_Note: This will not show information about old > Zookeeper-based consumers._{color} > {color:#205081}_TOPIC PARTITION CURRENT-OFFSET LOG-END-OFFSET LAG CONSUMER-ID > HOST CLIENT-ID_{color} > > {color:#33}That is strange. Can you please check the issue?{color} -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (KAFKA-7141) kafka-consumer-group doesn't describe existing group
[ https://issues.apache.org/jira/browse/KAFKA-7141?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16537665#comment-16537665 ] Vahid Hashemian commented on KAFKA-7141: [~kioria] thanks for reporting the issue. Do you see that output after manually committing offsets? Or before? It would be great help if you could provide the steps that reproduces the issue. Thanks! > kafka-consumer-group doesn't describe existing group > > > Key: KAFKA-7141 > URL: https://issues.apache.org/jira/browse/KAFKA-7141 > Project: Kafka > Issue Type: Bug > Components: admin >Affects Versions: 0.11.0.0, 1.0.1 >Reporter: Bohdana Panchenko >Priority: Major > > I am running two consumers: akka-stream-kafka consumer with standard config > section as described in the > [https://doc.akka.io/docs/akka-stream-kafka/current/consumer.html] and > kafka-console-consumer. > akka-stream-kafka consumer configuration looks like this > {color:#33}_akka.kafka.consumer{_{color} > {color:#33} _kafka-clients{_{color} > {color:#33} _group.id = "myakkastreamkafka-1"_{color} > {color:#33} _enable.auto.commit = false_{color} > } > {color:#33} }{color} > > I am able to see the both groups with the command > > *kafka-consumer-groups --bootstrap-server 127.0.0.1:9092 --list* > _Note: This will not show information about old Zookeeper-based consumers._ > > _myakkastreamkafka-1_ > _console-consumer-57171_ > {color:#33}I am able to view details about the console consumer > group{color} > *kafka-consumer-groups --describe --bootstrap-server 127.0.0.1:9092 --group > console-consumer-57171* > _{color:#205081}Note: This will not show information about old > Zookeeper-based consumers.{color}_ > _{color:#205081}TOPIC PARTITION CURRENT-OFFSET LOG-END-OFFSET LAG CONSUMER-ID > HOST CLIENT-ID{color}_ > _{color:#205081}STREAM-TEST 0 0 0 0 > consumer-1-6b928e07-196a-4322-9928-068681617878 /172.19.0.4 consumer-1{color}_ > {color:#33}But the command to describe my akka stream consumer gives me > empty output:{color} > *kafka-consumer-groups --describe --bootstrap-server 127.0.0.1:9092 --group > myakkastreamkafka-1* > {color:#205081}_Note: This will not show information about old > Zookeeper-based consumers._{color} > {color:#205081}_TOPIC PARTITION CURRENT-OFFSET LOG-END-OFFSET LAG CONSUMER-ID > HOST CLIENT-ID_{color} > > {color:#33}That is strange. Can you please check the issue?{color} -- This message was sent by Atlassian JIRA (v7.6.3#76005)