You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by jg...@apache.org on 2021/11/13 18:08:07 UTC

[kafka] branch trunk updated: KAFKA-13421; Reenable `testRollingBrokerRestartsWithSmallerMaxGroupSizeConfigDisruptsBigGroup` (#11485)

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

jgus 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 e9db5a1  KAFKA-13421; Reenable `testRollingBrokerRestartsWithSmallerMaxGroupSizeConfigDisruptsBigGroup` (#11485)
e9db5a1 is described below

commit e9db5a11e48ffeec88bb37feb5bc08b756bb6662
Author: Jason Gustafson <ja...@confluent.io>
AuthorDate: Sat Nov 13 10:06:16 2021 -0800

    KAFKA-13421; Reenable `testRollingBrokerRestartsWithSmallerMaxGroupSizeConfigDisruptsBigGroup` (#11485)
    
    This test was disabled in https://github.com/apache/kafka/commit/af8100b94fda4a27511797233e9845078ae8a69f. The reason the test was failing is that it assumes that the reference to `servers` can be mutated directly. The implementation in `IntegrationTestHarness` is intended to allow this by returning a mutable buffer, but the implementation actually returns a copy of the underlying collection. This caused the test case to create multiple `KafkaServer` instances instead of one as intend [...]
    
    Reviewers: David Jacot <dj...@confluent.io>
---
 core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala   | 1 -
 .../test/scala/unit/kafka/integration/KafkaServerTestHarness.scala   | 5 +++--
 2 files changed, 3 insertions(+), 3 deletions(-)

diff --git a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala
index 9fc8727f..90748e7 100644
--- a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala
+++ b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala
@@ -299,7 +299,6 @@ class ConsumerBounceTest extends AbstractConsumerTest with Logging {
     * Then, 1 consumer should be left out of the group.
     */
   @Test
-  @Disabled // To be re-enabled once we fix KAFKA-13421
   def testRollingBrokerRestartsWithSmallerMaxGroupSizeConfigDisruptsBigGroup(): Unit = {
     val group = "group-max-size-test"
     val topic = "group-max-size-test"
diff --git a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala
index e8fdaf5..9bd9f57 100755
--- a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala
+++ b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala
@@ -55,7 +55,7 @@ abstract class KafkaServerTestHarness extends QuorumTestHarness {
    */
   def servers: mutable.Buffer[KafkaServer] = {
     checkIsZKTest()
-    _brokers.map(_.asInstanceOf[KafkaServer])
+    _brokers.asInstanceOf[mutable.Buffer[KafkaServer]]
   }
 
   var brokerList: String = null
@@ -208,8 +208,9 @@ abstract class KafkaServerTestHarness extends QuorumTestHarness {
           threadNamePrefix = None,
           enableForwarding
         )
+      } else {
+        brokers(i).startup()
       }
-      _brokers(i).startup()
       alive(i) = true
     }
   }