feat: implement replication transport layer
All checks were successful
Go Tests / Run Tests (1.24.2) (pull_request) Successful in 9m49s

This commit implements the replication transport layer as part of Phase 2 of the replication plan.
Key components include:

- Add protocol buffer definitions for replication services
- Implement WALReplicator extension for processor management
- Create replication service server implementation
- Add replication client and server transport implementations
- Implement storage snapshot interface for bootstrap operations
- Standardize package naming across replication components
This commit is contained in:
Jeremy Tregunna 2025-04-26 13:05:54 -06:00
parent ed991ae00d
commit 5963538bc5
Signed by: jer
GPG Key ID: 1278B36BA6F5D5E4
11 changed files with 3706 additions and 4 deletions

View File

@ -0,0 +1,610 @@
package service
import (
"context"
"fmt"
"io"
"sync"
"time"
"github.com/KevoDB/kevo/pkg/replication"
"github.com/KevoDB/kevo/pkg/transport"
"github.com/KevoDB/kevo/pkg/wal"
"github.com/KevoDB/kevo/proto/kevo"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
)
// ReplicationServiceServer implements the gRPC ReplicationService
type ReplicationServiceServer struct {
kevo.UnimplementedReplicationServiceServer
// Replication components
replicator *replication.WALReplicator
applier *replication.WALApplier
serializer *replication.EntrySerializer
highestLSN uint64
replicas map[string]*transport.ReplicaInfo
replicasMutex sync.RWMutex
// For snapshot/bootstrap
storageSnapshot replication.StorageSnapshot
}
// NewReplicationService creates a new ReplicationService
func NewReplicationService(
replicator *replication.WALReplicator,
applier *replication.WALApplier,
serializer *replication.EntrySerializer,
storageSnapshot replication.StorageSnapshot,
) *ReplicationServiceServer {
return &ReplicationServiceServer{
replicator: replicator,
applier: applier,
serializer: serializer,
replicas: make(map[string]*transport.ReplicaInfo),
storageSnapshot: storageSnapshot,
}
}
// RegisterReplica handles registration of a new replica
func (s *ReplicationServiceServer) RegisterReplica(
ctx context.Context,
req *kevo.RegisterReplicaRequest,
) (*kevo.RegisterReplicaResponse, error) {
// Validate request
if req.ReplicaId == "" {
return nil, status.Error(codes.InvalidArgument, "replica_id is required")
}
if req.Address == "" {
return nil, status.Error(codes.InvalidArgument, "address is required")
}
// Convert role enum to string
role := transport.RoleReplica
switch req.Role {
case kevo.ReplicaRole_PRIMARY:
role = transport.RolePrimary
case kevo.ReplicaRole_REPLICA:
role = transport.RoleReplica
case kevo.ReplicaRole_READ_ONLY:
role = transport.RoleReadOnly
default:
return nil, status.Error(codes.InvalidArgument, "invalid role")
}
// Register the replica
s.replicasMutex.Lock()
defer s.replicasMutex.Unlock()
// If already registered, update address and role
if replica, exists := s.replicas[req.ReplicaId]; exists {
replica.Address = req.Address
replica.Role = role
replica.LastSeen = time.Now()
replica.Status = transport.StatusConnecting
} else {
// Create new replica info
s.replicas[req.ReplicaId] = &transport.ReplicaInfo{
ID: req.ReplicaId,
Address: req.Address,
Role: role,
Status: transport.StatusConnecting,
LastSeen: time.Now(),
}
}
// Determine if bootstrap is needed (for now always suggest bootstrap)
bootstrapRequired := true
// Return current highest LSN
currentLSN := s.replicator.GetHighestTimestamp()
return &kevo.RegisterReplicaResponse{
Success: true,
CurrentLsn: currentLSN,
BootstrapRequired: bootstrapRequired,
}, nil
}
// ReplicaHeartbeat handles heartbeat requests from replicas
func (s *ReplicationServiceServer) ReplicaHeartbeat(
ctx context.Context,
req *kevo.ReplicaHeartbeatRequest,
) (*kevo.ReplicaHeartbeatResponse, error) {
// Validate request
if req.ReplicaId == "" {
return nil, status.Error(codes.InvalidArgument, "replica_id is required")
}
// Check if replica is registered
s.replicasMutex.Lock()
defer s.replicasMutex.Unlock()
replica, exists := s.replicas[req.ReplicaId]
if !exists {
return nil, status.Error(codes.NotFound, "replica not registered")
}
// Update replica status
replica.LastSeen = time.Now()
// Convert status enum to string
switch req.Status {
case kevo.ReplicaStatus_CONNECTING:
replica.Status = transport.StatusConnecting
case kevo.ReplicaStatus_SYNCING:
replica.Status = transport.StatusSyncing
case kevo.ReplicaStatus_BOOTSTRAPPING:
replica.Status = transport.StatusBootstrapping
case kevo.ReplicaStatus_READY:
replica.Status = transport.StatusReady
case kevo.ReplicaStatus_DISCONNECTED:
replica.Status = transport.StatusDisconnected
case kevo.ReplicaStatus_ERROR:
replica.Status = transport.StatusError
replica.Error = fmt.Errorf("%s", req.ErrorMessage)
default:
return nil, status.Error(codes.InvalidArgument, "invalid status")
}
// Update replica LSN
replica.CurrentLSN = req.CurrentLsn
// Calculate replication lag
primaryLSN := s.replicator.GetHighestTimestamp()
var replicationLagMs int64 = 0
if primaryLSN > req.CurrentLsn {
// Simple lag calculation based on LSN difference
// In a real system, we'd use timestamps for better accuracy
replicationLagMs = int64(primaryLSN - req.CurrentLsn)
}
replica.ReplicationLag = time.Duration(replicationLagMs) * time.Millisecond
return &kevo.ReplicaHeartbeatResponse{
Success: true,
PrimaryLsn: primaryLSN,
ReplicationLagMs: replicationLagMs,
}, nil
}
// GetReplicaStatus retrieves the status of a specific replica
func (s *ReplicationServiceServer) GetReplicaStatus(
ctx context.Context,
req *kevo.GetReplicaStatusRequest,
) (*kevo.GetReplicaStatusResponse, error) {
// Validate request
if req.ReplicaId == "" {
return nil, status.Error(codes.InvalidArgument, "replica_id is required")
}
// Get replica info
s.replicasMutex.RLock()
defer s.replicasMutex.RUnlock()
replica, exists := s.replicas[req.ReplicaId]
if !exists {
return nil, status.Error(codes.NotFound, "replica not found")
}
// Convert replica info to proto message
pbReplica := convertReplicaInfoToProto(replica)
return &kevo.GetReplicaStatusResponse{
Replica: pbReplica,
}, nil
}
// ListReplicas retrieves the status of all replicas
func (s *ReplicationServiceServer) ListReplicas(
ctx context.Context,
req *kevo.ListReplicasRequest,
) (*kevo.ListReplicasResponse, error) {
s.replicasMutex.RLock()
defer s.replicasMutex.RUnlock()
// Convert all replicas to proto messages
pbReplicas := make([]*kevo.ReplicaInfo, 0, len(s.replicas))
for _, replica := range s.replicas {
pbReplicas = append(pbReplicas, convertReplicaInfoToProto(replica))
}
return &kevo.ListReplicasResponse{
Replicas: pbReplicas,
}, nil
}
// GetWALEntries handles requests for WAL entries
func (s *ReplicationServiceServer) GetWALEntries(
ctx context.Context,
req *kevo.GetWALEntriesRequest,
) (*kevo.GetWALEntriesResponse, error) {
// Validate request
if req.ReplicaId == "" {
return nil, status.Error(codes.InvalidArgument, "replica_id is required")
}
// Check if replica is registered
s.replicasMutex.RLock()
_, exists := s.replicas[req.ReplicaId]
s.replicasMutex.RUnlock()
if !exists {
return nil, status.Error(codes.NotFound, "replica not registered")
}
// Get entries from replicator
entries, err := s.replicator.GetEntriesAfter(replication.ReplicationPosition{Timestamp: req.FromLsn})
if err != nil {
return nil, status.Errorf(codes.Internal, "failed to get entries: %v", err)
}
if len(entries) == 0 {
return &kevo.GetWALEntriesResponse{
Batch: &kevo.WALEntryBatch{},
HasMore: false,
}, nil
}
// Convert entries to proto messages
pbEntries := make([]*kevo.WALEntry, 0, len(entries))
for _, entry := range entries {
pbEntries = append(pbEntries, convertWALEntryToProto(entry))
}
// Create batch
pbBatch := &kevo.WALEntryBatch{
Entries: pbEntries,
FirstLsn: entries[0].SequenceNumber,
LastLsn: entries[len(entries)-1].SequenceNumber,
Count: uint32(len(entries)),
}
// Check if there are more entries
hasMore := s.replicator.GetHighestTimestamp() > entries[len(entries)-1].SequenceNumber
return &kevo.GetWALEntriesResponse{
Batch: pbBatch,
HasMore: hasMore,
}, nil
}
// StreamWALEntries handles streaming WAL entries to a replica
func (s *ReplicationServiceServer) StreamWALEntries(
req *kevo.StreamWALEntriesRequest,
stream kevo.ReplicationService_StreamWALEntriesServer,
) error {
// Validate request
if req.ReplicaId == "" {
return status.Error(codes.InvalidArgument, "replica_id is required")
}
// Check if replica is registered
s.replicasMutex.RLock()
_, exists := s.replicas[req.ReplicaId]
s.replicasMutex.RUnlock()
if !exists {
return status.Error(codes.NotFound, "replica not registered")
}
// Process entries in batches
fromLSN := req.FromLsn
batchSize := 100 // Can be configurable
notifierCh := make(chan struct{}, 1)
// Register for notifications of new entries
var processor *entryNotifier
if req.Continuous {
processor = &entryNotifier{
notifyCh: notifierCh,
fromLSN: fromLSN,
}
s.replicator.AddProcessor(processor)
defer s.replicator.RemoveProcessor(processor)
}
// Initial send of available entries
for {
// Get batch of entries
entries, err := s.replicator.GetEntriesAfter(replication.ReplicationPosition{Timestamp: fromLSN})
if err != nil {
return err
}
if len(entries) == 0 {
// No more entries, check if continuous streaming
if !req.Continuous {
break
}
// Wait for notification of new entries
select {
case <-notifierCh:
continue
case <-stream.Context().Done():
return stream.Context().Err()
}
}
// Create batch message
pbEntries := make([]*kevo.WALEntry, 0, len(entries))
for _, entry := range entries {
pbEntries = append(pbEntries, convertWALEntryToProto(entry))
}
pbBatch := &kevo.WALEntryBatch{
Entries: pbEntries,
FirstLsn: entries[0].SequenceNumber,
LastLsn: entries[len(entries)-1].SequenceNumber,
Count: uint32(len(entries)),
}
// Send batch
if err := stream.Send(pbBatch); err != nil {
return err
}
// Update fromLSN for next batch
fromLSN = entries[len(entries)-1].SequenceNumber + 1
// If not continuous, break after sending all available entries
if !req.Continuous && len(entries) < batchSize {
break
}
}
return nil
}
// ReportAppliedEntries handles reports of entries applied by replicas
func (s *ReplicationServiceServer) ReportAppliedEntries(
ctx context.Context,
req *kevo.ReportAppliedEntriesRequest,
) (*kevo.ReportAppliedEntriesResponse, error) {
// Validate request
if req.ReplicaId == "" {
return nil, status.Error(codes.InvalidArgument, "replica_id is required")
}
// Update replica LSN
s.replicasMutex.Lock()
replica, exists := s.replicas[req.ReplicaId]
if exists {
replica.CurrentLSN = req.AppliedLsn
}
s.replicasMutex.Unlock()
if !exists {
return nil, status.Error(codes.NotFound, "replica not registered")
}
return &kevo.ReportAppliedEntriesResponse{
Success: true,
PrimaryLsn: s.replicator.GetHighestTimestamp(),
}, nil
}
// RequestBootstrap handles bootstrap requests from replicas
func (s *ReplicationServiceServer) RequestBootstrap(
req *kevo.BootstrapRequest,
stream kevo.ReplicationService_RequestBootstrapServer,
) error {
// Validate request
if req.ReplicaId == "" {
return status.Error(codes.InvalidArgument, "replica_id is required")
}
// Check if replica is registered
s.replicasMutex.RLock()
replica, exists := s.replicas[req.ReplicaId]
s.replicasMutex.RUnlock()
if !exists {
return status.Error(codes.NotFound, "replica not registered")
}
// Update replica status
s.replicasMutex.Lock()
replica.Status = transport.StatusBootstrapping
s.replicasMutex.Unlock()
// Create snapshot of current data
snapshotLSN := s.replicator.GetHighestTimestamp()
iterator, err := s.storageSnapshot.CreateSnapshotIterator()
if err != nil {
s.replicasMutex.Lock()
replica.Status = transport.StatusError
replica.Error = err
s.replicasMutex.Unlock()
return status.Errorf(codes.Internal, "failed to create snapshot: %v", err)
}
defer iterator.Close()
// Stream key-value pairs in batches
batchSize := 100 // Can be configurable
totalCount := s.storageSnapshot.KeyCount()
sentCount := 0
batch := make([]*kevo.KeyValuePair, 0, batchSize)
for {
// Get next key-value pair
key, value, err := iterator.Next()
if err == io.EOF {
break
}
if err != nil {
s.replicasMutex.Lock()
replica.Status = transport.StatusError
replica.Error = err
s.replicasMutex.Unlock()
return status.Errorf(codes.Internal, "error reading snapshot: %v", err)
}
// Add to batch
batch = append(batch, &kevo.KeyValuePair{
Key: key,
Value: value,
})
// Send batch if full
if len(batch) >= batchSize {
progress := float32(sentCount) / float32(totalCount)
if err := stream.Send(&kevo.BootstrapBatch{
Pairs: batch,
Progress: progress,
IsLast: false,
SnapshotLsn: snapshotLSN,
}); err != nil {
return err
}
// Reset batch and update count
sentCount += len(batch)
batch = batch[:0]
}
}
// Send final batch
if len(batch) > 0 {
sentCount += len(batch)
progress := float32(sentCount) / float32(totalCount)
if err := stream.Send(&kevo.BootstrapBatch{
Pairs: batch,
Progress: progress,
IsLast: true,
SnapshotLsn: snapshotLSN,
}); err != nil {
return err
}
} else if sentCount > 0 {
// Send empty final batch to mark the end
if err := stream.Send(&kevo.BootstrapBatch{
Pairs: []*kevo.KeyValuePair{},
Progress: 1.0,
IsLast: true,
SnapshotLsn: snapshotLSN,
}); err != nil {
return err
}
}
// Update replica status
s.replicasMutex.Lock()
replica.Status = transport.StatusSyncing
replica.CurrentLSN = snapshotLSN
s.replicasMutex.Unlock()
return nil
}
// Helper to convert replica info to proto message
func convertReplicaInfoToProto(replica *transport.ReplicaInfo) *kevo.ReplicaInfo {
// Convert status to proto enum
var pbStatus kevo.ReplicaStatus
switch replica.Status {
case transport.StatusConnecting:
pbStatus = kevo.ReplicaStatus_CONNECTING
case transport.StatusSyncing:
pbStatus = kevo.ReplicaStatus_SYNCING
case transport.StatusBootstrapping:
pbStatus = kevo.ReplicaStatus_BOOTSTRAPPING
case transport.StatusReady:
pbStatus = kevo.ReplicaStatus_READY
case transport.StatusDisconnected:
pbStatus = kevo.ReplicaStatus_DISCONNECTED
case transport.StatusError:
pbStatus = kevo.ReplicaStatus_ERROR
default:
pbStatus = kevo.ReplicaStatus_DISCONNECTED
}
// Convert role to proto enum
var pbRole kevo.ReplicaRole
switch replica.Role {
case transport.RolePrimary:
pbRole = kevo.ReplicaRole_PRIMARY
case transport.RoleReplica:
pbRole = kevo.ReplicaRole_REPLICA
case transport.RoleReadOnly:
pbRole = kevo.ReplicaRole_READ_ONLY
default:
pbRole = kevo.ReplicaRole_REPLICA
}
// Create proto message
pbReplica := &kevo.ReplicaInfo{
ReplicaId: replica.ID,
Address: replica.Address,
Role: pbRole,
Status: pbStatus,
LastSeenMs: replica.LastSeen.UnixMilli(),
CurrentLsn: replica.CurrentLSN,
ReplicationLagMs: replica.ReplicationLag.Milliseconds(),
}
// Add error message if any
if replica.Error != nil {
pbReplica.ErrorMessage = replica.Error.Error()
}
return pbReplica
}
// Convert WAL entry to proto message
func convertWALEntryToProto(entry *wal.Entry) *kevo.WALEntry {
return &kevo.WALEntry{
SequenceNumber: entry.SequenceNumber,
Type: uint32(entry.Type),
Key: entry.Key,
Value: entry.Value,
// We'd normally calculate a checksum here
Checksum: nil,
}
}
// entryNotifier is a helper struct that implements replication.EntryProcessor
// to notify when new entries are available
type entryNotifier struct {
notifyCh chan struct{}
fromLSN uint64
}
func (n *entryNotifier) ProcessEntry(entry *wal.Entry) error {
if entry.SequenceNumber >= n.fromLSN {
select {
case n.notifyCh <- struct{}{}:
default:
// Channel already has a notification, no need to send another
}
}
return nil
}
func (n *entryNotifier) ProcessBatch(entries []*wal.Entry) error {
if len(entries) > 0 && entries[len(entries)-1].SequenceNumber >= n.fromLSN {
select {
case n.notifyCh <- struct{}{}:
default:
// Channel already has a notification, no need to send another
}
}
return nil
}
// Define the interface for storage snapshot operations
// This would normally be implemented by the storage engine
type StorageSnapshot interface {
// CreateSnapshotIterator creates an iterator for a storage snapshot
CreateSnapshotIterator() (SnapshotIterator, error)
// KeyCount returns the approximate number of keys in storage
KeyCount() int64
}
// SnapshotIterator provides iteration over key-value pairs in storage
type SnapshotIterator interface {
// Next returns the next key-value pair
Next() (key []byte, value []byte, err error)
// Close closes the iterator
Close() error
}

