@ -16,6 +16,9 @@
@@ -16,6 +16,9 @@
* /
package org.apache.kafka.raft ;
import net.jqwik.api.ForAll ;
import net.jqwik.api.Property ;
import net.jqwik.api.constraints.IntRange ;
import org.apache.kafka.common.TopicPartition ;
import org.apache.kafka.common.Uuid ;
import org.apache.kafka.common.memory.MemoryPool ;
@ -32,7 +35,6 @@ import org.apache.kafka.raft.MockLog.LogBatch;
@@ -32,7 +35,6 @@ import org.apache.kafka.raft.MockLog.LogBatch;
import org.apache.kafka.raft.MockLog.LogEntry ;
import org.apache.kafka.raft.internals.BatchMemoryPool ;
import org.junit.jupiter.api.Tag ;
import org.junit.jupiter.api.Test ;
import java.io.IOException ;
import java.net.InetSocketAddress ;
@ -59,9 +61,45 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
@@ -59,9 +61,45 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertTrue ;
import static org.junit.jupiter.api.Assertions.fail ;
/ * *
* The simulation testing framework provides a way to verify quorum behavior under
* different conditions . It is similar to system testing in that the test involves
* independently executing nodes , but there are several important differences :
*
* 1 . Simulation behavior is deterministic provided an initial random seed . This
* makes it easy to reproduce and debug test failures .
* 2 . The simulation uses an in - memory message router instead of a real network .
* Not only is this much cheaper and faster , it provides an easy way to create
* flaky network conditions or even network partitions without losing the
* simulation determinism .
* 3 . Similarly , persistent state is stored in memory . We can nevertheless simulate
* different kinds of failures , such as the loss of unflushed data after a hard
* node restart using { @link MockLog } .
*
* The framework uses a single event scheduler in order to provide deterministic
* executions . Each test is setup as a specific scenario with a variable number of
* voters and observers . Much like system tests , there is typically a warmup
* period , followed by some cluster event ( such as a node failure ) , and then some
* logic to validate behavior after recovery .
*
* If any of the tests fail on a particular seed , the easiest way to reproduce
* the failure is to change the ` @Property ` annotation to specify the failing seed .
* For example :
*
* < pre >
* { @code
* @Property ( tries = 1 , seed = "-590031835267299290" , shrinking = ShrinkingMode . OFF )
* }
* < / pre >
*
* ( Note that we disable parameter shrinking since it is not too useful for simulation
* failures and this allows us to isolate a single execution , which makes the logging
* more useful if enabled . )
* /
@Tag ( "integration" )
public class RaftEventSimulationTest {
private static final TopicPartition METADATA_PARTITION = new TopicPartition ( "__cluster_metadata" , 0 ) ;
private static final TopicPartition METADATA_PARTITION = new TopicPartition ( "@ metadata" , 0 ) ;
private static final int ELECTION_TIMEOUT_MS = 1000 ;
private static final int ELECTION_JITTER_MS = 100 ;
private static final int FETCH_TIMEOUT_MS = 3000 ;
@ -70,335 +108,200 @@ public class RaftEventSimulationTest {
@@ -70,335 +108,200 @@ public class RaftEventSimulationTest {
private static final int FETCH_MAX_WAIT_MS = 100 ;
private static final int LINGER_MS = 0 ;
@Test
public void testInitialLeaderElectionQuorumSizeOne ( ) {
testInitialLeaderElection ( new QuorumConfig ( 1 ) ) ;
}
@Test
public void testInitialLeaderElectionQuorumSizeTwo ( ) {
testInitialLeaderElection ( new QuorumConfig ( 2 ) ) ;
}
@Test
public void testInitialLeaderElectionQuorumSizeThree ( ) {
testInitialLeaderElection ( new QuorumConfig ( 3 ) ) ;
}
@Test
public void testInitialLeaderElectionQuorumSizeFour ( ) {
testInitialLeaderElection ( new QuorumConfig ( 4 ) ) ;
}
@Test
public void testInitialLeaderElectionQuorumSizeFive ( ) {
testInitialLeaderElection ( new QuorumConfig ( 5 ) ) ;
}
private void testInitialLeaderElection ( QuorumConfig config ) {
for ( int seed = 0 ; seed < 100 ; seed + + ) {
Cluster cluster = new Cluster ( config , seed ) ;
MessageRouter router = new MessageRouter ( cluster ) ;
EventScheduler scheduler = schedulerWithDefaultInvariants ( cluster ) ;
cluster . startAll ( ) ;
schedulePolling ( scheduler , cluster , 3 , 5 ) ;
scheduler . schedule ( router : : deliverAll , 0 , 2 , 1 ) ;
scheduler . schedule ( new SequentialAppendAction ( cluster ) , 0 , 2 , 3 ) ;
scheduler . runUntil ( cluster : : hasConsistentLeader ) ;
scheduler . runUntil ( ( ) - > cluster . allReachedHighWatermark ( 10 ) ) ;
}
}
@Test
public void testElectionAfterLeaderFailureQuorumSizeThree ( ) {
testElectionAfterLeaderFailure ( new QuorumConfig ( 3 , 0 ) ) ;
}
@Test
public void testElectionAfterLeaderFailureQuorumSizeThreeAndTwoObservers ( ) {
testElectionAfterLeaderFailure ( new QuorumConfig ( 3 , 1 ) ) ;
}
@Test
public void testElectionAfterLeaderFailureQuorumSizeFour ( ) {
testElectionAfterLeaderFailure ( new QuorumConfig ( 4 , 0 ) ) ;
}
@Test
public void testElectionAfterLeaderFailureQuorumSizeFourAndTwoObservers ( ) {
testElectionAfterLeaderFailure ( new QuorumConfig ( 4 , 2 ) ) ;
}
@Test
public void testElectionAfterLeaderFailureQuorumSizeFive ( ) {
testElectionAfterLeaderFailure ( new QuorumConfig ( 5 , 0 ) ) ;
}
@Test
public void testElectionAfterLeaderFailureQuorumSizeFiveAndThreeObservers ( ) {
testElectionAfterLeaderFailure ( new QuorumConfig ( 5 , 3 ) ) ;
}
private void testElectionAfterLeaderFailure ( QuorumConfig config ) {
checkElectionAfterLeaderShutdown ( config , false ) ;
}
@Test
public void testElectionAfterLeaderGracefulShutdownQuorumSizeThree ( ) {
checkElectionAfterLeaderGracefulShutdown ( new QuorumConfig ( 3 , 0 ) ) ;
}
@Test
public void testElectionAfterLeaderGracefulShutdownQuorumSizeThreeAndTwoObservers ( ) {
checkElectionAfterLeaderGracefulShutdown ( new QuorumConfig ( 3 , 2 ) ) ;
}
@Test
public void testElectionAfterLeaderGracefulShutdownQuorumSizeFour ( ) {
checkElectionAfterLeaderGracefulShutdown ( new QuorumConfig ( 4 , 0 ) ) ;
}
@Test
public void testElectionAfterLeaderGracefulShutdownQuorumSizeFourAndTwoObservers ( ) {
checkElectionAfterLeaderGracefulShutdown ( new QuorumConfig ( 4 , 2 ) ) ;
}
@Test
public void testElectionAfterLeaderGracefulShutdownQuorumSizeFive ( ) {
checkElectionAfterLeaderGracefulShutdown ( new QuorumConfig ( 5 , 0 ) ) ;
}
@Test
public void testElectionAfterLeaderGracefulShutdownQuorumSizeFiveAndThreeObservers ( ) {
checkElectionAfterLeaderGracefulShutdown ( new QuorumConfig ( 5 , 3 ) ) ;
}
private void checkElectionAfterLeaderGracefulShutdown ( QuorumConfig config ) {
checkElectionAfterLeaderShutdown ( config , true ) ;
}
private void checkElectionAfterLeaderShutdown ( QuorumConfig config , boolean isGracefulShutdown ) {
for ( int seed = 0 ; seed < 100 ; seed + + ) {
Cluster cluster = new Cluster ( config , seed ) ;
MessageRouter router = new MessageRouter ( cluster ) ;
EventScheduler scheduler = schedulerWithDefaultInvariants ( cluster ) ;
// Seed the cluster with some data
cluster . startAll ( ) ;
schedulePolling ( scheduler , cluster , 3 , 5 ) ;
scheduler . schedule ( router : : deliverAll , 0 , 2 , 1 ) ;
scheduler . schedule ( new SequentialAppendAction ( cluster ) , 0 , 2 , 3 ) ;
scheduler . runUntil ( cluster : : hasConsistentLeader ) ;
scheduler . runUntil ( ( ) - > cluster . anyReachedHighWatermark ( 10 ) ) ;
// Shutdown the leader and write some more data. We can verify the new leader has been elected
// by verifying that the high watermark can still advance.
int leaderId = cluster . latestLeader ( ) . getAsInt ( ) ;
if ( isGracefulShutdown ) {
cluster . shutdown ( leaderId ) ;
} else {
cluster . kill ( leaderId ) ;
}
scheduler . runUntil ( ( ) - > cluster . allReachedHighWatermark ( 20 ) ) ;
long highWatermark = cluster . maxHighWatermarkReached ( ) ;
// Restart the node and verify it catches up
cluster . start ( leaderId ) ;
scheduler . runUntil ( ( ) - > cluster . allReachedHighWatermark ( highWatermark + 10 ) ) ;
}
}
@Test
public void testRecoveryAfterAllNodesFailQuorumSizeThree ( ) {
checkRecoveryAfterAllNodesFail ( new QuorumConfig ( 3 ) ) ;
}
@Test
public void testRecoveryAfterAllNodesFailQuorumSizeFour ( ) {
checkRecoveryAfterAllNodesFail ( new QuorumConfig ( 4 ) ) ;
}
@Test
public void testRecoveryAfterAllNodesFailQuorumSizeFive ( ) {
checkRecoveryAfterAllNodesFail ( new QuorumConfig ( 5 ) ) ;
}
private void checkRecoveryAfterAllNodesFail ( QuorumConfig config ) {
for ( int seed = 0 ; seed < 100 ; seed + + ) {
Cluster cluster = new Cluster ( config , seed ) ;
MessageRouter router = new MessageRouter ( cluster ) ;
EventScheduler scheduler = schedulerWithDefaultInvariants ( cluster ) ;
// Seed the cluster with some data
cluster . startAll ( ) ;
schedulePolling ( scheduler , cluster , 3 , 5 ) ;
scheduler . schedule ( router : : deliverAll , 0 , 2 , 1 ) ;
scheduler . schedule ( new SequentialAppendAction ( cluster ) , 0 , 2 , 3 ) ;
scheduler . runUntil ( cluster : : hasConsistentLeader ) ;
scheduler . runUntil ( ( ) - > cluster . anyReachedHighWatermark ( 10 ) ) ;
long highWatermark = cluster . maxHighWatermarkReached ( ) ;
// We kill all of the nodes. Then we bring back a majority and verify that
// they are able to elect a leader and continue making progress
cluster . killAll ( ) ;
Iterator < Integer > nodeIdsIterator = cluster . nodes ( ) . iterator ( ) ;
for ( int i = 0 ; i < cluster . majoritySize ( ) ; i + + ) {
Integer nodeId = nodeIdsIterator . next ( ) ;
cluster . start ( nodeId ) ;
}
scheduler . runUntil ( ( ) - > cluster . allReachedHighWatermark ( highWatermark + 10 ) ) ;
}
}
@Test
public void testElectionAfterLeaderNetworkPartitionQuorumSizeThree ( ) {
checkElectionAfterLeaderNetworkPartition ( new QuorumConfig ( 3 ) ) ;
}
@Test
public void testElectionAfterLeaderNetworkPartitionQuorumSizeThreeAndTwoObservers ( ) {
checkElectionAfterLeaderNetworkPartition ( new QuorumConfig ( 3 , 2 ) ) ;
}
@Test
public void testElectionAfterLeaderNetworkPartitionQuorumSizeFour ( ) {
checkElectionAfterLeaderNetworkPartition ( new QuorumConfig ( 4 ) ) ;
}
@Test
public void testElectionAfterLeaderNetworkPartitionQuorumSizeFourAndTwoObservers ( ) {
checkElectionAfterLeaderNetworkPartition ( new QuorumConfig ( 4 , 2 ) ) ;
}
@Test
public void testElectionAfterLeaderNetworkPartitionQuorumSizeFive ( ) {
checkElectionAfterLeaderNetworkPartition ( new QuorumConfig ( 5 ) ) ;
}
@Test
public void testElectionAfterLeaderNetworkPartitionQuorumSizeFiveAndThreeObservers ( ) {
checkElectionAfterLeaderNetworkPartition ( new QuorumConfig ( 5 , 3 ) ) ;
}
private void checkElectionAfterLeaderNetworkPartition ( QuorumConfig config ) {
for ( int seed = 0 ; seed < 100 ; seed + + ) {
Cluster cluster = new Cluster ( config , seed ) ;
MessageRouter router = new MessageRouter ( cluster ) ;
EventScheduler scheduler = schedulerWithDefaultInvariants ( cluster ) ;
// Seed the cluster with some data
cluster . startAll ( ) ;
schedulePolling ( scheduler , cluster , 3 , 5 ) ;
scheduler . schedule ( router : : deliverAll , 0 , 2 , 2 ) ;
scheduler . schedule ( new SequentialAppendAction ( cluster ) , 0 , 2 , 3 ) ;
scheduler . runUntil ( cluster : : hasConsistentLeader ) ;
scheduler . runUntil ( ( ) - > cluster . anyReachedHighWatermark ( 10 ) ) ;
// The leader gets partitioned off. We can verify the new leader has been elected
// by writing some data and ensuring that it gets replicated
int leaderId = cluster . latestLeader ( ) . getAsInt ( ) ;
router . filter ( leaderId , new DropAllTraffic ( ) ) ;
Set < Integer > nonPartitionedNodes = new HashSet < > ( cluster . nodes ( ) ) ;
nonPartitionedNodes . remove ( leaderId ) ;
scheduler . runUntil ( ( ) - > cluster . allReachedHighWatermark ( 20 , nonPartitionedNodes ) ) ;
}
@Property ( tries = 100 )
void canElectInitialLeader (
@ForAll Random random ,
@ForAll @IntRange ( min = 1 , max = 5 ) int numVoters ,
@ForAll @IntRange ( min = 0 , max = 5 ) int numObservers
) {
Cluster cluster = new Cluster ( numVoters , numObservers , random ) ;
MessageRouter router = new MessageRouter ( cluster ) ;
EventScheduler scheduler = schedulerWithDefaultInvariants ( cluster ) ;
cluster . startAll ( ) ;
schedulePolling ( scheduler , cluster , 3 , 5 ) ;
scheduler . schedule ( router : : deliverAll , 0 , 2 , 1 ) ;
scheduler . schedule ( new SequentialAppendAction ( cluster ) , 0 , 2 , 3 ) ;
scheduler . runUntil ( cluster : : hasConsistentLeader ) ;
scheduler . runUntil ( ( ) - > cluster . allReachedHighWatermark ( 10 ) ) ;
}
@Test
public void testElectionAfterMultiNodeNetworkPartitionQuorumSizeFive ( ) {
checkElectionAfterMultiNodeNetworkPartition ( new QuorumConfig ( 5 ) ) ;
@Property ( tries = 100 )
void canElectNewLeaderAfterOldLeaderFailure (
@ForAll Random random ,
@ForAll @IntRange ( min = 3 , max = 5 ) int numVoters ,
@ForAll @IntRange ( min = 0 , max = 5 ) int numObservers ,
@ForAll boolean isGracefulShutdown
) {
Cluster cluster = new Cluster ( numVoters , numObservers , random ) ;
MessageRouter router = new MessageRouter ( cluster ) ;
EventScheduler scheduler = schedulerWithDefaultInvariants ( cluster ) ;
// Seed the cluster with some data
cluster . startAll ( ) ;
schedulePolling ( scheduler , cluster , 3 , 5 ) ;
scheduler . schedule ( router : : deliverAll , 0 , 2 , 1 ) ;
scheduler . schedule ( new SequentialAppendAction ( cluster ) , 0 , 2 , 3 ) ;
scheduler . runUntil ( cluster : : hasConsistentLeader ) ;
scheduler . runUntil ( ( ) - > cluster . anyReachedHighWatermark ( 10 ) ) ;
// Shutdown the leader and write some more data. We can verify the new leader has been elected
// by verifying that the high watermark can still advance.
int leaderId = cluster . latestLeader ( ) . orElseThrow ( ( ) - >
new AssertionError ( "Failed to find current leader" )
) ;
if ( isGracefulShutdown ) {
cluster . shutdown ( leaderId ) ;
} else {
cluster . kill ( leaderId ) ;
}
scheduler . runUntil ( ( ) - > cluster . allReachedHighWatermark ( 20 ) ) ;
long highWatermark = cluster . maxHighWatermarkReached ( ) ;
// Restart the node and verify it catches up
cluster . start ( leaderId ) ;
scheduler . runUntil ( ( ) - > cluster . allReachedHighWatermark ( highWatermark + 10 ) ) ;
}
@Test
public void testElectionAfterMultiNodeNetworkPartitionQuorumSizeFiveAndTwoObservers ( ) {
checkElectionAfterMultiNodeNetworkPartition ( new QuorumConfig ( 5 , 2 ) ) ;
@Property ( tries = 100 )
void canRecoverAfterAllNodesKilled (
@ForAll Random random ,
@ForAll @IntRange ( min = 1 , max = 5 ) int numVoters ,
@ForAll @IntRange ( min = 0 , max = 5 ) int numObservers
) {
Cluster cluster = new Cluster ( numVoters , numObservers , random ) ;
MessageRouter router = new MessageRouter ( cluster ) ;
EventScheduler scheduler = schedulerWithDefaultInvariants ( cluster ) ;
// Seed the cluster with some data
cluster . startAll ( ) ;
schedulePolling ( scheduler , cluster , 3 , 5 ) ;
scheduler . schedule ( router : : deliverAll , 0 , 2 , 1 ) ;
scheduler . schedule ( new SequentialAppendAction ( cluster ) , 0 , 2 , 3 ) ;
scheduler . runUntil ( cluster : : hasConsistentLeader ) ;
scheduler . runUntil ( ( ) - > cluster . anyReachedHighWatermark ( 10 ) ) ;
long highWatermark = cluster . maxHighWatermarkReached ( ) ;
// We kill all of the nodes. Then we bring back a majority and verify that
// they are able to elect a leader and continue making progress
cluster . killAll ( ) ;
Iterator < Integer > nodeIdsIterator = cluster . nodes ( ) . iterator ( ) ;
for ( int i = 0 ; i < cluster . majoritySize ( ) ; i + + ) {
Integer nodeId = nodeIdsIterator . next ( ) ;
cluster . start ( nodeId ) ;
}
scheduler . runUntil ( ( ) - > cluster . allReachedHighWatermark ( highWatermark + 10 ) ) ;
}
private void checkElectionAfterMultiNodeNetworkPartition ( QuorumConfig config ) {
for ( int seed = 0 ; seed < 100 ; seed + + ) {
Cluster cluster = new Cluster ( config , seed ) ;
MessageRouter router = new MessageRouter ( cluster ) ;
EventScheduler scheduler = schedulerWithDefaultInvariants ( cluster ) ;
// Seed the cluster with some data
cluster . startAll ( ) ;
schedulePolling ( scheduler , cluster , 3 , 5 ) ;
scheduler . schedule ( router : : deliverAll , 0 , 2 , 2 ) ;
scheduler . schedule ( new SequentialAppendAction ( cluster ) , 0 , 2 , 3 ) ;
scheduler . runUntil ( cluster : : hasConsistentLeader ) ;
scheduler . runUntil ( ( ) - > cluster . anyReachedHighWatermark ( 10 ) ) ;
// Partition the nodes into two sets. Nodes are reachable within each set,
// but the two sets cannot communicate with each other. We should be able
// to make progress even if an election is needed in the larger set.
router . filter ( 0 , new DropOutboundRequestsFrom ( Utils . mkSet ( 2 , 3 , 4 ) ) ) ;
router . filter ( 1 , new DropOutboundRequestsFrom ( Utils . mkSet ( 2 , 3 , 4 ) ) ) ;
router . filter ( 2 , new DropOutboundRequestsFrom ( Utils . mkSet ( 0 , 1 ) ) ) ;
router . filter ( 3 , new DropOutboundRequestsFrom ( Utils . mkSet ( 0 , 1 ) ) ) ;
router . filter ( 4 , new DropOutboundRequestsFrom ( Utils . mkSet ( 0 , 1 ) ) ) ;
scheduler . runUntil ( ( ) - > cluster . anyReachedHighWatermark ( 20 ) ) ;
long minorityHighWatermark = cluster . maxHighWatermarkReached ( Utils . mkSet ( 0 , 1 ) ) ;
long majorityHighWatermark = cluster . maxHighWatermarkReached ( Utils . mkSet ( 2 , 3 , 4 ) ) ;
assertTrue ( majorityHighWatermark > minorityHighWatermark ) ;
// Now restore the partition and verify everyone catches up
router . filter ( 0 , new PermitAllTraffic ( ) ) ;
router . filter ( 1 , new PermitAllTraffic ( ) ) ;
router . filter ( 2 , new PermitAllTraffic ( ) ) ;
router . filter ( 3 , new PermitAllTraffic ( ) ) ;
router . filter ( 4 , new PermitAllTraffic ( ) ) ;
scheduler . runUntil ( ( ) - > cluster . allReachedHighWatermark ( 30 ) ) ;
}
@Property ( tries = 100 )
void canElectNewLeaderAfterOldLeaderPartitionedAway (
@ForAll Random random ,
@ForAll @IntRange ( min = 3 , max = 5 ) int numVoters ,
@ForAll @IntRange ( min = 0 , max = 5 ) int numObservers
) {
Cluster cluster = new Cluster ( numVoters , numObservers , random ) ;
MessageRouter router = new MessageRouter ( cluster ) ;
EventScheduler scheduler = schedulerWithDefaultInvariants ( cluster ) ;
// Seed the cluster with some data
cluster . startAll ( ) ;
schedulePolling ( scheduler , cluster , 3 , 5 ) ;
scheduler . schedule ( router : : deliverAll , 0 , 2 , 2 ) ;
scheduler . schedule ( new SequentialAppendAction ( cluster ) , 0 , 2 , 3 ) ;
scheduler . runUntil ( cluster : : hasConsistentLeader ) ;
scheduler . runUntil ( ( ) - > cluster . anyReachedHighWatermark ( 10 ) ) ;
// The leader gets partitioned off. We can verify the new leader has been elected
// by writing some data and ensuring that it gets replicated
int leaderId = cluster . latestLeader ( ) . orElseThrow ( ( ) - >
new AssertionError ( "Failed to find current leader" )
) ;
router . filter ( leaderId , new DropAllTraffic ( ) ) ;
Set < Integer > nonPartitionedNodes = new HashSet < > ( cluster . nodes ( ) ) ;
nonPartitionedNodes . remove ( leaderId ) ;
scheduler . runUntil ( ( ) - > cluster . allReachedHighWatermark ( 20 , nonPartitionedNodes ) ) ;
}
@Test
public void testBackToBackLeaderFailuresQuorumSizeThree ( ) {
checkBackToBackLeaderFailures ( new QuorumConfig ( 3 ) ) ;
@Property ( tries = 100 )
void canMakeProgressIfMajorityIsReachable (
@ForAll Random random ,
@ForAll @IntRange ( min = 0 , max = 3 ) int numObservers
) {
int numVoters = 5 ;
Cluster cluster = new Cluster ( numVoters , numObservers , random ) ;
MessageRouter router = new MessageRouter ( cluster ) ;
EventScheduler scheduler = schedulerWithDefaultInvariants ( cluster ) ;
// Seed the cluster with some data
cluster . startAll ( ) ;
schedulePolling ( scheduler , cluster , 3 , 5 ) ;
scheduler . schedule ( router : : deliverAll , 0 , 2 , 2 ) ;
scheduler . schedule ( new SequentialAppendAction ( cluster ) , 0 , 2 , 3 ) ;
scheduler . runUntil ( cluster : : hasConsistentLeader ) ;
scheduler . runUntil ( ( ) - > cluster . anyReachedHighWatermark ( 10 ) ) ;
// Partition the nodes into two sets. Nodes are reachable within each set,
// but the two sets cannot communicate with each other. We should be able
// to make progress even if an election is needed in the larger set.
router . filter ( 0 , new DropOutboundRequestsFrom ( Utils . mkSet ( 2 , 3 , 4 ) ) ) ;
router . filter ( 1 , new DropOutboundRequestsFrom ( Utils . mkSet ( 2 , 3 , 4 ) ) ) ;
router . filter ( 2 , new DropOutboundRequestsFrom ( Utils . mkSet ( 0 , 1 ) ) ) ;
router . filter ( 3 , new DropOutboundRequestsFrom ( Utils . mkSet ( 0 , 1 ) ) ) ;
router . filter ( 4 , new DropOutboundRequestsFrom ( Utils . mkSet ( 0 , 1 ) ) ) ;
scheduler . runUntil ( ( ) - > cluster . anyReachedHighWatermark ( 20 ) ) ;
long minorityHighWatermark = cluster . maxHighWatermarkReached ( Utils . mkSet ( 0 , 1 ) ) ;
long majorityHighWatermark = cluster . maxHighWatermarkReached ( Utils . mkSet ( 2 , 3 , 4 ) ) ;
assertTrue ( majorityHighWatermark > minorityHighWatermark ) ;
// Now restore the partition and verify everyone catches up
router . filter ( 0 , new PermitAllTraffic ( ) ) ;
router . filter ( 1 , new PermitAllTraffic ( ) ) ;
router . filter ( 2 , new PermitAllTraffic ( ) ) ;
router . filter ( 3 , new PermitAllTraffic ( ) ) ;
router . filter ( 4 , new PermitAllTraffic ( ) ) ;
scheduler . runUntil ( ( ) - > cluster . allReachedHighWatermark ( 30 ) ) ;
}
@Test
public void testBackToBackLeaderFailuresQuorumSizeFiveAndTwoObservers ( ) {
checkBackToBackLeaderFailures ( new QuorumConfig ( 5 , 2 ) ) ;
}
private void checkBackToBackLeaderFailures ( QuorumConfig config ) {
for ( int seed = 0 ; seed < 100 ; seed + + ) {
Cluster cluster = new Cluster ( config , seed ) ;
MessageRouter router = new MessageRouter ( cluster ) ;
EventScheduler scheduler = schedulerWithDefaultInvariants ( cluster ) ;
// Seed the cluster with some data
cluster . startAll ( ) ;
schedulePolling ( scheduler , cluster , 3 , 5 ) ;
scheduler . schedule ( router : : deliverAll , 0 , 2 , 5 ) ;
scheduler . schedule ( new SequentialAppendAction ( cluster ) , 0 , 2 , 3 ) ;
scheduler . runUntil ( cluster : : hasConsistentLeader ) ;
scheduler . runUntil ( ( ) - > cluster . anyReachedHighWatermark ( 10 ) ) ;
int leaderId = cluster . latestLeader ( ) . getAsInt ( ) ;
router . filter ( leaderId , new DropAllTraffic ( ) ) ;
scheduler . runUntil ( ( ) - > cluster . latestLeader ( ) . isPresent ( ) & & cluster . latestLeader ( ) . getAsInt ( ) ! = leaderId ) ;
// As soon as we have a new leader, restore traffic to the old leader and partition the new leader
int newLeaderId = cluster . latestLeader ( ) . getAsInt ( ) ;
router . filter ( leaderId , new PermitAllTraffic ( ) ) ;
router . filter ( newLeaderId , new DropAllTraffic ( ) ) ;
// Verify now that we can make progress
long targetHighWatermark = cluster . maxHighWatermarkReached ( ) + 10 ;
scheduler . runUntil ( ( ) - > cluster . anyReachedHighWatermark ( targetHighWatermark ) ) ;
}
@Property ( tries = 100 )
void canMakeProgressAfterBackToBackLeaderFailures (
@ForAll Random random ,
@ForAll @IntRange ( min = 3 , max = 5 ) int numVoters ,
@ForAll @IntRange ( min = 0 , max = 5 ) int numObservers
) {
Cluster cluster = new Cluster ( numVoters , numObservers , random ) ;
MessageRouter router = new MessageRouter ( cluster ) ;
EventScheduler scheduler = schedulerWithDefaultInvariants ( cluster ) ;
// Seed the cluster with some data
cluster . startAll ( ) ;
schedulePolling ( scheduler , cluster , 3 , 5 ) ;
scheduler . schedule ( router : : deliverAll , 0 , 2 , 5 ) ;
scheduler . schedule ( new SequentialAppendAction ( cluster ) , 0 , 2 , 3 ) ;
scheduler . runUntil ( cluster : : hasConsistentLeader ) ;
scheduler . runUntil ( ( ) - > cluster . anyReachedHighWatermark ( 10 ) ) ;
int leaderId = cluster . latestLeader ( ) . getAsInt ( ) ;
router . filter ( leaderId , new DropAllTraffic ( ) ) ;
scheduler . runUntil ( ( ) - > cluster . latestLeader ( ) . isPresent ( ) & & cluster . latestLeader ( ) . getAsInt ( ) ! = leaderId ) ;
// As soon as we have a new leader, restore traffic to the old leader and partition the new leader
int newLeaderId = cluster . latestLeader ( ) . getAsInt ( ) ;
router . filter ( leaderId , new PermitAllTraffic ( ) ) ;
router . filter ( newLeaderId , new DropAllTraffic ( ) ) ;
// Verify now that we can make progress
long targetHighWatermark = cluster . maxHighWatermarkReached ( ) + 10 ;
scheduler . runUntil ( ( ) - > cluster . anyReachedHighWatermark ( targetHighWatermark ) ) ;
}
private EventScheduler schedulerWithDefaultInvariants ( Cluster cluster ) {
@ -495,6 +398,8 @@ public class RaftEventSimulationTest {
@@ -495,6 +398,8 @@ public class RaftEventSimulationTest {
}
private static class EventScheduler {
private static final int MAX_ITERATIONS = 500000 ;
final AtomicInteger eventIdGenerator = new AtomicInteger ( 0 ) ;
final PriorityQueue < Event > queue = new PriorityQueue < > ( ) ;
final Random random ;
@ -525,9 +430,15 @@ public class RaftEventSimulationTest {
@@ -525,9 +430,15 @@ public class RaftEventSimulationTest {
}
void runUntil ( Supplier < Boolean > exitCondition ) {
while ( ! exitCondition . get ( ) ) {
if ( queue . isEmpty ( ) )
for ( int iteration = 0 ; iteration < MAX_ITERATIONS ; iteration + + ) {
if ( exitCondition . get ( ) ) {
break ;
}
if ( queue . isEmpty ( ) ) {
throw new IllegalStateException ( "Event queue exhausted before condition was satisfied" ) ;
}
Event event = queue . poll ( ) ;
long delayMs = Math . max ( event . deadlineMs - time . milliseconds ( ) , 0 ) ;
time . sleep ( delayMs ) ;
@ -535,23 +446,11 @@ public class RaftEventSimulationTest {
@@ -535,23 +446,11 @@ public class RaftEventSimulationTest {
invariants . forEach ( Invariant : : verify ) ;
}
validations . forEach ( Validation : : validate ) ;
}
}
private static class QuorumConfig {
final int numVoters ;
final int numObservers ;
private QuorumConfig ( int numVoters , int numObservers ) {
this . numVoters = numVoters ;
this . numObservers = numObservers ;
}
assertTrue ( exitCondition . get ( ) , "Simulation condition was not satisfied after "
+ MAX_ITERATIONS + " iterations" ) ;
private QuorumConfig ( int numVoters ) {
this ( numVoters , 0 ) ;
validations . forEach ( Validation : : validate ) ;
}
}
private static class PersistentState {
@ -568,16 +467,16 @@ public class RaftEventSimulationTest {
@@ -568,16 +467,16 @@ public class RaftEventSimulationTest {
final Map < Integer , PersistentState > nodes = new HashMap < > ( ) ;
final Map < Integer , RaftNode > running = new HashMap < > ( ) ;
private Cluster ( QuorumConfig config , int randomSeed ) {
this . random = new Random ( randomSeed ) ;
private Cluster ( int numVoters , int numObservers , Random random ) {
this . random = random ;
int nodeId = 0 ;
for ( ; nodeId < config . numVoters ; nodeId + + ) {
for ( ; nodeId < numVoters ; nodeId + + ) {
voters . add ( nodeId ) ;
nodes . put ( nodeId , new PersistentState ( ) ) ;
}
for ( ; nodeId < config . numVoters + config . numObservers ; nodeId + + ) {
for ( ; nodeId < numVoters + numObservers ; nodeId + + ) {
nodes . put ( nodeId , new PersistentState ( ) ) ;
}
}
@ -606,12 +505,12 @@ public class RaftEventSimulationTest {
@@ -606,12 +505,12 @@ public class RaftEventSimulationTest {
boolean anyReachedHighWatermark ( long offset ) {
return running . values ( ) . stream ( )
. anyMatch ( node - > node . client . quorum ( ) . highWatermark ( ) . map ( hw - > hw . offset ) . orElse ( 0L ) > offset ) ;
. anyMatch ( node - > node . highWatermark ( ) > offset ) ;
}
long maxHighWatermarkReached ( ) {
return running . values ( ) . stream ( )
. map ( node - > node . client . quorum ( ) . highWatermark ( ) . map ( hw - > hw . offset ) . orElse ( 0L ) )
. map ( RaftNode : : highWatermark )
. max ( Long : : compareTo )
. orElse ( 0L ) ;
}
@ -619,20 +518,19 @@ public class RaftEventSimulationTest {
@@ -619,20 +518,19 @@ public class RaftEventSimulationTest {
long maxHighWatermarkReached ( Set < Integer > nodeIds ) {
return running . values ( ) . stream ( )
. filter ( node - > nodeIds . contains ( node . nodeId ) )
. map ( node - > node . client . quorum ( ) . highWatermark ( ) . map ( hw - > hw . offset ) . orElse ( 0L ) )
. map ( RaftNode : : highWatermark )
. max ( Long : : compareTo )
. orElse ( 0L ) ;
}
boolean allReachedHighWatermark ( long offset , Set < Integer > nodeIds ) {
return nodeIds . stream ( )
. allMatch ( nodeId - > running . get ( nodeId ) . client . quorum ( ) . highWatermark ( ) . map ( hw - > hw . offset )
. orElse ( 0L ) > offset ) ;
. allMatch ( nodeId - > running . get ( nodeId ) . highWatermark ( ) > offset ) ;
}
boolean allReachedHighWatermark ( long offset ) {
return running . values ( ) . stream ( )
. allMatch ( node - > node . client . quorum ( ) . highWatermark ( ) . map ( hw - > hw . offset ) . orElse ( 0L ) > offset ) ;
. allMatch ( node - > node . highWatermark ( ) > offset ) ;
}
OptionalInt latestLeader ( ) {
@ -835,6 +733,17 @@ public class RaftEventSimulationTest {
@@ -835,6 +733,17 @@ public class RaftEventSimulationTest {
throw new RuntimeException ( "Uncaught exception during poll of node " + nodeId , e ) ;
}
}
long highWatermark ( ) {
return client . quorum ( ) . highWatermark ( )
. map ( hw - > hw . offset )
. orElse ( 0L ) ;
}
@Override
public String toString ( ) {
return "Node(id=" + nodeId + ", hw=" + highWatermark ( ) + ")" ;
}
}
private static class InflightRequest {