chia7712 commented on code in PR #19528: URL: https://github.com/apache/kafka/pull/19528#discussion_r2059628557
########## clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/MetricsDuringTopicCreationDeletionTest.java: ########## @@ -0,0 +1,185 @@ +/* + * 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 com.yammer.metrics.core.Gauge; +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 org.junit.jupiter.api.BeforeEach; + +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class MetricsDuringTopicCreationDeletionTest { + + 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<>(); + int topicNum = 2; + for (int n = 0; n < topicNum; n++) { + String topicName = "topic"; + topics.add(topicName + 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(); + } + } + + /* + * Checking all metrics we care in a single test is faster though it would be more elegant to have 3 @Test methods + */ + @ClusterTest( + types = {Type.KRAFT}, + serverProperties = { + @ClusterConfigProperty(key = ServerConfigs.DELETE_TOPIC_ENABLE_CONFIG, value = "true"), + @ClusterConfigProperty(key = "log.initial.task.delay.ms", value = "100"), + @ClusterConfigProperty(key = "log.segment.delete.delay.ms", value = "1000"), + @ClusterConfigProperty(key = ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, value = "false"), + // speed up the test for UnderReplicatedPartitions, which relies on the ISR expiry thread to execute concurrently with topic creation + // But the replica.lag.time.max.ms value still need to consider the slow Jenkins testing environment + @ClusterConfigProperty(key = ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_CONFIG, value = "4000") + } + ) + public void testMetricsDuringTopicCreateDelete() throws Exception { + + // For OfflinePartitionsCount and PreferredReplicaImbalanceCount even with https://issues.apache.org/jira/browse/KAFKA-4605 + // the test has worked reliably because the metric that gets triggered is the one generated by the first started server (controller) + final Gauge<Integer> offlinePartitionsCountGauge = getGauge("OfflinePartitionsCount"); + final int[] offlinePartitionsCount = {offlinePartitionsCountGauge.value()}; + + final Gauge<Integer> preferredReplicaImbalanceCountGauge = getGauge("PreferredReplicaImbalanceCount"); + final int[] preferredReplicaImbalanceCount = {preferredReplicaImbalanceCountGauge.value()}; + + // For UnderReplicatedPartitions, because of https://issues.apache.org/jira/browse/KAFKA-4605 + // we can't access the metrics value of each server. So instead we directly invoke the method + // replicaManager.underReplicatedPartitionCount() that defines the metrics value. + final int underReplicatedPartitionCount = 0; + + // Sanity check: ensure metrics haven't changed before test starts + assertEquals(initialOfflinePartitionsCount, offlinePartitionsCount[0]); + assertEquals(initialPreferredReplicaImbalanceCount, preferredReplicaImbalanceCount[0]); + assertEquals(initialUnderReplicatedPartitionCount, underReplicatedPartitionCount); + + // Thread that continuously checks for any change in metrics + running = true; + Thread thread = new Thread(() -> { Review Comment: Could you please consider rewriting the old-school code? for example: ```java 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; } } }); } }); return () -> { closed.set(true); f.join(); }; } ``` ```java try (var ignored = runThread()) { } ``` -- 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