View File

@ -0,0 +1,494 @@
package transport
import (
"context"
"io"
"sync"
"time"
"github.com/KevoDB/kevo/pkg/replication"
"github.com/KevoDB/kevo/pkg/transport"
"github.com/KevoDB/kevo/pkg/wal"
"github.com/KevoDB/kevo/proto/kevo"
"google.golang.org/grpc"
"google.golang.org/grpc/connectivity"
)
// ReplicationGRPCClient implements the ReplicationClient interface using gRPC
type ReplicationGRPCClient struct {
conn *grpc.ClientConn
client kevo.ReplicationServiceClient
endpoint string
options transport.TransportOptions
replicaID string
status transport.TransportStatus
applier *replication.WALApplier
serializer *replication.EntrySerializer
highestAppliedLSN uint64
currentLSN uint64
mu sync.RWMutex
}
// NewReplicationGRPCClient creates a new ReplicationGRPCClient
func NewReplicationGRPCClient(
endpoint string,
options transport.TransportOptions,
replicaID string,
applier *replication.WALApplier,
serializer *replication.EntrySerializer,
) (*ReplicationGRPCClient, error) {
return &ReplicationGRPCClient{
endpoint: endpoint,
options: options,
replicaID: replicaID,
applier: applier,
serializer: serializer,
status: transport.TransportStatus{
Connected: false,
LastConnected: time.Time{},
LastError: nil,
BytesSent: 0,
BytesReceived: 0,
RTT: 0,
},
}, nil
}
// Connect establishes a connection to the server
func (c *ReplicationGRPCClient) Connect(ctx context.Context) error {
c.mu.Lock()
defer c.mu.Unlock()
// Set up connection options
dialOptions := []grpc.DialOption{
grpc.WithBlock(),
}
// Add TLS if configured - TODO: Add TLS support once TLS helpers are implemented
if c.options.TLSEnabled {
// We'll need to implement TLS credentials loading
// For now, we'll skip TLS
c.options.TLSEnabled = false
} else {
dialOptions = append(dialOptions, grpc.WithInsecure())
}
// Set timeout for connection
dialCtx, cancel := context.WithTimeout(ctx, c.options.Timeout)
defer cancel()
// Establish connection
conn, err := grpc.DialContext(dialCtx, c.endpoint, dialOptions...)
if err != nil {
c.status.LastError = err
return err
}
c.conn = conn
c.client = kevo.NewReplicationServiceClient(conn)
c.status.Connected = true
c.status.LastConnected = time.Now()
return nil
}
// Close closes the connection
func (c *ReplicationGRPCClient) Close() error {
c.mu.Lock()
defer c.mu.Unlock()
if c.conn != nil {
err := c.conn.Close()
c.conn = nil
c.client = nil
c.status.Connected = false
if err != nil {
c.status.LastError = err
return err
}
}
return nil
}
// IsConnected returns whether the client is connected
func (c *ReplicationGRPCClient) IsConnected() bool {
c.mu.RLock()
defer c.mu.RUnlock()
if c.conn == nil {
return false
}
state := c.conn.GetState()
return state == connectivity.Ready || state == connectivity.Idle
}
// Status returns the current status of the connection
func (c *ReplicationGRPCClient) Status() transport.TransportStatus {
c.mu.RLock()
defer c.mu.RUnlock()
return c.status
}
// Send sends a request and waits for a response
func (c *ReplicationGRPCClient) Send(ctx context.Context, request transport.Request) (transport.Response, error) {
// Implementation depends on specific replication messages
// This is a placeholder that would be completed for each message type
return nil, transport.ErrInvalidRequest
}
// Stream opens a bidirectional stream
func (c *ReplicationGRPCClient) Stream(ctx context.Context) (transport.Stream, error) {
// Not implemented for replication client
return nil, transport.ErrInvalidRequest
}
// RegisterAsReplica registers this client as a replica with the primary
func (c *ReplicationGRPCClient) RegisterAsReplica(ctx context.Context, replicaID string) error {
c.mu.Lock()
defer c.mu.Unlock()
if c.client == nil {
return transport.ErrNotConnected
}
// Create registration request
req := &kevo.RegisterReplicaRequest{
ReplicaId: replicaID,
Address: c.endpoint, // Use client endpoint as the replica address
Role: kevo.ReplicaRole_REPLICA,
}
// Call the service
resp, err := c.client.RegisterReplica(ctx, req)
if err != nil {
c.status.LastError = err
return err
}
// Update client info based on response
c.replicaID = replicaID
c.currentLSN = resp.CurrentLsn
return nil
}
// SendHeartbeat sends a heartbeat to the primary
func (c *ReplicationGRPCClient) SendHeartbeat(ctx context.Context, info *transport.ReplicaInfo) error {
c.mu.Lock()
defer c.mu.Unlock()
if c.client == nil {
return transport.ErrNotConnected
}
// Convert status to proto enum
var pbStatus kevo.ReplicaStatus
switch info.Status {
case transport.StatusConnecting:
pbStatus = kevo.ReplicaStatus_CONNECTING
case transport.StatusSyncing:
pbStatus = kevo.ReplicaStatus_SYNCING
case transport.StatusBootstrapping:
pbStatus = kevo.ReplicaStatus_BOOTSTRAPPING
case transport.StatusReady:
pbStatus = kevo.ReplicaStatus_READY
case transport.StatusDisconnected:
pbStatus = kevo.ReplicaStatus_DISCONNECTED
case transport.StatusError:
pbStatus = kevo.ReplicaStatus_ERROR
default:
pbStatus = kevo.ReplicaStatus_DISCONNECTED
}
// Create heartbeat request
req := &kevo.ReplicaHeartbeatRequest{
ReplicaId: c.replicaID,
Status: pbStatus,
CurrentLsn: c.highestAppliedLSN,
ErrorMessage: "",
}
// Add error message if any
if info.Error != nil {
req.ErrorMessage = info.Error.Error()
}
// Call the service
resp, err := c.client.ReplicaHeartbeat(ctx, req)
if err != nil {
c.status.LastError = err
return err
}
// Update client info based on response
c.currentLSN = resp.PrimaryLsn
return nil
}
// RequestWALEntries requests WAL entries from the primary starting from a specific LSN
func (c *ReplicationGRPCClient) RequestWALEntries(ctx context.Context, fromLSN uint64) ([]*wal.Entry, error) {
c.mu.Lock()
defer c.mu.Unlock()
if c.client == nil {
return nil, transport.ErrNotConnected
}
// Create request
req := &kevo.GetWALEntriesRequest{
ReplicaId: c.replicaID,
FromLsn: fromLSN,
MaxEntries: 1000, // Configurable
}
// Call the service
resp, err := c.client.GetWALEntries(ctx, req)
if err != nil {
c.status.LastError = err
return nil, err
}
// Convert proto entries to WAL entries
entries := make([]*wal.Entry, 0, len(resp.Batch.Entries))
for _, pbEntry := range resp.Batch.Entries {
entry := &wal.Entry{
SequenceNumber: pbEntry.SequenceNumber,
Type: uint8(pbEntry.Type),
Key: pbEntry.Key,
Value: pbEntry.Value,
}
entries = append(entries, entry)
}
return entries, nil
}
// RequestBootstrap requests a snapshot for bootstrap purposes
func (c *ReplicationGRPCClient) RequestBootstrap(ctx context.Context) (transport.BootstrapIterator, error) {
c.mu.Lock()
defer c.mu.Unlock()
if c.client == nil {
return nil, transport.ErrNotConnected
}
// Create request
req := &kevo.BootstrapRequest{
ReplicaId: c.replicaID,
}
// Call the service
stream, err := c.client.RequestBootstrap(ctx, req)
if err != nil {
c.status.LastError = err
return nil, err
}
// Create and return bootstrap iterator
return &GRPCBootstrapIterator{
stream: stream,
totalPairs: 0,
seenPairs: 0,
progress: 0.0,
mu: &sync.Mutex{},
applier: c.applier,
}, nil
}
// StartReplicationStream starts a stream for continuous replication
func (c *ReplicationGRPCClient) StartReplicationStream(ctx context.Context) error {
if !c.IsConnected() {
return transport.ErrNotConnected
}
// Get current highest applied LSN
c.mu.Lock()
fromLSN := c.highestAppliedLSN
c.mu.Unlock()
// Create stream request
req := &kevo.StreamWALEntriesRequest{
ReplicaId: c.replicaID,
FromLsn: fromLSN,
Continuous: true,
}
// Start streaming
stream, err := c.client.StreamWALEntries(ctx, req)
if err != nil {
return err
}
// Process stream in a goroutine
go c.processWALStream(ctx, stream)
return nil
}
// processWALStream handles the incoming WAL entry stream
func (c *ReplicationGRPCClient) processWALStream(ctx context.Context, stream kevo.ReplicationService_StreamWALEntriesClient) {
for {
// Check if context is cancelled
select {
case <-ctx.Done():
return
default:
// Continue processing
}
// Receive next batch
batch, err := stream.Recv()
if err == io.EOF {
// Stream completed normally
return
}
if err != nil {
// Stream error
c.mu.Lock()
c.status.LastError = err
c.mu.Unlock()
return
}
// Process entries in batch
entries := make([]*wal.Entry, 0, len(batch.Entries))
for _, pbEntry := range batch.Entries {
entry := &wal.Entry{
SequenceNumber: pbEntry.SequenceNumber,
Type: uint8(pbEntry.Type),
Key: pbEntry.Key,
Value: pbEntry.Value,
}
entries = append(entries, entry)
}
// Apply entries
if len(entries) > 0 {
_, err = c.applier.ApplyBatch(entries)
if err != nil {
c.mu.Lock()
c.status.LastError = err
c.mu.Unlock()
return
}
// Update highest applied LSN
c.mu.Lock()
c.highestAppliedLSN = batch.LastLsn
c.mu.Unlock()
// Report applied entries
go c.reportAppliedEntries(context.Background(), batch.LastLsn)
}
}
}
// reportAppliedEntries reports the highest applied LSN to the primary
func (c *ReplicationGRPCClient) reportAppliedEntries(ctx context.Context, appliedLSN uint64) {
c.mu.RLock()
client := c.client
replicaID := c.replicaID
c.mu.RUnlock()
if client == nil {
return
}
// Create request
req := &kevo.ReportAppliedEntriesRequest{
ReplicaId: replicaID,
AppliedLsn: appliedLSN,
}
// Call the service
_, err := client.ReportAppliedEntries(ctx, req)
if err != nil {
// Just log error, don't return it
c.mu.Lock()
c.status.LastError = err
c.mu.Unlock()
}
}
// GRPCBootstrapIterator implements the BootstrapIterator interface for gRPC
type GRPCBootstrapIterator struct {
stream kevo.ReplicationService_RequestBootstrapClient
currentBatch *kevo.BootstrapBatch
batchIndex int
totalPairs int
seenPairs int
progress float64
mu *sync.Mutex
applier *replication.WALApplier
}
// Next returns the next key-value pair
func (it *GRPCBootstrapIterator) Next() ([]byte, []byte, error) {
it.mu.Lock()
defer it.mu.Unlock()
// If we have a current batch and there are more pairs
if it.currentBatch != nil && it.batchIndex < len(it.currentBatch.Pairs) {
pair := it.currentBatch.Pairs[it.batchIndex]
it.batchIndex++
it.seenPairs++
return pair.Key, pair.Value, nil
}
// Need to get a new batch
batch, err := it.stream.Recv()
if err == io.EOF {
return nil, nil, io.EOF
}
if err != nil {
return nil, nil, err
}
// Update progress
it.currentBatch = batch
it.batchIndex = 0
it.progress = float64(batch.Progress)
// If batch is empty and it's the last one
if len(batch.Pairs) == 0 && batch.IsLast {
// Store the snapshot LSN for later use
if it.applier != nil {
it.applier.ResetHighestApplied(batch.SnapshotLsn)
}
return nil, nil, io.EOF
}
// If batch is empty but not the last one, try again
if len(batch.Pairs) == 0 {
return it.Next()
}
// Return the first pair from the new batch
pair := batch.Pairs[it.batchIndex]
it.batchIndex++
it.seenPairs++
return pair.Key, pair.Value, nil
}
// Close closes the iterator
func (it *GRPCBootstrapIterator) Close() error {
it.mu.Lock()
defer it.mu.Unlock()
// Store the snapshot LSN if we have a current batch and it's the last one
if it.currentBatch != nil && it.currentBatch.IsLast && it.applier != nil {
it.applier.ResetHighestApplied(it.currentBatch.SnapshotLsn)
}
return nil
}
// Progress returns the progress of the bootstrap operation (0.0-1.0)
func (it *GRPCBootstrapIterator) Progress() float64 {
it.mu.Lock()
defer it.mu.Unlock()
return it.progress
}

