@ -21,7 +21,6 @@ import org.apache.kafka.common.utils.LogContext;
@@ -21,7 +21,6 @@ import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.MockTime ;
import org.apache.kafka.common.utils.Utils ;
import org.apache.kafka.raft.internals.BatchAccumulator ;
import org.junit.jupiter.api.Test ;
import org.junit.jupiter.params.ParameterizedTest ;
import org.junit.jupiter.params.provider.ValueSource ;
@ -29,16 +28,13 @@ import org.mockito.Mockito;
@@ -29,16 +28,13 @@ import org.mockito.Mockito;
import java.util.Arrays ;
import java.util.Collections ;
import java.util.List ;
import java.util.Objects ;
import java.util.Optional ;
import java.util.Set ;
import java.util.stream.Collectors ;
import static java.util.Collections.emptyList ;
import static java.util.Collections.emptySet ;
import static java.util.Collections.singleton ;
import static org.apache.kafka.common.utils.Utils.mkEntry ;
import static org.apache.kafka.common.utils.Utils.mkMap ;
import static org.apache.kafka.common.utils.Utils.mkSet ;
import static org.junit.jupiter.api.Assertions.assertEquals ;
import static org.junit.jupiter.api.Assertions.assertFalse ;
@ -103,12 +99,12 @@ public class LeaderStateTest {
@@ -103,12 +99,12 @@ public class LeaderStateTest {
public void testUpdateHighWatermarkQuorumSizeOne ( ) {
LeaderState < ? > state = newLeaderState ( singleton ( localId ) , 15L ) ;
assertEquals ( Optional . empty ( ) , state . highWatermark ( ) ) ;
assertFalse ( state . updateLocalState ( 0 , new LogOffsetMetadata ( 15L ) ) ) ;
assertFalse ( state . updateLocalState ( new LogOffsetMetadata ( 15L ) ) ) ;
assertEquals ( emptySet ( ) , state . nonAcknowledgingVoters ( ) ) ;
assertEquals ( Optional . empty ( ) , state . highWatermark ( ) ) ;
assertTrue ( state . updateLocalState ( 0 , new LogOffsetMetadata ( 16L ) ) ) ;
assertTrue ( state . updateLocalState ( new LogOffsetMetadata ( 16L ) ) ) ;
assertEquals ( Optional . of ( new LogOffsetMetadata ( 16L ) ) , state . highWatermark ( ) ) ;
assertTrue ( state . updateLocalState ( 0 , new LogOffsetMetadata ( 20 ) ) ) ;
assertTrue ( state . updateLocalState ( new LogOffsetMetadata ( 20 ) ) ) ;
assertEquals ( Optional . of ( new LogOffsetMetadata ( 20L ) ) , state . highWatermark ( ) ) ;
}
@ -116,10 +112,10 @@ public class LeaderStateTest {
@@ -116,10 +112,10 @@ public class LeaderStateTest {
public void testNonMonotonicLocalEndOffsetUpdate ( ) {
LeaderState < ? > state = newLeaderState ( singleton ( localId ) , 15L ) ;
assertEquals ( Optional . empty ( ) , state . highWatermark ( ) ) ;
assertTrue ( state . updateLocalState ( 0 , new LogOffsetMetadata ( 16L ) ) ) ;
assertTrue ( state . updateLocalState ( new LogOffsetMetadata ( 16L ) ) ) ;
assertEquals ( Optional . of ( new LogOffsetMetadata ( 16L ) ) , state . highWatermark ( ) ) ;
assertThrows ( IllegalStateException . class ,
( ) - > state . updateLocalState ( 0 , new LogOffsetMetadata ( 15L ) ) ) ;
( ) - > state . updateLocalState ( new LogOffsetMetadata ( 15L ) ) ) ;
}
@Test
@ -128,49 +124,51 @@ public class LeaderStateTest {
@@ -128,49 +124,51 @@ public class LeaderStateTest {
int node2 = 2 ;
int currentTime = 1000 ;
int fetchTime = 0 ;
int caughtu pTime = - 1 ;
int caughtU pTime = - 1 ;
LeaderState < ? > state = newLeaderState ( mkSet ( localId , node1 , node2 ) , 10L ) ;
assertEquals ( Optional . empty ( ) , state . highWatermark ( ) ) ;
assertFalse ( state . updateLocalState ( + + fetchTime , new LogOffsetMetadata ( 10L ) ) ) ;
assertFalse ( state . updateLocalState ( new LogOffsetMetadata ( 10L ) ) ) ;
assertEquals ( mkSet ( node1 , node2 ) , state . nonAcknowledgingVoters ( ) ) ;
assertEquals ( Optional . empty ( ) , state . highWatermark ( ) ) ;
// Node 1 falls behind
assertFalse ( state . updateLocalState ( + + fetchTime , new LogOffsetMetadata ( 10 L ) ) ) ;
assertFalse ( state . updateReplicaState ( node1 , + + fetchTime , new LogOffsetMetadata ( 10L ) , 11L ) ) ;
assertEquals ( currentTime , state . quorumResponseVoterStates ( currentTime ) . get ( localId ) . lastCaughtUpTimestamp ( ) ) ;
assertEquals ( caughtupTime , state . quorumResponseVoterStates ( currentTime ) . get ( node1 ) . lastCaughtUpTimestamp ( ) ) ;
assertFalse ( state . updateLocalState ( new LogOffsetMetadata ( 11 L ) ) ) ;
assertFalse ( state . updateReplicaState ( node1 , + + fetchTime , new LogOffsetMetadata ( 10L ) ) ) ;
assertEquals ( currentTime , describeVoterState ( state , localId , currentTime ) . lastCaughtUpTimestamp ( ) ) ;
assertEquals ( caughtUpTime , describeVoterState ( state , node1 , currentTime ) . lastCaughtUpTimestamp ( ) ) ;
// Node 1 catches up to leader
assertFals e ( state . updateReplicaState ( node1 , + + fetchTime , new LogOffsetMetadata ( 11L ) , 11L ) ) ;
caughtu pTime = fetchTime ;
assertEquals ( currentTime , state . quorumResponseVoterStates ( currentTime ) . get ( localId ) . lastCaughtUpTimestamp ( ) ) ;
assertEquals ( caughtupTime , state . quorumResponseVoterStates ( currentTime ) . get ( node1 ) . lastCaughtUpTimestamp ( ) ) ;
assertTru e ( state . updateReplicaState ( node1 , + + fetchTime , new LogOffsetMetadata ( 11L ) ) ) ;
caughtU pTime = fetchTime ;
assertEquals ( currentTime , describeVoterState ( state , localId , currentTime ) . lastCaughtUpTimestamp ( ) ) ;
assertEquals ( caughtUpTime , describeVoterState ( state , node1 , currentTime ) . lastCaughtUpTimestamp ( ) ) ;
// Node 1 falls behind
assertFalse ( state . updateReplicaState ( node1 , + + fetchTime , new LogOffsetMetadata ( 50L ) , 100L ) ) ;
assertEquals ( currentTime , state . quorumResponseVoterStates ( currentTime ) . get ( localId ) . lastCaughtUpTimestamp ( ) ) ;
assertEquals ( caughtupTime , state . quorumResponseVoterStates ( currentTime ) . get ( node1 ) . lastCaughtUpTimestamp ( ) ) ;
assertFalse ( state . updateLocalState ( new LogOffsetMetadata ( 100L ) ) ) ;
assertTrue ( state . updateReplicaState ( node1 , + + fetchTime , new LogOffsetMetadata ( 50L ) ) ) ;
assertEquals ( currentTime , describeVoterState ( state , localId , currentTime ) . lastCaughtUpTimestamp ( ) ) ;
assertEquals ( caughtUpTime , describeVoterState ( state , node1 , currentTime ) . lastCaughtUpTimestamp ( ) ) ;
// Node 1 catches up to the last fetch offset
int prevFetchTime = fetchTime ;
assertFalse ( state . updateReplicaState ( node1 , + + fetchTime , new LogOffsetMetadata ( 102L ) , 200L ) ) ;
caughtupTime = prevFetchTime ;
assertEquals ( currentTime , state . quorumResponseVoterStates ( currentTime ) . get ( localId ) . lastCaughtUpTimestamp ( ) ) ;
assertEquals ( caughtupTime , state . quorumResponseVoterStates ( currentTime ) . get ( node1 ) . lastCaughtUpTimestamp ( ) ) ;
assertFalse ( state . updateLocalState ( new LogOffsetMetadata ( 200L ) ) ) ;
assertTrue ( state . updateReplicaState ( node1 , + + fetchTime , new LogOffsetMetadata ( 100L ) ) ) ;
caughtUpTime = prevFetchTime ;
assertEquals ( currentTime , describeVoterState ( state , localId , currentTime ) . lastCaughtUpTimestamp ( ) ) ;
assertEquals ( caughtUpTime , describeVoterState ( state , node1 , currentTime ) . lastCaughtUpTimestamp ( ) ) ;
// Node2 has never caught up to leader
assertEquals ( - 1L , state . quorumResponseVoterStates ( currentTime ) . get ( node2 ) . lastCaughtUpTimestamp ( ) ) ;
assertTrue ( state . updateReplicaState ( node2 , + + fetchTime , new LogOffsetMetadata ( 202L ) , 300L ) ) ;
assertEquals ( - 1L , state . quorumResponseVoterStates ( currentTime ) . get ( node2 ) . lastCaughtUpTimestamp ( ) ) ;
assertFalse ( state . updateReplicaState ( node2 , + + fetchTime , new LogOffsetMetadata ( 250L ) , 300L ) ) ;
assertEquals ( - 1L , state . quorumResponseVoterStates ( currentTime ) . get ( node2 ) . lastCaughtUpTimestamp ( ) ) ;
assertEquals ( - 1L , describeVoterState ( state , node2 , currentTime ) . lastCaughtUpTimestamp ( ) ) ;
assertFalse ( state . updateLocalState ( new LogOffsetMetadata ( 300L ) ) ) ;
assertTrue ( state . updateReplicaState ( node2 , + + fetchTime , new LogOffsetMetadata ( 200L ) ) ) ;
assertEquals ( - 1L , describeVoterState ( state , node2 , currentTime ) . lastCaughtUpTimestamp ( ) ) ;
assertTrue ( state . updateReplicaState ( node2 , + + fetchTime , new LogOffsetMetadata ( 250L ) ) ) ;
assertEquals ( - 1L , describeVoterState ( state , node2 , currentTime ) . lastCaughtUpTimestamp ( ) ) ;
}
@Test
public void testLastCaughtUpTimeObserver ( ) {
int node1Index = 0 ;
int node1Id = 1 ;
int node1 = 1 ;
int currentTime = 1000 ;
int fetchTime = 0 ;
int caughtUpTime = - 1 ;
@ -179,42 +177,44 @@ public class LeaderStateTest {
@@ -179,42 +177,44 @@ public class LeaderStateTest {
assertEquals ( emptySet ( ) , state . nonAcknowledgingVoters ( ) ) ;
// Node 1 falls behind
assertTrue ( state . updateLocalState ( + + fetchTime , new LogOffsetMetadata ( 10 L ) ) ) ;
assertFalse ( state . updateReplicaState ( node1Id , + + fetchTime , new LogOffsetMetadata ( 10L ) , 11L ) ) ;
assertEquals ( currentTime , state . quorumResponseVoterStates ( currentTime ) . get ( localId ) . lastCaughtUpTimestamp ( ) ) ;
assertEquals ( caughtUpTime , state . quorumResponseObserverStates ( currentTime ) . get ( node1Index ) . lastCaughtUpTimestamp ( ) ) ;
assertTrue ( state . updateLocalState ( new LogOffsetMetadata ( 11 L ) ) ) ;
assertFalse ( state . updateReplicaState ( node1 , + + fetchTime , new LogOffsetMetadata ( 10L ) ) ) ;
assertEquals ( currentTime , describeVoterState ( state , localId , currentTime ) . lastCaughtUpTimestamp ( ) ) ;
assertEquals ( caughtUpTime , describeObserverState ( state , node1 , currentTime ) . lastCaughtUpTimestamp ( ) ) ;
// Node 1 catches up to leader
assertFalse ( state . updateReplicaState ( node1Id , + + fetchTime , new LogOffsetMetadata ( 11L ) , 11L ) ) ;
assertFalse ( state . updateReplicaState ( node1 , + + fetchTime , new LogOffsetMetadata ( 11L ) ) ) ;
caughtUpTime = fetchTime ;
assertEquals ( currentTime , state . quorumResponseVoterStates ( currentTime ) . get ( localId ) . lastCaughtUpTimestamp ( ) ) ;
assertEquals ( caughtUpTime , state . quorumResponseObserverStates ( currentTime ) . get ( node1Index ) . lastCaughtUpTimestamp ( ) ) ;
assertEquals ( currentTime , describeVoterState ( state , localId , currentTime ) . lastCaughtUpTimestamp ( ) ) ;
assertEquals ( caughtUpTime , describeObserverState ( state , node1 , currentTime ) . lastCaughtUpTimestamp ( ) ) ;
// Node 1 falls behind
assertFalse ( state . updateReplicaState ( node1Id , + + fetchTime , new LogOffsetMetadata ( 50L ) , 100L ) ) ;
assertEquals ( currentTime , state . quorumResponseVoterStates ( currentTime ) . get ( localId ) . lastCaughtUpTimestamp ( ) ) ;
assertEquals ( caughtUpTime , state . quorumResponseObserverStates ( currentTime ) . get ( node1Index ) . lastCaughtUpTimestamp ( ) ) ;
assertTrue ( state . updateLocalState ( new LogOffsetMetadata ( 100L ) ) ) ;
assertFalse ( state . updateReplicaState ( node1 , + + fetchTime , new LogOffsetMetadata ( 50L ) ) ) ;
assertEquals ( currentTime , describeVoterState ( state , localId , currentTime ) . lastCaughtUpTimestamp ( ) ) ;
assertEquals ( caughtUpTime , describeObserverState ( state , node1 , currentTime ) . lastCaughtUpTimestamp ( ) ) ;
// Node 1 catches up to the last fetch offset
int prevFetchTime = fetchTime ;
assertFalse ( state . updateReplicaState ( node1Id , + + fetchTime , new LogOffsetMetadata ( 102L ) , 200L ) ) ;
assertTrue ( state . updateLocalState ( new LogOffsetMetadata ( 200L ) ) ) ;
assertFalse ( state . updateReplicaState ( node1 , + + fetchTime , new LogOffsetMetadata ( 102L ) ) ) ;
caughtUpTime = prevFetchTime ;
assertEquals ( currentTime , state . quorumResponseVoterStates ( currentTime ) . get ( localId ) . lastCaughtUpTimestamp ( ) ) ;
assertEquals ( caughtUpTime , state . quorumResponseObserverStates ( currentTime ) . get ( node1Index ) . lastCaughtUpTimestamp ( ) ) ;
assertEquals ( currentTime , describeVoterState ( state , localId , currentTime ) . lastCaughtUpTimestamp ( ) ) ;
assertEquals ( caughtUpTime , describeObserverState ( state , node1 , currentTime ) . lastCaughtUpTimestamp ( ) ) ;
// Node 1 catches up to leader
assertFalse ( state . updateReplicaState ( node1Id , + + fetchTime , new LogOffsetMetadata ( 202L ) , 200L ) ) ;
assertFalse ( state . updateReplicaState ( node1 , + + fetchTime , new LogOffsetMetadata ( 200L ) ) ) ;
caughtUpTime = fetchTime ;
assertEquals ( currentTime , state . quorumResponseVoterStates ( currentTime ) . get ( localId ) . lastCaughtUpTimestamp ( ) ) ;
assertEquals ( caughtUpTime , state . quorumResponseObserverStates ( currentTime ) . get ( node1Index ) . lastCaughtUpTimestamp ( ) ) ;
assertEquals ( currentTime , describeVoterState ( state , localId , currentTime ) . lastCaughtUpTimestamp ( ) ) ;
assertEquals ( caughtUpTime , describeObserverState ( state , node1 , currentTime ) . lastCaughtUpTimestamp ( ) ) ;
}
@Test
public void testIdempotentEndOffsetUpdate ( ) {
LeaderState < ? > state = newLeaderState ( singleton ( localId ) , 15L ) ;
assertEquals ( Optional . empty ( ) , state . highWatermark ( ) ) ;
assertTrue ( state . updateLocalState ( 0 , new LogOffsetMetadata ( 16L ) ) ) ;
assertFalse ( state . updateLocalState ( 0 , new LogOffsetMetadata ( 16L ) ) ) ;
assertTrue ( state . updateLocalState ( new LogOffsetMetadata ( 16L ) ) ) ;
assertFalse ( state . updateLocalState ( new LogOffsetMetadata ( 16L ) ) ) ;
assertEquals ( Optional . of ( new LogOffsetMetadata ( 16L ) ) , state . highWatermark ( ) ) ;
}
@ -224,11 +224,11 @@ public class LeaderStateTest {
@@ -224,11 +224,11 @@ public class LeaderStateTest {
assertEquals ( Optional . empty ( ) , state . highWatermark ( ) ) ;
LogOffsetMetadata initialHw = new LogOffsetMetadata ( 16L , Optional . of ( new MockOffsetMetadata ( "bar" ) ) ) ;
assertTrue ( state . updateLocalState ( 0 , initialHw ) ) ;
assertTrue ( state . updateLocalState ( initialHw ) ) ;
assertEquals ( Optional . of ( initialHw ) , state . highWatermark ( ) ) ;
LogOffsetMetadata updateHw = new LogOffsetMetadata ( 16L , Optional . of ( new MockOffsetMetadata ( "baz" ) ) ) ;
assertTrue ( state . updateLocalState ( 0 , updateHw ) ) ;
assertTrue ( state . updateLocalState ( updateHw ) ) ;
assertEquals ( Optional . of ( updateHw ) , state . highWatermark ( ) ) ;
}
@ -236,15 +236,15 @@ public class LeaderStateTest {
@@ -236,15 +236,15 @@ public class LeaderStateTest {
public void testUpdateHighWatermarkQuorumSizeTwo ( ) {
int otherNodeId = 1 ;
LeaderState < ? > state = newLeaderState ( mkSet ( localId , otherNodeId ) , 10L ) ;
assertFalse ( state . updateLocalState ( 0 , new LogOffsetMetadata ( 13L ) ) ) ;
assertFalse ( state . updateLocalState ( new LogOffsetMetadata ( 13L ) ) ) ;
assertEquals ( singleton ( otherNodeId ) , state . nonAcknowledgingVoters ( ) ) ;
assertEquals ( Optional . empty ( ) , state . highWatermark ( ) ) ;
assertFalse ( state . updateReplicaState ( otherNodeId , 0 , new LogOffsetMetadata ( 10L ) , 11L ) ) ;
assertFalse ( state . updateReplicaState ( otherNodeId , 0 , new LogOffsetMetadata ( 10L ) ) ) ;
assertEquals ( emptySet ( ) , state . nonAcknowledgingVoters ( ) ) ;
assertEquals ( Optional . empty ( ) , state . highWatermark ( ) ) ;
assertTrue ( state . updateReplicaState ( otherNodeId , 0 , new LogOffsetMetadata ( 11L ) , 12L ) ) ;
assertTrue ( state . updateReplicaState ( otherNodeId , 0 , new LogOffsetMetadata ( 11L ) ) ) ;
assertEquals ( Optional . of ( new LogOffsetMetadata ( 11L ) ) , state . highWatermark ( ) ) ;
assertTrue ( state . updateReplicaState ( otherNodeId , 0 , new LogOffsetMetadata ( 13L ) , 14L ) ) ;
assertTrue ( state . updateReplicaState ( otherNodeId , 0 , new LogOffsetMetadata ( 13L ) ) ) ;
assertEquals ( Optional . of ( new LogOffsetMetadata ( 13L ) ) , state . highWatermark ( ) ) ;
}
@ -253,22 +253,22 @@ public class LeaderStateTest {
@@ -253,22 +253,22 @@ public class LeaderStateTest {
int node1 = 1 ;
int node2 = 2 ;
LeaderState < ? > state = newLeaderState ( mkSet ( localId , node1 , node2 ) , 10L ) ;
assertFalse ( state . updateLocalState ( 0 , new LogOffsetMetadata ( 15L ) ) ) ;
assertFalse ( state . updateLocalState ( new LogOffsetMetadata ( 15L ) ) ) ;
assertEquals ( mkSet ( node1 , node2 ) , state . nonAcknowledgingVoters ( ) ) ;
assertEquals ( Optional . empty ( ) , state . highWatermark ( ) ) ;
assertFalse ( state . updateReplicaState ( node1 , 0 , new LogOffsetMetadata ( 10L ) , 11L ) ) ;
assertFalse ( state . updateReplicaState ( node1 , 0 , new LogOffsetMetadata ( 10L ) ) ) ;
assertEquals ( singleton ( node2 ) , state . nonAcknowledgingVoters ( ) ) ;
assertEquals ( Optional . empty ( ) , state . highWatermark ( ) ) ;
assertFalse ( state . updateReplicaState ( node2 , 0 , new LogOffsetMetadata ( 10L ) , 11L ) ) ;
assertFalse ( state . updateReplicaState ( node2 , 0 , new LogOffsetMetadata ( 10L ) ) ) ;
assertEquals ( emptySet ( ) , state . nonAcknowledgingVoters ( ) ) ;
assertEquals ( Optional . empty ( ) , state . highWatermark ( ) ) ;
assertTrue ( state . updateReplicaState ( node2 , 0 , new LogOffsetMetadata ( 15L ) , 16L ) ) ;
assertTrue ( state . updateReplicaState ( node2 , 0 , new LogOffsetMetadata ( 15L ) ) ) ;
assertEquals ( Optional . of ( new LogOffsetMetadata ( 15L ) ) , state . highWatermark ( ) ) ;
assertFalse ( state . updateLocalState ( 0 , new LogOffsetMetadata ( 20L ) ) ) ;
assertFalse ( state . updateLocalState ( new LogOffsetMetadata ( 20L ) ) ) ;
assertEquals ( Optional . of ( new LogOffsetMetadata ( 15L ) ) , state . highWatermark ( ) ) ;
assertTrue ( state . updateReplicaState ( node1 , 0 , new LogOffsetMetadata ( 20L ) , 21L ) ) ;
assertTrue ( state . updateReplicaState ( node1 , 0 , new LogOffsetMetadata ( 20L ) ) ) ;
assertEquals ( Optional . of ( new LogOffsetMetadata ( 20L ) ) , state . highWatermark ( ) ) ;
assertFalse ( state . updateReplicaState ( node2 , 0 , new LogOffsetMetadata ( 20L ) , 21L ) ) ;
assertFalse ( state . updateReplicaState ( node2 , 0 , new LogOffsetMetadata ( 20L ) ) ) ;
assertEquals ( Optional . of ( new LogOffsetMetadata ( 20L ) ) , state . highWatermark ( ) ) ;
}
@ -277,14 +277,14 @@ public class LeaderStateTest {
@@ -277,14 +277,14 @@ public class LeaderStateTest {
MockTime time = new MockTime ( ) ;
int node1 = 1 ;
LeaderState < ? > state = newLeaderState ( mkSet ( localId , node1 ) , 0L ) ;
state . updateLocalState ( time . milliseconds ( ) , new LogOffsetMetadata ( 10L ) ) ;
state . updateReplicaState ( node1 , time . milliseconds ( ) , new LogOffsetMetadata ( 10L ) , 11L ) ;
state . updateLocalState ( new LogOffsetMetadata ( 10L ) ) ;
state . updateReplicaState ( node1 , time . milliseconds ( ) , new LogOffsetMetadata ( 10L ) ) ;
assertEquals ( Optional . of ( new LogOffsetMetadata ( 10L ) ) , state . highWatermark ( ) ) ;
// Follower crashes and disk is lost. It fetches an earlier offset to rebuild state.
// The leader will report an error in the logs, but will not let the high watermark rewind
assertFalse ( state . updateReplicaState ( node1 , time . milliseconds ( ) , new LogOffsetMetadata ( 5L ) , 11L ) ) ;
assertEquals ( 5L , state . quorumResponseVoterStates ( time . milliseconds ( ) ) . get ( node1 ) . logEndOffset ( ) ) ;
assertFalse ( state . updateReplicaState ( node1 , time . milliseconds ( ) , new LogOffsetMetadata ( 5L ) ) ) ;
assertEquals ( 5L , describeVoterState ( state , node1 , time . milliseconds ( ) ) . logEndOffset ( ) ) ;
assertEquals ( Optional . of ( new LogOffsetMetadata ( 10L ) ) , state . highWatermark ( ) ) ;
}
@ -302,21 +302,102 @@ public class LeaderStateTest {
@@ -302,21 +302,102 @@ public class LeaderStateTest {
}
@Test
public void testGetVoterStates ( ) {
int node1 = 1 ;
int node2 = 2 ;
public void testDescribeQuorumWithSingleVoter ( ) {
MockTime time = new MockTime ( ) ;
long leaderStartOffset = 10L ;
long leaderEndOffset = 15L ;
LeaderState < ? > state = setUpLeaderAndFollowers ( node1 , node2 , leaderStartOffset , leaderEndOffset ) ;
LeaderState < ? > state = newLeaderState ( mkSet ( localId ) , leaderStartOffset ) ;
// Until we have updated local state, high watermark should be uninitialized
assertEquals ( Optional . empty ( ) , state . highWatermark ( ) ) ;
DescribeQuorumResponseData . PartitionData partitionData = state . describeQuorum ( time . milliseconds ( ) ) ;
assertEquals ( - 1 , partitionData . highWatermark ( ) ) ;
assertEquals ( localId , partitionData . leaderId ( ) ) ;
assertEquals ( epoch , partitionData . leaderEpoch ( ) ) ;
assertEquals ( Collections . emptyList ( ) , partitionData . observers ( ) ) ;
assertEquals ( 1 , partitionData . currentVoters ( ) . size ( ) ) ;
assertEquals ( new DescribeQuorumResponseData . ReplicaState ( )
. setReplicaId ( localId )
. setLogEndOffset ( - 1 )
. setLastFetchTimestamp ( time . milliseconds ( ) )
. setLastCaughtUpTimestamp ( time . milliseconds ( ) ) ,
partitionData . currentVoters ( ) . get ( 0 ) ) ;
// Now update the high watermark and verify the describe output
assertTrue ( state . updateLocalState ( new LogOffsetMetadata ( leaderEndOffset ) ) ) ;
assertEquals ( Optional . of ( new LogOffsetMetadata ( leaderEndOffset ) ) , state . highWatermark ( ) ) ;
time . sleep ( 500 ) ;
partitionData = state . describeQuorum ( time . milliseconds ( ) ) ;
assertEquals ( leaderEndOffset , partitionData . highWatermark ( ) ) ;
assertEquals ( localId , partitionData . leaderId ( ) ) ;
assertEquals ( epoch , partitionData . leaderEpoch ( ) ) ;
assertEquals ( Collections . emptyList ( ) , partitionData . observers ( ) ) ;
assertEquals ( 1 , partitionData . currentVoters ( ) . size ( ) ) ;
assertEquals ( new DescribeQuorumResponseData . ReplicaState ( )
. setReplicaId ( localId )
. setLogEndOffset ( leaderEndOffset )
. setLastFetchTimestamp ( time . milliseconds ( ) )
. setLastCaughtUpTimestamp ( time . milliseconds ( ) ) ,
partitionData . currentVoters ( ) . get ( 0 ) ) ;
}
@Test
public void testDescribeQuorumWithMultipleVoters ( ) {
MockTime time = new MockTime ( ) ;
int activeFollowerId = 1 ;
int inactiveFollowerId = 2 ;
long leaderStartOffset = 10L ;
long leaderEndOffset = 15L ;
LeaderState < ? > state = newLeaderState ( mkSet ( localId , activeFollowerId , inactiveFollowerId ) , leaderStartOffset ) ;
assertFalse ( state . updateLocalState ( new LogOffsetMetadata ( leaderEndOffset ) ) ) ;
assertEquals ( Optional . empty ( ) , state . highWatermark ( ) ) ;
assertEquals ( mkMap (
mkEntry ( localId , leaderEndOffset ) ,
mkEntry ( node1 , leaderStartOffset ) ,
mkEntry ( node2 , leaderEndOffset )
) , state . quorumResponseVoterStates ( 0 )
. stream ( )
. collect ( Collectors . toMap ( DescribeQuorumResponseData . ReplicaState : : replicaId , DescribeQuorumResponseData . ReplicaState : : logEndOffset ) ) ) ;
long activeFollowerFetchTimeMs = time . milliseconds ( ) ;
assertTrue ( state . updateReplicaState ( activeFollowerId , activeFollowerFetchTimeMs , new LogOffsetMetadata ( leaderEndOffset ) ) ) ;
assertEquals ( Optional . of ( new LogOffsetMetadata ( leaderEndOffset ) ) , state . highWatermark ( ) ) ;
time . sleep ( 500 ) ;
DescribeQuorumResponseData . PartitionData partitionData = state . describeQuorum ( time . milliseconds ( ) ) ;
assertEquals ( leaderEndOffset , partitionData . highWatermark ( ) ) ;
assertEquals ( localId , partitionData . leaderId ( ) ) ;
assertEquals ( epoch , partitionData . leaderEpoch ( ) ) ;
assertEquals ( Collections . emptyList ( ) , partitionData . observers ( ) ) ;
List < DescribeQuorumResponseData . ReplicaState > voterStates = partitionData . currentVoters ( ) ;
assertEquals ( 3 , voterStates . size ( ) ) ;
DescribeQuorumResponseData . ReplicaState leaderState =
findReplicaOrFail ( localId , partitionData . currentVoters ( ) ) ;
assertEquals ( new DescribeQuorumResponseData . ReplicaState ( )
. setReplicaId ( localId )
. setLogEndOffset ( leaderEndOffset )
. setLastFetchTimestamp ( time . milliseconds ( ) )
. setLastCaughtUpTimestamp ( time . milliseconds ( ) ) ,
leaderState ) ;
DescribeQuorumResponseData . ReplicaState activeFollowerState =
findReplicaOrFail ( activeFollowerId , partitionData . currentVoters ( ) ) ;
assertEquals ( new DescribeQuorumResponseData . ReplicaState ( )
. setReplicaId ( activeFollowerId )
. setLogEndOffset ( leaderEndOffset )
. setLastFetchTimestamp ( activeFollowerFetchTimeMs )
. setLastCaughtUpTimestamp ( activeFollowerFetchTimeMs ) ,
activeFollowerState ) ;
DescribeQuorumResponseData . ReplicaState inactiveFollowerState =
findReplicaOrFail ( inactiveFollowerId , partitionData . currentVoters ( ) ) ;
assertEquals ( new DescribeQuorumResponseData . ReplicaState ( )
. setReplicaId ( inactiveFollowerId )
. setLogEndOffset ( - 1 )
. setLastFetchTimestamp ( - 1 )
. setLastCaughtUpTimestamp ( - 1 ) ,
inactiveFollowerState ) ;
}
private LeaderState < ? > setUpLeaderAndFollowers ( int follower1 ,
@ -324,37 +405,60 @@ public class LeaderStateTest {
@@ -324,37 +405,60 @@ public class LeaderStateTest {
long leaderStartOffset ,
long leaderEndOffset ) {
LeaderState < ? > state = newLeaderState ( mkSet ( localId , follower1 , follower2 ) , leaderStartOffset ) ;
state . updateLocalState ( 0 , new LogOffsetMetadata ( leaderEndOffset ) ) ;
state . updateLocalState ( new LogOffsetMetadata ( leaderEndOffset ) ) ;
assertEquals ( Optional . empty ( ) , state . highWatermark ( ) ) ;
state . updateReplicaState ( follower1 , 0 , new LogOffsetMetadata ( leaderStartOffset ) , leaderEndOffset ) ;
state . updateReplicaState ( follower2 , 0 , new LogOffsetMetadata ( leaderEndOffset ) , leaderEndOffset ) ;
state . updateReplicaState ( follower1 , 0 , new LogOffsetMetadata ( leaderStartOffset ) ) ;
state . updateReplicaState ( follower2 , 0 , new LogOffsetMetadata ( leaderEndOffset ) ) ;
return state ;
}
@Test
public void testGetObserverStatesWithObserver ( ) {
public void testDescribeQuorumWithObservers ( ) {
MockTime time = new MockTime ( ) ;
int observerId = 10 ;
long epochStartOffset = 10L ;
LeaderState < ? > state = newLeaderState ( mkSet ( localId ) , epochStartOffset ) ;
long timestamp = 20L ;
assertFalse ( state . updateReplicaState ( observerId , timestamp , new LogOffsetMetadata ( epochStartOffset ) , epochStartOffset + 10 ) ) ;
assertEquals ( Collections . singletonMap ( observerId , epochStartOffset ) ,
state . quorumResponseObserverStates ( timestamp )
. stream ( )
. collect ( Collectors . toMap ( DescribeQuorumResponseData . ReplicaState : : replicaId , DescribeQuorumResponseData . ReplicaState : : logEndOffset ) ) ) ;
assertTrue ( state . updateLocalState ( new LogOffsetMetadata ( epochStartOffset + 1 ) ) ) ;
assertEquals ( Optional . of ( new LogOffsetMetadata ( epochStartOffset + 1 ) ) , state . highWatermark ( ) ) ;
time . sleep ( 500 ) ;
long observerFetchTimeMs = time . milliseconds ( ) ;
assertFalse ( state . updateReplicaState ( observerId , observerFetchTimeMs , new LogOffsetMetadata ( epochStartOffset + 1 ) ) ) ;
time . sleep ( 500 ) ;
DescribeQuorumResponseData . PartitionData partitionData = state . describeQuorum ( time . milliseconds ( ) ) ;
assertEquals ( epochStartOffset + 1 , partitionData . highWatermark ( ) ) ;
assertEquals ( localId , partitionData . leaderId ( ) ) ;
assertEquals ( epoch , partitionData . leaderEpoch ( ) ) ;
assertEquals ( 1 , partitionData . currentVoters ( ) . size ( ) ) ;
assertEquals ( localId , partitionData . currentVoters ( ) . get ( 0 ) . replicaId ( ) ) ;
List < DescribeQuorumResponseData . ReplicaState > observerStates = partitionData . observers ( ) ;
assertEquals ( 1 , observerStates . size ( ) ) ;
DescribeQuorumResponseData . ReplicaState observerState = observerStates . get ( 0 ) ;
assertEquals ( new DescribeQuorumResponseData . ReplicaState ( )
. setReplicaId ( observerId )
. setLogEndOffset ( epochStartOffset + 1 )
. setLastFetchTimestamp ( observerFetchTimeMs )
. setLastCaughtUpTimestamp ( observerFetchTimeMs ) ,
observerState ) ;
}
@Test
public void testNoOpForNegativeRemoteNodeId ( ) {
int observerId = - 1 ;
MockTime time = new MockTime ( ) ;
int replicaId = - 1 ;
long epochStartOffset = 10L ;
LeaderState < ? > state = newLeaderState ( mkSet ( localId ) , epochStartOffset ) ;
assertFalse ( state . updateReplicaState ( observerId , 0 , new LogOffsetMetadata ( epochStartOffset ) , epochStartOffset + 10 ) ) ;
assertFalse ( state . updateReplicaState ( replica Id, 0 , new LogOffsetMetadata ( epochStartOffset ) ) ) ;
assertEquals ( emptyList ( ) , state . quorumResponseObserverStates ( 10 ) ) ;
DescribeQuorumResponseData . PartitionData partitionData = state . describeQuorum ( time . milliseconds ( ) ) ;
List < DescribeQuorumResponseData . ReplicaState > observerStates = partitionData . observers ( ) ;
assertEquals ( Collections . emptyList ( ) , observerStates ) ;
}
@Test
@ -364,14 +468,17 @@ public class LeaderStateTest {
@@ -364,14 +468,17 @@ public class LeaderStateTest {
long epochStartOffset = 10L ;
LeaderState < ? > state = newLeaderState ( mkSet ( localId ) , epochStartOffset ) ;
state . updateReplicaState ( observerId , time . milliseconds ( ) , new LogOffsetMetadata ( epochStartOffset ) , epochStartOffset + 10 ) ;
assertEquals ( singleton ( observerId ) ,
state . quorumResponseObserverStates ( time . milliseconds ( ) )
. stream ( ) . map ( o - > o . replicaId ( ) )
. collect ( Collectors . toSet ( ) ) ) ;
state . updateReplicaState ( observerId , time . milliseconds ( ) , new LogOffsetMetadata ( epochStartOffset ) ) ;
DescribeQuorumResponseData . PartitionData partitionData = state . describeQuorum ( time . milliseconds ( ) ) ;
List < DescribeQuorumResponseData . ReplicaState > observerStates = partitionData . observers ( ) ;
assertEquals ( 1 , observerStates . size ( ) ) ;
DescribeQuorumResponseData . ReplicaState observerState = observerStates . get ( 0 ) ;
assertEquals ( observerId , observerState . replicaId ( ) ) ;
time . sleep ( LeaderState . OBSERVER_SESSION_TIMEOUT_MS ) ;
assertEquals ( emptyList ( ) , state . quorumResponseObserverStates ( time . milliseconds ( ) ) ) ;
partitionData = state . describeQuorum ( time . milliseconds ( ) ) ;
assertEquals ( Collections . emptyList ( ) , partitionData . observers ( ) ) ;
}
@ParameterizedTest
@ -405,4 +512,34 @@ public class LeaderStateTest {
@@ -405,4 +512,34 @@ public class LeaderStateTest {
}
}
private DescribeQuorumResponseData . ReplicaState describeVoterState (
LeaderState state ,
int voterId ,
long currentTimeMs
) {
DescribeQuorumResponseData . PartitionData partitionData = state . describeQuorum ( currentTimeMs ) ;
return findReplicaOrFail ( voterId , partitionData . currentVoters ( ) ) ;
}
private DescribeQuorumResponseData . ReplicaState describeObserverState (
LeaderState state ,
int observerId ,
long currentTimeMs
) {
DescribeQuorumResponseData . PartitionData partitionData = state . describeQuorum ( currentTimeMs ) ;
return findReplicaOrFail ( observerId , partitionData . observers ( ) ) ;
}
private DescribeQuorumResponseData . ReplicaState findReplicaOrFail (
int replicaId ,
List < DescribeQuorumResponseData . ReplicaState > replicas
) {
return replicas . stream ( )
. filter ( observer - > observer . replicaId ( ) = = replicaId )
. findFirst ( )
. orElseThrow ( ( ) - > new AssertionError (
"Failed to find expected replica state for replica " + replicaId
) ) ;
}
}