Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
28 changes: 25 additions & 3 deletions raft/src/main/scala/zio/raft/Raft.scala
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@ class Raft[S, A <: Command](
peers: Peers,
private[raft] val raftState: Ref[State[S]],
commandsQueue: Queue[StreamItem[A, S]],
stateNotificationsQueue: Queue[StateNotification],
stable: Stable,
private[raft] val logStore: LogStore[A],
snapshotStore: SnapshotStore,
Expand All @@ -39,6 +40,9 @@ class Raft[S, A <: Command](
val batchSize = 100
val numberOfServers = peers.size + 1

def stateNotifications: ZStream[Any, Nothing, StateNotification] =
ZStream.fromQueue(stateNotificationsQueue)

private def stepDown(newTerm: Term, leaderId: Option[MemberId]) =
for
_ <- stable.newTerm(newTerm, None)
Expand All @@ -55,6 +59,15 @@ class Raft[S, A <: Command](
_ <- ZIO.logDebug(
s"memberId=${this.memberId} Following $leaderId $currentTerm"
)

_ <- (currentState, leaderId) match
case (_: Leader[S], _) =>
stateNotificationsQueue.offer(StateNotification.SteppedDown(leaderId))
case (f: State.Follower[S], Some(leaderId)) if f.leaderId != Some(leaderId) =>
stateNotificationsQueue.offer(StateNotification.LeaderChanged(leaderId))
case (_: State.Candidate[S], Some(leaderId)) =>
stateNotificationsQueue.offer(StateNotification.LeaderChanged(leaderId))
case _ => ZIO.unit
yield newTerm

private def convertToFollower(leaderId: MemberId) =
Expand All @@ -66,9 +79,14 @@ class Raft[S, A <: Command](

_ <- s match
case s: State.Follower[S] if s.leaderId != Some(leaderId) =>
ZIO.logDebug(
s"memberId=${this.memberId} Following $leaderId $currentTerm"
)
for
_ <- ZIO.logDebug(
s"memberId=${this.memberId} Following $leaderId $currentTerm"
)

// Leader changed, so let's notify
_ <- stateNotificationsQueue.offer(StateNotification.LeaderChanged(leaderId))
yield ()
case s: State.Follower[S] => ZIO.unit
case s =>
ZIO.logDebug(
Expand Down Expand Up @@ -514,6 +532,8 @@ class Raft[S, A <: Command](
PendingCommands.empty
)
)

_ <- stateNotificationsQueue.offer(StateNotification.SteppedUp)
yield ()

for
Expand Down Expand Up @@ -1046,12 +1066,14 @@ object Raft:
State.Follower[S](previousIndex, previousIndex, electionTimeout, None)
)
commandsQueue <- Queue.unbounded[StreamItem[A, S]] // TODO: should this be bounded for back-pressure?
stateNotificationsQueue <- Queue.unbounded[StateNotification]

raft = new Raft[S, A](
memberId,
peers,
state,
commandsQueue,
stateNotificationsQueue,
stable,
logStore,
snapshotStore,
Expand Down
6 changes: 6 additions & 0 deletions raft/src/main/scala/zio/raft/Types.scala
Original file line number Diff line number Diff line change
Expand Up @@ -126,3 +126,9 @@ object InstallSnapshotResult:
done: Boolean
) extends InstallSnapshotResult[A]
case class Failure[A <: Command](from: MemberId, term: Term, index: Index) extends InstallSnapshotResult[A]

sealed trait StateNotification
object StateNotification:
case object SteppedUp extends StateNotification
case class SteppedDown(leaderId: Option[MemberId]) extends StateNotification
case class LeaderChanged(leaderId: MemberId) extends StateNotification