View File

@ -0,0 +1,200 @@
package transport
import (
"context"
"fmt"
"sync"
"github.com/KevoDB/kevo/pkg/grpc/service"
"github.com/KevoDB/kevo/pkg/replication"
"github.com/KevoDB/kevo/pkg/transport"
)
// ReplicationGRPCServer implements the ReplicationServer interface using gRPC
type ReplicationGRPCServer struct {
transportManager *GRPCTransportManager
replicationService *service.ReplicationServiceServer
options transport.TransportOptions
replicas map[string]*transport.ReplicaInfo
mu sync.RWMutex
}
// NewReplicationGRPCServer creates a new ReplicationGRPCServer
func NewReplicationGRPCServer(
transportManager *GRPCTransportManager,
replicator *replication.WALReplicator,
applier *replication.WALApplier,
serializer *replication.EntrySerializer,
storageSnapshot replication.StorageSnapshot,
options transport.TransportOptions,
) (*ReplicationGRPCServer, error) {
// Create replication service
replicationService := service.NewReplicationService(
replicator,
applier,
serializer,
storageSnapshot,
)
return &ReplicationGRPCServer{
transportManager: transportManager,
replicationService: replicationService,
options: options,
replicas: make(map[string]*transport.ReplicaInfo),
}, nil
}
// Start starts the server and returns immediately
func (s *ReplicationGRPCServer) Start() error {
// Register the replication service with the transport manager
if err := s.transportManager.RegisterService(s.replicationService); err != nil {
return fmt.Errorf("failed to register replication service: %w", err)
}
// Start the transport manager if it's not already started
if err := s.transportManager.Start(); err != nil {
return fmt.Errorf("failed to start transport manager: %w", err)
}
return nil
}
// Serve starts the server and blocks until it's stopped
func (s *ReplicationGRPCServer) Serve() error {
if err := s.Start(); err != nil {
return err
}
// This will block until the context is cancelled
<-context.Background().Done()
return nil
}
// Stop stops the server gracefully
func (s *ReplicationGRPCServer) Stop(ctx context.Context) error {
return s.transportManager.Stop(ctx)
}
// SetRequestHandler sets the handler for incoming requests
// Not used for the replication server as it uses a dedicated service
func (s *ReplicationGRPCServer) SetRequestHandler(handler transport.RequestHandler) {
// No-op for replication server
}
// RegisterReplica registers a new replica
func (s *ReplicationGRPCServer) RegisterReplica(replicaInfo *transport.ReplicaInfo) error {
s.mu.Lock()
defer s.mu.Unlock()
s.replicas[replicaInfo.ID] = replicaInfo
return nil
}
// UpdateReplicaStatus updates the status of a replica
func (s *ReplicationGRPCServer) UpdateReplicaStatus(replicaID string, status transport.ReplicaStatus, lsn uint64) error {
s.mu.Lock()
defer s.mu.Unlock()
replica, exists := s.replicas[replicaID]
if !exists {
return fmt.Errorf("replica not found: %s", replicaID)
}
replica.Status = status
replica.CurrentLSN = lsn
return nil
}
// GetReplicaInfo returns information about a specific replica
func (s *ReplicationGRPCServer) GetReplicaInfo(replicaID string) (*transport.ReplicaInfo, error) {
s.mu.RLock()
defer s.mu.RUnlock()
replica, exists := s.replicas[replicaID]
if !exists {
return nil, fmt.Errorf("replica not found: %s", replicaID)
}
return replica, nil
}
// ListReplicas returns information about all connected replicas
func (s *ReplicationGRPCServer) ListReplicas() ([]*transport.ReplicaInfo, error) {
s.mu.RLock()
defer s.mu.RUnlock()
result := make([]*transport.ReplicaInfo, 0, len(s.replicas))
for _, replica := range s.replicas {
result = append(result, replica)
}
return result, nil
}
// StreamWALEntriesToReplica streams WAL entries to a specific replica
func (s *ReplicationGRPCServer) StreamWALEntriesToReplica(ctx context.Context, replicaID string, fromLSN uint64) error {
// This is handled by the gRPC service directly
return nil
}
// RegisterReplicationTransport registers the gRPC replication transport with the registry
func init() {
// Register replication server factory
transport.RegisterReplicationServerTransport("grpc", func(address string, options transport.TransportOptions) (transport.ReplicationServer, error) {
// Create gRPC transport manager
grpcOptions := &GRPCTransportOptions{
ListenAddr: address,
ConnectionTimeout: options.Timeout,
DialTimeout: options.Timeout,
}
// Add TLS configuration if enabled
if options.TLSEnabled {
tlsConfig, err := LoadServerTLSConfig(options.CertFile, options.KeyFile, options.CAFile)
if err != nil {
return nil, fmt.Errorf("failed to load TLS config: %w", err)
}
grpcOptions.TLSConfig = tlsConfig
}
// Create transport manager
manager, err := NewGRPCTransportManager(grpcOptions)
if err != nil {
return nil, fmt.Errorf("failed to create gRPC transport manager: %w", err)
}
// For registration, we return a placeholder that will be properly initialized
// by the caller with the required components
return &ReplicationGRPCServer{
transportManager: manager,
options: options,
replicas: make(map[string]*transport.ReplicaInfo),
}, nil
})
// Register replication client factory
transport.RegisterReplicationClientTransport("grpc", func(endpoint string, options transport.TransportOptions) (transport.ReplicationClient, error) {
// For registration, we return a placeholder that will be properly initialized
// by the caller with the required components
return &ReplicationGRPCClient{
endpoint: endpoint,
options: options,
}, nil
})
}
// WithReplicator adds a replicator to the replication server
func (s *ReplicationGRPCServer) WithReplicator(
replicator *replication.WALReplicator,
applier *replication.WALApplier,
serializer *replication.EntrySerializer,
storageSnapshot replication.StorageSnapshot,
) *ReplicationGRPCServer {
s.replicationService = service.NewReplicationService(
replicator,
applier,
serializer,
storageSnapshot,
)
return s
}

