chia7712 commented on code in PR #16658:
URL: https://github.com/apache/kafka/pull/16658#discussion_r1720795618


##########
core/src/test/scala/integration/kafka/admin/AdminFenceProducersIntegrationTest.scala:
##########
@@ -107,6 +109,22 @@ class AdminFenceProducersIntegrationTest extends 
IntegrationTestHarness {
     assertThrows(classOf[ProducerFencedException], () => 
producer.commitTransaction())
   }
 
+  @ParameterizedTest
+  @ValueSource(strings = Array("zk", "kraft"))
+  @Timeout(value = 30)
+  def testFenceProducerTimeoutMs(quorum: String): Unit = {
+    adminClient = {
+      val config = createConfig
+      config.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, 
s"localhost:${TestUtils.IncorrectBrokerPort}")
+      Admin.create(config)
+    }
+    try {
+      val e = assertThrows(classOf[ExecutionException], () => 
adminClient.fenceProducers(Collections.singletonList(txnId),
+        new FenceProducersOptions().timeoutMs(0)).all().get())
+      assertInstanceOf(classOf[TimeoutException], e.getCause)
+    } finally adminClient.close()

Review Comment:
   please use `adminClient.close(java.time.Duration.ofSeconds(0))` to avoid 
blocking



##########
core/src/test/java/kafka/admin/ClientTelemetryTest.java:
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.test.ClusterInstance;
+import kafka.test.annotation.ClusterConfigProperty;
+import kafka.test.annotation.ClusterTest;
+import kafka.test.annotation.Type;
+import kafka.test.junit.ClusterTestExtensions;
+
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.Producer;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.metrics.KafkaMetric;
+import org.apache.kafka.common.metrics.MetricsReporter;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.server.telemetry.ClientTelemetry;
+import org.apache.kafka.server.telemetry.ClientTelemetryReceiver;
+
+import org.junit.jupiter.api.extension.ExtendWith;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertSame;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+@ExtendWith(value = ClusterTestExtensions.class)
+public class ClientTelemetryTest {
+
+    @ClusterTest(types = Type.KRAFT,
+            serverProperties = @ClusterConfigProperty(key = 
AdminClientConfig.METRIC_REPORTER_CLASSES_CONFIG,
+                    value = 
"kafka.admin.ClientTelemetryTest$GetIdClientTelemetry"))
+    public void testClientInstanceId(ClusterInstance clusterInstance) throws 
InterruptedException, ExecutionException {

Review Comment:
   I don't think we need those producers and consumers. Could you please remove 
them?



##########
core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala:
##########
@@ -3005,6 +3005,36 @@ class PlaintextAdminIntegrationTest extends 
BaseAdminIntegrationTest {
     assertEquals(expected, config.value())
   }
 
+  @ParameterizedTest
+  @ValueSource(strings = Array("quorum=kraft"))
+  def testListClientMetricsResources(ignored: String): Unit = {
+    client = createAdminClient
+    client.createTopics(Collections.singleton(new NewTopic(topic, partition, 
0.toShort)))
+    assertTrue(client.listClientMetricsResources().all().get().isEmpty)
+    val name = "name"
+    val configResource = new 
ConfigResource(ConfigResource.Type.CLIENT_METRICS, name)
+    val configEntry = new ConfigEntry("interval.ms", "111")
+    val configOp = new AlterConfigOp(configEntry, AlterConfigOp.OpType.SET)
+    client.incrementalAlterConfigs(Collections.singletonMap(configResource, 
Collections.singletonList(configOp))).all().get()
+    def result = client.listClientMetricsResources().all().get()
+    TestUtils.waitUntilTrue(() => result != null, "metadata time out")

Review Comment:
   this is incorrect as the result is never null. Please see following example:
   ```scala
       TestUtils.waitUntilTrue(() => {
         val results = client.listClientMetricsResources().all().get()
         results.size() == 1 && results.iterator().next().equals(new 
ClientMetricsResourceListing(name))
       }, "metadata timeout")
   ```



##########
core/src/test/java/kafka/admin/ClientTelemetryTest.java:
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.test.ClusterInstance;
+import kafka.test.annotation.ClusterConfigProperty;
+import kafka.test.annotation.ClusterTest;
+import kafka.test.annotation.Type;
+import kafka.test.junit.ClusterTestExtensions;
+
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.Producer;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.metrics.KafkaMetric;
+import org.apache.kafka.common.metrics.MetricsReporter;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.server.telemetry.ClientTelemetry;
+import org.apache.kafka.server.telemetry.ClientTelemetryReceiver;
+
+import org.junit.jupiter.api.extension.ExtendWith;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertSame;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+@ExtendWith(value = ClusterTestExtensions.class)
+public class ClientTelemetryTest {
+
+    @ClusterTest(types = Type.KRAFT,
+            serverProperties = @ClusterConfigProperty(key = 
AdminClientConfig.METRIC_REPORTER_CLASSES_CONFIG,
+                    value = 
"kafka.admin.ClientTelemetryTest$GetIdClientTelemetry"))
+    public void testClientInstanceId(ClusterInstance clusterInstance) throws 
InterruptedException, ExecutionException {
+        Map<String, Object> configs = new HashMap<>();
+        configs.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, 
clusterInstance.bootstrapServers());
+        try (Admin admin = Admin.create(configs)) {
+            String testTopicName = "test_topic";
+            admin.createTopics(Collections.singletonList(new 
NewTopic(testTopicName, 1, (short) 1)));
+            clusterInstance.waitForTopic(testTopicName, 1);
+            
+            Map<String, Object> producerConfigs = new HashMap<>();
+            producerConfigs.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, 
clusterInstance.bootstrapServers());
+            producerConfigs.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, 
StringSerializer.class.getName());
+            producerConfigs.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, 
StringSerializer.class.getName());
+
+            try (Producer<String, String> producer = new 
KafkaProducer<>(producerConfigs)) {
+                producer.send(new ProducerRecord<>(testTopicName, 0, null, 
"bar")).get();
+                producer.flush();
+            }
+
+            Map<String, Object> consumerConfigs = new HashMap<>();
+            consumerConfigs.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, 
clusterInstance.bootstrapServers());
+            consumerConfigs.put(ConsumerConfig.GROUP_ID_CONFIG, 
UUID.randomUUID().toString());
+            consumerConfigs.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, 
StringDeserializer.class.getName());
+            
consumerConfigs.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, 
StringDeserializer.class.getName());
+
+            try (Consumer<String, String> consumer = new 
KafkaConsumer<>(consumerConfigs)) {
+                consumer.assign(Collections.singletonList(new 
TopicPartition(testTopicName, 0)));
+                consumer.seekToBeginning(Collections.singletonList(new 
TopicPartition(testTopicName, 0)));
+                List<String> values = new ArrayList<>();
+                ConsumerRecords<String, String> records = 
consumer.poll(Duration.ofMinutes(1));
+                for (ConsumerRecord<String, String> record : records) {
+                    values.add(record.value());
+                }
+                assertEquals(1, values.size());
+                assertEquals("bar", values.get(0));
+            }
+            Uuid uuid = admin.clientInstanceId(Duration.ofSeconds(3));
+            assertNotNull(uuid);
+            assertEquals(uuid, admin.clientInstanceId(Duration.ofSeconds(3)));
+        }
+    }
+
+
+    @ClusterTest(types = Type.KRAFT)
+    public void testMetrics(ClusterInstance clusterInstance) {
+        Map<String, Object> configs = new HashMap<>();
+        configs.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, 
clusterInstance.bootstrapServers());
+        List<String> expectedMetricsName = Arrays.asList("request-size-max", 
"io-wait-ratio", "response-total",
+                "version",
+                "io-time-ns-avg", "network-io-rate");
+        try (Admin admin = Admin.create(configs)) {
+            List<String> actual = new ArrayList<>();

Review Comment:
   ```java
               Set<String> actualMetricsName = 
admin.metrics().keySet().stream().map(MetricName::name).collect(Collectors.toSet());
               expectedMetricsName.forEach(n -> 
assertTrue(actualMetricsName.contains(n), "actual metrics name: " + 
actualMetricsName
                " expected: " + n));
   ```



-- 
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

Reply via email to