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

huxihx edited comment on KAFKA-7141 at 7/11/18 6:24 AM:
--------------------------------------------------------

[~wushujames] Agreed. This makes no sense that whether to commit offsets 
impacts 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}
 


was (Author: huxi_2b):
[~wushujames] Agreed. This makes no sense that 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:#333333}_akka.kafka.consumer{_{color}
> {color:#333333}  _kafka-clients{_{color}
> {color:#333333}    _group.id = "myakkastreamkafka-1"_{color}
> {color:#333333}   _enable.auto.commit = false_{color}
> }
> {color:#333333} }{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:#333333}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:#333333}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:#333333}That is strange. Can you please check the issue?{color}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to