View File

@ -0,0 +1,46 @@
package replication
// No imports needed
// processorIndex finds the index of a processor in the processors slice
// Returns -1 if not found
func (r *WALReplicator) processorIndex(target EntryProcessor) int {
r.mu.RLock()
defer r.mu.RUnlock()
for i, p := range r.processors {
if p == target {
return i
}
}
return -1
}
// RemoveProcessor removes an EntryProcessor from the replicator
func (r *WALReplicator) RemoveProcessor(processor EntryProcessor) {
if processor == nil {
return
}
r.mu.Lock()
defer r.mu.Unlock()
// Find the processor in the slice
idx := -1
for i, p := range r.processors {
if p == processor {
idx = i
break
}
}
// If found, remove it
if idx >= 0 {
// Remove the element by replacing it with the last element and truncating
lastIdx := len(r.processors) - 1
if idx < lastIdx {
r.processors[idx] = r.processors[lastIdx]
}
r.processors = r.processors[:lastIdx]
}
}

View File

@ -0,0 +1,87 @@
package replication
import (
"io"
)
// StorageSnapshot provides an interface for taking a snapshot of the storage
// for replication bootstrap purposes.
type StorageSnapshot interface {
// CreateSnapshotIterator creates an iterator for a storage snapshot
CreateSnapshotIterator() (SnapshotIterator, error)
// KeyCount returns the approximate number of keys in storage
KeyCount() int64
}
// SnapshotIterator provides iteration over key-value pairs in storage
type SnapshotIterator interface {
// Next returns the next key-value pair
// Returns io.EOF when there are no more items
Next() (key []byte, value []byte, err error)
// Close closes the iterator
Close() error
}
// StorageSnapshotProvider is implemented by storage engines that support snapshots
type StorageSnapshotProvider interface {
// CreateSnapshot creates a snapshot of the current storage state
CreateSnapshot() (StorageSnapshot, error)
}
// MemoryStorageSnapshot is a simple in-memory implementation of StorageSnapshot
// Useful for testing or small datasets
type MemoryStorageSnapshot struct {
Pairs []KeyValuePair
position int
}
// KeyValuePair represents a key-value pair in storage
type KeyValuePair struct {
Key []byte
Value []byte
}
// CreateSnapshotIterator creates an iterator for a memory storage snapshot
func (m *MemoryStorageSnapshot) CreateSnapshotIterator() (SnapshotIterator, error) {
return &MemorySnapshotIterator{
snapshot: m,
position: 0,
}, nil
}
// KeyCount returns the number of keys in the snapshot
func (m *MemoryStorageSnapshot) KeyCount() int64 {
return int64(len(m.Pairs))
}
// MemorySnapshotIterator is an iterator for MemoryStorageSnapshot
type MemorySnapshotIterator struct {
snapshot *MemoryStorageSnapshot
position int
}
// Next returns the next key-value pair
func (it *MemorySnapshotIterator) Next() ([]byte, []byte, error) {
if it.position >= len(it.snapshot.Pairs) {
return nil, nil, io.EOF
}
pair := it.snapshot.Pairs[it.position]
it.position++
return pair.Key, pair.Value, nil
}
// Close closes the iterator
func (it *MemorySnapshotIterator) Close() error {
return nil
}
// NewMemoryStorageSnapshot creates a new in-memory storage snapshot
func NewMemoryStorageSnapshot(pairs []KeyValuePair) *MemoryStorageSnapshot {
return &MemoryStorageSnapshot{
Pairs: pairs,
}
}

