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


##########
core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala:
##########
@@ -3257,3 +3287,4 @@ object PlaintextAdminIntegrationTest {
     assertEquals(LogConfig.DEFAULT_COMPRESSION_TYPE, 
configs.get(brokerResource).get(ServerConfigs.COMPRESSION_TYPE_CONFIG).value)
   }
 }
+

Review Comment:
   please remove this change.



##########
core/src/test/scala/integration/kafka/admin/AdminFenceProducersIntegrationTest.scala:
##########
@@ -107,6 +110,43 @@ 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)
+    }
+    producer.initTransactions()
+    producer.beginTransaction()
+    producer.send(record).get()
+    producer.commitTransaction()
+    def timeoutOption = new FenceProducersOptions()
+    try {
+      adminClient.fenceProducers(Collections.singletonList(txnId), 
timeoutOption.timeoutMs(0)).all().get()
+    } catch {
+      case ee: ExecutionException =>
+        assertInstanceOf(classOf[TimeoutException], ee.getCause) //ok
+    }
+    producer.beginTransaction()
+    try {
+      producer.send(record).get()
+    } catch {
+      case _: ProducerFencedException => //ok
+      case ee: ExecutionException =>
+        assertInstanceOf(classOf[ProducerFencedException], ee.getCause) //ok
+      case e: Exception =>
+        throw e
+    }
+    assertDoesNotThrow(new Executable() {

Review Comment:
   This is the last execution, so we don't need to wrap it by 
`assertDoesNotThrow`



##########
core/src/test/scala/integration/kafka/admin/AdminFenceProducersIntegrationTest.scala:
##########
@@ -107,6 +110,43 @@ 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)
+    }
+    producer.initTransactions()
+    producer.beginTransaction()
+    producer.send(record).get()
+    producer.commitTransaction()
+    def timeoutOption = new FenceProducersOptions()

Review Comment:
   ```scala
       val e = assertThrows(classOf[ExecutionException], () => 
adminClient.fenceProducers(Collections.singletonList(txnId), new 
FenceProducersOptions().timeoutMs(0)).all().get())
       assertInstanceOf(classOf[TimeoutException], e.getCause)
   ```



##########
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
+    def newTopic = new NewTopic(topic, partition, 0.toShort)
+    client.createTopics(Collections.singleton(newTopic))

Review Comment:
   `client.createTopics(Collections.singleton(new NewTopic(topic, partition, 
0.toShort)))`



##########
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
+    def newTopic = new NewTopic(topic, partition, 0.toShort)
+    client.createTopics(Collections.singleton(newTopic))
+    assertTrue(client.listClientMetricsResources().all().get().isEmpty)
+    def name = "name"
+    def configResource = new 
ConfigResource(ConfigResource.Type.CLIENT_METRICS, name)
+    val configEntry = new ConfigEntry("interval.ms", "111")
+    def configOp = new AlterConfigOp(configEntry, AlterConfigOp.OpType.SET)
+    client.incrementalAlterConfigs(Collections.singletonMap(configResource, 
Collections.singletonList(configOp))).all().get()
+    def result = client.listClientMetricsResources().all().get()
+    def expected = Collections.singletonList(new 
ClientMetricsResourceListing(name))
+    assertEquals(new util.HashSet(expected), new util.HashSet(result))

Review Comment:
   the metadata could be unsync, so please use `waitUntilTrue` for this test 
case



##########
core/src/test/scala/integration/kafka/admin/AdminFenceProducersIntegrationTest.scala:
##########
@@ -107,6 +110,43 @@ class AdminFenceProducersIntegrationTest extends 
IntegrationTestHarness {
     assertThrows(classOf[ProducerFencedException], () => 
producer.commitTransaction())
   }
 
+  @ParameterizedTest
+  @ValueSource(strings = Array("zk", "kraft"))
+  @Timeout(value = 30)
+  def testFenceProducerTimeoutMs(quorum: String): Unit = {

Review Comment:
   this test case is used to verify the timeout, so we don't need to send 
transaction actually. 
   
   ```scala
     @ParameterizedTest
     @ValueSource(strings = Array("zk", "kraft"))
     @Timeout(value = 30)
     def testFenceProducerTimeoutMs(quorum: String): Unit = {
       val config = createConfig
       config.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, 
s"localhost:${TestUtils.IncorrectBrokerPort}")
       val admin = Admin.create(config)
       try {
         val e = assertThrows(classOf[ExecutionException], () => 
admin.fenceProducers(Collections.singletonList(txnId),
           new FenceProducersOptions().timeoutMs(0)).all().get())
         assertInstanceOf(classOf[TimeoutException], e.getCause)
       } finally admin.close(java.time.Duration.ofSeconds(0))
     }
   ```



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