Integrate etcd/raft life cycle to Gitaly's WAL log management
Overview
This MR marks the initial integration of the Raft consensus algorithm into Gitaly's Transaction Manager. Given its size, this PR serves as a reference for subsequent, smaller PRs where individual feature implementations are extracted for detailed reviews and integration.
What happens after this MR is merged?
If Raft is enabled, every partition on a node is converted to a single-node cluster. It means:
- When the Raft group is bootstrapped for the first time, the node performs a mandatory election.
- After a restart, the Raft group is resumed using persisted info.
- All log entries now flow through the Raft Manager.
- Occasionally, Raft state machine inserts internal log entries, such as ConfChange or empty log entries. Those entries should not interrupt the functionality of TransactionManager.
- All the functionalities should work well.
As the node is the only node of the cluster, it is always the primary. There is no "real" election, replication or failover. It's highly likely that the flow will break in some non-conventional scenarios when more members join the cluster. The test suite doesn't cover multi-node scenarios just yet. It also doesn't work with an existing partition having more than one log entries (empty partitions are fine).
High-level design
The Raft algorithm will help Gitaly manage distributed transactions more reliably by ensuring consensus across nodes, which is crucial for maintaining consistency and facilitating leader election and membership changes in clusters. This integration leverages the etcd/raft
library for its robust implementation and community validation. The library's usage is described in detail in this section.
The Raft Manager implements storage.LogManager interface. By default, Transaction Manager uses log.Manager. All log entries are appended to the file-system WAL. As soon as an entry is persisted, it's ready to be applied by Transaction Manager. When Raft is enabled, the whole flow is handled by raftmgr.Manager. Network transmission and quorum acknowledgment are now added to the flow. To reduce tight coupling and complexity, the flow is encapsulated inside raftmgr.Manager. Both log.Manager and raftmgr.Manager share the same interface.
The responsibilities of three crucial components are:
- log.Manager handles local log management
- raftmgr.Manager handles distributed log management
- partition.TransactionManager manages transactions, concurrency, snapshots, conflicts, etc.
So, the interaction flow between Raft and Transaction manager looks like following:
┌──────────┐
│Local Disk│
└────▲─────┘
│
log.Manager
▲
TransactionManager New Transaction │ Without Raft
│ ▼ │
└─►Initialize─►txn.Commit()─►Verify─► AppendLogEntry()──►Apply
│ │
│ │ With Raft
▼ ▼
┌─►Initialize─────...──────────────────Propose
│ │
raftmgr.Manager etcd/raft state machine
┌───────┴────────┐
▼ ▼
raftmgr.Storage raftmgr.Transport
│ │
▼ │
log.Manager Network
│ │
┌─────▼────┐ ┌─────▼────┐
│Local Disk│ │Raft Group│
└──────────┘ └──────────┘
What has been done
-
Core Raft manager implementation. -
Persistent storage -
Interaction with the Transaction manager. -
Testing infrastructure for Raft. -
Transaction manager tests pass when Raft is enabled -
The whole test suite passes when Raft is enabled
header | header |
---|---|
![]() |
![]() |
How to verify Raft on the local environment?
- Run the test suite with
GITALY_TEST_WAL=true GITALY_TEST_RAFT=YesPlease
- Start Gitaly with the following config:
[transactions]
enabled = true
[raft]
enabled = true
cluster_id = "9e099d78-b599-4e0d-9b92-37d5dcc4e569"
References and future works
- Previous attempt: Draft: Initial Raft Integration for Gitaly's Tr... (!7385 - closed)
- Prior work leading to this MR:
- Implement Raft storage component (!7584 - merged)
- Embed raftpb to Gitaly for direct use (!7561 - merged)
- storage: Refactor some storage components to pr... (!7556 - merged)
- storage: Generalize position tracking in LogMan... (!7541 - merged)
- storage: Add CompareAndAppendLogEntry to log ma... (!7540 - merged)
- storage: Move log entry pruning to a concurrent... (!7521 - merged)
- Extract log management out of TransactionManager (!7424 - merged)
- Networking is out of the picture. Although the log data is packed before sending, the messages are swallowed by a no-op transportation. It will be implemented here: raft: Implement networking handlers for Raft (#6401 - closed)
- The only peer is the current node. The node ID is set to 1 statically. The node ID will be automatically assigned to a replica based on the ConfChange sequence. The transportation layer will capture the initial message and start replicas proactively: raft: Boot partition Raft group in replicas pro... (#6304)
- The proposal doesn't wait until a log entry is applied. Gitaly unlocks the caller as soon as a log entry is committed. This approach aligns with the current flow of Transaction Manager.
- Autocompacting is ignored. raft: Add auto compaction support (#6463)