diff --git a/bin/kafka-consumer-groups.sh b/bin/kafka-consumer-groups.sh index feb063de756..6dde7d708f7 100755 --- a/bin/kafka-consumer-groups.sh +++ b/bin/kafka-consumer-groups.sh @@ -14,4 +14,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -exec $(dirname $0)/kafka-run-class.sh kafka.admin.ConsumerGroupCommand "$@" +exec $(dirname $0)/kafka-run-class.sh org.apache.kafka.tools.consumer.group.ConsumerGroupCommand "$@" diff --git a/bin/windows/kafka-consumer-groups.bat b/bin/windows/kafka-consumer-groups.bat index e027b9e6bfe..bdec36be41d 100644 --- a/bin/windows/kafka-consumer-groups.bat +++ b/bin/windows/kafka-consumer-groups.bat @@ -14,4 +14,4 @@ rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. rem See the License for the specific language governing permissions and rem limitations under the License. -"%~dp0kafka-run-class.bat" kafka.admin.ConsumerGroupCommand %* +"%~dp0kafka-run-class.bat" org.apache.kafka.tools.consumer.group.ConsumerGroupCommand %* diff --git a/build.gradle b/build.gradle index fb5cbf5869d..88d919b27c6 100644 --- a/build.gradle +++ b/build.gradle @@ -1980,6 +1980,9 @@ project(':tools') { implementation project(':log4j-appender') implementation project(':tools:tools-api') implementation libs.argparse4j + implementation libs.jacksonDatabind + implementation libs.jacksonDataformatCsv + implementation libs.jacksonJDK8Datatypes implementation libs.slf4jApi implementation libs.log4j implementation libs.joptSimple diff --git a/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala b/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala deleted file mode 100755 index 160b9a70aae..00000000000 --- a/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala +++ /dev/null @@ -1,1220 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.admin - -import com.fasterxml.jackson.databind.{ObjectReader, ObjectWriter} - -import java.time.{Duration, Instant} -import java.util.{Collections, Properties} -import com.fasterxml.jackson.dataformat.csv.CsvMapper -import com.fasterxml.jackson.module.scala.DefaultScalaModule -import kafka.utils._ -import kafka.utils.Implicits._ -import org.apache.kafka.clients.admin._ -import org.apache.kafka.clients.consumer.OffsetAndMetadata -import org.apache.kafka.clients.CommonClientConfigs -import org.apache.kafka.common.utils.Utils -import org.apache.kafka.common.{ConsumerGroupState, GroupType, KafkaException, Node, TopicPartition} -import org.apache.kafka.server.util.{CommandDefaultOptions, CommandLineUtils} - -import scala.jdk.CollectionConverters._ -import scala.collection.mutable.ListBuffer -import scala.collection.{Map, Seq, immutable, mutable} -import scala.util.{Failure, Success, Try} -import joptsimple.{OptionException, OptionSpec, OptionSpecBuilder} -import org.apache.kafka.common.protocol.Errors - -import scala.collection.immutable.TreeMap -import scala.reflect.ClassTag -import org.apache.kafka.common.requests.ListOffsetsResponse - -object ConsumerGroupCommand extends Logging { - - def main(args: Array[String]): Unit = { - - val opts = new ConsumerGroupCommandOptions(args) - try { - opts.checkArgs() - CommandLineUtils.maybePrintHelpOrVersion(opts, "This tool helps to list all consumer groups, describe a consumer group, delete consumer group info, or reset consumer group offsets.") - - // should have exactly one action - val actions = Seq(opts.listOpt, opts.describeOpt, opts.deleteOpt, opts.resetOffsetsOpt, opts.deleteOffsetsOpt).count(opts.options.has) - if (actions != 1) - CommandLineUtils.printUsageAndExit(opts.parser, "Command must include exactly one action: --list, --describe, --delete, --reset-offsets, --delete-offsets") - - run(opts) - } catch { - case e: OptionException => - CommandLineUtils.printUsageAndExit(opts.parser, e.getMessage) - } - } - - def run(opts: ConsumerGroupCommandOptions): Unit = { - val consumerGroupService = new ConsumerGroupService(opts) - try { - if (opts.options.has(opts.listOpt)) - consumerGroupService.listGroups() - else if (opts.options.has(opts.describeOpt)) - consumerGroupService.describeGroups() - else if (opts.options.has(opts.deleteOpt)) - consumerGroupService.deleteGroups() - else if (opts.options.has(opts.resetOffsetsOpt)) { - val offsetsToReset = consumerGroupService.resetOffsets() - if (opts.options.has(opts.exportOpt)) { - val exported = consumerGroupService.exportOffsetsToCsv(offsetsToReset) - println(exported) - } else - printOffsetsToReset(offsetsToReset) - } - else if (opts.options.has(opts.deleteOffsetsOpt)) { - consumerGroupService.deleteOffsets() - } - } catch { - case e: IllegalArgumentException => - CommandLineUtils.printUsageAndExit(opts.parser, e.getMessage) - case e: Throwable => - printError(s"Executing consumer group command failed due to ${e.getMessage}", Some(e)) - } finally { - consumerGroupService.close() - } - } - - def consumerGroupStatesFromString(input: String): Set[ConsumerGroupState] = { - val parsedStates = input.split(',').map(s => ConsumerGroupState.parse(s.trim)).toSet - if (parsedStates.contains(ConsumerGroupState.UNKNOWN)) { - val validStates = ConsumerGroupState.values().filter(_ != ConsumerGroupState.UNKNOWN) - throw new IllegalArgumentException(s"Invalid state list '$input'. Valid states are: ${validStates.mkString(", ")}") - } - parsedStates - } - - def consumerGroupTypesFromString(input: String): Set[GroupType] = { - val parsedTypes = input.toLowerCase.split(',').map(s => GroupType.parse(s.trim)).toSet - if (parsedTypes.contains(GroupType.UNKNOWN)) { - val validTypes = GroupType.values().filter(_ != GroupType.UNKNOWN) - throw new IllegalArgumentException(s"Invalid types list '$input'. Valid types are: ${validTypes.mkString(", ")}") - } - parsedTypes - } - - val MISSING_COLUMN_VALUE = "-" - - private def printError(msg: String, e: Option[Throwable] = None): Unit = { - println(s"\nError: $msg") - e.foreach(_.printStackTrace()) - } - - private def printOffsetsToReset(groupAssignmentsToReset: Map[String, Map[TopicPartition, OffsetAndMetadata]]): Unit = { - val format = "%-30s %-30s %-10s %-15s" - if (groupAssignmentsToReset.nonEmpty) - println("\n" + format.format("GROUP", "TOPIC", "PARTITION", "NEW-OFFSET")) - for { - (groupId, assignment) <- groupAssignmentsToReset - (consumerAssignment, offsetAndMetadata) <- assignment - } { - println(format.format( - groupId, - consumerAssignment.topic, - consumerAssignment.partition, - offsetAndMetadata.offset)) - } - } - - case class PartitionAssignmentState(group: String, coordinator: Option[Node], topic: Option[String], - partition: Option[Int], offset: Option[Long], lag: Option[Long], - consumerId: Option[String], host: Option[String], - clientId: Option[String], logEndOffset: Option[Long]) - - private[admin] case class MemberAssignmentState(group: String, consumerId: String, host: String, clientId: String, groupInstanceId: String, - numPartitions: Int, assignment: List[TopicPartition]) - - private[admin] case class GroupState(group: String, coordinator: Node, assignmentStrategy: String, state: String, numMembers: Int) - - private[admin] sealed trait CsvRecord - private[admin] case class CsvRecordWithGroup(group: String, topic: String, partition: Int, offset: Long) extends CsvRecord - private[admin] case class CsvRecordNoGroup(topic: String, partition: Int, offset: Long) extends CsvRecord - private[admin] object CsvRecordWithGroup { - val fields: Array[String] = Array("group", "topic", "partition", "offset") - } - private[admin] object CsvRecordNoGroup { - val fields: Array[String] = Array("topic", "partition", "offset") - } - // Example: CsvUtils().readerFor[CsvRecordWithoutGroup] - private[admin] case class CsvUtils() { - val mapper = new CsvMapper - mapper.registerModule(DefaultScalaModule) - def readerFor[T <: CsvRecord : ClassTag]: ObjectReader = { - val schema = getSchema[T] - val clazz = implicitly[ClassTag[T]].runtimeClass - mapper.readerFor(clazz).`with`(schema) - } - def writerFor[T <: CsvRecord : ClassTag]: ObjectWriter = { - val schema = getSchema[T] - val clazz = implicitly[ClassTag[T]].runtimeClass - mapper.writerFor(clazz).`with`(schema) - } - private def getSchema[T <: CsvRecord : ClassTag] = { - val clazz = implicitly[ClassTag[T]].runtimeClass - - val fields = - if (classOf[CsvRecordWithGroup] == clazz) CsvRecordWithGroup.fields - else if (classOf[CsvRecordNoGroup] == clazz) CsvRecordNoGroup.fields - else throw new IllegalStateException(s"Unhandled class $clazz") - - val schema = mapper.schemaFor(clazz).sortedBy(fields: _*) - schema - } - } - - class ConsumerGroupService(val opts: ConsumerGroupCommandOptions, - private[admin] val configOverrides: Map[String, String] = Map.empty) { - - private val adminClient = createAdminClient(configOverrides) - - // We have to make sure it is evaluated once and available - private lazy val resetPlanFromFile: Option[Map[String, Map[TopicPartition, OffsetAndMetadata]]] = { - if (opts.options.has(opts.resetFromFileOpt)) { - val resetPlanPath = opts.options.valueOf(opts.resetFromFileOpt) - val resetPlanCsv = Utils.readFileAsString(resetPlanPath) - val resetPlan = parseResetPlan(resetPlanCsv) - Some(resetPlan) - } else None - } - - def listGroups(): Unit = { - val includeType = opts.options.has(opts.typeOpt) - val includeState = opts.options.has(opts.stateOpt) - - if (includeType || includeState) { - val types = typeValues() - val states = stateValues() - val listings = listConsumerGroupsWithFilters(types, states) - - printGroupInfo(listings, includeType, includeState) - - } else { - listConsumerGroups().foreach(println(_)) - } - } - - private def stateValues(): Set[ConsumerGroupState] = { - val stateValue = opts.options.valueOf(opts.stateOpt) - if (stateValue == null || stateValue.isEmpty) - Set[ConsumerGroupState]() - else - consumerGroupStatesFromString(stateValue) - } - - private def typeValues(): Set[GroupType] = { - val typeValue = opts.options.valueOf(opts.typeOpt) - if (typeValue == null || typeValue.isEmpty) - Set[GroupType]() - else - consumerGroupTypesFromString(typeValue) - } - - private def printGroupInfo(groups: List[ConsumerGroupListing], includeType: Boolean, includeState: Boolean): Unit = { - def groupId(groupListing: ConsumerGroupListing): String = groupListing.groupId - def groupType(groupListing: ConsumerGroupListing): String = groupListing.`type`().orElse(GroupType.UNKNOWN).toString - def groupState(groupListing: ConsumerGroupListing): String = groupListing.state.orElse(ConsumerGroupState.UNKNOWN).toString - - val maxGroupLen = groups.foldLeft(15)((maxLen, groupListing) => Math.max(maxLen, groupId(groupListing).length)) + 10 - var format = s"%-${maxGroupLen}s" - var header = List("GROUP") - var extractors: List[ConsumerGroupListing => String] = List(groupId) - - if (includeType) { - header = header :+ "TYPE" - extractors = extractors :+ groupType _ - format += " %-20s" - } - - if (includeState) { - header = header :+ "STATE" - extractors = extractors :+ groupState _ - format += " %-20s" - } - - println(format.format(header: _*)) - - groups.foreach { groupListing => - val info = extractors.map(extractor => extractor(groupListing)) - println(format.format(info: _*)) - } - } - - def listConsumerGroups(): List[String] = { - val result = adminClient.listConsumerGroups(withTimeoutMs(new ListConsumerGroupsOptions)) - val listings = result.all.get.asScala - listings.map(_.groupId).toList - } - - def listConsumerGroupsWithFilters(types: Set[GroupType], states: Set[ConsumerGroupState]): List[ConsumerGroupListing] = { - val listConsumerGroupsOptions = withTimeoutMs(new ListConsumerGroupsOptions()) - listConsumerGroupsOptions - .inStates(states.asJava) - .withTypes(types.asJava) - val result = adminClient.listConsumerGroups(listConsumerGroupsOptions) - result.all.get.asScala.toList - } - - private def shouldPrintMemberState(group: String, state: Option[String], numRows: Option[Int]): Boolean = { - // numRows contains the number of data rows, if any, compiled from the API call in the caller method. - // if it's undefined or 0, there is no relevant group information to display. - numRows match { - case None => - printError(s"The consumer group '$group' does not exist.") - false - case Some(num) => state match { - case Some("Dead") => - printError(s"Consumer group '$group' does not exist.") - case Some("Empty") => - Console.err.println(s"\nConsumer group '$group' has no active members.") - case Some("PreparingRebalance") | Some("CompletingRebalance") | Some("Assigning") | Some("Reconciling") => - Console.err.println(s"\nWarning: Consumer group '$group' is rebalancing.") - case Some("Stable") => - case other => - // the control should never reach here - throw new KafkaException(s"Expected a valid consumer group state, but found '${other.getOrElse("NONE")}'.") - } - !state.contains("Dead") && num > 0 - } - } - - private def size(colOpt: Option[Seq[Object]]): Option[Int] = colOpt.map(_.size) - - private def printOffsets(offsets: Map[String, (Option[String], Option[Seq[PartitionAssignmentState]])]): Unit = { - for ((groupId, (state, assignments)) <- offsets) { - if (shouldPrintMemberState(groupId, state, size(assignments))) { - // find proper columns width - var (maxGroupLen, maxTopicLen, maxConsumerIdLen, maxHostLen) = (15, 15, 15, 15) - assignments match { - case None => // do nothing - case Some(consumerAssignments) => - consumerAssignments.foreach { consumerAssignment => - maxGroupLen = Math.max(maxGroupLen, consumerAssignment.group.length) - maxTopicLen = Math.max(maxTopicLen, consumerAssignment.topic.getOrElse(MISSING_COLUMN_VALUE).length) - maxConsumerIdLen = Math.max(maxConsumerIdLen, consumerAssignment.consumerId.getOrElse(MISSING_COLUMN_VALUE).length) - maxHostLen = Math.max(maxHostLen, consumerAssignment.host.getOrElse(MISSING_COLUMN_VALUE).length) - } - } - - val format = s"%${-maxGroupLen}s %${-maxTopicLen}s %-10s %-15s %-15s %-15s %${-maxConsumerIdLen}s %${-maxHostLen}s %s" - println("\n" + format - .format("GROUP", "TOPIC", "PARTITION", "CURRENT-OFFSET", "LOG-END-OFFSET", "LAG", "CONSUMER-ID", "HOST", "CLIENT-ID")) - - assignments match { - case None => // do nothing - case Some(consumerAssignments) => - consumerAssignments.foreach { consumerAssignment => - println(format.format( - consumerAssignment.group, - consumerAssignment.topic.getOrElse(MISSING_COLUMN_VALUE), consumerAssignment.partition.getOrElse(MISSING_COLUMN_VALUE), - consumerAssignment.offset.getOrElse(MISSING_COLUMN_VALUE), consumerAssignment.logEndOffset.getOrElse(MISSING_COLUMN_VALUE), - consumerAssignment.lag.getOrElse(MISSING_COLUMN_VALUE), consumerAssignment.consumerId.getOrElse(MISSING_COLUMN_VALUE), - consumerAssignment.host.getOrElse(MISSING_COLUMN_VALUE), consumerAssignment.clientId.getOrElse(MISSING_COLUMN_VALUE)) - ) - } - } - } - } - } - - private def printMembers(members: Map[String, (Option[String], Option[Seq[MemberAssignmentState]])], verbose: Boolean): Unit = { - for ((groupId, (state, assignments)) <- members) { - if (shouldPrintMemberState(groupId, state, size(assignments))) { - // find proper columns width - var (maxGroupLen, maxConsumerIdLen, maxGroupInstanceIdLen, maxHostLen, maxClientIdLen, includeGroupInstanceId) = (15, 15, 17, 15, 15, false) - assignments match { - case None => // do nothing - case Some(memberAssignments) => - memberAssignments.foreach { memberAssignment => - maxGroupLen = Math.max(maxGroupLen, memberAssignment.group.length) - maxConsumerIdLen = Math.max(maxConsumerIdLen, memberAssignment.consumerId.length) - maxGroupInstanceIdLen = Math.max(maxGroupInstanceIdLen, memberAssignment.groupInstanceId.length) - maxHostLen = Math.max(maxHostLen, memberAssignment.host.length) - maxClientIdLen = Math.max(maxClientIdLen, memberAssignment.clientId.length) - includeGroupInstanceId = includeGroupInstanceId || memberAssignment.groupInstanceId.nonEmpty - } - } - - val wideFormat = s"%${-maxGroupLen}s %${-maxConsumerIdLen}s %${-maxGroupInstanceIdLen}s %${-maxHostLen}s %${-maxClientIdLen}s %-15s " - val shortFormat = s"%${-maxGroupLen}s %${-maxConsumerIdLen}s %${-maxHostLen}s %${-maxClientIdLen}s %-15s " - - if (includeGroupInstanceId) { - print("\n" + wideFormat - .format("GROUP", "CONSUMER-ID", "GROUP-INSTANCE-ID", "HOST", "CLIENT-ID", "#PARTITIONS")) - } else { - print("\n" + shortFormat - .format("GROUP", "CONSUMER-ID", "HOST", "CLIENT-ID", "#PARTITIONS")) - } - if (verbose) - print(s"%s".format("ASSIGNMENT")) - println() - - assignments match { - case None => // do nothing - case Some(memberAssignments) => - memberAssignments.foreach { memberAssignment => - if (includeGroupInstanceId) { - print(wideFormat.format( - memberAssignment.group, memberAssignment.consumerId, memberAssignment.groupInstanceId, memberAssignment.host, - memberAssignment.clientId, memberAssignment.numPartitions)) - } else { - print(shortFormat.format( - memberAssignment.group, memberAssignment.consumerId, memberAssignment.host, memberAssignment.clientId, memberAssignment.numPartitions)) - } - if (verbose) { - val partitions = memberAssignment.assignment match { - case List() => MISSING_COLUMN_VALUE - case assignment => - assignment.groupBy(_.topic).map { - case (topic, partitionList) => topic + partitionList.map(_.partition).sorted.mkString("(", ",", ")") - }.toList.sorted.mkString(", ") - } - print(s"%s".format(partitions)) - } - println() - } - } - } - } - } - - private def printStates(states: Map[String, GroupState]): Unit = { - for ((groupId, state) <- states) { - if (shouldPrintMemberState(groupId, Some(state.state), Some(1))) { - val coordinator = s"${state.coordinator.host}:${state.coordinator.port} (${state.coordinator.idString})" - val coordinatorColLen = Math.max(25, coordinator.length) - val format = s"\n%${-coordinatorColLen}s %-25s %-20s %-15s %s" - print(format.format("GROUP", "COORDINATOR (ID)", "ASSIGNMENT-STRATEGY", "STATE", "#MEMBERS")) - print(format.format(state.group, coordinator, state.assignmentStrategy, state.state, state.numMembers)) - println() - } - } - } - - def describeGroups(): Unit = { - val groupIds = - if (opts.options.has(opts.allGroupsOpt)) listConsumerGroups() - else opts.options.valuesOf(opts.groupOpt).asScala - val membersOptPresent = opts.options.has(opts.membersOpt) - val stateOptPresent = opts.options.has(opts.stateOpt) - val offsetsOptPresent = opts.options.has(opts.offsetsOpt) - val subActions = Seq(membersOptPresent, offsetsOptPresent, stateOptPresent).count(_ == true) - - if (subActions == 0 || offsetsOptPresent) { - val offsets = collectGroupsOffsets(groupIds) - printOffsets(offsets) - } else if (membersOptPresent) { - val members = collectGroupsMembers(groupIds, opts.options.has(opts.verboseOpt)) - printMembers(members, opts.options.has(opts.verboseOpt)) - } else { - val states = collectGroupsState(groupIds) - printStates(states) - } - } - - private def collectConsumerAssignment(group: String, - coordinator: Option[Node], - topicPartitions: Seq[TopicPartition], - getPartitionOffset: TopicPartition => Option[Long], - consumerIdOpt: Option[String], - hostOpt: Option[String], - clientIdOpt: Option[String]): Array[PartitionAssignmentState] = { - if (topicPartitions.isEmpty) { - Array[PartitionAssignmentState]( - PartitionAssignmentState(group, coordinator, None, None, None, getLag(None, None), consumerIdOpt, hostOpt, clientIdOpt, None) - ) - } - else - describePartitions(group, coordinator, topicPartitions.sortBy(_.partition), getPartitionOffset, consumerIdOpt, hostOpt, clientIdOpt) - } - - private def getLag(offset: Option[Long], logEndOffset: Option[Long]): Option[Long] = - offset.filter(_ != -1).flatMap(offset => logEndOffset.map(_ - offset)) - - private def describePartitions(group: String, - coordinator: Option[Node], - topicPartitions: Seq[TopicPartition], - getPartitionOffset: TopicPartition => Option[Long], - consumerIdOpt: Option[String], - hostOpt: Option[String], - clientIdOpt: Option[String]): Array[PartitionAssignmentState] = { - - def getDescribePartitionResult(topicPartition: TopicPartition, logEndOffsetOpt: Option[Long]): PartitionAssignmentState = { - val offset = getPartitionOffset(topicPartition) - PartitionAssignmentState(group, coordinator, Option(topicPartition.topic), Option(topicPartition.partition), offset, - getLag(offset, logEndOffsetOpt), consumerIdOpt, hostOpt, clientIdOpt, logEndOffsetOpt) - } - - getLogEndOffsets(topicPartitions).map { - logEndOffsetResult => - logEndOffsetResult._2 match { - case LogOffsetResult.LogOffset(logEndOffset) => getDescribePartitionResult(logEndOffsetResult._1, Some(logEndOffset)) - case LogOffsetResult.Unknown => getDescribePartitionResult(logEndOffsetResult._1, None) - case LogOffsetResult.Ignore => null - } - }.toArray - } - - def resetOffsets(): Map[String, Map[TopicPartition, OffsetAndMetadata]] = { - val groupIds = - if (opts.options.has(opts.allGroupsOpt)) listConsumerGroups() - else opts.options.valuesOf(opts.groupOpt).asScala - - val consumerGroups = adminClient.describeConsumerGroups( - groupIds.asJava, - withTimeoutMs(new DescribeConsumerGroupsOptions) - ).describedGroups() - - val result = - consumerGroups.asScala.foldLeft(immutable.Map[String, Map[TopicPartition, OffsetAndMetadata]]()) { - case (acc, (groupId, groupDescription)) => - groupDescription.get.state().toString match { - case "Empty" | "Dead" => - val partitionsToReset = getPartitionsToReset(groupId) - val preparedOffsets = prepareOffsetsToReset(groupId, partitionsToReset) - - // Dry-run is the default behavior if --execute is not specified - val dryRun = opts.options.has(opts.dryRunOpt) || !opts.options.has(opts.executeOpt) - if (!dryRun) { - adminClient.alterConsumerGroupOffsets( - groupId, - preparedOffsets.asJava, - withTimeoutMs(new AlterConsumerGroupOffsetsOptions) - ).all.get - } - acc.updated(groupId, preparedOffsets) - case currentState => - printError(s"Assignments can only be reset if the group '$groupId' is inactive, but the current state is $currentState.") - acc.updated(groupId, Map.empty) - } - } - result - } - - def deleteOffsets(groupId: String, topics: List[String]): (Errors, Map[TopicPartition, Throwable]) = { - val partitionLevelResult = mutable.Map[TopicPartition, Throwable]() - - val (topicWithPartitions, topicWithoutPartitions) = topics.partition(_.contains(":")) - val knownPartitions = topicWithPartitions.flatMap(parseTopicsWithPartitions) - - // Get the partitions of topics that the user did not explicitly specify the partitions - val describeTopicsResult = adminClient.describeTopics( - topicWithoutPartitions.asJava, - withTimeoutMs(new DescribeTopicsOptions)) - - val unknownPartitions = describeTopicsResult.topicNameValues().asScala.flatMap { case (topic, future) => - Try(future.get()) match { - case Success(description) => description.partitions().asScala.map { partition => - new TopicPartition(topic, partition.partition()) - } - case Failure(e) => - partitionLevelResult += new TopicPartition(topic, -1) -> e - List.empty - } - } - - val partitions = knownPartitions ++ unknownPartitions - - val deleteResult = adminClient.deleteConsumerGroupOffsets( - groupId, - partitions.toSet.asJava, - withTimeoutMs(new DeleteConsumerGroupOffsetsOptions) - ) - - var topLevelException = Errors.NONE - Try(deleteResult.all.get) match { - case Success(_) => - case Failure(e) => topLevelException = Errors.forException(e.getCause) - } - - partitions.foreach { partition => - Try(deleteResult.partitionResult(partition).get()) match { - case Success(_) => partitionLevelResult += partition -> null - case Failure(e) => partitionLevelResult += partition -> e - } - } - - (topLevelException, partitionLevelResult) - } - - def deleteOffsets(): Unit = { - val groupId = opts.options.valueOf(opts.groupOpt) - val topics = opts.options.valuesOf(opts.topicOpt).asScala.toList - - val (topLevelResult, partitionLevelResult) = deleteOffsets(groupId, topics) - - topLevelResult match { - case Errors.NONE => - println(s"Request succeed for deleting offsets with topic ${topics.mkString(", ")} group $groupId") - case Errors.INVALID_GROUP_ID => - printError(s"'$groupId' is not valid.") - case Errors.GROUP_ID_NOT_FOUND => - printError(s"'$groupId' does not exist.") - case Errors.GROUP_AUTHORIZATION_FAILED => - printError(s"Access to '$groupId' is not authorized.") - case Errors.NON_EMPTY_GROUP => - printError(s"Deleting offsets of a consumer group '$groupId' is forbidden if the group is not empty.") - case Errors.GROUP_SUBSCRIBED_TO_TOPIC | - Errors.TOPIC_AUTHORIZATION_FAILED | - Errors.UNKNOWN_TOPIC_OR_PARTITION => - printError(s"Encounter some partition level error, see the follow-up details:") - case _ => - printError(s"Encounter some unknown error: $topLevelResult") - } - - val format = "%-30s %-15s %-15s" - println("\n" + format.format("TOPIC", "PARTITION", "STATUS")) - partitionLevelResult.toList.sortBy(t => t._1.topic + t._1.partition.toString).foreach { case (tp, error) => - println(format.format( - tp.topic, - if (tp.partition >= 0) tp.partition else "Not Provided", - if (error != null) s"Error: ${error.getMessage}" else "Successful" - )) - } - } - - private[admin] def describeConsumerGroups(groupIds: Seq[String]): mutable.Map[String, ConsumerGroupDescription] = { - adminClient.describeConsumerGroups( - groupIds.asJava, - withTimeoutMs(new DescribeConsumerGroupsOptions) - ).describedGroups().asScala.map { - case (groupId, groupDescriptionFuture) => (groupId, groupDescriptionFuture.get()) - } - } - - /** - * Returns the state of the specified consumer group and partition assignment states - */ - def collectGroupOffsets(groupId: String): (Option[String], Option[Seq[PartitionAssignmentState]]) = { - collectGroupsOffsets(List(groupId)).getOrElse(groupId, (None, None)) - } - - /** - * Returns states of the specified consumer groups and partition assignment states - */ - private def collectGroupsOffsets(groupIds: Seq[String]): TreeMap[String, (Option[String], Option[Seq[PartitionAssignmentState]])] = { - val consumerGroups = describeConsumerGroups(groupIds) - - val groupOffsets = TreeMap[String, (Option[String], Option[Seq[PartitionAssignmentState]])]() ++ (for ((groupId, consumerGroup) <- consumerGroups) yield { - val state = consumerGroup.state - val committedOffsets = getCommittedOffsets(groupId) - // The admin client returns `null` as a value to indicate that there is not committed offset for a partition. - def getPartitionOffset(tp: TopicPartition): Option[Long] = committedOffsets.get(tp).filter(_ != null).map(_.offset) - var assignedTopicPartitions = ListBuffer[TopicPartition]() - val rowsWithConsumer = consumerGroup.members.asScala.filterNot(_.assignment.topicPartitions.isEmpty).toSeq - .sortBy(_.assignment.topicPartitions.size)(Ordering[Int].reverse).flatMap { consumerSummary => - val topicPartitions = consumerSummary.assignment.topicPartitions.asScala - assignedTopicPartitions = assignedTopicPartitions ++ topicPartitions - collectConsumerAssignment(groupId, Option(consumerGroup.coordinator), topicPartitions.toList, - getPartitionOffset, Some(s"${consumerSummary.consumerId}"), Some(s"${consumerSummary.host}"), - Some(s"${consumerSummary.clientId}")) - } - val unassignedPartitions = committedOffsets.filterNot { case (tp, _) => assignedTopicPartitions.contains(tp) } - val rowsWithoutConsumer = if (unassignedPartitions.nonEmpty) { - collectConsumerAssignment( - groupId, - Option(consumerGroup.coordinator), - unassignedPartitions.keySet.toSeq, - getPartitionOffset, - Some(MISSING_COLUMN_VALUE), - Some(MISSING_COLUMN_VALUE), - Some(MISSING_COLUMN_VALUE)).toSeq - } else - Seq.empty - - groupId -> (Some(state.toString), Some(rowsWithConsumer ++ rowsWithoutConsumer)) - }).toMap - - groupOffsets - } - - private[admin] def collectGroupMembers(groupId: String, verbose: Boolean): (Option[String], Option[Seq[MemberAssignmentState]]) = { - collectGroupsMembers(Seq(groupId), verbose)(groupId) - } - - private[admin] def collectGroupsMembers(groupIds: Seq[String], verbose: Boolean): TreeMap[String, (Option[String], Option[Seq[MemberAssignmentState]])] = { - val consumerGroups = describeConsumerGroups(groupIds) - TreeMap[String, (Option[String], Option[Seq[MemberAssignmentState]])]() ++ (for ((groupId, consumerGroup) <- consumerGroups) yield { - val state = consumerGroup.state.toString - val memberAssignmentStates = consumerGroup.members().asScala.map(consumer => - MemberAssignmentState( - groupId, - consumer.consumerId, - consumer.host, - consumer.clientId, - consumer.groupInstanceId.orElse(""), - consumer.assignment.topicPartitions.size(), - if (verbose) consumer.assignment.topicPartitions.asScala.toList else List() - )).toList - groupId -> (Some(state), Option(memberAssignmentStates)) - }).toMap - } - - private[admin] def collectGroupState(groupId: String): GroupState = { - collectGroupsState(Seq(groupId))(groupId) - } - - private[admin] def collectGroupsState(groupIds: Seq[String]): TreeMap[String, GroupState] = { - val consumerGroups = describeConsumerGroups(groupIds) - TreeMap[String, GroupState]() ++ (for ((groupId, groupDescription) <- consumerGroups) yield { - groupId -> GroupState( - groupId, - groupDescription.coordinator, - groupDescription.partitionAssignor(), - groupDescription.state.toString, - groupDescription.members().size - ) - }).toMap - } - - private def getLogEndOffsets(topicPartitions: Seq[TopicPartition]): Map[TopicPartition, LogOffsetResult] = { - val endOffsets = topicPartitions.map { topicPartition => - topicPartition -> OffsetSpec.latest - }.toMap - val offsets = adminClient.listOffsets( - endOffsets.asJava, - withTimeoutMs(new ListOffsetsOptions) - ).all.get - topicPartitions.map { topicPartition => - Option(offsets.get(topicPartition)) match { - case Some(listOffsetsResultInfo) => topicPartition -> LogOffsetResult.LogOffset(listOffsetsResultInfo.offset) - case _ => topicPartition -> LogOffsetResult.Unknown - } - }.toMap - } - - private def getLogStartOffsets(topicPartitions: Seq[TopicPartition]): Map[TopicPartition, LogOffsetResult] = { - val startOffsets = topicPartitions.map { topicPartition => - topicPartition -> OffsetSpec.earliest - }.toMap - val offsets = adminClient.listOffsets( - startOffsets.asJava, - withTimeoutMs(new ListOffsetsOptions) - ).all.get - topicPartitions.map { topicPartition => - Option(offsets.get(topicPartition)) match { - case Some(listOffsetsResultInfo) => topicPartition -> LogOffsetResult.LogOffset(listOffsetsResultInfo.offset) - case _ => topicPartition -> LogOffsetResult.Unknown - } - }.toMap - } - - private def getLogTimestampOffsets(topicPartitions: Seq[TopicPartition], timestamp: java.lang.Long): Map[TopicPartition, LogOffsetResult] = { - val timestampOffsets = topicPartitions.map { topicPartition => - topicPartition -> OffsetSpec.forTimestamp(timestamp) - }.toMap - val offsets = adminClient.listOffsets( - timestampOffsets.asJava, - withTimeoutMs(new ListOffsetsOptions) - ).all.get - val (successfulOffsetsForTimes, unsuccessfulOffsetsForTimes) = - offsets.asScala.partition(_._2.offset != ListOffsetsResponse.UNKNOWN_OFFSET) - - val successfulLogTimestampOffsets = successfulOffsetsForTimes.map { - case (topicPartition, listOffsetsResultInfo) => topicPartition -> LogOffsetResult.LogOffset(listOffsetsResultInfo.offset) - }.toMap - - unsuccessfulOffsetsForTimes.foreach { entry => - println(s"\nWarn: Partition " + entry._1.partition() + " from topic " + entry._1.topic() + - " is empty. Falling back to latest known offset.") - } - - successfulLogTimestampOffsets ++ getLogEndOffsets(unsuccessfulOffsetsForTimes.keySet.toSeq) - } - - def close(): Unit = { - adminClient.close() - } - - // Visibility for testing - protected def createAdminClient(configOverrides: Map[String, String]): Admin = { - val props = if (opts.options.has(opts.commandConfigOpt)) Utils.loadProps(opts.options.valueOf(opts.commandConfigOpt)) else new Properties() - props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, opts.options.valueOf(opts.bootstrapServerOpt)) - configOverrides.forKeyValue { (k, v) => props.put(k, v)} - Admin.create(props) - } - - private def withTimeoutMs [T <: AbstractOptions[T]] (options : T) = { - val t = opts.options.valueOf(opts.timeoutMsOpt).intValue() - options.timeoutMs(t) - } - - private def parseTopicsWithPartitions(topicArg: String): Seq[TopicPartition] = { - def partitionNum(partition: String): Int = { - try { - partition.toInt - } catch { - case _: NumberFormatException => - throw new IllegalArgumentException(s"Invalid partition '$partition' specified in topic arg '$topicArg''") - } - } - topicArg.split(":") match { - case Array(topic, partitions) => - partitions.split(",").map(partition => new TopicPartition(topic, partitionNum(partition))) - case _ => - throw new IllegalArgumentException(s"Invalid topic arg '$topicArg', expected topic name and partitions") - } - } - - private def parseTopicPartitionsToReset(topicArgs: Seq[String]): Seq[TopicPartition] = { - val (topicsWithPartitions, topics) = topicArgs.partition(_.contains(":")) - val specifiedPartitions = topicsWithPartitions.flatMap(parseTopicsWithPartitions) - - val unspecifiedPartitions = if (topics.nonEmpty) { - val descriptionMap = adminClient.describeTopics( - topics.asJava, - withTimeoutMs(new DescribeTopicsOptions) - ).allTopicNames().get.asScala - descriptionMap.flatMap { case (topic, description) => - description.partitions().asScala.map { tpInfo => - new TopicPartition(topic, tpInfo.partition) - } - } - } else - Seq.empty - specifiedPartitions ++ unspecifiedPartitions - } - - private def getPartitionsToReset(groupId: String): Seq[TopicPartition] = { - if (opts.options.has(opts.allTopicsOpt)) { - getCommittedOffsets(groupId).keys.toSeq - } else if (opts.options.has(opts.topicOpt)) { - val topics = opts.options.valuesOf(opts.topicOpt).asScala - parseTopicPartitionsToReset(topics) - } else { - if (opts.options.has(opts.resetFromFileOpt)) - Nil - else - ToolsUtils.printUsageAndExit(opts.parser, "One of the reset scopes should be defined: --all-topics, --topic.") - } - } - - private def getCommittedOffsets(groupId: String): Map[TopicPartition, OffsetAndMetadata] = { - adminClient.listConsumerGroupOffsets( - Collections.singletonMap(groupId, new ListConsumerGroupOffsetsSpec), - withTimeoutMs(new ListConsumerGroupOffsetsOptions()) - ).partitionsToOffsetAndMetadata(groupId).get().asScala - } - - type GroupMetadata = immutable.Map[String, immutable.Map[TopicPartition, OffsetAndMetadata]] - private def parseResetPlan(resetPlanCsv: String): GroupMetadata = { - def updateGroupMetadata(group: String, topic: String, partition: Int, offset: Long, acc: GroupMetadata) = { - val topicPartition = new TopicPartition(topic, partition) - val offsetAndMetadata = new OffsetAndMetadata(offset) - val dataMap = acc.getOrElse(group, immutable.Map()).updated(topicPartition, offsetAndMetadata) - acc.updated(group, dataMap) - } - val csvReader = CsvUtils().readerFor[CsvRecordNoGroup] - val lines = resetPlanCsv.split("\n") - val isSingleGroupQuery = opts.options.valuesOf(opts.groupOpt).size() == 1 - val isOldCsvFormat = lines.headOption.flatMap(line => - Try(csvReader.readValue[CsvRecordNoGroup](line)).toOption).nonEmpty - // Single group CSV format: "topic,partition,offset" - val dataMap = if (isSingleGroupQuery && isOldCsvFormat) { - val group = opts.options.valueOf(opts.groupOpt) - lines.foldLeft(immutable.Map[String, immutable.Map[TopicPartition, OffsetAndMetadata]]()) { (acc, line) => - val CsvRecordNoGroup(topic, partition, offset) = csvReader.readValue[CsvRecordNoGroup](line) - updateGroupMetadata(group, topic, partition, offset, acc) - } - // Multiple group CSV format: "group,topic,partition,offset" - } else { - val csvReader = CsvUtils().readerFor[CsvRecordWithGroup] - lines.foldLeft(immutable.Map[String, immutable.Map[TopicPartition, OffsetAndMetadata]]()) { (acc, line) => - val CsvRecordWithGroup(group, topic, partition, offset) = csvReader.readValue[CsvRecordWithGroup](line) - updateGroupMetadata(group, topic, partition, offset, acc) - } - } - dataMap - } - - private def prepareOffsetsToReset(groupId: String, - partitionsToReset: Seq[TopicPartition]): Map[TopicPartition, OffsetAndMetadata] = { - if (opts.options.has(opts.resetToOffsetOpt)) { - val offset = opts.options.valueOf(opts.resetToOffsetOpt) - checkOffsetsRange(partitionsToReset.map((_, offset)).toMap).map { - case (topicPartition, newOffset) => (topicPartition, new OffsetAndMetadata(newOffset)) - } - } else if (opts.options.has(opts.resetToEarliestOpt)) { - val logStartOffsets = getLogStartOffsets(partitionsToReset) - partitionsToReset.map { topicPartition => - logStartOffsets.get(topicPartition) match { - case Some(LogOffsetResult.LogOffset(offset)) => (topicPartition, new OffsetAndMetadata(offset)) - case _ => ToolsUtils.printUsageAndExit(opts.parser, s"Error getting starting offset of topic partition: $topicPartition") - } - }.toMap - } else if (opts.options.has(opts.resetToLatestOpt)) { - val logEndOffsets = getLogEndOffsets(partitionsToReset) - partitionsToReset.map { topicPartition => - logEndOffsets.get(topicPartition) match { - case Some(LogOffsetResult.LogOffset(offset)) => (topicPartition, new OffsetAndMetadata(offset)) - case _ => ToolsUtils.printUsageAndExit(opts.parser, s"Error getting ending offset of topic partition: $topicPartition") - } - }.toMap - } else if (opts.options.has(opts.resetShiftByOpt)) { - val currentCommittedOffsets = getCommittedOffsets(groupId) - val requestedOffsets = partitionsToReset.map { topicPartition => - val shiftBy = opts.options.valueOf(opts.resetShiftByOpt) - val currentOffset = currentCommittedOffsets.getOrElse(topicPartition, - throw new IllegalArgumentException(s"Cannot shift offset for partition $topicPartition since there is no current committed offset")).offset - (topicPartition, currentOffset + shiftBy) - }.toMap - checkOffsetsRange(requestedOffsets).map { - case (topicPartition, newOffset) => (topicPartition, new OffsetAndMetadata(newOffset)) - } - } else if (opts.options.has(opts.resetToDatetimeOpt)) { - val timestamp = Utils.getDateTime(opts.options.valueOf(opts.resetToDatetimeOpt)) - val logTimestampOffsets = getLogTimestampOffsets(partitionsToReset, timestamp) - partitionsToReset.map { topicPartition => - val logTimestampOffset = logTimestampOffsets.get(topicPartition) - logTimestampOffset match { - case Some(LogOffsetResult.LogOffset(offset)) => (topicPartition, new OffsetAndMetadata(offset)) - case _ => ToolsUtils.printUsageAndExit(opts.parser, s"Error getting offset by timestamp of topic partition: $topicPartition") - } - }.toMap - } else if (opts.options.has(opts.resetByDurationOpt)) { - val duration = opts.options.valueOf(opts.resetByDurationOpt) - val durationParsed = Duration.parse(duration) - val now = Instant.now() - durationParsed.negated().addTo(now) - val timestamp = now.minus(durationParsed).toEpochMilli - val logTimestampOffsets = getLogTimestampOffsets(partitionsToReset, timestamp) - partitionsToReset.map { topicPartition => - val logTimestampOffset = logTimestampOffsets.get(topicPartition) - logTimestampOffset match { - case Some(LogOffsetResult.LogOffset(offset)) => (topicPartition, new OffsetAndMetadata(offset)) - case _ => ToolsUtils.printUsageAndExit(opts.parser, s"Error getting offset by timestamp of topic partition: $topicPartition") - } - }.toMap - } else if (resetPlanFromFile.isDefined) { - resetPlanFromFile.map(resetPlan => resetPlan.get(groupId).map { resetPlanForGroup => - val requestedOffsets = resetPlanForGroup.keySet.map { topicPartition => - topicPartition -> resetPlanForGroup(topicPartition).offset - }.toMap - checkOffsetsRange(requestedOffsets).map { - case (topicPartition, newOffset) => (topicPartition, new OffsetAndMetadata(newOffset)) - } - } match { - case Some(resetPlanForGroup) => resetPlanForGroup - case None => - printError(s"No reset plan for group $groupId found") - Map[TopicPartition, OffsetAndMetadata]() - }).getOrElse(Map.empty) - } else if (opts.options.has(opts.resetToCurrentOpt)) { - val currentCommittedOffsets = getCommittedOffsets(groupId) - val (partitionsToResetWithCommittedOffset, partitionsToResetWithoutCommittedOffset) = - partitionsToReset.partition(currentCommittedOffsets.keySet.contains(_)) - - val preparedOffsetsForPartitionsWithCommittedOffset = partitionsToResetWithCommittedOffset.map { topicPartition => - (topicPartition, new OffsetAndMetadata(currentCommittedOffsets.get(topicPartition) match { - case Some(offset) => offset.offset - case None => throw new IllegalStateException(s"Expected a valid current offset for topic partition: $topicPartition") - })) - }.toMap - - val preparedOffsetsForPartitionsWithoutCommittedOffset = getLogEndOffsets(partitionsToResetWithoutCommittedOffset).map { - case (topicPartition, LogOffsetResult.LogOffset(offset)) => (topicPartition, new OffsetAndMetadata(offset)) - case (topicPartition, _) => ToolsUtils.printUsageAndExit(opts.parser, s"Error getting ending offset of topic partition: $topicPartition") - } - - preparedOffsetsForPartitionsWithCommittedOffset ++ preparedOffsetsForPartitionsWithoutCommittedOffset - } else { - ToolsUtils.printUsageAndExit(opts.parser, "Option '%s' requires one of the following scenarios: %s".format(opts.resetOffsetsOpt, opts.allResetOffsetScenarioOpts)) - } - } - - private def checkOffsetsRange(requestedOffsets: Map[TopicPartition, Long]) = { - val logStartOffsets = getLogStartOffsets(requestedOffsets.keySet.toSeq) - val logEndOffsets = getLogEndOffsets(requestedOffsets.keySet.toSeq) - requestedOffsets.map { case (topicPartition, offset) => (topicPartition, - logEndOffsets.get(topicPartition) match { - case Some(LogOffsetResult.LogOffset(endOffset)) if offset > endOffset => - warn(s"New offset ($offset) is higher than latest offset for topic partition $topicPartition. Value will be set to $endOffset") - endOffset - - case Some(_) => logStartOffsets.get(topicPartition) match { - case Some(LogOffsetResult.LogOffset(startOffset)) if offset < startOffset => - warn(s"New offset ($offset) is lower than earliest offset for topic partition $topicPartition. Value will be set to $startOffset") - startOffset - - case _ => offset - } - - case None => // the control should not reach here - throw new IllegalStateException(s"Unexpected non-existing offset value for topic partition $topicPartition") - }) - } - } - - def exportOffsetsToCsv(assignments: Map[String, Map[TopicPartition, OffsetAndMetadata]]): String = { - val isSingleGroupQuery = opts.options.valuesOf(opts.groupOpt).size() == 1 - val csvWriter = - if (isSingleGroupQuery) CsvUtils().writerFor[CsvRecordNoGroup] - else CsvUtils().writerFor[CsvRecordWithGroup] - val rows = assignments.flatMap { case (groupId, partitionInfo) => - partitionInfo.map { case (k: TopicPartition, v: OffsetAndMetadata) => - val csvRecord = - if (isSingleGroupQuery) CsvRecordNoGroup(k.topic, k.partition, v.offset) - else CsvRecordWithGroup(groupId, k.topic, k.partition, v.offset) - csvWriter.writeValueAsString(csvRecord) - } - } - rows.mkString("") - } - - def deleteGroups(): Map[String, Throwable] = { - val groupIds = - if (opts.options.has(opts.allGroupsOpt)) listConsumerGroups() - else opts.options.valuesOf(opts.groupOpt).asScala - - val groupsToDelete = adminClient.deleteConsumerGroups( - groupIds.asJava, - withTimeoutMs(new DeleteConsumerGroupsOptions) - ).deletedGroups().asScala - - val result = groupsToDelete.map { case (g, f) => - Try(f.get) match { - case Success(_) => g -> null - case Failure(e) => g -> e - } - } - - val (success, failed) = result.partition { - case (_, error) => error == null - } - - if (failed.isEmpty) { - println(s"Deletion of requested consumer groups (${success.keySet.mkString("'", "', '", "'")}) was successful.") - } - else { - printError("Deletion of some consumer groups failed:") - failed.foreach { - case (group, error) => println(s"* Group '$group' could not be deleted due to: ${error.toString}") - } - if (success.nonEmpty) - println(s"\nThese consumer groups were deleted successfully: ${success.keySet.mkString("'", "', '", "'")}") - } - - result.toMap - } - } - - sealed trait LogOffsetResult - - private object LogOffsetResult { - case class LogOffset(value: Long) extends LogOffsetResult - case object Unknown extends LogOffsetResult - case object Ignore extends LogOffsetResult - } - - class ConsumerGroupCommandOptions(args: Array[String]) extends CommandDefaultOptions(args) { - val BootstrapServerDoc = "REQUIRED: The server(s) to connect to." - private val GroupDoc = "The consumer group we wish to act on." - private val TopicDoc = "The topic whose consumer group information should be deleted or topic whose should be included in the reset offset process. " + - "In `reset-offsets` case, partitions can be specified using this format: `topic1:0,1,2`, where 0,1,2 are the partition to be included in the process. " + - "Reset-offsets also supports multiple topic inputs." - private val AllTopicsDoc = "Consider all topics assigned to a group in the `reset-offsets` process." - private val ListDoc = "List all consumer groups." - private val DescribeDoc = "Describe consumer group and list offset lag (number of messages not yet processed) related to given group." - private val AllGroupsDoc = "Apply to all consumer groups." - val nl: String = System.getProperty("line.separator") - private val DeleteDoc = "Pass in groups to delete topic partition offsets and ownership information " + - "over the entire consumer group. For instance --group g1 --group g2" - private val TimeoutMsDoc = "The timeout that can be set for some use cases. For example, it can be used when describing the group " + - "to specify the maximum amount of time in milliseconds to wait before the group stabilizes (when the group is just created, " + - "or is going through some changes)." - val CommandConfigDoc: String = "Property file containing configs to be passed to Admin Client and Consumer." - private val ResetOffsetsDoc = "Reset offsets of consumer group. Supports one consumer group at the time, and instances should be inactive" + nl + - "Has 2 execution options: --dry-run (the default) to plan which offsets to reset, and --execute to update the offsets. " + - "Additionally, the --export option is used to export the results to a CSV format." + nl + - "You must choose one of the following reset specifications: --to-datetime, --by-duration, --to-earliest, " + - "--to-latest, --shift-by, --from-file, --to-current, --to-offset." + nl + - "To define the scope use --all-topics or --topic. One scope must be specified unless you use '--from-file'." - private val DryRunDoc = "Only show results without executing changes on Consumer Groups. Supported operations: reset-offsets." - private val ExecuteDoc = "Execute operation. Supported operations: reset-offsets." - private val ExportDoc = "Export operation execution to a CSV file. Supported operations: reset-offsets." - private val ResetToOffsetDoc = "Reset offsets to a specific offset." - private val ResetFromFileDoc = "Reset offsets to values defined in CSV file." - private val ResetToDatetimeDoc = "Reset offsets to offset from datetime. Format: 'YYYY-MM-DDTHH:mm:SS.sss'" - private val ResetByDurationDoc = "Reset offsets to offset by duration from current timestamp. Format: 'PnDTnHnMnS'" - private val ResetToEarliestDoc = "Reset offsets to earliest offset." - private val ResetToLatestDoc = "Reset offsets to latest offset." - private val ResetToCurrentDoc = "Reset offsets to current offset." - private val ResetShiftByDoc = "Reset offsets shifting current offset by 'n', where 'n' can be positive or negative." - private val MembersDoc = "Describe members of the group. This option may be used with '--describe' and '--bootstrap-server' options only." + nl + - "Example: --bootstrap-server localhost:9092 --describe --group group1 --members" - private val VerboseDoc = "Provide additional information, if any, when describing the group. This option may be used " + - "with '--offsets'/'--members'/'--state' and '--bootstrap-server' options only." + nl + "Example: --bootstrap-server localhost:9092 --describe --group group1 --members --verbose" - private val OffsetsDoc = "Describe the group and list all topic partitions in the group along with their offset lag. " + - "This is the default sub-action of and may be used with '--describe' and '--bootstrap-server' options only." + nl + - "Example: --bootstrap-server localhost:9092 --describe --group group1 --offsets" - private val StateDoc = "When specified with '--describe', includes the state of the group." + nl + - "Example: --bootstrap-server localhost:9092 --describe --group group1 --state" + nl + - "When specified with '--list', it displays the state of all groups. It can also be used to list groups with specific states." + nl + - "Example: --bootstrap-server localhost:9092 --list --state stable,empty" + nl + - "This option may be used with '--describe', '--list' and '--bootstrap-server' options only." - private val TypeDoc = "When specified with '--list', it displays the types of all the groups. It can also be used to list groups with specific types." + nl + - "Example: --bootstrap-server localhost:9092 --list --type classic,consumer" + nl + - "This option may be used with the '--list' option only." - private val DeleteOffsetsDoc = "Delete offsets of consumer group. Supports one consumer group at the time, and multiple topics." - - val bootstrapServerOpt: OptionSpec[String] = parser.accepts("bootstrap-server", BootstrapServerDoc) - .withRequiredArg - .describedAs("server to connect to") - .ofType(classOf[String]) - val groupOpt: OptionSpec[String] = parser.accepts("group", GroupDoc) - .withRequiredArg - .describedAs("consumer group") - .ofType(classOf[String]) - val topicOpt: OptionSpec[String] = parser.accepts("topic", TopicDoc) - .withRequiredArg - .describedAs("topic") - .ofType(classOf[String]) - val allTopicsOpt: OptionSpecBuilder = parser.accepts("all-topics", AllTopicsDoc) - val listOpt: OptionSpecBuilder = parser.accepts("list", ListDoc) - val describeOpt: OptionSpecBuilder = parser.accepts("describe", DescribeDoc) - val allGroupsOpt: OptionSpecBuilder = parser.accepts("all-groups", AllGroupsDoc) - val deleteOpt: OptionSpecBuilder = parser.accepts("delete", DeleteDoc) - val timeoutMsOpt: OptionSpec[Long] = parser.accepts("timeout", TimeoutMsDoc) - .withRequiredArg - .describedAs("timeout (ms)") - .ofType(classOf[Long]) - .defaultsTo(5000) - val commandConfigOpt: OptionSpec[String] = parser.accepts("command-config", CommandConfigDoc) - .withRequiredArg - .describedAs("command config property file") - .ofType(classOf[String]) - val resetOffsetsOpt: OptionSpecBuilder = parser.accepts("reset-offsets", ResetOffsetsDoc) - val deleteOffsetsOpt: OptionSpecBuilder = parser.accepts("delete-offsets", DeleteOffsetsDoc) - val dryRunOpt: OptionSpecBuilder = parser.accepts("dry-run", DryRunDoc) - val executeOpt: OptionSpecBuilder = parser.accepts("execute", ExecuteDoc) - val exportOpt: OptionSpecBuilder = parser.accepts("export", ExportDoc) - val resetToOffsetOpt: OptionSpec[Long] = parser.accepts("to-offset", ResetToOffsetDoc) - .withRequiredArg() - .describedAs("offset") - .ofType(classOf[Long]) - val resetFromFileOpt: OptionSpec[String] = parser.accepts("from-file", ResetFromFileDoc) - .withRequiredArg() - .describedAs("path to CSV file") - .ofType(classOf[String]) - val resetToDatetimeOpt: OptionSpec[String] = parser.accepts("to-datetime", ResetToDatetimeDoc) - .withRequiredArg() - .describedAs("datetime") - .ofType(classOf[String]) - val resetByDurationOpt: OptionSpec[String] = parser.accepts("by-duration", ResetByDurationDoc) - .withRequiredArg() - .describedAs("duration") - .ofType(classOf[String]) - val resetToEarliestOpt: OptionSpecBuilder = parser.accepts("to-earliest", ResetToEarliestDoc) - val resetToLatestOpt: OptionSpecBuilder = parser.accepts("to-latest", ResetToLatestDoc) - val resetToCurrentOpt: OptionSpecBuilder = parser.accepts("to-current", ResetToCurrentDoc) - val resetShiftByOpt: OptionSpec[Long] = parser.accepts("shift-by", ResetShiftByDoc) - .withRequiredArg() - .describedAs("number-of-offsets") - .ofType(classOf[Long]) - val membersOpt: OptionSpecBuilder = parser.accepts("members", MembersDoc) - .availableIf(describeOpt) - val verboseOpt: OptionSpecBuilder = parser.accepts("verbose", VerboseDoc) - .availableIf(describeOpt) - val offsetsOpt: OptionSpecBuilder = parser.accepts("offsets", OffsetsDoc) - .availableIf(describeOpt) - val stateOpt: OptionSpec[String] = parser.accepts("state", StateDoc) - .availableIf(describeOpt, listOpt) - .withOptionalArg() - .ofType(classOf[String]) - val typeOpt: OptionSpec[String] = parser.accepts("type", TypeDoc) - .availableIf(listOpt) - .withOptionalArg() - .ofType(classOf[String]) - - options = parser.parse(args : _*) - - private val allGroupSelectionScopeOpts = immutable.Set[OptionSpec[_]](groupOpt, allGroupsOpt) - private val allConsumerGroupLevelOpts = immutable.Set[OptionSpec[_]](listOpt, describeOpt, deleteOpt, resetOffsetsOpt) - val allResetOffsetScenarioOpts: Set[OptionSpec[_]] = immutable.Set[OptionSpec[_]](resetToOffsetOpt, resetShiftByOpt, - resetToDatetimeOpt, resetByDurationOpt, resetToEarliestOpt, resetToLatestOpt, resetToCurrentOpt, resetFromFileOpt) - private val allDeleteOffsetsOpts = immutable.Set[OptionSpec[_]](groupOpt, topicOpt) - - def checkArgs(): Unit = { - - CommandLineUtils.checkRequiredArgs(parser, options, bootstrapServerOpt) - - if (options.has(describeOpt)) { - if (!options.has(groupOpt) && !options.has(allGroupsOpt)) - CommandLineUtils.printUsageAndExit(parser, - s"Option $describeOpt takes one of these options: ${allGroupSelectionScopeOpts.mkString(", ")}") - val mutuallyExclusiveOpts: Set[OptionSpec[_]] = Set(membersOpt, offsetsOpt, stateOpt) - if (mutuallyExclusiveOpts.toList.map(o => if (options.has(o)) 1 else 0).sum > 1) { - CommandLineUtils.printUsageAndExit(parser, - s"Option $describeOpt takes at most one of these options: ${mutuallyExclusiveOpts.mkString(", ")}") - } - if (options.has(stateOpt) && options.valueOf(stateOpt) != null) - CommandLineUtils.printUsageAndExit(parser, - s"Option $describeOpt does not take a value for $stateOpt") - } else { - if (options.has(timeoutMsOpt)) - debug(s"Option $timeoutMsOpt is applicable only when $describeOpt is used.") - } - - if (options.has(deleteOpt)) { - if (!options.has(groupOpt) && !options.has(allGroupsOpt)) - CommandLineUtils.printUsageAndExit(parser, - s"Option $deleteOpt takes one of these options: ${allGroupSelectionScopeOpts.mkString(", ")}") - if (options.has(topicOpt)) - CommandLineUtils.printUsageAndExit(parser, s"The consumer does not support topic-specific offset " + - "deletion from a consumer group.") - } - - if (options.has(deleteOffsetsOpt)) { - if (!options.has(groupOpt) || !options.has(topicOpt)) - CommandLineUtils.printUsageAndExit(parser, - s"Option $deleteOffsetsOpt takes the following options: ${allDeleteOffsetsOpts.mkString(", ")}") - } - - if (options.has(resetOffsetsOpt)) { - if (options.has(dryRunOpt) && options.has(executeOpt)) - CommandLineUtils.printUsageAndExit(parser, s"Option $resetOffsetsOpt only accepts one of $executeOpt and $dryRunOpt") - - if (!options.has(dryRunOpt) && !options.has(executeOpt)) { - Console.err.println("WARN: No action will be performed as the --execute option is missing." + - "In a future major release, the default behavior of this command will be to prompt the user before " + - "executing the reset rather than doing a dry run. You should add the --dry-run option explicitly " + - "if you are scripting this command and want to keep the current default behavior without prompting.") - } - - if (!options.has(groupOpt) && !options.has(allGroupsOpt)) - CommandLineUtils.printUsageAndExit(parser, - s"Option $resetOffsetsOpt takes one of these options: ${allGroupSelectionScopeOpts.mkString(", ")}") - CommandLineUtils.checkInvalidArgs(parser, options, resetToOffsetOpt, (allResetOffsetScenarioOpts - resetToOffsetOpt).asJava) - CommandLineUtils.checkInvalidArgs(parser, options, resetToDatetimeOpt, (allResetOffsetScenarioOpts - resetToDatetimeOpt).asJava) - CommandLineUtils.checkInvalidArgs(parser, options, resetByDurationOpt, (allResetOffsetScenarioOpts - resetByDurationOpt).asJava) - CommandLineUtils.checkInvalidArgs(parser, options, resetToEarliestOpt, (allResetOffsetScenarioOpts - resetToEarliestOpt).asJava) - CommandLineUtils.checkInvalidArgs(parser, options, resetToLatestOpt, (allResetOffsetScenarioOpts - resetToLatestOpt).asJava) - CommandLineUtils.checkInvalidArgs(parser, options, resetToCurrentOpt, (allResetOffsetScenarioOpts - resetToCurrentOpt).asJava) - CommandLineUtils.checkInvalidArgs(parser, options, resetShiftByOpt, (allResetOffsetScenarioOpts - resetShiftByOpt).asJava) - CommandLineUtils.checkInvalidArgs(parser, options, resetFromFileOpt, (allResetOffsetScenarioOpts - resetFromFileOpt).asJava) - } - - CommandLineUtils.checkInvalidArgs(parser, options, groupOpt, (allGroupSelectionScopeOpts - groupOpt).asJava) - CommandLineUtils.checkInvalidArgs(parser, options, groupOpt, (allConsumerGroupLevelOpts - describeOpt - deleteOpt - resetOffsetsOpt).asJava) - CommandLineUtils.checkInvalidArgs(parser, options, topicOpt, (allConsumerGroupLevelOpts - deleteOpt - resetOffsetsOpt).asJava ) - } - } -} diff --git a/core/src/main/scala/kafka/utils/ToolsUtils.scala b/core/src/main/scala/kafka/utils/ToolsUtils.scala index edd79c00030..ffb6214d0b6 100644 --- a/core/src/main/scala/kafka/utils/ToolsUtils.scala +++ b/core/src/main/scala/kafka/utils/ToolsUtils.scala @@ -69,7 +69,7 @@ object ToolsUtils { /** * This is a simple wrapper around `CommandLineUtils.printUsageAndExit`. * It is needed for tools migration (KAFKA-14525), as there is no Java equivalent for return type `Nothing`. - * Can be removed once [[kafka.admin.ConsumerGroupCommand]] and [[kafka.tools.ConsoleProducer]] are migrated. + * Can be removed once [[kafka.tools.ConsoleProducer]] are migrated. * * @param parser Command line options parser. * @param message Error message. diff --git a/tools/src/main/java/org/apache/kafka/tools/ToolsUtils.java b/tools/src/main/java/org/apache/kafka/tools/ToolsUtils.java index 9c6a7a0d1cc..394f5078c46 100644 --- a/tools/src/main/java/org/apache/kafka/tools/ToolsUtils.java +++ b/tools/src/main/java/org/apache/kafka/tools/ToolsUtils.java @@ -16,9 +16,11 @@ */ package org.apache.kafka.tools; +import joptsimple.OptionParser; import org.apache.kafka.common.Metric; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.server.util.CommandLineUtils; import java.io.PrintStream; import java.util.Arrays; @@ -155,4 +157,17 @@ public class ToolsUtils { return res; } + /** + * This is a simple wrapper around `CommandLineUtils.printUsageAndExit`. + * It is needed for tools migration (KAFKA-14525), as there is no Java equivalent for return type `Nothing`. + * Can be removed once [[kafka.tools.ConsoleConsumer]] + * and [[kafka.tools.ConsoleProducer]] are migrated. + * + * @param parser Command line options parser. + * @param message Error message. + */ + public static void printUsageAndExit(OptionParser parser, String message) { + CommandLineUtils.printUsageAndExit(parser, message); + throw new AssertionError("printUsageAndExit should not return, but it did."); + } } diff --git a/tools/src/main/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommand.java b/tools/src/main/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommand.java new file mode 100644 index 00000000000..2b055de6161 --- /dev/null +++ b/tools/src/main/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommand.java @@ -0,0 +1,1235 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.tools.consumer.group; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectReader; +import com.fasterxml.jackson.databind.ObjectWriter; +import joptsimple.OptionException; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.admin.AbstractOptions; +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.admin.AlterConsumerGroupOffsetsOptions; +import org.apache.kafka.clients.admin.ConsumerGroupDescription; +import org.apache.kafka.clients.admin.ConsumerGroupListing; +import org.apache.kafka.clients.admin.DeleteConsumerGroupOffsetsOptions; +import org.apache.kafka.clients.admin.DeleteConsumerGroupOffsetsResult; +import org.apache.kafka.clients.admin.DeleteConsumerGroupsOptions; +import org.apache.kafka.clients.admin.DescribeConsumerGroupsOptions; +import org.apache.kafka.clients.admin.DescribeTopicsOptions; +import org.apache.kafka.clients.admin.DescribeTopicsResult; +import org.apache.kafka.clients.admin.ListConsumerGroupOffsetsOptions; +import org.apache.kafka.clients.admin.ListConsumerGroupOffsetsSpec; +import org.apache.kafka.clients.admin.ListConsumerGroupsOptions; +import org.apache.kafka.clients.admin.ListConsumerGroupsResult; +import org.apache.kafka.clients.admin.ListOffsetsOptions; +import org.apache.kafka.clients.admin.ListOffsetsResult.ListOffsetsResultInfo; +import org.apache.kafka.clients.admin.MemberDescription; +import org.apache.kafka.clients.admin.OffsetSpec; +import org.apache.kafka.clients.admin.TopicDescription; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.ConsumerGroupState; +import org.apache.kafka.common.GroupType; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.KafkaFuture; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.requests.ListOffsetsResponse; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.server.util.CommandLineUtils; +import org.apache.kafka.tools.ToolsUtils; +import org.apache.kafka.tools.Tuple2; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.text.ParseException; +import java.time.Duration; +import java.time.Instant; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.OptionalInt; +import java.util.Properties; +import java.util.Set; +import java.util.TreeMap; +import java.util.concurrent.ExecutionException; +import java.util.function.BiFunction; +import java.util.function.Function; +import java.util.function.ToIntFunction; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +public class ConsumerGroupCommand { + private static final Logger LOGGER = LoggerFactory.getLogger(ConsumerGroupCommand.class); + + static final String MISSING_COLUMN_VALUE = "-"; + + public static void main(String[] args) { + ConsumerGroupCommandOptions opts = ConsumerGroupCommandOptions.fromArgs(args); + try { + CommandLineUtils.maybePrintHelpOrVersion(opts, "This tool helps to list all consumer groups, describe a consumer group, delete consumer group info, or reset consumer group offsets."); + + // should have exactly one action + long actions = Stream.of(opts.listOpt, opts.describeOpt, opts.deleteOpt, opts.resetOffsetsOpt, opts.deleteOffsetsOpt).filter(opts.options::has).count(); + if (actions != 1) + CommandLineUtils.printUsageAndExit(opts.parser, "Command must include exactly one action: --list, --describe, --delete, --reset-offsets, --delete-offsets"); + + run(opts); + } catch (OptionException e) { + CommandLineUtils.printUsageAndExit(opts.parser, e.getMessage()); + } + } + + static void run(ConsumerGroupCommandOptions opts) { + try (ConsumerGroupService consumerGroupService = new ConsumerGroupService(opts, Collections.emptyMap())) { + if (opts.options.has(opts.listOpt)) + consumerGroupService.listGroups(); + else if (opts.options.has(opts.describeOpt)) + consumerGroupService.describeGroups(); + else if (opts.options.has(opts.deleteOpt)) + consumerGroupService.deleteGroups(); + else if (opts.options.has(opts.resetOffsetsOpt)) { + Map> offsetsToReset = consumerGroupService.resetOffsets(); + if (opts.options.has(opts.exportOpt)) { + String exported = consumerGroupService.exportOffsetsToCsv(offsetsToReset); + System.out.println(exported); + } else + printOffsetsToReset(offsetsToReset); + } else if (opts.options.has(opts.deleteOffsetsOpt)) { + consumerGroupService.deleteOffsets(); + } + } catch (IllegalArgumentException e) { + CommandLineUtils.printUsageAndExit(opts.parser, e.getMessage()); + } catch (Throwable e) { + printError("Executing consumer group command failed due to " + e.getMessage(), Optional.of(e)); + } + } + + static Set consumerGroupStatesFromString(String input) { + Set parsedStates = Arrays.stream(input.split(",")).map(s -> ConsumerGroupState.parse(s.trim())).collect(Collectors.toSet()); + if (parsedStates.contains(ConsumerGroupState.UNKNOWN)) { + Collection validStates = Arrays.stream(ConsumerGroupState.values()).filter(s -> s != ConsumerGroupState.UNKNOWN).collect(Collectors.toList()); + throw new IllegalArgumentException("Invalid state list '" + input + "'. Valid states are: " + Utils.join(validStates, ", ")); + } + return parsedStates; + } + + @SuppressWarnings("Regexp") + static Set consumerGroupTypesFromString(String input) { + Set parsedTypes = Stream.of(input.toLowerCase().split(",")).map(s -> GroupType.parse(s.trim())).collect(Collectors.toSet()); + if (parsedTypes.contains(GroupType.UNKNOWN)) { + List validTypes = Arrays.stream(GroupType.values()).filter(t -> t != GroupType.UNKNOWN).map(Object::toString).collect(Collectors.toList()); + throw new IllegalArgumentException("Invalid types list '" + input + "'. Valid types are: " + String.join(", ", validTypes)); + } + return parsedTypes; + } + + static void printError(String msg, Optional e) { + System.out.println("\nError: " + msg); + e.ifPresent(Throwable::printStackTrace); + } + + static void printOffsetsToReset(Map> groupAssignmentsToReset) { + String format = "%-30s %-30s %-10s %-15s"; + if (!groupAssignmentsToReset.isEmpty()) + System.out.printf("\n" + format, "GROUP", "TOPIC", "PARTITION", "NEW-OFFSET"); + + groupAssignmentsToReset.forEach((groupId, assignment) -> + assignment.forEach((consumerAssignment, offsetAndMetadata) -> + System.out.printf(format, + groupId, + consumerAssignment.topic(), + consumerAssignment.partition(), + offsetAndMetadata.offset()))); + } + + @SuppressWarnings("ClassFanOutComplexity") + static class ConsumerGroupService implements AutoCloseable { + final ConsumerGroupCommandOptions opts; + final Map configOverrides; + private final Admin adminClient; + + ConsumerGroupService(ConsumerGroupCommandOptions opts, Map configOverrides) { + this.opts = opts; + this.configOverrides = configOverrides; + try { + this.adminClient = createAdminClient(configOverrides); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + Optional>> resetPlanFromFile() { + if (opts.options.has(opts.resetFromFileOpt)) { + try { + String resetPlanPath = opts.options.valueOf(opts.resetFromFileOpt); + String resetPlanCsv = Utils.readFileAsString(resetPlanPath); + Map> resetPlan = parseResetPlan(resetPlanCsv); + return Optional.of(resetPlan); + } catch (IOException e) { + throw new RuntimeException(e); + } + } else return Optional.empty(); + } + + void listGroups() throws ExecutionException, InterruptedException { + boolean includeType = opts.options.has(opts.typeOpt); + boolean includeState = opts.options.has(opts.stateOpt); + + if (includeType || includeState) { + Set types = typeValues(); + Set states = stateValues(); + List listings = listConsumerGroupsWithFilters(types, states); + + printGroupInfo(listings, includeType, includeState); + } else { + listConsumerGroups().forEach(System.out::println); + } + } + + private Set stateValues() { + String stateValue = opts.options.valueOf(opts.stateOpt); + return (stateValue == null || stateValue.isEmpty()) + ? Collections.emptySet() + : consumerGroupStatesFromString(stateValue); + } + + private Set typeValues() { + String typeValue = opts.options.valueOf(opts.typeOpt); + return (typeValue == null || typeValue.isEmpty()) + ? Collections.emptySet() + : consumerGroupTypesFromString(typeValue); + } + + private void printGroupInfo(List groups, boolean includeType, boolean includeState) { + Function groupId = ConsumerGroupListing::groupId; + Function groupType = groupListing -> groupListing.type().orElse(GroupType.UNKNOWN).toString(); + Function groupState = groupListing -> groupListing.state().orElse(ConsumerGroupState.UNKNOWN).toString(); + + OptionalInt maybeMax = groups.stream().mapToInt(groupListing -> Math.max(15, groupId.apply(groupListing).length())).max(); + int maxGroupLen = maybeMax.orElse(15) + 10; + String format = "%-" + maxGroupLen + "s"; + List header = new ArrayList<>(); + header.add("GROUP"); + List> extractors = new ArrayList<>(); + extractors.add(groupId); + + if (includeType) { + header.add("TYPE"); + extractors.add(groupType); + format += " %-20s"; + } + + if (includeState) { + header.add("STATE"); + extractors.add(groupState); + format += " %-20s"; + } + + System.out.printf(format + "%n", header.toArray(new Object[0])); + + for (ConsumerGroupListing groupListing : groups) { + Object[] info = extractors.stream().map(extractor -> extractor.apply(groupListing)).toArray(Object[]::new); + System.out.printf(format + "%n", info); + } + } + + List listConsumerGroups() { + try { + ListConsumerGroupsResult result = adminClient.listConsumerGroups(withTimeoutMs(new ListConsumerGroupsOptions())); + Collection listings = result.all().get(); + return listings.stream().map(ConsumerGroupListing::groupId).collect(Collectors.toList()); + } catch (InterruptedException | ExecutionException e) { + throw new RuntimeException(e); + } + } + + List listConsumerGroupsWithFilters(Set types, Set states) throws ExecutionException, InterruptedException { + ListConsumerGroupsOptions listConsumerGroupsOptions = withTimeoutMs(new ListConsumerGroupsOptions()); + listConsumerGroupsOptions + .inStates(states) + .withTypes(types); + ListConsumerGroupsResult result = adminClient.listConsumerGroups(listConsumerGroupsOptions); + return new ArrayList<>(result.all().get()); + } + + private boolean shouldPrintMemberState(String group, Optional state, Optional numRows) { + // numRows contains the number of data rows, if any, compiled from the API call in the caller method. + // if it's undefined or 0, there is no relevant group information to display. + if (!numRows.isPresent()) { + printError("The consumer group '" + group + "' does not exist.", Optional.empty()); + return false; + } + + int num = numRows.get(); + + String state0 = state.orElse("NONE"); + switch (state0) { + case "Dead": + printError("Consumer group '" + group + "' does not exist.", Optional.empty()); + break; + case "Empty": + System.err.println("\nConsumer group '" + group + "' has no active members."); + break; + case "PreparingRebalance": + case "CompletingRebalance": + case "Assigning": + case "Reconciling": + System.err.println("\nWarning: Consumer group '" + group + "' is rebalancing."); + break; + case "Stable": + break; + default: + // the control should never reach here + throw new KafkaException("Expected a valid consumer group state, but found '" + state0 + "'."); + } + + return !state0.contains("Dead") && num > 0; + } + + private Optional size(Optional> colOpt) { + return colOpt.map(Collection::size); + } + + private void printOffsets(Map, Optional>>> offsets) { + offsets.forEach((groupId, tuple) -> { + Optional state = tuple.v1; + Optional> assignments = tuple.v2; + + if (shouldPrintMemberState(groupId, state, size(assignments))) { + String format = printOffsetFormat(assignments); + + System.out.printf(format, "GROUP", "TOPIC", "PARTITION", "CURRENT-OFFSET", "LOG-END-OFFSET", "LAG", "CONSUMER-ID", "HOST", "CLIENT-ID"); + + if (assignments.isPresent()) { + Collection consumerAssignments = assignments.get(); + for (PartitionAssignmentState consumerAssignment : consumerAssignments) { + System.out.printf(format, + consumerAssignment.group, + consumerAssignment.topic.orElse(MISSING_COLUMN_VALUE), consumerAssignment.partition.map(Object::toString).orElse(MISSING_COLUMN_VALUE), + consumerAssignment.offset.map(Object::toString).orElse(MISSING_COLUMN_VALUE), consumerAssignment.logEndOffset.map(Object::toString).orElse(MISSING_COLUMN_VALUE), + consumerAssignment.lag.map(Object::toString).orElse(MISSING_COLUMN_VALUE), consumerAssignment.consumerId.orElse(MISSING_COLUMN_VALUE), + consumerAssignment.host.orElse(MISSING_COLUMN_VALUE), consumerAssignment.clientId.orElse(MISSING_COLUMN_VALUE) + ); + } + } + } + }); + } + + private static String printOffsetFormat(Optional> assignments) { + // find proper columns width + int maxGroupLen = 15, maxTopicLen = 15, maxConsumerIdLen = 15, maxHostLen = 15; + if (assignments.isPresent()) { + Collection consumerAssignments = assignments.get(); + for (PartitionAssignmentState consumerAssignment : consumerAssignments) { + maxGroupLen = Math.max(maxGroupLen, consumerAssignment.group.length()); + maxTopicLen = Math.max(maxTopicLen, consumerAssignment.topic.orElse(MISSING_COLUMN_VALUE).length()); + maxConsumerIdLen = Math.max(maxConsumerIdLen, consumerAssignment.consumerId.orElse(MISSING_COLUMN_VALUE).length()); + maxHostLen = Math.max(maxHostLen, consumerAssignment.host.orElse(MISSING_COLUMN_VALUE).length()); + + } + } + + String format = "\n%" + (-maxGroupLen) + "s %" + (-maxTopicLen) + "s %-10s %-15s %-15s %-15s %" + (-maxConsumerIdLen) + "s %" + (-maxHostLen) + "s %s"; + return format; + } + + private void printMembers(Map, Optional>>> members, boolean verbose) { + members.forEach((groupId, tuple) -> { + Optional state = tuple.v1; + Optional> assignments = tuple.v2; + int maxGroupLen = 15, maxConsumerIdLen = 15, maxGroupInstanceIdLen = 17, maxHostLen = 15, maxClientIdLen = 15; + boolean includeGroupInstanceId = false; + + if (shouldPrintMemberState(groupId, state, size(assignments))) { + // find proper columns width + if (assignments.isPresent()) { + for (MemberAssignmentState memberAssignment : assignments.get()) { + maxGroupLen = Math.max(maxGroupLen, memberAssignment.group.length()); + maxConsumerIdLen = Math.max(maxConsumerIdLen, memberAssignment.consumerId.length()); + maxGroupInstanceIdLen = Math.max(maxGroupInstanceIdLen, memberAssignment.groupInstanceId.length()); + maxHostLen = Math.max(maxHostLen, memberAssignment.host.length()); + maxClientIdLen = Math.max(maxClientIdLen, memberAssignment.clientId.length()); + includeGroupInstanceId = includeGroupInstanceId || !memberAssignment.groupInstanceId.isEmpty(); + } + } + } + + String format0 = "%" + -maxGroupLen + "s %" + -maxConsumerIdLen + "s %" + -maxGroupInstanceIdLen + "s %" + -maxHostLen + "s %" + -maxClientIdLen + "s %-15s "; + String format1 = "%" + -maxGroupLen + "s %" + -maxConsumerIdLen + "s %" + -maxHostLen + "s %" + -maxClientIdLen + "s %-15s "; + + if (includeGroupInstanceId) { + System.out.printf("\n" + format0, "GROUP", "CONSUMER-ID", "GROUP-INSTANCE-ID", "HOST", "CLIENT-ID", "#PARTITIONS"); + } else { + System.out.printf("\n" + format1, "GROUP", "CONSUMER-ID", "HOST", "CLIENT-ID", "#PARTITIONS"); + } + if (verbose) + System.out.printf("%s", "ASSIGNMENT"); + System.out.println(); + + if (assignments.isPresent()) { + for (MemberAssignmentState memberAssignment : assignments.get()) { + if (includeGroupInstanceId) { + System.out.printf(format0, memberAssignment.group, memberAssignment.consumerId, + memberAssignment.groupInstanceId, memberAssignment.host, memberAssignment.clientId, + memberAssignment.numPartitions); + } else { + System.out.printf(format1, memberAssignment.group, memberAssignment.consumerId, + memberAssignment.host, memberAssignment.clientId, memberAssignment.numPartitions); + } + if (verbose) { + String partitions; + + if (memberAssignment.assignment.isEmpty()) + partitions = MISSING_COLUMN_VALUE; + else { + Map> grouped = new HashMap<>(); + memberAssignment.assignment.forEach( + tp -> grouped.computeIfAbsent(tp.topic(), key -> new ArrayList<>()).add(tp)); + partitions = grouped.values().stream().map(topicPartitions -> + topicPartitions.stream().map(TopicPartition::partition).map(Object::toString).sorted().collect(Collectors.joining(",", "(", ")")) + ).sorted().collect(Collectors.joining(", ")); + } + System.out.printf("%s", partitions); + } + System.out.println(); + } + } + }); + } + + private void printStates(Map states) { + states.forEach((groupId, state) -> { + if (shouldPrintMemberState(groupId, Optional.of(state.state), Optional.of(1))) { + String coordinator = state.coordinator.host() + ":" + state.coordinator.port() + " (" + state.coordinator.idString() + ")"; + int coordinatorColLen = Math.max(25, coordinator.length()); + + String format = "\n%" + -coordinatorColLen + "s %-25s %-20s %-15s %s"; + + System.out.printf(format, "GROUP", "COORDINATOR (ID)", "ASSIGNMENT-STRATEGY", "STATE", "#MEMBERS"); + System.out.printf(format, state.group, coordinator, state.assignmentStrategy, state.state, state.numMembers); + System.out.println(); + } + }); + } + + void describeGroups() throws Exception { + Collection groupIds = opts.options.has(opts.allGroupsOpt) + ? listConsumerGroups() + : opts.options.valuesOf(opts.groupOpt); + boolean membersOptPresent = opts.options.has(opts.membersOpt); + boolean stateOptPresent = opts.options.has(opts.stateOpt); + boolean offsetsOptPresent = opts.options.has(opts.offsetsOpt); + long subActions = Stream.of(membersOptPresent, offsetsOptPresent, stateOptPresent).filter(x -> x).count(); + + if (subActions == 0 || offsetsOptPresent) { + TreeMap, Optional>>> offsets + = collectGroupsOffsets(groupIds); + printOffsets(offsets); + } else if (membersOptPresent) { + TreeMap, Optional>>> members + = collectGroupsMembers(groupIds, opts.options.has(opts.verboseOpt)); + printMembers(members, opts.options.has(opts.verboseOpt)); + } else { + TreeMap states = collectGroupsState(groupIds); + printStates(states); + } + } + + private Collection collectConsumerAssignment( + String group, + Optional coordinator, + Collection topicPartitions, + Function> getPartitionOffset, + Optional consumerIdOpt, + Optional hostOpt, + Optional clientIdOpt + ) { + if (topicPartitions.isEmpty()) { + return Collections.singleton( + new PartitionAssignmentState(group, coordinator, Optional.empty(), Optional.empty(), Optional.empty(), + getLag(Optional.empty(), Optional.empty()), consumerIdOpt, hostOpt, clientIdOpt, Optional.empty()) + ); + } else { + List topicPartitionsSorted = topicPartitions.stream().sorted(Comparator.comparingInt(TopicPartition::partition)).collect(Collectors.toList()); + return describePartitions(group, coordinator, topicPartitionsSorted, getPartitionOffset, consumerIdOpt, hostOpt, clientIdOpt); + } + } + + private Optional getLag(Optional offset, Optional logEndOffset) { + return offset.filter(o -> o != -1).flatMap(offset0 -> logEndOffset.map(end -> end - offset0)); + } + + private Collection describePartitions(String group, + Optional coordinator, + List topicPartitions, + Function> getPartitionOffset, + Optional consumerIdOpt, + Optional hostOpt, + Optional clientIdOpt) { + BiFunction, PartitionAssignmentState> getDescribePartitionResult = (topicPartition, logEndOffsetOpt) -> { + Optional offset = getPartitionOffset.apply(topicPartition); + return new PartitionAssignmentState(group, coordinator, Optional.of(topicPartition.topic()), + Optional.of(topicPartition.partition()), offset, getLag(offset, logEndOffsetOpt), + consumerIdOpt, hostOpt, clientIdOpt, logEndOffsetOpt); + }; + + return getLogEndOffsets(topicPartitions).entrySet().stream().map(logEndOffsetResult -> { + if (logEndOffsetResult.getValue() instanceof LogOffset) + return getDescribePartitionResult.apply( + logEndOffsetResult.getKey(), + Optional.of(((LogOffset) logEndOffsetResult.getValue()).value) + ); + else if (logEndOffsetResult.getValue() instanceof Unknown) + return getDescribePartitionResult.apply(logEndOffsetResult.getKey(), Optional.empty()); + else if (logEndOffsetResult.getValue() instanceof Ignore) + return null; + + throw new IllegalStateException("Unknown LogOffset subclass: " + logEndOffsetResult.getValue()); + }).collect(Collectors.toList()); + } + + Map> resetOffsets() { + List groupIds = opts.options.has(opts.allGroupsOpt) + ? listConsumerGroups() + : opts.options.valuesOf(opts.groupOpt); + + Map> consumerGroups = adminClient.describeConsumerGroups( + groupIds, + withTimeoutMs(new DescribeConsumerGroupsOptions()) + ).describedGroups(); + + Map> result = new HashMap<>(); + + consumerGroups.forEach((groupId, groupDescription) -> { + try { + String state = groupDescription.get().state().toString(); + switch (state) { + case "Empty": + case "Dead": + Collection partitionsToReset = getPartitionsToReset(groupId); + Map preparedOffsets = prepareOffsetsToReset(groupId, partitionsToReset); + + // Dry-run is the default behavior if --execute is not specified + boolean dryRun = opts.options.has(opts.dryRunOpt) || !opts.options.has(opts.executeOpt); + if (!dryRun) { + adminClient.alterConsumerGroupOffsets( + groupId, + preparedOffsets, + withTimeoutMs(new AlterConsumerGroupOffsetsOptions()) + ).all().get(); + } + + result.put(groupId, preparedOffsets); + + break; + default: + printError("Assignments can only be reset if the group '" + groupId + "' is inactive, but the current state is " + state + ".", Optional.empty()); + result.put(groupId, Collections.emptyMap()); + } + } catch (InterruptedException | ExecutionException e) { + throw new RuntimeException(e); + } + }); + + return result; + } + + Tuple2> deleteOffsets(String groupId, List topics) { + Map partitionLevelResult = new HashMap<>(); + Set topicWithPartitions = new HashSet<>(); + Set topicWithoutPartitions = new HashSet<>(); + + for (String topic : topics) { + if (topic.contains(":")) + topicWithPartitions.add(topic); + else + topicWithoutPartitions.add(topic); + } + + List knownPartitions = topicWithPartitions.stream().flatMap(this::parseTopicsWithPartitions).collect(Collectors.toList()); + + // Get the partitions of topics that the user did not explicitly specify the partitions + DescribeTopicsResult describeTopicsResult = adminClient.describeTopics( + topicWithoutPartitions, + withTimeoutMs(new DescribeTopicsOptions())); + + Iterator unknownPartitions = describeTopicsResult.topicNameValues().entrySet().stream().flatMap(e -> { + String topic = e.getKey(); + try { + return e.getValue().get().partitions().stream().map(partition -> + new TopicPartition(topic, partition.partition())); + } catch (ExecutionException | InterruptedException err) { + partitionLevelResult.put(new TopicPartition(topic, -1), err); + return Stream.empty(); + } + }).iterator(); + + Set partitions = new HashSet<>(knownPartitions); + + unknownPartitions.forEachRemaining(partitions::add); + + DeleteConsumerGroupOffsetsResult deleteResult = adminClient.deleteConsumerGroupOffsets( + groupId, + partitions, + withTimeoutMs(new DeleteConsumerGroupOffsetsOptions()) + ); + + Errors topLevelException = Errors.NONE; + + try { + deleteResult.all().get(); + } catch (ExecutionException | InterruptedException e) { + topLevelException = Errors.forException(e.getCause()); + } + + partitions.forEach(partition -> { + try { + deleteResult.partitionResult(partition).get(); + partitionLevelResult.put(partition, null); + } catch (ExecutionException | InterruptedException e) { + partitionLevelResult.put(partition, e); + } + }); + + return new Tuple2<>(topLevelException, partitionLevelResult); + } + + void deleteOffsets() { + String groupId = opts.options.valueOf(opts.groupOpt); + List topics = opts.options.valuesOf(opts.topicOpt); + + Tuple2> res = deleteOffsets(groupId, topics); + + Errors topLevelResult = res.v1; + Map partitionLevelResult = res.v2; + + switch (topLevelResult) { + case NONE: + System.out.println("Request succeed for deleting offsets with topic " + Utils.mkString(topics.stream(), "", "", ", ") + " group " + groupId); + break; + case INVALID_GROUP_ID: + printError("'" + groupId + "' is not valid.", Optional.empty()); + break; + case GROUP_ID_NOT_FOUND: + printError("'" + groupId + "' does not exist.", Optional.empty()); + break; + case GROUP_AUTHORIZATION_FAILED: + printError("Access to '" + groupId + "' is not authorized.", Optional.empty()); + break; + case NON_EMPTY_GROUP: + printError("Deleting offsets of a consumer group '" + groupId + "' is forbidden if the group is not empty.", Optional.empty()); + break; + case GROUP_SUBSCRIBED_TO_TOPIC: + case TOPIC_AUTHORIZATION_FAILED: + case UNKNOWN_TOPIC_OR_PARTITION: + printError("Encounter some partition level error, see the follow-up details:", Optional.empty()); + break; + default: + printError("Encounter some unknown error: " + topLevelResult, Optional.empty()); + } + + String format = "%-30s %-15s %-15s"; + + System.out.printf("\n" + format, "TOPIC", "PARTITION", "STATUS"); + partitionLevelResult.entrySet().stream() + .sorted(Comparator.comparing(e -> e.getKey().topic() + e.getKey().partition())) + .forEach(e -> { + TopicPartition tp = e.getKey(); + Throwable error = e.getValue(); + System.out.printf(format, + tp.topic(), + tp.partition() >= 0 ? tp.partition() : "Not Provided", + error != null ? "Error: :" + error.getMessage() : "Successful" + ); + }); + } + + Map describeConsumerGroups(Collection groupIds) throws Exception { + Map res = new HashMap<>(); + Map> stringKafkaFutureMap = adminClient.describeConsumerGroups( + groupIds, + withTimeoutMs(new DescribeConsumerGroupsOptions()) + ).describedGroups(); + + for (Map.Entry> e : stringKafkaFutureMap.entrySet()) { + res.put(e.getKey(), e.getValue().get()); + } + return res; + } + + /** + * Returns the state of the specified consumer group and partition assignment states + */ + Tuple2, Optional>> collectGroupOffsets(String groupId) throws Exception { + return collectGroupsOffsets(Collections.singletonList(groupId)).getOrDefault(groupId, new Tuple2<>(Optional.empty(), Optional.empty())); + } + + /** + * Returns states of the specified consumer groups and partition assignment states + */ + TreeMap, Optional>>> collectGroupsOffsets(Collection groupIds) throws Exception { + Map consumerGroups = describeConsumerGroups(groupIds); + TreeMap, Optional>>> groupOffsets = new TreeMap<>(); + + consumerGroups.forEach((groupId, consumerGroup) -> { + ConsumerGroupState state = consumerGroup.state(); + Map committedOffsets = getCommittedOffsets(groupId); + // The admin client returns `null` as a value to indicate that there is not committed offset for a partition. + Function> getPartitionOffset = tp -> Optional.ofNullable(committedOffsets.get(tp)).map(OffsetAndMetadata::offset); + List assignedTopicPartitions = new ArrayList<>(); + Comparator comparator = + Comparator.comparingInt(m -> m.assignment().topicPartitions().size()).reversed(); + List rowsWithConsumer = new ArrayList<>(); + consumerGroup.members().stream().filter(m -> !m.assignment().topicPartitions().isEmpty()) + .sorted(comparator) + .forEach(consumerSummary -> { + Set topicPartitions = consumerSummary.assignment().topicPartitions(); + assignedTopicPartitions.addAll(topicPartitions); + rowsWithConsumer.addAll(collectConsumerAssignment( + groupId, + Optional.of(consumerGroup.coordinator()), + topicPartitions, + getPartitionOffset, + Optional.of(consumerSummary.consumerId()), + Optional.of(consumerSummary.host()), + Optional.of(consumerSummary.clientId())) + ); + }); + Map unassignedPartitions = new HashMap<>(); + committedOffsets.entrySet().stream().filter(e -> !assignedTopicPartitions.contains(e.getKey())) + .forEach(e -> unassignedPartitions.put(e.getKey(), e.getValue())); + Collection rowsWithoutConsumer = !unassignedPartitions.isEmpty() + ? collectConsumerAssignment( + groupId, + Optional.of(consumerGroup.coordinator()), + unassignedPartitions.keySet(), + getPartitionOffset, + Optional.of(MISSING_COLUMN_VALUE), + Optional.of(MISSING_COLUMN_VALUE), + Optional.of(MISSING_COLUMN_VALUE)) + : Collections.emptyList(); + + rowsWithConsumer.addAll(rowsWithoutConsumer); + + groupOffsets.put(groupId, new Tuple2<>(Optional.of(state.toString()), Optional.of(rowsWithConsumer))); + }); + + return groupOffsets; + } + + Tuple2, Optional>> collectGroupMembers(String groupId, boolean verbose) throws Exception { + return collectGroupsMembers(Collections.singleton(groupId), verbose).get(groupId); + } + + TreeMap, Optional>>> collectGroupsMembers(Collection groupIds, boolean verbose) throws Exception { + Map consumerGroups = describeConsumerGroups(groupIds); + TreeMap, Optional>>> res = new TreeMap<>(); + + consumerGroups.forEach((groupId, consumerGroup) -> { + String state = consumerGroup.state().toString(); + List memberAssignmentStates = consumerGroup.members().stream().map(consumer -> + new MemberAssignmentState( + groupId, + consumer.consumerId(), + consumer.host(), + consumer.clientId(), + consumer.groupInstanceId().orElse(""), + consumer.assignment().topicPartitions().size(), + new ArrayList<>(verbose ? consumer.assignment().topicPartitions() : Collections.emptySet()) + )).collect(Collectors.toList()); + res.put(groupId, new Tuple2<>(Optional.of(state), Optional.of(memberAssignmentStates))); + }); + return res; + } + + GroupState collectGroupState(String groupId) throws Exception { + return collectGroupsState(Collections.singleton(groupId)).get(groupId); + } + + TreeMap collectGroupsState(Collection groupIds) throws Exception { + Map consumerGroups = describeConsumerGroups(groupIds); + TreeMap res = new TreeMap<>(); + consumerGroups.forEach((groupId, groupDescription) -> + res.put(groupId, new GroupState( + groupId, + groupDescription.coordinator(), + groupDescription.partitionAssignor(), + groupDescription.state().toString(), + groupDescription.members().size() + ))); + return res; + } + + private Map getLogEndOffsets(Collection topicPartitions) { + return getLogOffsets(topicPartitions, OffsetSpec.latest()); + } + + private Map getLogStartOffsets(Collection topicPartitions) { + return getLogOffsets(topicPartitions, OffsetSpec.earliest()); + } + + private Map getLogOffsets(Collection topicPartitions, OffsetSpec offsetSpec) { + try { + Map startOffsets = topicPartitions.stream() + .collect(Collectors.toMap(Function.identity(), tp -> offsetSpec)); + + Map offsets = adminClient.listOffsets( + startOffsets, + withTimeoutMs(new ListOffsetsOptions()) + ).all().get(); + + return topicPartitions.stream().collect(Collectors.toMap( + Function.identity(), + tp -> offsets.containsKey(tp) + ? new LogOffset(offsets.get(tp).offset()) + : new Unknown() + )); + } catch (InterruptedException | ExecutionException e) { + throw new RuntimeException(e); + } + } + + private Map getLogTimestampOffsets(Collection topicPartitions, long timestamp) { + try { + Map timestampOffsets = topicPartitions.stream() + .collect(Collectors.toMap(Function.identity(), tp -> OffsetSpec.forTimestamp(timestamp))); + + Map offsets = adminClient.listOffsets( + timestampOffsets, + withTimeoutMs(new ListOffsetsOptions()) + ).all().get(); + + Map successfulOffsetsForTimes = new HashMap<>(); + Map unsuccessfulOffsetsForTimes = new HashMap<>(); + + offsets.forEach((tp, offsetsResultInfo) -> { + if (offsetsResultInfo.offset() != ListOffsetsResponse.UNKNOWN_OFFSET) + successfulOffsetsForTimes.put(tp, offsetsResultInfo); + else + unsuccessfulOffsetsForTimes.put(tp, offsetsResultInfo); + }); + + Map successfulLogTimestampOffsets = successfulOffsetsForTimes.entrySet().stream() + .collect(Collectors.toMap(Map.Entry::getKey, e -> new LogOffset(e.getValue().offset()))); + + unsuccessfulOffsetsForTimes.forEach((tp, offsetResultInfo) -> + System.out.println("\nWarn: Partition " + tp.partition() + " from topic " + tp.topic() + + " is empty. Falling back to latest known offset.")); + + successfulLogTimestampOffsets.putAll(getLogEndOffsets(unsuccessfulOffsetsForTimes.keySet())); + + return successfulLogTimestampOffsets; + } catch (InterruptedException | ExecutionException e) { + throw new RuntimeException(e); + } + } + + @Override + public void close() { + adminClient.close(); + } + + // Visibility for testing + protected Admin createAdminClient(Map configOverrides) throws IOException { + Properties props = opts.options.has(opts.commandConfigOpt) ? Utils.loadProps(opts.options.valueOf(opts.commandConfigOpt)) : new Properties(); + props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, opts.options.valueOf(opts.bootstrapServerOpt)); + props.putAll(configOverrides); + return Admin.create(props); + } + + private > T withTimeoutMs(T options) { + int t = opts.options.valueOf(opts.timeoutMsOpt).intValue(); + return options.timeoutMs(t); + } + + private Stream parseTopicsWithPartitions(String topicArg) { + ToIntFunction partitionNum = partition -> { + try { + return Integer.parseInt(partition); + } catch (NumberFormatException e) { + throw new IllegalArgumentException("Invalid partition '" + partition + "' specified in topic arg '" + topicArg + "''"); + } + }; + + String[] arr = topicArg.split(":"); + + if (arr.length != 2) + throw new IllegalArgumentException("Invalid topic arg '" + topicArg + "', expected topic name and partitions"); + + String topic = arr[0]; + String partitions = arr[1]; + + return Arrays.stream(partitions.split(",")). + map(partition -> new TopicPartition(topic, partitionNum.applyAsInt(partition))); + } + + private List parseTopicPartitionsToReset(List topicArgs) throws ExecutionException, InterruptedException { + List topicsWithPartitions = new ArrayList<>(); + List topics = new ArrayList<>(); + + topicArgs.forEach(topicArg -> { + if (topicArg.contains(":")) + topicsWithPartitions.add(topicArg); + else + topics.add(topicArg); + }); + + List specifiedPartitions = topicsWithPartitions.stream().flatMap(this::parseTopicsWithPartitions).collect(Collectors.toList()); + + List unspecifiedPartitions = new ArrayList<>(); + + if (!topics.isEmpty()) { + Map descriptionMap = adminClient.describeTopics( + topics, + withTimeoutMs(new DescribeTopicsOptions()) + ).allTopicNames().get(); + + descriptionMap.forEach((topic, description) -> + description.partitions().forEach(tpInfo -> unspecifiedPartitions.add(new TopicPartition(topic, tpInfo.partition()))) + ); + } + + specifiedPartitions.addAll(unspecifiedPartitions); + + return specifiedPartitions; + } + + private Collection getPartitionsToReset(String groupId) throws ExecutionException, InterruptedException { + if (opts.options.has(opts.allTopicsOpt)) { + return getCommittedOffsets(groupId).keySet(); + } else if (opts.options.has(opts.topicOpt)) { + List topics = opts.options.valuesOf(opts.topicOpt); + return parseTopicPartitionsToReset(topics); + } else { + if (!opts.options.has(opts.resetFromFileOpt)) + CommandLineUtils.printUsageAndExit(opts.parser, "One of the reset scopes should be defined: --all-topics, --topic."); + + return Collections.emptyList(); + } + } + + private Map getCommittedOffsets(String groupId) { + try { + return adminClient.listConsumerGroupOffsets( + Collections.singletonMap(groupId, new ListConsumerGroupOffsetsSpec()), + withTimeoutMs(new ListConsumerGroupOffsetsOptions()) + ).partitionsToOffsetAndMetadata(groupId).get(); + } catch (InterruptedException | ExecutionException e) { + throw new RuntimeException(e); + } + } + + private Map> parseResetPlan(String resetPlanCsv) { + ObjectReader csvReader = CsvUtils.readerFor(CsvUtils.CsvRecordNoGroup.class); + String[] lines = resetPlanCsv.split("\n"); + boolean isSingleGroupQuery = opts.options.valuesOf(opts.groupOpt).size() == 1; + boolean isOldCsvFormat = false; + try { + if (lines.length > 0) { + csvReader.readValue(lines[0], CsvUtils.CsvRecordNoGroup.class); + isOldCsvFormat = true; + } + } catch (IOException e) { + e.printStackTrace(); + // Ignore. + } + + Map> dataMap = new HashMap<>(); + + try { + // Single group CSV format: "topic,partition,offset" + if (isSingleGroupQuery && isOldCsvFormat) { + String group = opts.options.valueOf(opts.groupOpt); + for (String line : lines) { + CsvUtils.CsvRecordNoGroup rec = csvReader.readValue(line, CsvUtils.CsvRecordNoGroup.class); + dataMap.computeIfAbsent(group, k -> new HashMap<>()) + .put(new TopicPartition(rec.getTopic(), rec.getPartition()), new OffsetAndMetadata(rec.getOffset())); + } + } else { + csvReader = CsvUtils.readerFor(CsvUtils.CsvRecordWithGroup.class); + for (String line : lines) { + CsvUtils.CsvRecordWithGroup rec = csvReader.readValue(line, CsvUtils.CsvRecordWithGroup.class); + dataMap.computeIfAbsent(rec.getGroup(), k -> new HashMap<>()) + .put(new TopicPartition(rec.getTopic(), rec.getPartition()), new OffsetAndMetadata(rec.getOffset())); + } + } + } catch (IOException e) { + throw new RuntimeException(e); + } + + return dataMap; + } + + @SuppressWarnings("CyclomaticComplexity") + private Map prepareOffsetsToReset(String groupId, Collection partitionsToReset) { + if (opts.options.has(opts.resetToOffsetOpt)) { + long offset = opts.options.valueOf(opts.resetToOffsetOpt); + return checkOffsetsRange(partitionsToReset.stream().collect(Collectors.toMap(Function.identity(), tp -> offset))) + .entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, e -> new OffsetAndMetadata(e.getValue()))); + } else if (opts.options.has(opts.resetToEarliestOpt)) { + Map logStartOffsets = getLogStartOffsets(partitionsToReset); + return partitionsToReset.stream().collect(Collectors.toMap(Function.identity(), topicPartition -> { + LogOffsetResult logOffsetResult = logStartOffsets.get(topicPartition); + + if (!(logOffsetResult instanceof LogOffset)) { + ToolsUtils.printUsageAndExit(opts.parser, "Error getting starting offset of topic partition: " + topicPartition); + return null; + } + + return new OffsetAndMetadata(((LogOffset) logOffsetResult).value); + })); + } else if (opts.options.has(opts.resetToLatestOpt)) { + Map logEndOffsets = getLogEndOffsets(partitionsToReset); + return partitionsToReset.stream().collect(Collectors.toMap(Function.identity(), topicPartition -> { + LogOffsetResult logOffsetResult = logEndOffsets.get(topicPartition); + + if (!(logOffsetResult instanceof LogOffset)) { + ToolsUtils.printUsageAndExit(opts.parser, "Error getting ending offset of topic partition: " + topicPartition); + return null; + } + + return new OffsetAndMetadata(((LogOffset) logOffsetResult).value); + })); + } else if (opts.options.has(opts.resetShiftByOpt)) { + Map currentCommittedOffsets = getCommittedOffsets(groupId); + Map requestedOffsets = partitionsToReset.stream().collect(Collectors.toMap(Function.identity(), topicPartition -> { + long shiftBy = opts.options.valueOf(opts.resetShiftByOpt); + OffsetAndMetadata currentOffset = currentCommittedOffsets.get(topicPartition); + + if (currentOffset == null) { + throw new IllegalArgumentException("Cannot shift offset for partition " + topicPartition + " since there is no current committed offset"); + } + + return currentOffset.offset() + shiftBy; + })); + return checkOffsetsRange(requestedOffsets).entrySet().stream() + .collect(Collectors.toMap(Map.Entry::getKey, e -> new OffsetAndMetadata(e.getValue()))); + } else if (opts.options.has(opts.resetToDatetimeOpt)) { + try { + long timestamp = Utils.getDateTime(opts.options.valueOf(opts.resetToDatetimeOpt)); + Map logTimestampOffsets = getLogTimestampOffsets(partitionsToReset, timestamp); + return partitionsToReset.stream().collect(Collectors.toMap(Function.identity(), topicPartition -> { + LogOffsetResult logTimestampOffset = logTimestampOffsets.get(topicPartition); + + if (!(logTimestampOffset instanceof LogOffset)) { + ToolsUtils.printUsageAndExit(opts.parser, "Error getting offset by timestamp of topic partition: " + topicPartition); + return null; + } + + return new OffsetAndMetadata(((LogOffset) logTimestampOffset).value); + })); + } catch (ParseException e) { + throw new RuntimeException(e); + } + } else if (opts.options.has(opts.resetByDurationOpt)) { + String duration = opts.options.valueOf(opts.resetByDurationOpt); + Duration durationParsed = Duration.parse(duration); + Instant now = Instant.now(); + durationParsed.negated().addTo(now); + long timestamp = now.minus(durationParsed).toEpochMilli(); + Map logTimestampOffsets = getLogTimestampOffsets(partitionsToReset, timestamp); + return partitionsToReset.stream().collect(Collectors.toMap(Function.identity(), topicPartition -> { + LogOffsetResult logTimestampOffset = logTimestampOffsets.get(topicPartition); + + if (!(logTimestampOffset instanceof LogOffset)) { + ToolsUtils.printUsageAndExit(opts.parser, "Error getting offset by timestamp of topic partition: " + topicPartition); + return null; + } + + return new OffsetAndMetadata(((LogOffset) logTimestampOffset).value); + })); + } else if (resetPlanFromFile().isPresent()) { + return resetPlanFromFile().map(resetPlan -> { + Map resetPlanForGroup = resetPlan.get(groupId); + + if (resetPlanForGroup == null) { + printError("No reset plan for group " + groupId + " found", Optional.empty()); + return Collections.emptyMap(); + } + + Map requestedOffsets = resetPlanForGroup.keySet().stream().collect(Collectors.toMap( + Function.identity(), + topicPartition -> resetPlanForGroup.get(topicPartition).offset())); + + return checkOffsetsRange(requestedOffsets).entrySet().stream() + .collect(Collectors.toMap(Map.Entry::getKey, e -> new OffsetAndMetadata(e.getValue()))); + }).orElseGet(Collections::emptyMap); + } else if (opts.options.has(opts.resetToCurrentOpt)) { + Map currentCommittedOffsets = getCommittedOffsets(groupId); + Collection partitionsToResetWithCommittedOffset = new ArrayList<>(); + Collection partitionsToResetWithoutCommittedOffset = new ArrayList<>(); + + for (TopicPartition topicPartition : partitionsToReset) { + if (currentCommittedOffsets.containsKey(topicPartition)) + partitionsToResetWithCommittedOffset.add(topicPartition); + else + partitionsToResetWithoutCommittedOffset.add(topicPartition); + } + + Map preparedOffsetsForPartitionsWithCommittedOffset = partitionsToResetWithCommittedOffset.stream() + .collect(Collectors.toMap(Function.identity(), topicPartition -> { + OffsetAndMetadata committedOffset = currentCommittedOffsets.get(topicPartition); + + if (committedOffset == null) { + throw new IllegalStateException("Expected a valid current offset for topic partition: " + topicPartition); + } + + return new OffsetAndMetadata(committedOffset.offset()); + })); + + Map preparedOffsetsForPartitionsWithoutCommittedOffset = getLogEndOffsets(partitionsToResetWithoutCommittedOffset) + .entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, e -> { + if (!(e.getValue() instanceof LogOffset)) { + ToolsUtils.printUsageAndExit(opts.parser, "Error getting ending offset of topic partition: " + e.getKey()); + return null; + } + + return new OffsetAndMetadata(((LogOffset) e.getValue()).value); + })); + + preparedOffsetsForPartitionsWithCommittedOffset.putAll(preparedOffsetsForPartitionsWithoutCommittedOffset); + + return preparedOffsetsForPartitionsWithCommittedOffset; + } + + ToolsUtils.printUsageAndExit(opts.parser, String.format("Option '%s' requires one of the following scenarios: %s", opts.resetOffsetsOpt, opts.allResetOffsetScenarioOpts)); + return null; + } + + private Map checkOffsetsRange(Map requestedOffsets) { + Map logStartOffsets = getLogStartOffsets(requestedOffsets.keySet()); + Map logEndOffsets = getLogEndOffsets(requestedOffsets.keySet()); + + Map res = new HashMap<>(); + + requestedOffsets.forEach((topicPartition, offset) -> { + LogOffsetResult logEndOffset = logEndOffsets.get(topicPartition); + + if (logEndOffset != null) { + if (logEndOffset instanceof LogOffset && offset > ((LogOffset) logEndOffset).value) { + long endOffset = ((LogOffset) logEndOffset).value; + LOGGER.warn("New offset (" + offset + ") is higher than latest offset for topic partition " + topicPartition + ". Value will be set to " + endOffset); + res.put(topicPartition, endOffset); + } else { + LogOffsetResult logStartOffset = logStartOffsets.get(topicPartition); + + if (logStartOffset instanceof LogOffset && offset < ((LogOffset) logStartOffset).value) { + long startOffset = ((LogOffset) logStartOffset).value; + LOGGER.warn("New offset (" + offset + ") is lower than earliest offset for topic partition " + topicPartition + ". Value will be set to " + startOffset); + res.put(topicPartition, startOffset); + } else + res.put(topicPartition, offset); + } + } else { + // the control should not reach here + throw new IllegalStateException("Unexpected non-existing offset value for topic partition " + topicPartition); + } + }); + + return res; + } + + String exportOffsetsToCsv(Map> assignments) { + boolean isSingleGroupQuery = opts.options.valuesOf(opts.groupOpt).size() == 1; + ObjectWriter csvWriter = isSingleGroupQuery + ? CsvUtils.writerFor(CsvUtils.CsvRecordNoGroup.class) + : CsvUtils.writerFor(CsvUtils.CsvRecordWithGroup.class); + + return Utils.mkString(assignments.entrySet().stream().flatMap(e -> { + String groupId = e.getKey(); + Map partitionInfo = e.getValue(); + + return partitionInfo.entrySet().stream().map(e1 -> { + TopicPartition k = e1.getKey(); + OffsetAndMetadata v = e1.getValue(); + Object csvRecord = isSingleGroupQuery + ? new CsvUtils.CsvRecordNoGroup(k.topic(), k.partition(), v.offset()) + : new CsvUtils.CsvRecordWithGroup(groupId, k.topic(), k.partition(), v.offset()); + + try { + return csvWriter.writeValueAsString(csvRecord); + } catch (JsonProcessingException err) { + throw new RuntimeException(err); + } + }); + }), "", "", ""); + } + + Map deleteGroups() { + List groupIds = opts.options.has(opts.allGroupsOpt) + ? listConsumerGroups() + : opts.options.valuesOf(opts.groupOpt); + + Map> groupsToDelete = adminClient.deleteConsumerGroups( + groupIds, + withTimeoutMs(new DeleteConsumerGroupsOptions()) + ).deletedGroups(); + + Map success = new HashMap<>(); + Map failed = new HashMap<>(); + + groupsToDelete.forEach((g, f) -> { + try { + f.get(); + success.put(g, null); + } catch (ExecutionException | InterruptedException e) { + failed.put(g, e); + } + }); + + if (failed.isEmpty()) + System.out.println("Deletion of requested consumer groups (" + Utils.mkString(success.keySet().stream(), "'", "'", "', '") + ") was successful."); + else { + printError("Deletion of some consumer groups failed:", Optional.empty()); + failed.forEach((group, error) -> System.out.println("* Group '" + group + "' could not be deleted due to: " + error)); + + if (!success.isEmpty()) + System.out.println("\nThese consumer groups were deleted successfully: " + Utils.mkString(success.keySet().stream(), "'", "', '", "'")); + } + + failed.putAll(success); + + return failed; + } + } + + interface LogOffsetResult { } + + private static class LogOffset implements LogOffsetResult { + final long value; + + LogOffset(long value) { + this.value = value; + } + } + + private static class Unknown implements LogOffsetResult { } + + private static class Ignore implements LogOffsetResult { } +} diff --git a/tools/src/main/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommandOptions.java b/tools/src/main/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommandOptions.java index 045d296444d..3eb811bcaf8 100644 --- a/tools/src/main/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommandOptions.java +++ b/tools/src/main/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommandOptions.java @@ -31,89 +31,99 @@ import static org.apache.kafka.common.utils.Utils.join; import static org.apache.kafka.tools.ToolsUtils.minus; public class ConsumerGroupCommandOptions extends CommandDefaultOptions { - public static final Logger LOGGER = LoggerFactory.getLogger(ConsumerGroupCommandOptions.class); + private static final Logger LOGGER = LoggerFactory.getLogger(ConsumerGroupCommandOptions.class); - public static final String BOOTSTRAP_SERVER_DOC = "REQUIRED: The server(s) to connect to."; - public static final String GROUP_DOC = "The consumer group we wish to act on."; - public static final String TOPIC_DOC = "The topic whose consumer group information should be deleted or topic whose should be included in the reset offset process. " + + private static final String BOOTSTRAP_SERVER_DOC = "REQUIRED: The server(s) to connect to."; + private static final String GROUP_DOC = "The consumer group we wish to act on."; + private static final String TOPIC_DOC = "The topic whose consumer group information should be deleted or topic whose should be included in the reset offset process. " + "In `reset-offsets` case, partitions can be specified using this format: `topic1:0,1,2`, where 0,1,2 are the partition to be included in the process. " + "Reset-offsets also supports multiple topic inputs."; - public static final String ALL_TOPICS_DOC = "Consider all topics assigned to a group in the `reset-offsets` process."; - public static final String LIST_DOC = "List all consumer groups."; - public static final String DESCRIBE_DOC = "Describe consumer group and list offset lag (number of messages not yet processed) related to given group."; - public static final String ALL_GROUPS_DOC = "Apply to all consumer groups."; - public static final String NL = System.lineSeparator(); - public static final String DELETE_DOC = "Pass in groups to delete topic partition offsets and ownership information " + + private static final String ALL_TOPICS_DOC = "Consider all topics assigned to a group in the `reset-offsets` process."; + private static final String LIST_DOC = "List all consumer groups."; + private static final String DESCRIBE_DOC = "Describe consumer group and list offset lag (number of messages not yet processed) related to given group."; + private static final String ALL_GROUPS_DOC = "Apply to all consumer groups."; + private static final String NL = System.lineSeparator(); + private static final String DELETE_DOC = "Pass in groups to delete topic partition offsets and ownership information " + "over the entire consumer group. For instance --group g1 --group g2"; - public static final String TIMEOUT_MS_DOC = "The timeout that can be set for some use cases. For example, it can be used when describing the group " + + private static final String TIMEOUT_MS_DOC = "The timeout that can be set for some use cases. For example, it can be used when describing the group " + "to specify the maximum amount of time in milliseconds to wait before the group stabilizes (when the group is just created, " + "or is going through some changes)."; - public static final String COMMAND_CONFIG_DOC = "Property file containing configs to be passed to Admin Client and Consumer."; - public static final String RESET_OFFSETS_DOC = "Reset offsets of consumer group. Supports one consumer group at the time, and instances should be inactive" + NL + + private static final String COMMAND_CONFIG_DOC = "Property file containing configs to be passed to Admin Client and Consumer."; + private static final String RESET_OFFSETS_DOC = "Reset offsets of consumer group. Supports one consumer group at the time, and instances should be inactive" + NL + "Has 2 execution options: --dry-run (the default) to plan which offsets to reset, and --execute to update the offsets. " + "Additionally, the --export option is used to export the results to a CSV format." + NL + "You must choose one of the following reset specifications: --to-datetime, --by-duration, --to-earliest, " + "--to-latest, --shift-by, --from-file, --to-current, --to-offset." + NL + "To define the scope use --all-topics or --topic. One scope must be specified unless you use '--from-file'."; - public static final String DRY_RUN_DOC = "Only show results without executing changes on Consumer Groups. Supported operations: reset-offsets."; - public static final String EXECUTE_DOC = "Execute operation. Supported operations: reset-offsets."; - public static final String EXPORT_DOC = "Export operation execution to a CSV file. Supported operations: reset-offsets."; - public static final String RESET_TO_OFFSET_DOC = "Reset offsets to a specific offset."; - public static final String RESET_FROM_FILE_DOC = "Reset offsets to values defined in CSV file."; - public static final String RESET_TO_DATETIME_DOC = "Reset offsets to offset from datetime. Format: 'YYYY-MM-DDTHH:mm:SS.sss'"; - public static final String RESET_BY_DURATION_DOC = "Reset offsets to offset by duration from current timestamp. Format: 'PnDTnHnMnS'"; - public static final String RESET_TO_EARLIEST_DOC = "Reset offsets to earliest offset."; - public static final String RESET_TO_LATEST_DOC = "Reset offsets to latest offset."; - public static final String RESET_TO_CURRENT_DOC = "Reset offsets to current offset."; - public static final String RESET_SHIFT_BY_DOC = "Reset offsets shifting current offset by 'n', where 'n' can be positive or negative."; - public static final String MEMBERS_DOC = "Describe members of the group. This option may be used with '--describe' and '--bootstrap-server' options only." + NL + + private static final String DRY_RUN_DOC = "Only show results without executing changes on Consumer Groups. Supported operations: reset-offsets."; + private static final String EXECUTE_DOC = "Execute operation. Supported operations: reset-offsets."; + private static final String EXPORT_DOC = "Export operation execution to a CSV file. Supported operations: reset-offsets."; + private static final String RESET_TO_OFFSET_DOC = "Reset offsets to a specific offset."; + private static final String RESET_FROM_FILE_DOC = "Reset offsets to values defined in CSV file."; + private static final String RESET_TO_DATETIME_DOC = "Reset offsets to offset from datetime. Format: 'YYYY-MM-DDTHH:mm:SS.sss'"; + private static final String RESET_BY_DURATION_DOC = "Reset offsets to offset by duration from current timestamp. Format: 'PnDTnHnMnS'"; + private static final String RESET_TO_EARLIEST_DOC = "Reset offsets to earliest offset."; + private static final String RESET_TO_LATEST_DOC = "Reset offsets to latest offset."; + private static final String RESET_TO_CURRENT_DOC = "Reset offsets to current offset."; + private static final String RESET_SHIFT_BY_DOC = "Reset offsets shifting current offset by 'n', where 'n' can be positive or negative."; + private static final String MEMBERS_DOC = "Describe members of the group. This option may be used with '--describe' and '--bootstrap-server' options only." + NL + "Example: --bootstrap-server localhost:9092 --describe --group group1 --members"; - public static final String VERBOSE_DOC = "Provide additional information, if any, when describing the group. This option may be used " + + private static final String VERBOSE_DOC = "Provide additional information, if any, when describing the group. This option may be used " + "with '--offsets'/'--members'/'--state' and '--bootstrap-server' options only." + NL + "Example: --bootstrap-server localhost:9092 --describe --group group1 --members --verbose"; - public static final String OFFSETS_DOC = "Describe the group and list all topic partitions in the group along with their offset lag. " + + private static final String OFFSETS_DOC = "Describe the group and list all topic partitions in the group along with their offset lag. " + "This is the default sub-action of and may be used with '--describe' and '--bootstrap-server' options only." + NL + "Example: --bootstrap-server localhost:9092 --describe --group group1 --offsets"; - public static final String STATE_DOC = "When specified with '--describe', includes the state of the group." + NL + + private static final String STATE_DOC = "When specified with '--describe', includes the state of the group." + NL + "Example: --bootstrap-server localhost:9092 --describe --group group1 --state" + NL + "When specified with '--list', it displays the state of all groups. It can also be used to list groups with specific states." + NL + "Example: --bootstrap-server localhost:9092 --list --state stable,empty" + NL + "This option may be used with '--describe', '--list' and '--bootstrap-server' options only."; - public static final String DELETE_OFFSETS_DOC = "Delete offsets of consumer group. Supports one consumer group at the time, and multiple topics."; + private static final String TYPE_DOC = "When specified with '--list', it displays the types of all the groups. It can also be used to list groups with specific types." + NL + + "Example: --bootstrap-server localhost:9092 --list --type classic,consumer" + NL + + "This option may be used with the '--list' option only."; + private static final String DELETE_OFFSETS_DOC = "Delete offsets of consumer group. Supports one consumer group at the time, and multiple topics."; - public final OptionSpec bootstrapServerOpt; - public final OptionSpec groupOpt; - public final OptionSpec topicOpt; - public final OptionSpec allTopicsOpt; - public final OptionSpec listOpt; - public final OptionSpec describeOpt; - public final OptionSpec allGroupsOpt; - public final OptionSpec deleteOpt; - public final OptionSpec timeoutMsOpt; - public final OptionSpec commandConfigOpt; - public final OptionSpec resetOffsetsOpt; - public final OptionSpec deleteOffsetsOpt; - public final OptionSpec dryRunOpt; - public final OptionSpec executeOpt; - public final OptionSpec exportOpt; - public final OptionSpec resetToOffsetOpt; - public final OptionSpec resetFromFileOpt; - public final OptionSpec resetToDatetimeOpt; - public final OptionSpec resetByDurationOpt; - public final OptionSpec resetToEarliestOpt; - public final OptionSpec resetToLatestOpt; - public final OptionSpec resetToCurrentOpt; - public final OptionSpec resetShiftByOpt; - public final OptionSpec membersOpt; - public final OptionSpec verboseOpt; - public final OptionSpec offsetsOpt; - public final OptionSpec stateOpt; + final OptionSpec bootstrapServerOpt; + final OptionSpec groupOpt; + final OptionSpec topicOpt; + final OptionSpec allTopicsOpt; + final OptionSpec listOpt; + final OptionSpec describeOpt; + final OptionSpec allGroupsOpt; + final OptionSpec deleteOpt; + final OptionSpec timeoutMsOpt; + final OptionSpec commandConfigOpt; + final OptionSpec resetOffsetsOpt; + final OptionSpec deleteOffsetsOpt; + final OptionSpec dryRunOpt; + final OptionSpec executeOpt; + final OptionSpec exportOpt; + final OptionSpec resetToOffsetOpt; + final OptionSpec resetFromFileOpt; + final OptionSpec resetToDatetimeOpt; + final OptionSpec resetByDurationOpt; + final OptionSpec resetToEarliestOpt; + final OptionSpec resetToLatestOpt; + final OptionSpec resetToCurrentOpt; + final OptionSpec resetShiftByOpt; + final OptionSpec membersOpt; + final OptionSpec verboseOpt; + final OptionSpec offsetsOpt; + final OptionSpec stateOpt; + final OptionSpec typeOpt; - public final Set> allGroupSelectionScopeOpts; - public final Set> allConsumerGroupLevelOpts; - public final Set> allResetOffsetScenarioOpts; - public final Set> allDeleteOffsetsOpts; + final Set> allGroupSelectionScopeOpts; + final Set> allConsumerGroupLevelOpts; + final Set> allResetOffsetScenarioOpts; + final Set> allDeleteOffsetsOpts; - public ConsumerGroupCommandOptions(String[] args) { + public static ConsumerGroupCommandOptions fromArgs(String[] args) { + ConsumerGroupCommandOptions opts = new ConsumerGroupCommandOptions(args); + opts.checkArgs(); + return opts; + } + + private ConsumerGroupCommandOptions(String[] args) { super(args); bootstrapServerOpt = parser.accepts("bootstrap-server", BOOTSTRAP_SERVER_DOC) @@ -180,6 +190,10 @@ public class ConsumerGroupCommandOptions extends CommandDefaultOptions { .availableIf(describeOpt, listOpt) .withOptionalArg() .ofType(String.class); + typeOpt = parser.accepts("type", TYPE_DOC) + .availableIf(listOpt) + .withOptionalArg() + .ofType(String.class); allGroupSelectionScopeOpts = new HashSet<>(Arrays.asList(groupOpt, allGroupsOpt)); allConsumerGroupLevelOpts = new HashSet<>(Arrays.asList(listOpt, describeOpt, deleteOpt, resetOffsetsOpt)); @@ -191,7 +205,7 @@ public class ConsumerGroupCommandOptions extends CommandDefaultOptions { } @SuppressWarnings({"CyclomaticComplexity", "NPathComplexity"}) - public void checkArgs() { + void checkArgs() { CommandLineUtils.checkRequiredArgs(parser, options, bootstrapServerOpt); if (options.has(describeOpt)) { diff --git a/tools/src/main/java/org/apache/kafka/tools/consumer/group/CsvUtils.java b/tools/src/main/java/org/apache/kafka/tools/consumer/group/CsvUtils.java new file mode 100644 index 00000000000..60a8bb22be9 --- /dev/null +++ b/tools/src/main/java/org/apache/kafka/tools/consumer/group/CsvUtils.java @@ -0,0 +1,157 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.tools.consumer.group; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.ObjectReader; +import com.fasterxml.jackson.databind.ObjectWriter; +import com.fasterxml.jackson.dataformat.csv.CsvMapper; +import com.fasterxml.jackson.dataformat.csv.CsvSchema; + +public class CsvUtils { + private final static CsvMapper MAPPER = new CsvMapper(); + + static ObjectReader readerFor(Class clazz) { + return MAPPER.readerFor(clazz).with(getSchema(clazz)); + } + + static ObjectWriter writerFor(Class clazz) { + return MAPPER.writerFor(clazz).with(getSchema(clazz)); + } + + private static CsvSchema getSchema(Class clazz) { + String[] fields; + if (CsvRecordWithGroup.class == clazz) + fields = CsvRecordWithGroup.FIELDS; + else if (CsvRecordNoGroup.class == clazz) + fields = CsvRecordNoGroup.FIELDS; + else + throw new IllegalStateException("Unhandled class " + clazz); + + return MAPPER.schemaFor(clazz).sortedBy(fields); + } + + public static class CsvRecordWithGroup { + public static final String[] FIELDS = new String[] {"group", "topic", "partition", "offset"}; + + @JsonProperty + private String group; + + @JsonProperty + private String topic; + + @JsonProperty + private int partition; + + @JsonProperty + private long offset; + + /** + * Required for jackson. + */ + public CsvRecordWithGroup() { + } + + public CsvRecordWithGroup(String group, String topic, int partition, long offset) { + this.group = group; + this.topic = topic; + this.partition = partition; + this.offset = offset; + } + + public void setGroup(String group) { + this.group = group; + } + + public String getGroup() { + return group; + } + + public String getTopic() { + return topic; + } + + public void setTopic(String topic) { + this.topic = topic; + } + + public int getPartition() { + return partition; + } + + public void setPartition(int partition) { + this.partition = partition; + } + + public long getOffset() { + return offset; + } + + public void setOffset(long offset) { + this.offset = offset; + } + } + + public static class CsvRecordNoGroup { + public static final String[] FIELDS = new String[]{"topic", "partition", "offset"}; + + @JsonProperty + private String topic; + + @JsonProperty + private int partition; + + @JsonProperty + private long offset; + + /** + * Required for jackson. + */ + public CsvRecordNoGroup() { + } + + public CsvRecordNoGroup(String topic, int partition, long offset) { + this.topic = topic; + this.partition = partition; + this.offset = offset; + } + + public String getTopic() { + return topic; + } + + public void setTopic(String topic) { + this.topic = topic; + } + + public int getPartition() { + return partition; + } + + public void setPartition(int partition) { + this.partition = partition; + } + + public long getOffset() { + return offset; + } + + public void setOffset(long offset) { + this.offset = offset; + } + } +} diff --git a/tools/src/main/java/org/apache/kafka/tools/consumer/group/GroupState.java b/tools/src/main/java/org/apache/kafka/tools/consumer/group/GroupState.java new file mode 100644 index 00000000000..04a3b2eae19 --- /dev/null +++ b/tools/src/main/java/org/apache/kafka/tools/consumer/group/GroupState.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.tools.consumer.group; + +import org.apache.kafka.common.Node; + +class GroupState { + final String group; + final Node coordinator; + final String assignmentStrategy; + final String state; + final int numMembers; + + GroupState(String group, Node coordinator, String assignmentStrategy, String state, int numMembers) { + this.group = group; + this.coordinator = coordinator; + this.assignmentStrategy = assignmentStrategy; + this.state = state; + this.numMembers = numMembers; + } +} diff --git a/tools/src/main/java/org/apache/kafka/tools/consumer/group/MemberAssignmentState.java b/tools/src/main/java/org/apache/kafka/tools/consumer/group/MemberAssignmentState.java index 040cb1c741e..9ac46428338 100644 --- a/tools/src/main/java/org/apache/kafka/tools/consumer/group/MemberAssignmentState.java +++ b/tools/src/main/java/org/apache/kafka/tools/consumer/group/MemberAssignmentState.java @@ -21,15 +21,15 @@ import org.apache.kafka.common.TopicPartition; import java.util.List; class MemberAssignmentState { - public final String group; - public final String consumerId; - public final String host; - public final String clientId; - public final String groupInstanceId; - public final int numPartitions; - public final List assignment; + final String group; + final String consumerId; + final String host; + final String clientId; + final String groupInstanceId; + final int numPartitions; + final List assignment; - public MemberAssignmentState(String group, String consumerId, String host, String clientId, String groupInstanceId, + MemberAssignmentState(String group, String consumerId, String host, String clientId, String groupInstanceId, int numPartitions, List assignment) { this.group = group; this.consumerId = consumerId; diff --git a/tools/src/main/java/org/apache/kafka/tools/consumer/group/PartitionAssignmentState.java b/tools/src/main/java/org/apache/kafka/tools/consumer/group/PartitionAssignmentState.java index 396032f0a0c..9e45f2b05e7 100644 --- a/tools/src/main/java/org/apache/kafka/tools/consumer/group/PartitionAssignmentState.java +++ b/tools/src/main/java/org/apache/kafka/tools/consumer/group/PartitionAssignmentState.java @@ -19,25 +19,23 @@ package org.apache.kafka.tools.consumer.group; import org.apache.kafka.common.Node; import java.util.Optional; -import java.util.OptionalInt; -import java.util.OptionalLong; class PartitionAssignmentState { - public final String group; - public final Optional coordinator; - public final Optional topic; - public final OptionalInt partition; - public final OptionalLong offset; - public final OptionalLong lag; - public final Optional consumerId; - public final Optional host; - public final Optional clientId; - public final OptionalLong logEndOffset; + final String group; + final Optional coordinator; + final Optional topic; + final Optional partition; + final Optional offset; + final Optional lag; + final Optional consumerId; + final Optional host; + final Optional clientId; + final Optional logEndOffset; - public PartitionAssignmentState(String group, Optional coordinator, Optional topic, - OptionalInt partition, OptionalLong offset, OptionalLong lag, + PartitionAssignmentState(String group, Optional coordinator, Optional topic, + Optional partition, Optional offset, Optional lag, Optional consumerId, Optional host, Optional clientId, - OptionalLong logEndOffset) { + Optional logEndOffset) { this.group = group; this.coordinator = coordinator; this.topic = topic; diff --git a/tools/src/test/java/org/apache/kafka/tools/ToolsTestUtils.java b/tools/src/test/java/org/apache/kafka/tools/ToolsTestUtils.java index 83fa31bf5e7..0bb2c9c9d1b 100644 --- a/tools/src/test/java/org/apache/kafka/tools/ToolsTestUtils.java +++ b/tools/src/test/java/org/apache/kafka/tools/ToolsTestUtils.java @@ -16,8 +16,8 @@ */ package org.apache.kafka.tools; -import kafka.utils.TestInfoUtils; import kafka.server.DynamicConfig; +import kafka.utils.TestInfoUtils; import kafka.utils.TestUtils; import org.apache.kafka.clients.admin.Admin; import org.apache.kafka.clients.admin.AlterConfigOp; @@ -210,6 +210,66 @@ public class ToolsTestUtils { return org.apache.kafka.test.TestUtils.tempFile(sb.toString()); } + /** + * Capture the console output during the execution of the provided function. + */ + public static String grabConsoleOutput(Runnable f) { + ByteArrayOutputStream buf = new ByteArrayOutputStream(); + PrintStream out = new PrintStream(buf); + PrintStream out0 = System.out; + + System.setOut(out); + try { + f.run(); + } finally { + System.setOut(out0); + } + out.flush(); + return buf.toString(); + } + + /** + * Capture the console error during the execution of the provided function. + */ + public static String grabConsoleError(Runnable f) { + ByteArrayOutputStream buf = new ByteArrayOutputStream(); + PrintStream err = new PrintStream(buf); + PrintStream err0 = System.err; + + System.setErr(err); + try { + f.run(); + } finally { + System.setErr(err0); + } + err.flush(); + return buf.toString(); + } + + /** + * Capture both the console output and console error during the execution of the provided function. + */ + public static Tuple2 grabConsoleOutputAndError(Runnable f) { + ByteArrayOutputStream outBuf = new ByteArrayOutputStream(); + ByteArrayOutputStream errBuf = new ByteArrayOutputStream(); + PrintStream out = new PrintStream(outBuf); + PrintStream err = new PrintStream(errBuf); + PrintStream out0 = System.out; + PrintStream err0 = System.err; + + System.setOut(out); + System.setErr(err); + try { + f.run(); + } finally { + System.setOut(out0); + System.setErr(err0); + } + out.flush(); + err.flush(); + return new Tuple2<>(outBuf.toString(), errBuf.toString()); + } + public static class MockExitProcedure implements Exit.Procedure { private boolean hasExited = false; private int statusCode; diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/AuthorizerIntegrationTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/AuthorizerIntegrationTest.java index 3094572b049..a0ad01bad95 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/AuthorizerIntegrationTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/AuthorizerIntegrationTest.java @@ -16,45 +16,30 @@ */ package org.apache.kafka.tools.consumer.group; -import kafka.admin.ConsumerGroupCommand; import kafka.api.AbstractAuthorizerIntegrationTest; import kafka.security.authorizer.AclEntry; import org.apache.kafka.common.acl.AccessControlEntry; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; -import scala.collection.immutable.Map$; +import scala.collection.JavaConverters; import java.util.Collections; -import java.util.Properties; import static org.apache.kafka.common.acl.AclOperation.DESCRIBE; import static org.apache.kafka.common.acl.AclPermissionType.ALLOW; import static org.apache.kafka.tools.ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME; -import static org.apache.kafka.tools.consumer.group.ConsumerGroupCommandTest.set; public class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { + @SuppressWarnings({"deprecation"}) @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_NAME) @ValueSource(strings = {"zk", "kraft"}) - public void testDescribeGroupCliWithGroupDescribe(String quorum) { - addAndVerifyAcls(set(Collections.singleton(new AccessControlEntry(ClientPrincipal().toString(), AclEntry.WildcardHost(), DESCRIBE, ALLOW))), groupResource()); + public void testDescribeGroupCliWithGroupDescribe(String quorum) throws Exception { + addAndVerifyAcls(JavaConverters.asScalaSet(Collections.singleton(new AccessControlEntry(ClientPrincipal().toString(), AclEntry.WildcardHost(), DESCRIBE, ALLOW))).toSet(), groupResource()); String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", group()}; - ConsumerGroupCommand.ConsumerGroupCommandOptions opts = new ConsumerGroupCommand.ConsumerGroupCommandOptions(cgcArgs); - ConsumerGroupCommand.ConsumerGroupService consumerGroupService = new ConsumerGroupCommand.ConsumerGroupService(opts, Map$.MODULE$.empty()); + ConsumerGroupCommandOptions opts = ConsumerGroupCommandOptions.fromArgs(cgcArgs); + ConsumerGroupCommand.ConsumerGroupService consumerGroupService = new ConsumerGroupCommand.ConsumerGroupService(opts, Collections.emptyMap()); consumerGroupService.describeGroups(); consumerGroupService.close(); } - - private void createTopicWithBrokerPrincipal(String topic) { - // Note the principal builder implementation maps all connections on the - // inter-broker listener to the broker principal. - createTopic( - topic, - 1, - 1, - new Properties(), - interBrokerListenerName(), - new Properties() - ); - } } diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommandTest.java index e3ee39f6b1d..e1daa37f197 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommandTest.java @@ -17,7 +17,6 @@ package org.apache.kafka.tools.consumer.group; import kafka.api.BaseConsumerTest; -import kafka.admin.ConsumerGroupCommand; import kafka.server.KafkaConfig; import kafka.utils.TestUtils; import org.apache.kafka.clients.admin.AdminClientConfig; @@ -41,7 +40,6 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; -import java.util.HashSet; import java.util.List; import java.util.Locale; import java.util.Map; @@ -130,10 +128,10 @@ public class ConsumerGroupCommandTest extends kafka.integration.KafkaServerTestH } ConsumerGroupCommand.ConsumerGroupService getConsumerGroupService(String[] args) { - ConsumerGroupCommand.ConsumerGroupCommandOptions opts = new ConsumerGroupCommand.ConsumerGroupCommandOptions(args); + ConsumerGroupCommandOptions opts = ConsumerGroupCommandOptions.fromArgs(args); ConsumerGroupCommand.ConsumerGroupService service = new ConsumerGroupCommand.ConsumerGroupService( opts, - asScala(Collections.singletonMap(AdminClientConfig.RETRIES_CONFIG, Integer.toString(Integer.MAX_VALUE))) + Collections.singletonMap(AdminClientConfig.RETRIES_CONFIG, Integer.toString(Integer.MAX_VALUE)) ); consumerGroupService.add(0, service); @@ -352,14 +350,4 @@ public class ConsumerGroupCommandTest extends kafka.integration.KafkaServerTestH static Seq seq(Collection seq) { return JavaConverters.asScalaIteratorConverter(seq.iterator()).asScala().toSeq(); } - - @SuppressWarnings("deprecation") - static scala.collection.Map asScala(Map jmap) { - return JavaConverters.mapAsScalaMap(jmap); - } - - @SuppressWarnings({"deprecation"}) - static scala.collection.immutable.Set set(final Collection set) { - return JavaConverters.asScalaSet(new HashSet<>(set)).toSet(); - } } diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupServiceTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupServiceTest.java index 8c75823648f..b7583c1c44f 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupServiceTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupServiceTest.java @@ -16,7 +16,6 @@ */ package org.apache.kafka.tools.consumer.group; -import kafka.admin.ConsumerGroupCommand; import org.apache.kafka.clients.admin.Admin; import org.apache.kafka.clients.admin.AdminClientTestUtils; import org.apache.kafka.clients.admin.ConsumerGroupDescription; @@ -39,15 +38,10 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicPartitionInfo; import org.apache.kafka.common.internals.KafkaFutureImpl; import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.tools.Tuple2; import org.junit.jupiter.api.Test; import org.mockito.ArgumentMatcher; import org.mockito.ArgumentMatchers; -import scala.Option; -import scala.Some; -import scala.Tuple2; -import scala.collection.JavaConverters; -import scala.collection.Seq; -import scala.collection.immutable.Map$; import java.util.ArrayList; import java.util.Arrays; @@ -86,7 +80,7 @@ public class ConsumerGroupServiceTest { private final Admin admin = mock(Admin.class); @Test - public void testAdminRequestsForDescribeOffsets() { + public void testAdminRequestsForDescribeOffsets() throws Exception { String[] args = new String[]{"--bootstrap-server", "localhost:9092", "--group", GROUP, "--describe", "--offsets"}; ConsumerGroupCommand.ConsumerGroupService groupService = consumerGroupService(args); @@ -97,10 +91,10 @@ public class ConsumerGroupServiceTest { when(admin.listOffsets(offsetsArgMatcher(), any())) .thenReturn(listOffsetsResult()); - Tuple2, Option>> statesAndAssignments = groupService.collectGroupOffsets(GROUP); - assertEquals(Some.apply("Stable"), statesAndAssignments._1); - assertTrue(statesAndAssignments._2.isDefined()); - assertEquals(TOPIC_PARTITIONS.size(), statesAndAssignments._2.get().size()); + Tuple2, Optional>> statesAndAssignments = groupService.collectGroupOffsets(GROUP); + assertEquals(Optional.of("Stable"), statesAndAssignments.v1); + assertTrue(statesAndAssignments.v2.isPresent()); + assertEquals(TOPIC_PARTITIONS.size(), statesAndAssignments.v2.get().size()); verify(admin, times(1)).describeConsumerGroups(ArgumentMatchers.eq(Collections.singletonList(GROUP)), any()); verify(admin, times(1)).listConsumerGroupOffsets(ArgumentMatchers.eq(listConsumerGroupOffsetsSpec()), any()); @@ -108,7 +102,7 @@ public class ConsumerGroupServiceTest { } @Test - public void testAdminRequestsForDescribeNegativeOffsets() { + public void testAdminRequestsForDescribeNegativeOffsets() throws Exception { String[] args = new String[]{"--bootstrap-server", "localhost:9092", "--group", GROUP, "--describe", "--offsets"}; ConsumerGroupCommand.ConsumerGroupService groupService = consumerGroupService(args); @@ -170,20 +164,15 @@ public class ConsumerGroupServiceTest { )).thenReturn(new ListOffsetsResult(endOffsets.entrySet().stream().filter(e -> unassignedTopicPartitions.contains(e.getKey())) .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)))); - Tuple2, Option>> statesAndAssignments = groupService.collectGroupOffsets(GROUP); - Option state = statesAndAssignments._1; - Option> assignments = statesAndAssignments._2; + Tuple2, Optional>> statesAndAssignments = groupService.collectGroupOffsets(GROUP); + Optional state = statesAndAssignments.v1; + Optional> assignments = statesAndAssignments.v2; - Map> returnedOffsets = new HashMap<>(); - assignments.foreach(results -> { - results.foreach(assignment -> { - returnedOffsets.put( - new TopicPartition(assignment.topic().get(), (Integer) assignment.partition().get()), - assignment.offset().isDefined() ? Optional.of((Long) assignment.offset().get()) : Optional.empty()); - return null; - }); - return null; - }); + Map> returnedOffsets = assignments.map(results -> + results.stream().collect(Collectors.toMap( + assignment -> new TopicPartition(assignment.topic.get(), assignment.partition.get()), + assignment -> assignment.offset)) + ).orElse(Collections.emptyMap()); Map> expectedOffsets = new HashMap<>(); @@ -194,7 +183,7 @@ public class ConsumerGroupServiceTest { expectedOffsets.put(testTopicPartition4, Optional.of(100L)); expectedOffsets.put(testTopicPartition5, Optional.empty()); - assertEquals(Some.apply("Stable"), state); + assertEquals(Optional.of("Stable"), state); assertEquals(expectedOffsets, returnedOffsets); verify(admin, times(1)).describeConsumerGroups(ArgumentMatchers.eq(Collections.singletonList(GROUP)), any()); @@ -220,9 +209,9 @@ public class ConsumerGroupServiceTest { when(admin.listOffsets(offsetsArgMatcher(), any())) .thenReturn(listOffsetsResult()); - scala.collection.Map> resetResult = groupService.resetOffsets(); - assertEquals(set(Collections.singletonList(GROUP)), resetResult.keySet()); - assertEquals(set(TOPIC_PARTITIONS), resetResult.get(GROUP).get().keys().toSet()); + Map> resetResult = groupService.resetOffsets(); + assertEquals(Collections.singleton(GROUP), resetResult.keySet()); + assertEquals(new HashSet<>(TOPIC_PARTITIONS), resetResult.get(GROUP).keySet()); verify(admin, times(1)).describeConsumerGroups(ArgumentMatchers.eq(Collections.singletonList(GROUP)), any()); verify(admin, times(1)).describeTopics(ArgumentMatchers.eq(topicsWithoutPartitionsSpecified), any()); @@ -230,9 +219,9 @@ public class ConsumerGroupServiceTest { } private ConsumerGroupCommand.ConsumerGroupService consumerGroupService(String[] args) { - return new ConsumerGroupCommand.ConsumerGroupService(new kafka.admin.ConsumerGroupCommand.ConsumerGroupCommandOptions(args), Map$.MODULE$.empty()) { + return new ConsumerGroupCommand.ConsumerGroupService(ConsumerGroupCommandOptions.fromArgs(args), Collections.emptyMap()) { @Override - public Admin createAdminClient(scala.collection.Map configOverrides) { + protected Admin createAdminClient(Map configOverrides) { return admin; } }; @@ -291,9 +280,4 @@ public class ConsumerGroupServiceTest { private Map listConsumerGroupOffsetsSpec() { return Collections.singletonMap(GROUP, new ListConsumerGroupOffsetsSpec()); } - - @SuppressWarnings({"deprecation"}) - private static scala.collection.immutable.Set set(final Collection set) { - return JavaConverters.asScalaSet(new HashSet<>(set)).toSet(); - } } diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteConsumerGroupsTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteConsumerGroupsTest.java index 0a1bcb2daef..95fe212c531 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteConsumerGroupsTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteConsumerGroupsTest.java @@ -17,17 +17,18 @@ package org.apache.kafka.tools.consumer.group; import joptsimple.OptionException; -import kafka.admin.ConsumerGroupCommand; import org.apache.kafka.clients.consumer.GroupProtocol; import org.apache.kafka.clients.consumer.RangeAssignor; import org.apache.kafka.common.errors.GroupIdNotFoundException; import org.apache.kafka.common.errors.GroupNotEmptyException; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.test.TestUtils; +import org.apache.kafka.tools.ToolsTestUtils; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; import java.util.Arrays; +import java.util.HashSet; import java.util.Map; import java.util.Objects; import java.util.Optional; @@ -60,10 +61,7 @@ public class DeleteConsumerGroupsTest extends ConsumerGroupCommandTest { String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--delete", "--group", missingGroup}; ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); - String output = kafka.utils.TestUtils.grabConsoleOutput(() -> { - service.deleteGroups(); - return null; - }); + String output = ToolsTestUtils.grabConsoleOutput(service::deleteGroups); assertTrue(output.contains("Group '" + missingGroup + "' could not be deleted due to:") && output.contains(Errors.GROUP_ID_NOT_FOUND.message()), "The expected error (" + Errors.GROUP_ID_NOT_FOUND + ") was not detected while deleting consumer group"); } @@ -78,8 +76,8 @@ public class DeleteConsumerGroupsTest extends ConsumerGroupCommandTest { String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--delete", "--group", missingGroup}; ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); - scala.collection.Map result = service.deleteGroups(); - assertTrue(result.size() == 1 && result.contains(missingGroup) && result.get(missingGroup).get().getCause() instanceof GroupIdNotFoundException, + Map result = service.deleteGroups(); + assertTrue(result.size() == 1 && result.containsKey(missingGroup) && result.get(missingGroup).getCause() instanceof GroupIdNotFoundException, "The expected error (" + Errors.GROUP_ID_NOT_FOUND + ") was not detected while deleting consumer group"); } @@ -94,14 +92,11 @@ public class DeleteConsumerGroupsTest extends ConsumerGroupCommandTest { ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); TestUtils.waitForCondition( - () -> service.collectGroupMembers(GROUP, false)._2.get().size() == 1, + () -> service.collectGroupMembers(GROUP, false).v2.get().size() == 1, "The group did not initialize as expected." ); - String output = kafka.utils.TestUtils.grabConsoleOutput(() -> { - service.deleteGroups(); - return null; - }); + String output = ToolsTestUtils.grabConsoleOutput(service::deleteGroups); assertTrue(output.contains("Group '" + GROUP + "' could not be deleted due to:") && output.contains(Errors.NON_EMPTY_GROUP.message()), "The expected error (" + Errors.NON_EMPTY_GROUP + ") was not detected while deleting consumer group. Output was: (" + output + ")"); } @@ -117,14 +112,14 @@ public class DeleteConsumerGroupsTest extends ConsumerGroupCommandTest { ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); TestUtils.waitForCondition( - () -> service.collectGroupMembers(GROUP, false)._2.get().size() == 1, + () -> service.collectGroupMembers(GROUP, false).v2.get().size() == 1, "The group did not initialize as expected." ); - scala.collection.Map result = service.deleteGroups(); - assertNotNull(result.get(GROUP).get(), + Map result = service.deleteGroups(); + assertNotNull(result.get(GROUP), "Group was deleted successfully, but it shouldn't have been. Result was:(" + result + ")"); - assertTrue(result.size() == 1 && result.contains(GROUP) && result.get(GROUP).get().getCause() instanceof GroupNotEmptyException, + assertTrue(result.size() == 1 && result.containsKey(GROUP) && result.get(GROUP).getCause() instanceof GroupNotEmptyException, "The expected error (" + Errors.NON_EMPTY_GROUP + ") was not detected while deleting consumer group. Result was:(" + result + ")"); } @@ -139,21 +134,18 @@ public class DeleteConsumerGroupsTest extends ConsumerGroupCommandTest { ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); TestUtils.waitForCondition( - () -> service.listConsumerGroups().contains(GROUP) && Objects.equals(service.collectGroupState(GROUP).state(), "Stable"), + () -> service.listConsumerGroups().contains(GROUP) && Objects.equals(service.collectGroupState(GROUP).state, "Stable"), "The group did not initialize as expected." ); executor.shutdown(); TestUtils.waitForCondition( - () -> Objects.equals(service.collectGroupState(GROUP).state(), "Empty"), + () -> Objects.equals(service.collectGroupState(GROUP).state, "Empty"), "The group did not become empty as expected." ); - String output = kafka.utils.TestUtils.grabConsoleOutput(() -> { - service.deleteGroups(); - return null; - }); + String output = ToolsTestUtils.grabConsoleOutput(service::deleteGroups); assertTrue(output.contains("Deletion of requested consumer groups ('" + GROUP + "') was successful."), "The consumer group could not be deleted as expected"); } @@ -173,10 +165,10 @@ public class DeleteConsumerGroupsTest extends ConsumerGroupCommandTest { ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); TestUtils.waitForCondition(() -> - Objects.equals(service.listConsumerGroups().toSet(), set(groups.keySet())) && + new HashSet<>(service.listConsumerGroups()).equals(groups.keySet()) && groups.keySet().stream().allMatch(groupId -> { try { - return Objects.equals(service.collectGroupState(groupId).state(), "Stable"); + return Objects.equals(service.collectGroupState(groupId).state, "Stable"); } catch (Exception e) { throw new RuntimeException(e); } @@ -189,17 +181,14 @@ public class DeleteConsumerGroupsTest extends ConsumerGroupCommandTest { TestUtils.waitForCondition(() -> groups.keySet().stream().allMatch(groupId -> { try { - return Objects.equals(service.collectGroupState(groupId).state(), "Empty"); + return Objects.equals(service.collectGroupState(groupId).state, "Empty"); } catch (Exception e) { throw new RuntimeException(e); } }), "The group did not become empty as expected."); - String output = kafka.utils.TestUtils.grabConsoleOutput(() -> { - service.deleteGroups(); - return null; - }).trim(); + String output = ToolsTestUtils.grabConsoleOutput(service::deleteGroups).trim(); Set expectedGroupsForDeletion = groups.keySet(); Set deletedGroupsGrepped = Arrays.stream(output.substring(output.indexOf('(') + 1, output.indexOf(')')).split(",")) .map(str -> str.replaceAll("'", "").trim()).collect(Collectors.toSet()); @@ -220,17 +209,17 @@ public class DeleteConsumerGroupsTest extends ConsumerGroupCommandTest { ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); TestUtils.waitForCondition( - () -> service.listConsumerGroups().contains(GROUP) && Objects.equals(service.collectGroupState(GROUP).state(), "Stable"), + () -> service.listConsumerGroups().contains(GROUP) && Objects.equals(service.collectGroupState(GROUP).state, "Stable"), "The group did not initialize as expected."); executor.shutdown(); TestUtils.waitForCondition( - () -> Objects.equals(service.collectGroupState(GROUP).state(), "Empty"), + () -> Objects.equals(service.collectGroupState(GROUP).state, "Empty"), "The group did not become empty as expected."); - scala.collection.Map result = service.deleteGroups(); - assertTrue(result.size() == 1 && result.contains(GROUP) && result.get(GROUP).get() == null, + Map result = service.deleteGroups(); + assertTrue(result.size() == 1 && result.containsKey(GROUP) && result.get(GROUP) == null, "The consumer group could not be deleted as expected"); } @@ -246,23 +235,20 @@ public class DeleteConsumerGroupsTest extends ConsumerGroupCommandTest { ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); TestUtils.waitForCondition( - () -> service.listConsumerGroups().contains(GROUP) && Objects.equals(service.collectGroupState(GROUP).state(), "Stable"), + () -> service.listConsumerGroups().contains(GROUP) && Objects.equals(service.collectGroupState(GROUP).state, "Stable"), "The group did not initialize as expected."); executor.shutdown(); TestUtils.waitForCondition( - () -> Objects.equals(service.collectGroupState(GROUP).state(), "Empty"), + () -> Objects.equals(service.collectGroupState(GROUP).state, "Empty"), "The group did not become empty as expected."); cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--delete", "--group", GROUP, "--group", missingGroup}; ConsumerGroupCommand.ConsumerGroupService service2 = getConsumerGroupService(cgcArgs); - String output = kafka.utils.TestUtils.grabConsoleOutput(() -> { - service2.deleteGroups(); - return null; - }); + String output = ToolsTestUtils.grabConsoleOutput(service2::deleteGroups); assertTrue(output.contains("Group '" + missingGroup + "' could not be deleted due to:") && output.contains(Errors.GROUP_ID_NOT_FOUND.message()) && output.contains("These consumer groups were deleted successfully: '" + GROUP + "'"), @@ -281,23 +267,23 @@ public class DeleteConsumerGroupsTest extends ConsumerGroupCommandTest { ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); TestUtils.waitForCondition( - () -> service.listConsumerGroups().contains(GROUP) && Objects.equals(service.collectGroupState(GROUP).state(), "Stable"), + () -> service.listConsumerGroups().contains(GROUP) && Objects.equals(service.collectGroupState(GROUP).state, "Stable"), "The group did not initialize as expected."); executor.shutdown(); TestUtils.waitForCondition( - () -> Objects.equals(service.collectGroupState(GROUP).state(), "Empty"), + () -> Objects.equals(service.collectGroupState(GROUP).state, "Empty"), "The group did not become empty as expected."); cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--delete", "--group", GROUP, "--group", missingGroup}; ConsumerGroupCommand.ConsumerGroupService service2 = getConsumerGroupService(cgcArgs); - scala.collection.Map result = service2.deleteGroups(); + Map result = service2.deleteGroups(); assertTrue(result.size() == 2 && - result.contains(GROUP) && result.get(GROUP).get() == null && - result.contains(missingGroup) && - result.get(missingGroup).get().getMessage().contains(Errors.GROUP_ID_NOT_FOUND.message()), + result.containsKey(GROUP) && result.get(GROUP) == null && + result.containsKey(missingGroup) && + result.get(missingGroup).getMessage().contains(Errors.GROUP_ID_NOT_FOUND.message()), "The consumer group deletion did not work as expected"); } diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteOffsetsConsumerGroupCommandIntegrationTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteOffsetsConsumerGroupCommandIntegrationTest.java index 2ac093923ce..05fd00071e1 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteOffsetsConsumerGroupCommandIntegrationTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteOffsetsConsumerGroupCommandIntegrationTest.java @@ -16,7 +16,6 @@ */ package org.apache.kafka.tools.consumer.group; -import kafka.admin.ConsumerGroupCommand; import kafka.utils.TestUtils; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerConfig; @@ -30,10 +29,12 @@ import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.serialization.ByteArraySerializer; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.server.config.Defaults; +import org.apache.kafka.tools.Tuple2; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; import java.util.Collections; +import java.util.Map; import java.util.Properties; import java.util.concurrent.ExecutionException; @@ -59,8 +60,8 @@ public class DeleteOffsetsConsumerGroupCommandIntegrationTest extends ConsumerGr String topic = "foo:1"; ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(getArgs(group, topic)); - scala.Tuple2> res = service.deleteOffsets(group, seq(Collections.singleton(topic)).toList()); - assertEquals(Errors.GROUP_ID_NOT_FOUND, res._1); + Tuple2> res = service.deleteOffsets(group, Collections.singletonList(topic)); + assertEquals(Errors.GROUP_ID_NOT_FOUND, res.v1); } @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_NAME) @@ -144,18 +145,18 @@ public class DeleteOffsetsConsumerGroupCommandIntegrationTest extends ConsumerGr withConsumerGroup.accept(() -> { String topic = inputPartition >= 0 ? inputTopic + ":" + inputPartition : inputTopic; ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(getArgs(GROUP, topic)); - scala.Tuple2> res = service.deleteOffsets(GROUP, seq(Collections.singletonList(topic)).toList()); - Errors topLevelError = res._1; - scala.collection.Map partitions = res._2; + Tuple2> res = service.deleteOffsets(GROUP, Collections.singletonList(topic)); + Errors topLevelError = res.v1; + Map partitions = res.v2; TopicPartition tp = new TopicPartition(inputTopic, expectedPartition); // Partition level error should propagate to top level, unless this is due to a missed partition attempt. if (inputPartition >= 0) { assertEquals(expectedError, topLevelError); } if (expectedError == Errors.NONE) - assertNull(partitions.get(tp).get()); + assertNull(partitions.get(tp)); else - assertEquals(expectedError.exception(), partitions.get(tp).get().getCause()); + assertEquals(expectedError.exception(), partitions.get(tp).getCause()); }); } diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/DescribeConsumerGroupTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/DescribeConsumerGroupTest.java index f0277d18cd6..85f9f5b836a 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/DescribeConsumerGroupTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/DescribeConsumerGroupTest.java @@ -16,7 +16,6 @@ */ package org.apache.kafka.tools.consumer.group; -import kafka.admin.ConsumerGroupCommand; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.RangeAssignor; import org.apache.kafka.clients.consumer.RoundRobinAssignor; @@ -24,17 +23,15 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.utils.Exit; import org.apache.kafka.test.TestUtils; +import org.apache.kafka.tools.ToolsTestUtils; +import org.apache.kafka.tools.Tuple2; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.MethodSource; import org.junit.jupiter.params.provider.ValueSource; -import scala.Function0; -import scala.Function1; -import scala.Option; -import scala.collection.Seq; -import scala.runtime.BoxedUnit; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.Objects; @@ -43,6 +40,8 @@ import java.util.Properties; import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Predicate; +import java.util.stream.Collectors; import static org.apache.kafka.test.TestUtils.RANDOM; import static org.apache.kafka.tools.ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES; @@ -81,7 +80,7 @@ public class DescribeConsumerGroupTest extends ConsumerGroupCommandTest { cgcArgs.addAll(describeType); ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs.toArray(new String[0])); - String output = kafka.utils.TestUtils.grabConsoleOutput(describeGroups(service)); + String output = ToolsTestUtils.grabConsoleOutput(describeGroups(service)); assertTrue(output.contains("Consumer group '" + missingGroup + "' does not exist."), "Expected error was not detected for describe option '" + String.join(" ", describeType) + "'"); } @@ -129,7 +128,7 @@ public class DescribeConsumerGroupTest extends ConsumerGroupCommandTest { @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES) @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"}) - public void testDescribeOffsetsOfNonExistingGroup(String quorum, String groupProtocol) { + public void testDescribeOffsetsOfNonExistingGroup(String quorum, String groupProtocol) throws Exception { String group = "missing.group"; createOffsetsTopic(listenerName(), new Properties()); @@ -139,14 +138,14 @@ public class DescribeConsumerGroupTest extends ConsumerGroupCommandTest { String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", group}; ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); - scala.Tuple2, Option>> res = service.collectGroupOffsets(group); - assertTrue(res._1.map(s -> s.contains("Dead")).getOrElse(() -> false) && res._2.map(Seq::isEmpty).getOrElse(() -> false), + Tuple2, Optional>> res = service.collectGroupOffsets(group); + assertTrue(res.v1.map(s -> s.contains("Dead")).orElse(false) && res.v2.map(Collection::isEmpty).orElse(false), "Expected the state to be 'Dead', with no members in the group '" + group + "'."); } @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES) @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"}) - public void testDescribeMembersOfNonExistingGroup(String quorum, String groupProtocol) { + public void testDescribeMembersOfNonExistingGroup(String quorum, String groupProtocol) throws Exception { String group = "missing.group"; createOffsetsTopic(listenerName(), new Properties()); @@ -156,18 +155,18 @@ public class DescribeConsumerGroupTest extends ConsumerGroupCommandTest { String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", group}; ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); - scala.Tuple2, Option>> res = service.collectGroupMembers(group, false); - assertTrue(res._1.map(s -> s.contains("Dead")).getOrElse(() -> false) && res._2.map(Seq::isEmpty).getOrElse(() -> false), + Tuple2, Optional>> res = service.collectGroupMembers(group, false); + assertTrue(res.v1.map(s -> s.contains("Dead")).orElse(false) && res.v2.map(Collection::isEmpty).orElse(false), "Expected the state to be 'Dead', with no members in the group '" + group + "'."); - scala.Tuple2, Option>> res2 = service.collectGroupMembers(group, true); - assertTrue(res2._1.map(s -> s.contains("Dead")).getOrElse(() -> false) && res2._2.map(Seq::isEmpty).getOrElse(() -> false), + Tuple2, Optional>> res2 = service.collectGroupMembers(group, true); + assertTrue(res2.v1.map(s -> s.contains("Dead")).orElse(false) && res2.v2.map(Collection::isEmpty).orElse(false), "Expected the state to be 'Dead', with no members in the group '" + group + "' (verbose option)."); } @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES) @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"}) - public void testDescribeStateOfNonExistingGroup(String quorum, String groupProtocol) { + public void testDescribeStateOfNonExistingGroup(String quorum, String groupProtocol) throws Exception { String group = "missing.group"; createOffsetsTopic(listenerName(), new Properties()); @@ -177,9 +176,9 @@ public class DescribeConsumerGroupTest extends ConsumerGroupCommandTest { String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", group}; ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); - ConsumerGroupCommand.GroupState state = service.collectGroupState(group); - assertTrue(Objects.equals(state.state(), "Dead") && state.numMembers() == 0 && - state.coordinator() != null && !brokers().filter(s -> s.config().brokerId() == state.coordinator().id()).isEmpty(), + GroupState state = service.collectGroupState(group); + assertTrue(Objects.equals(state.state, "Dead") && state.numMembers == 0 && + state.coordinator != null && !brokers().filter(s -> s.config().brokerId() == state.coordinator.id()).isEmpty(), "Expected the state to be 'Dead', with no members in the group '" + group + "'." ); } @@ -198,8 +197,8 @@ public class DescribeConsumerGroupTest extends ConsumerGroupCommandTest { ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs.toArray(new String[0])); TestUtils.waitForCondition(() -> { - scala.Tuple2 res = kafka.utils.TestUtils.grabConsoleOutputAndError(describeGroups(service)); - return res._1.trim().split("\n").length == 2 && res._2.isEmpty(); + Tuple2 res = ToolsTestUtils.grabConsoleOutputAndError(describeGroups(service)); + return res.v1.trim().split("\n").length == 2 && res.v2.isEmpty(); }, "Expected a data row and no error in describe results with describe type " + String.join(" ", describeType) + "."); } } @@ -227,9 +226,9 @@ public class DescribeConsumerGroupTest extends ConsumerGroupCommandTest { ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs.toArray(new String[0])); TestUtils.waitForCondition(() -> { - scala.Tuple2 res = kafka.utils.TestUtils.grabConsoleOutputAndError(describeGroups(service)); - long numLines = Arrays.stream(res._1.trim().split("\n")).filter(line -> !line.isEmpty()).count(); - return (numLines == expectedNumLines) && res._2.isEmpty(); + Tuple2 res = ToolsTestUtils.grabConsoleOutputAndError(describeGroups(service)); + long numLines = Arrays.stream(res.v1.trim().split("\n")).filter(line -> !line.isEmpty()).count(); + return (numLines == expectedNumLines) && res.v2.isEmpty(); }, "Expected a data row and no error in describe results with describe type " + String.join(" ", describeType) + "."); } } @@ -253,9 +252,9 @@ public class DescribeConsumerGroupTest extends ConsumerGroupCommandTest { ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs.toArray(new String[0])); TestUtils.waitForCondition(() -> { - scala.Tuple2 res = kafka.utils.TestUtils.grabConsoleOutputAndError(describeGroups(service)); - long numLines = Arrays.stream(res._1.trim().split("\n")).filter(s -> !s.isEmpty()).count(); - return (numLines == expectedNumLines) && res._2.isEmpty(); + Tuple2 res = ToolsTestUtils.grabConsoleOutputAndError(describeGroups(service)); + long numLines = Arrays.stream(res.v1.trim().split("\n")).filter(s -> !s.isEmpty()).count(); + return (numLines == expectedNumLines) && res.v2.isEmpty(); }, "Expected a data row and no error in describe results with describe type " + String.join(" ", describeType) + "."); } } @@ -272,26 +271,28 @@ public class DescribeConsumerGroupTest extends ConsumerGroupCommandTest { ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); TestUtils.waitForCondition(() -> { - scala.Tuple2, Option>> groupOffsets = service.collectGroupOffsets(GROUP); - Option state = groupOffsets._1; - Option> assignments = groupOffsets._2; + Tuple2, Optional>> groupOffsets = service.collectGroupOffsets(GROUP); + Optional state = groupOffsets.v1; + Optional> assignments = groupOffsets.v2; - Function1 isGrp = s -> Objects.equals(s.group(), GROUP); + Predicate isGrp = s -> Objects.equals(s.group, GROUP); - boolean res = state.map(s -> s.contains("Stable")).getOrElse(() -> false) && - assignments.isDefined() && - assignments.get().count(isGrp) == 1; + boolean res = state.map(s -> s.contains("Stable")).orElse(false) && + assignments.isPresent() && + assignments.get().stream().filter(isGrp).count() == 1; if (!res) return false; - @SuppressWarnings("cast") - ConsumerGroupCommand.PartitionAssignmentState partitionState = - (ConsumerGroupCommand.PartitionAssignmentState) assignments.get().filter(isGrp).head(); + Optional maybePartitionState = assignments.get().stream().filter(isGrp).findFirst(); + if (!maybePartitionState.isPresent()) + return false; - return !partitionState.consumerId().map(s0 -> s0.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE())).getOrElse(() -> false) && - !partitionState.clientId().map(s0 -> s0.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE())).getOrElse(() -> false) && - !partitionState.host().map(h -> h.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE())).getOrElse(() -> false); + PartitionAssignmentState partitionState = maybePartitionState.get(); + + return !partitionState.consumerId.map(s0 -> s0.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE)).orElse(false) && + !partitionState.clientId.map(s0 -> s0.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE)).orElse(false) && + !partitionState.host.map(h -> h.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE)).orElse(false); }, "Expected a 'Stable' group status, rows and valid values for consumer id / client id / host columns in describe results for group " + GROUP + "."); } @@ -306,32 +307,34 @@ public class DescribeConsumerGroupTest extends ConsumerGroupCommandTest { ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); TestUtils.waitForCondition(() -> { - scala.Tuple2, Option>> groupMembers = service.collectGroupMembers(GROUP, false); - Option state = groupMembers._1; - Option> assignments = groupMembers._2; + Tuple2, Optional>> groupMembers = service.collectGroupMembers(GROUP, false); + Optional state = groupMembers.v1; + Optional> assignments = groupMembers.v2; - Function1 isGrp = s -> Objects.equals(s.group(), GROUP); + Predicate isGrp = s -> Objects.equals(s.group, GROUP); - boolean res = state.map(s -> s.contains("Stable")).getOrElse(() -> false) && - assignments.isDefined() && - assignments.get().count(s -> Objects.equals(s.group(), GROUP)) == 1; + boolean res = state.map(s -> s.contains("Stable")).orElse(false) && + assignments.isPresent() && + assignments.get().stream().filter(s -> Objects.equals(s.group, GROUP)).count() == 1; if (!res) return false; - @SuppressWarnings("cast") - ConsumerGroupCommand.MemberAssignmentState assignmentState = - (ConsumerGroupCommand.MemberAssignmentState) assignments.get().filter(isGrp).head(); + Optional maybeAssignmentState = assignments.get().stream().filter(isGrp).findFirst(); + if (!maybeAssignmentState.isPresent()) + return false; - return !Objects.equals(assignmentState.consumerId(), ConsumerGroupCommand.MISSING_COLUMN_VALUE()) && - !Objects.equals(assignmentState.clientId(), ConsumerGroupCommand.MISSING_COLUMN_VALUE()) && - !Objects.equals(assignmentState.host(), ConsumerGroupCommand.MISSING_COLUMN_VALUE()); + MemberAssignmentState assignmentState = maybeAssignmentState.get(); + + return !Objects.equals(assignmentState.consumerId, ConsumerGroupCommand.MISSING_COLUMN_VALUE) && + !Objects.equals(assignmentState.clientId, ConsumerGroupCommand.MISSING_COLUMN_VALUE) && + !Objects.equals(assignmentState.host, ConsumerGroupCommand.MISSING_COLUMN_VALUE); }, "Expected a 'Stable' group status, rows and valid member information for group " + GROUP + "."); - scala.Tuple2, Option>> res = service.collectGroupMembers(GROUP, true); + Tuple2, Optional>> res = service.collectGroupMembers(GROUP, true); - if (res._2.isDefined()) { - assertTrue(res._2.get().size() == 1 && res._2.get().iterator().next().assignment().size() == 1, + if (res.v2.isPresent()) { + assertTrue(res.v2.get().size() == 1 && res.v2.get().iterator().next().assignment.size() == 1, "Expected a topic partition assigned to the single group member for group " + GROUP); } else { fail("Expected partition assignments for members of group " + GROUP); @@ -354,12 +357,12 @@ public class DescribeConsumerGroupTest extends ConsumerGroupCommandTest { ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); TestUtils.waitForCondition(() -> { - ConsumerGroupCommand.GroupState state = service.collectGroupState(GROUP); - return Objects.equals(state.state(), "Stable") && - state.numMembers() == 1 && - Objects.equals(state.assignmentStrategy(), "range") && - state.coordinator() != null && - brokers().count(s -> s.config().brokerId() == state.coordinator().id()) > 0; + GroupState state = service.collectGroupState(GROUP); + return Objects.equals(state.state, "Stable") && + state.numMembers == 1 && + Objects.equals(state.assignmentStrategy, "range") && + state.coordinator != null && + brokers().count(s -> s.config().brokerId() == state.coordinator.id()) > 0; }, "Expected a 'Stable' group status, with one member and round robin assignment strategy for group " + GROUP + "."); } @@ -381,12 +384,12 @@ public class DescribeConsumerGroupTest extends ConsumerGroupCommandTest { ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); TestUtils.waitForCondition(() -> { - ConsumerGroupCommand.GroupState state = service.collectGroupState(GROUP); - return Objects.equals(state.state(), "Stable") && - state.numMembers() == 1 && - Objects.equals(state.assignmentStrategy(), expectedName) && - state.coordinator() != null && - brokers().count(s -> s.config().brokerId() == state.coordinator().id()) > 0; + GroupState state = service.collectGroupState(GROUP); + return Objects.equals(state.state, "Stable") && + state.numMembers == 1 && + Objects.equals(state.assignmentStrategy, expectedName) && + state.coordinator != null && + brokers().count(s -> s.config().brokerId() == state.coordinator.id()) > 0; }, "Expected a 'Stable' group status, with one member and " + expectedName + " assignment strategy for group " + GROUP + "."); } @@ -404,14 +407,14 @@ public class DescribeConsumerGroupTest extends ConsumerGroupCommandTest { ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs.toArray(new String[0])); TestUtils.waitForCondition(() -> { - scala.Tuple2 res = kafka.utils.TestUtils.grabConsoleOutputAndError(describeGroups(service)); - return res._1.trim().split("\n").length == 2 && res._2.isEmpty(); + Tuple2 res = ToolsTestUtils.grabConsoleOutputAndError(describeGroups(service)); + return res.v1.trim().split("\n").length == 2 && res.v2.isEmpty(); }, "Expected describe group results with one data row for describe type '" + String.join(" ", describeType) + "'"); // stop the consumer so the group has no active member anymore executor.shutdown(); TestUtils.waitForCondition( - () -> kafka.utils.TestUtils.grabConsoleError(describeGroups(service)).contains("Consumer group '" + group + "' has no active members."), + () -> ToolsTestUtils.grabConsoleError(describeGroups(service)).contains("Consumer group '" + group + "' has no active members."), "Expected no active member in describe group results with describe type " + String.join(" ", describeType)); } } @@ -428,26 +431,25 @@ public class DescribeConsumerGroupTest extends ConsumerGroupCommandTest { ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); TestUtils.waitForCondition(() -> { - scala.Tuple2, Option>> res = service.collectGroupOffsets(GROUP); - return res._1.map(s -> s.contains("Stable")).getOrElse(() -> false) - && res._2.map(c -> c.exists(assignment -> Objects.equals(assignment.group(), GROUP) && assignment.offset().isDefined())).getOrElse(() -> false); + Tuple2, Optional>> res = service.collectGroupOffsets(GROUP); + return res.v1.map(s -> s.contains("Stable")).orElse(false) + && res.v2.map(c -> c.stream().anyMatch(assignment -> Objects.equals(assignment.group, GROUP) && assignment.offset.isPresent())).orElse(false); }, "Expected the group to initially become stable, and to find group in assignments after initial offset commit."); // stop the consumer so the group has no active member anymore executor.shutdown(); TestUtils.waitForCondition(() -> { - scala.Tuple2, Option>> offsets = service.collectGroupOffsets(GROUP); - Option state = offsets._1; - Option> assignments = offsets._2; - @SuppressWarnings("unchecked") - Seq testGroupAssignments = assignments.get().filter(a -> Objects.equals(a.group(), GROUP)).toSeq(); - ConsumerGroupCommand.PartitionAssignmentState assignment = testGroupAssignments.head(); - return state.map(s -> s.contains("Empty")).getOrElse(() -> false) && + Tuple2, Optional>> offsets = service.collectGroupOffsets(GROUP); + Optional state = offsets.v1; + Optional> assignments = offsets.v2; + List testGroupAssignments = assignments.get().stream().filter(a -> Objects.equals(a.group, GROUP)).collect(Collectors.toList()); + PartitionAssignmentState assignment = testGroupAssignments.get(0); + return state.map(s -> s.contains("Empty")).orElse(false) && testGroupAssignments.size() == 1 && - assignment.consumerId().map(c -> c.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE())).getOrElse(() -> false) && // the member should be gone - assignment.clientId().map(c -> c.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE())).getOrElse(() -> false) && - assignment.host().map(c -> c.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE())).getOrElse(() -> false); + assignment.consumerId.map(c -> c.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE)).orElse(false) && // the member should be gone + assignment.clientId.map(c -> c.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE)).orElse(false) && + assignment.host.map(c -> c.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE)).orElse(false); }, "failed to collect group offsets"); } @@ -463,17 +465,17 @@ public class DescribeConsumerGroupTest extends ConsumerGroupCommandTest { ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); TestUtils.waitForCondition(() -> { - scala.Tuple2, Option>> res = service.collectGroupMembers(GROUP, false); - return res._1.map(s -> s.contains("Stable")).getOrElse(() -> false) - && res._2.map(c -> c.exists(m -> Objects.equals(m.group(), GROUP))).getOrElse(() -> false); + Tuple2, Optional>> res = service.collectGroupMembers(GROUP, false); + return res.v1.map(s -> s.contains("Stable")).orElse(false) + && res.v2.map(c -> c.stream().anyMatch(m -> Objects.equals(m.group, GROUP))).orElse(false); }, "Expected the group to initially become stable, and to find group in assignments after initial offset commit."); // stop the consumer so the group has no active member anymore executor.shutdown(); TestUtils.waitForCondition(() -> { - scala.Tuple2, Option>> res = service.collectGroupMembers(GROUP, false); - return res._1.map(s -> s.contains("Empty")).getOrElse(() -> false) && res._2.isDefined() && res._2.get().isEmpty(); + Tuple2, Optional>> res = service.collectGroupMembers(GROUP, false); + return res.v1.map(s -> s.contains("Empty")).orElse(false) && res.v2.isPresent() && res.v2.get().isEmpty(); }, "Expected no member in describe group members results for group '" + GROUP + "'"); } @@ -489,19 +491,19 @@ public class DescribeConsumerGroupTest extends ConsumerGroupCommandTest { ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); TestUtils.waitForCondition(() -> { - ConsumerGroupCommand.GroupState state = service.collectGroupState(GROUP); - return Objects.equals(state.state(), "Stable") && - state.numMembers() == 1 && - state.coordinator() != null && - brokers().count(s -> s.config().brokerId() == state.coordinator().id()) > 0; + GroupState state = service.collectGroupState(GROUP); + return Objects.equals(state.state, "Stable") && + state.numMembers == 1 && + state.coordinator != null && + brokers().count(s -> s.config().brokerId() == state.coordinator.id()) > 0; }, "Expected the group '" + GROUP + "' to initially become stable, and have a single member."); // stop the consumer so the group has no active member anymore executor.shutdown(); TestUtils.waitForCondition(() -> { - ConsumerGroupCommand.GroupState state = service.collectGroupState(GROUP); - return Objects.equals(state.state(), "Empty") && state.numMembers() == 0; + GroupState state = service.collectGroupState(GROUP); + return Objects.equals(state.state, "Empty") && state.numMembers == 0; }, "Expected the group '" + GROUP + "' to become empty after the only member leaving."); } @@ -519,9 +521,9 @@ public class DescribeConsumerGroupTest extends ConsumerGroupCommandTest { ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs.toArray(new String[0])); TestUtils.waitForCondition(() -> { - scala.Tuple2 res = kafka.utils.TestUtils.grabConsoleOutputAndError(describeGroups(service)); + Tuple2 res = ToolsTestUtils.grabConsoleOutputAndError(describeGroups(service)); int expectedNumRows = DESCRIBE_TYPE_MEMBERS.contains(describeType) ? 3 : 2; - return res._2.isEmpty() && res._1.trim().split("\n").length == expectedNumRows; + return res.v2.isEmpty() && res.v1.trim().split("\n").length == expectedNumRows; }, "Expected a single data row in describe group result with describe type '" + String.join(" ", describeType) + "'"); } } @@ -538,11 +540,11 @@ public class DescribeConsumerGroupTest extends ConsumerGroupCommandTest { ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); TestUtils.waitForCondition(() -> { - scala.Tuple2, Option>> res = service.collectGroupOffsets(GROUP); - return res._1.map(s -> s.contains("Stable")).getOrElse(() -> false) && - res._2.isDefined() && - res._2.get().count(s -> Objects.equals(s.group(), GROUP)) == 1 && - res._2.get().count(x -> Objects.equals(x.group(), GROUP) && x.partition().isDefined()) == 1; + Tuple2, Optional>> res = service.collectGroupOffsets(GROUP); + return res.v1.map(s -> s.contains("Stable")).isPresent() && + res.v2.isPresent() && + res.v2.get().stream().filter(s -> Objects.equals(s.group, GROUP)).count() == 1 && + res.v2.get().stream().filter(x -> Objects.equals(x.group, GROUP) && x.partition.isPresent()).count() == 1; }, "Expected rows for consumers with no assigned partitions in describe group results"); } @@ -558,18 +560,18 @@ public class DescribeConsumerGroupTest extends ConsumerGroupCommandTest { ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); TestUtils.waitForCondition(() -> { - scala.Tuple2, Option>> res = service.collectGroupMembers(GROUP, false); - return res._1.map(s -> s.contains("Stable")).getOrElse(() -> false) && - res._2.isDefined() && - res._2.get().count(s -> Objects.equals(s.group(), GROUP)) == 2 && - res._2.get().count(x -> Objects.equals(x.group(), GROUP) && x.numPartitions() == 1) == 1 && - res._2.get().count(x -> Objects.equals(x.group(), GROUP) && x.numPartitions() == 0) == 1 && - res._2.get().forall(s -> s.assignment().isEmpty()); + Tuple2, Optional>> res = service.collectGroupMembers(GROUP, false); + return res.v1.map(s -> s.contains("Stable")).orElse(false) && + res.v2.isPresent() && + res.v2.get().stream().filter(s -> Objects.equals(s.group, GROUP)).count() == 2 && + res.v2.get().stream().filter(x -> Objects.equals(x.group, GROUP) && x.numPartitions == 1).count() == 1 && + res.v2.get().stream().filter(x -> Objects.equals(x.group, GROUP) && x.numPartitions == 0).count() == 1 && + res.v2.get().stream().allMatch(s -> s.assignment.isEmpty()); }, "Expected rows for consumers with no assigned partitions in describe group results"); - scala.Tuple2, Option>> res = service.collectGroupMembers(GROUP, true); - assertTrue(res._1.map(s -> s.contains("Stable")).getOrElse(() -> false) - && res._2.map(c -> c.exists(s -> !s.assignment().isEmpty())).getOrElse(() -> false), + Tuple2, Optional>> res = service.collectGroupMembers(GROUP, true); + assertTrue(res.v1.map(s -> s.contains("Stable")).orElse(false) + && res.v2.map(c -> c.stream().anyMatch(s -> !s.assignment.isEmpty())).orElse(false), "Expected additional columns in verbose version of describe members"); } @@ -585,8 +587,8 @@ public class DescribeConsumerGroupTest extends ConsumerGroupCommandTest { ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); TestUtils.waitForCondition(() -> { - ConsumerGroupCommand.GroupState state = service.collectGroupState(GROUP); - return Objects.equals(state.state(), "Stable") && state.numMembers() == 2; + GroupState state = service.collectGroupState(GROUP); + return Objects.equals(state.state, "Stable") && state.numMembers == 2; }, "Expected two consumers in describe group results"); } @@ -606,9 +608,9 @@ public class DescribeConsumerGroupTest extends ConsumerGroupCommandTest { ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs.toArray(new String[0])); TestUtils.waitForCondition(() -> { - scala.Tuple2 res = kafka.utils.TestUtils.grabConsoleOutputAndError(describeGroups(service)); + Tuple2 res = ToolsTestUtils.grabConsoleOutputAndError(describeGroups(service)); int expectedNumRows = DESCRIBE_TYPE_STATE.contains(describeType) ? 2 : 3; - return res._2.isEmpty() && res._1.trim().split("\n").length == expectedNumRows; + return res.v2.isEmpty() && res.v1.trim().split("\n").length == expectedNumRows; }, "Expected a single data row in describe group result with describe type '" + String.join(" ", describeType) + "'"); } } @@ -627,12 +629,12 @@ public class DescribeConsumerGroupTest extends ConsumerGroupCommandTest { ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); TestUtils.waitForCondition(() -> { - scala.Tuple2, Option>> res = service.collectGroupOffsets(GROUP); - return res._1.map(s -> s.contains("Stable")).getOrElse(() -> false) && - res._2.isDefined() && - res._2.get().count(s -> Objects.equals(s.group(), GROUP)) == 2 && - res._2.get().count(x -> Objects.equals(x.group(), GROUP) && x.partition().isDefined()) == 2 && - res._2.get().count(x -> Objects.equals(x.group(), GROUP) && !x.partition().isDefined()) == 0; + Tuple2, Optional>> res = service.collectGroupOffsets(GROUP); + return res.v1.map(s -> s.contains("Stable")).orElse(false) && + res.v2.isPresent() && + res.v2.get().stream().filter(s -> Objects.equals(s.group, GROUP)).count() == 2 && + res.v2.get().stream().filter(x -> Objects.equals(x.group, GROUP) && x.partition.isPresent()).count() == 2 && + res.v2.get().stream().noneMatch(x -> Objects.equals(x.group, GROUP) && !x.partition.isPresent()); }, "Expected two rows (one row per consumer) in describe group results."); } @@ -650,16 +652,16 @@ public class DescribeConsumerGroupTest extends ConsumerGroupCommandTest { ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); TestUtils.waitForCondition(() -> { - scala.Tuple2, Option>> res = service.collectGroupMembers(GROUP, false); - return res._1.map(s -> s.contains("Stable")).getOrElse(() -> false) && - res._2.isDefined() && - res._2.get().count(s -> Objects.equals(s.group(), GROUP)) == 2 && - res._2.get().count(x -> Objects.equals(x.group(), GROUP) && x.numPartitions() == 1) == 2 && - res._2.get().count(x -> Objects.equals(x.group(), GROUP) && x.numPartitions() == 0) == 0; + Tuple2, Optional>> res = service.collectGroupMembers(GROUP, false); + return res.v1.map(s -> s.contains("Stable")).orElse(false) && + res.v2.isPresent() && + res.v2.get().stream().filter(s -> Objects.equals(s.group, GROUP)).count() == 2 && + res.v2.get().stream().filter(x -> Objects.equals(x.group, GROUP) && x.numPartitions == 1).count() == 2 && + res.v2.get().stream().noneMatch(x -> Objects.equals(x.group, GROUP) && x.numPartitions == 0); }, "Expected two rows (one row per consumer) in describe group members results."); - scala.Tuple2, Option>> res = service.collectGroupMembers(GROUP, true); - assertTrue(res._1.map(s -> s.contains("Stable")).getOrElse(() -> false) && res._2.map(s -> s.count(x -> x.assignment().isEmpty())).getOrElse(() -> 0) == 0, + Tuple2, Optional>> res = service.collectGroupMembers(GROUP, true); + assertTrue(res.v1.map(s -> s.contains("Stable")).orElse(false) && res.v2.map(s -> s.stream().filter(x -> x.assignment.isEmpty()).count()).orElse(0L) == 0, "Expected additional columns in verbose version of describe members"); } @@ -677,8 +679,8 @@ public class DescribeConsumerGroupTest extends ConsumerGroupCommandTest { ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); TestUtils.waitForCondition(() -> { - ConsumerGroupCommand.GroupState state = service.collectGroupState(GROUP); - return Objects.equals(state.state(), "Stable") && Objects.equals(state.group(), GROUP) && state.numMembers() == 2; + GroupState state = service.collectGroupState(GROUP); + return Objects.equals(state.state, "Stable") && Objects.equals(state.group, GROUP) && state.numMembers == 2; }, "Expected a stable group with two members in describe group state result."); } @@ -696,9 +698,9 @@ public class DescribeConsumerGroupTest extends ConsumerGroupCommandTest { ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); TestUtils.waitForCondition(() -> { - scala.Tuple2, Option>> res = service.collectGroupOffsets(GROUP); - return res._1.map(s -> s.contains("Empty")).getOrElse(() -> false) - && res._2.isDefined() && res._2.get().count(s -> Objects.equals(s.group(), GROUP)) == 2; + Tuple2, Optional>> res = service.collectGroupOffsets(GROUP); + return res.v1.map(s -> s.contains("Empty")).orElse(false) + && res.v2.isPresent() && res.v2.get().stream().filter(s -> Objects.equals(s.group, GROUP)).count() == 2; }, "Expected a stable group with two members in describe group state result."); } @@ -796,32 +798,33 @@ public class DescribeConsumerGroupTest extends ConsumerGroupCommandTest { ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); TestUtils.waitForCondition(() -> { - scala.Tuple2, Option>> groupOffsets = service.collectGroupOffsets(GROUP); + Tuple2, Optional>> groupOffsets = service.collectGroupOffsets(GROUP); - Function1 isGrp = s -> Objects.equals(s.group(), GROUP); + Predicate isGrp = s -> Objects.equals(s.group, GROUP); - boolean res = groupOffsets._1.map(s -> s.contains("Stable")).getOrElse(() -> false) && - groupOffsets._2.isDefined() && - groupOffsets._2.get().count(isGrp) == 1; + boolean res = groupOffsets.v1.map(s -> s.contains("Stable")).orElse(false) && + groupOffsets.v2.isPresent() && + groupOffsets.v2.get().stream().filter(isGrp).count() == 1; if (!res) return false; - @SuppressWarnings("cast") - ConsumerGroupCommand.PartitionAssignmentState assignmentState = - (ConsumerGroupCommand.PartitionAssignmentState) groupOffsets._2.get().filter(isGrp).head(); + Optional maybeAssignmentState = groupOffsets.v2.get().stream().filter(isGrp).findFirst(); + if (!maybeAssignmentState.isPresent()) + return false; - return assignmentState.consumerId().map(c -> !c.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE())).getOrElse(() -> false) && - assignmentState.clientId().map(c -> !c.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE())).getOrElse(() -> false) && - assignmentState.host().map(h -> !h.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE())).getOrElse(() -> false); + PartitionAssignmentState assignmentState = maybeAssignmentState.get(); + + return assignmentState.consumerId.map(c -> !c.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE)).orElse(false) && + assignmentState.clientId.map(c -> !c.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE)).orElse(false) && + assignmentState.host.map(h -> !h.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE)).orElse(false); }, "Expected a 'Stable' group status, rows and valid values for consumer id / client id / host columns in describe results for non-offset-committing group " + GROUP + "."); } - private Function0 describeGroups(ConsumerGroupCommand.ConsumerGroupService service) { + private Runnable describeGroups(ConsumerGroupCommand.ConsumerGroupService service) { return () -> { try { service.describeGroups(); - return null; } catch (Exception e) { throw new RuntimeException(e); } diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ListConsumerGroupTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ListConsumerGroupTest.java index ba5ebd254fc..6fd2928da9f 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ListConsumerGroupTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ListConsumerGroupTest.java @@ -17,16 +17,17 @@ package org.apache.kafka.tools.consumer.group; import joptsimple.OptionException; -import kafka.admin.ConsumerGroupCommand; import org.apache.kafka.clients.admin.ConsumerGroupListing; import org.apache.kafka.common.ConsumerGroupState; import org.apache.kafka.common.GroupType; import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.Test; +import org.apache.kafka.tools.ToolsTestUtils; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.MethodSource; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.EnumSet; import java.util.HashSet; @@ -58,11 +59,11 @@ public class ListConsumerGroupTest extends ConsumerGroupCommandTest { String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--list"}; ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); - scala.collection.Set expectedGroups = set(Arrays.asList(GROUP, simpleGroup, PROTOCOL_GROUP)); - final AtomicReference foundGroups = new AtomicReference<>(); + Set expectedGroups = set(Arrays.asList(GROUP, simpleGroup, PROTOCOL_GROUP)); + final AtomicReference foundGroups = new AtomicReference<>(); TestUtils.waitForCondition(() -> { - foundGroups.set(service.listConsumerGroups().toSet()); + foundGroups.set(set(service.listConsumerGroups())); return Objects.equals(expectedGroups, foundGroups.get()); }, "Expected --list to show groups " + expectedGroups + ", but found " + foundGroups.get() + "."); } @@ -272,7 +273,7 @@ public class ListConsumerGroupTest extends ConsumerGroupCommandTest { @Test public void testConsumerGroupStatesFromString() { - scala.collection.Set result = ConsumerGroupCommand.consumerGroupStatesFromString("Stable"); + Set result = ConsumerGroupCommand.consumerGroupStatesFromString("Stable"); assertEquals(set(Collections.singleton(ConsumerGroupState.STABLE)), result); result = ConsumerGroupCommand.consumerGroupStatesFromString("Stable, PreparingRebalance"); @@ -299,7 +300,7 @@ public class ListConsumerGroupTest extends ConsumerGroupCommandTest { @Test public void testConsumerGroupTypesFromString() { - scala.collection.Set result = ConsumerGroupCommand.consumerGroupTypesFromString("consumer"); + Set result = ConsumerGroupCommand.consumerGroupTypesFromString("consumer"); assertEquals(set(Collections.singleton(GroupType.CONSUMER)), result); result = ConsumerGroupCommand.consumerGroupTypesFromString("consumer, classic"); @@ -475,9 +476,9 @@ public class ListConsumerGroupTest extends ConsumerGroupCommandTest { Set stateFilterSet, Set expectedListing ) throws Exception { - final AtomicReference foundListing = new AtomicReference<>(); + final AtomicReference> foundListing = new AtomicReference<>(); TestUtils.waitForCondition(() -> { - foundListing.set(service.listConsumerGroupsWithFilters(set(typeFilterSet), set(stateFilterSet)).toSet()); + foundListing.set(set(service.listConsumerGroupsWithFilters(set(typeFilterSet), set(stateFilterSet)))); return Objects.equals(set(expectedListing), foundListing.get()); }, () -> "Expected to show groups " + expectedListing + ", but found " + foundListing.get() + "."); } @@ -498,7 +499,7 @@ public class ListConsumerGroupTest extends ConsumerGroupCommandTest { ) throws InterruptedException { final AtomicReference out = new AtomicReference<>(""); TestUtils.waitForCondition(() -> { - String output = runAndGrabConsoleOutput(args); + String output = ToolsTestUtils.grabConsoleOutput(() -> ConsumerGroupCommand.main(args.toArray(new String[0]))); out.set(output); int index = 0; @@ -522,12 +523,7 @@ public class ListConsumerGroupTest extends ConsumerGroupCommandTest { }, () -> String.format("Expected header=%s and groups=%s, but found:%n%s", expectedHeader, expectedRows, out.get())); } - private static String runAndGrabConsoleOutput( - List args - ) { - return kafka.utils.TestUtils.grabConsoleOutput(() -> { - ConsumerGroupCommand.main(args.toArray(new String[0])); - return null; - }); + public static Set set(Collection set) { + return new HashSet<>(set); } } diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ResetConsumerGroupOffsetTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ResetConsumerGroupOffsetTest.java index dca26345223..4d805ce68f8 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ResetConsumerGroupOffsetTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ResetConsumerGroupOffsetTest.java @@ -17,7 +17,6 @@ package org.apache.kafka.tools.consumer.group; import joptsimple.OptionException; -import kafka.admin.ConsumerGroupCommand; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.GroupProtocol; import org.apache.kafka.clients.consumer.OffsetAndMetadata; @@ -25,7 +24,6 @@ import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.Test; -import scala.Option; import java.io.BufferedWriter; import java.io.File; @@ -98,10 +96,10 @@ public class ResetConsumerGroupOffsetTest extends ConsumerGroupCommandTest { ConsumerGroupCommand.ConsumerGroupService consumerGroupCommand = getConsumerGroupService(args); // Make sure we got a coordinator TestUtils.waitForCondition( - () -> Objects.equals(consumerGroupCommand.collectGroupState(group).coordinator().host(), "localhost"), + () -> Objects.equals(consumerGroupCommand.collectGroupState(group).coordinator.host(), "localhost"), "Can't find a coordinator"); - Option> resetOffsets = consumerGroupCommand.resetOffsets().get(group); - assertTrue(resetOffsets.isDefined() && resetOffsets.get().isEmpty()); + Map resetOffsets = consumerGroupCommand.resetOffsets().get(group); + assertTrue(resetOffsets.isEmpty()); assertTrue(committedOffsets(TOPIC, group).isEmpty()); } @@ -211,7 +209,7 @@ public class ResetConsumerGroupOffsetTest extends ConsumerGroupCommandTest { } @Test - public void testResetOffsetsByDurationFallbackToLatestWhenNoRecords() throws Exception { + public void testResetOffsetsByDurationFallbackToLatestWhenNoRecords() { String topic = "foo2"; String[] args = buildArgsForGroup(GROUP, "--topic", topic, "--by-duration", "PT1M", "--execute"); createTopic(topic, 1, 1, new Properties(), listenerName(), new Properties()); @@ -326,7 +324,7 @@ public class ResetConsumerGroupOffsetTest extends ConsumerGroupCommandTest { TopicPartition tp1 = new TopicPartition(topic1, 0); TopicPartition tp2 = new TopicPartition(topic2, 0); - Map allResetOffsets = toOffsetMap(consumerGroupCommand.resetOffsets().get(GROUP)); + Map allResetOffsets = toOffsetMap(resetOffsets(consumerGroupCommand).get(GROUP)); Map expMap = new HashMap<>(); expMap.put(tp1, 0L); expMap.put(tp2, 0L); @@ -357,7 +355,7 @@ public class ResetConsumerGroupOffsetTest extends ConsumerGroupCommandTest { TopicPartition tp1 = new TopicPartition(topic1, 1); TopicPartition tp2 = new TopicPartition(topic2, 1); - Map allResetOffsets = toOffsetMap(consumerGroupCommand.resetOffsets().get(GROUP)); + Map allResetOffsets = toOffsetMap(resetOffsets(consumerGroupCommand).get(GROUP)); Map expMap = new HashMap<>(); expMap.put(tp1, 0L); expMap.put(tp2, 0L); @@ -386,7 +384,7 @@ public class ResetConsumerGroupOffsetTest extends ConsumerGroupCommandTest { File file = TestUtils.tempFile("reset", ".csv"); - scala.collection.Map> exportedOffsets = consumerGroupCommand.resetOffsets(); + Map> exportedOffsets = consumerGroupCommand.resetOffsets(); BufferedWriter bw = new BufferedWriter(new FileWriter(file)); bw.write(consumerGroupCommand.exportOffsetsToCsv(exportedOffsets)); bw.close(); @@ -398,7 +396,7 @@ public class ResetConsumerGroupOffsetTest extends ConsumerGroupCommandTest { String[] cgcArgsExec = buildArgsForGroup(GROUP, "--all-topics", "--from-file", file.getCanonicalPath(), "--dry-run"); ConsumerGroupCommand.ConsumerGroupService consumerGroupCommandExec = getConsumerGroupService(cgcArgsExec); - scala.collection.Map> importedOffsets = consumerGroupCommandExec.resetOffsets(); + Map> importedOffsets = consumerGroupCommandExec.resetOffsets(); assertEquals(exp1, toOffsetMap(importedOffsets.get(GROUP))); adminZkClient().deleteTopic(topic); @@ -430,7 +428,7 @@ public class ResetConsumerGroupOffsetTest extends ConsumerGroupCommandTest { File file = TestUtils.tempFile("reset", ".csv"); - scala.collection.Map> exportedOffsets = consumerGroupCommand.resetOffsets(); + Map> exportedOffsets = consumerGroupCommand.resetOffsets(); BufferedWriter bw = new BufferedWriter(new FileWriter(file)); bw.write(consumerGroupCommand.exportOffsetsToCsv(exportedOffsets)); bw.close(); @@ -447,14 +445,14 @@ public class ResetConsumerGroupOffsetTest extends ConsumerGroupCommandTest { // Multiple --group's offset import String[] cgcArgsExec = buildArgsForGroups(Arrays.asList(group1, group2), "--all-topics", "--from-file", file.getCanonicalPath(), "--dry-run"); ConsumerGroupCommand.ConsumerGroupService consumerGroupCommandExec = getConsumerGroupService(cgcArgsExec); - scala.collection.Map> importedOffsets = consumerGroupCommandExec.resetOffsets(); + Map> importedOffsets = consumerGroupCommandExec.resetOffsets(); assertEquals(exp1, toOffsetMap(importedOffsets.get(group1))); assertEquals(exp2, toOffsetMap(importedOffsets.get(group2))); // Single --group offset import using "group,topic,partition,offset" csv format String[] cgcArgsExec2 = buildArgsForGroup(group1, "--all-topics", "--from-file", file.getCanonicalPath(), "--dry-run"); ConsumerGroupCommand.ConsumerGroupService consumerGroupCommandExec2 = getConsumerGroupService(cgcArgsExec2); - scala.collection.Map> importedOffsets2 = consumerGroupCommandExec2.resetOffsets(); + Map> importedOffsets2 = consumerGroupCommandExec2.resetOffsets(); assertEquals(exp1, toOffsetMap(importedOffsets2.get(group1))); adminZkClient().deleteTopic(TOPIC); @@ -503,9 +501,9 @@ public class ResetConsumerGroupOffsetTest extends ConsumerGroupCommandTest { private void awaitConsumerGroupInactive(ConsumerGroupCommand.ConsumerGroupService consumerGroupService, String group) throws Exception { TestUtils.waitForCondition(() -> { - String state = consumerGroupService.collectGroupState(group).state(); + String state = consumerGroupService.collectGroupState(group).state; return Objects.equals(state, "Empty") || Objects.equals(state, "Dead"); - }, "Expected that consumer group is inactive. Actual state: " + consumerGroupService.collectGroupState(group).state()); + }, "Expected that consumer group is inactive. Actual state: " + consumerGroupService.collectGroupState(group).state); } private void resetAndAssertOffsets(String[] args, @@ -521,25 +519,17 @@ public class ResetConsumerGroupOffsetTest extends ConsumerGroupCommandTest { Map> expectedOffsets = topics.stream().collect(Collectors.toMap( Function.identity(), topic -> Collections.singletonMap(new TopicPartition(topic, 0), expectedOffset))); - scala.collection.Map> resetOffsetsResultByGroup = consumerGroupCommand.resetOffsets(); + Map> resetOffsetsResultByGroup = resetOffsets(consumerGroupCommand); try { for (final String topic : topics) { - resetOffsetsResultByGroup.foreach(entry -> { - String group = entry._1; - scala.collection.Map partitionInfo = entry._2; + resetOffsetsResultByGroup.forEach((group, partitionInfo) -> { Map priorOffsets = committedOffsets(topic, group); - Map offsets = new HashMap<>(); - partitionInfo.foreach(partitionInfoEntry -> { - TopicPartition tp = partitionInfoEntry._1; - OffsetAndMetadata offsetAndMetadata = partitionInfoEntry._2; - if (Objects.equals(tp.topic(), topic)) - offsets.put(tp, offsetAndMetadata.offset()); - return null; - }); - assertEquals(expectedOffsets.get(topic), offsets); + assertEquals(expectedOffsets.get(topic), + partitionInfo.entrySet().stream() + .filter(entry -> Objects.equals(entry.getKey().topic(), topic)) + .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().offset()))); assertEquals(dryRun ? priorOffsets : expectedOffsets.get(topic), committedOffsets(topic, group)); - return null; }); } } finally { @@ -550,35 +540,22 @@ public class ResetConsumerGroupOffsetTest extends ConsumerGroupCommandTest { private void resetAndAssertOffsetsCommitted(ConsumerGroupCommand.ConsumerGroupService consumerGroupService, Map expectedOffsets, String topic) { - scala.collection.Map> allResetOffsets = consumerGroupService.resetOffsets(); + Map> allResetOffsets = resetOffsets(consumerGroupService); - allResetOffsets.foreach(entry -> { - String group = entry._1; - scala.collection.Map offsetsInfo = entry._2; - offsetsInfo.foreach(offsetInfoEntry -> { - TopicPartition tp = offsetInfoEntry._1; - OffsetAndMetadata offsetMetadata = offsetInfoEntry._2; + allResetOffsets.forEach((group, offsetsInfo) -> { + offsetsInfo.forEach((tp, offsetMetadata) -> { assertEquals(offsetMetadata.offset(), expectedOffsets.get(tp)); assertEquals(expectedOffsets, committedOffsets(topic, group)); - return null; }); - return null; }); } - Map toOffsetMap(Option> map) { - assertTrue(map.isDefined()); - Map res = new HashMap<>(); - map.foreach(m -> { - m.foreach(entry -> { - TopicPartition tp = entry._1; - OffsetAndMetadata offsetAndMetadata = entry._2; - res.put(tp, offsetAndMetadata.offset()); - return null; - }); - return null; - }); - return res; + private Map> resetOffsets(ConsumerGroupCommand.ConsumerGroupService consumerGroupService) { + return consumerGroupService.resetOffsets(); + } + + Map toOffsetMap(Map map) { + return map.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().offset())); } private String[] addTo(String[] args, String...extra) { diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/SaslClientsWithInvalidCredentialsTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/SaslClientsWithInvalidCredentialsTest.java index 06b727a59a8..1ce1f909dd5 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/SaslClientsWithInvalidCredentialsTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/SaslClientsWithInvalidCredentialsTest.java @@ -16,7 +16,6 @@ */ package org.apache.kafka.tools.consumer.group; -import kafka.admin.ConsumerGroupCommand; import kafka.api.AbstractSaslTest; import kafka.api.Both$; import kafka.utils.JaasTestUtils; @@ -36,7 +35,6 @@ import scala.Option; import scala.Some$; import scala.collection.JavaConverters; import scala.collection.Seq; -import scala.collection.immutable.Map$; import java.io.File; import java.io.IOException; @@ -164,8 +162,8 @@ public class SaslClientsWithInvalidCredentialsTest extends AbstractSaslTest { "--describe", "--group", "test.group", "--command-config", propsFile.getAbsolutePath()}; - ConsumerGroupCommand.ConsumerGroupCommandOptions opts = new ConsumerGroupCommand.ConsumerGroupCommandOptions(cgcArgs); - return new ConsumerGroupCommand.ConsumerGroupService(opts, Map$.MODULE$.empty()); + ConsumerGroupCommandOptions opts = ConsumerGroupCommandOptions.fromArgs(cgcArgs); + return new ConsumerGroupCommand.ConsumerGroupService(opts, Collections.emptyMap()); } private void verifyAuthenticationException(Executable action) {