This is an automated email from the ASF dual-hosted git repository.

lianetm pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
     new 023f9c26e60 KAFKA-18529: ConsumerRebootstrapTest should run for async 
consumer (#18554)
023f9c26e60 is described below

commit 023f9c26e60c0710891abd148cce52c1dadaf7cd
Author: TaiJuWu <[email protected]>
AuthorDate: Sat Jan 25 03:33:20 2025 +0800

    KAFKA-18529: ConsumerRebootstrapTest should run for async consumer (#18554)
    
    Reviewers: Kirk True <[email protected]>, Chia-Ping Tsai 
<[email protected]>, Lianet Magrans <[email protected]>
---
 .../integration/kafka/api/ConsumerRebootstrapTest.scala  | 16 ++++++++--------
 .../kafka/api/PlaintextAdminIntegrationTest.scala        |  6 +++---
 2 files changed, 11 insertions(+), 11 deletions(-)

diff --git 
a/core/src/test/scala/integration/kafka/api/ConsumerRebootstrapTest.scala 
b/core/src/test/scala/integration/kafka/api/ConsumerRebootstrapTest.scala
index a20e6954f8e..ea9345f7265 100644
--- a/core/src/test/scala/integration/kafka/api/ConsumerRebootstrapTest.scala
+++ b/core/src/test/scala/integration/kafka/api/ConsumerRebootstrapTest.scala
@@ -17,7 +17,7 @@
 package kafka.api
 
 import kafka.api.ConsumerRebootstrapTest._
-import 
kafka.server.QuorumTestHarness.getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly
+import 
kafka.server.QuorumTestHarness.getTestQuorumAndGroupProtocolParametersAll
 import kafka.utils.{TestInfoUtils, TestUtils}
 import org.apache.kafka.clients.CommonClientConfigs
 import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord}
@@ -135,12 +135,12 @@ object ConsumerRebootstrapTest {
 
   final val RebootstrapTestName = 
s"${TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames}.useRebootstrapTriggerMs={2}"
   def rebootstrapTestParams: stream.Stream[Arguments] = {
-    assertEquals(1, 
getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly.count())
-    val args = getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly
-      .findFirst().get.get
-    stream.Stream.of(
-      Arguments.of((args :+ true):_*),
-      Arguments.of((args :+ false):_*)
-    )
+    getTestQuorumAndGroupProtocolParametersAll
+      .flatMap { baseArgs =>
+        stream.Stream.of(
+          Arguments.of((baseArgs.get :+ true):_*),
+          Arguments.of((baseArgs.get :+ false):_*)
+        )
+      }
   }
 }
diff --git 
a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala 
b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala
index 0f4174e1250..b4e9d984129 100644
--- 
a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala
+++ 
b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala
@@ -207,10 +207,10 @@ class PlaintextAdminIntegrationTest extends 
BaseAdminIntegrationTest {
     Admin.create(config)
   }
 
-  @ParameterizedTest(name = 
TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
-  @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
   @Timeout(10)
-  def testDescribeUserScramCredentialsTimeout(quorum: String, groupProtocol: 
String): Unit = {
+  @ParameterizedTest
+  @ValueSource(strings = Array("kraft"))
+  def testDescribeUserScramCredentialsTimeout(quorum: String): Unit = {
     client = createInvalidAdminClient()
     try {
       // test describeUserScramCredentials(List<String> users, 
DescribeUserScramCredentialsOptions options)

Reply via email to