[jira] [Commented] (KAFKA-7141) kafka-consumer-group doesn't describe existing group

2018-07-20 Thread ASF GitHub Bot (JIRA)


[ 
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

2018-07-11 Thread huxihx (JIRA)


[ 
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

2018-07-11 Thread ASF GitHub Bot (JIRA)


[ 
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

2018-07-10 Thread James Cheng (JIRA)


[ 
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

2018-07-10 Thread Vahid Hashemian (JIRA)


[ 
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

2018-07-10 Thread Bohdana Panchenko (JIRA)


[ 
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

2018-07-10 Thread Bohdana Panchenko (JIRA)


[ 
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

2018-07-10 Thread Bohdana Panchenko (JIRA)


[ 
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

2018-07-09 Thread Vahid Hashemian (JIRA)


[ 
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)