From aaaab78bd871194fd4ff392b51a019cd084769f3 Mon Sep 17 00:00:00 2001 From: frankvicky Date: Fri, 17 Jan 2025 23:56:44 +0800 Subject: [PATCH] KAFKA-18561: Remove withKip853Rpc and replace with withRaftProtocol JIRA: KAFKA-18561 Remove withKip853Rpc and replace with withRaftProtocol and adjust corresponding test. --- .../raft/KafkaRaftClientReconfigTest.java | 97 +- .../raft/KafkaRaftClientSnapshotTest.java | 319 ++--- .../kafka/raft/KafkaRaftClientTest.java | 1131 +++++++++-------- .../kafka/raft/RaftClientTestContext.java | 9 - 4 files changed, 796 insertions(+), 760 deletions(-) diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java index 493083831d1de..15b367033fd5d 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java @@ -59,6 +59,7 @@ import java.util.stream.Stream; import static org.apache.kafka.raft.KafkaRaftClientTest.replicaKey; +import static org.apache.kafka.raft.RaftClientTestContext.RaftProtocol.KIP_853_PROTOCOL; import static org.apache.kafka.snapshot.Snapshots.BOOTSTRAP_SNAPSHOT_ID; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -75,7 +76,7 @@ public void testLeaderWritesBootstrapRecords() throws Exception { VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .withBootstrapSnapshot(Optional.of(voters)) .withUnknownLeader(0) .build(); @@ -150,7 +151,7 @@ public void testBootstrapCheckpointIsNotReturnedOnFetch() throws Exception { VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .withBootstrapSnapshot(Optional.of(voters)) .withUnknownLeader(0) .build(); @@ -229,7 +230,7 @@ public void testFollowerDoesNotRequestLeaderBootstrapSnapshot() throws Exception VoterSet voters = VoterSetTest.voterSet(Stream.of(local, leader)); RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .withBootstrapSnapshot(Optional.of(voters)) .withElectedLeader(epoch, leader.id()) .build(); @@ -259,7 +260,7 @@ public void testFollowerReadsKRaftBootstrapRecords() throws Exception { int epoch = 5; RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .withBootstrapSnapshot(Optional.of(voterSet)) .withElectedLeader(epoch, leader.id()) .build(); @@ -328,7 +329,7 @@ public void testAddVoter() throws Exception { VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .withBootstrapSnapshot(Optional.of(voters)) .withUnknownLeader(3) .build(); @@ -407,7 +408,7 @@ void testAddVoterInvalidClusterId() throws Exception { VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .withBootstrapSnapshot(Optional.of(voters)) .withUnknownLeader(3) .build(); @@ -444,7 +445,7 @@ void testAddVoterToNotLeader() throws Exception { VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .withBootstrapSnapshot(Optional.of(voters)) .withUnknownLeader(3) .build(); @@ -472,7 +473,7 @@ void testAddVoterWithMissingDefaultListener() throws Exception { VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .withBootstrapSnapshot(Optional.of(voters)) .withUnknownLeader(3) .build(); @@ -502,7 +503,7 @@ void testAddVoterWithPendingAddVoter() throws Exception { VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .withBootstrapSnapshot(Optional.of(voters)) .withUnknownLeader(3) .build(); @@ -558,7 +559,7 @@ void testAddVoterWithoutFencedPreviousLeaders() throws Exception { VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .withBootstrapSnapshot(Optional.of(voters)) .withUnknownLeader(3) .build(); @@ -596,7 +597,7 @@ void testAddVoterWithKraftVersion0() throws Exception { VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .withStaticVoters(voters) .withUnknownLeader(3) .build(); @@ -641,7 +642,7 @@ void testAddVoterWithExistingVoter() throws Exception { VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .withBootstrapSnapshot(Optional.of(voters)) .withUnknownLeader(3) .build(); @@ -686,7 +687,7 @@ void testAddVoterTimeout() throws Exception { VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .withBootstrapSnapshot(Optional.of(voters)) .withUnknownLeader(3) .build(); @@ -758,7 +759,7 @@ void testAddVoterWithApiVersionsFromIncorrectNode() throws Exception { VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .withBootstrapSnapshot(Optional.of(voters)) .withUnknownLeader(3) .build(); @@ -818,7 +819,7 @@ void testAddVoterInvalidFeatureVersion() throws Exception { VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .withBootstrapSnapshot(Optional.of(voters)) .withUnknownLeader(3) .build(); @@ -878,7 +879,7 @@ void testAddVoterWithLaggingNewVoter() throws Exception { VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .withBootstrapSnapshot(Optional.of(voters)) .withUnknownLeader(3) .build(); @@ -931,7 +932,7 @@ void testAddVoterFailsWhenLosingLeadership() throws Exception { VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .withBootstrapSnapshot(Optional.of(voters)) .withUnknownLeader(3) .build(); @@ -980,7 +981,7 @@ void testAddVoterWithMissingDirectoryId() throws Exception { VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .withBootstrapSnapshot(Optional.of(voters)) .withUnknownLeader(3) .build(); @@ -1019,7 +1020,7 @@ public void testRemoveVoter() throws Exception { VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower1, follower2)); RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .withBootstrapSnapshot(Optional.of(voters)) .withUnknownLeader(3) .build(); @@ -1068,7 +1069,7 @@ public void testRemoveVoterIsLeader() throws Exception { VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower1, follower2)); RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .withBootstrapSnapshot(Optional.of(voters)) .withUnknownLeader(3) .build(); @@ -1140,7 +1141,7 @@ public void testRemoveVoterInvalidClusterId() throws Exception { VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower1, follower2)); RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .withBootstrapSnapshot(Optional.of(voters)) .withUnknownLeader(3) .build(); @@ -1169,7 +1170,7 @@ void testRemoveVoterToNotLeader() throws Exception { VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower1, follower2)); RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .withBootstrapSnapshot(Optional.of(voters)) .withUnknownLeader(3) .build(); @@ -1189,7 +1190,7 @@ void testRemoveVoterWithPendingRemoveVoter() throws Exception { VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower1, follower2)); RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .withBootstrapSnapshot(Optional.of(voters)) .withUnknownLeader(3) .build(); @@ -1227,7 +1228,7 @@ void testRemoveVoterWithoutFencedPreviousLeaders() throws Exception { VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower1, follower2)); RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .withBootstrapSnapshot(Optional.of(voters)) .withUnknownLeader(3) .build(); @@ -1249,7 +1250,7 @@ void testRemoveVoterWithKraftVersion0() throws Exception { VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower1, follower2)); RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .withStaticVoters(voters) .withUnknownLeader(3) .build(); @@ -1279,7 +1280,7 @@ void testRemoveVoterWithNoneVoter() throws Exception { VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower1, follower2)); RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .withBootstrapSnapshot(Optional.of(voters)) .withUnknownLeader(3) .build(); @@ -1309,7 +1310,7 @@ void testRemoveVoterWithNoneVoterId() throws Exception { VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower1, follower2)); RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .withBootstrapSnapshot(Optional.of(voters)) .withUnknownLeader(3) .build(); @@ -1341,7 +1342,7 @@ void testRemoveVoterToEmptyVoterSet() throws Exception { VoterSet voters = VoterSetTest.voterSet(Stream.of(local)); RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .withBootstrapSnapshot(Optional.of(voters)) .build(); assertEquals(OptionalInt.of(local.id()), context.currentLeader()); @@ -1361,7 +1362,7 @@ void testRemoveVoterTimedOut() throws Exception { VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower1, follower2)); RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .withBootstrapSnapshot(Optional.of(voters)) .withUnknownLeader(3) .build(); @@ -1405,7 +1406,7 @@ void testRemoveVoterFailsWhenLosingLeadership() throws Exception { VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower1, follower2)); RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .withBootstrapSnapshot(Optional.of(voters)) .withUnknownLeader(3) .build(); @@ -1447,7 +1448,7 @@ void testAddVoterWithPendingRemoveVoter() throws Exception { VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower1, follower2)); RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .withBootstrapSnapshot(Optional.of(voters)) .withUnknownLeader(3) .build(); @@ -1492,7 +1493,7 @@ void testRemoveVoterWithPendingAddVoter() throws Exception { VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .withBootstrapSnapshot(Optional.of(voters)) .withUnknownLeader(3) .build(); @@ -1540,7 +1541,7 @@ void testUpdateVoter() throws Exception { VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .withBootstrapSnapshot(Optional.of(voters)) .withUnknownLeader(3) .build(); @@ -1609,7 +1610,7 @@ void testLeaderUpdatesVoter() throws Exception { Endpoints localListeners = Endpoints.fromInetSocketAddresses(listenersMap); RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .withBootstrapSnapshot(Optional.of(voters)) .withUnknownLeader(3) .withLocalListeners(localListeners) @@ -1645,7 +1646,7 @@ public void testUpdateVoterInvalidClusterId() throws Exception { VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .withBootstrapSnapshot(Optional.of(voters)) .withUnknownLeader(3) .build(); @@ -1696,7 +1697,7 @@ void testUpdateVoterOldEpoch() throws Exception { VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .withBootstrapSnapshot(Optional.of(voters)) .withUnknownLeader(3) .build(); @@ -1729,7 +1730,7 @@ void testUpdateVoterNewEpoch() throws Exception { VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .withBootstrapSnapshot(Optional.of(voters)) .withUnknownLeader(3) .build(); @@ -1762,7 +1763,7 @@ void testUpdateVoterToNotLeader() throws Exception { VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .withBootstrapSnapshot(Optional.of(voters)) .withUnknownLeader(3) .build(); @@ -1791,7 +1792,7 @@ void testUpdateVoterWithoutFencedPreviousLeaders() throws Exception { VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .withBootstrapSnapshot(Optional.of(voters)) .withUnknownLeader(3) .build(); @@ -1836,7 +1837,7 @@ void testUpdateVoterWithKraftVersion0() throws Exception { VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .withStaticVoters(voters) .withUnknownLeader(3) .build(); @@ -1887,7 +1888,7 @@ void testUpdateVoterWithNoneVoter() throws Exception { VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .withBootstrapSnapshot(Optional.of(voters)) .withUnknownLeader(3) .build(); @@ -1938,7 +1939,7 @@ void testUpdateVoterWithNoneVoterId() throws Exception { VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .withBootstrapSnapshot(Optional.of(voters)) .withUnknownLeader(3) .build(); @@ -1989,7 +1990,7 @@ void testUpdateVoterWithPendingAddVoter() throws Exception { VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .withBootstrapSnapshot(Optional.of(voters)) .withUnknownLeader(3) .build(); @@ -2072,7 +2073,7 @@ void testFollowerSendsUpdateVoter() throws Exception { Endpoints localListeners = Endpoints.fromInetSocketAddresses(listenersMap); RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .withBootstrapSnapshot(Optional.of(voters)) .withElectedLeader(epoch, voter1.id()) .withLocalListeners(localListeners) @@ -2133,7 +2134,7 @@ void testFollowerSendsUpdateVoterWhenDifferent() throws Exception { int epoch = 4; RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .withBootstrapSnapshot(Optional.of(voters)) .withElectedLeader(epoch, voter1.id()) .build(); @@ -2193,7 +2194,7 @@ void testUpdateVoterResponseCausesEpochChange() throws Exception { Endpoints localListeners = Endpoints.fromInetSocketAddresses(listenersMap); RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .withBootstrapSnapshot(Optional.of(voters)) .withElectedLeader(epoch, voter1.id()) .withLocalListeners(localListeners) @@ -2252,7 +2253,7 @@ void testObserverDiscoversLeaderWithUnknownVoters() throws Exception { int epoch = 3; RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .withBootstrapSnapshot(Optional.empty()) .withUnknownLeader(epoch) .withBootstrapServers(Optional.of(Collections.singletonList(bootstrapAddress))) @@ -2275,7 +2276,7 @@ public void testHandleBeginQuorumRequestMoreEndpoints() throws Exception { RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) .withBootstrapSnapshot(Optional.of(voters)) .withElectedLeader(leaderEpoch, leader.id()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .build(); context.client.poll(); diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java index 1f3307f9adaef..9778559c776e7 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java @@ -36,7 +36,9 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.CsvSource; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.EnumSource; +import org.junit.jupiter.params.provider.MethodSource; import org.junit.jupiter.params.provider.ValueSource; import java.io.IOException; @@ -52,6 +54,9 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Stream; +import static org.apache.kafka.raft.RaftClientTestContext.Builder; +import static org.apache.kafka.raft.RaftClientTestContext.MockListener; +import static org.apache.kafka.raft.RaftClientTestContext.RaftProtocol; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertInstanceOf; @@ -59,6 +64,9 @@ import static org.junit.jupiter.api.Assertions.assertTrue; public final class KafkaRaftClientSnapshotTest { + private static final String KIP_595_PROTOCOL = "KIP_595_PROTOCOL"; + private static final String KIP_853_PROTOCOL = "KIP_853_PROTOCOL"; + @Test public void testLatestSnapshotId() throws Exception { int localId = randomReplicaId(); @@ -67,7 +75,7 @@ public void testLatestSnapshotId() throws Exception { int epoch = 2; OffsetAndEpoch snapshotId = new OffsetAndEpoch(3, 1); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .appendToLog(snapshotId.epoch(), Arrays.asList("a", "b", "c")) .appendToLog(snapshotId.epoch(), Arrays.asList("d", "e", "f")) .withEmptySnapshot(snapshotId) @@ -85,7 +93,7 @@ public void testLatestSnapshotIdMissing() throws Exception { int epoch = 2; OffsetAndEpoch snapshotId = new OffsetAndEpoch(3, 1); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .appendToLog(snapshotId.epoch(), Arrays.asList("a", "b", "c")) .appendToLog(snapshotId.epoch(), Arrays.asList("d", "e", "f")) .withElectedLeader(epoch, leaderId) @@ -95,15 +103,15 @@ public void testLatestSnapshotIdMissing() throws Exception { } @ParameterizedTest - @CsvSource({ "false,false", "false,true", "true,false", "true,true" }) - public void testLeaderListenerNotified(boolean entireLog, boolean withKip853Rpc) throws Exception { + @MethodSource("generateKip853TestMatrix") + public void testLeaderListenerNotified(boolean entireLog, RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); ReplicaKey otherNodeKey = replicaKey(localId + 1, false); Set voters = Set.of(localId, otherNodeKey.id()); OffsetAndEpoch snapshotId = new OffsetAndEpoch(3, 1); - RaftClientTestContext.Builder contextBuilder = new RaftClientTestContext.Builder(localId, voters) - .withKip853Rpc(withKip853Rpc) + Builder contextBuilder = new Builder(localId, voters) + .withRaftProtocol(raftProtocol) .appendToLog(snapshotId.epoch(), Arrays.asList("a", "b", "c")) .appendToLog(snapshotId.epoch(), Arrays.asList("d", "e", "f")) .withEmptySnapshot(snapshotId); @@ -140,7 +148,7 @@ public void testFollowerListenerNotified(boolean entireLog) throws Exception { int epoch = 2; OffsetAndEpoch snapshotId = new OffsetAndEpoch(3, 1); - RaftClientTestContext.Builder contextBuilder = new RaftClientTestContext.Builder(localId, voters) + Builder contextBuilder = new Builder(localId, voters) .appendToLog(snapshotId.epoch(), Arrays.asList("a", "b", "c")) .appendToLog(snapshotId.epoch(), Arrays.asList("d", "e", "f")) .withEmptySnapshot(snapshotId) @@ -182,7 +190,7 @@ public void testSecondListenerNotified(boolean entireLog) throws Exception { int epoch = 2; OffsetAndEpoch snapshotId = new OffsetAndEpoch(3, 1); - RaftClientTestContext.Builder contextBuilder = new RaftClientTestContext.Builder(localId, voters) + Builder contextBuilder = new Builder(localId, voters) .appendToLog(snapshotId.epoch(), Arrays.asList("a", "b", "c")) .appendToLog(snapshotId.epoch(), Arrays.asList("d", "e", "f")) .withEmptySnapshot(snapshotId) @@ -208,7 +216,7 @@ public void testSecondListenerNotified(boolean entireLog) throws Exception { context.pollUntilRequest(); context.assertSentFetchRequest(epoch, localLogEndOffset, snapshotId.epoch()); - RaftClientTestContext.MockListener secondListener = new RaftClientTestContext.MockListener(OptionalInt.of(localId)); + MockListener secondListener = new MockListener(OptionalInt.of(localId)); context.client.register(secondListener); context.client.poll(); @@ -220,15 +228,15 @@ public void testSecondListenerNotified(boolean entireLog) throws Exception { } @ParameterizedTest - @ValueSource(booleans = { false, true }) - public void testListenerRenotified(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class, names = {KIP_595_PROTOCOL, KIP_853_PROTOCOL}) + public void testListenerRenotified(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, isSupport853(raftProtocol)); Set voters = Set.of(localId, otherNodeKey.id()); OffsetAndEpoch snapshotId = new OffsetAndEpoch(3, 1); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withKip853Rpc(withKip853Rpc) + RaftClientTestContext context = new Builder(localId, voters) + .withRaftProtocol(raftProtocol) .appendToLog(snapshotId.epoch(), Arrays.asList("a", "b", "c")) .appendToLog(snapshotId.epoch(), Arrays.asList("d", "e", "f")) .appendToLog(snapshotId.epoch(), Arrays.asList("g", "h", "i")) @@ -276,16 +284,16 @@ public void testListenerRenotified(boolean withKip853Rpc) throws Exception { } @ParameterizedTest - @ValueSource(booleans = { false, true }) - public void testLeaderImmediatelySendsSnapshotId(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class, names = {KIP_595_PROTOCOL, KIP_853_PROTOCOL}) + public void testLeaderImmediatelySendsSnapshotId(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, isSupport853(raftProtocol)); Set voters = Set.of(localId, otherNodeKey.id()); OffsetAndEpoch snapshotId = new OffsetAndEpoch(3, 4); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withUnknownLeader(snapshotId.epoch()) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .appendToLog(snapshotId.epoch(), Arrays.asList("a", "b", "c")) .appendToLog(snapshotId.epoch(), Arrays.asList("d", "e", "f")) .appendToLog(snapshotId.epoch(), Arrays.asList("g", "h", "i")) @@ -310,15 +318,15 @@ public void testLeaderImmediatelySendsSnapshotId(boolean withKip853Rpc) throws E } @ParameterizedTest - @ValueSource(booleans = { false, true }) - public void testFetchRequestOffsetLessThanLogStart(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class, names = {KIP_595_PROTOCOL, KIP_853_PROTOCOL}) + public void testFetchRequestOffsetLessThanLogStart(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, isSupport853(raftProtocol)); Set voters = Set.of(localId, otherNodeKey.id()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withAppendLingerMs(1) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -359,16 +367,16 @@ public void testFetchRequestOffsetLessThanLogStart(boolean withKip853Rpc) throws } @ParameterizedTest - @ValueSource(booleans = { false, true }) - public void testFetchRequestOffsetAtZero(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class, names = {KIP_595_PROTOCOL, KIP_853_PROTOCOL}) + public void testFetchRequestOffsetAtZero(RaftProtocol raftProtocol) throws Exception { // When the follower sends a FETCH request at offset 0, reply with snapshot id if it exists int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, isSupport853(raftProtocol)); Set voters = Set.of(localId, otherNodeKey.id()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withAppendLingerMs(1) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -408,19 +416,19 @@ public void testFetchRequestOffsetAtZero(boolean withKip853Rpc) throws Exception } @ParameterizedTest - @ValueSource(booleans = { false, true }) - public void testFetchRequestWithLargerLastFetchedEpoch(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class, names = {KIP_595_PROTOCOL, KIP_853_PROTOCOL}) + public void testFetchRequestWithLargerLastFetchedEpoch(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, isSupport853(raftProtocol)); Set voters = Set.of(localId, otherNodeKey.id()); OffsetAndEpoch oldestSnapshotId = new OffsetAndEpoch(3, 2); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .appendToLog(oldestSnapshotId.epoch(), Arrays.asList("a", "b", "c")) .appendToLog(oldestSnapshotId.epoch(), Arrays.asList("d", "e", "f")) .withAppendLingerMs(1) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -449,20 +457,20 @@ public void testFetchRequestWithLargerLastFetchedEpoch(boolean withKip853Rpc) th } @ParameterizedTest - @ValueSource(booleans = { false, true }) - public void testFetchRequestTruncateToLogStart(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class, names = {KIP_595_PROTOCOL, KIP_853_PROTOCOL}) + public void testFetchRequestTruncateToLogStart(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, isSupport853(raftProtocol)); int syncNodeId = otherNodeKey.id() + 1; Set voters = Set.of(localId, otherNodeKey.id(), syncNodeId); OffsetAndEpoch oldestSnapshotId = new OffsetAndEpoch(3, 2); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .appendToLog(oldestSnapshotId.epoch(), Arrays.asList("a", "b", "c")) .appendToLog(oldestSnapshotId.epoch() + 2, Arrays.asList("d", "e", "f")) .withAppendLingerMs(1) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -499,21 +507,21 @@ public void testFetchRequestTruncateToLogStart(boolean withKip853Rpc) throws Exc } @ParameterizedTest - @ValueSource(booleans = { false, true }) - public void testFetchRequestAtLogStartOffsetWithValidEpoch(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class, names = {KIP_595_PROTOCOL, KIP_853_PROTOCOL}) + public void testFetchRequestAtLogStartOffsetWithValidEpoch(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, isSupport853(raftProtocol)); int syncNodeId = otherNodeKey.id() + 1; Set voters = Set.of(localId, otherNodeKey.id(), syncNodeId); OffsetAndEpoch oldestSnapshotId = new OffsetAndEpoch(3, 2); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .appendToLog(oldestSnapshotId.epoch(), Arrays.asList("a", "b", "c")) .appendToLog(oldestSnapshotId.epoch(), Arrays.asList("d", "e", "f")) .appendToLog(oldestSnapshotId.epoch() + 2, Arrays.asList("g", "h", "i")) .withAppendLingerMs(1) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -545,21 +553,21 @@ public void testFetchRequestAtLogStartOffsetWithValidEpoch(boolean withKip853Rpc } @ParameterizedTest - @ValueSource(booleans = { false, true }) - public void testFetchRequestAtLogStartOffsetWithInvalidEpoch(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class, names = {KIP_595_PROTOCOL, KIP_853_PROTOCOL}) + public void testFetchRequestAtLogStartOffsetWithInvalidEpoch(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, isSupport853(raftProtocol)); int syncNodeId = otherNodeKey.id() + 1; Set voters = Set.of(localId, otherNodeKey.id(), syncNodeId); OffsetAndEpoch oldestSnapshotId = new OffsetAndEpoch(3, 2); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .appendToLog(oldestSnapshotId.epoch(), Arrays.asList("a", "b", "c")) .appendToLog(oldestSnapshotId.epoch(), Arrays.asList("d", "e", "f")) .appendToLog(oldestSnapshotId.epoch() + 2, Arrays.asList("g", "h", "i")) .withAppendLingerMs(1) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -597,23 +605,21 @@ public void testFetchRequestAtLogStartOffsetWithInvalidEpoch(boolean withKip853R } @ParameterizedTest - @ValueSource(booleans = { false, true }) - public void testFetchRequestWithLastFetchedEpochLessThanOldestSnapshot( - boolean withKip853Rpc - ) throws Exception { + @EnumSource(value = RaftProtocol.class, names = {KIP_595_PROTOCOL, KIP_853_PROTOCOL}) + public void testFetchRequestWithLastFetchedEpochLessThanOldestSnapshot(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, isSupport853(raftProtocol)); int syncNodeId = otherNodeKey.id() + 1; Set voters = Set.of(localId, otherNodeKey.id(), syncNodeId); OffsetAndEpoch oldestSnapshotId = new OffsetAndEpoch(3, 2); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .appendToLog(oldestSnapshotId.epoch(), Arrays.asList("a", "b", "c")) .appendToLog(oldestSnapshotId.epoch(), Arrays.asList("d", "e", "f")) .appendToLog(oldestSnapshotId.epoch() + 2, Arrays.asList("g", "h", "i")) .withAppendLingerMs(1) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -650,14 +656,14 @@ public void testFetchRequestWithLastFetchedEpochLessThanOldestSnapshot( } @ParameterizedTest - @ValueSource(booleans = { false, true }) - public void testFetchSnapshotRequestMissingSnapshot(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class, names = {KIP_595_PROTOCOL, KIP_853_PROTOCOL}) + public void testFetchSnapshotRequestMissingSnapshot(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); Set voters = Set.of(localId, localId + 1); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withUnknownLeader(3) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -686,9 +692,8 @@ public void testFetchSnapshotRequestBootstrapSnapshot() throws Exception { Stream.of(localKey, replicaKey(localKey.id() + 1, true)) ); - RaftClientTestContext context = new RaftClientTestContext - .Builder(localKey.id(), localKey.directoryId().get()) - .withKip853Rpc(true) + RaftClientTestContext context = new Builder(localKey.id(), localKey.directoryId().get()) + .withRaftProtocol(RaftProtocol.KIP_853_PROTOCOL) .withBootstrapSnapshot(Optional.of(voters)) .withUnknownLeader(3) .build(); @@ -713,15 +718,15 @@ public void testFetchSnapshotRequestBootstrapSnapshot() throws Exception { } @ParameterizedTest - @ValueSource(booleans = { false, true }) - public void testFetchSnapshotRequestUnknownPartition(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class, names = {KIP_595_PROTOCOL, KIP_853_PROTOCOL}) + public void testFetchSnapshotRequestUnknownPartition(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); Set voters = Set.of(localId, localId + 1); TopicPartition topicPartition = new TopicPartition("unknown", 0); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withUnknownLeader(3) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -744,16 +749,16 @@ public void testFetchSnapshotRequestUnknownPartition(boolean withKip853Rpc) thro } @ParameterizedTest - @ValueSource(booleans = { false, true }) - public void testFetchSnapshotRequestAsLeader(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class, names = {KIP_595_PROTOCOL, KIP_853_PROTOCOL}) + public void testFetchSnapshotRequestAsLeader(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); Set voters = Set.of(localId, localId + 1); OffsetAndEpoch snapshotId = new OffsetAndEpoch(1, 1); List records = Arrays.asList("foo", "bar"); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .appendToLog(snapshotId.epoch(), Collections.singletonList("a")) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -795,21 +800,20 @@ public void testFetchSnapshotRequestAsLeader(boolean withKip853Rpc) throws Excep } @ParameterizedTest - @ValueSource(booleans = { false, true }) - public void testLeaderShouldResignLeadershipIfNotGetFetchSnapshotRequestFromMajorityVoters( - boolean withKip853Rpc - ) throws Exception { + @EnumSource(value = RaftProtocol.class, names = {KIP_595_PROTOCOL, KIP_853_PROTOCOL}) + public void testLeaderShouldResignLeadershipIfNotGetFetchSnapshotRequestFromMajorityVoters(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey voter1 = replicaKey(localId + 1, withKip853Rpc); - ReplicaKey voter2 = replicaKey(localId + 2, withKip853Rpc); - ReplicaKey observer3 = replicaKey(localId + 3, withKip853Rpc); + boolean support853 = isSupport853(raftProtocol); + ReplicaKey voter1 = replicaKey(localId + 1, support853); + ReplicaKey voter2 = replicaKey(localId + 2, support853); + ReplicaKey observer3 = replicaKey(localId + 3, support853); Set voters = Set.of(localId, voter1.id(), voter2.id()); OffsetAndEpoch snapshotId = new OffsetAndEpoch(1, 1); List records = Arrays.asList("foo", "bar"); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .appendToLog(snapshotId.epoch(), Collections.singletonList("a")) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); int resignLeadershipTimeout = context.checkQuorumTimeoutMs; @@ -890,16 +894,16 @@ public void testLeaderShouldResignLeadershipIfNotGetFetchSnapshotRequestFromMajo } @ParameterizedTest - @ValueSource(booleans = { false, true }) - public void testPartialFetchSnapshotRequestAsLeader(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class, names = {KIP_595_PROTOCOL, KIP_853_PROTOCOL}) + public void testPartialFetchSnapshotRequestAsLeader(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); Set voters = Set.of(localId, localId + 1); OffsetAndEpoch snapshotId = new OffsetAndEpoch(2, 1); List records = Arrays.asList("foo", "bar"); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .appendToLog(snapshotId.epoch(), records) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -971,17 +975,17 @@ public void testPartialFetchSnapshotRequestAsLeader(boolean withKip853Rpc) throw } @ParameterizedTest - @ValueSource(booleans = { false, true }) - public void testFetchSnapshotRequestAsFollower(boolean withKip853Rpc) throws IOException { + @EnumSource(value = RaftProtocol.class, names = {KIP_595_PROTOCOL, KIP_853_PROTOCOL}) + public void testFetchSnapshotRequestAsFollower(RaftProtocol raftProtocol) throws IOException { int localId = randomReplicaId(); int leaderId = localId + 1; Set voters = Set.of(localId, leaderId); int epoch = 2; OffsetAndEpoch snapshotId = Snapshots.BOOTSTRAP_SNAPSHOT_ID; - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withElectedLeader(epoch, leaderId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.deliverRequest( @@ -1003,16 +1007,16 @@ public void testFetchSnapshotRequestAsFollower(boolean withKip853Rpc) throws IOE } @ParameterizedTest - @ValueSource(booleans = { false, true }) - public void testFetchSnapshotRequestWithInvalidPosition(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class, names = {KIP_595_PROTOCOL, KIP_853_PROTOCOL}) + public void testFetchSnapshotRequestWithInvalidPosition(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); Set voters = Set.of(localId, localId + 1); OffsetAndEpoch snapshotId = new OffsetAndEpoch(1, 1); List records = Arrays.asList("foo", "bar"); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .appendToLog(snapshotId.epoch(), Collections.singletonList("a")) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -1063,15 +1067,15 @@ public void testFetchSnapshotRequestWithInvalidPosition(boolean withKip853Rpc) t } @ParameterizedTest - @ValueSource(booleans = { false, true }) - public void testFetchSnapshotRequestWithOlderEpoch(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class, names = {KIP_595_PROTOCOL, KIP_853_PROTOCOL}) + public void testFetchSnapshotRequestWithOlderEpoch(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); Set voters = Set.of(localId, localId + 1); OffsetAndEpoch snapshotId = Snapshots.BOOTSTRAP_SNAPSHOT_ID; - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withUnknownLeader(1) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -1096,15 +1100,15 @@ public void testFetchSnapshotRequestWithOlderEpoch(boolean withKip853Rpc) throws } @ParameterizedTest - @ValueSource(booleans = { false, true }) - public void testFetchSnapshotRequestWithNewerEpoch(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class, names = {KIP_595_PROTOCOL, KIP_853_PROTOCOL}) + public void testFetchSnapshotRequestWithNewerEpoch(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); Set voters = Set.of(localId, localId + 1); OffsetAndEpoch snapshotId = Snapshots.BOOTSTRAP_SNAPSHOT_ID; - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withUnknownLeader(1) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -1129,8 +1133,8 @@ public void testFetchSnapshotRequestWithNewerEpoch(boolean withKip853Rpc) throws } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testFetchResponseWithInvalidSnapshotId(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class, names = {KIP_595_PROTOCOL, KIP_853_PROTOCOL}) + public void testFetchResponseWithInvalidSnapshotId(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int leaderId = localId + 1; Set voters = Set.of(localId, leaderId); @@ -1139,9 +1143,9 @@ public void testFetchResponseWithInvalidSnapshotId(boolean withKip853Rpc) throws OffsetAndEpoch invalidEndOffset = new OffsetAndEpoch(-1L, 1); int slept = 0; - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withElectedLeader(epoch, leaderId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.pollUntilRequest(); @@ -1192,17 +1196,17 @@ public void testFetchResponseWithInvalidSnapshotId(boolean withKip853Rpc) throws } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testFetchResponseWithSnapshotId(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class, names = {KIP_595_PROTOCOL, KIP_853_PROTOCOL}) + public void testFetchResponseWithSnapshotId(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int leaderId = localId + 1; Set voters = Set.of(localId, leaderId); int epoch = 2; OffsetAndEpoch snapshotId = new OffsetAndEpoch(100L, 1); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withElectedLeader(epoch, leaderId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.pollUntilRequest(); @@ -1265,17 +1269,17 @@ public void testFetchResponseWithSnapshotId(boolean withKip853Rpc) throws Except } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testFetchSnapshotResponsePartialData(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class, names = {KIP_595_PROTOCOL, KIP_853_PROTOCOL}) + public void testFetchSnapshotResponsePartialData(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int leaderId = localId + 1; Set voters = Set.of(localId, leaderId); int epoch = 2; OffsetAndEpoch snapshotId = new OffsetAndEpoch(100L, 1); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withElectedLeader(epoch, leaderId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.pollUntilRequest(); @@ -1370,17 +1374,17 @@ public void testFetchSnapshotResponsePartialData(boolean withKip853Rpc) throws E } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testFetchSnapshotResponseMissingSnapshot(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class, names = {KIP_595_PROTOCOL, KIP_853_PROTOCOL}) + public void testFetchSnapshotResponseMissingSnapshot(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int leaderId = localId + 1; Set voters = Set.of(localId, leaderId); int epoch = 2; OffsetAndEpoch snapshotId = new OffsetAndEpoch(100L, 1); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withElectedLeader(epoch, leaderId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.pollUntilRequest(); @@ -1431,8 +1435,8 @@ public void testFetchSnapshotResponseMissingSnapshot(boolean withKip853Rpc) thro } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testFetchSnapshotResponseFromNewerEpochNotLeader(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class, names = {KIP_595_PROTOCOL, KIP_853_PROTOCOL}) + public void testFetchSnapshotResponseFromNewerEpochNotLeader(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int firstLeaderId = localId + 1; int secondLeaderId = firstLeaderId + 1; @@ -1440,9 +1444,9 @@ public void testFetchSnapshotResponseFromNewerEpochNotLeader(boolean withKip853R int epoch = 2; OffsetAndEpoch snapshotId = new OffsetAndEpoch(100L, 1); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withElectedLeader(epoch, firstLeaderId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.pollUntilRequest(); @@ -1493,17 +1497,17 @@ public void testFetchSnapshotResponseFromNewerEpochNotLeader(boolean withKip853R } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testFetchSnapshotResponseFromNewerEpochLeader(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class, names = {KIP_595_PROTOCOL, KIP_853_PROTOCOL}) + public void testFetchSnapshotResponseFromNewerEpochLeader(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int leaderId = localId + 1; Set voters = Set.of(localId, leaderId); int epoch = 2; OffsetAndEpoch snapshotId = new OffsetAndEpoch(100L, 1); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withElectedLeader(epoch, leaderId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.pollUntilRequest(); @@ -1554,17 +1558,17 @@ public void testFetchSnapshotResponseFromNewerEpochLeader(boolean withKip853Rpc) } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testFetchSnapshotResponseFromOlderEpoch(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class, names = {KIP_595_PROTOCOL, KIP_853_PROTOCOL}) + public void testFetchSnapshotResponseFromOlderEpoch(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int leaderId = localId + 1; Set voters = Set.of(localId, leaderId); int epoch = 2; OffsetAndEpoch snapshotId = new OffsetAndEpoch(100L, 1); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withElectedLeader(epoch, leaderId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.pollUntilRequest(); @@ -1625,17 +1629,17 @@ public void testFetchSnapshotResponseFromOlderEpoch(boolean withKip853Rpc) throw } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testFetchSnapshotResponseWithInvalidId(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class, names = {KIP_595_PROTOCOL, KIP_853_PROTOCOL}) + public void testFetchSnapshotResponseWithInvalidId(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int leaderId = localId + 1; Set voters = Set.of(localId, leaderId); int epoch = 2; OffsetAndEpoch snapshotId = new OffsetAndEpoch(100L, 1); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withElectedLeader(epoch, leaderId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.pollUntilRequest(); @@ -1741,17 +1745,17 @@ public void testFetchSnapshotResponseWithInvalidId(boolean withKip853Rpc) throws } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testFetchSnapshotResponseToNotFollower(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class, names = {KIP_595_PROTOCOL, KIP_853_PROTOCOL}) + public void testFetchSnapshotResponseToNotFollower(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int leaderId = localId + 1; Set voters = Set.of(localId, leaderId); int epoch = 2; OffsetAndEpoch snapshotId = new OffsetAndEpoch(100L, 1); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withElectedLeader(epoch, leaderId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.pollUntilRequest(); @@ -1815,17 +1819,15 @@ public void testFetchSnapshotResponseToNotFollower(boolean withKip853Rpc) throws } @ParameterizedTest - @ValueSource(booleans = { false, true }) - public void testFetchSnapshotRequestClusterIdValidation( - boolean withKip853Rpc - ) throws Exception { + @EnumSource(value = RaftProtocol.class, names = {KIP_595_PROTOCOL, KIP_853_PROTOCOL}) + public void testFetchSnapshotRequestClusterIdValidation(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNode = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNode = replicaKey(localId + 1, isSupport853(raftProtocol)); Set voters = Set.of(localId, otherNode.id()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withUnknownLeader(4) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -1893,8 +1895,8 @@ public void testFetchSnapshotRequestClusterIdValidation( } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testCreateSnapshotAsLeaderWithInvalidSnapshotId(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class, names = {KIP_595_PROTOCOL, KIP_853_PROTOCOL}) + public void testCreateSnapshotAsLeaderWithInvalidSnapshotId(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; Set voters = Set.of(localId, otherNodeId); @@ -1903,10 +1905,10 @@ public void testCreateSnapshotAsLeaderWithInvalidSnapshotId(boolean withKip853Rp List appendRecords = Arrays.asList("a", "b", "c"); OffsetAndEpoch invalidSnapshotId1 = new OffsetAndEpoch(4, epoch); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .appendToLog(epoch, appendRecords) .withAppendLingerMs(1) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -1983,18 +1985,18 @@ public void testCreateSnapshotAsLeaderWithInvalidSnapshotId(boolean withKip853Rp } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testCreateSnapshotAsFollowerWithInvalidSnapshotId(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class, names = {KIP_595_PROTOCOL, KIP_853_PROTOCOL}) + public void testCreateSnapshotAsFollowerWithInvalidSnapshotId(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int leaderId = localId + 1; int otherFollowerId = localId + 2; int epoch = 5; Set voters = Set.of(localId, leaderId, otherFollowerId); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .appendToLog(1, List.of("a")) .withElectedLeader(epoch, leaderId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.assertElectedLeader(epoch, leaderId); @@ -2265,4 +2267,17 @@ public ByteBuffer buffer() { return data; } } + + private boolean isSupport853(RaftProtocol raftProtocol) { + return raftProtocol == RaftProtocol.KIP_853_PROTOCOL; + } + + private static Stream generateKip853TestMatrix() { + return Stream.of( + Arguments.of(false, RaftProtocol.KIP_595_PROTOCOL), + Arguments.of(false, RaftProtocol.KIP_853_PROTOCOL), + Arguments.of(true, RaftProtocol.KIP_595_PROTOCOL), + Arguments.of(true, RaftProtocol.KIP_853_PROTOCOL) + ); + } } diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java index 48b8c8b72c107..82f1f900ff249 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java @@ -45,9 +45,9 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.CsvSource; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.EnumSource; import org.junit.jupiter.params.provider.MethodSource; -import org.junit.jupiter.params.provider.ValueSource; import org.mockito.Mockito; import java.io.IOException; @@ -69,8 +69,14 @@ import java.util.stream.Stream; import static java.util.Collections.singletonList; +import static org.apache.kafka.raft.RaftClientTestContext.Builder; import static org.apache.kafka.raft.RaftClientTestContext.Builder.DEFAULT_ELECTION_TIMEOUT_MS; +import static org.apache.kafka.raft.RaftClientTestContext.MockListener; +import static org.apache.kafka.raft.RaftClientTestContext.RaftProtocol; +import static org.apache.kafka.raft.RaftClientTestContext.RaftProtocol.KIP_595_PROTOCOL; import static org.apache.kafka.raft.RaftClientTestContext.RaftProtocol.KIP_853_PROTOCOL; +import static org.apache.kafka.raft.RaftClientTestContext.assertMatchingRecords; +import static org.apache.kafka.raft.RaftClientTestContext.verifyLeaderChangeMessage; import static org.apache.kafka.test.TestUtils.assertFutureThrows; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -82,35 +88,38 @@ @SuppressWarnings({"ClassDataAbstractionCoupling", "ClassFanOutComplexity"}) public class KafkaRaftClientTest { + private static final String KIP_595_PROTOCOL_NAME = "KIP_595_PROTOCOL"; + private static final String KIP_853_PROTOCOL_NAME = "KIP_853_PROTOCOL"; + @Test public void testNodeDirectoryId() { int localId = randomReplicaId(); assertThrows( IllegalArgumentException.class, - new RaftClientTestContext.Builder(localId, Uuid.ZERO_UUID)::build + new Builder(localId, Uuid.ZERO_UUID)::build ); } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testInitializeSingleMemberQuorum(boolean withKip853Rpc) throws IOException { + @EnumSource(value = RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testInitializeSingleMemberQuorum(RaftProtocol raftProtocol) throws IOException { int localId = randomReplicaId(); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, Collections.singleton(localId)) - .withKip853Rpc(withKip853Rpc) + RaftClientTestContext context = new Builder(localId, Collections.singleton(localId)) + .withRaftProtocol(raftProtocol) .build(); context.assertElectedLeader(1, localId); assertEquals(context.log.endOffset().offset(), context.client.logEndOffset()); } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testInitializeAsLeaderFromStateStoreSingleMemberQuorum(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testInitializeAsLeaderFromStateStoreSingleMemberQuorum(RaftProtocol raftProtocol) throws Exception { // Start off as leader. We should still bump the epoch after initialization int localId = randomReplicaId(); int initialEpoch = 2; Set voters = Collections.singleton(localId); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withKip853Rpc(withKip853Rpc) + RaftClientTestContext context = new Builder(localId, voters) + .withRaftProtocol(raftProtocol) .withElectedLeader(initialEpoch, localId) .build(); @@ -123,18 +132,18 @@ public void testInitializeAsLeaderFromStateStoreSingleMemberQuorum(boolean withK } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testRejectVotesFromSameEpochAfterResigningLeadership(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testRejectVotesFromSameEpochAfterResigningLeadership(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int remoteId = localId + 1; - ReplicaKey remoteKey = replicaKey(remoteId, withKip853Rpc); + ReplicaKey remoteKey = replicaKey(remoteId, isSupport853(raftProtocol)); Set voters = Set.of(localId, remoteKey.id()); int epoch = 2; - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .updateRandom(r -> r.mockNextInt(DEFAULT_ELECTION_TIMEOUT_MS, 0)) .withElectedLeader(epoch, localId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); assertEquals(0L, context.log.endOffset().offset()); @@ -155,18 +164,18 @@ public void testRejectVotesFromSameEpochAfterResigningLeadership(boolean withKip } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testRejectVotesFromSameEpochAfterResigningCandidacy(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testRejectVotesFromSameEpochAfterResigningCandidacy(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int remoteId = localId + 1; - ReplicaKey remoteKey = replicaKey(remoteId, withKip853Rpc); + ReplicaKey remoteKey = replicaKey(remoteId, isSupport853(raftProtocol)); Set voters = Set.of(localId, remoteKey.id()); int epoch = 2; - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .updateRandom(r -> r.mockNextInt(DEFAULT_ELECTION_TIMEOUT_MS, 0)) .withVotedCandidate(epoch, ReplicaKey.of(localId, ReplicaKey.NO_DIRECTORY_ID)) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); assertEquals(0L, context.log.endOffset().offset()); @@ -187,18 +196,18 @@ public void testRejectVotesFromSameEpochAfterResigningCandidacy(boolean withKip8 } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testGrantVotesFromHigherEpochAfterResigningLeadership(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testGrantVotesFromHigherEpochAfterResigningLeadership(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int remoteId = localId + 1; - ReplicaKey remoteKey = replicaKey(remoteId, withKip853Rpc); + ReplicaKey remoteKey = replicaKey(remoteId, isSupport853(raftProtocol)); Set voters = Set.of(localId, remoteKey.id()); int epoch = 2; - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .updateRandom(r -> r.mockNextInt(DEFAULT_ELECTION_TIMEOUT_MS, 0)) .withElectedLeader(epoch, localId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); // Resign from leader, will restart in resigned state @@ -224,18 +233,18 @@ public void testGrantVotesFromHigherEpochAfterResigningLeadership(boolean withKi } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testGrantVotesFromHigherEpochAfterResigningCandidacy(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testGrantVotesFromHigherEpochAfterResigningCandidacy(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int remoteId = localId + 1; - ReplicaKey remoteKey = replicaKey(remoteId, withKip853Rpc); + ReplicaKey remoteKey = replicaKey(remoteId, isSupport853(raftProtocol)); Set voters = Set.of(localId, remoteKey.id()); int epoch = 2; - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .updateRandom(r -> r.mockNextInt(DEFAULT_ELECTION_TIMEOUT_MS, 0)) .withVotedCandidate(epoch, ReplicaKey.of(localId, ReplicaKey.NO_DIRECTORY_ID)) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); // Resign from candidate, will restart in candidate state @@ -261,15 +270,15 @@ public void testGrantVotesFromHigherEpochAfterResigningCandidacy(boolean withKip } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testGrantVotesWhenShuttingDown(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testGrantVotesWhenShuttingDown(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int remoteId = localId + 1; - ReplicaKey remoteKey = replicaKey(remoteId, withKip853Rpc); + ReplicaKey remoteKey = replicaKey(remoteId, isSupport853(raftProtocol)); Set voters = Set.of(localId, remoteKey.id()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withKip853Rpc(withKip853Rpc) + RaftClientTestContext context = new Builder(localId, voters) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -303,17 +312,17 @@ public void testGrantVotesWhenShuttingDown(boolean withKip853Rpc) throws Excepti } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testInitializeAsResignedAndUnableToContactQuorum(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testInitializeAsResignedAndUnableToContactQuorum(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int remoteId = localId + 1; Set voters = Set.of(localId, remoteId); int epoch = 2; - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .updateRandom(r -> r.mockNextInt(DEFAULT_ELECTION_TIMEOUT_MS, 0)) .withElectedLeader(epoch, localId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); // Resign from leader, will restart in resigned state @@ -341,16 +350,16 @@ public void testInitializeAsResignedAndUnableToContactQuorum(boolean withKip853R } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testInitializeAsResignedLeaderFromStateStore(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testInitializeAsResignedLeaderFromStateStore(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int remoteId = localId + 1; Set voters = Set.of(localId, remoteId); int epoch = 2; - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .updateRandom(r -> r.mockNextInt(DEFAULT_ELECTION_TIMEOUT_MS, 0)) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .withElectedLeader(epoch, localId) .build(); @@ -380,24 +389,24 @@ public void testInitializeAsResignedLeaderFromStateStore(boolean withKip853Rpc) } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testAppendFailedWithNotLeaderException(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testAppendFailedWithNotLeaderException(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int remoteId = localId + 1; Set voters = Set.of(localId, remoteId); int epoch = 2; - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withUnknownLeader(epoch) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); assertThrows(NotLeaderException.class, () -> context.client.prepareAppend(epoch, Arrays.asList("a", "b"))); } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testAppendFailedWithBufferAllocationException(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testAppendFailedWithBufferAllocationException(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; Set voters = Set.of(localId, otherNodeId); @@ -409,9 +418,9 @@ public void testAppendFailedWithBufferAllocationException(boolean withKip853Rpc) .thenReturn(buffer) // Buffer for the leader message control record .thenReturn(null); // Buffer for the prepareAppend call - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withMemoryPool(memoryPool) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -423,14 +432,14 @@ public void testAppendFailedWithBufferAllocationException(boolean withKip853Rpc) } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testAppendFailedWithFencedEpoch(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testAppendFailedWithFencedEpoch(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; Set voters = Set.of(localId, otherNodeId); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withKip853Rpc(withKip853Rpc) + RaftClientTestContext context = new Builder(localId, voters) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -444,14 +453,14 @@ public void testAppendFailedWithFencedEpoch(boolean withKip853Rpc) throws Except } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testAppendFailedWithRecordBatchTooLargeException(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testAppendFailedWithRecordBatchTooLargeException(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; Set voters = Set.of(localId, otherNodeId); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withKip853Rpc(withKip853Rpc) + RaftClientTestContext context = new Builder(localId, voters) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -470,8 +479,8 @@ public void testAppendFailedWithRecordBatchTooLargeException(boolean withKip853R } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testEndQuorumEpochRetriesWhileResigned(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testEndQuorumEpochRetriesWhileResigned(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int voter1 = localId + 1; int voter2 = localId + 2; @@ -482,11 +491,11 @@ public void testEndQuorumEpochRetriesWhileResigned(boolean withKip853Rpc) throws // Note that we intentionally set a request timeout which is smaller than // the election timeout so that we can still in the Resigned state and // verify retry behavior. - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withElectionTimeoutMs(10000) .withRequestTimeoutMs(5000) .withElectedLeader(epoch, localId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.pollUntilRequest(); @@ -516,15 +525,15 @@ public void testEndQuorumEpochRetriesWhileResigned(boolean withKip853Rpc) throws } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testResignWillCompleteFetchPurgatory(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testResignWillCompleteFetchPurgatory(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int remoteId = localId + 1; - ReplicaKey otherNodeKey = replicaKey(remoteId, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(remoteId, isSupport853(raftProtocol)); Set voters = Set.of(localId, otherNodeKey.id()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withKip853Rpc(withKip853Rpc) + RaftClientTestContext context = new Builder(localId, voters) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -555,14 +564,14 @@ public void testResignWillCompleteFetchPurgatory(boolean withKip853Rpc) throws E } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testResignInOlderEpochIgnored(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testResignInOlderEpochIgnored(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; Set voters = Set.of(localId, otherNodeId); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withKip853Rpc(withKip853Rpc) + RaftClientTestContext context = new Builder(localId, voters) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -579,17 +588,17 @@ public void testResignInOlderEpochIgnored(boolean withKip853Rpc) throws Exceptio } @ParameterizedTest - @ValueSource(booleans = { true, false }) + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) public void testHandleBeginQuorumEpochAfterUserInitiatedResign( - boolean withKip853Rpc + RaftProtocol raftProtocol ) throws Exception { int localId = randomReplicaId(); int remoteId1 = localId + 1; int remoteId2 = localId + 2; Set voters = Set.of(localId, remoteId1, remoteId2); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withKip853Rpc(withKip853Rpc) + RaftClientTestContext context = new Builder(localId, voters) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -609,15 +618,15 @@ public void testHandleBeginQuorumEpochAfterUserInitiatedResign( } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testBeginQuorumEpochHeartbeat(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testBeginQuorumEpochHeartbeat(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int remoteId1 = localId + 1; int remoteId2 = localId + 2; Set voters = Set.of(localId, remoteId1, remoteId2); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withKip853Rpc(withKip853Rpc) + RaftClientTestContext context = new Builder(localId, voters) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -640,19 +649,19 @@ public void testBeginQuorumEpochHeartbeat(boolean withKip853Rpc) throws Exceptio } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testLeaderShouldResignLeadershipIfNotGetFetchRequestFromMajorityVoters(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testLeaderShouldResignLeadershipIfNotGetFetchRequestFromMajorityVoters(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int remoteId1 = localId + 1; int remoteId2 = localId + 2; int observerId = localId + 3; - ReplicaKey remoteKey1 = replicaKey(remoteId1, withKip853Rpc); - ReplicaKey remoteKey2 = replicaKey(remoteId2, withKip853Rpc); - ReplicaKey observerKey3 = replicaKey(observerId, withKip853Rpc); + ReplicaKey remoteKey1 = replicaKey(remoteId1, isSupport853(raftProtocol)); + ReplicaKey remoteKey2 = replicaKey(remoteId2, isSupport853(raftProtocol)); + ReplicaKey observerKey3 = replicaKey(observerId, isSupport853(raftProtocol)); Set voters = Set.of(localId, remoteKey1.id(), remoteKey2.id()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withKip853Rpc(withKip853Rpc) + RaftClientTestContext context = new Builder(localId, voters) + .withRaftProtocol(raftProtocol) .build(); int resignLeadershipTimeout = context.checkQuorumTimeoutMs; @@ -700,13 +709,13 @@ public void testLeaderShouldResignLeadershipIfNotGetFetchRequestFromMajorityVote } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testLeaderShouldNotResignLeadershipIfOnlyOneVoters(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testLeaderShouldNotResignLeadershipIfOnlyOneVoters(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); Set voters = Set.of(localId); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withKip853Rpc(withKip853Rpc) + RaftClientTestContext context = new Builder(localId, voters) + .withRaftProtocol(raftProtocol) .build(); assertEquals(OptionalInt.of(localId), context.currentLeader()); @@ -719,14 +728,14 @@ public void testLeaderShouldNotResignLeadershipIfOnlyOneVoters(boolean withKip85 } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testElectionTimeoutAfterUserInitiatedResign(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testElectionTimeoutAfterUserInitiatedResign(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; Set voters = Set.of(localId, otherNodeId); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withKip853Rpc(withKip853Rpc) + RaftClientTestContext context = new Builder(localId, voters) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -754,7 +763,7 @@ public void testElectionTimeoutAfterUserInitiatedResign(boolean withKip853Rpc) t assertFalse(context.channel.hasSentRequests()); // Any `Fetch` received in the resigned state should result in a NOT_LEADER error. - ReplicaKey observer = replicaKey(-1, withKip853Rpc); + ReplicaKey observer = replicaKey(-1, isSupport853(raftProtocol)); context.deliverRequest(context.fetchRequest(1, observer, 0, 0, 0)); context.pollUntilResponse(); context.assertSentFetchPartitionResponse( @@ -777,14 +786,14 @@ public void testElectionTimeoutAfterUserInitiatedResign(boolean withKip853Rpc) t } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testCannotResignWithLargerEpochThanCurrentEpoch(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testCannotResignWithLargerEpochThanCurrentEpoch(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; Set voters = Set.of(localId, otherNodeId); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withKip853Rpc(withKip853Rpc) + RaftClientTestContext context = new Builder(localId, voters) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -793,16 +802,16 @@ public void testCannotResignWithLargerEpochThanCurrentEpoch(boolean withKip853Rp } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testCannotResignIfNotLeader(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testCannotResignIfNotLeader(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int leaderEpoch = 2; Set voters = Set.of(localId, otherNodeId); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withElectedLeader(leaderEpoch, otherNodeId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); assertEquals(OptionalInt.of(otherNodeId), context.currentLeader()); @@ -810,15 +819,15 @@ public void testCannotResignIfNotLeader(boolean withKip853Rpc) throws Exception } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testCannotResignIfObserver(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testCannotResignIfObserver(RaftProtocol raftProtocol) throws Exception { int leaderId = randomReplicaId(); int otherNodeId = randomReplicaId() + 1; int epoch = 5; Set voters = Set.of(leaderId, otherNodeId); - RaftClientTestContext context = new RaftClientTestContext.Builder(OptionalInt.empty(), voters) - .withKip853Rpc(withKip853Rpc) + RaftClientTestContext context = new Builder(OptionalInt.empty(), voters) + .withRaftProtocol(raftProtocol) .build(); context.pollUntilRequest(); @@ -838,15 +847,15 @@ public void testCannotResignIfObserver(boolean withKip853Rpc) throws Exception { } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testInitializeAsCandidateFromStateStore(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testInitializeAsCandidateFromStateStore(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); // Need 3 node to require a 2-node majority Set voters = Set.of(localId, localId + 1, localId + 2); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withVotedCandidate(2, ReplicaKey.of(localId, ReplicaKey.NO_DIRECTORY_ID)) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.assertVotedCandidate(2, localId); assertEquals(0L, context.log.endOffset().offset()); @@ -858,13 +867,13 @@ public void testInitializeAsCandidateFromStateStore(boolean withKip853Rpc) throw } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testInitializeAsUnattachedAndBecomeLeader(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testInitializeAsUnattachedAndBecomeLeader(RaftProtocol raftProtocol) throws Exception { final int localId = randomReplicaId(); final int otherNodeId = localId + 1; Set voters = Set.of(localId, otherNodeId); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withKip853Rpc(withKip853Rpc) + RaftClientTestContext context = new Builder(localId, voters) + .withRaftProtocol(raftProtocol) .build(); context.assertUnknownLeaderAndNoVotedCandidate(0); @@ -922,20 +931,20 @@ public void testInitializeAsUnattachedAndBecomeLeader(boolean withKip853Rpc) thr Record record = batch.iterator().next(); assertEquals(electionTimestamp, record.timestamp()); - RaftClientTestContext.verifyLeaderChangeMessage(localId, Arrays.asList(localId, otherNodeId), + verifyLeaderChangeMessage(localId, Arrays.asList(localId, otherNodeId), Arrays.asList(otherNodeId, localId), record.key(), record.value()); } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testInitializeAsCandidateAndBecomeLeaderQuorumOfThree(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testInitializeAsCandidateAndBecomeLeaderQuorumOfThree(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); final int firstNodeId = localId + 1; final int secondNodeId = localId + 2; Set voters = Set.of(localId, firstNodeId, secondNodeId); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withVotedCandidate(2, ReplicaKey.of(localId, ReplicaKey.NO_DIRECTORY_ID)) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); assertTrue(context.client.quorum().isCandidate()); context.pollUntilRequest(); @@ -971,16 +980,16 @@ public void testInitializeAsCandidateAndBecomeLeaderQuorumOfThree(boolean withKi Record record = batch.iterator().next(); assertEquals(electionTimestamp, record.timestamp()); - RaftClientTestContext.verifyLeaderChangeMessage(localId, Arrays.asList(localId, firstNodeId, secondNodeId), + verifyLeaderChangeMessage(localId, Arrays.asList(localId, firstNodeId, secondNodeId), Arrays.asList(voterId, localId), record.key(), record.value()); } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testInitializeAsOnlyVoterWithEmptyElectionState(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testInitializeAsOnlyVoterWithEmptyElectionState(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, Set.of(localId)) - .withKip853Rpc(withKip853Rpc) + RaftClientTestContext context = new Builder(localId, Set.of(localId)) + .withRaftProtocol(raftProtocol) .build(); context.assertElectedLeader(1, localId); assertEquals(0L, context.log.endOffset().offset()); @@ -990,7 +999,7 @@ public void testInitializeAsOnlyVoterWithEmptyElectionState(boolean withKip853Rp @Test public void testInitializeAsFollowerAndOnlyVoter() throws Exception { int localId = randomReplicaId(); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, Set.of(localId)) + RaftClientTestContext context = new Builder(localId, Set.of(localId)) .withRaftProtocol(KIP_853_PROTOCOL) .withElectedLeader(2, localId + 1) .build(); @@ -1002,7 +1011,7 @@ public void testInitializeAsFollowerAndOnlyVoter() throws Exception { @Test public void testInitializeAsCandidateAndOnlyVoter() throws Exception { int localId = randomReplicaId(); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, Set.of(localId)) + RaftClientTestContext context = new Builder(localId, Set.of(localId)) .withRaftProtocol(KIP_853_PROTOCOL) .withVotedCandidate(2, ReplicaKey.of(localId, ReplicaKey.NO_DIRECTORY_ID)) .build(); @@ -1013,7 +1022,7 @@ public void testInitializeAsCandidateAndOnlyVoter() throws Exception { @Test public void testInitializeAsResignedAndOnlyVoter() throws Exception { int localId = randomReplicaId(); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, Set.of(localId)) + RaftClientTestContext context = new Builder(localId, Set.of(localId)) .withRaftProtocol(KIP_853_PROTOCOL) .withElectedLeader(2, localId) .build(); @@ -1022,16 +1031,16 @@ public void testInitializeAsResignedAndOnlyVoter() throws Exception { } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testHandleBeginQuorumRequest(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testHandleBeginQuorumRequest(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, isSupport853(raftProtocol)); int votedCandidateEpoch = 2; Set voters = Set.of(localId, otherNodeKey.id()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withVotedCandidate(votedCandidateEpoch, otherNodeKey) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.deliverRequest(context.beginEpochRequest(votedCandidateEpoch, otherNodeKey.id())); @@ -1054,10 +1063,10 @@ public void testHandleBeginQuorumRequestMoreEndpoints() throws Exception { VoterSet voters = VoterSetTest.voterSet(Stream.of(local, leader)); - RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) + RaftClientTestContext context = new Builder(local.id(), local.directoryId().get()) .withStaticVoters(voters) .withElectedLeader(leaderEpoch, leader.id()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .build(); context.client.poll(); @@ -1086,16 +1095,16 @@ public void testHandleBeginQuorumRequestMoreEndpoints() throws Exception { } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testHandleBeginQuorumResponse(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testHandleBeginQuorumResponse(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int leaderEpoch = 2; Set voters = Set.of(localId, otherNodeId); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withElectedLeader(leaderEpoch, localId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.deliverRequest(context.beginEpochRequest(leaderEpoch + 1, otherNodeId)); @@ -1105,18 +1114,18 @@ public void testHandleBeginQuorumResponse(boolean withKip853Rpc) throws Exceptio } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testEndQuorumIgnoredAsCandidateIfOlderEpoch(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testEndQuorumIgnoredAsCandidateIfOlderEpoch(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int epoch = 5; int jitterMs = 85; Set voters = Set.of(localId, otherNodeId); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .updateRandom(r -> r.mockNextInt(jitterMs)) .withUnknownLeader(epoch - 1) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToCandidate(); @@ -1150,14 +1159,14 @@ public void testEndQuorumIgnoredAsCandidateIfOlderEpoch(boolean withKip853Rpc) t } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testEndQuorumIgnoredAsLeaderIfOlderEpoch(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testEndQuorumIgnoredAsLeaderIfOlderEpoch(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int voter2 = localId + 1; - ReplicaKey voter3 = replicaKey(localId + 2, withKip853Rpc); + ReplicaKey voter3 = replicaKey(localId + 2, isSupport853(raftProtocol)); Set voters = Set.of(localId, voter2, voter3.id()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withUnknownLeader(6) .build(); @@ -1179,19 +1188,19 @@ public void testEndQuorumIgnoredAsLeaderIfOlderEpoch(boolean withKip853Rpc) thro } @ParameterizedTest - @ValueSource(booleans = { true, false }) + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) public void testEndQuorumStartsNewElectionImmediatelyIfFollowerUnattached( - boolean withKip853Rpc + RaftProtocol raftProtocol ) throws Exception { int localId = randomReplicaId(); int voter2 = localId + 1; - ReplicaKey voter3 = replicaKey(localId + 2, withKip853Rpc); + ReplicaKey voter3 = replicaKey(localId + 2, isSupport853(raftProtocol)); int epoch = 2; Set voters = Set.of(localId, voter2, voter3.id()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withUnknownLeader(epoch) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.deliverRequest( @@ -1211,8 +1220,8 @@ public void testEndQuorumStartsNewElectionImmediatelyIfFollowerUnattached( } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testAccumulatorClearedAfterBecomingFollower(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testAccumulatorClearedAfterBecomingFollower(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int lingerMs = 50; @@ -1223,10 +1232,10 @@ public void testAccumulatorClearedAfterBecomingFollower(boolean withKip853Rpc) t Mockito.when(memoryPool.tryAllocate(KafkaRaftClient.MAX_BATCH_SIZE_BYTES)) .thenReturn(buffer); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withAppendLingerMs(lingerMs) .withMemoryPool(memoryPool) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -1244,10 +1253,10 @@ public void testAccumulatorClearedAfterBecomingFollower(boolean withKip853Rpc) t } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testAccumulatorClearedAfterBecomingVoted(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testAccumulatorClearedAfterBecomingVoted(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, isSupport853(raftProtocol)); int lingerMs = 50; Set voters = Set.of(localId, otherNodeKey.id()); @@ -1256,10 +1265,10 @@ public void testAccumulatorClearedAfterBecomingVoted(boolean withKip853Rpc) thro Mockito.when(memoryPool.tryAllocate(KafkaRaftClient.MAX_BATCH_SIZE_BYTES)) .thenReturn(buffer); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withAppendLingerMs(lingerMs) .withMemoryPool(memoryPool) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -1278,10 +1287,10 @@ public void testAccumulatorClearedAfterBecomingVoted(boolean withKip853Rpc) thro } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testAccumulatorClearedAfterBecomingUnattached(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testAccumulatorClearedAfterBecomingUnattached(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, isSupport853(raftProtocol)); int lingerMs = 50; Set voters = Set.of(localId, otherNodeKey.id()); @@ -1290,10 +1299,10 @@ public void testAccumulatorClearedAfterBecomingUnattached(boolean withKip853Rpc) Mockito.when(memoryPool.tryAllocate(KafkaRaftClient.MAX_BATCH_SIZE_BYTES)) .thenReturn(buffer); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withAppendLingerMs(lingerMs) .withMemoryPool(memoryPool) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -1311,8 +1320,8 @@ public void testAccumulatorClearedAfterBecomingUnattached(boolean withKip853Rpc) } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testChannelWokenUpIfLingerTimeoutReachedWithoutAppend(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testChannelWokenUpIfLingerTimeoutReachedWithoutAppend(RaftProtocol raftProtocol) throws Exception { // This test verifies that the client will set its poll timeout accounting // for the lingerMs of a pending append int localId = randomReplicaId(); @@ -1320,9 +1329,9 @@ public void testChannelWokenUpIfLingerTimeoutReachedWithoutAppend(boolean withKi int lingerMs = 50; Set voters = Set.of(localId, otherNodeId); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withAppendLingerMs(lingerMs) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -1347,8 +1356,8 @@ public void testChannelWokenUpIfLingerTimeoutReachedWithoutAppend(boolean withKi } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testChannelWokenUpIfLingerTimeoutReachedDuringAppend(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testChannelWokenUpIfLingerTimeoutReachedDuringAppend(RaftProtocol raftProtocol) throws Exception { // This test verifies that the client will get woken up immediately // if the linger timeout has expired during an append int localId = randomReplicaId(); @@ -1356,9 +1365,9 @@ public void testChannelWokenUpIfLingerTimeoutReachedDuringAppend(boolean withKip int lingerMs = 50; Set voters = Set.of(localId, otherNodeId); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withAppendLingerMs(lingerMs) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -1384,16 +1393,16 @@ public void testChannelWokenUpIfLingerTimeoutReachedDuringAppend(boolean withKip } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testHandleEndQuorumRequest(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testHandleEndQuorumRequest(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int oldLeaderId = localId + 1; int leaderEpoch = 2; Set voters = Set.of(localId, oldLeaderId); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withElectedLeader(leaderEpoch, oldLeaderId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.deliverRequest( @@ -1413,17 +1422,17 @@ public void testHandleEndQuorumRequest(boolean withKip853Rpc) throws Exception { } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testHandleEndQuorumRequestWithLowerPriorityToBecomeLeader(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testHandleEndQuorumRequestWithLowerPriorityToBecomeLeader(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey oldLeaderKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey oldLeaderKey = replicaKey(localId + 1, isSupport853(raftProtocol)); int leaderEpoch = 2; - ReplicaKey preferredNextLeader = replicaKey(localId + 2, withKip853Rpc); + ReplicaKey preferredNextLeader = replicaKey(localId + 2, isSupport853(raftProtocol)); Set voters = Set.of(localId, oldLeaderKey.id(), preferredNextLeader.id()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withElectedLeader(leaderEpoch, oldLeaderKey.id()) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.deliverRequest( @@ -1456,15 +1465,15 @@ public void testHandleEndQuorumRequestWithLowerPriorityToBecomeLeader(boolean wi } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testVoteRequestTimeout(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testVoteRequestTimeout(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int epoch = 1; Set voters = Set.of(localId, otherNodeId); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withKip853Rpc(withKip853Rpc) + RaftClientTestContext context = new Builder(localId, voters) + .withRaftProtocol(raftProtocol) .build(); context.assertUnknownLeaderAndNoVotedCandidate(0); @@ -1498,16 +1507,16 @@ public void testVoteRequestTimeout(boolean withKip853Rpc) throws Exception { } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testHandleValidVoteRequestAsFollower(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testHandleValidVoteRequestAsFollower(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int epoch = 2; - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, isSupport853(raftProtocol)); Set voters = Set.of(localId, otherNodeKey.id()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withUnknownLeader(epoch) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.deliverRequest(context.voteRequest(epoch, otherNodeKey, epoch - 1, 1)); @@ -1519,17 +1528,17 @@ public void testHandleValidVoteRequestAsFollower(boolean withKip853Rpc) throws E } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testHandleVoteRequestAsFollowerWithElectedLeader(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testHandleVoteRequestAsFollowerWithElectedLeader(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int epoch = 2; - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, isSupport853(raftProtocol)); int electedLeaderId = localId + 2; Set voters = Set.of(localId, otherNodeKey.id(), electedLeaderId); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withElectedLeader(epoch, electedLeaderId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.deliverRequest(context.voteRequest(epoch, otherNodeKey, epoch - 1, 1)); @@ -1541,17 +1550,17 @@ public void testHandleVoteRequestAsFollowerWithElectedLeader(boolean withKip853R } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testHandleVoteRequestAsFollowerWithVotedCandidate(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testHandleVoteRequestAsFollowerWithVotedCandidate(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int epoch = 2; - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); - ReplicaKey votedCandidateKey = replicaKey(localId + 2, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, isSupport853(raftProtocol)); + ReplicaKey votedCandidateKey = replicaKey(localId + 2, isSupport853(raftProtocol)); Set voters = Set.of(localId, otherNodeKey.id(), votedCandidateKey.id()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withVotedCandidate(epoch, votedCandidateKey) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.deliverRequest(context.voteRequest(epoch, otherNodeKey, epoch - 1, 1)); @@ -1562,17 +1571,17 @@ public void testHandleVoteRequestAsFollowerWithVotedCandidate(boolean withKip853 } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testHandleVoteRequestAsProspective(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testHandleVoteRequestAsProspective(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int epoch = 2; - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, isSupport853(raftProtocol)); int electedLeaderId = localId + 2; Set voters = Set.of(localId, otherNodeKey.id(), electedLeaderId); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withElectedLeader(epoch, electedLeaderId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); // Sleep a little to ensure that we become a prospective @@ -1590,17 +1599,17 @@ public void testHandleVoteRequestAsProspective(boolean withKip853Rpc) throws Exc } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testHandleVoteRequestAsProspectiveWithVotedCandidate(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testHandleVoteRequestAsProspectiveWithVotedCandidate(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int epoch = 2; - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); - ReplicaKey votedCandidateKey = replicaKey(localId + 2, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, isSupport853(raftProtocol)); + ReplicaKey votedCandidateKey = replicaKey(localId + 2, isSupport853(raftProtocol)); Set voters = Set.of(localId, otherNodeKey.id(), votedCandidateKey.id()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withVotedCandidate(epoch, votedCandidateKey) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); // Sleep a little to ensure that we become a prospective @@ -1617,16 +1626,16 @@ public void testHandleVoteRequestAsProspectiveWithVotedCandidate(boolean withKip } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testHandleInvalidVoteRequestWithOlderEpoch(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testHandleInvalidVoteRequestWithOlderEpoch(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int epoch = 2; - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, isSupport853(raftProtocol)); Set voters = Set.of(localId, otherNodeKey.id()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withUnknownLeader(epoch) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.deliverRequest(context.voteRequest(epoch - 1, otherNodeKey, epoch - 2, 1)); @@ -1637,17 +1646,17 @@ public void testHandleInvalidVoteRequestWithOlderEpoch(boolean withKip853Rpc) th } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testHandleVoteRequestAsObserver(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testHandleVoteRequestAsObserver(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int epoch = 2; - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, isSupport853(raftProtocol)); int otherNodeId2 = localId + 2; Set voters = Set.of(otherNodeKey.id(), otherNodeId2); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withUnknownLeader(epoch) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.deliverRequest(context.voteRequest(epoch + 1, otherNodeKey, epoch, 1)); @@ -1658,15 +1667,15 @@ public void testHandleVoteRequestAsObserver(boolean withKip853Rpc) throws Except } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testLeaderIgnoreVoteRequestOnSameEpoch(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testLeaderIgnoreVoteRequestOnSameEpoch(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, isSupport853(raftProtocol)); Set voters = Set.of(localId, otherNodeKey.id()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withUnknownLeader(2) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -1681,15 +1690,15 @@ public void testLeaderIgnoreVoteRequestOnSameEpoch(boolean withKip853Rpc) throws } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testListenerCommitCallbackAfterLeaderWrite(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testListenerCommitCallbackAfterLeaderWrite(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, isSupport853(raftProtocol)); Set voters = Set.of(localId, otherNodeKey.id()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withUnknownLeader(4) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -1727,15 +1736,15 @@ public void testListenerCommitCallbackAfterLeaderWrite(boolean withKip853Rpc) th } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testLeaderImmediatelySendsDivergingEpoch(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testLeaderImmediatelySendsDivergingEpoch(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, isSupport853(raftProtocol)); Set voters = Set.of(localId, otherNodeKey.id()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withUnknownLeader(5) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .appendToLog(1, Arrays.asList("a", "b", "c")) .appendToLog(3, Arrays.asList("d", "e", "f")) .appendToLog(5, Arrays.asList("g", "h", "i")) @@ -1759,16 +1768,16 @@ public void testLeaderImmediatelySendsDivergingEpoch(boolean withKip853Rpc) thro } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testCandidateIgnoreVoteRequestOnSameEpoch(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testCandidateIgnoreVoteRequestOnSameEpoch(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, isSupport853(raftProtocol)); int leaderEpoch = 2; Set voters = Set.of(localId, otherNodeKey.id()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withVotedCandidate(leaderEpoch, ReplicaKey.of(localId, ReplicaKey.NO_DIRECTORY_ID)) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.pollUntilRequest(); @@ -1780,17 +1789,17 @@ public void testCandidateIgnoreVoteRequestOnSameEpoch(boolean withKip853Rpc) thr } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testCandidateBackoffElection(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testCandidateBackoffElection(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int epoch = 1; int exponentialFactor = 85; // set it large enough so that replica will bound on jitter Set voters = Set.of(localId, otherNodeId); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .updateRandom(r -> r.mockNextInt(exponentialFactor)) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.assertUnknownLeaderAndNoVotedCandidate(0); @@ -1856,17 +1865,17 @@ public void testCandidateBackoffElection(boolean withKip853Rpc) throws Exception } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testCandidateElectionTimeout(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testCandidateElectionTimeout(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int epoch = 1; int jitter = 100; Set voters = Set.of(localId, otherNodeId); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .updateRandom(r -> r.mockNextInt(jitter)) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.assertUnknownLeaderAndNoVotedCandidate(0); @@ -1899,16 +1908,16 @@ public void testCandidateElectionTimeout(boolean withKip853Rpc) throws Exception } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testInitializeAsFollowerEmptyLog(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testInitializeAsFollowerEmptyLog(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int epoch = 5; Set voters = Set.of(localId, otherNodeId); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withElectedLeader(epoch, otherNodeId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.assertElectedLeader(epoch, otherNodeId); @@ -1919,18 +1928,18 @@ public void testInitializeAsFollowerEmptyLog(boolean withKip853Rpc) throws Excep } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testInitializeAsFollowerNonEmptyLog(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testInitializeAsFollowerNonEmptyLog(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int epoch = 5; int lastEpoch = 3; Set voters = Set.of(localId, otherNodeId); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withElectedLeader(epoch, otherNodeId) .appendToLog(lastEpoch, singletonList("foo")) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.assertElectedLeader(epoch, otherNodeId); @@ -1940,18 +1949,18 @@ public void testInitializeAsFollowerNonEmptyLog(boolean withKip853Rpc) throws Ex } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testVoterBecomeProspectiveAfterFetchTimeout(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testVoterBecomeProspectiveAfterFetchTimeout(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int epoch = 5; int lastEpoch = 3; Set voters = Set.of(localId, otherNodeId); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withElectedLeader(epoch, otherNodeId) .appendToLog(lastEpoch, singletonList("foo")) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.assertElectedLeader(epoch, otherNodeId); @@ -1966,18 +1975,18 @@ public void testVoterBecomeProspectiveAfterFetchTimeout(boolean withKip853Rpc) t } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testFollowerAsObserverDoesNotBecomeProspectiveAfterFetchTimeout(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testFollowerAsObserverDoesNotBecomeProspectiveAfterFetchTimeout(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int epoch = 5; int lastEpoch = 3; Set voters = Set.of(otherNodeId); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withElectedLeader(epoch, otherNodeId) .appendToLog(lastEpoch, singletonList("foo")) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.assertElectedLeader(epoch, otherNodeId); @@ -1989,23 +1998,23 @@ public void testFollowerAsObserverDoesNotBecomeProspectiveAfterFetchTimeout(bool assertTrue(context.client.quorum().isFollower()); // transitions to unattached - context.deliverRequest(context.voteRequest(epoch + 1, replicaKey(otherNodeId, withKip853Rpc), epoch, 1)); + context.deliverRequest(context.voteRequest(epoch + 1, replicaKey(otherNodeId, isSupport853(raftProtocol)), epoch, 1)); context.pollUntilResponse(); context.assertSentVoteResponse(Errors.NONE, epoch + 1, OptionalInt.empty(), true); assertTrue(context.client.quorum().isUnattached()); } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testUnattachedAsObserverDoesNotBecomeProspectiveAfterElectionTimeout(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testUnattachedAsObserverDoesNotBecomeProspectiveAfterElectionTimeout(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int epoch = 5; Set voters = Set.of(otherNodeId); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withUnknownLeader(epoch) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.pollUntilRequest(); @@ -2019,7 +2028,7 @@ public void testUnattachedAsObserverDoesNotBecomeProspectiveAfterElectionTimeout // confirm no vote request was sent assertEquals(0, context.channel.drainSendQueue().size()); - context.deliverRequest(context.voteRequest(epoch + 1, replicaKey(otherNodeId, withKip853Rpc), epoch, 0)); + context.deliverRequest(context.voteRequest(epoch + 1, replicaKey(otherNodeId, isSupport853(raftProtocol)), epoch, 0)); context.pollUntilResponse(); // observer can vote context.assertSentVoteResponse(Errors.NONE, epoch + 1, OptionalInt.empty(), true); @@ -2033,17 +2042,17 @@ public void testUnattachedAsObserverDoesNotBecomeProspectiveAfterElectionTimeout } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testUnattachedAsVoterCanBecomeFollowerAfterFindingLeader(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testUnattachedAsVoterCanBecomeFollowerAfterFindingLeader(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int leaderNodeId = localId + 2; int epoch = 5; Set voters = Set.of(localId, otherNodeId, leaderNodeId); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withUnknownLeader(epoch) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.pollUntilRequest(); @@ -2064,16 +2073,16 @@ public void testUnattachedAsVoterCanBecomeFollowerAfterFindingLeader(boolean wit } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testInitializeObserverNoPreviousState(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testInitializeObserverNoPreviousState(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int leaderId = localId + 1; int otherNodeId = localId + 2; int epoch = 5; Set voters = Set.of(leaderId, otherNodeId); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withKip853Rpc(withKip853Rpc) + RaftClientTestContext context = new Builder(localId, voters) + .withRaftProtocol(raftProtocol) .build(); context.pollUntilRequest(); @@ -2092,8 +2101,8 @@ public void testInitializeObserverNoPreviousState(boolean withKip853Rpc) throws } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testObserverQuorumDiscoveryFailure(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testObserverQuorumDiscoveryFailure(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int leaderId = localId + 1; int epoch = 5; @@ -2103,9 +2112,9 @@ public void testObserverQuorumDiscoveryFailure(boolean withKip853Rpc) throws Exc .map(RaftClientTestContext::mockAddress) .collect(Collectors.toList()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withBootstrapServers(Optional.of(bootstrapServers)) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.pollUntilRequest(); @@ -2138,8 +2147,8 @@ public void testObserverQuorumDiscoveryFailure(boolean withKip853Rpc) throws Exc } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testObserverSendDiscoveryFetchAfterFetchTimeout(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testObserverSendDiscoveryFetchAfterFetchTimeout(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int leaderId = localId + 1; int otherNodeId = localId + 2; @@ -2150,9 +2159,9 @@ public void testObserverSendDiscoveryFetchAfterFetchTimeout(boolean withKip853Rp .map(RaftClientTestContext::mockAddress) .collect(Collectors.toList()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withBootstrapServers(Optional.of(bootstrapServers)) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.pollUntilRequest(); @@ -2179,8 +2188,8 @@ public void testObserverSendDiscoveryFetchAfterFetchTimeout(boolean withKip853Rp } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testObserverHandleRetryFetchToBootstrapServer(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testObserverHandleRetryFetchToBootstrapServer(RaftProtocol raftProtocol) throws Exception { // This test tries to check that KRaft is able to handle a retrying Fetch request to // a boostrap server after a Fetch request to the leader. int localId = randomReplicaId(); @@ -2193,9 +2202,9 @@ public void testObserverHandleRetryFetchToBootstrapServer(boolean withKip853Rpc) .map(RaftClientTestContext::mockAddress) .collect(Collectors.toList()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withBootstrapServers(Optional.of(bootstrapServers)) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); // Expect a fetch request to one of the bootstrap servers @@ -2253,8 +2262,8 @@ public void testObserverHandleRetryFetchToBootstrapServer(boolean withKip853Rpc) } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testObserverHandleRetryFetchToLeader(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testObserverHandleRetryFetchToLeader(RaftProtocol raftProtocol) throws Exception { // This test tries to check that KRaft is able to handle a retrying Fetch request to // the leader after a Fetch request to the bootstrap server. int localId = randomReplicaId(); @@ -2267,9 +2276,9 @@ public void testObserverHandleRetryFetchToLeader(boolean withKip853Rpc) throws E .map(RaftClientTestContext::mockAddress) .collect(Collectors.toList()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withBootstrapServers(Optional.of(bootstrapServers)) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); // Expect a fetch request to one of the bootstrap servers @@ -2312,15 +2321,15 @@ public void testObserverHandleRetryFetchToLeader(boolean withKip853Rpc) throws E } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testInvalidFetchRequest(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testInvalidFetchRequest(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, isSupport853(raftProtocol)); Set voters = Set.of(localId, otherNodeKey.id()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withUnknownLeader(4) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -2365,7 +2374,7 @@ public void testLeaderStateUpdateWithDifferentFetchRequestVersions(short version int epoch = 5; Set voters = Set.of(localId, otherNodeKey.id()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withUnknownLeader(epoch - 1) .build(); context.assertUnknownLeaderAndNoVotedCandidate(epoch - 1); @@ -2388,15 +2397,15 @@ public void testLeaderStateUpdateWithDifferentFetchRequestVersions(short version } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testFetchRequestClusterIdValidation(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testFetchRequestClusterIdValidation(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, isSupport853(raftProtocol)); Set voters = Set.of(localId, otherNodeKey.id()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withUnknownLeader(4) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -2424,14 +2433,14 @@ public void testFetchRequestClusterIdValidation(boolean withKip853Rpc) throws Ex } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testVoteRequestClusterIdValidation(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testVoteRequestClusterIdValidation(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, isSupport853(raftProtocol)); Set voters = Set.of(localId, otherNodeKey.id()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withKip853Rpc(withKip853Rpc) + RaftClientTestContext context = new Builder(localId, voters) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -2464,8 +2473,8 @@ public void testInvalidVoterReplicaVoteRequest() throws Exception { ReplicaKey otherNodeKey = replicaKey(localId + 1, true); Set voters = Set.of(localId, otherNodeKey.id()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withKip853Rpc(true) + RaftClientTestContext context = new Builder(localId, voters) + .withRaftProtocol(KIP_853_PROTOCOL) .build(); context.unattachedToLeader(); @@ -2510,9 +2519,9 @@ public void testInvalidVoterReplicaBeginQuorumEpochRequest() throws Exception { int epoch = 5; Set voters = Set.of(localId, voter2, voter3); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withUnknownLeader(epoch - 1) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .build(); context.assertUnknownLeaderAndNoVotedCandidate(epoch - 1); @@ -2556,15 +2565,15 @@ public void testInvalidVoterReplicaBeginQuorumEpochRequest() throws Exception { } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testBeginQuorumEpochRequestClusterIdValidation(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testBeginQuorumEpochRequestClusterIdValidation(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; Set voters = Set.of(localId, otherNodeId); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withUnknownLeader(4) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -2592,15 +2601,15 @@ public void testBeginQuorumEpochRequestClusterIdValidation(boolean withKip853Rpc } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testEndQuorumEpochRequestClusterIdValidation(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testEndQuorumEpochRequestClusterIdValidation(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, isSupport853(raftProtocol)); Set voters = Set.of(localId, otherNodeKey.id()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withUnknownLeader(4) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -2628,21 +2637,21 @@ public void testEndQuorumEpochRequestClusterIdValidation(boolean withKip853Rpc) } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testLeaderAcceptVoteFromObserver(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testLeaderAcceptVoteFromObserver(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; Set voters = Set.of(localId, otherNodeId); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withUnknownLeader(4) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); int epoch = context.currentEpoch(); - ReplicaKey observerKey = replicaKey(localId + 2, withKip853Rpc); + ReplicaKey observerKey = replicaKey(localId + 2, isSupport853(raftProtocol)); context.deliverRequest(context.voteRequest(epoch - 1, observerKey, 0, 0)); context.client.poll(); context.assertSentVoteResponse(Errors.FENCED_LEADER_EPOCH, epoch, OptionalInt.of(localId), false); @@ -2653,16 +2662,16 @@ public void testLeaderAcceptVoteFromObserver(boolean withKip853Rpc) throws Excep } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testInvalidVoteRequest(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testInvalidVoteRequest(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, isSupport853(raftProtocol)); int epoch = 5; Set voters = Set.of(localId, otherNodeKey.id()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withElectedLeader(epoch, otherNodeKey.id()) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.assertElectedLeader(epoch, otherNodeKey.id()); @@ -2698,15 +2707,15 @@ public void testInvalidVoteRequest(boolean withKip853Rpc) throws Exception { } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testPurgatoryFetchTimeout(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testPurgatoryFetchTimeout(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, isSupport853(raftProtocol)); Set voters = Set.of(localId, otherNodeKey.id()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withUnknownLeader(4) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -2726,15 +2735,15 @@ public void testPurgatoryFetchTimeout(boolean withKip853Rpc) throws Exception { } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testPurgatoryFetchSatisfiedByWrite(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testPurgatoryFetchSatisfiedByWrite(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, isSupport853(raftProtocol)); Set voters = Set.of(localId, otherNodeKey.id()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withUnknownLeader(4) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -2752,20 +2761,20 @@ public void testPurgatoryFetchSatisfiedByWrite(boolean withKip853Rpc) throws Exc context.client.poll(); MemoryRecords fetchedRecords = context.assertSentFetchPartitionResponse(Errors.NONE, epoch, OptionalInt.of(localId)); - RaftClientTestContext.assertMatchingRecords(appendRecords, fetchedRecords); + assertMatchingRecords(appendRecords, fetchedRecords); } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testPurgatoryFetchCompletedByFollowerTransition(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testPurgatoryFetchCompletedByFollowerTransition(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey voterKey2 = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey voterKey2 = replicaKey(localId + 1, isSupport853(raftProtocol)); int voter3 = localId + 2; Set voters = Set.of(localId, voterKey2.id(), voter3); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withUnknownLeader(4) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -2792,17 +2801,17 @@ public void testPurgatoryFetchCompletedByFollowerTransition(boolean withKip853Rp } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testFetchResponseIgnoredAfterBecomingProspective(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testFetchResponseIgnoredAfterBecomingProspective(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int epoch = 5; // The other node starts out as the leader Set voters = Set.of(localId, otherNodeId); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withElectedLeader(epoch, otherNodeId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.assertElectedLeader(epoch, otherNodeId); @@ -2829,9 +2838,9 @@ public void testFetchResponseIgnoredAfterBecomingProspective(boolean withKip853R } @ParameterizedTest - @ValueSource(booleans = { true, false }) + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) public void testFetchResponseIgnoredAfterBecomingFollowerOfDifferentLeader( - boolean withKip853Rpc + RaftProtocol raftProtocol ) throws Exception { int localId = randomReplicaId(); int voter2 = localId + 1; @@ -2840,9 +2849,9 @@ public void testFetchResponseIgnoredAfterBecomingFollowerOfDifferentLeader( // Start out with `voter2` as the leader Set voters = Set.of(localId, voter2, voter3); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withElectedLeader(epoch, voter2) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.assertElectedLeader(epoch, voter2); @@ -2870,17 +2879,17 @@ public void testFetchResponseIgnoredAfterBecomingFollowerOfDifferentLeader( } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testVoteResponseIgnoredAfterBecomingFollower(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testVoteResponseIgnoredAfterBecomingFollower(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int voter2 = localId + 1; int voter3 = localId + 2; int epoch = 5; Set voters = Set.of(localId, voter2, voter3); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withUnknownLeader(epoch) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.assertUnknownLeaderAndNoVotedCandidate(epoch); context.unattachedToCandidate(); @@ -2924,8 +2933,8 @@ public void testVoteResponseIgnoredAfterBecomingFollower(boolean withKip853Rpc) } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testFollowerLeaderRediscoveryAfterBrokerNotAvailableError(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testFollowerLeaderRediscoveryAfterBrokerNotAvailableError(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int leaderId = localId + 1; int otherNodeId = localId + 2; @@ -2936,9 +2945,9 @@ public void testFollowerLeaderRediscoveryAfterBrokerNotAvailableError(boolean wi .map(RaftClientTestContext::mockAddress) .collect(Collectors.toList()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withBootstrapServers(Optional.of(bootstrapServers)) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .withElectedLeader(epoch, leaderId) .build(); @@ -2963,8 +2972,8 @@ public void testFollowerLeaderRediscoveryAfterBrokerNotAvailableError(boolean wi } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testFollowerLeaderRediscoveryAfterRequestTimeout(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testFollowerLeaderRediscoveryAfterRequestTimeout(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int leaderId = localId + 1; int otherNodeId = localId + 2; @@ -2975,9 +2984,9 @@ public void testFollowerLeaderRediscoveryAfterRequestTimeout(boolean withKip853R .map(RaftClientTestContext::mockAddress) .collect(Collectors.toList()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withBootstrapServers(Optional.of(bootstrapServers)) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .withElectedLeader(epoch, leaderId) .build(); @@ -2998,8 +3007,8 @@ public void testFollowerLeaderRediscoveryAfterRequestTimeout(boolean withKip853R } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testObserverLeaderRediscoveryAfterBrokerNotAvailableError(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testObserverLeaderRediscoveryAfterBrokerNotAvailableError(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int leaderId = localId + 1; int otherNodeId = localId + 2; @@ -3010,9 +3019,9 @@ public void testObserverLeaderRediscoveryAfterBrokerNotAvailableError(boolean wi .map(RaftClientTestContext::mockAddress) .collect(Collectors.toList()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withBootstrapServers(Optional.of(bootstrapServers)) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.discoverLeaderAsObserver(leaderId, epoch); @@ -3047,8 +3056,8 @@ public void testObserverLeaderRediscoveryAfterBrokerNotAvailableError(boolean wi } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testObserverLeaderRediscoveryAfterRequestTimeout(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testObserverLeaderRediscoveryAfterRequestTimeout(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int leaderId = localId + 1; int otherNodeId = localId + 2; @@ -3059,9 +3068,9 @@ public void testObserverLeaderRediscoveryAfterRequestTimeout(boolean withKip853R .map(RaftClientTestContext::mockAddress) .collect(Collectors.toList()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withBootstrapServers(Optional.of(bootstrapServers)) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.discoverLeaderAsObserver(leaderId, epoch); @@ -3092,14 +3101,14 @@ public void testObserverLeaderRediscoveryAfterRequestTimeout(boolean withKip853R } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testLeaderGracefulShutdown(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testLeaderGracefulShutdown(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, isSupport853(raftProtocol)); Set voters = Set.of(localId, otherNodeKey.id()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withKip853Rpc(withKip853Rpc) + RaftClientTestContext context = new Builder(localId, voters) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -3139,15 +3148,15 @@ public void testLeaderGracefulShutdown(boolean withKip853Rpc) throws Exception { } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testEndQuorumEpochSentBasedOnFetchOffset(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testEndQuorumEpochSentBasedOnFetchOffset(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey closeFollower = replicaKey(localId + 2, withKip853Rpc); - ReplicaKey laggingFollower = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey closeFollower = replicaKey(localId + 2, isSupport853(raftProtocol)); + ReplicaKey laggingFollower = replicaKey(localId + 1, isSupport853(raftProtocol)); Set voters = Set.of(localId, closeFollower.id(), laggingFollower.id()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withKip853Rpc(withKip853Rpc) + RaftClientTestContext context = new Builder(localId, voters) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -3190,15 +3199,15 @@ public void testEndQuorumEpochSentBasedOnFetchOffset(boolean withKip853Rpc) thro } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testDescribeQuorumNonLeader(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testDescribeQuorumNonLeader(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey voter2 = replicaKey(localId + 1, withKip853Rpc); - ReplicaKey voter3 = replicaKey(localId + 2, withKip853Rpc); + ReplicaKey voter2 = replicaKey(localId + 1, isSupport853(raftProtocol)); + ReplicaKey voter3 = replicaKey(localId + 2, isSupport853(raftProtocol)); int epoch = 2; Set voters = Set.of(localId, voter2.id(), voter3.id()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withUnknownLeader(epoch) .build(); @@ -3221,16 +3230,16 @@ public void testDescribeQuorumNonLeader(boolean withKip853Rpc) throws Exception } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testDescribeQuorumWithOnlyStaticVoters(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testDescribeQuorumWithOnlyStaticVoters(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); ReplicaKey local = replicaKey(localId, true); ReplicaKey follower1 = replicaKey(localId + 1, true); Set voters = Set.of(localId, follower1.id()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, local.directoryId().get()) + RaftClientTestContext context = new Builder(localId, local.directoryId().get()) .withStaticVoters(voters) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -3256,24 +3265,25 @@ public void testDescribeQuorumWithOnlyStaticVoters(boolean withKip853Rpc) throws } @ParameterizedTest - @CsvSource({ "true, true", "true, false", "false, false" }) - public void testDescribeQuorumWithFollowers(boolean withKip853Rpc, boolean withBootstrapSnapshot) throws Exception { + @MethodSource("generateTestDescribeQuorumMatrix") + public void testDescribeQuorumWithFollowers(RaftProtocol raftProtocol, boolean withBootstrapSnapshot) throws Exception { int localId = randomReplicaId(); int followerId1 = localId + 1; int followerId2 = localId + 2; + boolean support853 = isSupport853(raftProtocol); ReplicaKey local = replicaKey(localId, withBootstrapSnapshot); // local directory id must exist Uuid localDirectoryId = local.directoryId().orElse(Uuid.randomUuid()); ReplicaKey bootstrapFollower1 = replicaKey(followerId1, withBootstrapSnapshot); // if withBootstrapSnapshot is false, directory ids are still needed by the static voter set - Uuid followerDirectoryId1 = bootstrapFollower1.directoryId().orElse(withKip853Rpc ? Uuid.randomUuid() : ReplicaKey.NO_DIRECTORY_ID); + Uuid followerDirectoryId1 = bootstrapFollower1.directoryId().orElse(support853 ? Uuid.randomUuid() : ReplicaKey.NO_DIRECTORY_ID); ReplicaKey follower1 = ReplicaKey.of(followerId1, followerDirectoryId1); ReplicaKey bootstrapFollower2 = replicaKey(followerId2, withBootstrapSnapshot); - Uuid followerDirectoryId2 = bootstrapFollower2.directoryId().orElse(withKip853Rpc ? Uuid.randomUuid() : ReplicaKey.NO_DIRECTORY_ID); + Uuid followerDirectoryId2 = bootstrapFollower2.directoryId().orElse(support853 ? Uuid.randomUuid() : ReplicaKey.NO_DIRECTORY_ID); ReplicaKey follower2 = ReplicaKey.of(followerId2, followerDirectoryId2); - RaftClientTestContext.Builder builder = new RaftClientTestContext.Builder(localId, localDirectoryId) - .withKip853Rpc(withKip853Rpc); + Builder builder = new Builder(localId, localDirectoryId) + .withRaftProtocol(raftProtocol); if (withBootstrapSnapshot) { VoterSet bootstrapVoterSet = VoterSetTest.voterSet(Stream.of(local, bootstrapFollower1, bootstrapFollower2)); @@ -3368,18 +3378,19 @@ public void testDescribeQuorumWithFollowers(boolean withKip853Rpc, boolean withB } @ParameterizedTest - @CsvSource({ "true, true", "true, false", "false, false" }) - public void testDescribeQuorumWithObserver(boolean withKip853Rpc, boolean withBootstrapSnapshot) throws Exception { + @MethodSource("generateTestDescribeQuorumMatrix") + public void testDescribeQuorumWithObserver(RaftProtocol raftProtocol, boolean withBootstrapSnapshot) throws Exception { int localId = randomReplicaId(); int followerId = localId + 1; ReplicaKey local = replicaKey(localId, withBootstrapSnapshot); Uuid localDirectoryId = local.directoryId().orElse(Uuid.randomUuid()); ReplicaKey bootstrapFollower = replicaKey(followerId, withBootstrapSnapshot); - Uuid followerDirectoryId = bootstrapFollower.directoryId().orElse(withKip853Rpc ? Uuid.randomUuid() : ReplicaKey.NO_DIRECTORY_ID); + boolean support853 = isSupport853(raftProtocol); + Uuid followerDirectoryId = bootstrapFollower.directoryId().orElse(support853 ? Uuid.randomUuid() : ReplicaKey.NO_DIRECTORY_ID); ReplicaKey follower = ReplicaKey.of(followerId, followerDirectoryId); - RaftClientTestContext.Builder builder = new RaftClientTestContext.Builder(localId, localDirectoryId) - .withKip853Rpc(withKip853Rpc); + Builder builder = new Builder(localId, localDirectoryId) + .withRaftProtocol(raftProtocol); if (withBootstrapSnapshot) { VoterSet bootstrapVoterSet = VoterSetTest.voterSet(Stream.of(local, bootstrapFollower)); @@ -3403,7 +3414,7 @@ public void testDescribeQuorumWithObserver(boolean withKip853Rpc, boolean withBo context.assertSentFetchPartitionResponse(expectedHW, epoch); // Create observer - ReplicaKey observer = replicaKey(localId + 2, withKip853Rpc); + ReplicaKey observer = replicaKey(localId + 2, support853); Uuid observerDirectoryId = observer.directoryId().orElse(ReplicaKey.NO_DIRECTORY_ID); context.time.sleep(100); long observerFetchTime = context.time.milliseconds(); @@ -3511,18 +3522,18 @@ public void testDescribeQuorumWithObserver(boolean withKip853Rpc, boolean withBo } @ParameterizedTest - @CsvSource({ "true, true", "true, false", "false, false" }) - public void testDescribeQuorumNonMonotonicFollowerFetch(boolean withKip853Rpc, boolean withBootstrapSnapshot) throws Exception { + @MethodSource("generateTestDescribeQuorumMatrix") + public void testDescribeQuorumNonMonotonicFollowerFetch(RaftProtocol raftProtocol, boolean withBootstrapSnapshot) throws Exception { int localId = randomReplicaId(); ReplicaKey local = replicaKey(localId, withBootstrapSnapshot); Uuid localDirectoryId = local.directoryId().orElse(Uuid.randomUuid()); int followerId = localId + 1; ReplicaKey bootstrapFollower = replicaKey(followerId, withBootstrapSnapshot); - Uuid followerDirectoryId = bootstrapFollower.directoryId().orElse(withKip853Rpc ? Uuid.randomUuid() : ReplicaKey.NO_DIRECTORY_ID); + Uuid followerDirectoryId = bootstrapFollower.directoryId().orElse(isSupport853(raftProtocol) ? Uuid.randomUuid() : ReplicaKey.NO_DIRECTORY_ID); ReplicaKey follower = ReplicaKey.of(followerId, followerDirectoryId); - RaftClientTestContext.Builder builder = new RaftClientTestContext.Builder(localId, localDirectoryId) - .withKip853Rpc(withKip853Rpc); + Builder builder = new Builder(localId, localDirectoryId) + .withRaftProtocol(raftProtocol); if (withBootstrapSnapshot) { VoterSet bootstrapVoterSet = VoterSetTest.voterSet(Stream.of(local, bootstrapFollower)); builder.withBootstrapSnapshot(Optional.of(bootstrapVoterSet)); @@ -3587,8 +3598,8 @@ public void testDescribeQuorumNonMonotonicFollowerFetch(boolean withKip853Rpc, b } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testStaticVotersIgnoredWithBootstrapSnapshot(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testStaticVotersIgnoredWithBootstrapSnapshot(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); ReplicaKey local = replicaKey(localId, true); ReplicaKey follower = replicaKey(localId + 1, true); @@ -3597,9 +3608,9 @@ public void testStaticVotersIgnoredWithBootstrapSnapshot(boolean withKip853Rpc) Set staticVoters = Set.of(localId, follower.id()); VoterSet voterSet = VoterSetTest.voterSet(Stream.of(local, follower, follower2)); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, local.directoryId().get()) + RaftClientTestContext context = new Builder(localId, local.directoryId().get()) .withStaticVoters(staticVoters) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .withBootstrapSnapshot(Optional.of(voterSet)) .build(); @@ -3608,22 +3619,23 @@ public void testStaticVotersIgnoredWithBootstrapSnapshot(boolean withKip853Rpc) // check describe quorum response has both followers context.deliverRequest(context.describeQuorumRequest()); context.pollUntilResponse(); + boolean support853 = isSupport853(raftProtocol); List expectedVoterStates = Arrays.asList( new ReplicaState() .setReplicaId(localId) - .setReplicaDirectoryId(withKip853Rpc ? local.directoryId().get() : ReplicaKey.NO_DIRECTORY_ID) + .setReplicaDirectoryId(support853 ? local.directoryId().get() : ReplicaKey.NO_DIRECTORY_ID) .setLogEndOffset(3L) .setLastFetchTimestamp(context.time.milliseconds()) .setLastCaughtUpTimestamp(context.time.milliseconds()), new ReplicaState() .setReplicaId(follower.id()) - .setReplicaDirectoryId(withKip853Rpc ? follower.directoryId().get() : ReplicaKey.NO_DIRECTORY_ID) + .setReplicaDirectoryId(support853 ? follower.directoryId().get() : ReplicaKey.NO_DIRECTORY_ID) .setLogEndOffset(-1L) .setLastFetchTimestamp(-1) .setLastCaughtUpTimestamp(-1), new ReplicaState() .setReplicaId(follower2.id()) - .setReplicaDirectoryId(withKip853Rpc ? follower2.directoryId().get() : ReplicaKey.NO_DIRECTORY_ID) + .setReplicaDirectoryId(support853 ? follower2.directoryId().get() : ReplicaKey.NO_DIRECTORY_ID) .setLogEndOffset(-1L) .setLastFetchTimestamp(-1) .setLastCaughtUpTimestamp(-1)); @@ -3632,15 +3644,15 @@ public void testStaticVotersIgnoredWithBootstrapSnapshot(boolean withKip853Rpc) @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testLeaderGracefulShutdownTimeout(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testLeaderGracefulShutdownTimeout(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; Set voters = Set.of(localId, otherNodeId); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withUnknownLeader(1) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -3670,16 +3682,16 @@ public void testLeaderGracefulShutdownTimeout(boolean withKip853Rpc) throws Exce } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testFollowerGracefulShutdown(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testFollowerGracefulShutdown(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int epoch = 5; Set voters = Set.of(localId, otherNodeId); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withElectedLeader(epoch, otherNodeId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.assertElectedLeader(epoch, otherNodeId); @@ -3697,16 +3709,16 @@ public void testFollowerGracefulShutdown(boolean withKip853Rpc) throws Exception } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testObserverGracefulShutdown(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testObserverGracefulShutdown(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int voter1 = localId + 1; int voter2 = localId + 2; Set voters = Set.of(voter1, voter2); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withUnknownLeader(5) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.client.poll(); context.assertUnknownLeaderAndNoVotedCandidate(5); @@ -3724,11 +3736,11 @@ public void testObserverGracefulShutdown(boolean withKip853Rpc) throws Exception } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testGracefulShutdownSingleMemberQuorum(boolean withKip853Rpc) throws IOException { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testGracefulShutdownSingleMemberQuorum(RaftProtocol raftProtocol) throws IOException { int localId = randomReplicaId(); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, Collections.singleton(localId)) - .withKip853Rpc(withKip853Rpc) + RaftClientTestContext context = new Builder(localId, Collections.singleton(localId)) + .withRaftProtocol(raftProtocol) .build(); context.assertElectedLeader(1, localId); @@ -3742,16 +3754,16 @@ public void testGracefulShutdownSingleMemberQuorum(boolean withKip853Rpc) throws } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testFollowerReplication(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testFollowerReplication(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int epoch = 5; Set voters = Set.of(localId, otherNodeId); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withElectedLeader(epoch, otherNodeId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.assertElectedLeader(epoch, otherNodeId); @@ -3772,16 +3784,16 @@ public void testFollowerReplication(boolean withKip853Rpc) throws Exception { } @ParameterizedTest - @CsvSource({ "true, true", "true, false", "false, true", "false, false" }) - public void testObserverReplication(boolean withKip853Rpc, boolean alwaysFlush) throws Exception { + @MethodSource("generateTestObserverReplication") + public void testObserverReplication(RaftProtocol raftProtocol, boolean alwaysFlush) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int epoch = 5; Set voters = Set.of(otherNodeId); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withElectedLeader(epoch, otherNodeId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .withAlwaysFlush(alwaysFlush) .build(); context.assertElectedLeader(epoch, otherNodeId); @@ -3804,16 +3816,16 @@ public void testObserverReplication(boolean withKip853Rpc, boolean alwaysFlush) } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testEmptyRecordSetInFetchResponse(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testEmptyRecordSetInFetchResponse(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int epoch = 5; Set voters = Set.of(localId, otherNodeId); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withElectedLeader(epoch, otherNodeId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.assertElectedLeader(epoch, otherNodeId); @@ -3871,17 +3883,17 @@ public void testEmptyRecordSetInFetchResponse(boolean withKip853Rpc) throws Exce } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testFetchShouldBeTreatedAsLeaderAcknowledgement(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testFetchShouldBeTreatedAsLeaderAcknowledgement(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, isSupport853(raftProtocol)); int epoch = 5; Set voters = Set.of(localId, otherNodeKey.id()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .updateRandom(r -> r.mockNextInt(DEFAULT_ELECTION_TIMEOUT_MS, 0)) .withUnknownLeader(epoch - 1) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToCandidate(); @@ -3907,13 +3919,13 @@ public void testFetchShouldBeTreatedAsLeaderAcknowledgement(boolean withKip853Rp } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testLeaderAppendSingleMemberQuorum(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testLeaderAppendSingleMemberQuorum(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); Set voters = Collections.singleton(localId); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withKip853Rpc(withKip853Rpc) + RaftClientTestContext context = new Builder(localId, voters) + .withRaftProtocol(raftProtocol) .build(); long now = context.time.milliseconds(); @@ -3937,7 +3949,7 @@ public void testLeaderAppendSingleMemberQuorum(boolean withKip853Rpc) throws Exc assertEquals(OptionalLong.of(4L), context.client.highWatermark()); // Now try reading it - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, isSupport853(raftProtocol)); List batches = new ArrayList<>(2); boolean appended = true; @@ -3970,7 +3982,7 @@ public void testLeaderAppendSingleMemberQuorum(boolean withKip853Rpc) throws Exc Record record = readRecords.get(0); assertEquals(now, record.timestamp()); - RaftClientTestContext.verifyLeaderChangeMessage(localId, Collections.singletonList(localId), + verifyLeaderChangeMessage(localId, Collections.singletonList(localId), Collections.singletonList(localId), record.key(), record.value()); MutableRecordBatch batch = batches.get(1); @@ -3984,19 +3996,19 @@ public void testLeaderAppendSingleMemberQuorum(boolean withKip853Rpc) throws Exc } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testFollowerLogReconciliation(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testFollowerLogReconciliation(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int epoch = 5; int lastEpoch = 3; Set voters = Set.of(localId, otherNodeId); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withElectedLeader(epoch, otherNodeId) .appendToLog(lastEpoch, Arrays.asList("foo", "bar")) .appendToLog(lastEpoch, singletonList("baz")) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.assertElectedLeader(epoch, otherNodeId); @@ -4021,12 +4033,12 @@ public void testFollowerLogReconciliation(boolean withKip853Rpc) throws Exceptio } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testMetrics(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testMetrics(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int epoch = 1; - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, Collections.singleton(localId)) - .withKip853Rpc(withKip853Rpc) + RaftClientTestContext context = new Builder(localId, Collections.singleton(localId)) + .withRaftProtocol(raftProtocol) .build(); context.pollUntil(() -> context.log.endOffset().offset() == 1L); @@ -4069,15 +4081,15 @@ public void testMetrics(boolean withKip853Rpc) throws Exception { } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testClusterAuthorizationFailedInFetch(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testClusterAuthorizationFailedInFetch(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int epoch = 5; Set voters = Set.of(localId, otherNodeId); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withKip853Rpc(withKip853Rpc) + RaftClientTestContext context = new Builder(localId, voters) + .withRaftProtocol(raftProtocol) .withElectedLeader(epoch, otherNodeId) .build(); @@ -4097,17 +4109,17 @@ public void testClusterAuthorizationFailedInFetch(boolean withKip853Rpc) throws } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testClusterAuthorizationFailedInBeginQuorumEpoch(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testClusterAuthorizationFailedInBeginQuorumEpoch(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int epoch = 5; Set voters = Set.of(localId, otherNodeId); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .updateRandom(r -> r.mockNextInt(DEFAULT_ELECTION_TIMEOUT_MS, 0)) .withUnknownLeader(epoch - 1) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.time.sleep(context.electionTimeoutMs()); @@ -4127,16 +4139,16 @@ public void testClusterAuthorizationFailedInBeginQuorumEpoch(boolean withKip853R } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testClusterAuthorizationFailedInVote(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testClusterAuthorizationFailedInVote(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int epoch = 5; Set voters = Set.of(localId, otherNodeId); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withUnknownLeader(epoch - 1) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); // Become a candidate @@ -4153,15 +4165,15 @@ public void testClusterAuthorizationFailedInVote(boolean withKip853Rpc) throws E } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testClusterAuthorizationFailedInEndQuorumEpoch(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testClusterAuthorizationFailedInEndQuorumEpoch(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; Set voters = Set.of(localId, otherNodeId); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withUnknownLeader(1) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -4179,16 +4191,14 @@ public void testClusterAuthorizationFailedInEndQuorumEpoch(boolean withKip853Rpc } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testHandleLeaderChangeFiresAfterListenerReachesEpochStartOffsetOnEmptyLog( - boolean withKip853Rpc - ) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testHandleLeaderChangeFiresAfterListenerReachesEpochStartOffsetOnEmptyLog(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, isSupport853(raftProtocol)); Set voters = Set.of(localId, otherNodeKey.id()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withKip853Rpc(withKip853Rpc) + RaftClientTestContext context = new Builder(localId, voters) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -4226,12 +4236,10 @@ public void testHandleLeaderChangeFiresAfterListenerReachesEpochStartOffsetOnEmp } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testHandleLeaderChangeFiresAfterListenerReachesEpochStartOffset( - boolean withKip853Rpc - ) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testHandleLeaderChangeFiresAfterListenerReachesEpochStartOffset(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, isSupport853(raftProtocol)); int epoch = 5; Set voters = Set.of(localId, otherNodeKey.id()); @@ -4240,12 +4248,12 @@ public void testHandleLeaderChangeFiresAfterListenerReachesEpochStartOffset( List batch3 = Arrays.asList("7", "8", "9"); List> expectedBatches = Arrays.asList(batch1, batch2, batch3); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .appendToLog(1, batch1) .appendToLog(1, batch2) .appendToLog(2, batch3) .withUnknownLeader(epoch - 1) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -4295,10 +4303,10 @@ public void testHandleLeaderChangeFiresAfterListenerReachesEpochStartOffset( } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testLateRegisteredListenerCatchesUp(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testLateRegisteredListenerCatchesUp(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, isSupport853(raftProtocol)); int epoch = 5; Set voters = Set.of(localId, otherNodeKey.id()); @@ -4306,12 +4314,12 @@ public void testLateRegisteredListenerCatchesUp(boolean withKip853Rpc) throws Ex List batch2 = Arrays.asList("4", "5", "6"); List batch3 = Arrays.asList("7", "8", "9"); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .appendToLog(1, batch1) .appendToLog(1, batch2) .appendToLog(2, batch3) .withUnknownLeader(epoch - 1) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -4328,7 +4336,7 @@ public void testLateRegisteredListenerCatchesUp(boolean withKip853Rpc) throws Ex assertEquals(9L, context.listener.claimedEpochStartOffset(epoch)); // Register a second listener and allow it to catch up to the high watermark - RaftClientTestContext.MockListener secondListener = new RaftClientTestContext.MockListener(OptionalInt.of(localId)); + MockListener secondListener = new MockListener(OptionalInt.of(localId)); context.client.register(secondListener); context.pollUntil(() -> OptionalInt.of(epoch).equals(secondListener.currentClaimedEpoch())); assertEquals(OptionalLong.of(9L), secondListener.lastCommitOffset()); @@ -4338,8 +4346,8 @@ public void testLateRegisteredListenerCatchesUp(boolean withKip853Rpc) throws Ex } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testReregistrationChangesListenerContext(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testReregistrationChangesListenerContext(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int epoch = 5; @@ -4349,12 +4357,12 @@ public void testReregistrationChangesListenerContext(boolean withKip853Rpc) thro List batch2 = Arrays.asList("4", "5", "6"); List batch3 = Arrays.asList("7", "8", "9"); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .appendToLog(1, batch1) .appendToLog(1, batch2) .appendToLog(2, batch3) .withUnknownLeader(epoch - 1) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -4366,7 +4374,7 @@ public void testReregistrationChangesListenerContext(boolean withKip853Rpc) thro context.pollUntil(() -> OptionalLong.of(9).equals(context.listener.lastCommitOffset())); // Register a second listener - RaftClientTestContext.MockListener secondListener = new RaftClientTestContext.MockListener(OptionalInt.of(localId)); + MockListener secondListener = new MockListener(OptionalInt.of(localId)); context.client.register(secondListener); context.pollUntil(() -> OptionalLong.of(9).equals(secondListener.lastCommitOffset())); context.client.unregister(secondListener); @@ -4382,16 +4390,16 @@ public void testReregistrationChangesListenerContext(boolean withKip853Rpc) thro } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testHandleCommitCallbackFiresAfterFollowerHighWatermarkAdvances(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testHandleCommitCallbackFiresAfterFollowerHighWatermarkAdvances(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int epoch = 5; Set voters = Set.of(localId, otherNodeId); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withElectedLeader(epoch, otherNodeId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); assertEquals(OptionalLong.empty(), context.client.highWatermark()); @@ -4441,21 +4449,21 @@ public void testHandleCommitCallbackFiresAfterFollowerHighWatermarkAdvances(bool } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testHandleCommitCallbackFiresInVotedState(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testHandleCommitCallbackFiresInVotedState(RaftProtocol raftProtocol) throws Exception { // This test verifies that the state machine can still catch up even while // an election is in progress as long as the high watermark is known. int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, isSupport853(raftProtocol)); int epoch = 7; Set voters = Set.of(localId, otherNodeKey.id()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .appendToLog(2, Arrays.asList("a", "b", "c")) .appendToLog(4, Arrays.asList("d", "e", "f")) .appendToLog(4, Arrays.asList("g", "h", "i")) .withUnknownLeader(epoch - 1) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); // Start off as the leader and receive a fetch to initialize the high watermark @@ -4472,7 +4480,7 @@ public void testHandleCommitCallbackFiresInVotedState(boolean withKip853Rpc) thr assertEquals(OptionalLong.of(10L), context.client.highWatermark()); // Register another listener and verify that it catches up while we remain 'voted' - RaftClientTestContext.MockListener secondListener = new RaftClientTestContext.MockListener( + MockListener secondListener = new MockListener( OptionalInt.of(localId) ); context.client.register(secondListener); @@ -4488,21 +4496,21 @@ public void testHandleCommitCallbackFiresInVotedState(boolean withKip853Rpc) thr } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testHandleCommitCallbackFiresInCandidateState(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testHandleCommitCallbackFiresInCandidateState(RaftProtocol raftProtocol) throws Exception { // This test verifies that the state machine can still catch up even while // an election is in progress as long as the high watermark is known. int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, isSupport853(raftProtocol)); int epoch = 7; Set voters = Set.of(localId, otherNodeKey.id()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .appendToLog(2, Arrays.asList("a", "b", "c")) .appendToLog(4, Arrays.asList("d", "e", "f")) .appendToLog(4, Arrays.asList("g", "h", "i")) .withUnknownLeader(epoch - 1) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); // Start off as the leader and receive a fetch to initialize the high watermark @@ -4527,7 +4535,7 @@ public void testHandleCommitCallbackFiresInCandidateState(boolean withKip853Rpc) context.assertVotedCandidate(candidateEpoch, localId); // Register another listener and verify that it catches up - RaftClientTestContext.MockListener secondListener = new RaftClientTestContext.MockListener( + MockListener secondListener = new MockListener( OptionalInt.of(localId) ); context.client.register(secondListener); @@ -4543,9 +4551,9 @@ public void testHandleCommitCallbackFiresInCandidateState(boolean withKip853Rpc) } @ParameterizedTest - @ValueSource(booleans = { true, false }) + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) public void testHandleLeaderChangeFiresAfterUnattachedRegistration( - boolean withKip853Rpc + RaftProtocol raftProtocol ) throws Exception { // When registering a listener while the replica is unattached, it should get notified // with the current epoch @@ -4555,13 +4563,13 @@ public void testHandleLeaderChangeFiresAfterUnattachedRegistration( int epoch = 7; Set voters = Set.of(localId, otherNodeId); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withUnknownLeader(epoch) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); // Register another listener and verify that it is notified of latest epoch - RaftClientTestContext.MockListener secondListener = new RaftClientTestContext.MockListener( + MockListener secondListener = new MockListener( OptionalInt.of(localId) ); context.client.register(secondListener); @@ -4581,8 +4589,8 @@ public void testHandleLeaderChangeFiresAfterUnattachedRegistration( } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testHandleLeaderChangeFiresAfterFollowerRegistration(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testHandleLeaderChangeFiresAfterFollowerRegistration(RaftProtocol raftProtocol) throws Exception { // When registering a listener while the replica is a follower, it should get notified with // the current leader and epoch int localId = randomReplicaId(); @@ -4590,13 +4598,13 @@ public void testHandleLeaderChangeFiresAfterFollowerRegistration(boolean withKip int epoch = 7; Set voters = Set.of(localId, otherNodeId); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withElectedLeader(epoch, otherNodeId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); // Register another listener and verify that it is notified of latest leader and epoch - RaftClientTestContext.MockListener secondListener = new RaftClientTestContext.MockListener( + MockListener secondListener = new MockListener( OptionalInt.of(localId) ); context.client.register(secondListener); @@ -4607,8 +4615,8 @@ public void testHandleLeaderChangeFiresAfterFollowerRegistration(boolean withKip } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testHandleLeaderChangeFiresAfterResignRegistration(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testHandleLeaderChangeFiresAfterResignRegistration(RaftProtocol raftProtocol) throws Exception { // When registering a listener while the replica is resigned, it should not get notified with // the current leader and epoch int localId = randomReplicaId(); @@ -4616,9 +4624,9 @@ public void testHandleLeaderChangeFiresAfterResignRegistration(boolean withKip85 int epoch = 7; Set voters = Set.of(localId, otherNodeId); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new Builder(localId, voters) .withElectedLeader(epoch, localId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.client.poll(); @@ -4626,7 +4634,7 @@ public void testHandleLeaderChangeFiresAfterResignRegistration(boolean withKip85 assertEquals(LeaderAndEpoch.UNKNOWN, context.listener.currentLeaderAndEpoch()); // Register another listener and verify that it is not notified of latest leader and epoch - RaftClientTestContext.MockListener secondListener = new RaftClientTestContext.MockListener( + MockListener secondListener = new MockListener( OptionalInt.of(localId) ); context.client.register(secondListener); @@ -4637,8 +4645,8 @@ public void testHandleLeaderChangeFiresAfterResignRegistration(boolean withKip85 } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testObserverFetchWithNoLocalId(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftClientTestContext.RaftProtocol.class, names = {KIP_595_PROTOCOL_NAME, KIP_853_PROTOCOL_NAME}) + public void testObserverFetchWithNoLocalId(RaftProtocol raftProtocol) throws Exception { // When no `localId` is defined, the client will behave as an observer. // This is designed for tooling/debugging use cases. int leaderId = randomReplicaId(); @@ -4648,9 +4656,9 @@ public void testObserverFetchWithNoLocalId(boolean withKip853Rpc) throws Excepti .map(RaftClientTestContext::mockAddress) .collect(Collectors.toList()); - RaftClientTestContext context = new RaftClientTestContext.Builder(OptionalInt.empty(), voters) + RaftClientTestContext context = new Builder(OptionalInt.empty(), voters) .withBootstrapServers(Optional.of(bootstrapServers)) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); // First fetch discovers the current leader and epoch @@ -4689,6 +4697,10 @@ public void testObserverFetchWithNoLocalId(boolean withKip853Rpc) throws Excepti assertEquals(3, context.log.lastFetchedEpoch()); } + private boolean isSupport853(RaftProtocol raftProtocol) { + return raftProtocol == KIP_853_PROTOCOL; + } + private static KafkaMetric getMetric(final Metrics metrics, final String name) { return metrics.metrics().get(metrics.metricName(name, "raft-metrics")); } @@ -4701,4 +4713,21 @@ static ReplicaKey replicaKey(int id, boolean withDirectoryId) { static int randomReplicaId() { return ThreadLocalRandom.current().nextInt(1025); } + + private static Stream generateTestDescribeQuorumMatrix() { + return Stream.of( + Arguments.of(KIP_853_PROTOCOL, true), + Arguments.of(KIP_853_PROTOCOL, false), + Arguments.of(KIP_595_PROTOCOL, false) + ); + } + + private static Stream generateTestObserverReplication() { + return Stream.of( + Arguments.of(KIP_853_PROTOCOL, true), + Arguments.of(KIP_853_PROTOCOL, false), + Arguments.of(KIP_853_PROTOCOL, true), + Arguments.of(KIP_595_PROTOCOL, false) + ); + } } diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java index ee840ff59ed77..db293e4cec4bb 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java @@ -99,7 +99,6 @@ import java.util.stream.Stream; import static org.apache.kafka.raft.LeaderState.CHECK_QUORUM_TIMEOUT_FACTOR; -import static org.apache.kafka.raft.RaftClientTestContext.RaftProtocol.KIP_853_PROTOCOL; import static org.apache.kafka.raft.RaftUtil.hasValidTopicPartition; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -293,14 +292,6 @@ Builder withBootstrapServers(Optional> bootstrapServers) return this; } - // deprecated, use withRpc instead - Builder withKip853Rpc(boolean withKip853Rpc) { - if (withKip853Rpc) { - this.raftProtocol = KIP_853_PROTOCOL; - } - return this; - } - Builder withRaftProtocol(RaftProtocol raftProtocol) { this.raftProtocol = raftProtocol; return this;