diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientClusterAuthTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientClusterAuthTest.java index 62c8e11769817..ebae5bd26401a 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientClusterAuthTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientClusterAuthTest.java @@ -38,14 +38,14 @@ public class KafkaRaftClientClusterAuthTest { @ParameterizedTest @ValueSource(booleans = { true, false }) - void testClusterAuthorizationFailedInFetch(boolean withKip853Rpc) throws Exception { + void testClusterAuthorizationFailedInFetch(boolean isReconfigSupported) 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) + .withRaftProtocol(isReconfigSupported ? RaftProtocol.KIP_853_PROTOCOL : RaftProtocol.KIP_595_PROTOCOL) .withElectedLeader(epoch, otherNodeId) .build(); @@ -66,7 +66,7 @@ void testClusterAuthorizationFailedInFetch(boolean withKip853Rpc) throws Excepti @ParameterizedTest @ValueSource(booleans = { true, false }) - void testClusterAuthorizationFailedInBeginQuorumEpoch(boolean withKip853Rpc) throws Exception { + void testClusterAuthorizationFailedInBeginQuorumEpoch(boolean isReconfigSupported) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int epoch = 5; @@ -75,7 +75,7 @@ void testClusterAuthorizationFailedInBeginQuorumEpoch(boolean withKip853Rpc) thr RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .updateRandom(r -> r.mockNextInt(DEFAULT_ELECTION_TIMEOUT_MS, 0)) .withUnknownLeader(epoch - 1) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(isReconfigSupported ? RaftProtocol.KIP_853_PROTOCOL : RaftProtocol.KIP_595_PROTOCOL) .build(); context.time.sleep(context.electionTimeoutMs()); @@ -96,7 +96,7 @@ void testClusterAuthorizationFailedInBeginQuorumEpoch(boolean withKip853Rpc) thr @ParameterizedTest @ValueSource(booleans = { true, false }) - void testClusterAuthorizationFailedInVote(boolean withKip853Rpc) throws Exception { + void testClusterAuthorizationFailedInVote(boolean isReconfigSupported) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int epoch = 5; @@ -104,7 +104,7 @@ void testClusterAuthorizationFailedInVote(boolean withKip853Rpc) throws Exceptio RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withUnknownLeader(epoch - 1) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(isReconfigSupported ? RaftProtocol.KIP_853_PROTOCOL : RaftProtocol.KIP_595_PROTOCOL) .build(); // Become a candidate @@ -122,14 +122,14 @@ void testClusterAuthorizationFailedInVote(boolean withKip853Rpc) throws Exceptio @ParameterizedTest @ValueSource(booleans = { true, false }) - void testClusterAuthorizationFailedInEndQuorumEpoch(boolean withKip853Rpc) throws Exception { + void testClusterAuthorizationFailedInEndQuorumEpoch(boolean isReconfigSupported) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; Set voters = Set.of(localId, otherNodeId); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withUnknownLeader(1) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(isReconfigSupported ? RaftProtocol.KIP_853_PROTOCOL : RaftProtocol.KIP_595_PROTOCOL) .build(); context.unattachedToLeader(); diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientFetchTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientFetchTest.java index ade509d8051f4..4e515c14886a7 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientFetchTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientFetchTest.java @@ -67,7 +67,7 @@ private static void testFetchResponseWithInvalidRecord(MemoryRecords records, in VoterSetTest.voterSet(Stream.of(local, electedLeader)), KRaftVersion.KRAFT_VERSION_1 ) .withElectedLeader(epoch, electedLeader.id()) - .withRaftProtocol(RaftClientTestContext.RaftProtocol.KIP_996_PROTOCOL) + .withRaftProtocol(RaftProtocol.KIP_996_PROTOCOL) .build(); context.pollUntilRequest(); @@ -102,7 +102,7 @@ void testReplicationOfHigherPartitionLeaderEpoch() throws Exception { VoterSetTest.voterSet(Stream.of(local, electedLeader)), KRaftVersion.KRAFT_VERSION_1 ) .withElectedLeader(epoch, electedLeader.id()) - .withRaftProtocol(RaftClientTestContext.RaftProtocol.KIP_996_PROTOCOL) + .withRaftProtocol(RaftProtocol.KIP_996_PROTOCOL) .build(); context.pollUntilRequest(); diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java index 4efe3e3600ae3..7fc146f30ffc1 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java @@ -21,7 +21,6 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.record.MemoryRecords; -import org.apache.kafka.raft.RaftClientTestContext.RaftProtocol; import org.apache.kafka.server.common.KRaftVersion; import org.junit.jupiter.params.ParameterizedTest; @@ -35,7 +34,7 @@ import static org.apache.kafka.raft.KafkaRaftClientTest.randomReplicaId; import static org.apache.kafka.raft.KafkaRaftClientTest.replicaKey; -import static org.apache.kafka.raft.RaftClientTestContext.RaftProtocol.KIP_996_PROTOCOL; +import static org.apache.kafka.raft.RaftProtocol.KIP_996_PROTOCOL; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; 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 0c4c1780919b2..ad6f941a7a8de 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java @@ -62,6 +62,7 @@ import java.util.stream.Stream; import static org.apache.kafka.raft.KafkaRaftClientTest.replicaKey; +import static org.apache.kafka.raft.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; @@ -79,7 +80,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(); @@ -154,7 +155,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(); @@ -215,7 +216,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(); @@ -245,7 +246,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(); @@ -314,7 +315,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(); @@ -398,7 +399,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(); @@ -435,7 +436,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(); @@ -463,7 +464,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(); @@ -493,7 +494,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(); @@ -549,7 +550,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(); @@ -587,7 +588,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(); @@ -632,7 +633,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(); @@ -677,7 +678,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(); @@ -749,7 +750,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(); @@ -809,7 +810,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(); @@ -869,7 +870,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(); @@ -922,7 +923,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(); @@ -977,7 +978,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(); @@ -1016,7 +1017,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(); @@ -1069,7 +1070,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(); @@ -1147,7 +1148,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(); @@ -1176,7 +1177,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(); @@ -1196,7 +1197,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(); @@ -1234,7 +1235,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(); @@ -1256,7 +1257,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(); @@ -1286,7 +1287,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(); @@ -1316,7 +1317,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(); @@ -1348,7 +1349,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()); @@ -1368,7 +1369,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(); @@ -1412,7 +1413,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(); @@ -1454,7 +1455,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(); @@ -1499,7 +1500,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(); @@ -1547,7 +1548,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(); @@ -1616,7 +1617,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) @@ -1652,7 +1653,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(); @@ -1703,7 +1704,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(); @@ -1736,7 +1737,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(); @@ -1769,7 +1770,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(); @@ -1798,7 +1799,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(); @@ -1843,7 +1844,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(); @@ -1894,7 +1895,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(); @@ -1945,7 +1946,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(); @@ -1996,7 +1997,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(); @@ -2079,7 +2080,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) @@ -2140,7 +2141,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(); @@ -2200,7 +2201,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) @@ -2259,7 +2260,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(List.of(bootstrapAddress))) @@ -2282,7 +2283,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(); @@ -2318,7 +2319,7 @@ public void testLeaderMetricsAreReset() 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(); 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 a0d45a349ee80..05374bb558529 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java @@ -37,6 +37,7 @@ 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.EnumSource; import org.junit.jupiter.params.provider.ValueSource; import java.io.IOException; @@ -94,14 +95,14 @@ public void testLatestSnapshotIdMissing() throws Exception { @ParameterizedTest @CsvSource({ "false,false", "false,true", "true,false", "true,true" }) - public void testLeaderListenerNotified(boolean entireLog, boolean withKip853Rpc) throws Exception { + public void testLeaderListenerNotified(boolean entireLog, boolean isReconfigSupported) 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) + .withRaftProtocol(isReconfigSupported ? RaftProtocol.KIP_853_PROTOCOL : RaftProtocol.KIP_595_PROTOCOL) .appendToLog(snapshotId.epoch(), List.of("a", "b", "c")) .appendToLog(snapshotId.epoch(), List.of("d", "e", "f")) .withEmptySnapshot(snapshotId); @@ -218,15 +219,15 @@ public void testSecondListenerNotified(boolean entireLog) throws Exception { } @ParameterizedTest - @ValueSource(booleans = { false, true }) - public void testListenerRenotified(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testListenerRenotified(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, raftProtocol.isReconfigSupported()); Set voters = Set.of(localId, otherNodeKey.id()); OffsetAndEpoch snapshotId = new OffsetAndEpoch(3, 1); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .appendToLog(snapshotId.epoch(), List.of("a", "b", "c")) .appendToLog(snapshotId.epoch(), List.of("d", "e", "f")) .appendToLog(snapshotId.epoch(), List.of("g", "h", "i")) @@ -274,16 +275,16 @@ public void testListenerRenotified(boolean withKip853Rpc) throws Exception { } @ParameterizedTest - @ValueSource(booleans = { false, true }) - public void testLeaderImmediatelySendsSnapshotId(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testLeaderImmediatelySendsSnapshotId(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, raftProtocol.isReconfigSupported()); Set voters = Set.of(localId, otherNodeKey.id()); OffsetAndEpoch snapshotId = new OffsetAndEpoch(3, 4); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withUnknownLeader(snapshotId.epoch()) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .appendToLog(snapshotId.epoch(), List.of("a", "b", "c")) .appendToLog(snapshotId.epoch(), List.of("d", "e", "f")) .appendToLog(snapshotId.epoch(), List.of("g", "h", "i")) @@ -308,15 +309,15 @@ public void testLeaderImmediatelySendsSnapshotId(boolean withKip853Rpc) throws E } @ParameterizedTest - @ValueSource(booleans = { false, true }) - public void testFetchRequestOffsetLessThanLogStart(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testFetchRequestOffsetLessThanLogStart(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, raftProtocol.isReconfigSupported()); Set voters = Set.of(localId, otherNodeKey.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withAppendLingerMs(1) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -357,16 +358,16 @@ public void testFetchRequestOffsetLessThanLogStart(boolean withKip853Rpc) throws } @ParameterizedTest - @ValueSource(booleans = { false, true }) - public void testFetchRequestOffsetAtZero(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + 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, raftProtocol.isReconfigSupported()); Set voters = Set.of(localId, otherNodeKey.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withAppendLingerMs(1) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -406,10 +407,10 @@ public void testFetchRequestOffsetAtZero(boolean withKip853Rpc) throws Exception } @ParameterizedTest - @ValueSource(booleans = { false, true }) - public void testFetchRequestWithLargerLastFetchedEpoch(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testFetchRequestWithLargerLastFetchedEpoch(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, raftProtocol.isReconfigSupported()); Set voters = Set.of(localId, otherNodeKey.id()); OffsetAndEpoch oldestSnapshotId = new OffsetAndEpoch(3, 2); @@ -418,7 +419,7 @@ public void testFetchRequestWithLargerLastFetchedEpoch(boolean withKip853Rpc) th .appendToLog(oldestSnapshotId.epoch(), List.of("a", "b", "c")) .appendToLog(oldestSnapshotId.epoch(), List.of("d", "e", "f")) .withAppendLingerMs(1) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -447,10 +448,10 @@ public void testFetchRequestWithLargerLastFetchedEpoch(boolean withKip853Rpc) th } @ParameterizedTest - @ValueSource(booleans = { false, true }) - public void testFetchRequestTruncateToLogStart(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testFetchRequestTruncateToLogStart(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, raftProtocol.isReconfigSupported()); int syncNodeId = otherNodeKey.id() + 1; Set voters = Set.of(localId, otherNodeKey.id(), syncNodeId); @@ -460,7 +461,7 @@ public void testFetchRequestTruncateToLogStart(boolean withKip853Rpc) throws Exc .appendToLog(oldestSnapshotId.epoch(), List.of("a", "b", "c")) .appendToLog(oldestSnapshotId.epoch() + 2, List.of("d", "e", "f")) .withAppendLingerMs(1) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -497,10 +498,10 @@ public void testFetchRequestTruncateToLogStart(boolean withKip853Rpc) throws Exc } @ParameterizedTest - @ValueSource(booleans = { false, true }) - public void testFetchRequestAtLogStartOffsetWithValidEpoch(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testFetchRequestAtLogStartOffsetWithValidEpoch(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, raftProtocol.isReconfigSupported()); int syncNodeId = otherNodeKey.id() + 1; Set voters = Set.of(localId, otherNodeKey.id(), syncNodeId); @@ -511,7 +512,7 @@ public void testFetchRequestAtLogStartOffsetWithValidEpoch(boolean withKip853Rpc .appendToLog(oldestSnapshotId.epoch(), List.of("d", "e", "f")) .appendToLog(oldestSnapshotId.epoch() + 2, List.of("g", "h", "i")) .withAppendLingerMs(1) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -543,10 +544,10 @@ public void testFetchRequestAtLogStartOffsetWithValidEpoch(boolean withKip853Rpc } @ParameterizedTest - @ValueSource(booleans = { false, true }) - public void testFetchRequestAtLogStartOffsetWithInvalidEpoch(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testFetchRequestAtLogStartOffsetWithInvalidEpoch(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, raftProtocol.isReconfigSupported()); int syncNodeId = otherNodeKey.id() + 1; Set voters = Set.of(localId, otherNodeKey.id(), syncNodeId); @@ -557,7 +558,7 @@ public void testFetchRequestAtLogStartOffsetWithInvalidEpoch(boolean withKip853R .appendToLog(oldestSnapshotId.epoch(), List.of("d", "e", "f")) .appendToLog(oldestSnapshotId.epoch() + 2, List.of("g", "h", "i")) .withAppendLingerMs(1) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -595,12 +596,10 @@ public void testFetchRequestAtLogStartOffsetWithInvalidEpoch(boolean withKip853R } @ParameterizedTest - @ValueSource(booleans = { false, true }) - public void testFetchRequestWithLastFetchedEpochLessThanOldestSnapshot( - boolean withKip853Rpc - ) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testFetchRequestWithLastFetchedEpochLessThanOldestSnapshot(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, raftProtocol.isReconfigSupported()); int syncNodeId = otherNodeKey.id() + 1; Set voters = Set.of(localId, otherNodeKey.id(), syncNodeId); @@ -611,7 +610,7 @@ public void testFetchRequestWithLastFetchedEpochLessThanOldestSnapshot( .appendToLog(oldestSnapshotId.epoch(), List.of("d", "e", "f")) .appendToLog(oldestSnapshotId.epoch() + 2, List.of("g", "h", "i")) .withAppendLingerMs(1) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -648,14 +647,14 @@ public void testFetchRequestWithLastFetchedEpochLessThanOldestSnapshot( } @ParameterizedTest - @ValueSource(booleans = { false, true }) - public void testFetchSnapshotRequestMissingSnapshot(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testFetchSnapshotRequestMissingSnapshot(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); Set voters = Set.of(localId, localId + 1); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withUnknownLeader(3) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -684,9 +683,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 RaftClientTestContext.Builder(localKey.id(), localKey.directoryId().get()) + .withRaftProtocol(RaftProtocol.KIP_853_PROTOCOL) .withBootstrapSnapshot(Optional.of(voters)) .withUnknownLeader(3) .build(); @@ -711,15 +709,15 @@ public void testFetchSnapshotRequestBootstrapSnapshot() throws Exception { } @ParameterizedTest - @ValueSource(booleans = { false, true }) - public void testFetchSnapshotRequestUnknownPartition(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + 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) .withUnknownLeader(3) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -742,8 +740,8 @@ public void testFetchSnapshotRequestUnknownPartition(boolean withKip853Rpc) thro } @ParameterizedTest - @ValueSource(booleans = { false, true }) - public void testFetchSnapshotRequestAsLeader(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testFetchSnapshotRequestAsLeader(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); Set voters = Set.of(localId, localId + 1); OffsetAndEpoch snapshotId = new OffsetAndEpoch(1, 1); @@ -751,7 +749,7 @@ public void testFetchSnapshotRequestAsLeader(boolean withKip853Rpc) throws Excep RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .appendToLog(snapshotId.epoch(), List.of("a")) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -793,21 +791,20 @@ public void testFetchSnapshotRequestAsLeader(boolean withKip853Rpc) throws Excep } @ParameterizedTest - @ValueSource(booleans = { false, true }) - public void testLeaderShouldResignLeadershipIfNotGetFetchSnapshotRequestFromMajorityVoters( - boolean withKip853Rpc - ) throws Exception { + @EnumSource(value = RaftProtocol.class) + 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 reconfigSupported = raftProtocol.isReconfigSupported(); + ReplicaKey voter1 = replicaKey(localId + 1, reconfigSupported); + ReplicaKey voter2 = replicaKey(localId + 2, reconfigSupported); + ReplicaKey observer3 = replicaKey(localId + 3, reconfigSupported); Set voters = Set.of(localId, voter1.id(), voter2.id()); OffsetAndEpoch snapshotId = new OffsetAndEpoch(1, 1); List records = List.of("foo", "bar"); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .appendToLog(snapshotId.epoch(), List.of("a")) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); int resignLeadershipTimeout = context.checkQuorumTimeoutMs; @@ -888,8 +885,8 @@ public void testLeaderShouldResignLeadershipIfNotGetFetchSnapshotRequestFromMajo } @ParameterizedTest - @ValueSource(booleans = { false, true }) - public void testPartialFetchSnapshotRequestAsLeader(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testPartialFetchSnapshotRequestAsLeader(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); Set voters = Set.of(localId, localId + 1); OffsetAndEpoch snapshotId = new OffsetAndEpoch(2, 1); @@ -897,7 +894,7 @@ public void testPartialFetchSnapshotRequestAsLeader(boolean withKip853Rpc) throw RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .appendToLog(snapshotId.epoch(), records) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -969,8 +966,8 @@ public void testPartialFetchSnapshotRequestAsLeader(boolean withKip853Rpc) throw } @ParameterizedTest - @ValueSource(booleans = { false, true }) - public void testFetchSnapshotRequestAsFollower(boolean withKip853Rpc) throws IOException { + @EnumSource(value = RaftProtocol.class) + public void testFetchSnapshotRequestAsFollower(RaftProtocol raftProtocol) throws IOException { int localId = randomReplicaId(); int leaderId = localId + 1; Set voters = Set.of(localId, leaderId); @@ -979,7 +976,7 @@ public void testFetchSnapshotRequestAsFollower(boolean withKip853Rpc) throws IOE RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withElectedLeader(epoch, leaderId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.deliverRequest( @@ -1001,8 +998,8 @@ public void testFetchSnapshotRequestAsFollower(boolean withKip853Rpc) throws IOE } @ParameterizedTest - @ValueSource(booleans = { false, true }) - public void testFetchSnapshotRequestWithInvalidPosition(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testFetchSnapshotRequestWithInvalidPosition(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); Set voters = Set.of(localId, localId + 1); OffsetAndEpoch snapshotId = new OffsetAndEpoch(1, 1); @@ -1010,7 +1007,7 @@ public void testFetchSnapshotRequestWithInvalidPosition(boolean withKip853Rpc) t RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .appendToLog(snapshotId.epoch(), List.of("a")) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -1061,15 +1058,15 @@ public void testFetchSnapshotRequestWithInvalidPosition(boolean withKip853Rpc) t } @ParameterizedTest - @ValueSource(booleans = { false, true }) - public void testFetchSnapshotRequestWithOlderEpoch(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + 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) .withUnknownLeader(1) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -1094,15 +1091,15 @@ public void testFetchSnapshotRequestWithOlderEpoch(boolean withKip853Rpc) throws } @ParameterizedTest - @ValueSource(booleans = { false, true }) - public void testFetchSnapshotRequestWithNewerEpoch(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + 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) .withUnknownLeader(1) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -1127,8 +1124,8 @@ public void testFetchSnapshotRequestWithNewerEpoch(boolean withKip853Rpc) throws } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testFetchResponseWithInvalidSnapshotId(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testFetchResponseWithInvalidSnapshotId(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int leaderId = localId + 1; Set voters = Set.of(localId, leaderId); @@ -1139,7 +1136,7 @@ public void testFetchResponseWithInvalidSnapshotId(boolean withKip853Rpc) throws RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withElectedLeader(epoch, leaderId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.pollUntilRequest(); @@ -1190,8 +1187,8 @@ public void testFetchResponseWithInvalidSnapshotId(boolean withKip853Rpc) throws } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testFetchResponseWithSnapshotId(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testFetchResponseWithSnapshotId(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int leaderId = localId + 1; Set voters = Set.of(localId, leaderId); @@ -1200,7 +1197,7 @@ public void testFetchResponseWithSnapshotId(boolean withKip853Rpc) throws Except RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withElectedLeader(epoch, leaderId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.pollUntilRequest(); @@ -1263,8 +1260,8 @@ public void testFetchResponseWithSnapshotId(boolean withKip853Rpc) throws Except } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testFetchSnapshotResponsePartialData(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testFetchSnapshotResponsePartialData(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int leaderId = localId + 1; Set voters = Set.of(localId, leaderId); @@ -1273,7 +1270,7 @@ public void testFetchSnapshotResponsePartialData(boolean withKip853Rpc) throws E RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withElectedLeader(epoch, leaderId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.pollUntilRequest(); @@ -1368,8 +1365,8 @@ public void testFetchSnapshotResponsePartialData(boolean withKip853Rpc) throws E } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testFetchSnapshotResponseMissingSnapshot(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testFetchSnapshotResponseMissingSnapshot(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int leaderId = localId + 1; Set voters = Set.of(localId, leaderId); @@ -1378,7 +1375,7 @@ public void testFetchSnapshotResponseMissingSnapshot(boolean withKip853Rpc) thro RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withElectedLeader(epoch, leaderId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.pollUntilRequest(); @@ -1429,8 +1426,8 @@ public void testFetchSnapshotResponseMissingSnapshot(boolean withKip853Rpc) thro } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testFetchSnapshotResponseFromNewerEpochNotLeader(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testFetchSnapshotResponseFromNewerEpochNotLeader(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int firstLeaderId = localId + 1; int secondLeaderId = firstLeaderId + 1; @@ -1440,7 +1437,7 @@ public void testFetchSnapshotResponseFromNewerEpochNotLeader(boolean withKip853R RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withElectedLeader(epoch, firstLeaderId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.pollUntilRequest(); @@ -1491,8 +1488,8 @@ public void testFetchSnapshotResponseFromNewerEpochNotLeader(boolean withKip853R } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testFetchSnapshotResponseFromNewerEpochLeader(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testFetchSnapshotResponseFromNewerEpochLeader(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int leaderId = localId + 1; Set voters = Set.of(localId, leaderId); @@ -1501,7 +1498,7 @@ public void testFetchSnapshotResponseFromNewerEpochLeader(boolean withKip853Rpc) RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withElectedLeader(epoch, leaderId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.pollUntilRequest(); @@ -1552,8 +1549,8 @@ public void testFetchSnapshotResponseFromNewerEpochLeader(boolean withKip853Rpc) } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testFetchSnapshotResponseFromOlderEpoch(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testFetchSnapshotResponseFromOlderEpoch(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int leaderId = localId + 1; Set voters = Set.of(localId, leaderId); @@ -1562,7 +1559,7 @@ public void testFetchSnapshotResponseFromOlderEpoch(boolean withKip853Rpc) throw RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withElectedLeader(epoch, leaderId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.pollUntilRequest(); @@ -1623,8 +1620,8 @@ public void testFetchSnapshotResponseFromOlderEpoch(boolean withKip853Rpc) throw } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testFetchSnapshotResponseWithInvalidId(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testFetchSnapshotResponseWithInvalidId(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int leaderId = localId + 1; Set voters = Set.of(localId, leaderId); @@ -1633,7 +1630,7 @@ public void testFetchSnapshotResponseWithInvalidId(boolean withKip853Rpc) throws RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withElectedLeader(epoch, leaderId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.pollUntilRequest(); @@ -1739,8 +1736,8 @@ public void testFetchSnapshotResponseWithInvalidId(boolean withKip853Rpc) throws } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testFetchSnapshotResponseToNotFollower(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testFetchSnapshotResponseToNotFollower(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int leaderId = localId + 1; Set voters = Set.of(localId, leaderId); @@ -1749,7 +1746,7 @@ public void testFetchSnapshotResponseToNotFollower(boolean withKip853Rpc) throws RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withElectedLeader(epoch, leaderId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.pollUntilRequest(); @@ -1813,17 +1810,15 @@ public void testFetchSnapshotResponseToNotFollower(boolean withKip853Rpc) throws } @ParameterizedTest - @ValueSource(booleans = { false, true }) - public void testFetchSnapshotRequestClusterIdValidation( - boolean withKip853Rpc - ) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testFetchSnapshotRequestClusterIdValidation(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNode = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNode = replicaKey(localId + 1, raftProtocol.isReconfigSupported()); Set voters = Set.of(localId, otherNode.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withUnknownLeader(4) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -1891,8 +1886,8 @@ public void testFetchSnapshotRequestClusterIdValidation( } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testCreateSnapshotAsLeaderWithInvalidSnapshotId(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testCreateSnapshotAsLeaderWithInvalidSnapshotId(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; Set voters = Set.of(localId, otherNodeId); @@ -1904,7 +1899,7 @@ public void testCreateSnapshotAsLeaderWithInvalidSnapshotId(boolean withKip853Rp RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .appendToLog(epoch, appendRecords) .withAppendLingerMs(1) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -1981,8 +1976,8 @@ public void testCreateSnapshotAsLeaderWithInvalidSnapshotId(boolean withKip853Rp } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testCreateSnapshotAsFollowerWithInvalidSnapshotId(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testCreateSnapshotAsFollowerWithInvalidSnapshotId(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int leaderId = localId + 1; int otherFollowerId = localId + 2; @@ -1992,7 +1987,7 @@ public void testCreateSnapshotAsFollowerWithInvalidSnapshotId(boolean withKip853 RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .appendToLog(1, List.of("a")) .withElectedLeader(epoch, leaderId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.assertElectedLeader(epoch, leaderId); 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 a1e92f6afcf6c..b47c27548bb80 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java @@ -43,9 +43,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; @@ -64,7 +64,8 @@ import java.util.stream.Stream; import static org.apache.kafka.raft.RaftClientTestContext.Builder.DEFAULT_ELECTION_TIMEOUT_MS; -import static org.apache.kafka.raft.RaftClientTestContext.RaftProtocol.KIP_853_PROTOCOL; +import static org.apache.kafka.raft.RaftProtocol.KIP_595_PROTOCOL; +import static org.apache.kafka.raft.RaftProtocol.KIP_853_PROTOCOL; import static org.apache.kafka.test.TestUtils.assertFutureThrows; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -86,25 +87,25 @@ public void testNodeDirectoryId() { } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testInitializeSingleMemberQuorum(boolean withKip853Rpc) throws IOException { + @EnumSource(value = RaftProtocol.class) + public void testInitializeSingleMemberQuorum(RaftProtocol raftProtocol) throws IOException { int localId = randomReplicaId(); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, Set.of(localId)) - .withKip853Rpc(withKip853Rpc) + .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) + 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 = Set.of(localId); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .withElectedLeader(initialEpoch, localId) .build(); @@ -117,18 +118,18 @@ public void testInitializeAsLeaderFromStateStoreSingleMemberQuorum(boolean withK } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testRejectVotesFromSameEpochAfterResigningLeadership(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testRejectVotesFromSameEpochAfterResigningLeadership(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int remoteId = localId + 1; - ReplicaKey remoteKey = replicaKey(remoteId, withKip853Rpc); + ReplicaKey remoteKey = replicaKey(remoteId, raftProtocol.isReconfigSupported()); Set voters = Set.of(localId, remoteKey.id()); int epoch = 2; RaftClientTestContext context = new RaftClientTestContext.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()); @@ -149,18 +150,18 @@ public void testRejectVotesFromSameEpochAfterResigningLeadership(boolean withKip } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testRejectVotesFromSameEpochAfterResigningCandidacy(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testRejectVotesFromSameEpochAfterResigningCandidacy(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int remoteId = localId + 1; - ReplicaKey remoteKey = replicaKey(remoteId, withKip853Rpc); + ReplicaKey remoteKey = replicaKey(remoteId, raftProtocol.isReconfigSupported()); Set voters = Set.of(localId, remoteKey.id()); int epoch = 2; RaftClientTestContext context = new RaftClientTestContext.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()); @@ -181,18 +182,18 @@ public void testRejectVotesFromSameEpochAfterResigningCandidacy(boolean withKip8 } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testGrantVotesFromHigherEpochAfterResigningLeadership(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testGrantVotesFromHigherEpochAfterResigningLeadership(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int remoteId = localId + 1; - ReplicaKey remoteKey = replicaKey(remoteId, withKip853Rpc); + ReplicaKey remoteKey = replicaKey(remoteId, raftProtocol.isReconfigSupported()); Set voters = Set.of(localId, remoteKey.id()); int epoch = 2; RaftClientTestContext context = new RaftClientTestContext.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 @@ -218,18 +219,18 @@ public void testGrantVotesFromHigherEpochAfterResigningLeadership(boolean withKi } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testGrantVotesFromHigherEpochAfterResigningCandidacy(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testGrantVotesFromHigherEpochAfterResigningCandidacy(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int remoteId = localId + 1; - ReplicaKey remoteKey = replicaKey(remoteId, withKip853Rpc); + ReplicaKey remoteKey = replicaKey(remoteId, raftProtocol.isReconfigSupported()); Set voters = Set.of(localId, remoteKey.id()); int epoch = 2; RaftClientTestContext context = new RaftClientTestContext.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 @@ -255,15 +256,15 @@ public void testGrantVotesFromHigherEpochAfterResigningCandidacy(boolean withKip } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testGrantVotesWhenShuttingDown(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testGrantVotesWhenShuttingDown(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int remoteId = localId + 1; - ReplicaKey remoteKey = replicaKey(remoteId, withKip853Rpc); + ReplicaKey remoteKey = replicaKey(remoteId, raftProtocol.isReconfigSupported()); Set voters = Set.of(localId, remoteKey.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -297,8 +298,8 @@ public void testGrantVotesWhenShuttingDown(boolean withKip853Rpc) throws Excepti } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testInitializeAsResignedAndUnableToContactQuorum(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testInitializeAsResignedAndUnableToContactQuorum(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int remoteId = localId + 1; Set voters = Set.of(localId, remoteId); @@ -307,7 +308,7 @@ public void testInitializeAsResignedAndUnableToContactQuorum(boolean withKip853R RaftClientTestContext context = new RaftClientTestContext.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 @@ -335,8 +336,8 @@ public void testInitializeAsResignedAndUnableToContactQuorum(boolean withKip853R } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testInitializeAsResignedLeaderFromStateStore(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testInitializeAsResignedLeaderFromStateStore(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int remoteId = localId + 1; Set voters = Set.of(localId, remoteId); @@ -344,7 +345,7 @@ public void testInitializeAsResignedLeaderFromStateStore(boolean withKip853Rpc) RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .updateRandom(r -> r.mockNextInt(DEFAULT_ELECTION_TIMEOUT_MS, 0)) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .withElectedLeader(epoch, localId) .build(); @@ -374,8 +375,8 @@ public void testInitializeAsResignedLeaderFromStateStore(boolean withKip853Rpc) } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testAppendFailedWithNotLeaderException(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testAppendFailedWithNotLeaderException(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int remoteId = localId + 1; Set voters = Set.of(localId, remoteId); @@ -383,15 +384,15 @@ public void testAppendFailedWithNotLeaderException(boolean withKip853Rpc) throws RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withUnknownLeader(epoch) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); assertThrows(NotLeaderException.class, () -> context.client.prepareAppend(epoch, List.of("a", "b"))); } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testAppendFailedWithBufferAllocationException(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testAppendFailedWithBufferAllocationException(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; Set voters = Set.of(localId, otherNodeId); @@ -405,7 +406,7 @@ public void testAppendFailedWithBufferAllocationException(boolean withKip853Rpc) RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withMemoryPool(memoryPool) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -417,14 +418,14 @@ public void testAppendFailedWithBufferAllocationException(boolean withKip853Rpc) } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testAppendFailedWithFencedEpoch(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + 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) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -438,14 +439,14 @@ public void testAppendFailedWithFencedEpoch(boolean withKip853Rpc) throws Except } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testAppendFailedWithRecordBatchTooLargeException(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + 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) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -464,8 +465,8 @@ public void testAppendFailedWithRecordBatchTooLargeException(boolean withKip853R } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testEndQuorumEpochRetriesWhileResigned(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testEndQuorumEpochRetriesWhileResigned(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int voter1 = localId + 1; int voter2 = localId + 2; @@ -480,7 +481,7 @@ public void testEndQuorumEpochRetriesWhileResigned(boolean withKip853Rpc) throws .withElectionTimeoutMs(10000) .withRequestTimeoutMs(5000) .withElectedLeader(epoch, localId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.pollUntilRequest(); @@ -510,15 +511,15 @@ public void testEndQuorumEpochRetriesWhileResigned(boolean withKip853Rpc) throws } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testResignWillCompleteFetchPurgatory(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testResignWillCompleteFetchPurgatory(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int remoteId = localId + 1; - ReplicaKey otherNodeKey = replicaKey(remoteId, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(remoteId, raftProtocol.isReconfigSupported()); Set voters = Set.of(localId, otherNodeKey.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -549,14 +550,14 @@ public void testResignWillCompleteFetchPurgatory(boolean withKip853Rpc) throws E } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testResignInOlderEpochIgnored(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + 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) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -573,9 +574,9 @@ public void testResignInOlderEpochIgnored(boolean withKip853Rpc) throws Exceptio } @ParameterizedTest - @ValueSource(booleans = { true, false }) + @EnumSource(value = RaftProtocol.class) public void testHandleBeginQuorumEpochAfterUserInitiatedResign( - boolean withKip853Rpc + RaftProtocol raftProtocol ) throws Exception { int localId = randomReplicaId(); int remoteId1 = localId + 1; @@ -583,7 +584,7 @@ public void testHandleBeginQuorumEpochAfterUserInitiatedResign( Set voters = Set.of(localId, remoteId1, remoteId2); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -603,15 +604,15 @@ public void testHandleBeginQuorumEpochAfterUserInitiatedResign( } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testBeginQuorumEpochHeartbeat(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + 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) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -634,19 +635,20 @@ public void testBeginQuorumEpochHeartbeat(boolean withKip853Rpc) throws Exceptio } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testLeaderShouldResignLeadershipIfNotGetFetchRequestFromMajorityVoters(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + 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); + boolean reconfigSupported = raftProtocol.isReconfigSupported(); + ReplicaKey remoteKey1 = replicaKey(remoteId1, reconfigSupported); + ReplicaKey remoteKey2 = replicaKey(remoteId2, reconfigSupported); + ReplicaKey observerKey3 = replicaKey(observerId, reconfigSupported); Set voters = Set.of(localId, remoteKey1.id(), remoteKey2.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); int resignLeadershipTimeout = context.checkQuorumTimeoutMs; @@ -694,13 +696,13 @@ public void testLeaderShouldResignLeadershipIfNotGetFetchRequestFromMajorityVote } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testLeaderShouldNotResignLeadershipIfOnlyOneVoters(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testLeaderShouldNotResignLeadershipIfOnlyOneVoters(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); Set voters = Set.of(localId); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); assertEquals(OptionalInt.of(localId), context.currentLeader()); @@ -713,14 +715,14 @@ public void testLeaderShouldNotResignLeadershipIfOnlyOneVoters(boolean withKip85 } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testElectionTimeoutAfterUserInitiatedResign(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + 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) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -748,7 +750,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, raftProtocol.isReconfigSupported()); context.deliverRequest(context.fetchRequest(1, observer, 0, 0, 0)); context.pollUntilResponse(); context.assertSentFetchPartitionResponse( @@ -771,14 +773,14 @@ public void testElectionTimeoutAfterUserInitiatedResign(boolean withKip853Rpc) t } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testCannotResignWithLargerEpochThanCurrentEpoch(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + 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) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -787,8 +789,8 @@ public void testCannotResignWithLargerEpochThanCurrentEpoch(boolean withKip853Rp } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testCannotResignIfNotLeader(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testCannotResignIfNotLeader(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int leaderEpoch = 2; @@ -796,7 +798,7 @@ public void testCannotResignIfNotLeader(boolean withKip853Rpc) throws Exception RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withElectedLeader(leaderEpoch, otherNodeId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); assertEquals(OptionalInt.of(otherNodeId), context.currentLeader()); @@ -804,15 +806,15 @@ public void testCannotResignIfNotLeader(boolean withKip853Rpc) throws Exception } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testCannotResignIfObserver(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + 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) + .withRaftProtocol(raftProtocol) .build(); context.pollUntilRequest(); @@ -832,15 +834,15 @@ public void testCannotResignIfObserver(boolean withKip853Rpc) throws Exception { } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testInitializeAsCandidateFromStateStore(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + 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) .withVotedCandidate(2, ReplicaKey.of(localId, ReplicaKey.NO_DIRECTORY_ID)) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.assertVotedCandidate(2, localId); assertEquals(0L, context.log.endOffset().offset()); @@ -852,13 +854,13 @@ public void testInitializeAsCandidateFromStateStore(boolean withKip853Rpc) throw } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testInitializeAsUnattachedAndBecomeLeader(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + 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) + .withRaftProtocol(raftProtocol) .build(); context.assertUnknownLeaderAndNoVotedCandidate(0); @@ -921,15 +923,15 @@ public void testInitializeAsUnattachedAndBecomeLeader(boolean withKip853Rpc) thr } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testInitializeAsCandidateAndBecomeLeaderQuorumOfThree(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + 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) .withVotedCandidate(2, ReplicaKey.of(localId, ReplicaKey.NO_DIRECTORY_ID)) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); assertTrue(context.client.quorum().isCandidate()); context.pollUntilRequest(); @@ -970,11 +972,11 @@ public void testInitializeAsCandidateAndBecomeLeaderQuorumOfThree(boolean withKi } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testInitializeAsOnlyVoterWithEmptyElectionState(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testInitializeAsOnlyVoterWithEmptyElectionState(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, Set.of(localId)) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.assertElectedLeader(1, localId); assertEquals(0L, context.log.endOffset().offset()); @@ -1016,16 +1018,16 @@ public void testInitializeAsResignedAndOnlyVoter() throws Exception { } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testHandleBeginQuorumRequest(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testHandleBeginQuorumRequest(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, raftProtocol.isReconfigSupported()); int votedCandidateEpoch = 2; Set voters = Set.of(localId, otherNodeKey.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withVotedCandidate(votedCandidateEpoch, otherNodeKey) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.deliverRequest(context.beginEpochRequest(votedCandidateEpoch, otherNodeKey.id())); @@ -1051,7 +1053,7 @@ public void testHandleBeginQuorumRequestMoreEndpoints() throws Exception { RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) .withStaticVoters(voters) .withElectedLeader(leaderEpoch, leader.id()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .build(); context.client.poll(); @@ -1080,8 +1082,8 @@ public void testHandleBeginQuorumRequestMoreEndpoints() throws Exception { } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testHandleBeginQuorumResponse(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testHandleBeginQuorumResponse(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int leaderEpoch = 2; @@ -1089,7 +1091,7 @@ public void testHandleBeginQuorumResponse(boolean withKip853Rpc) throws Exceptio RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withElectedLeader(leaderEpoch, localId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.deliverRequest(context.beginEpochRequest(leaderEpoch + 1, otherNodeId)); @@ -1099,8 +1101,8 @@ public void testHandleBeginQuorumResponse(boolean withKip853Rpc) throws Exceptio } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testEndQuorumIgnoredAsCandidateIfOlderEpoch(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testEndQuorumIgnoredAsCandidateIfOlderEpoch(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int epoch = 5; @@ -1110,7 +1112,7 @@ public void testEndQuorumIgnoredAsCandidateIfOlderEpoch(boolean withKip853Rpc) t RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .updateRandom(r -> r.mockNextInt(jitterMs)) .withUnknownLeader(epoch - 1) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToCandidate(); @@ -1144,11 +1146,11 @@ public void testEndQuorumIgnoredAsCandidateIfOlderEpoch(boolean withKip853Rpc) t } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testEndQuorumIgnoredAsLeaderIfOlderEpoch(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + 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, raftProtocol.isReconfigSupported()); Set voters = Set.of(localId, voter2, voter3.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) @@ -1173,19 +1175,19 @@ public void testEndQuorumIgnoredAsLeaderIfOlderEpoch(boolean withKip853Rpc) thro } @ParameterizedTest - @ValueSource(booleans = { true, false }) + @EnumSource(value = RaftProtocol.class) 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, raftProtocol.isReconfigSupported()); int epoch = 2; Set voters = Set.of(localId, voter2, voter3.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withUnknownLeader(epoch) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.deliverRequest( @@ -1205,8 +1207,8 @@ public void testEndQuorumStartsNewElectionImmediatelyIfFollowerUnattached( } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testAccumulatorClearedAfterBecomingFollower(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testAccumulatorClearedAfterBecomingFollower(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int lingerMs = 50; @@ -1220,7 +1222,7 @@ public void testAccumulatorClearedAfterBecomingFollower(boolean withKip853Rpc) t RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withAppendLingerMs(lingerMs) .withMemoryPool(memoryPool) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -1238,10 +1240,10 @@ public void testAccumulatorClearedAfterBecomingFollower(boolean withKip853Rpc) t } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testAccumulatorClearedAfterBecomingVoted(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testAccumulatorClearedAfterBecomingVoted(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, raftProtocol.isReconfigSupported()); int lingerMs = 50; Set voters = Set.of(localId, otherNodeKey.id()); @@ -1253,7 +1255,7 @@ public void testAccumulatorClearedAfterBecomingVoted(boolean withKip853Rpc) thro RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withAppendLingerMs(lingerMs) .withMemoryPool(memoryPool) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -1272,10 +1274,10 @@ public void testAccumulatorClearedAfterBecomingVoted(boolean withKip853Rpc) thro } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testAccumulatorClearedAfterBecomingUnattached(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testAccumulatorClearedAfterBecomingUnattached(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, raftProtocol.isReconfigSupported()); int lingerMs = 50; Set voters = Set.of(localId, otherNodeKey.id()); @@ -1287,7 +1289,7 @@ public void testAccumulatorClearedAfterBecomingUnattached(boolean withKip853Rpc) RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withAppendLingerMs(lingerMs) .withMemoryPool(memoryPool) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -1305,8 +1307,8 @@ public void testAccumulatorClearedAfterBecomingUnattached(boolean withKip853Rpc) } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testChannelWokenUpIfLingerTimeoutReachedWithoutAppend(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + 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(); @@ -1316,7 +1318,7 @@ public void testChannelWokenUpIfLingerTimeoutReachedWithoutAppend(boolean withKi RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withAppendLingerMs(lingerMs) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -1341,8 +1343,8 @@ public void testChannelWokenUpIfLingerTimeoutReachedWithoutAppend(boolean withKi } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testChannelWokenUpIfLingerTimeoutReachedDuringAppend(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + 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(); @@ -1352,7 +1354,7 @@ public void testChannelWokenUpIfLingerTimeoutReachedDuringAppend(boolean withKip RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withAppendLingerMs(lingerMs) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -1378,8 +1380,8 @@ public void testChannelWokenUpIfLingerTimeoutReachedDuringAppend(boolean withKip } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testHandleEndQuorumRequest(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testHandleEndQuorumRequest(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int oldLeaderId = localId + 1; int leaderEpoch = 2; @@ -1387,7 +1389,7 @@ public void testHandleEndQuorumRequest(boolean withKip853Rpc) throws Exception { RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withElectedLeader(leaderEpoch, oldLeaderId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.deliverRequest( @@ -1407,17 +1409,18 @@ public void testHandleEndQuorumRequest(boolean withKip853Rpc) throws Exception { } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testHandleEndQuorumRequestWithLowerPriorityToBecomeLeader(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testHandleEndQuorumRequestWithLowerPriorityToBecomeLeader(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey oldLeaderKey = replicaKey(localId + 1, withKip853Rpc); + boolean reconfigSupported = raftProtocol.isReconfigSupported(); + ReplicaKey oldLeaderKey = replicaKey(localId + 1, reconfigSupported); int leaderEpoch = 2; - ReplicaKey preferredNextLeader = replicaKey(localId + 2, withKip853Rpc); + ReplicaKey preferredNextLeader = replicaKey(localId + 2, reconfigSupported); Set voters = Set.of(localId, oldLeaderKey.id(), preferredNextLeader.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withElectedLeader(leaderEpoch, oldLeaderKey.id()) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.deliverRequest( @@ -1450,15 +1453,15 @@ public void testHandleEndQuorumRequestWithLowerPriorityToBecomeLeader(boolean wi } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testVoteRequestTimeout(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + 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) + .withRaftProtocol(raftProtocol) .build(); context.assertUnknownLeaderAndNoVotedCandidate(0); @@ -1492,16 +1495,16 @@ public void testVoteRequestTimeout(boolean withKip853Rpc) throws Exception { } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testHandleValidVoteRequestAsFollower(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testHandleValidVoteRequestAsFollower(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int epoch = 2; - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, raftProtocol.isReconfigSupported()); Set voters = Set.of(localId, otherNodeKey.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withUnknownLeader(epoch) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.deliverRequest(context.voteRequest(epoch, otherNodeKey, epoch - 1, 1)); @@ -1513,17 +1516,17 @@ public void testHandleValidVoteRequestAsFollower(boolean withKip853Rpc) throws E } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testHandleVoteRequestAsFollowerWithElectedLeader(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testHandleVoteRequestAsFollowerWithElectedLeader(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int epoch = 2; - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, raftProtocol.isReconfigSupported()); int electedLeaderId = localId + 2; Set voters = Set.of(localId, otherNodeKey.id(), electedLeaderId); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withElectedLeader(epoch, electedLeaderId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.deliverRequest(context.voteRequest(epoch, otherNodeKey, epoch - 1, 1)); @@ -1535,17 +1538,18 @@ public void testHandleVoteRequestAsFollowerWithElectedLeader(boolean withKip853R } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testHandleVoteRequestAsFollowerWithVotedCandidate(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + 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); + boolean reconfigSupported = raftProtocol.isReconfigSupported(); + ReplicaKey otherNodeKey = replicaKey(localId + 1, reconfigSupported); + ReplicaKey votedCandidateKey = replicaKey(localId + 2, reconfigSupported); Set voters = Set.of(localId, otherNodeKey.id(), votedCandidateKey.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withVotedCandidate(epoch, votedCandidateKey) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.deliverRequest(context.voteRequest(epoch, otherNodeKey, epoch - 1, 1)); @@ -1556,17 +1560,17 @@ public void testHandleVoteRequestAsFollowerWithVotedCandidate(boolean withKip853 } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testHandleVoteRequestAsProspective(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testHandleVoteRequestAsProspective(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int epoch = 2; - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, raftProtocol.isReconfigSupported()); int electedLeaderId = localId + 2; Set voters = Set.of(localId, otherNodeKey.id(), electedLeaderId); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withElectedLeader(epoch, electedLeaderId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); // Sleep a little to ensure that we become a prospective @@ -1584,17 +1588,18 @@ public void testHandleVoteRequestAsProspective(boolean withKip853Rpc) throws Exc } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testHandleVoteRequestAsProspectiveWithVotedCandidate(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + 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); + boolean reconfigSupported = raftProtocol.isReconfigSupported(); + ReplicaKey otherNodeKey = replicaKey(localId + 1, reconfigSupported); + ReplicaKey votedCandidateKey = replicaKey(localId + 2, reconfigSupported); Set voters = Set.of(localId, otherNodeKey.id(), votedCandidateKey.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withVotedCandidate(epoch, votedCandidateKey) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); // Sleep a little to ensure that we become a prospective @@ -1611,16 +1616,16 @@ public void testHandleVoteRequestAsProspectiveWithVotedCandidate(boolean withKip } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testHandleInvalidVoteRequestWithOlderEpoch(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testHandleInvalidVoteRequestWithOlderEpoch(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int epoch = 2; - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, raftProtocol.isReconfigSupported()); Set voters = Set.of(localId, otherNodeKey.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withUnknownLeader(epoch) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.deliverRequest(context.voteRequest(epoch - 1, otherNodeKey, epoch - 2, 1)); @@ -1631,17 +1636,17 @@ public void testHandleInvalidVoteRequestWithOlderEpoch(boolean withKip853Rpc) th } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testHandleVoteRequestAsObserver(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testHandleVoteRequestAsObserver(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int epoch = 2; - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, raftProtocol.isReconfigSupported()); int otherNodeId2 = localId + 2; Set voters = Set.of(otherNodeKey.id(), otherNodeId2); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withUnknownLeader(epoch) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.deliverRequest(context.voteRequest(epoch + 1, otherNodeKey, epoch, 1)); @@ -1652,15 +1657,15 @@ public void testHandleVoteRequestAsObserver(boolean withKip853Rpc) throws Except } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testLeaderIgnoreVoteRequestOnSameEpoch(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testLeaderIgnoreVoteRequestOnSameEpoch(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, raftProtocol.isReconfigSupported()); Set voters = Set.of(localId, otherNodeKey.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withUnknownLeader(2) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -1675,15 +1680,15 @@ public void testLeaderIgnoreVoteRequestOnSameEpoch(boolean withKip853Rpc) throws } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testListenerCommitCallbackAfterLeaderWrite(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testListenerCommitCallbackAfterLeaderWrite(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, raftProtocol.isReconfigSupported()); Set voters = Set.of(localId, otherNodeKey.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withUnknownLeader(4) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -1721,15 +1726,15 @@ public void testListenerCommitCallbackAfterLeaderWrite(boolean withKip853Rpc) th } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testLeaderImmediatelySendsDivergingEpoch(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testLeaderImmediatelySendsDivergingEpoch(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, raftProtocol.isReconfigSupported()); Set voters = Set.of(localId, otherNodeKey.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withUnknownLeader(5) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .appendToLog(1, List.of("a", "b", "c")) .appendToLog(3, List.of("d", "e", "f")) .appendToLog(5, List.of("g", "h", "i")) @@ -1753,16 +1758,16 @@ public void testLeaderImmediatelySendsDivergingEpoch(boolean withKip853Rpc) thro } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testCandidateIgnoreVoteRequestOnSameEpoch(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testCandidateIgnoreVoteRequestOnSameEpoch(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, raftProtocol.isReconfigSupported()); int leaderEpoch = 2; Set voters = Set.of(localId, otherNodeKey.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withVotedCandidate(leaderEpoch, ReplicaKey.of(localId, ReplicaKey.NO_DIRECTORY_ID)) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.pollUntilRequest(); @@ -1774,8 +1779,8 @@ public void testCandidateIgnoreVoteRequestOnSameEpoch(boolean withKip853Rpc) thr } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testCandidateBackoffElection(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testCandidateBackoffElection(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int epoch = 1; @@ -1784,7 +1789,7 @@ public void testCandidateBackoffElection(boolean withKip853Rpc) throws Exception RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .updateRandom(r -> r.mockNextInt(exponentialFactor)) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.assertUnknownLeaderAndNoVotedCandidate(0); @@ -1850,8 +1855,8 @@ public void testCandidateBackoffElection(boolean withKip853Rpc) throws Exception } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testCandidateElectionTimeout(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testCandidateElectionTimeout(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int epoch = 1; @@ -1860,7 +1865,7 @@ public void testCandidateElectionTimeout(boolean withKip853Rpc) throws Exception RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .updateRandom(r -> r.mockNextInt(jitter)) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.assertUnknownLeaderAndNoVotedCandidate(0); @@ -1893,8 +1898,8 @@ public void testCandidateElectionTimeout(boolean withKip853Rpc) throws Exception } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testInitializeAsFollowerEmptyLog(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testInitializeAsFollowerEmptyLog(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int epoch = 5; @@ -1902,7 +1907,7 @@ public void testInitializeAsFollowerEmptyLog(boolean withKip853Rpc) throws Excep RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withElectedLeader(epoch, otherNodeId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.assertElectedLeader(epoch, otherNodeId); @@ -1913,8 +1918,8 @@ public void testInitializeAsFollowerEmptyLog(boolean withKip853Rpc) throws Excep } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testInitializeAsFollowerNonEmptyLog(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testInitializeAsFollowerNonEmptyLog(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int epoch = 5; @@ -1924,7 +1929,7 @@ public void testInitializeAsFollowerNonEmptyLog(boolean withKip853Rpc) throws Ex RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withElectedLeader(epoch, otherNodeId) .appendToLog(lastEpoch, List.of("foo")) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.assertElectedLeader(epoch, otherNodeId); @@ -1934,8 +1939,8 @@ public void testInitializeAsFollowerNonEmptyLog(boolean withKip853Rpc) throws Ex } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testVoterBecomeProspectiveAfterFetchTimeout(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testVoterBecomeProspectiveAfterFetchTimeout(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int epoch = 5; @@ -1945,7 +1950,7 @@ public void testVoterBecomeProspectiveAfterFetchTimeout(boolean withKip853Rpc) t RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withElectedLeader(epoch, otherNodeId) .appendToLog(lastEpoch, List.of("foo")) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.assertElectedLeader(epoch, otherNodeId); @@ -1960,8 +1965,8 @@ public void testVoterBecomeProspectiveAfterFetchTimeout(boolean withKip853Rpc) t } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testFollowerAsObserverDoesNotBecomeProspectiveAfterFetchTimeout(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testFollowerAsObserverDoesNotBecomeProspectiveAfterFetchTimeout(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int epoch = 5; @@ -1971,7 +1976,7 @@ public void testFollowerAsObserverDoesNotBecomeProspectiveAfterFetchTimeout(bool RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withElectedLeader(epoch, otherNodeId) .appendToLog(lastEpoch, List.of("foo")) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.assertElectedLeader(epoch, otherNodeId); @@ -1983,15 +1988,15 @@ 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, raftProtocol.isReconfigSupported()), 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 = RaftProtocol.class) + public void testUnattachedAsObserverDoesNotBecomeProspectiveAfterElectionTimeout(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int epoch = 5; @@ -1999,7 +2004,7 @@ public void testUnattachedAsObserverDoesNotBecomeProspectiveAfterElectionTimeout RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withUnknownLeader(epoch) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.pollUntilRequest(); @@ -2013,7 +2018,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, raftProtocol.isReconfigSupported()), epoch, 0)); context.pollUntilResponse(); // observer can vote context.assertSentVoteResponse(Errors.NONE, epoch + 1, OptionalInt.empty(), true); @@ -2027,8 +2032,8 @@ public void testUnattachedAsObserverDoesNotBecomeProspectiveAfterElectionTimeout } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testUnattachedAsVoterCanBecomeFollowerAfterFindingLeader(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testUnattachedAsVoterCanBecomeFollowerAfterFindingLeader(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int leaderNodeId = localId + 2; @@ -2037,7 +2042,7 @@ public void testUnattachedAsVoterCanBecomeFollowerAfterFindingLeader(boolean wit RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withUnknownLeader(epoch) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.pollUntilRequest(); @@ -2058,8 +2063,8 @@ public void testUnattachedAsVoterCanBecomeFollowerAfterFindingLeader(boolean wit } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testInitializeObserverNoPreviousState(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testInitializeObserverNoPreviousState(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int leaderId = localId + 1; int otherNodeId = localId + 2; @@ -2067,7 +2072,7 @@ public void testInitializeObserverNoPreviousState(boolean withKip853Rpc) throws Set voters = Set.of(leaderId, otherNodeId); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.pollUntilRequest(); @@ -2086,8 +2091,8 @@ public void testInitializeObserverNoPreviousState(boolean withKip853Rpc) throws } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testObserverQuorumDiscoveryFailure(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testObserverQuorumDiscoveryFailure(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int leaderId = localId + 1; int epoch = 5; @@ -2099,7 +2104,7 @@ public void testObserverQuorumDiscoveryFailure(boolean withKip853Rpc) throws Exc RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withBootstrapServers(Optional.of(bootstrapServers)) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.pollUntilRequest(); @@ -2132,8 +2137,8 @@ public void testObserverQuorumDiscoveryFailure(boolean withKip853Rpc) throws Exc } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testObserverSendDiscoveryFetchAfterFetchTimeout(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testObserverSendDiscoveryFetchAfterFetchTimeout(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int leaderId = localId + 1; int otherNodeId = localId + 2; @@ -2146,7 +2151,7 @@ public void testObserverSendDiscoveryFetchAfterFetchTimeout(boolean withKip853Rp RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withBootstrapServers(Optional.of(bootstrapServers)) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.pollUntilRequest(); @@ -2173,8 +2178,8 @@ public void testObserverSendDiscoveryFetchAfterFetchTimeout(boolean withKip853Rp } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testObserverHandleRetryFetchToBootstrapServer(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + 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(); @@ -2189,7 +2194,7 @@ public void testObserverHandleRetryFetchToBootstrapServer(boolean withKip853Rpc) RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withBootstrapServers(Optional.of(bootstrapServers)) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); // Expect a fetch request to one of the bootstrap servers @@ -2247,8 +2252,8 @@ public void testObserverHandleRetryFetchToBootstrapServer(boolean withKip853Rpc) } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testObserverHandleRetryFetchToLeader(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + 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(); @@ -2263,7 +2268,7 @@ public void testObserverHandleRetryFetchToLeader(boolean withKip853Rpc) throws E RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withBootstrapServers(Optional.of(bootstrapServers)) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); // Expect a fetch request to one of the bootstrap servers @@ -2306,15 +2311,15 @@ public void testObserverHandleRetryFetchToLeader(boolean withKip853Rpc) throws E } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testInvalidFetchRequest(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testInvalidFetchRequest(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, raftProtocol.isReconfigSupported()); Set voters = Set.of(localId, otherNodeKey.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withUnknownLeader(4) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -2382,15 +2387,15 @@ public void testLeaderStateUpdateWithDifferentFetchRequestVersions(short version } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testFetchRequestClusterIdValidation(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testFetchRequestClusterIdValidation(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, raftProtocol.isReconfigSupported()); Set voters = Set.of(localId, otherNodeKey.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withUnknownLeader(4) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -2418,14 +2423,14 @@ public void testFetchRequestClusterIdValidation(boolean withKip853Rpc) throws Ex } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testVoteRequestClusterIdValidation(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testVoteRequestClusterIdValidation(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, raftProtocol.isReconfigSupported()); Set voters = Set.of(localId, otherNodeKey.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -2459,7 +2464,7 @@ public void testInvalidVoterReplicaVoteRequest() throws Exception { Set voters = Set.of(localId, otherNodeKey.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .build(); context.unattachedToLeader(); @@ -2506,7 +2511,7 @@ public void testInvalidVoterReplicaBeginQuorumEpochRequest() throws Exception { RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withUnknownLeader(epoch - 1) - .withKip853Rpc(true) + .withRaftProtocol(KIP_853_PROTOCOL) .build(); context.assertUnknownLeaderAndNoVotedCandidate(epoch - 1); @@ -2550,15 +2555,15 @@ public void testInvalidVoterReplicaBeginQuorumEpochRequest() throws Exception { } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testBeginQuorumEpochRequestClusterIdValidation(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + 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) .withUnknownLeader(4) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -2586,15 +2591,15 @@ public void testBeginQuorumEpochRequestClusterIdValidation(boolean withKip853Rpc } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testEndQuorumEpochRequestClusterIdValidation(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testEndQuorumEpochRequestClusterIdValidation(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, raftProtocol.isReconfigSupported()); Set voters = Set.of(localId, otherNodeKey.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withUnknownLeader(4) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -2622,21 +2627,21 @@ public void testEndQuorumEpochRequestClusterIdValidation(boolean withKip853Rpc) } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testLeaderAcceptVoteFromObserver(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + 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) .withUnknownLeader(4) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); int epoch = context.currentEpoch(); - ReplicaKey observerKey = replicaKey(localId + 2, withKip853Rpc); + ReplicaKey observerKey = replicaKey(localId + 2, raftProtocol.isReconfigSupported()); context.deliverRequest(context.voteRequest(epoch - 1, observerKey, 0, 0)); context.client.poll(); context.assertSentVoteResponse(Errors.FENCED_LEADER_EPOCH, epoch, OptionalInt.of(localId), false); @@ -2647,16 +2652,16 @@ public void testLeaderAcceptVoteFromObserver(boolean withKip853Rpc) throws Excep } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testInvalidVoteRequest(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testInvalidVoteRequest(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, raftProtocol.isReconfigSupported()); int epoch = 5; Set voters = Set.of(localId, otherNodeKey.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withElectedLeader(epoch, otherNodeKey.id()) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.assertElectedLeader(epoch, otherNodeKey.id()); @@ -2692,15 +2697,15 @@ public void testInvalidVoteRequest(boolean withKip853Rpc) throws Exception { } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testPurgatoryFetchTimeout(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testPurgatoryFetchTimeout(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, raftProtocol.isReconfigSupported()); Set voters = Set.of(localId, otherNodeKey.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withUnknownLeader(4) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -2720,15 +2725,15 @@ public void testPurgatoryFetchTimeout(boolean withKip853Rpc) throws Exception { } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testPurgatoryFetchSatisfiedByWrite(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testPurgatoryFetchSatisfiedByWrite(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, raftProtocol.isReconfigSupported()); Set voters = Set.of(localId, otherNodeKey.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withUnknownLeader(4) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -2750,16 +2755,16 @@ public void testPurgatoryFetchSatisfiedByWrite(boolean withKip853Rpc) throws Exc } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testPurgatoryFetchCompletedByFollowerTransition(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testPurgatoryFetchCompletedByFollowerTransition(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey voterKey2 = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey voterKey2 = replicaKey(localId + 1, raftProtocol.isReconfigSupported()); int voter3 = localId + 2; Set voters = Set.of(localId, voterKey2.id(), voter3); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withUnknownLeader(4) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -2786,8 +2791,8 @@ public void testPurgatoryFetchCompletedByFollowerTransition(boolean withKip853Rp } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testFetchResponseIgnoredAfterBecomingProspective(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testFetchResponseIgnoredAfterBecomingProspective(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int epoch = 5; @@ -2796,7 +2801,7 @@ public void testFetchResponseIgnoredAfterBecomingProspective(boolean withKip853R RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withElectedLeader(epoch, otherNodeId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.assertElectedLeader(epoch, otherNodeId); @@ -2823,9 +2828,9 @@ public void testFetchResponseIgnoredAfterBecomingProspective(boolean withKip853R } @ParameterizedTest - @ValueSource(booleans = { true, false }) + @EnumSource(value = RaftProtocol.class) public void testFetchResponseIgnoredAfterBecomingFollowerOfDifferentLeader( - boolean withKip853Rpc + RaftProtocol raftProtocol ) throws Exception { int localId = randomReplicaId(); int voter2 = localId + 1; @@ -2836,7 +2841,7 @@ public void testFetchResponseIgnoredAfterBecomingFollowerOfDifferentLeader( RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withElectedLeader(epoch, voter2) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.assertElectedLeader(epoch, voter2); @@ -2864,8 +2869,8 @@ public void testFetchResponseIgnoredAfterBecomingFollowerOfDifferentLeader( } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testVoteResponseIgnoredAfterBecomingFollower(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testVoteResponseIgnoredAfterBecomingFollower(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int voter2 = localId + 1; int voter3 = localId + 2; @@ -2874,7 +2879,7 @@ public void testVoteResponseIgnoredAfterBecomingFollower(boolean withKip853Rpc) RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withUnknownLeader(epoch) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.assertUnknownLeaderAndNoVotedCandidate(epoch); context.unattachedToCandidate(); @@ -2918,8 +2923,8 @@ public void testVoteResponseIgnoredAfterBecomingFollower(boolean withKip853Rpc) } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testFollowerLeaderRediscoveryAfterBrokerNotAvailableError(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testFollowerLeaderRediscoveryAfterBrokerNotAvailableError(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int leaderId = localId + 1; int otherNodeId = localId + 2; @@ -2932,7 +2937,7 @@ public void testFollowerLeaderRediscoveryAfterBrokerNotAvailableError(boolean wi RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withBootstrapServers(Optional.of(bootstrapServers)) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .withElectedLeader(epoch, leaderId) .build(); @@ -2957,8 +2962,8 @@ public void testFollowerLeaderRediscoveryAfterBrokerNotAvailableError(boolean wi } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testFollowerLeaderRediscoveryAfterRequestTimeout(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testFollowerLeaderRediscoveryAfterRequestTimeout(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int leaderId = localId + 1; int otherNodeId = localId + 2; @@ -2971,7 +2976,7 @@ public void testFollowerLeaderRediscoveryAfterRequestTimeout(boolean withKip853R RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withBootstrapServers(Optional.of(bootstrapServers)) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .withElectedLeader(epoch, leaderId) .build(); @@ -2992,8 +2997,8 @@ public void testFollowerLeaderRediscoveryAfterRequestTimeout(boolean withKip853R } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testObserverLeaderRediscoveryAfterBrokerNotAvailableError(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testObserverLeaderRediscoveryAfterBrokerNotAvailableError(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int leaderId = localId + 1; int otherNodeId = localId + 2; @@ -3006,7 +3011,7 @@ public void testObserverLeaderRediscoveryAfterBrokerNotAvailableError(boolean wi RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withBootstrapServers(Optional.of(bootstrapServers)) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.discoverLeaderAsObserver(leaderId, epoch); @@ -3041,8 +3046,8 @@ public void testObserverLeaderRediscoveryAfterBrokerNotAvailableError(boolean wi } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testObserverLeaderRediscoveryAfterRequestTimeout(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testObserverLeaderRediscoveryAfterRequestTimeout(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int leaderId = localId + 1; int otherNodeId = localId + 2; @@ -3055,7 +3060,7 @@ public void testObserverLeaderRediscoveryAfterRequestTimeout(boolean withKip853R RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withBootstrapServers(Optional.of(bootstrapServers)) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.discoverLeaderAsObserver(leaderId, epoch); @@ -3086,14 +3091,14 @@ public void testObserverLeaderRediscoveryAfterRequestTimeout(boolean withKip853R } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testLeaderGracefulShutdown(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testLeaderGracefulShutdown(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, raftProtocol.isReconfigSupported()); Set voters = Set.of(localId, otherNodeKey.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -3133,15 +3138,16 @@ public void testLeaderGracefulShutdown(boolean withKip853Rpc) throws Exception { } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testEndQuorumEpochSentBasedOnFetchOffset(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testEndQuorumEpochSentBasedOnFetchOffset(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey closeFollower = replicaKey(localId + 2, withKip853Rpc); - ReplicaKey laggingFollower = replicaKey(localId + 1, withKip853Rpc); + boolean reconfigSupported = raftProtocol.isReconfigSupported(); + ReplicaKey closeFollower = replicaKey(localId + 2, reconfigSupported); + ReplicaKey laggingFollower = replicaKey(localId + 1, reconfigSupported); Set voters = Set.of(localId, closeFollower.id(), laggingFollower.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -3184,11 +3190,12 @@ public void testEndQuorumEpochSentBasedOnFetchOffset(boolean withKip853Rpc) thro } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testDescribeQuorumNonLeader(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testDescribeQuorumNonLeader(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey voter2 = replicaKey(localId + 1, withKip853Rpc); - ReplicaKey voter3 = replicaKey(localId + 2, withKip853Rpc); + boolean reconfigSupported = raftProtocol.isReconfigSupported(); + ReplicaKey voter2 = replicaKey(localId + 1, reconfigSupported); + ReplicaKey voter3 = replicaKey(localId + 2, reconfigSupported); int epoch = 2; Set voters = Set.of(localId, voter2.id(), voter3.id()); @@ -3215,8 +3222,8 @@ public void testDescribeQuorumNonLeader(boolean withKip853Rpc) throws Exception } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testDescribeQuorumWithOnlyStaticVoters(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testDescribeQuorumWithOnlyStaticVoters(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); ReplicaKey local = replicaKey(localId, true); ReplicaKey follower1 = replicaKey(localId + 1, true); @@ -3224,7 +3231,7 @@ public void testDescribeQuorumWithOnlyStaticVoters(boolean withKip853Rpc) throws RaftClientTestContext context = new RaftClientTestContext.Builder(localId, local.directoryId().get()) .withStaticVoters(voters) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -3250,24 +3257,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 reconfigSupported = raftProtocol.isReconfigSupported(); 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(reconfigSupported ? 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(reconfigSupported ? Uuid.randomUuid() : ReplicaKey.NO_DIRECTORY_ID); ReplicaKey follower2 = ReplicaKey.of(followerId2, followerDirectoryId2); RaftClientTestContext.Builder builder = new RaftClientTestContext.Builder(localId, localDirectoryId) - .withKip853Rpc(withKip853Rpc); + .withRaftProtocol(raftProtocol); if (withBootstrapSnapshot) { VoterSet bootstrapVoterSet = VoterSetTest.voterSet(Stream.of(local, bootstrapFollower1, bootstrapFollower2)); @@ -3362,18 +3370,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 reconfigSupported = raftProtocol.isReconfigSupported(); + Uuid followerDirectoryId = bootstrapFollower.directoryId().orElse(reconfigSupported ? Uuid.randomUuid() : ReplicaKey.NO_DIRECTORY_ID); ReplicaKey follower = ReplicaKey.of(followerId, followerDirectoryId); RaftClientTestContext.Builder builder = new RaftClientTestContext.Builder(localId, localDirectoryId) - .withKip853Rpc(withKip853Rpc); + .withRaftProtocol(raftProtocol); if (withBootstrapSnapshot) { VoterSet bootstrapVoterSet = VoterSetTest.voterSet(Stream.of(local, bootstrapFollower)); @@ -3397,7 +3406,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, reconfigSupported); Uuid observerDirectoryId = observer.directoryId().orElse(ReplicaKey.NO_DIRECTORY_ID); context.time.sleep(100); long observerFetchTime = context.time.milliseconds(); @@ -3505,18 +3514,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(raftProtocol.isReconfigSupported() ? Uuid.randomUuid() : ReplicaKey.NO_DIRECTORY_ID); ReplicaKey follower = ReplicaKey.of(followerId, followerDirectoryId); RaftClientTestContext.Builder builder = new RaftClientTestContext.Builder(localId, localDirectoryId) - .withKip853Rpc(withKip853Rpc); + .withRaftProtocol(raftProtocol); if (withBootstrapSnapshot) { VoterSet bootstrapVoterSet = VoterSetTest.voterSet(Stream.of(local, bootstrapFollower)); builder.withBootstrapSnapshot(Optional.of(bootstrapVoterSet)); @@ -3581,8 +3590,8 @@ public void testDescribeQuorumNonMonotonicFollowerFetch(boolean withKip853Rpc, b } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testStaticVotersIgnoredWithBootstrapSnapshot(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testStaticVotersIgnoredWithBootstrapSnapshot(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); ReplicaKey local = replicaKey(localId, true); ReplicaKey follower = replicaKey(localId + 1, true); @@ -3593,7 +3602,7 @@ public void testStaticVotersIgnoredWithBootstrapSnapshot(boolean withKip853Rpc) RaftClientTestContext context = new RaftClientTestContext.Builder(localId, local.directoryId().get()) .withStaticVoters(staticVoters) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .withBootstrapSnapshot(Optional.of(voterSet)) .build(); @@ -3602,22 +3611,23 @@ public void testStaticVotersIgnoredWithBootstrapSnapshot(boolean withKip853Rpc) // check describe quorum response has both followers context.deliverRequest(context.describeQuorumRequest()); context.pollUntilResponse(); + boolean reconfigSupported = raftProtocol.isReconfigSupported(); List expectedVoterStates = List.of( new ReplicaState() .setReplicaId(localId) - .setReplicaDirectoryId(withKip853Rpc ? local.directoryId().get() : ReplicaKey.NO_DIRECTORY_ID) + .setReplicaDirectoryId(reconfigSupported ? 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(reconfigSupported ? 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(reconfigSupported ? follower2.directoryId().get() : ReplicaKey.NO_DIRECTORY_ID) .setLogEndOffset(-1L) .setLastFetchTimestamp(-1) .setLastCaughtUpTimestamp(-1)); @@ -3626,15 +3636,15 @@ public void testStaticVotersIgnoredWithBootstrapSnapshot(boolean withKip853Rpc) @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testLeaderGracefulShutdownTimeout(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + 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) .withUnknownLeader(1) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -3664,8 +3674,8 @@ public void testLeaderGracefulShutdownTimeout(boolean withKip853Rpc) throws Exce } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testFollowerGracefulShutdown(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testFollowerGracefulShutdown(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int epoch = 5; @@ -3673,7 +3683,7 @@ public void testFollowerGracefulShutdown(boolean withKip853Rpc) throws Exception RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withElectedLeader(epoch, otherNodeId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.assertElectedLeader(epoch, otherNodeId); @@ -3691,8 +3701,8 @@ public void testFollowerGracefulShutdown(boolean withKip853Rpc) throws Exception } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testObserverGracefulShutdown(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testObserverGracefulShutdown(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int voter1 = localId + 1; int voter2 = localId + 2; @@ -3700,7 +3710,7 @@ public void testObserverGracefulShutdown(boolean withKip853Rpc) throws Exception RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withUnknownLeader(5) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.client.poll(); context.assertUnknownLeaderAndNoVotedCandidate(5); @@ -3718,11 +3728,11 @@ public void testObserverGracefulShutdown(boolean withKip853Rpc) throws Exception } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testGracefulShutdownSingleMemberQuorum(boolean withKip853Rpc) throws IOException { + @EnumSource(value = RaftProtocol.class) + public void testGracefulShutdownSingleMemberQuorum(RaftProtocol raftProtocol) throws IOException { int localId = randomReplicaId(); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, Set.of(localId)) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.assertElectedLeader(1, localId); @@ -3736,8 +3746,8 @@ public void testGracefulShutdownSingleMemberQuorum(boolean withKip853Rpc) throws } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testFollowerReplication(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testFollowerReplication(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int epoch = 5; @@ -3745,7 +3755,7 @@ public void testFollowerReplication(boolean withKip853Rpc) throws Exception { RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withElectedLeader(epoch, otherNodeId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.assertElectedLeader(epoch, otherNodeId); @@ -3766,8 +3776,8 @@ 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; @@ -3775,7 +3785,7 @@ public void testObserverReplication(boolean withKip853Rpc, boolean alwaysFlush) RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withElectedLeader(epoch, otherNodeId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .withAlwaysFlush(alwaysFlush) .build(); context.assertElectedLeader(epoch, otherNodeId); @@ -3798,8 +3808,8 @@ public void testObserverReplication(boolean withKip853Rpc, boolean alwaysFlush) } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testEmptyRecordSetInFetchResponse(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testEmptyRecordSetInFetchResponse(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int epoch = 5; @@ -3807,7 +3817,7 @@ public void testEmptyRecordSetInFetchResponse(boolean withKip853Rpc) throws Exce RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withElectedLeader(epoch, otherNodeId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.assertElectedLeader(epoch, otherNodeId); @@ -3865,17 +3875,17 @@ public void testEmptyRecordSetInFetchResponse(boolean withKip853Rpc) throws Exce } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testFetchShouldBeTreatedAsLeaderAcknowledgement(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testFetchShouldBeTreatedAsLeaderAcknowledgement(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, raftProtocol.isReconfigSupported()); int epoch = 5; Set voters = Set.of(localId, otherNodeKey.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .updateRandom(r -> r.mockNextInt(DEFAULT_ELECTION_TIMEOUT_MS, 0)) .withUnknownLeader(epoch - 1) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToCandidate(); @@ -3901,13 +3911,13 @@ public void testFetchShouldBeTreatedAsLeaderAcknowledgement(boolean withKip853Rp } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testLeaderAppendSingleMemberQuorum(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testLeaderAppendSingleMemberQuorum(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); Set voters = Set.of(localId); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); long now = context.time.milliseconds(); @@ -3931,7 +3941,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, raftProtocol.isReconfigSupported()); List batches = new ArrayList<>(2); boolean appended = true; @@ -3978,8 +3988,8 @@ public void testLeaderAppendSingleMemberQuorum(boolean withKip853Rpc) throws Exc } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testFollowerLogReconciliation(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testFollowerLogReconciliation(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int epoch = 5; @@ -3990,7 +4000,7 @@ public void testFollowerLogReconciliation(boolean withKip853Rpc) throws Exceptio .withElectedLeader(epoch, otherNodeId) .appendToLog(lastEpoch, List.of("foo", "bar")) .appendToLog(lastEpoch, List.of("baz")) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.assertElectedLeader(epoch, otherNodeId); @@ -4015,12 +4025,12 @@ public void testFollowerLogReconciliation(boolean withKip853Rpc) throws Exceptio } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testMetrics(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testMetrics(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int epoch = 1; RaftClientTestContext context = new RaftClientTestContext.Builder(localId, Set.of(localId)) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.pollUntil(() -> context.log.endOffset().offset() == 1L); @@ -4059,16 +4069,14 @@ public void testMetrics(boolean withKip853Rpc) throws Exception { } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testHandleLeaderChangeFiresAfterListenerReachesEpochStartOffsetOnEmptyLog( - boolean withKip853Rpc - ) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testHandleLeaderChangeFiresAfterListenerReachesEpochStartOffsetOnEmptyLog(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, raftProtocol.isReconfigSupported()); Set voters = Set.of(localId, otherNodeKey.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -4106,12 +4114,10 @@ public void testHandleLeaderChangeFiresAfterListenerReachesEpochStartOffsetOnEmp } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testHandleLeaderChangeFiresAfterListenerReachesEpochStartOffset( - boolean withKip853Rpc - ) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testHandleLeaderChangeFiresAfterListenerReachesEpochStartOffset(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, raftProtocol.isReconfigSupported()); int epoch = 5; Set voters = Set.of(localId, otherNodeKey.id()); @@ -4125,7 +4131,7 @@ public void testHandleLeaderChangeFiresAfterListenerReachesEpochStartOffset( .appendToLog(1, batch2) .appendToLog(2, batch3) .withUnknownLeader(epoch - 1) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -4175,10 +4181,10 @@ public void testHandleLeaderChangeFiresAfterListenerReachesEpochStartOffset( } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testLateRegisteredListenerCatchesUp(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testLateRegisteredListenerCatchesUp(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey otherNodeKey = replicaKey(localId + 1, raftProtocol.isReconfigSupported()); int epoch = 5; Set voters = Set.of(localId, otherNodeKey.id()); @@ -4191,7 +4197,7 @@ public void testLateRegisteredListenerCatchesUp(boolean withKip853Rpc) throws Ex .appendToLog(1, batch2) .appendToLog(2, batch3) .withUnknownLeader(epoch - 1) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -4218,8 +4224,8 @@ public void testLateRegisteredListenerCatchesUp(boolean withKip853Rpc) throws Ex } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testReregistrationChangesListenerContext(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testReregistrationChangesListenerContext(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int epoch = 5; @@ -4234,7 +4240,7 @@ public void testReregistrationChangesListenerContext(boolean withKip853Rpc) thro .appendToLog(1, batch2) .appendToLog(2, batch3) .withUnknownLeader(epoch - 1) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.unattachedToLeader(); @@ -4262,8 +4268,8 @@ public void testReregistrationChangesListenerContext(boolean withKip853Rpc) thro } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testHandleCommitCallbackFiresAfterFollowerHighWatermarkAdvances(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + public void testHandleCommitCallbackFiresAfterFollowerHighWatermarkAdvances(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int epoch = 5; @@ -4271,7 +4277,7 @@ public void testHandleCommitCallbackFiresAfterFollowerHighWatermarkAdvances(bool RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withElectedLeader(epoch, otherNodeId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); assertEquals(OptionalLong.empty(), context.client.highWatermark()); @@ -4321,12 +4327,12 @@ public void testHandleCommitCallbackFiresAfterFollowerHighWatermarkAdvances(bool } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testHandleCommitCallbackFiresInVotedState(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + 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, raftProtocol.isReconfigSupported()); int epoch = 7; Set voters = Set.of(localId, otherNodeKey.id()); @@ -4335,7 +4341,7 @@ public void testHandleCommitCallbackFiresInVotedState(boolean withKip853Rpc) thr .appendToLog(4, List.of("d", "e", "f")) .appendToLog(4, List.of("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 @@ -4368,12 +4374,12 @@ public void testHandleCommitCallbackFiresInVotedState(boolean withKip853Rpc) thr } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testHandleCommitCallbackFiresInCandidateState(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + 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, raftProtocol.isReconfigSupported()); int epoch = 7; Set voters = Set.of(localId, otherNodeKey.id()); @@ -4382,7 +4388,7 @@ public void testHandleCommitCallbackFiresInCandidateState(boolean withKip853Rpc) .appendToLog(4, List.of("d", "e", "f")) .appendToLog(4, List.of("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 @@ -4423,9 +4429,9 @@ public void testHandleCommitCallbackFiresInCandidateState(boolean withKip853Rpc) } @ParameterizedTest - @ValueSource(booleans = { true, false }) + @EnumSource(value = RaftProtocol.class) 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 @@ -4437,7 +4443,7 @@ public void testHandleLeaderChangeFiresAfterUnattachedRegistration( RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withUnknownLeader(epoch) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); // Register another listener and verify that it is notified of latest epoch @@ -4461,8 +4467,8 @@ public void testHandleLeaderChangeFiresAfterUnattachedRegistration( } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testHandleLeaderChangeFiresAfterFollowerRegistration(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + 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(); @@ -4472,7 +4478,7 @@ public void testHandleLeaderChangeFiresAfterFollowerRegistration(boolean withKip RaftClientTestContext context = new RaftClientTestContext.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 @@ -4487,8 +4493,8 @@ public void testHandleLeaderChangeFiresAfterFollowerRegistration(boolean withKip } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testHandleLeaderChangeFiresAfterResignRegistration(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + 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(); @@ -4498,7 +4504,7 @@ public void testHandleLeaderChangeFiresAfterResignRegistration(boolean withKip85 RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withElectedLeader(epoch, localId) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); context.client.poll(); @@ -4517,8 +4523,8 @@ public void testHandleLeaderChangeFiresAfterResignRegistration(boolean withKip85 } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testObserverFetchWithNoLocalId(boolean withKip853Rpc) throws Exception { + @EnumSource(value = RaftProtocol.class) + 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(); @@ -4530,7 +4536,7 @@ public void testObserverFetchWithNoLocalId(boolean withKip853Rpc) throws Excepti RaftClientTestContext context = new RaftClientTestContext.Builder(OptionalInt.empty(), voters) .withBootstrapServers(Optional.of(bootstrapServers)) - .withKip853Rpc(withKip853Rpc) + .withRaftProtocol(raftProtocol) .build(); // First fetch discovers the current leader and epoch @@ -4581,4 +4587,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_595_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 c104af0ce7606..0027dd5f75853 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java @@ -100,7 +100,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; @@ -295,14 +294,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; @@ -2224,33 +2215,4 @@ public void handleLoadSnapshot(SnapshotReader reader) { snapshot = Optional.of(reader); } } - - /** - * Determines what versions of RPCs are in use. Note, these are ordered from oldest to newest, and are - * cumulative. E.g. KIP_996_PROTOCOL includes KIP_853_PROTOCOL and KIP_595_PROTOCOL changes - */ - enum RaftProtocol { - // kraft support - KIP_595_PROTOCOL, - // dynamic quorum reconfiguration support - KIP_853_PROTOCOL, - // preVote support - KIP_996_PROTOCOL; - - boolean isKRaftSupported() { - return isAtLeast(KIP_595_PROTOCOL); - } - - boolean isReconfigSupported() { - return isAtLeast(KIP_853_PROTOCOL); - } - - boolean isPreVoteSupported() { - return isAtLeast(KIP_996_PROTOCOL); - } - - private boolean isAtLeast(RaftProtocol otherRpc) { - return this.compareTo(otherRpc) >= 0; - } - } } diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftProtocol.java b/raft/src/test/java/org/apache/kafka/raft/RaftProtocol.java new file mode 100644 index 0000000000000..2312ae6c3f494 --- /dev/null +++ b/raft/src/test/java/org/apache/kafka/raft/RaftProtocol.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.raft; + +/** + * Determines what versions of RPCs are in use. Note, these are ordered from oldest to newest, and are + * cumulative. E.g. KIP_996_PROTOCOL includes KIP_853_PROTOCOL and KIP_595_PROTOCOL changes + */ +enum RaftProtocol { + // kraft support + KIP_595_PROTOCOL, + // dynamic quorum reconfiguration support + KIP_853_PROTOCOL, + // preVote support + KIP_996_PROTOCOL; + + boolean isKRaftSupported() { + return isAtLeast(KIP_595_PROTOCOL); + } + + boolean isReconfigSupported() { + return isAtLeast(KIP_853_PROTOCOL); + } + + boolean isPreVoteSupported() { + return isAtLeast(KIP_996_PROTOCOL); + } + + private boolean isAtLeast(RaftProtocol otherRpc) { + return this.compareTo(otherRpc) >= 0; + } +}