View File

@ -0,0 +1,401 @@
package transport
import (
"context"
"testing"
"time"
"github.com/KevoDB/kevo/pkg/wal"
)
// MockReplicationClient implements ReplicationClient for testing
type MockReplicationClient struct {
connected bool
registeredAsReplica bool
heartbeatSent bool
walEntriesRequested bool
bootstrapRequested bool
replicaID string
walEntries []*wal.Entry
bootstrapIterator BootstrapIterator
status TransportStatus
}
func NewMockReplicationClient() *MockReplicationClient {
return &MockReplicationClient{
connected: false,
registeredAsReplica: false,
heartbeatSent: false,
walEntriesRequested: false,
bootstrapRequested: false,
status: TransportStatus{
Connected: false,
LastConnected: time.Time{},
LastError: nil,
BytesSent: 0,
BytesReceived: 0,
RTT: 0,
},
}
}
func (c *MockReplicationClient) Connect(ctx context.Context) error {
c.connected = true
c.status.Connected = true
c.status.LastConnected = time.Now()
return nil
}
func (c *MockReplicationClient) Close() error {
c.connected = false
c.status.Connected = false
return nil
}
func (c *MockReplicationClient) IsConnected() bool {
return c.connected
}
func (c *MockReplicationClient) Status() TransportStatus {
return c.status
}
func (c *MockReplicationClient) Send(ctx context.Context, request Request) (Response, error) {
return nil, ErrInvalidRequest
}
func (c *MockReplicationClient) Stream(ctx context.Context) (Stream, error) {
return nil, ErrInvalidRequest
}
func (c *MockReplicationClient) RegisterAsReplica(ctx context.Context, replicaID string) error {
c.registeredAsReplica = true
c.replicaID = replicaID
return nil
}
func (c *MockReplicationClient) SendHeartbeat(ctx context.Context, status *ReplicaInfo) error {
c.heartbeatSent = true
return nil
}
func (c *MockReplicationClient) RequestWALEntries(ctx context.Context, fromLSN uint64) ([]*wal.Entry, error) {
c.walEntriesRequested = true
return c.walEntries, nil
}
func (c *MockReplicationClient) RequestBootstrap(ctx context.Context) (BootstrapIterator, error) {
c.bootstrapRequested = true
return c.bootstrapIterator, nil
}
// MockBootstrapIterator implements BootstrapIterator for testing
type MockBootstrapIterator struct {
pairs []struct{ key, value []byte }
position int
progress float64
closed bool
}
func NewMockBootstrapIterator() *MockBootstrapIterator {
return &MockBootstrapIterator{
pairs: []struct{ key, value []byte }{
{[]byte("key1"), []byte("value1")},
{[]byte("key2"), []byte("value2")},
{[]byte("key3"), []byte("value3")},
},
position: 0,
progress: 0.0,
closed: false,
}
}
func (it *MockBootstrapIterator) Next() ([]byte, []byte, error) {
if it.position >= len(it.pairs) {
return nil, nil, nil
}
pair := it.pairs[it.position]
it.position++
it.progress = float64(it.position) / float64(len(it.pairs))
return pair.key, pair.value, nil
}
func (it *MockBootstrapIterator) Close() error {
it.closed = true
return nil
}
func (it *MockBootstrapIterator) Progress() float64 {
return it.progress
}
// Tests
func TestReplicationClientInterface(t *testing.T) {
// Create a mock client
client := NewMockReplicationClient()
// Test Connect
ctx := context.Background()
err := client.Connect(ctx)
if err != nil {
t.Errorf("Connect failed: %v", err)
}
// Test IsConnected
if !client.IsConnected() {
t.Errorf("Expected client to be connected")
}
// Test Status
status := client.Status()
if !status.Connected {
t.Errorf("Expected status.Connected to be true")
}
// Test RegisterAsReplica
err = client.RegisterAsReplica(ctx, "replica1")
if err != nil {
t.Errorf("RegisterAsReplica failed: %v", err)
}
if !client.registeredAsReplica {
t.Errorf("Expected client to be registered as replica")
}
if client.replicaID != "replica1" {
t.Errorf("Expected replicaID to be 'replica1', got '%s'", client.replicaID)
}
// Test SendHeartbeat
replicaInfo := &ReplicaInfo{
ID: "replica1",
Address: "localhost:50051",
Role: RoleReplica,
Status: StatusReady,
LastSeen: time.Now(),
CurrentLSN: 100,
ReplicationLag: 0,
}
err = client.SendHeartbeat(ctx, replicaInfo)
if err != nil {
t.Errorf("SendHeartbeat failed: %v", err)
}
if !client.heartbeatSent {
t.Errorf("Expected heartbeat to be sent")
}
// Test RequestWALEntries
client.walEntries = []*wal.Entry{
{SequenceNumber: 101, Type: 1, Key: []byte("key1"), Value: []byte("value1")},
{SequenceNumber: 102, Type: 1, Key: []byte("key2"), Value: []byte("value2")},
}
entries, err := client.RequestWALEntries(ctx, 100)
if err != nil {
t.Errorf("RequestWALEntries failed: %v", err)
}
if !client.walEntriesRequested {
t.Errorf("Expected WAL entries to be requested")
}
if len(entries) != 2 {
t.Errorf("Expected 2 entries, got %d", len(entries))
}
// Test RequestBootstrap
client.bootstrapIterator = NewMockBootstrapIterator()
iterator, err := client.RequestBootstrap(ctx)
if err != nil {
t.Errorf("RequestBootstrap failed: %v", err)
}
if !client.bootstrapRequested {
t.Errorf("Expected bootstrap to be requested")
}
// Test iterator
key, value, err := iterator.Next()
if err != nil {
t.Errorf("Iterator.Next failed: %v", err)
}
if string(key) != "key1" || string(value) != "value1" {
t.Errorf("Expected key1/value1, got %s/%s", string(key), string(value))
}
progress := iterator.Progress()
if progress != 1.0/3.0 {
t.Errorf("Expected progress to be 1/3, got %f", progress)
}
// Test Close
err = client.Close()
if err != nil {
t.Errorf("Close failed: %v", err)
}
if client.IsConnected() {
t.Errorf("Expected client to be disconnected")
}
// Test iterator Close
err = iterator.Close()
if err != nil {
t.Errorf("Iterator.Close failed: %v", err)
}
mockIter := iterator.(*MockBootstrapIterator)
if !mockIter.closed {
t.Errorf("Expected iterator to be closed")
}
}
// MockReplicationServer implements ReplicationServer for testing
type MockReplicationServer struct {
started bool
stopped bool
replicas map[string]*ReplicaInfo
streamingReplicas map[string]bool
}
func NewMockReplicationServer() *MockReplicationServer {
return &MockReplicationServer{
started: false,
stopped: false,
replicas: make(map[string]*ReplicaInfo),
streamingReplicas: make(map[string]bool),
}
}
func (s *MockReplicationServer) Start() error {
s.started = true
return nil
}
func (s *MockReplicationServer) Serve() error {
s.started = true
return nil
}
func (s *MockReplicationServer) Stop(ctx context.Context) error {
s.stopped = true
return nil
}
func (s *MockReplicationServer) SetRequestHandler(handler RequestHandler) {
// No-op for testing
}
func (s *MockReplicationServer) RegisterReplica(replicaInfo *ReplicaInfo) error {
s.replicas[replicaInfo.ID] = replicaInfo
return nil
}
func (s *MockReplicationServer) UpdateReplicaStatus(replicaID string, status ReplicaStatus, lsn uint64) error {
replica, exists := s.replicas[replicaID]
if !exists {
return ErrInvalidRequest
}
replica.Status = status
replica.CurrentLSN = lsn
return nil
}
func (s *MockReplicationServer) GetReplicaInfo(replicaID string) (*ReplicaInfo, error) {
replica, exists := s.replicas[replicaID]
if !exists {
return nil, ErrInvalidRequest
}
return replica, nil
}
func (s *MockReplicationServer) ListReplicas() ([]*ReplicaInfo, error) {
result := make([]*ReplicaInfo, 0, len(s.replicas))
for _, replica := range s.replicas {
result = append(result, replica)
}
return result, nil
}
func (s *MockReplicationServer) StreamWALEntriesToReplica(ctx context.Context, replicaID string, fromLSN uint64) error {
_, exists := s.replicas[replicaID]
if !exists {
return ErrInvalidRequest
}
s.streamingReplicas[replicaID] = true
return nil
}
func TestReplicationServerInterface(t *testing.T) {
// Create a mock server
server := NewMockReplicationServer()
// Test Start
err := server.Start()
if err != nil {
t.Errorf("Start failed: %v", err)
}
if !server.started {
t.Errorf("Expected server to be started")
}
// Test RegisterReplica
replica1 := &ReplicaInfo{
ID: "replica1",
Address: "localhost:50051",
Role: RoleReplica,
Status: StatusConnecting,
LastSeen: time.Now(),
CurrentLSN: 0,
ReplicationLag: 0,
}
err = server.RegisterReplica(replica1)
if err != nil {
t.Errorf("RegisterReplica failed: %v", err)
}
// Test UpdateReplicaStatus
err = server.UpdateReplicaStatus("replica1", StatusReady, 100)
if err != nil {
t.Errorf("UpdateReplicaStatus failed: %v", err)
}
// Test GetReplicaInfo
replica, err := server.GetReplicaInfo("replica1")
if err != nil {
t.Errorf("GetReplicaInfo failed: %v", err)
}
if replica.Status != StatusReady {
t.Errorf("Expected status to be StatusReady, got %v", replica.Status)
}
if replica.CurrentLSN != 100 {
t.Errorf("Expected LSN to be 100, got %d", replica.CurrentLSN)
}
// Test ListReplicas
replicas, err := server.ListReplicas()
if err != nil {
t.Errorf("ListReplicas failed: %v", err)
}
if len(replicas) != 1 {
t.Errorf("Expected 1 replica, got %d", len(replicas))
}
// Test StreamWALEntriesToReplica
ctx := context.Background()
err = server.StreamWALEntriesToReplica(ctx, "replica1", 0)
if err != nil {
t.Errorf("StreamWALEntriesToReplica failed: %v", err)
}
if !server.streamingReplicas["replica1"] {
t.Errorf("Expected replica1 to be streaming")
}
// Test Stop
err = server.Stop(ctx)
if err != nil {
t.Errorf("Stop failed: %v", err)
}
if !server.stopped {
t.Errorf("Expected server to be stopped")
}
}

