Context
A Raft-based alternative for replication is currently being developed. This issue aims to present the proposed architecture and implementation plan.
Architecture
Consensus layer (under pkg/raftstore/consensus)
This is the Raft engine which wraps the etcd Raft library and includes transport and Raft log persistence.
Consensus runs a handleReady() goroutine which processes the node.Ready() updates provided by etcd Raft. On every iteration, it persists entries and snapshots to the WAL, sends the required messages (Heartbeats, Votes etc. as decided by the underlying etcd Raft) to peers, applies the committed entries to the state machine then calls node.Advance() to communicate to the library that it is done processing the update. The general workflow of the goroutine is as follows:

Storage handles persisting hard state, entries and snapshots by wrapping etcd's WAL implementation.
Generic Store (pkg/raftstore/store.go)
The generic raftstore implements the store.Store[R] interface and connects the consensus layer with the specific DBs layers (currently only scd). The raftstore Transact() implementation builds a proposal from the serialized op data attached to the context, registers the action as a pending transaction then calls consensus.Propose and blocks. When the proposal comes back as a committed entry, the pending transaction is executed. Since the other nodes don't have access to the action, they use per-operation appliers to apply the entry. This means that the current design implies duplicated logic between the API handlers and these appliers.
Scd in-memory store (under pkg/scd/store/raftstore)
This is the scd specific implementation of the Raftstore it contains a repo struct which is an in-memory implementation of a scd store:
type Repo struct {
mu sync.RWMutex
Data
}
type Data struct {
Subscriptions map[dssmodels.ID]*scdmodels.Subscription `json:"subscriptions"`
OperationalIntents map[dssmodels.ID]*scdmodels.OperationalIntent `json:"operational_intents"`
Constraints map[dssmodels.ID]*scdmodels.Constraint `json:"constraints"`
UssAvailability map[dssmodels.Manager]*scdmodels.UssAvailabilityStatus `json:"uss_availability"`
}
Implementation plan
#1451: Adds the raftstore flag and store stubs
#1460: Adds Raft related dependencies
Future PRs:
- Implement transport in the empty consensus struct. DSS nodes should be able to connect to each other at this point.
- Add WAL and snapshot persistence.
- Add the core consensus logic with trivial kvstore to test logic.
- Implement the generic raftstore (commit reading and
Transact)
- Implement in-memory raftstore with appliers => Raftstore fully functional with SCD.
- Encrypted communication.
Future changes:
- Add RID and aux.
- Configuration changes.
- Avoid the applier code duplication.
- Optimizations: batching Raft proposals + tuning snapshot catchup entries parameter.
Context
A Raft-based alternative for replication is currently being developed. This issue aims to present the proposed architecture and implementation plan.
Architecture
Consensus layer (under
pkg/raftstore/consensus)This is the Raft engine which wraps the etcd Raft library and includes transport and Raft log persistence.

Consensusruns ahandleReady()goroutine which processes thenode.Ready()updates provided by etcd Raft. On every iteration, it persists entries and snapshots to the WAL, sends the required messages (Heartbeats, Votes etc. as decided by the underlying etcd Raft) to peers, applies the committed entries to the state machine then callsnode.Advance()to communicate to the library that it is done processing the update. The general workflow of the goroutine is as follows:Storagehandles persisting hard state, entries and snapshots by wrapping etcd's WAL implementation.Generic Store (
pkg/raftstore/store.go)The generic raftstore implements the
store.Store[R]interface and connects the consensus layer with the specific DBs layers (currently only scd). The raftstore Transact() implementation builds a proposal from the serialized op data attached to the context, registers the action as a pending transaction then callsconsensus.Proposeand blocks. When the proposal comes back as a committed entry, the pending transaction is executed. Since the other nodes don't have access to the action, they use per-operation appliers to apply the entry. This means that the current design implies duplicated logic between the API handlers and these appliers.Scd in-memory store (under
pkg/scd/store/raftstore)This is the scd specific implementation of the Raftstore it contains a repo struct which is an in-memory implementation of a scd store:
Implementation plan
#1451: Adds the raftstore flag and store stubs
#1460: Adds Raft related dependencies
Future PRs:
Transact)Future changes: