FrankYang0529 commented on code in PR #19528: URL: https://github.com/apache/kafka/pull/19528#discussion_r2061207943
########## clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/MetricsDuringTopicCreationDeletionTest.java: ########## @@ -0,0 +1,194 @@ +/* + * 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.clients; + +import org.apache.kafka.common.test.ClusterInstance; +import org.apache.kafka.common.test.api.ClusterConfigProperty; +import org.apache.kafka.common.test.api.ClusterTest; +import org.apache.kafka.common.test.api.Type; +import org.apache.kafka.server.config.ReplicationConfigs; +import org.apache.kafka.server.config.ServerConfigs; +import org.apache.kafka.server.config.ServerLogConfigs; +import org.apache.kafka.server.metrics.KafkaYammerMetrics; + +import com.yammer.metrics.core.Gauge; + +import org.junit.jupiter.api.BeforeEach; + +import java.io.Closeable; +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicBoolean; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class MetricsDuringTopicCreationDeletionTest { + + private static final String TOPIC_NAME_PREFIX = "topic"; + private static final int TOPIC_NUM = 2; + private static final int CREATE_DELETE_ITERATIONS = 3; + private static final int REPLICATION_FACTOR = 1; + private static final int PARTITION_NUM = 3; + + private final ClusterInstance clusterInstance; + private final List<String> topics; + private volatile boolean running = true; + + private int initialOfflinePartitionsCount = 0; + private int initialPreferredReplicaImbalanceCount = 0; + private int initialUnderReplicatedPartitionCount = 0; + + public MetricsDuringTopicCreationDeletionTest(ClusterInstance clusterInstance) { + this.clusterInstance = clusterInstance; + this.topics = new ArrayList<>(); + for (int n = 0; n < TOPIC_NUM; n++) { + topics.add(TOPIC_NAME_PREFIX + n); + } + } + + /* + * Captures initial values of key controller metrics. + * These will be compared with final values to detect any changes. + */ + @BeforeEach + public void setUp() { + try { + initialOfflinePartitionsCount = getGauge("OfflinePartitionsCount").value(); + initialPreferredReplicaImbalanceCount = getGauge("PreferredReplicaImbalanceCount").value(); + initialUnderReplicatedPartitionCount = getGauge("UnderReplicatedPartitions").value(); Review Comment: Will these initial values get other value after `setUp` function? If not, we can remove this block and set them as final value. ```java private final int initialOfflinePartitionsCount = 0; private final int initialPreferredReplicaImbalanceCount = 0; private final int initialUnderReplicatedPartitionCount = 0; ``` ########## test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/ClusterInstance.java: ########## @@ -290,6 +290,19 @@ default void createTopic(String topicName, int partitions, short replicas, Map<S } } + /** + * Deletes a topic and waits for the deletion to complete. + * + * @param topicName The name of the topic to delete + * @throws InterruptedException If the operation is interrupted + */ + default void deleteTopic(String topicName) throws InterruptedException { + try (Admin admin = admin()) { + admin.deleteTopics(Collections.singletonList(topicName)); Review Comment: ```suggestion admin.deleteTopics(List.of(topicName)); ``` ########## clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/MetricsDuringTopicCreationDeletionTest.java: ########## @@ -0,0 +1,194 @@ +/* + * 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.clients; + +import org.apache.kafka.common.test.ClusterInstance; +import org.apache.kafka.common.test.api.ClusterConfigProperty; +import org.apache.kafka.common.test.api.ClusterTest; +import org.apache.kafka.common.test.api.Type; +import org.apache.kafka.server.config.ReplicationConfigs; +import org.apache.kafka.server.config.ServerConfigs; +import org.apache.kafka.server.config.ServerLogConfigs; +import org.apache.kafka.server.metrics.KafkaYammerMetrics; + +import com.yammer.metrics.core.Gauge; + +import org.junit.jupiter.api.BeforeEach; + +import java.io.Closeable; +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicBoolean; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class MetricsDuringTopicCreationDeletionTest { + + private static final String TOPIC_NAME_PREFIX = "topic"; + private static final int TOPIC_NUM = 2; + private static final int CREATE_DELETE_ITERATIONS = 3; + private static final int REPLICATION_FACTOR = 1; + private static final int PARTITION_NUM = 3; + + private final ClusterInstance clusterInstance; + private final List<String> topics; + private volatile boolean running = true; + + private int initialOfflinePartitionsCount = 0; + private int initialPreferredReplicaImbalanceCount = 0; + private int initialUnderReplicatedPartitionCount = 0; + + public MetricsDuringTopicCreationDeletionTest(ClusterInstance clusterInstance) { + this.clusterInstance = clusterInstance; + this.topics = new ArrayList<>(); + for (int n = 0; n < TOPIC_NUM; n++) { + topics.add(TOPIC_NAME_PREFIX + n); + } + } + + /* + * Captures initial values of key controller metrics. + * These will be compared with final values to detect any changes. + */ + @BeforeEach + public void setUp() { + try { + initialOfflinePartitionsCount = getGauge("OfflinePartitionsCount").value(); + initialPreferredReplicaImbalanceCount = getGauge("PreferredReplicaImbalanceCount").value(); + initialUnderReplicatedPartitionCount = getGauge("UnderReplicatedPartitions").value(); + } catch (Exception e) { + e.printStackTrace(); + } + } + + private Closeable runThread() { + var closed = new AtomicBoolean(false); + var f = CompletableFuture.runAsync(() -> { + while (!closed.get()) { + clusterInstance.brokers().values().forEach(broker -> { + if (running) { + // Get UnderReplicatedPartitions through JMX + Optional<Integer> underReplicatedCount = KafkaYammerMetrics.defaultRegistry().allMetrics().entrySet().stream() + .filter(entry -> entry.getKey().getName().endsWith("UnderReplicatedPartitions")) + .map(entry -> ((Gauge<Integer>) entry.getValue()).value()) + .findFirst(); + + int count = underReplicatedCount.orElse(0); + if (count != initialUnderReplicatedPartitionCount) { + running = false; + } + } + }); Review Comment: This block doesn't really use `broker` variable. We can simplify it as following: ```suggestion if (running) { // Get UnderReplicatedPartitions through JMX Optional<Integer> underReplicatedCount = KafkaYammerMetrics.defaultRegistry().allMetrics().entrySet().stream() .filter(entry -> entry.getKey().getName().endsWith("UnderReplicatedPartitions")) .map(entry -> ((Gauge<Integer>) entry.getValue()).value()) .findFirst(); int count = underReplicatedCount.orElse(0); if (count != initialUnderReplicatedPartitionCount) { running = false; } } ``` -- 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