1464
proto/kevo/replication.pb.go Normal file

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,400 @@
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
// versions:
// - protoc-gen-go-grpc v1.5.1
// - protoc v3.20.3
// source: proto/kevo/replication.proto
package kevo
import (
context "context"
grpc "google.golang.org/grpc"
codes "google.golang.org/grpc/codes"
status "google.golang.org/grpc/status"
)
// This is a compile-time assertion to ensure that this generated file
// is compatible with the grpc package it is being compiled against.
// Requires gRPC-Go v1.64.0 or later.
const _ = grpc.SupportPackageIsVersion9
const (
ReplicationService_RegisterReplica_FullMethodName = "/kevo.ReplicationService/RegisterReplica"
ReplicationService_ReplicaHeartbeat_FullMethodName = "/kevo.ReplicationService/ReplicaHeartbeat"
ReplicationService_GetReplicaStatus_FullMethodName = "/kevo.ReplicationService/GetReplicaStatus"
ReplicationService_ListReplicas_FullMethodName = "/kevo.ReplicationService/ListReplicas"
ReplicationService_GetWALEntries_FullMethodName = "/kevo.ReplicationService/GetWALEntries"
ReplicationService_StreamWALEntries_FullMethodName = "/kevo.ReplicationService/StreamWALEntries"
ReplicationService_ReportAppliedEntries_FullMethodName = "/kevo.ReplicationService/ReportAppliedEntries"
ReplicationService_RequestBootstrap_FullMethodName = "/kevo.ReplicationService/RequestBootstrap"
)
// ReplicationServiceClient is the client API for ReplicationService service.
//
// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream.
type ReplicationServiceClient interface {
// Replica Registration and Status
RegisterReplica(ctx context.Context, in *RegisterReplicaRequest, opts ...grpc.CallOption) (*RegisterReplicaResponse, error)
ReplicaHeartbeat(ctx context.Context, in *ReplicaHeartbeatRequest, opts ...grpc.CallOption) (*ReplicaHeartbeatResponse, error)
GetReplicaStatus(ctx context.Context, in *GetReplicaStatusRequest, opts ...grpc.CallOption) (*GetReplicaStatusResponse, error)
ListReplicas(ctx context.Context, in *ListReplicasRequest, opts ...grpc.CallOption) (*ListReplicasResponse, error)
// WAL Replication
GetWALEntries(ctx context.Context, in *GetWALEntriesRequest, opts ...grpc.CallOption) (*GetWALEntriesResponse, error)
StreamWALEntries(ctx context.Context, in *StreamWALEntriesRequest, opts ...grpc.CallOption) (grpc.ServerStreamingClient[WALEntryBatch], error)
ReportAppliedEntries(ctx context.Context, in *ReportAppliedEntriesRequest, opts ...grpc.CallOption) (*ReportAppliedEntriesResponse, error)
// Bootstrap Operations
RequestBootstrap(ctx context.Context, in *BootstrapRequest, opts ...grpc.CallOption) (grpc.ServerStreamingClient[BootstrapBatch], error)
}
type replicationServiceClient struct {
cc grpc.ClientConnInterface
}
func NewReplicationServiceClient(cc grpc.ClientConnInterface) ReplicationServiceClient {
return &replicationServiceClient{cc}
}
func (c *replicationServiceClient) RegisterReplica(ctx context.Context, in *RegisterReplicaRequest, opts ...grpc.CallOption) (*RegisterReplicaResponse, error) {
cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...)
out := new(RegisterReplicaResponse)
err := c.cc.Invoke(ctx, ReplicationService_RegisterReplica_FullMethodName, in, out, cOpts...)
if err != nil {
return nil, err
}
return out, nil
}
func (c *replicationServiceClient) ReplicaHeartbeat(ctx context.Context, in *ReplicaHeartbeatRequest, opts ...grpc.CallOption) (*ReplicaHeartbeatResponse, error) {
cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...)
out := new(ReplicaHeartbeatResponse)
err := c.cc.Invoke(ctx, ReplicationService_ReplicaHeartbeat_FullMethodName, in, out, cOpts...)
if err != nil {
return nil, err
}
return out, nil
}
func (c *replicationServiceClient) GetReplicaStatus(ctx context.Context, in *GetReplicaStatusRequest, opts ...grpc.CallOption) (*GetReplicaStatusResponse, error) {
cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...)
out := new(GetReplicaStatusResponse)
err := c.cc.Invoke(ctx, ReplicationService_GetReplicaStatus_FullMethodName, in, out, cOpts...)
if err != nil {
return nil, err
}
return out, nil
}
func (c *replicationServiceClient) ListReplicas(ctx context.Context, in *ListReplicasRequest, opts ...grpc.CallOption) (*ListReplicasResponse, error) {
cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...)
out := new(ListReplicasResponse)
err := c.cc.Invoke(ctx, ReplicationService_ListReplicas_FullMethodName, in, out, cOpts...)
if err != nil {
return nil, err
}
return out, nil
}
func (c *replicationServiceClient) GetWALEntries(ctx context.Context, in *GetWALEntriesRequest, opts ...grpc.CallOption) (*GetWALEntriesResponse, error) {
cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...)
out := new(GetWALEntriesResponse)
err := c.cc.Invoke(ctx, ReplicationService_GetWALEntries_FullMethodName, in, out, cOpts...)
if err != nil {
return nil, err
}
return out, nil
}
func (c *replicationServiceClient) StreamWALEntries(ctx context.Context, in *StreamWALEntriesRequest, opts ...grpc.CallOption) (grpc.ServerStreamingClient[WALEntryBatch], error) {
cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...)
stream, err := c.cc.NewStream(ctx, &ReplicationService_ServiceDesc.Streams[0], ReplicationService_StreamWALEntries_FullMethodName, cOpts...)
if err != nil {
return nil, err
}
x := &grpc.GenericClientStream[StreamWALEntriesRequest, WALEntryBatch]{ClientStream: stream}
if err := x.ClientStream.SendMsg(in); err != nil {
return nil, err
}
if err := x.ClientStream.CloseSend(); err != nil {
return nil, err
}
return x, nil
}
// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name.
type ReplicationService_StreamWALEntriesClient = grpc.ServerStreamingClient[WALEntryBatch]
func (c *replicationServiceClient) ReportAppliedEntries(ctx context.Context, in *ReportAppliedEntriesRequest, opts ...grpc.CallOption) (*ReportAppliedEntriesResponse, error) {
cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...)
out := new(ReportAppliedEntriesResponse)
err := c.cc.Invoke(ctx, ReplicationService_ReportAppliedEntries_FullMethodName, in, out, cOpts...)
if err != nil {
return nil, err
}
return out, nil
}
func (c *replicationServiceClient) RequestBootstrap(ctx context.Context, in *BootstrapRequest, opts ...grpc.CallOption) (grpc.ServerStreamingClient[BootstrapBatch], error) {
cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...)
stream, err := c.cc.NewStream(ctx, &ReplicationService_ServiceDesc.Streams[1], ReplicationService_RequestBootstrap_FullMethodName, cOpts...)
if err != nil {
return nil, err
}
x := &grpc.GenericClientStream[BootstrapRequest, BootstrapBatch]{ClientStream: stream}
if err := x.ClientStream.SendMsg(in); err != nil {
return nil, err
}
if err := x.ClientStream.CloseSend(); err != nil {
return nil, err
}
return x, nil
}
// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name.
type ReplicationService_RequestBootstrapClient = grpc.ServerStreamingClient[BootstrapBatch]
// ReplicationServiceServer is the server API for ReplicationService service.
// All implementations must embed UnimplementedReplicationServiceServer
// for forward compatibility.
type ReplicationServiceServer interface {
// Replica Registration and Status
RegisterReplica(context.Context, *RegisterReplicaRequest) (*RegisterReplicaResponse, error)
ReplicaHeartbeat(context.Context, *ReplicaHeartbeatRequest) (*ReplicaHeartbeatResponse, error)
GetReplicaStatus(context.Context, *GetReplicaStatusRequest) (*GetReplicaStatusResponse, error)
ListReplicas(context.Context, *ListReplicasRequest) (*ListReplicasResponse, error)
// WAL Replication
GetWALEntries(context.Context, *GetWALEntriesRequest) (*GetWALEntriesResponse, error)
StreamWALEntries(*StreamWALEntriesRequest, grpc.ServerStreamingServer[WALEntryBatch]) error
ReportAppliedEntries(context.Context, *ReportAppliedEntriesRequest) (*ReportAppliedEntriesResponse, error)
// Bootstrap Operations
RequestBootstrap(*BootstrapRequest, grpc.ServerStreamingServer[BootstrapBatch]) error
mustEmbedUnimplementedReplicationServiceServer()
}
// UnimplementedReplicationServiceServer must be embedded to have
// forward compatible implementations.
//
// NOTE: this should be embedded by value instead of pointer to avoid a nil
// pointer dereference when methods are called.
type UnimplementedReplicationServiceServer struct{}
func (UnimplementedReplicationServiceServer) RegisterReplica(context.Context, *RegisterReplicaRequest) (*RegisterReplicaResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method RegisterReplica not implemented")
}
func (UnimplementedReplicationServiceServer) ReplicaHeartbeat(context.Context, *ReplicaHeartbeatRequest) (*ReplicaHeartbeatResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method ReplicaHeartbeat not implemented")
}
func (UnimplementedReplicationServiceServer) GetReplicaStatus(context.Context, *GetReplicaStatusRequest) (*GetReplicaStatusResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method GetReplicaStatus not implemented")
}
func (UnimplementedReplicationServiceServer) ListReplicas(context.Context, *ListReplicasRequest) (*ListReplicasResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method ListReplicas not implemented")
}
func (UnimplementedReplicationServiceServer) GetWALEntries(context.Context, *GetWALEntriesRequest) (*GetWALEntriesResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method GetWALEntries not implemented")
}
func (UnimplementedReplicationServiceServer) StreamWALEntries(*StreamWALEntriesRequest, grpc.ServerStreamingServer[WALEntryBatch]) error {
return status.Errorf(codes.Unimplemented, "method StreamWALEntries not implemented")
}
func (UnimplementedReplicationServiceServer) ReportAppliedEntries(context.Context, *ReportAppliedEntriesRequest) (*ReportAppliedEntriesResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method ReportAppliedEntries not implemented")
}
func (UnimplementedReplicationServiceServer) RequestBootstrap(*BootstrapRequest, grpc.ServerStreamingServer[BootstrapBatch]) error {
return status.Errorf(codes.Unimplemented, "method RequestBootstrap not implemented")
}
func (UnimplementedReplicationServiceServer) mustEmbedUnimplementedReplicationServiceServer() {}
func (UnimplementedReplicationServiceServer) testEmbeddedByValue() {}
// UnsafeReplicationServiceServer may be embedded to opt out of forward compatibility for this service.
// Use of this interface is not recommended, as added methods to ReplicationServiceServer will
// result in compilation errors.
type UnsafeReplicationServiceServer interface {
mustEmbedUnimplementedReplicationServiceServer()
}
func RegisterReplicationServiceServer(s grpc.ServiceRegistrar, srv ReplicationServiceServer) {
// If the following call pancis, it indicates UnimplementedReplicationServiceServer was
// embedded by pointer and is nil. This will cause panics if an
// unimplemented method is ever invoked, so we test this at initialization
// time to prevent it from happening at runtime later due to I/O.
if t, ok := srv.(interface{ testEmbeddedByValue() }); ok {
t.testEmbeddedByValue()
}
s.RegisterService(&ReplicationService_ServiceDesc, srv)
}
func _ReplicationService_RegisterReplica_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(RegisterReplicaRequest)
if err := dec(in); err != nil {
return nil, err
}
if interceptor == nil {
return srv.(ReplicationServiceServer).RegisterReplica(ctx, in)
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: ReplicationService_RegisterReplica_FullMethodName,
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(ReplicationServiceServer).RegisterReplica(ctx, req.(*RegisterReplicaRequest))
}
return interceptor(ctx, in, info, handler)
}
func _ReplicationService_ReplicaHeartbeat_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(ReplicaHeartbeatRequest)
if err := dec(in); err != nil {
return nil, err
}
if interceptor == nil {
return srv.(ReplicationServiceServer).ReplicaHeartbeat(ctx, in)
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: ReplicationService_ReplicaHeartbeat_FullMethodName,
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(ReplicationServiceServer).ReplicaHeartbeat(ctx, req.(*ReplicaHeartbeatRequest))
}
return interceptor(ctx, in, info, handler)
}
func _ReplicationService_GetReplicaStatus_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(GetReplicaStatusRequest)
if err := dec(in); err != nil {
return nil, err
}
if interceptor == nil {
return srv.(ReplicationServiceServer).GetReplicaStatus(ctx, in)
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: ReplicationService_GetReplicaStatus_FullMethodName,
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(ReplicationServiceServer).GetReplicaStatus(ctx, req.(*GetReplicaStatusRequest))
}
return interceptor(ctx, in, info, handler)
}
func _ReplicationService_ListReplicas_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(ListReplicasRequest)
if err := dec(in); err != nil {
return nil, err
}
if interceptor == nil {
return srv.(ReplicationServiceServer).ListReplicas(ctx, in)
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: ReplicationService_ListReplicas_FullMethodName,
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(ReplicationServiceServer).ListReplicas(ctx, req.(*ListReplicasRequest))
}
return interceptor(ctx, in, info, handler)
}
func _ReplicationService_GetWALEntries_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(GetWALEntriesRequest)
if err := dec(in); err != nil {
return nil, err
}
if interceptor == nil {
return srv.(ReplicationServiceServer).GetWALEntries(ctx, in)
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: ReplicationService_GetWALEntries_FullMethodName,
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(ReplicationServiceServer).GetWALEntries(ctx, req.(*GetWALEntriesRequest))
}
return interceptor(ctx, in, info, handler)
}
func _ReplicationService_StreamWALEntries_Handler(srv interface{}, stream grpc.ServerStream) error {
m := new(StreamWALEntriesRequest)
if err := stream.RecvMsg(m); err != nil {
return err
}
return srv.(ReplicationServiceServer).StreamWALEntries(m, &grpc.GenericServerStream[StreamWALEntriesRequest, WALEntryBatch]{ServerStream: stream})
}
// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name.
type ReplicationService_StreamWALEntriesServer = grpc.ServerStreamingServer[WALEntryBatch]
func _ReplicationService_ReportAppliedEntries_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(ReportAppliedEntriesRequest)
if err := dec(in); err != nil {
return nil, err
}
if interceptor == nil {
return srv.(ReplicationServiceServer).ReportAppliedEntries(ctx, in)
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: ReplicationService_ReportAppliedEntries_FullMethodName,
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(ReplicationServiceServer).ReportAppliedEntries(ctx, req.(*ReportAppliedEntriesRequest))
}
return interceptor(ctx, in, info, handler)
}
func _ReplicationService_RequestBootstrap_Handler(srv interface{}, stream grpc.ServerStream) error {
m := new(BootstrapRequest)
if err := stream.RecvMsg(m); err != nil {
return err
}
return srv.(ReplicationServiceServer).RequestBootstrap(m, &grpc.GenericServerStream[BootstrapRequest, BootstrapBatch]{ServerStream: stream})
}
// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name.
type ReplicationService_RequestBootstrapServer = grpc.ServerStreamingServer[BootstrapBatch]
// ReplicationService_ServiceDesc is the grpc.ServiceDesc for ReplicationService service.
// It's only intended for direct use with grpc.RegisterService,
// and not to be introspected or modified (even as a copy)
var ReplicationService_ServiceDesc = grpc.ServiceDesc{
ServiceName: "kevo.ReplicationService",
HandlerType: (*ReplicationServiceServer)(nil),
Methods: []grpc.MethodDesc{
{
MethodName: "RegisterReplica",
Handler: _ReplicationService_RegisterReplica_Handler,
},
{
MethodName: "ReplicaHeartbeat",
Handler: _ReplicationService_ReplicaHeartbeat_Handler,
},
{
MethodName: "GetReplicaStatus",
Handler: _ReplicationService_GetReplicaStatus_Handler,
},
{
MethodName: "ListReplicas",
Handler: _ReplicationService_ListReplicas_Handler,
},
{
MethodName: "GetWALEntries",
Handler: _ReplicationService_GetWALEntries_Handler,
},
{
MethodName: "ReportAppliedEntries",
Handler: _ReplicationService_ReportAppliedEntries_Handler,
},
},
Streams: []grpc.StreamDesc{
{
StreamName: "StreamWALEntries",
Handler: _ReplicationService_StreamWALEntries_Handler,
ServerStreams: true,
},
{
StreamName: "RequestBootstrap",
Handler: _ReplicationService_RequestBootstrap_Handler,
ServerStreams: true,
},
},
Metadata: "proto/kevo/replication.proto",
}

