12 KiB
Replication System Documentation
The replication system in Kevo implements a primary-replica architecture that allows scaling read operations across multiple replica nodes while maintaining a single writer (primary node). It ensures that replicas maintain a crash-resilient, consistent copy of the primary's data by streaming Write-Ahead Log (WAL) entries in strict logical order.
Overview
The replication system streams WAL entries from the primary node to replica nodes in real-time. It guarantees:
- Durability: All data is persisted before acknowledgment.
- Exactly-once application: WAL entries are applied in order without duplication.
- Crash resilience: Both primary and replicas can recover cleanly after restart.
- Simplicity: Designed to be minimal, efficient, and extensible.
- Transparent Client Experience: Client SDKs automatically handle routing between primary and replicas.
The WAL sequence number acts as a Lamport clock to provide total ordering across all operations.
Implementation Details
The replication system is implemented across several packages:
-
pkg/replication: Core replication functionality
- Primary implementation
- Replica implementation
- WAL streaming protocol
- Batching and compression
-
pkg/engine: Engine integration
- EngineFacade integration with ReplicationManager
- Read-only mode for replicas
-
pkg/client: Client SDK integration
- Node role discovery protocol
- Automatic operation routing
- Failover handling
Node Roles
Kevo supports three node roles:
-
Standalone: A single node with no replication
- Handles both reads and writes
- Default mode when replication is not configured
-
Primary: The single writer node in a replication cluster
- Processes all write operations
- Streams WAL entries to replicas
- Can serve read operations but typically offloads them to replicas
-
Replica: Read-only nodes that replicate data from the primary
- Process read operations
- Apply WAL entries from the primary
- Reject write operations with redirection information
Replication Manager
The ReplicationManager
is the central component of the replication system. It:
- Handles node configuration and setup
- Starts the appropriate mode (primary or replica) based on configuration
- Integrates with the storage engine and WAL
- Exposes replication topology information
- Manages the replication state machine
Configuration
The ReplicationManager is configured via the ManagerConfig
struct:
type ManagerConfig struct {
Enabled bool // Enable replication
Mode string // "primary", "replica", or "standalone"
ListenAddr string // Address for primary to listen on (e.g., ":50053")
PrimaryAddr string // Address of the primary (for replica mode)
// Advanced settings
MaxBatchSize int // Maximum batch size for streaming
RetentionTime time.Duration // How long to retain WAL entries
CompressionEnabled bool // Enable compression
}
Status Information
The ReplicationManager provides status information through its Status()
method:
// Example status information
{
"enabled": true,
"mode": "primary",
"active": true,
"listen_address": ":50053",
"connected_replicas": 2,
"last_sequence": 12345,
"bytes_transferred": 1048576
}
Primary Node Implementation
The primary node is responsible for:
- Observing WAL entries as they are written
- Streaming entries to connected replicas
- Handling acknowledgments from replicas
- Tracking replica state and lag
WAL Observer
The primary implements the WALEntryObserver
interface to be notified of new WAL entries:
// Simplified implementation
func (p *Primary) OnEntryWritten(entry *wal.Entry) {
p.buffer.Add(entry)
p.notifyReplicas()
}
Streaming Implementation
The primary streams entries using a gRPC service:
// Simplified streaming implementation
func (p *Primary) StreamWAL(req *proto.WALStreamRequest, stream proto.WALReplication_StreamWALServer) error {
startSeq := req.StartSequence
// Send initial entries from WAL
entries, err := p.wal.GetEntriesFrom(startSeq)
if err != nil {
return err
}
if err := p.sendEntries(entries, stream); err != nil {
return err
}
// Subscribe to new entries
subscription := p.subscribe()
defer p.unsubscribe(subscription)
for {
select {
case entries := <-subscription.Entries():
if err := p.sendEntries(entries, stream); err != nil {
return err
}
case <-stream.Context().Done():
return stream.Context().Err()
}
}
}
Replica Node Implementation
The replica node is responsible for:
- Connecting to the primary
- Receiving WAL entries
- Applying entries to the local storage engine
- Acknowledging successfully applied entries
State Machine
The replica uses a state machine to manage its lifecycle:
CONNECTING → STREAMING_ENTRIES → APPLYING_ENTRIES → FSYNC_PENDING → ACKNOWLEDGING → WAITING_FOR_DATA
Entry Application
Entries are applied in strict sequence order:
// Simplified implementation
func (r *Replica) applyEntries(entries []*wal.Entry) error {
// Verify entries are in proper sequence
for _, entry := range entries {
if entry.Sequence != r.nextExpectedSequence {
return ErrSequenceGap
}
r.nextExpectedSequence++
}
// Apply entries to the engine
if err := r.engine.ApplyBatch(entries); err != nil {
return err
}
// Update last applied sequence
r.lastAppliedSequence = entries[len(entries)-1].Sequence
return nil
}
Client SDK Integration
The client SDK provides a seamless experience for applications using Kevo with replication:
- Node Role Discovery: On connection, clients discover the node's role and replication topology
- Automatic Write Redirection: Write operations to replicas are transparently redirected to the primary
- Read Distribution: When connected to a primary with replicas, reads can be distributed to replicas
- Connection Recovery: Connection failures are handled with automatic retry and reconnection
Node Information
When connecting, the client retrieves node information:
// NodeInfo structure returned by the server
type NodeInfo struct {
Role string // "primary", "replica", or "standalone"
PrimaryAddr string // Address of the primary node (for replicas)
Replicas []ReplicaInfo // Available replica nodes (for primary)
LastSequence uint64 // Last applied sequence number
ReadOnly bool // Whether the node is in read-only mode
}
// Example ReplicaInfo
type ReplicaInfo struct {
Address string // Host:port of the replica
LastSequence uint64 // Last applied sequence number
Available bool // Whether the replica is available
Region string // Optional region information
Meta map[string]string // Additional metadata
}
Smart Routing
The client automatically routes operations to the appropriate node:
// Get retrieves a value by key
// If connected to a primary with replicas, it will route reads to a replica
func (c *Client) Get(ctx context.Context, key []byte) ([]byte, bool, error) {
// Check if we should route to replica
shouldUseReplica := c.nodeInfo != nil &&
c.nodeInfo.Role == "primary" &&
len(c.replicaConn) > 0
if shouldUseReplica {
// Select a replica for reading
selectedReplica := c.replicaConn[0]
// Try the replica first
resp, err := selectedReplica.Send(ctx, request)
// Fall back to primary if replica fails
if err != nil {
resp, err = c.client.Send(ctx, request)
}
} else {
// Use default connection
resp, err = c.client.Send(ctx, request)
}
// Process response...
}
// Put stores a key-value pair
// If connected to a replica, it will automatically route the write to the primary
func (c *Client) Put(ctx context.Context, key, value []byte) (bool, error) {
// Check if we should route to primary
shouldUsePrimary := c.nodeInfo != nil &&
c.nodeInfo.Role == "replica" &&
c.primaryConn != nil
if shouldUsePrimary {
// Use primary connection for writes when connected to replica
resp, err = c.primaryConn.Send(ctx, request)
} else {
// Use default connection
resp, err = c.client.Send(ctx, request)
// If we get a read-only error, try to discover topology and retry
if err != nil && isReadOnlyError(err) {
if err := c.discoverTopology(ctx); err == nil {
// Retry with primary if we now have one
if c.primaryConn != nil {
resp, err = c.primaryConn.Send(ctx, request)
}
}
}
}
// Process response...
}
Server Configuration
To run a Kevo server with replication, use the following configuration options:
Standalone Mode (Default)
kevo -server [database_path]
Primary Mode
kevo -server [database_path] -replication.enabled=true -replication.mode=primary -replication.listen=:50053
Replica Mode
kevo -server [database_path] -replication.enabled=true -replication.mode=replica -replication.primary=localhost:50053
Implementation Considerations
Durability
- Primary: All entries are durably written to WAL before being streamed
- Replica: Entries are applied and fsynced before acknowledgment
- WAL retention on primary ensures replicas can recover from short-term failures
Consistency
- Primary is always the source of truth for writes
- Replicas may temporarily lag behind the primary
- Last sequence number indicates replication status
- Clients can choose to verify replica freshness for critical operations
Performance
- Batch size is configurable to balance latency and throughput
- Compression can be enabled to reduce network bandwidth
- Read operations can be distributed across replicas for scaling
- Replicas operate in read-only mode, eliminating write contention
Fault Tolerance
- Replica node restart: Recover local state, catch up missing entries
- Primary node restart: Resume serving WAL entries to replicas
- Network failures: Automatic reconnection with exponential backoff
- Gap detection: Replicas verify sequence continuity
Protocol Details
The replication protocol is defined using Protocol Buffers:
service WALReplication {
rpc StreamWAL (WALStreamRequest) returns (stream WALStreamResponse);
rpc Acknowledge (Ack) returns (AckResponse);
}
message WALStreamRequest {
uint64 start_sequence = 1;
uint32 protocol_version = 2;
bool compression_supported = 3;
}
message WALStreamResponse {
repeated WALEntry entries = 1;
bool compressed = 2;
}
message WALEntry {
uint64 sequence_number = 1;
bytes payload = 2;
FragmentType fragment_type = 3;
}
message Ack {
uint64 acknowledged_up_to = 1;
}
message AckResponse {
bool success = 1;
string message = 2;
}
The protocol ensures:
- Entries are streamed in order
- Gaps are detected using sequence numbers
- Large entries can be fragmented
- Compression is negotiated for efficiency
Limitations and Trade-offs
- Single Writer Model: The system follows a strict single-writer architecture, limiting write throughput to a single primary node
- Replica Lag: Replicas may be slightly behind the primary, requiring careful consideration for read-after-write scenarios
- Manual Failover: The system does not implement automatic failover; if the primary fails, manual intervention is required
- Cold Start: If WAL entries are pruned, new replicas require a full resync from the primary
Future Work
The current implementation provides a robust foundation for replication, with several planned enhancements:
- Multi-region Replication: Optimize for cross-region replication
- Replica Groups: Support for replica tiers and read preferences
- Snapshot Transfer: Efficient initialization of new replicas without WAL replay
- Flow Control: Backpressure mechanisms to handle slow replicas