Skip to content

Commit 57ae6d6

Browse files
authored
KAFKA-18695 Remove quorum=kraft and kip932 from all integration tests (#19633)
Currently, the quorum uses kraft by default, so there's no need to specify it explicitly. For kip932 and isShareGroupTest, they are no longer used after #19542 . Reviewers: PoAn Yang <[email protected]>, Ken Huang <[email protected]>, Chia-Ping Tsai <[email protected]>
1 parent 54fd136 commit 57ae6d6

File tree

55 files changed

+776
-1211
lines changed

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

55 files changed

+776
-1211
lines changed

core/src/test/java/kafka/server/integration/EligibleLeaderReplicasIntegrationTest.java

Lines changed: 9 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -50,9 +50,8 @@
5050

5151
import org.junit.jupiter.api.AfterEach;
5252
import org.junit.jupiter.api.BeforeEach;
53+
import org.junit.jupiter.api.Test;
5354
import org.junit.jupiter.api.TestInfo;
54-
import org.junit.jupiter.params.ParameterizedTest;
55-
import org.junit.jupiter.params.provider.ValueSource;
5655

5756
import java.io.File;
5857
import java.time.Duration;
@@ -137,9 +136,8 @@ public void close() throws Exception {
137136
if (adminClient != null) adminClient.close();
138137
}
139138

140-
@ParameterizedTest
141-
@ValueSource(strings = {"kraft"})
142-
public void testHighWatermarkShouldNotAdvanceIfUnderMinIsr(String quorum) throws ExecutionException, InterruptedException {
139+
@Test
140+
public void testHighWatermarkShouldNotAdvanceIfUnderMinIsr() throws ExecutionException, InterruptedException {
143141
adminClient.createTopics(
144142
List.of(new NewTopic(testTopicName, 1, (short) 4))).all().get();
145143
TestUtils.waitForPartitionMetadata(brokers(), testTopicName, 0, 1000);
@@ -224,9 +222,8 @@ void waitUntilOneMessageIsConsumed(Consumer consumer) {
224222
);
225223
}
226224

227-
@ParameterizedTest
228-
@ValueSource(strings = {"kraft"})
229-
public void testElrMemberCanBeElected(String quorum) throws ExecutionException, InterruptedException {
225+
@Test
226+
public void testElrMemberCanBeElected() throws ExecutionException, InterruptedException {
230227
adminClient.createTopics(
231228
List.of(new NewTopic(testTopicName, 1, (short) 4))).all().get();
232229
TestUtils.waitForPartitionMetadata(brokers(), testTopicName, 0, 1000);
@@ -300,9 +297,8 @@ public void testElrMemberCanBeElected(String quorum) throws ExecutionException,
300297
}
301298
}
302299

303-
@ParameterizedTest
304-
@ValueSource(strings = {"kraft"})
305-
public void testElrMemberShouldBeKickOutWhenUncleanShutdown(String quorum) throws ExecutionException, InterruptedException {
300+
@Test
301+
public void testElrMemberShouldBeKickOutWhenUncleanShutdown() throws ExecutionException, InterruptedException {
306302
adminClient.createTopics(
307303
List.of(new NewTopic(testTopicName, 1, (short) 4))).all().get();
308304
TestUtils.waitForPartitionMetadata(brokers(), testTopicName, 0, 1000);
@@ -361,9 +357,8 @@ public void testElrMemberShouldBeKickOutWhenUncleanShutdown(String quorum) throw
361357
/*
362358
This test is only valid for KIP-966 part 1. When the unclean recovery is implemented, it should be removed.
363359
*/
364-
@ParameterizedTest
365-
@ValueSource(strings = {"kraft"})
366-
public void testLastKnownLeaderShouldBeElectedIfEmptyElr(String quorum) throws ExecutionException, InterruptedException {
360+
@Test
361+
public void testLastKnownLeaderShouldBeElectedIfEmptyElr() throws ExecutionException, InterruptedException {
367362
adminClient.createTopics(
368363
List.of(new NewTopic(testTopicName, 1, (short) 4))).all().get();
369364
TestUtils.waitForPartitionMetadata(brokers(), testTopicName, 0, 1000);

core/src/test/scala/integration/kafka/admin/RemoteTopicCrudTest.scala

Lines changed: 44 additions & 64 deletions
Original file line numberDiff line numberDiff line change
@@ -28,9 +28,9 @@ import org.apache.kafka.server.config.ServerLogConfigs
2828
import org.apache.kafka.server.log.remote.storage.{NoOpRemoteLogMetadataManager, NoOpRemoteStorageManager, RemoteLogManagerConfig, RemoteLogSegmentId, RemoteLogSegmentMetadata, RemoteLogSegmentState}
2929
import org.junit.jupiter.api.Assertions._
3030
import org.junit.jupiter.api.function.Executable
31-
import org.junit.jupiter.api.{BeforeEach, Tag, TestInfo}
31+
import org.junit.jupiter.api.{BeforeEach, Tag, Test, TestInfo}
3232
import org.junit.jupiter.params.ParameterizedTest
33-
import org.junit.jupiter.params.provider.{CsvSource, ValueSource}
33+
import org.junit.jupiter.params.provider.CsvSource
3434

3535
import java.util
3636
import java.util.concurrent.atomic.AtomicInteger
@@ -73,9 +73,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
7373
testTopicName = s"${info.getTestMethod.get().getName}-${Random.alphanumeric.take(10).mkString}"
7474
}
7575

76-
@ParameterizedTest
77-
@ValueSource(strings = Array("kraft"))
78-
def testCreateRemoteTopicWithValidRetentionTime(quorum: String): Unit = {
76+
@Test
77+
def testCreateRemoteTopicWithValidRetentionTime(): Unit = {
7978
val topicConfig = new Properties()
8079
topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true")
8180
topicConfig.put(TopicConfig.RETENTION_MS_CONFIG, "200")
@@ -85,9 +84,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
8584
verifyRemoteLogTopicConfigs(topicConfig)
8685
}
8786

88-
@ParameterizedTest
89-
@ValueSource(strings = Array("kraft"))
90-
def testCreateRemoteTopicWithValidRetentionSize(quorum: String): Unit = {
87+
@Test
88+
def testCreateRemoteTopicWithValidRetentionSize(): Unit = {
9189
val topicConfig = new Properties()
9290
topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true")
9391
topicConfig.put(TopicConfig.RETENTION_BYTES_CONFIG, "512")
@@ -97,9 +95,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
9795
verifyRemoteLogTopicConfigs(topicConfig)
9896
}
9997

100-
@ParameterizedTest
101-
@ValueSource(strings = Array("kraft"))
102-
def testCreateRemoteTopicWithInheritedLocalRetentionTime(quorum: String): Unit = {
98+
@Test
99+
def testCreateRemoteTopicWithInheritedLocalRetentionTime(): Unit = {
103100
// inherited local retention ms is 1000
104101
val topicConfig = new Properties()
105102
topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true")
@@ -109,9 +106,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
109106
verifyRemoteLogTopicConfigs(topicConfig)
110107
}
111108

112-
@ParameterizedTest
113-
@ValueSource(strings = Array("kraft"))
114-
def testCreateRemoteTopicWithInheritedLocalRetentionSize(quorum: String): Unit = {
109+
@Test
110+
def testCreateRemoteTopicWithInheritedLocalRetentionSize(): Unit = {
115111
// inherited local retention bytes is 1024
116112
val topicConfig = new Properties()
117113
topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true")
@@ -121,9 +117,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
121117
verifyRemoteLogTopicConfigs(topicConfig)
122118
}
123119

124-
@ParameterizedTest
125-
@ValueSource(strings = Array("kraft"))
126-
def testCreateRemoteTopicWithInvalidRetentionTime(quorum: String): Unit = {
120+
@Test
121+
def testCreateRemoteTopicWithInvalidRetentionTime(): Unit = {
127122
// inherited local retention ms is 1000
128123
val topicConfig = new Properties()
129124
topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true")
@@ -133,9 +128,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
133128
topicConfig = topicConfig))
134129
}
135130

136-
@ParameterizedTest
137-
@ValueSource(strings = Array("kraft"))
138-
def testCreateRemoteTopicWithInvalidRetentionSize(quorum: String): Unit = {
131+
@Test
132+
def testCreateRemoteTopicWithInvalidRetentionSize(): Unit = {
139133
// inherited local retention bytes is 1024
140134
val topicConfig = new Properties()
141135
topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true")
@@ -145,9 +139,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
145139
topicConfig = topicConfig))
146140
}
147141

148-
@ParameterizedTest
149-
@ValueSource(strings = Array("kraft"))
150-
def testCreateCompactedRemoteStorage(quorum: String): Unit = {
142+
@Test
143+
def testCreateCompactedRemoteStorage(): Unit = {
151144
val topicConfig = new Properties()
152145
topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true")
153146
topicConfig.put(TopicConfig.CLEANUP_POLICY_CONFIG, "compact")
@@ -158,8 +151,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
158151

159152
// `remote.log.delete.on.disable` and `remote.log.copy.disable` only works in KRaft mode.
160153
@ParameterizedTest
161-
@CsvSource(Array("kraft,true,true", "kraft,true,false", "kraft,false,true", "kraft,false,false"))
162-
def testCreateRemoteTopicWithCopyDisabledAndDeleteOnDisable(quorum: String, copyDisabled: Boolean, deleteOnDisable: Boolean): Unit = {
154+
@CsvSource(Array("true,true", "true,false", "false,true", "false,false"))
155+
def testCreateRemoteTopicWithCopyDisabledAndDeleteOnDisable(copyDisabled: Boolean, deleteOnDisable: Boolean): Unit = {
163156
val topicConfig = new Properties()
164157
topicConfig.put(TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG, copyDisabled.toString)
165158
topicConfig.put(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, deleteOnDisable.toString)
@@ -169,9 +162,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
169162
}
170163

171164
// `remote.log.delete.on.disable` only works in KRaft mode.
172-
@ParameterizedTest
173-
@ValueSource(strings = Array("kraft"))
174-
def testCreateTopicRetentionMsValidationWithRemoteCopyDisabled(quorum: String): Unit = {
165+
@Test
166+
def testCreateTopicRetentionMsValidationWithRemoteCopyDisabled(): Unit = {
175167
val testTopicName2 = testTopicName + "2"
176168
val testTopicName3 = testTopicName + "3"
177169
val errorMsgMs = "When `remote.log.copy.disable` is set to true, the `local.retention.ms` and `retention.ms` " +
@@ -235,9 +227,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
235227
admin.incrementalAlterConfigs(configs).all().get()
236228
}
237229

238-
@ParameterizedTest
239-
@ValueSource(strings = Array("kraft"))
240-
def testCreateTopicRetentionBytesValidationWithRemoteCopyDisabled(quorum: String): Unit = {
230+
@Test
231+
def testCreateTopicRetentionBytesValidationWithRemoteCopyDisabled(): Unit = {
241232
val testTopicName2 = testTopicName + "2"
242233
val testTopicName3 = testTopicName + "3"
243234
val errorMsgBytes = "When `remote.log.copy.disable` is set to true, the `local.retention.bytes` and `retention.bytes` " +
@@ -300,9 +291,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
300291
admin.incrementalAlterConfigs(configs).all().get()
301292
}
302293

303-
@ParameterizedTest
304-
@ValueSource(strings = Array("kraft"))
305-
def testEnableRemoteLogOnExistingTopicTest(quorum: String): Unit = {
294+
@Test
295+
def testEnableRemoteLogOnExistingTopicTest(): Unit = {
306296
val admin = createAdminClient()
307297
val topicConfig = new Properties()
308298
TestUtils.createTopicWithAdmin(admin, testTopicName, brokers, controllerServers, numPartitions, numReplicationFactor,
@@ -318,9 +308,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
318308
verifyRemoteLogTopicConfigs(topicConfig)
319309
}
320310

321-
@ParameterizedTest
322-
@ValueSource(strings = Array("kraft"))
323-
def testEnableRemoteLogWhenSystemRemoteStorageIsDisabled(quorum: String): Unit = {
311+
@Test
312+
def testEnableRemoteLogWhenSystemRemoteStorageIsDisabled(): Unit = {
324313
val admin = createAdminClient()
325314

326315
val topicConfigWithRemoteStorage = new Properties()
@@ -342,9 +331,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
342331
assertTrue(errorMessage.getMessage.contains("Tiered Storage functionality is disabled in the broker"))
343332
}
344333

345-
@ParameterizedTest
346-
@ValueSource(strings = Array("kraft"))
347-
def testUpdateTopicConfigWithValidRetentionTimeTest(quorum: String): Unit = {
334+
@Test
335+
def testUpdateTopicConfigWithValidRetentionTimeTest(): Unit = {
348336
val admin = createAdminClient()
349337
val topicConfig = new Properties()
350338
topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true")
@@ -363,9 +351,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
363351
verifyRemoteLogTopicConfigs(topicConfig)
364352
}
365353

366-
@ParameterizedTest
367-
@ValueSource(strings = Array("kraft"))
368-
def testUpdateTopicConfigWithValidRetentionSizeTest(quorum: String): Unit = {
354+
@Test
355+
def testUpdateTopicConfigWithValidRetentionSizeTest(): Unit = {
369356
val admin = createAdminClient()
370357
val topicConfig = new Properties()
371358
topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true")
@@ -384,9 +371,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
384371
verifyRemoteLogTopicConfigs(topicConfig)
385372
}
386373

387-
@ParameterizedTest
388-
@ValueSource(strings = Array("kraft"))
389-
def testUpdateTopicConfigWithInheritedLocalRetentionTime(quorum: String): Unit = {
374+
@Test
375+
def testUpdateTopicConfigWithInheritedLocalRetentionTime(): Unit = {
390376
val admin = createAdminClient()
391377
val topicConfig = new Properties()
392378
topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true")
@@ -404,9 +390,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
404390
() => admin.incrementalAlterConfigs(configs).all().get())
405391
}
406392

407-
@ParameterizedTest
408-
@ValueSource(strings = Array("kraft"))
409-
def testUpdateTopicConfigWithInheritedLocalRetentionSize(quorum: String): Unit = {
393+
@Test
394+
def testUpdateTopicConfigWithInheritedLocalRetentionSize(): Unit = {
410395
val admin = createAdminClient()
411396
val topicConfig = new Properties()
412397
topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true")
@@ -425,9 +410,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
425410
}
426411

427412
// The remote storage config validation on controller level only works in KRaft
428-
@ParameterizedTest
429-
@ValueSource(strings = Array("kraft"))
430-
def testUpdateTopicConfigWithDisablingRemoteStorage(quorum: String): Unit = {
413+
@Test
414+
def testUpdateTopicConfigWithDisablingRemoteStorage(): Unit = {
431415
val admin = createAdminClient()
432416
val topicConfig = new Properties
433417
topicConfig.setProperty(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true")
@@ -446,9 +430,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
446430
"If you want to disable remote storage and delete all remote data, please set `remote.storage.enable=false,remote.log.delete.on.disable=true`.")
447431
}
448432

449-
@ParameterizedTest
450-
@ValueSource(strings = Array("kraft"))
451-
def testUpdateTopicConfigWithDisablingRemoteStorageWithDeleteOnDisable(quorum: String): Unit = {
433+
@Test
434+
def testUpdateTopicConfigWithDisablingRemoteStorageWithDeleteOnDisable(): Unit = {
452435
val admin = createAdminClient()
453436
val topicConfig = new Properties
454437
topicConfig.setProperty(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true")
@@ -473,9 +456,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
473456
verifyRemoteLogTopicConfigs(newProps)
474457
}
475458

476-
@ParameterizedTest
477-
@ValueSource(strings = Array("kraft"))
478-
def testTopicDeletion(quorum: String): Unit = {
459+
@Test
460+
def testTopicDeletion(): Unit = {
479461
MyRemoteStorageManager.deleteSegmentEventCounter.set(0)
480462
val numPartitions = 2
481463
val topicConfig = new Properties()
@@ -492,9 +474,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
492474
"Remote log segments should be deleted only once by the leader")
493475
}
494476

495-
@ParameterizedTest
496-
@ValueSource(strings = Array("kraft"))
497-
def testClusterWideDisablementOfTieredStorageWithEnabledTieredTopic(quorum: String): Unit = {
477+
@Test
478+
def testClusterWideDisablementOfTieredStorageWithEnabledTieredTopic(): Unit = {
498479
val topicConfig = new Properties()
499480
topicConfig.setProperty(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true")
500481

@@ -510,9 +491,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
510491
faultHandler.setIgnore(true)
511492
}
512493

513-
@ParameterizedTest
514-
@ValueSource(strings = Array("kraft"))
515-
def testClusterWithoutTieredStorageStartsSuccessfullyIfTopicWithTieringDisabled(quorum: String): Unit = {
494+
@Test
495+
def testClusterWithoutTieredStorageStartsSuccessfullyIfTopicWithTieringDisabled(): Unit = {
516496
val topicConfig = new Properties()
517497
topicConfig.setProperty(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, false.toString)
518498

core/src/test/scala/integration/kafka/api/AdminClientWithPoliciesIntegrationTest.scala

Lines changed: 7 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -29,9 +29,7 @@ import org.apache.kafka.server.policy.AlterConfigPolicy
2929
import org.apache.kafka.storage.internals.log.LogConfig
3030
import org.apache.kafka.test.TestUtils.assertFutureThrows
3131
import org.junit.jupiter.api.Assertions.{assertEquals, assertNull, assertTrue}
32-
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo, Timeout}
33-
import org.junit.jupiter.params.ParameterizedTest
34-
import org.junit.jupiter.params.provider.ValueSource
32+
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo, Timeout}
3533

3634
import scala.collection.mutable
3735
import scala.jdk.CollectionConverters._
@@ -79,9 +77,8 @@ class AdminClientWithPoliciesIntegrationTest extends KafkaServerTestHarness with
7977
props.put(ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_CONFIG, classOf[Policy])
8078
}
8179

82-
@ParameterizedTest
83-
@ValueSource(strings = Array("kraft"))
84-
def testValidAlterConfigs(quorum: String): Unit = {
80+
@Test
81+
def testValidAlterConfigs(): Unit = {
8582
client = Admin.create(createConfig)
8683
// Create topics
8784
val topic1 = "describe-alter-configs-topic-1"
@@ -100,16 +97,14 @@ class AdminClientWithPoliciesIntegrationTest extends KafkaServerTestHarness with
10097
PlaintextAdminIntegrationTest.checkValidAlterConfigs(client, this, topicResource1, topicResource2, maxMessageBytes, retentionMs)
10198
}
10299

103-
@ParameterizedTest
104-
@ValueSource(strings = Array("kraft"))
105-
def testInvalidAlterConfigs(quorum: String): Unit = {
100+
@Test
101+
def testInvalidAlterConfigs(): Unit = {
106102
client = Admin.create(createConfig)
107103
PlaintextAdminIntegrationTest.checkInvalidAlterConfigs(this, client)
108104
}
109105

110-
@ParameterizedTest
111-
@ValueSource(strings = Array("kraft"))
112-
def testInvalidAlterConfigsDueToPolicy(quorum: String): Unit = {
106+
@Test
107+
def testInvalidAlterConfigsDueToPolicy(): Unit = {
113108
client = Admin.create(createConfig)
114109

115110
// Create topics

0 commit comments

Comments
 (0)