hachikuji commented on code in PR #12206: URL: https://github.com/apache/kafka/pull/12206#discussion_r890689748
########## clients/src/main/java/org/apache/kafka/common/requests/DescribeQuorumResponse.java: ########## @@ -93,4 +94,25 @@ public static DescribeQuorumResponseData singletonResponse(TopicPartition topicP public static DescribeQuorumResponse parse(ByteBuffer buffer, short version) { return new DescribeQuorumResponse(new DescribeQuorumResponseData(new ByteBufferAccessor(buffer), version)); } + + /** + * Get the replica info for the given topic name and partition. + * @param topicName Name of the topic to fetch + * @param partition Index of the parition to fetch + * @param getVoterInfo Return the voter information if true, return observers otherwise + * @return List of {@link ReplicaState} + */ + private List<ReplicaState> getReplicaInfo(String topicName, Integer partition, boolean getVoterInfo) { Review Comment: nit: are we using this anymore? ########## clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java: ########## @@ -4321,6 +4328,94 @@ void handleFailure(Throwable throwable) { return new UpdateFeaturesResult(new HashMap<>(updateFutures)); } + @Override + public DescribeMetadataQuorumResult describeMetadataQuorum(DescribeMetadataQuorumOptions options) { + NodeProvider provider = new LeastLoadedNodeProvider(); + + final KafkaFutureImpl<QuorumInfo> future = new KafkaFutureImpl<>(); + final long now = time.milliseconds(); + final Call call = new Call( + "describeMetadataQuorum", calcDeadlineMs(now, options.timeoutMs()), provider) { + + private QuorumInfo createQuorumResult(final DescribeQuorumResponseData.PartitionData partition) { + List<QuorumInfo.ReplicaState> voters = new ArrayList<>(); + List<QuorumInfo.ReplicaState> observers = new ArrayList<>(); + partition.currentVoters().forEach(v -> { + voters.add(new QuorumInfo.ReplicaState(v.replicaId(), + v.logEndOffset(), + OptionalLong.of(v.lastFetchTimestamp()), + OptionalLong.of(v.lastCaughtUpTimestamp()))); + }); + partition.observers().forEach(o -> { + observers.add(new QuorumInfo.ReplicaState(o.replicaId(), + o.logEndOffset(), + OptionalLong.of(o.lastFetchTimestamp()), + OptionalLong.of(o.lastCaughtUpTimestamp()))); + }); + QuorumInfo info = new QuorumInfo(partition.leaderId(), voters, observers); + return info; + } + + @Override + DescribeQuorumRequest.Builder createRequest(int timeoutMs) { + return new Builder(DescribeQuorumRequest.singletonRequest( + new TopicPartition(METADATA_TOPIC_NAME, METADATA_TOPIC_PARTITION.partition()))); + } + + @Override + void handleResponse(AbstractResponse response) { + final DescribeQuorumResponse quorumResponse = (DescribeQuorumResponse) response; + try { + if (quorumResponse.data().errorCode() != Errors.NONE.code()) { + throw Errors.forCode(quorumResponse.data().errorCode()).exception(); + } + if (quorumResponse.data().topics().size() > 1) { + String msg = String.format("DescribeMetadataQuorum received {} topics when 1 was expected", + quorumResponse.data().topics().size()); + log.debug(msg); + throw new UnknownServerException(msg); + } + DescribeQuorumResponseData.TopicData topic = quorumResponse.data().topics().get(0); + if (!topic.topicName().equals(METADATA_TOPIC_NAME)) { + String msg = String.format("DescribeMetadataQuorum received a topic with name {} when {} was expected", + topic.topicName(), METADATA_TOPIC_NAME); + log.debug(msg); + throw new UnknownServerException(msg); + } + if (topic.partitions().size() > 1) { + String msg = String.format("DescribeMetadataQuorum received a topic {} with {} partitions when 1 was expected", + topic.topicName(), topic.partitions().size()); + log.debug(msg); + throw new UnknownServerException(msg); + } + DescribeQuorumResponseData.PartitionData partition = topic.partitions().get(0); + if (partition.partitionIndex() != METADATA_TOPIC_PARTITION.partition()) { + String msg = String.format("DescribeMetadataQuorum received a single partition with index {} when {} was expected", + partition.partitionIndex(), METADATA_TOPIC_PARTITION.partition()); + log.debug(msg); + throw new UnknownServerException(msg); + } + if (partition.errorCode() != Errors.NONE.code()) { + throw Errors.forCode(partition.errorCode()).exception(); + } + future.complete(createQuorumResult(partition)); + } catch (RuntimeException e) { + throw e; + } catch (Exception e) { Review Comment: Which exceptions are we catching here? ########## clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java: ########## @@ -4321,6 +4328,94 @@ void handleFailure(Throwable throwable) { return new UpdateFeaturesResult(new HashMap<>(updateFutures)); } + @Override + public DescribeMetadataQuorumResult describeMetadataQuorum(DescribeMetadataQuorumOptions options) { + NodeProvider provider = new LeastLoadedNodeProvider(); + + final KafkaFutureImpl<QuorumInfo> future = new KafkaFutureImpl<>(); + final long now = time.milliseconds(); + final Call call = new Call( + "describeMetadataQuorum", calcDeadlineMs(now, options.timeoutMs()), provider) { + + private QuorumInfo createQuorumResult(final DescribeQuorumResponseData.PartitionData partition) { + List<QuorumInfo.ReplicaState> voters = new ArrayList<>(); + List<QuorumInfo.ReplicaState> observers = new ArrayList<>(); + partition.currentVoters().forEach(v -> { + voters.add(new QuorumInfo.ReplicaState(v.replicaId(), + v.logEndOffset(), + OptionalLong.of(v.lastFetchTimestamp()), + OptionalLong.of(v.lastCaughtUpTimestamp()))); + }); + partition.observers().forEach(o -> { + observers.add(new QuorumInfo.ReplicaState(o.replicaId(), + o.logEndOffset(), + OptionalLong.of(o.lastFetchTimestamp()), + OptionalLong.of(o.lastCaughtUpTimestamp()))); + }); + QuorumInfo info = new QuorumInfo(partition.leaderId(), voters, observers); + return info; + } + + @Override + DescribeQuorumRequest.Builder createRequest(int timeoutMs) { + return new Builder(DescribeQuorumRequest.singletonRequest( + new TopicPartition(METADATA_TOPIC_NAME, METADATA_TOPIC_PARTITION.partition()))); + } + + @Override + void handleResponse(AbstractResponse response) { + final DescribeQuorumResponse quorumResponse = (DescribeQuorumResponse) response; + try { + if (quorumResponse.data().errorCode() != Errors.NONE.code()) { + throw Errors.forCode(quorumResponse.data().errorCode()).exception(); + } + if (quorumResponse.data().topics().size() > 1) { + String msg = String.format("DescribeMetadataQuorum received {} topics when 1 was expected", + quorumResponse.data().topics().size()); + log.debug(msg); + throw new UnknownServerException(msg); + } + DescribeQuorumResponseData.TopicData topic = quorumResponse.data().topics().get(0); + if (!topic.topicName().equals(METADATA_TOPIC_NAME)) { + String msg = String.format("DescribeMetadataQuorum received a topic with name {} when {} was expected", + topic.topicName(), METADATA_TOPIC_NAME); + log.debug(msg); + throw new UnknownServerException(msg); + } + if (topic.partitions().size() > 1) { + String msg = String.format("DescribeMetadataQuorum received a topic {} with {} partitions when 1 was expected", + topic.topicName(), topic.partitions().size()); + log.debug(msg); + throw new UnknownServerException(msg); + } + DescribeQuorumResponseData.PartitionData partition = topic.partitions().get(0); + if (partition.partitionIndex() != METADATA_TOPIC_PARTITION.partition()) { + String msg = String.format("DescribeMetadataQuorum received a single partition with index {} when {} was expected", + partition.partitionIndex(), METADATA_TOPIC_PARTITION.partition()); + log.debug(msg); + throw new UnknownServerException(msg); + } + if (partition.errorCode() != Errors.NONE.code()) { + throw Errors.forCode(partition.errorCode()).exception(); + } + future.complete(createQuorumResult(partition)); + } catch (RuntimeException e) { + throw e; Review Comment: Is the intent to retry here? For some errors, such as auth failures, we probably would rather fail fast. ########## clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java: ########## @@ -4321,6 +4330,73 @@ void handleFailure(Throwable throwable) { return new UpdateFeaturesResult(new HashMap<>(updateFutures)); } + @Override + public DescribeMetadataQuorumResult describeMetadataQuorum(DescribeMetadataQuorumOptions options) { + NodeProvider provider = new LeastLoadedNodeProvider(); + + final KafkaFutureImpl<QuorumInfo> future = new KafkaFutureImpl<>(); + final long now = time.milliseconds(); + final Call call = new Call( + "describeMetadataQuorum", calcDeadlineMs(now, options.timeoutMs()), provider) { + + private QuorumInfo createQuorumResult(final DescribeQuorumResponse response) { + Integer partition = 0; + String topicName = response.getTopicNameByIndex(0); + Integer leaderId = response.getPartitionLeaderId(topicName, partition); + List<QuorumInfo.ReplicaState> voters = new ArrayList<>(); + List<QuorumInfo.ReplicaState> observers = new ArrayList<>(); + response.getVoterInfo(topicName, partition).forEach(v -> { + voters.add(new QuorumInfo.ReplicaState(v.replicaId(), + v.logEndOffset(), + OptionalLong.of(v.lastFetchTimestamp()), + OptionalLong.of(v.lastCaughtUpTimestamp()))); + }); + response.getObserverInfo(topicName, partition).forEach(o -> { + observers.add(new QuorumInfo.ReplicaState(o.replicaId(), + o.logEndOffset(), + OptionalLong.of(o.lastFetchTimestamp()), + OptionalLong.of(o.lastCaughtUpTimestamp()))); + }); + QuorumInfo info = new QuorumInfo(topicName, leaderId, voters, observers); + return info; + } + + @Override + DescribeQuorumRequest.Builder createRequest(int timeoutMs) { Review Comment: You may have missed the comment here. ########## core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala: ########## @@ -778,4 +778,35 @@ class KRaftClusterTest { cluster.close() } } + + @Test + def testDescribeQuorumRequestToBrokers() : Unit = { + val cluster = new KafkaClusterTestKit.Builder( + new TestKitNodes.Builder(). + setNumBrokerNodes(4). + setNumControllerNodes(3).build()).build() + try { + cluster.format + cluster.startup + for (i <- 0 to 3) { + TestUtils.waitUntilTrue(() => cluster.brokers.get(i).brokerState == BrokerState.RUNNING, + "Broker Never started up") + } + val props = cluster.clientProperties() + props.put(AdminClientConfig.CLIENT_ID_CONFIG, this.getClass.getName) + val admin = Admin.create(props) + try { + val quorumState = admin.describeMetadataQuorum(new DescribeMetadataQuorumOptions) + val quorumInfo = quorumState.quorumInfo().get() + + assertEquals(0, quorumInfo.leaderId()) + assertEquals(3, quorumInfo.voters.size()) + assertEquals(0, quorumInfo.observers.size()) + } finally { + admin.close() + } + } finally { + cluster.close Review Comment: nit: we usually add parenthesis for mutators -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org