Distributed databases face an interesting challenge: how do you ensure that multiple database nodes scattered across different machines, data centers, or even continents agree on the order and outcome of database transactions? This is where consensus algorithms come into play.
Multi-Paxos is one of the most influential consensus protocols and is used in several modern distributed databases, including Google Spanner and MySQL Group Replication.
In this write-up, we explore Multi-Paxos from the ground up, explain why it is particularly efficient for database transactions, and examine how real-world systems implement it to achieve strong consistency with high performance.
Prerequisites
To get the most out of this article, you should be familiar with:
- Why consistency matters (all nodes seeing the same data)
- The challenges of network failures and message delays
If you are new to these concepts, think of distributed systems like a group of people trying to work together while communicating only through unreliable phone calls that might arrive late, out of order, or not at all.
Consensus in Distributed Databases
When you run a database on a single machine, transaction ordering is straightforward. The database engine processes requests sequentially, and there is a clear before-and-after relationship between operations. But what happens when the database spans multiple servers?
Consider a banking application where a user transfers $100 from Account A to Account B. In a distributed database, this transaction may be handled by several nodes, each with its own copy of the data. For the system to be correct, every node must agree on whether the transfer happened and in what order relative to other transactions.
If the nodes do not agree, the system can end up in an inconsistent state. One node might believe the transfer succeeded and update the balances, while another might think it failed and keep the old values.
This problem is formally known as distributed consensus. The consensus problem requires nodes in a distributed system to agree on a single decision despite failures, network partitions, and message delays.
With consensus, all nodes reach the same outcome:
Node 1: Transfer $100 from A to B -> Committed at time T1
Node 2: Transfer $100 from A to B -> Committed at time T1
Node 3: Transfer $100 from A to B -> Committed at time T1
Without consensus, each node may see a different reality:
Replica 1: Transfer committed, A = $900, B = $1100
Replica 2: Transfer aborted, A = $1000, B = $1000
Replica 3: Transfer pending (never received a final decision)
In this situation, the database no longer has a single source of truth, which is exactly what consensus protocols are designed to prevent.
Key Terms and Concepts
Before diving into Paxos, here are some essential terms:
- Quorum: A majority of nodes. In a 5-node cluster, a quorum is 3 nodes. Quorums ensure that any two groups of nodes must overlap, preventing conflicting decisions.
- Proposal number: A unique, increasing number that orders proposals. Think of it like a timestamp that ensures newer proposals take precedence over older ones.
- Ballot: In Multi-Paxos, the ballot is the leadership epoch identifier. It changes only when leadership changes, not for every proposal.
- Leader: The node responsible for coordinating proposals.
Intuition Behind Paxos
Before diving into the technical details, here is a real-world analogy. Imagine a group of friends trying to decide on a restaurant for dinner over text messages, but messages can arrive late or get lost.
- Proposers are friends who suggest restaurants
- Acceptors are friends who vote on suggestions
- Learners are friends who learn the final decision
To prevent endless disagreement, friends promise not to accept older suggestions once they have seen a newer one.
If someone proposes “Italian food (suggestion #5)”, and then someone else tries to propose “Chinese food (suggestion #3)”, everyone ignores the Chinese food suggestion because they have already seen a higher-numbered proposal.
This is exactly what Paxos does.
Paxos
Before diving into Multi-Paxos, let’s talk about the basic Paxos algorithm. Paxos was introduced by Leslie Lamport in his 1989 paper. It solves the problem of getting a group of nodes to agree on a single value in the presence of failures.
Paxos defines three roles that participants can take:
- Proposers: propose values for the group to agree upon
- Acceptors: vote on and remember what they have accepted
- Learners: learn about the chosen value after consensus is reached
In practice, a single node often plays all three roles simultaneously.
The Two-Phase Protocol
Basic Paxos operates in two phases to reach agreement. Think of Phase 1 as “reserving the right to propose” and Phase 2 as “actually proposing a value.”
Phase 1: Prepare Phase (Reserve)
A proposer selects a unique proposal number n and sends a prepare request to the acceptors. The proposal number must be higher than any number the proposer has previously used.
Why do proposal numbers need to increase? Because higher numbers represent more recent attempts. If older proposals could override newer ones, the system could get stuck in an endless loop of conflicting decisions. By requiring monotonically increasing proposal numbers, we ensure that the system makes forward progress.
# Proposer
def prepare(self, proposal_number):
for acceptor in self.acceptors:
send(acceptor, Prepare(n=proposal_number))
When an acceptor receives a prepare request with proposal number n, it checks if n is greater than or equal to any proposal number it has previously promised to.
If so, the acceptor makes a promise not to accept any proposals with numbers less than n. By promising to ignore lower-numbered proposals, acceptors ensure that once they have seen proposal #10, they will not accept proposal #5, even if it arrives late.
The acceptor also returns any value it has previously accepted, along with that value’s proposal number (if any). This is critical for maintaining consistency. If someone has already partially agreed on a value, new proposers must respect that decision.
# Acceptor handles prepare request
def on_prepare(self, request):
if request.n >= self.promised_n:
self.promised_n = request.n
return Promise(
n=request.n,
accepted_n=self.accepted_n,
accepted_value=self.accepted_value
)
else:
return Reject()
Phase 2: Accept Phase (Commit)
If the proposer receives promises from a majority of acceptors, it proceeds to propose a value. This is where the protocol ensures consistency: if any acceptor has already accepted a value, the proposer must propose that value rather than its own.
Specifically, the proposer must select the value associated with the highest accepted proposal number among all the promise responses. If no acceptor has accepted a value yet, the proposer is free to propose its own value.
Why this rule? Because if a value was accepted by any acceptor in the past, it might have already been chosen (if enough other acceptors also accepted it). By proposing the value with the highest ballot number, the new proposer ensures it does not violate any previous decisions.
# Proposer sends accept request
def propose(self, proposal_number, responses):
value = self.own_value
highest_n = -1
for response in responses:
if response.accepted_n is not None and response.accepted_n > highest_n:
highest_n = response.accepted_n
value = response.accepted_value
for acceptor in self.acceptors:
send(acceptor, Accept(n=highest_n, value=value))
An acceptor accepts the proposal if it has not promised to a higher-numbered proposal since responding to the prepare request. If it accepts, it records the proposal number and value.
# Acceptor handles accept request
def on_accept(self, request):
if request.n >= self.promised_n:
self.promised_n = request.n
self.accepted_n = request.n
self.accepted_value = request.value
return Accepted()
else:
return Reject()
Once a majority of acceptors accept a value, that value is chosen.
When Things Go Wrong
What happens if the proposer fails to receive acceptances from a majority? This occurs when another proposer runs in parallel. For example, proposer P1 sends Prepare(n=10) and gets promises from a majority. But before P1 sends Accept(10, v), proposer P2 sends Prepare(n=11) and gets promises from a majority. Now when P1 tries to send Accept(10, v), the acceptors reject it because they have promised to ignore proposals numbered less than 11.
In this case, P1 must start over with a new, higher proposal number (say, 12).
However, if the proposer learns during the prepare phase that one or more acceptors have already accepted a value, it does not restart the protocol. Instead, it simply proposes that value in the current accept phase (as seen in the above pseudocode), abandoning its own original value for this round. This ensures that partially-agreed-upon values are eventually finalized.
Paxos to Multi-Paxos
Basic Paxos decides on a single value. But databases need to agree on a sequence of values (a log of transactions). Running independent instances of basic Paxos for each transaction would work, but it would be inefficient.
The key idea of Multi-Paxos is that if the same proposer (called the leader) handles all proposals, the prepare phase can be run once and then reused for many subsequent proposals.
In Multi-Paxos, we have a concept of a ballot, which is the leadership epoch identifier that orders competing leaders. Let me explain…
Unlike basic Paxos, where proposal numbers increment for every proposal, in Multi-Paxos the ballot number changes only when leadership changes. During stable leadership, the same ballot is reused for many log entries.
So, we can think of Multi-Paxos as a way to ensure that every node eventually has the same log, in the same order.
Stable Leader
In basic Paxos, every proposal requires two phases: prepare and accept. With a stable leader, we skip the prepare phase for subsequent proposals.
Basic Paxos for 3 values:
Value 1: Prepare -> Accept (4 message delays)
Value 2: Prepare -> Accept (4 message delays)
Value 3: Prepare -> Accept (4 message delays)
Total: 12 message delays
Multi-Paxos with stable leader:
Value 1: Prepare -> Accept (4 message delays, establishes leadership)
Value 2: Accept only (2 message delays)
Value 3: Accept only (2 message delays)
Total: 8 message delays
This cuts the message delay nearly in half for subsequent operations, which is significant for latency-sensitive database applications.
Replicated Log
Multi-Paxos maintains a replicated log where each entry is decided by a separate Paxos instance. The leader assigns monotonic log positions (also called slots or indices) to incoming requests and runs the accept phase to get each entry chosen.
Leader's Log:
Index 0: BEGIN TRANSACTION; INSERT INTO users (id, name) VALUES (1, 'Alice')
Index 1: COMMIT
Index 2: BEGIN TRANSACTION; UPDATE accounts SET balance = balance - 100
Index 3: UPDATE accounts SET balance = balance + 100
Index 4: COMMIT
All replicas apply the log entries in the same order, ensuring they end up in identical states. This is the foundation of state machine replication: if all replicas start from the same state and apply the same operations in the same order, they will end up in the same final state.
class MultiPaxosLeader:
def __init__(self):
self.next_index = 0
self.ballot_number = 0
def propose(self, value):
# Assign value to next log position
index = self.next_index
self.next_index += 1
# Run accept phase (prepare already done)
for acceptor in self.acceptors:
send(acceptor, Accept(
ballot=self.ballot_number,
index=index,
value=value
))
Multi-Paxos in Database Transactions
It might not be clear from the previous explanation, so let us trace through how a database transaction flows through a Multi-Paxos based system.
Transaction Initiation
A client sends a transaction request to one of the database nodes. This node might be the leader or might need to forward the request to the leader.
Client -> Node 2: BEGIN TRANSACTION;
UPDATE inventory SET quantity = quantity - 1 WHERE item_id = 42;
COMMIT;
Node 2 (not leader) -> Node 1 (leader): Forward transaction
Leader Processing
The leader assigns the transaction to a log position and broadcasts an accept request to the acceptors.
Leader (Node 1):
- Receives transaction
- Assigns log index 47
- Broadcasts Accept(ballot=5, index=47, value=<transaction>)
Acceptor Responses
Each acceptor checks the ballot number against its promises. If valid, it accepts the value, persists it to disk, and responds.
class Acceptor:
def on_accept(self, request):
if request.ballot >= self.promised_ballot:
# Persist to disk before responding
self.log[request.index] = {
'ballot': request.ballot,
'value': request.value
}
self.persist()
return Accepted(ballot=request.ballot, index=request.index)
else:
return Rejected(current_ballot=self.promised_ballot)
Commit and Execution
Once the leader receives acceptances from a majority, the transaction is considered committed. The leader notifies learners, and all replicas execute the transaction.
Leader:
- Receives Accept from Node 2: OK
- Receives Accept from Node 3: OK
- Majority achieved (2 out of 3)
- Transaction at index 47 is committed
- Notify all replicas to execute
Applying Locally
Each replica applies committed transactions in log order to its local database state:
class Replica:
def apply_committed(self, index, transaction):
# Wait for all prior entries to be applied
while self.last_applied < index - 1:
wait()
# Execute transaction against local database
self.database.execute(transaction)
self.last_applied = index
Efficiency of Multi-Paxos
Multi-Paxos achieves several efficiency gains that make it well-suited for database workloads.
Reduced Message Complexity
With a stable leader, Multi-Paxos reduces the messages per operation:
- Basic Paxos: 4 message delays (prepare + accept)
- Multi-Paxos steady state: 2 message delays (accept only)
For a 5-node cluster processing 10,000 transactions per second, this reduction saves millions of messages per hour.
Batching
Leaders batch multiple transactions into a single log entry, reducing the consensus overhead across many operations.
class BatchingLeader:
def __init__(self, batch_size=100, batch_timeout_ms=10):
self.pending = []
self.batch_size = batch_size
self.batch_timeout = batch_timeout_ms
def submit(self, transaction):
self.pending.append(transaction)
if len(self.pending) >= self.batch_size:
self.flush_batch()
def flush_batch(self):
batch = self.pending
self.pending = []
# Single consensus round for entire batch
self.propose(Batch(transactions=batch))
Pipelining
Pipelining allows multiple consensus instances to be in flight simultaneously, hiding network latency:
Without pipelining:
Time 0: Propose index 0
Time RTT: Index 0 committed
Time RTT: Propose index 1
Time 2RTT: Index 1 committed
With pipelining (window size 3):
Time 0: Propose index 0, 1, 2
Time RTT: Index 0, 1, 2 committed
Time RTT: Propose index 3, 4, 5
Read Optimization
Multi-Paxos enables efficient read operations. Since the leader always has the most recent committed state, reads can be served directly from the leader without running consensus.
class LeaderWithReads:
def handle_read(self, query):
# Verify we are still leader
if not self.is_leader():
raise NotLeaderError()
# Serve read directly, no consensus needed
return self.database.execute(query)
For read-heavy workloads, this optimization dramatically improves throughput since reads avoid the consensus protocol entirely.
Leader Election in Multi-Paxos
A critical aspect of Multi-Paxos is handling leader changes. The leader might fail, become partitioned from the rest of the cluster, or need to be changed for operational reasons (though deliberate changes for load balancing are rare in practice; most systems maintain a stable leader and only change on failure).
Detecting Leader Failure
Nodes detect leader failure through timeouts. If a follower does not hear from the leader within a certain time window, it suspects the leader has failed and can attempt to become the new leader.
Election
When a node suspects the leader has failed, it attempts to become the new leader by running the prepare phase with a higher ballot number.
class Node:
def suspect_leader_failure(self):
# Generate higher ballot number
new_ballot = self.generate_higher_ballot()
# Run prepare phase
promises = []
for acceptor in self.acceptors:
response = send(acceptor, Prepare(ballot=new_ballot))
if response.is_promise:
promises.append(response)
if len(promises) > len(self.acceptors) / 2:
# Become new leader
self.ballot = new_ballot
self.is_leader = True
# Recover any uncommitted entries
self.recover_from_promises(promises)
Log Recovery
When a new leader is elected, it must ensure all previously committed entries are preserved. The promises from acceptors include information about accepted values, allowing the new leader to recover the log state.
def recover_from_promises(self, promises):
# For each log position, find the highest-ballot accepted value
for index in range(self.first_uncommitted, self.highest_seen):
highest_ballot = -1
value = None
for promise in promises:
if index in promise.accepted:
if promise.accepted[index].ballot > highest_ballot:
highest_ballot = promise.accepted[index].ballot
value = promise.accepted[index].value
if value is not None:
# Re-propose this value to ensure it is committed
self.propose_at(index, value)
Leader Leases
To prevent multiple leaders from operating simultaneously (which would reduce efficiency but not violate safety), many systems use leader leases. A leader holds a time-bounded lease that gives it the exclusive right to propose.
class LeaderWithLease:
def __init__(self):
self.lease_duration = timedelta(seconds=10)
self.lease_expiry = None
def renew_lease(self):
# Get lease extension from majority
votes = self.request_lease_extension()
if len(votes) > len(self.acceptors) / 2:
self.lease_expiry = datetime.now() + self.lease_duration
def can_propose(self):
return self.lease_expiry and datetime.now() < self.lease_expiry
Leader Failure Scenario
Let’s walk through what happens when a leader fails mid-transaction:
Initial State:
- Node 1 is leader (ballot=5)
- Transaction T1 committed at index 10
- Transaction T2 being processed at index 11
Time T0: Leader Node 1 crashes while processing T2
- Node 1 sent Accept(ballot=5, index=11, value=T2) to Node 2
- Node 1 crashed before sending to Node 3 or Node 4
- Node 2 accepted and persisted T2
- Nodes 3 and 4 never saw T2
Time T1: Node 3 detects leader timeout (no heartbeat from Node 1)
- Node 3 initiates election with ballot=6
Time T2: Node 3 sends Prepare(ballot=6) to all acceptors
- Node 2 responds: Promise(ballot=6, accepted={(11, 5, T2)})
- Node 3 responds: Promise(ballot=6, accepted={})
- Node 4 responds: Promise(ballot=6, accepted={})
Time T3: Node 3 becomes leader and recovers log
- Sees that Node 2 accepted T2 at index 11 with ballot 5
- Must re-propose T2 to ensure it is committed
- Sends Accept(ballot=6, index=11, value=T2) to all acceptors
Time T4: All acceptors accept
- T2 is now committed at index 11
- System continues normal operation with Node 3 as leader
The key insight: even though the old leader crashed, the transaction it started was not lost. The new leader discovered it during recovery and ensured it was completed.
Multi-Paxos in Databases
Google Spanner
Spanner, Google’s globally distributed database, uses Paxos at its core. Each tablet (a contiguous range of rows) is replicated across multiple datacenters using a Paxos group.
Key characteristics:
- Each Paxos group has 5 replicas spread across datacenters
- One replica acts as the Paxos leader and handles all writes
- TrueTime API provides globally synchronized timestamps
- Two-phase commit coordinates transactions across Paxos groups
Zone A Zone B Zone C
+--------+ +--------+ +--------+
| Tablet |<----->| Tablet |<----->| Tablet |
| (Lead) | | (Repl) | | (Repl) |
+--------+ +--------+ +--------+
| | |
+----------------+----------------+
Paxos Group
When a transaction spans multiple Paxos groups, Spanner runs two-phase commit (2PC) on top of Paxos. The 2PC coordinator and participants are themselves Paxos groups, making the overall protocol fault-tolerant.
Handling Hot Spots
When certain keys receive disproportionate traffic, Multi-Paxos can become a bottleneck since all writes must go through the leader.
Mitigation strategies:
- Range partitioning: Split hot ranges into smaller Paxos groups
- Read replicas: Serve reads from followers with bounded staleness
- Client-side caching: Cache immutable or slowly-changing data
class HotSpotMitigation:
def __init__(self, split_threshold=10000): # ops/sec
self.split_threshold = split_threshold
self.metrics = {}
def record_access(self, key_range):
self.metrics[key_range] = self.metrics.get(key_range, 0) + 1
def should_split(self, key_range):
return self.metrics.get(key_range, 0) > self.split_threshold
def split_range(self, key_range):
# Create two new Paxos groups for the split ranges
mid = (key_range.start + key_range.end) // 2
return [
Range(key_range.start, mid),
Range(mid, key_range.end)
]
Comparison with Alternative Approaches
When to Use Multi-Paxos
Multi-Paxos is well-suited for:
- Transactions requiring strong consistency
- Systems with relatively low write contention
- Applications needing mature, battle-tested technology
Consider alternatives when:
- Write throughput is extremely high
- Clients are globally distributed
- Eventual consistency is acceptable
Common Misconceptions
- “Why can’t we just use timestamps to order events?”
Timestamps from different machines are not reliable because clocks can drift. Even with clock synchronization protocols like NTP, there is always some uncertainty. Multi-Paxos does not rely on synchronized clocks, making it more robust.
- “Why do we need a majority instead of unanimous agreement?”
Requiring unanimous agreement means a single node failure would halt the entire system. By requiring only a majority, the system can continue operating even if some nodes fail. The quorum property ensures that any two majorities overlap, preventing conflicting decisions.
- “Does Multi-Paxos guarantee that transactions will complete quickly?”
Multi-Paxos guarantees safety (all nodes agree on the same values) but not liveness (that operations will complete in bounded time). In practice, with reasonable timeouts and failure detection, Multi-Paxos provides good performance, but under extreme conditions (like network partitions), progress may stall until the partition heals.
Implementation Considerations
Durable Storage
Every Paxos acceptor must persist its promises and accepted values before responding. This is crucial for correctness: without durable storage, a crashed and restarted node could violate promises it made before crashing, potentially leading to conflicting decisions.
Membership Changes
Adding or removing nodes from a Paxos group requires care to maintain safety. Most systems use configuration change protocols that treat membership changes as special log entries that go through consensus.
Log Compaction
As the log grows, older entries must be compacted to manage storage. Systems typically take periodic snapshots of the database state and then truncate the log. New nodes or nodes recovering from failures load the snapshot and then replay only recent log entries.
Fault Tolerance and Safety Guarantees
Safety Properties
Multi-Paxos guarantees two fundamental safety properties:
- Agreement: No two replicas will ever commit different values for the same log index. This holds regardless of message delays, reordering, or node failures.
- Validity: Only values that were actually proposed can be committed. The system cannot fabricate values out of thin air.
Handling Node Failures
Multi-Paxos tolerates f failures in a cluster of 2f+1 nodes. This means a 5-node cluster can survive 2 simultaneous failures while continuing to process transactions.
5-node cluster failure tolerance:
Healthy cluster: [N1] [N2] [N3] [N4] [N5] -> Quorum = 3
1 node fails: [N1] [N2] [N3] [N4] [X ] -> Still operational
2 nodes fail: [N1] [N2] [N3] [X ] [X ] -> Still operational (barely)
3 nodes fail: [N1] [N2] [X ] [X ] [X ] -> Unavailable (no quorum)
When a node fails:
- If it was a follower, the cluster continues normally
- If it was the leader, a new leader is elected
- Once the node recovers, it catches up from the current leader
class NodeRecovery:
def recover_after_crash(self):
# Load persisted state
self.promised_ballot = self.storage.load_promised_ballot()
self.accepted = self.storage.load_accepted_values()
self.committed_index = self.storage.load_committed_index()
# Find current leader
leader = self.discover_leader()
# Catch up on missed entries
missing_entries = leader.get_entries_since(self.committed_index)
for entry in missing_entries:
self.apply(entry)
# Resume normal operation
self.start_accepting_requests()
Network Partition Handling
Network partitions can split a cluster into isolated groups. Multi-Paxos handles this safely:
Partition scenario:
[N1] [N2] | network | [N3] [N4] [N5]
2 nodes 3 nodes
Left partition: Cannot form quorum, stops accepting writes
Right partition: Can form quorum, continues operating
After partition heals:
N1 and N2 discover they are behind
They catch up from N3, N4, or N5
Cluster resumes unified operation
The quorum requirement ensures that at most one partition can make progress, preventing split-brain scenarios where two groups of nodes make conflicting decisions.
Integration with Database Transactions
Multi-Paxos provides ordered log replication, but database transactions require additional mechanisms for full ACID compliance.
Paxos with Write-Ahead Logging
Databases typically maintain their own write-ahead log (WAL) in addition to the Paxos log. Both are leveraged to provide better guarantees.
class DatabaseWithPaxos:
def execute_transaction(self, transaction):
# Phase 1: Write to local WAL
wal_record = self.wal.append(transaction)
# Phase 2: Replicate via Paxos
paxos_index = self.paxos.propose(transaction)
# Phase 3: Wait for Paxos commit
self.paxos.wait_for_commit(paxos_index)
# Phase 4: Apply to database
self.apply_to_database(transaction)
# Phase 5: Acknowledge to client
return Success(commit_index=paxos_index)
Concurrency Control
Multi-Paxos orders transactions but does not handle concurrency control. Databases layer locking or MVCC (Multi-Version Concurrency Control) on top:
class TransactionManager:
def __init__(self):
self.lock_table = LockTable()
self.paxos = MultiPaxos()
def begin_transaction(self):
return Transaction(id=uuid4())
def read(self, transaction, key):
# Acquire shared lock
self.lock_table.acquire_shared(transaction, key)
# Read from committed state
return self.database.read(key)
def write(self, transaction, key, value):
# Acquire exclusive lock
self.lock_table.acquire_exclusive(transaction, key)
# Buffer write
transaction.write_set[key] = value
def commit(self, transaction):
# Replicate write set via Paxos
record = CommitRecord(
transaction_id=transaction.id,
writes=transaction.write_set
)
self.paxos.propose(record)
# Wait for commit
self.paxos.wait_for_commit()
# Apply and release locks
self.apply_writes(transaction.write_set)
self.lock_table.release_all(transaction)
Multi-Partition Transactions
When a transaction spans multiple Paxos groups, two-phase commit coordinates the groups:
class TwoPhaseCommitCoordinator:
def commit_distributed(self, transaction, paxos_groups):
# Phase 1: Prepare all participants
prepare_results = []
for group in paxos_groups:
result = group.prepare(transaction)
prepare_results.append(result)
# Check if all prepared
if all(r.is_prepared() for r in prepare_results):
# Phase 2: Commit all participants
for group in paxos_groups:
group.commit(transaction)
return Committed()
else:
# Abort all participants
for group in paxos_groups:
group.abort(transaction)
return Aborted()
The key insight is that each prepare and commit operation is itself replicated through Paxos, making the coordinator and participants fault-tolerant.
By the way, I have a detailed video explaining Two Phase Commit, with a highly practical example. Give it a watch.
Other Optimizations
Read Quorums
Instead of always reading from the leader, clients can read from any majority of replicas:
class QuorumRead:
def read(self, key):
# Query majority of replicas
responses = []
for replica in self.replicas[:self.quorum_size]:
response = replica.read_local(key)
responses.append(response)
# Return value with highest version
return max(responses, key=lambda r: r.version).value
This distributes read load but requires quorum coordination for each read.
Lease-Based Reads
An alternative is leader leases, where the leader is guaranteed to be the authoritative source for a time window:
class LeaseBasedReads:
def read(self, key):
if self.is_leader and self.lease_valid():
# Serve directly, no coordination needed
return self.local_database.read(key)
else:
# Forward to leader
return self.leader.read(key)
def lease_valid(self):
return datetime.now() < self.lease_expiry - self.clock_skew_bound
The clock skew bound accounts for the possibility of clocks being slightly out of sync.
Witness Replicas
Witness replicas participate in voting but do not store full data. While they help maintain quorum more cheaply, they reduce durability guarantees and are somewhat controversial in practice.
Full replica: Stores data + votes on proposals
Witness replica: Votes on proposals only
Benefit: Cheaper to maintain quorum
Use case: Third datacenter for tie-breaking
class WitnessReplica:
def on_accept(self, request):
# Track ballot and index, but not full value
self.accepted_ballots[request.index] = request.ballot
# No need to store the actual value
return Accepted(ballot=request.ballot, index=request.index)
Debugging and Observability
Leader Thrashing
Frequent leader changes indicate network issues or misconfigured timeouts. Potential solutions
- Increase election timeout
- Fix network issues
- Check for asymmetric network partitions
Log Divergence
Replicas falling behind (gap between leader and follower commit indices) can cause consistency issues. Potential solutions
- Increase follower resources
- Check network bandwidth
- Reduce batch sizes temporarily
Conclusion
Multi-Paxos is a robust foundation for building distributed database transactions. By maintaining a stable leader and reusing the prepare phase, it achieves the low latency that database workloads demand.
When combined with optimizations like batching and pipelining, Multi-Paxos delivers high throughput while maintaining strong consistency guarantees.
The protocol is equipped to handle leader failures gracefully, recover log state during elections, and integrate with two-phase commit for multi-partition transactions. While it has limitations (like potential write bottlenecks at the leader), its proven correctness and battle-tested implementations make it a cornerstone of modern distributed databases.