Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
17 changes: 17 additions & 0 deletions bin/kafka-metadata-quorum.sh
Original file line number Diff line number Diff line change
@@ -0,0 +1,17 @@
#!/bin/bash
# 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.

exec $(dirname $0)/kafka-run-class.sh kafka.admin.MetadataQuorumCommand "$@"
17 changes: 17 additions & 0 deletions bin/windows/kafka-metatada-quorum.bat
Original file line number Diff line number Diff line change
@@ -0,0 +1,17 @@
@echo off
rem Licensed to the Apache Software Foundation (ASF) under one or more
rem contributor license agreements. See the NOTICE file distributed with
rem this work for additional information regarding copyright ownership.
rem The ASF licenses this file to You under the Apache License, Version 2.0
rem (the "License"); you may not use this file except in compliance with
rem the License. You may obtain a copy of the License at
rem
rem http://www.apache.org/licenses/LICENSE-2.0
rem
rem Unless required by applicable law or agreed to in writing, software
rem distributed under the License is distributed on an "AS IS" BASIS,
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.MetadataQuorumCommand %*
1 change: 1 addition & 0 deletions build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -1705,6 +1705,7 @@ project(':tools') {

dependencies {
implementation project(':clients')
implementation project(':server-common')
implementation project(':log4j-appender')
implementation libs.argparse4j
implementation libs.jacksonDatabind
Expand Down
1 change: 1 addition & 0 deletions checkstyle/import-control.xml
Original file line number Diff line number Diff line change
Expand Up @@ -366,6 +366,7 @@

<subpackage name="tools">
<allow pkg="org.apache.kafka.common"/>
<allow pkg="org.apache.kafka.server.util" />
<allow pkg="org.apache.kafka.clients.admin" />
<allow pkg="org.apache.kafka.clients.producer" />
<allow pkg="org.apache.kafka.clients.consumer" />
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -4359,6 +4359,8 @@ private QuorumInfo.ReplicaState translateReplicaState(DescribeQuorumResponseData
private QuorumInfo createQuorumResult(final DescribeQuorumResponseData.PartitionData partition) {
return new QuorumInfo(
partition.leaderId(),
partition.leaderEpoch(),
partition.highWatermark(),
partition.currentVoters().stream().map(v -> translateReplicaState(v)).collect(Collectors.toList()),
partition.observers().stream().map(o -> translateReplicaState(o)).collect(Collectors.toList()));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,11 +25,15 @@
*/
public class QuorumInfo {
private final Integer leaderId;
private final Integer leaderEpoch;
private final Long highWatermark;
private final List<ReplicaState> voters;
private final List<ReplicaState> observers;

QuorumInfo(Integer leaderId, List<ReplicaState> voters, List<ReplicaState> observers) {
QuorumInfo(Integer leaderId, Integer leaderEpoch, Long highWatermark, List<ReplicaState> voters, List<ReplicaState> observers) {
this.leaderId = leaderId;
this.leaderEpoch = leaderEpoch;
this.highWatermark = highWatermark;
this.voters = voters;
this.observers = observers;
}
Expand All @@ -38,6 +42,14 @@ public Integer leaderId() {
return leaderId;
}

public Integer leaderEpoch() {
return leaderEpoch;
}

public Long highWatermark() {
return highWatermark;
}

public List<ReplicaState> voters() {
return voters;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -608,7 +608,7 @@ private static ApiVersionsResponse prepareApiVersionsResponseForDescribeFeatures
}

private static QuorumInfo defaultQuorumInfo(Boolean emptyOptionals) {
return new QuorumInfo(1,
return new QuorumInfo(1, 1, 1L,
singletonList(new QuorumInfo.ReplicaState(1, 100,
emptyOptionals ? OptionalLong.empty() : OptionalLong.of(1000),
emptyOptionals ? OptionalLong.empty() : OptionalLong.of(1000))),
Expand Down
172 changes: 172 additions & 0 deletions core/src/main/scala/kafka/admin/MetadataQuorumCommand.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,172 @@
/**
* 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 kafka.tools.TerseFailure
import kafka.utils.Exit
import net.sourceforge.argparse4j.ArgumentParsers
import net.sourceforge.argparse4j.impl.Arguments.{fileType, storeTrue}
import net.sourceforge.argparse4j.inf.Subparsers
import org.apache.kafka.clients._
import org.apache.kafka.clients.admin.{Admin, QuorumInfo}
import org.apache.kafka.common.utils.Utils
import org.apache.kafka.server.util.ToolsUtils.prettyPrintTable

import java.io.File
import java.util.Properties
import scala.jdk.CollectionConverters._

/**
* A tool for describing quorum status
*/
object MetadataQuorumCommand {

def main(args: Array[String]): Unit = {
val res = mainNoExit(args)
Exit.exit(res)
}

def mainNoExit(args: Array[String]): Int = {
val parser = ArgumentParsers
.newArgumentParser("kafka-metadata-quorum")
.defaultHelp(true)
.description("This tool describes kraft metadata quorum status.")
parser
.addArgument("--bootstrap-server")
.help("A comma-separated list of host:port pairs to use for establishing the connection to the Kafka cluster.")
.required(true)

parser
.addArgument("--command-config")
.`type`(fileType())
.help("Property file containing configs to be passed to Admin Client.")
val subparsers = parser.addSubparsers().dest("command")
addDescribeParser(subparsers)

var admin: Admin = null
try {
val namespace = parser.parseArgsOrFail(args)
val command = namespace.getString("command")

val commandConfig = namespace.get[File]("command_config")
val props = if (commandConfig != null) {
if (!commandConfig.exists()) {
throw new TerseFailure(s"Properties file ${commandConfig.getPath} does not exists!")
}
Utils.loadProps(commandConfig.getPath)
} else {
new Properties()
}
props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, namespace.getString("bootstrap_server"))
admin = Admin.create(props)

if (command == "describe") {
if (namespace.getBoolean("status") && namespace.getBoolean("replication")) {
throw new TerseFailure(s"Only one of --status or --replication should be specified with describe sub-command")
} else if (namespace.getBoolean("replication")) {
handleDescribeReplication(admin)
} else if (namespace.getBoolean("status")) {
handleDescribeStatus(admin)
} else {
throw new TerseFailure(s"One of --status or --replication must be specified with describe sub-command")
}
} else {
throw new IllegalStateException(s"Unknown command: $command, only 'describe' is supported")
}
0
} catch {
case e: TerseFailure =>
Console.err.println(e.getMessage)
1
} finally {
if (admin != null) {
admin.close()
}
}
}

def addDescribeParser(subparsers: Subparsers): Unit = {
val describeParser = subparsers
.addParser("describe")
.help("Describe the metadata quorum info")

val statusArgs = describeParser.addArgumentGroup("Status")
statusArgs
.addArgument("--status")
.help(
"A short summary of the quorum status and the other provides detailed information about the status of replication.")
.action(storeTrue())
val replicationArgs = describeParser.addArgumentGroup("Replication")
replicationArgs
.addArgument("--replication")
.help("Detailed information about the status of replication")
.action(storeTrue())
}

private def handleDescribeReplication(admin: Admin): Unit = {
val quorumInfo = admin.describeMetadataQuorum.quorumInfo.get
val leaderId = quorumInfo.leaderId
val leader = quorumInfo.voters.asScala.filter(_.replicaId == leaderId).head

def convertQuorumInfo(infos: Seq[QuorumInfo.ReplicaState], status: String): Seq[Array[String]] =
infos.map { info =>
Array(info.replicaId,
info.logEndOffset,
leader.logEndOffset - info.logEndOffset,
info.lastFetchTimeMs.orElse(-1),
info.lastCaughtUpTimeMs.orElse(-1),
status
).map(_.toString)
}
prettyPrintTable(
Array("NodeId", "LogEndOffset", "Lag", "LastFetchTimeMs", "LastCaughtUpTimeMs", "Status"),
(convertQuorumInfo(Seq(leader), "Leader")
++ convertQuorumInfo(quorumInfo.voters.asScala.filter(_.replicaId != leaderId).toSeq, "Follower")
++ convertQuorumInfo(quorumInfo.observers.asScala.toSeq, "Observer")).asJava,
scala.Console.out
)
}

private def handleDescribeStatus(admin: Admin): Unit = {
val clusterId = admin.describeCluster.clusterId.get
val quorumInfo = admin.describeMetadataQuorum.quorumInfo.get
val leaderId = quorumInfo.leaderId
val leader = quorumInfo.voters.asScala.filter(_.replicaId == leaderId).head
val maxLagFollower = quorumInfo.voters.asScala
.minBy(_.logEndOffset)
val maxFollowerLag = leader.logEndOffset - maxLagFollower.logEndOffset
val maxFollowerLagTimeMs =
if (leader == maxLagFollower) {
0
} else if (leader.lastCaughtUpTimeMs.isPresent && maxLagFollower.lastCaughtUpTimeMs.isPresent) {
leader.lastCaughtUpTimeMs.getAsLong - maxLagFollower.lastCaughtUpTimeMs.getAsLong
} else {
-1
}
println(
s"""|ClusterId: $clusterId
|LeaderId: ${quorumInfo.leaderId}
|LeaderEpoch: ${quorumInfo.leaderEpoch}
|HighWatermark: ${quorumInfo.highWatermark}
|MaxFollowerLag: $maxFollowerLag
|MaxFollowerLagTimeMs: $maxFollowerLagTimeMs
|CurrentVoters: ${quorumInfo.voters.asScala.map(_.replicaId).mkString("[", ",", "]")}
|CurrentObservers: ${quorumInfo.observers.asScala.map(_.replicaId).mkString("[", ",", "]")}
|""".stripMargin
)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -183,6 +183,10 @@ public Map<Integer, BrokerFeatures> brokerFeatures() {
));
}

public Collection<ControllerServer> controllerServers() {
return controllers().collect(Collectors.toList());
}

@Override
public ClusterType clusterType() {
return ClusterType.RAFT;
Expand Down
Loading