403 lines
12 KiB
Markdown
403 lines
12 KiB
Markdown
# 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:
|
|
|
|
1. **pkg/replication**: Core replication functionality
|
|
- Primary implementation
|
|
- Replica implementation
|
|
- WAL streaming protocol
|
|
- Batching and compression
|
|
|
|
2. **pkg/engine**: Engine integration
|
|
- EngineFacade integration with ReplicationManager
|
|
- Read-only mode for replicas
|
|
|
|
3. **pkg/client**: Client SDK integration
|
|
- Node role discovery protocol
|
|
- Automatic operation routing
|
|
- Failover handling
|
|
|
|
## Node Roles
|
|
|
|
Kevo supports three node roles:
|
|
|
|
1. **Standalone**: A single node with no replication
|
|
- Handles both reads and writes
|
|
- Default mode when replication is not configured
|
|
|
|
2. **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
|
|
|
|
3. **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:
|
|
|
|
1. Handles node configuration and setup
|
|
2. Starts the appropriate mode (primary or replica) based on configuration
|
|
3. Integrates with the storage engine and WAL
|
|
4. Exposes replication topology information
|
|
5. Manages the replication state machine
|
|
|
|
### Configuration
|
|
|
|
The ReplicationManager is configured via the `ManagerConfig` struct:
|
|
|
|
```go
|
|
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:
|
|
|
|
```go
|
|
// 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:
|
|
|
|
1. Observing WAL entries as they are written
|
|
2. Streaming entries to connected replicas
|
|
3. Handling acknowledgments from replicas
|
|
4. Tracking replica state and lag
|
|
|
|
### WAL Observer
|
|
|
|
The primary implements the `WALEntryObserver` interface to be notified of new WAL entries:
|
|
|
|
```go
|
|
// 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:
|
|
|
|
```go
|
|
// 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:
|
|
|
|
1. Connecting to the primary
|
|
2. Receiving WAL entries
|
|
3. Applying entries to the local storage engine
|
|
4. 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:
|
|
|
|
```go
|
|
// 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:
|
|
|
|
1. **Node Role Discovery**: On connection, clients discover the node's role and replication topology
|
|
2. **Automatic Write Redirection**: Write operations to replicas are transparently redirected to the primary
|
|
3. **Read Distribution**: When connected to a primary with replicas, reads can be distributed to replicas
|
|
4. **Connection Recovery**: Connection failures are handled with automatic retry and reconnection
|
|
|
|
### Node Information
|
|
|
|
When connecting, the client retrieves node information:
|
|
|
|
```go
|
|
// 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:
|
|
|
|
```go
|
|
// 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)
|
|
|
|
```bash
|
|
kevo -server [database_path]
|
|
```
|
|
|
|
### Primary Mode
|
|
|
|
```bash
|
|
kevo -server [database_path] -replication.enabled=true -replication.mode=primary -replication.listen=:50053
|
|
```
|
|
|
|
### Replica Mode
|
|
|
|
```bash
|
|
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:
|
|
|
|
```proto
|
|
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
|
|
|
|
1. **Single Writer Model**: The system follows a strict single-writer architecture, limiting write throughput to a single primary node
|
|
2. **Replica Lag**: Replicas may be slightly behind the primary, requiring careful consideration for read-after-write scenarios
|
|
3. **Manual Failover**: The system does not implement automatic failover; if the primary fails, manual intervention is required
|
|
4. **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:
|
|
|
|
1. **Multi-region Replication**: Optimize for cross-region replication
|
|
2. **Replica Groups**: Support for replica tiers and read preferences
|
|
3. **Snapshot Transfer**: Efficient initialization of new replicas without WAL replay
|
|
4. **Flow Control**: Backpressure mechanisms to handle slow replicas |