View File

@ -4,7 +4,7 @@
// protoc v3.20.3
// source: proto/kevo/service.proto
package proto
package kevo
import (
protoreflect "google.golang.org/protobuf/reflect/protoreflect"
@ -1911,7 +1911,7 @@ const file_proto_kevo_service_proto_rawDesc = "" +
"\bTxDelete\x12\x15.kevo.TxDeleteRequest\x1a\x16.kevo.TxDeleteResponse\x125\n" +
"\x06TxScan\x12\x13.kevo.TxScanRequest\x1a\x14.kevo.TxScanResponse0\x01\x129\n" +
"\bGetStats\x12\x15.kevo.GetStatsRequest\x1a\x16.kevo.GetStatsResponse\x126\n" +
"\aCompact\x12\x14.kevo.CompactRequest\x1a\x15.kevo.CompactResponseB5Z3github.com/jeremytregunna/kevo/pkg/grpc/proto;protob\x06proto3"
"\aCompact\x12\x14.kevo.CompactRequest\x1a\x15.kevo.CompactResponseB#Z!github.com/KevoDB/kevo/proto/kevob\x06proto3"
var (
file_proto_kevo_service_proto_rawDescOnce sync.Once

View File

@ -2,7 +2,7 @@ syntax = "proto3";
package kevo;
option go_package = "github.com/jeremytregunna/kevo/pkg/grpc/proto;proto";
option go_package = "github.com/KevoDB/kevo/proto/kevo";
service KevoService {
// Key-Value Operations

View File

@ -4,7 +4,7 @@
// - protoc v3.20.3
// source: proto/kevo/service.proto
package proto
package kevo
import (
context "context"