feat: Add replication manager to manage primary/replica
- Primary nodes will connect to the WAL for observations, start a gRPC server for replication, and shutdown properly - Replica nodes will connect to the primary, apply received entries to local storage, and enforce read-only mode for consistency - Integrates the decision primary/replica/standalone into the kevo cli
This commit is contained in:
parent
83163db067
commit
4429836929
@ -92,6 +92,12 @@ type Config struct {
|
||||
TLSCertFile string
|
||||
TLSKeyFile string
|
||||
TLSCAFile string
|
||||
|
||||
// Replication settings
|
||||
ReplicationEnabled bool
|
||||
ReplicationMode string // "primary", "replica", or "standalone"
|
||||
ReplicationAddr string // Address for replication service
|
||||
PrimaryAddr string // Address of primary (for replicas)
|
||||
}
|
||||
|
||||
func main() {
|
||||
@ -162,6 +168,12 @@ func parseFlags() Config {
|
||||
tlsKeyFile := flag.String("key", "", "TLS private key file path")
|
||||
tlsCAFile := flag.String("ca", "", "TLS CA certificate file for client verification")
|
||||
|
||||
// Replication options
|
||||
replicationEnabled := flag.Bool("replication", false, "Enable replication")
|
||||
replicationMode := flag.String("replication-mode", "standalone", "Replication mode: primary, replica, or standalone")
|
||||
replicationAddr := flag.String("replication-address", "localhost:50052", "Address for replication service")
|
||||
primaryAddr := flag.String("primary", "localhost:50052", "Address of primary node (for replicas)")
|
||||
|
||||
// Parse flags
|
||||
flag.Parse()
|
||||
|
||||
@ -180,6 +192,12 @@ func parseFlags() Config {
|
||||
TLSCertFile: *tlsCertFile,
|
||||
TLSKeyFile: *tlsKeyFile,
|
||||
TLSCAFile: *tlsCAFile,
|
||||
|
||||
// Replication settings
|
||||
ReplicationEnabled: *replicationEnabled,
|
||||
ReplicationMode: *replicationMode,
|
||||
ReplicationAddr: *replicationAddr,
|
||||
PrimaryAddr: *primaryAddr,
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -10,6 +10,7 @@ import (
|
||||
"github.com/KevoDB/kevo/pkg/engine/interfaces"
|
||||
"github.com/KevoDB/kevo/pkg/engine/transaction"
|
||||
grpcservice "github.com/KevoDB/kevo/pkg/grpc/service"
|
||||
"github.com/KevoDB/kevo/pkg/replication"
|
||||
pb "github.com/KevoDB/kevo/proto/kevo"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/credentials"
|
||||
@ -18,12 +19,13 @@ import (
|
||||
|
||||
// Server represents the Kevo server
|
||||
type Server struct {
|
||||
eng interfaces.Engine
|
||||
txRegistry interfaces.TxRegistry
|
||||
listener net.Listener
|
||||
grpcServer *grpc.Server
|
||||
kevoService *grpcservice.KevoServiceServer
|
||||
config Config
|
||||
eng interfaces.Engine
|
||||
txRegistry interfaces.TxRegistry
|
||||
listener net.Listener
|
||||
grpcServer *grpc.Server
|
||||
kevoService *grpcservice.KevoServiceServer
|
||||
config Config
|
||||
replicationManager *replication.Manager
|
||||
}
|
||||
|
||||
// NewServer creates a new server instance
|
||||
@ -50,8 +52,9 @@ func (s *Server) Start() error {
|
||||
var serverOpts []grpc.ServerOption
|
||||
|
||||
// Add TLS if configured
|
||||
var tlsConfig *tls.Config
|
||||
if s.config.TLSEnabled {
|
||||
tlsConfig := &tls.Config{
|
||||
tlsConfig = &tls.Config{
|
||||
MinVersion: tls.VersionTLS12,
|
||||
}
|
||||
|
||||
@ -94,6 +97,37 @@ func (s *Server) Start() error {
|
||||
s.kevoService = grpcservice.NewKevoServiceServer(s.eng, s.txRegistry)
|
||||
pb.RegisterKevoServiceServer(s.grpcServer, s.kevoService)
|
||||
|
||||
// Initialize replication if enabled
|
||||
if s.config.ReplicationEnabled {
|
||||
// Create replication manager config
|
||||
replicationConfig := &replication.ManagerConfig{
|
||||
Enabled: true,
|
||||
Mode: s.config.ReplicationMode,
|
||||
PrimaryAddr: s.config.PrimaryAddr,
|
||||
ListenAddr: s.config.ReplicationAddr,
|
||||
TLSConfig: tlsConfig,
|
||||
ForceReadOnly: true,
|
||||
}
|
||||
|
||||
// Create the replication manager
|
||||
s.replicationManager, err = replication.NewManager(s.eng, replicationConfig)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to create replication manager: %w", err)
|
||||
}
|
||||
|
||||
// Start the replication service
|
||||
if err := s.replicationManager.Start(); err != nil {
|
||||
return fmt.Errorf("failed to start replication: %w", err)
|
||||
}
|
||||
|
||||
fmt.Printf("Replication started in %s mode\n", s.config.ReplicationMode)
|
||||
|
||||
// If in replica mode, the engine should now be read-only
|
||||
if s.config.ReplicationMode == "replica" {
|
||||
fmt.Println("Running as replica: database is in read-only mode")
|
||||
}
|
||||
}
|
||||
|
||||
fmt.Println("gRPC server initialized")
|
||||
return nil
|
||||
}
|
||||
@ -110,7 +144,17 @@ func (s *Server) Serve() error {
|
||||
|
||||
// Shutdown gracefully shuts down the server
|
||||
func (s *Server) Shutdown(ctx context.Context) error {
|
||||
// First, gracefully stop the gRPC server if it exists
|
||||
// First, stop the replication manager if it exists
|
||||
if s.replicationManager != nil {
|
||||
fmt.Println("Stopping replication manager...")
|
||||
if err := s.replicationManager.Stop(); err != nil {
|
||||
fmt.Printf("Warning: Failed to stop replication manager: %v\n", err)
|
||||
} else {
|
||||
fmt.Println("Replication manager stopped")
|
||||
}
|
||||
}
|
||||
|
||||
// Next, gracefully stop the gRPC server if it exists
|
||||
if s.grpcServer != nil {
|
||||
fmt.Println("Gracefully stopping gRPC server...")
|
||||
|
||||
|
@ -7,4 +7,6 @@ var (
|
||||
ErrEngineClosed = errors.New("engine is closed")
|
||||
// ErrKeyNotFound is returned when a key is not found
|
||||
ErrKeyNotFound = errors.New("key not found")
|
||||
// ErrReadOnlyMode is returned when write operations are attempted while the engine is in read-only mode
|
||||
ErrReadOnlyMode = errors.New("engine is in read-only mode (replica)")
|
||||
)
|
||||
|
@ -35,7 +35,8 @@ type EngineFacade struct {
|
||||
stats stats.Collector
|
||||
|
||||
// State
|
||||
closed atomic.Bool
|
||||
closed atomic.Bool
|
||||
readOnly atomic.Bool // Flag to indicate if the engine is in read-only mode (for replicas)
|
||||
}
|
||||
|
||||
// We keep the Engine name used in legacy code, but redirect it to our new implementation
|
||||
@ -115,6 +116,11 @@ func (e *EngineFacade) Put(key, value []byte) error {
|
||||
return ErrEngineClosed
|
||||
}
|
||||
|
||||
// Reject writes in read-only mode
|
||||
if e.readOnly.Load() {
|
||||
return ErrReadOnlyMode
|
||||
}
|
||||
|
||||
// Track the operation start
|
||||
e.stats.TrackOperation(stats.OpPut)
|
||||
|
||||
@ -173,6 +179,11 @@ func (e *EngineFacade) Delete(key []byte) error {
|
||||
return ErrEngineClosed
|
||||
}
|
||||
|
||||
// Reject writes in read-only mode
|
||||
if e.readOnly.Load() {
|
||||
return ErrReadOnlyMode
|
||||
}
|
||||
|
||||
// Track the operation start
|
||||
e.stats.TrackOperation(stats.OpDelete)
|
||||
|
||||
@ -264,6 +275,11 @@ func (e *EngineFacade) BeginTransaction(readOnly bool) (interfaces.Transaction,
|
||||
return nil, ErrEngineClosed
|
||||
}
|
||||
|
||||
// Force read-only mode if engine is in read-only mode
|
||||
if e.readOnly.Load() {
|
||||
readOnly = true
|
||||
}
|
||||
|
||||
// Track the operation start
|
||||
e.stats.TrackOperation(stats.OpTxBegin)
|
||||
|
||||
@ -299,6 +315,11 @@ func (e *EngineFacade) ApplyBatch(entries []*wal.Entry) error {
|
||||
return ErrEngineClosed
|
||||
}
|
||||
|
||||
// Reject writes in read-only mode
|
||||
if e.readOnly.Load() {
|
||||
return ErrReadOnlyMode
|
||||
}
|
||||
|
||||
// Track the operation - using a custom operation type might be good in the future
|
||||
e.stats.TrackOperation(stats.OpPut) // Using OpPut since batch operations are primarily writes
|
||||
|
||||
|
42
pkg/engine/replication.go
Normal file
42
pkg/engine/replication.go
Normal file
@ -0,0 +1,42 @@
|
||||
package engine
|
||||
|
||||
import (
|
||||
"github.com/KevoDB/kevo/pkg/common/log"
|
||||
"github.com/KevoDB/kevo/pkg/wal"
|
||||
)
|
||||
|
||||
// GetWAL exposes the WAL for replication purposes
|
||||
func (e *EngineFacade) GetWAL() *wal.WAL {
|
||||
// This is an enhancement to the EngineFacade to support replication
|
||||
// It's used by the replication manager to access the WAL
|
||||
if e.storage == nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
// Get WAL from storage manager
|
||||
// For now, we'll use type assertion since the interface doesn't
|
||||
// have a GetWAL method
|
||||
type walProvider interface {
|
||||
GetWAL() *wal.WAL
|
||||
}
|
||||
|
||||
if provider, ok := e.storage.(walProvider); ok {
|
||||
return provider.GetWAL()
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// SetReadOnly sets the engine to read-only mode for replicas
|
||||
func (e *EngineFacade) SetReadOnly(readOnly bool) {
|
||||
// This is an enhancement to the EngineFacade to support replication
|
||||
// Setting this will force the engine to reject write operations
|
||||
// Used by replicas to ensure they don't accept direct writes
|
||||
e.readOnly.Store(readOnly)
|
||||
log.Info("Engine read-only mode set to: %v", readOnly)
|
||||
}
|
||||
|
||||
// IsReadOnly returns whether the engine is in read-only mode
|
||||
func (e *EngineFacade) IsReadOnly() bool {
|
||||
return e.readOnly.Load()
|
||||
}
|
14
pkg/engine/storage/manager_wal.go
Normal file
14
pkg/engine/storage/manager_wal.go
Normal file
@ -0,0 +1,14 @@
|
||||
package storage
|
||||
|
||||
import (
|
||||
"github.com/KevoDB/kevo/pkg/wal"
|
||||
)
|
||||
|
||||
// GetWAL returns the storage manager's WAL instance
|
||||
// This is used by the replication manager to access the WAL
|
||||
func (m *Manager) GetWAL() *wal.WAL {
|
||||
m.mu.RLock()
|
||||
defer m.mu.RUnlock()
|
||||
|
||||
return m.wal
|
||||
}
|
384
pkg/replication/manager.go
Normal file
384
pkg/replication/manager.go
Normal file
@ -0,0 +1,384 @@
|
||||
// Package replication implements the primary-replica replication protocol for the Kevo database.
|
||||
package replication
|
||||
|
||||
import (
|
||||
"context"
|
||||
"crypto/tls"
|
||||
"fmt"
|
||||
"net"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/KevoDB/kevo/pkg/common/log"
|
||||
"github.com/KevoDB/kevo/pkg/engine/interfaces"
|
||||
proto "github.com/KevoDB/kevo/pkg/replication/proto"
|
||||
"github.com/KevoDB/kevo/pkg/wal"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/credentials"
|
||||
"google.golang.org/grpc/keepalive"
|
||||
)
|
||||
|
||||
// ManagerConfig defines the configuration for the replication manager
|
||||
type ManagerConfig struct {
|
||||
// Whether replication is enabled
|
||||
Enabled bool
|
||||
|
||||
// The replication mode: "primary", "replica", or "standalone"
|
||||
Mode string
|
||||
|
||||
// Address of the primary node (for replicas)
|
||||
PrimaryAddr string
|
||||
|
||||
// Address to listen on (for primaries)
|
||||
ListenAddr string
|
||||
|
||||
// Configuration for primary node
|
||||
PrimaryConfig *PrimaryConfig
|
||||
|
||||
// Configuration for replica node
|
||||
ReplicaConfig *ReplicaConfig
|
||||
|
||||
// TLS configuration
|
||||
TLSConfig *tls.Config
|
||||
|
||||
// Read-only mode enforcement for replicas
|
||||
ForceReadOnly bool
|
||||
}
|
||||
|
||||
// DefaultManagerConfig returns a default configuration for the replication manager
|
||||
func DefaultManagerConfig() *ManagerConfig {
|
||||
return &ManagerConfig{
|
||||
Enabled: false,
|
||||
Mode: "standalone",
|
||||
PrimaryAddr: "localhost:50052",
|
||||
ListenAddr: ":50052",
|
||||
PrimaryConfig: DefaultPrimaryConfig(),
|
||||
ReplicaConfig: DefaultReplicaConfig(),
|
||||
ForceReadOnly: true,
|
||||
}
|
||||
}
|
||||
|
||||
// Manager handles the setup and management of replication
|
||||
type Manager struct {
|
||||
config *ManagerConfig
|
||||
engine interfaces.Engine
|
||||
primary *Primary
|
||||
replica *Replica
|
||||
grpcServer *grpc.Server
|
||||
serviceStatus bool
|
||||
walApplier *EngineApplier
|
||||
lastApplied uint64
|
||||
mu sync.RWMutex
|
||||
ctx context.Context
|
||||
cancel context.CancelFunc
|
||||
}
|
||||
|
||||
// EngineApplier implements the WALEntryApplier interface for applying WAL entries to an engine
|
||||
type EngineApplier struct {
|
||||
engine interfaces.Engine
|
||||
}
|
||||
|
||||
// NewEngineApplier creates a new engine applier
|
||||
func NewEngineApplier(engine interfaces.Engine) *EngineApplier {
|
||||
return &EngineApplier{engine: engine}
|
||||
}
|
||||
|
||||
// Apply applies a WAL entry to the engine
|
||||
func (e *EngineApplier) Apply(entry *wal.Entry) error {
|
||||
switch entry.Type {
|
||||
case wal.OpTypePut:
|
||||
return e.engine.Put(entry.Key, entry.Value)
|
||||
case wal.OpTypeDelete:
|
||||
return e.engine.Delete(entry.Key)
|
||||
case wal.OpTypeBatch:
|
||||
return e.engine.ApplyBatch([]*wal.Entry{entry})
|
||||
default:
|
||||
return fmt.Errorf("unsupported WAL entry type: %d", entry.Type)
|
||||
}
|
||||
}
|
||||
|
||||
// Sync ensures all applied entries are persisted
|
||||
func (e *EngineApplier) Sync() error {
|
||||
// Force a flush of in-memory tables to ensure durability
|
||||
return e.engine.FlushImMemTables()
|
||||
}
|
||||
|
||||
// NewManager creates a new replication manager
|
||||
func NewManager(engine interfaces.Engine, config *ManagerConfig) (*Manager, error) {
|
||||
if config == nil {
|
||||
config = DefaultManagerConfig()
|
||||
}
|
||||
|
||||
if !config.Enabled {
|
||||
return &Manager{
|
||||
config: config,
|
||||
engine: engine,
|
||||
serviceStatus: false,
|
||||
}, nil
|
||||
}
|
||||
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
|
||||
return &Manager{
|
||||
config: config,
|
||||
engine: engine,
|
||||
serviceStatus: false,
|
||||
walApplier: NewEngineApplier(engine),
|
||||
ctx: ctx,
|
||||
cancel: cancel,
|
||||
}, nil
|
||||
}
|
||||
|
||||
// Start initializes and starts the replication service
|
||||
func (m *Manager) Start() error {
|
||||
m.mu.Lock()
|
||||
defer m.mu.Unlock()
|
||||
|
||||
if !m.config.Enabled {
|
||||
log.Info("Replication not enabled, skipping initialization")
|
||||
return nil
|
||||
}
|
||||
|
||||
log.Info("Starting replication in %s mode", m.config.Mode)
|
||||
|
||||
switch m.config.Mode {
|
||||
case "primary":
|
||||
return m.startPrimary()
|
||||
case "replica":
|
||||
return m.startReplica()
|
||||
case "standalone":
|
||||
log.Info("Running in standalone mode (no replication)")
|
||||
return nil
|
||||
default:
|
||||
return fmt.Errorf("invalid replication mode: %s", m.config.Mode)
|
||||
}
|
||||
}
|
||||
|
||||
// Stop halts the replication service
|
||||
func (m *Manager) Stop() error {
|
||||
m.mu.Lock()
|
||||
defer m.mu.Unlock()
|
||||
|
||||
if !m.serviceStatus {
|
||||
return nil
|
||||
}
|
||||
|
||||
// Cancel the context to signal shutdown to all goroutines
|
||||
if m.cancel != nil {
|
||||
m.cancel()
|
||||
}
|
||||
|
||||
// Shut down gRPC server
|
||||
if m.grpcServer != nil {
|
||||
m.grpcServer.GracefulStop()
|
||||
m.grpcServer = nil
|
||||
}
|
||||
|
||||
// Stop the replica
|
||||
if m.replica != nil {
|
||||
if err := m.replica.Stop(); err != nil {
|
||||
log.Error("Error stopping replica: %v", err)
|
||||
}
|
||||
m.replica = nil
|
||||
}
|
||||
|
||||
// Close the primary
|
||||
if m.primary != nil {
|
||||
if err := m.primary.Close(); err != nil {
|
||||
log.Error("Error closing primary: %v", err)
|
||||
}
|
||||
m.primary = nil
|
||||
}
|
||||
|
||||
m.serviceStatus = false
|
||||
log.Info("Replication service stopped")
|
||||
return nil
|
||||
}
|
||||
|
||||
// Status returns the current status of the replication service
|
||||
func (m *Manager) Status() map[string]interface{} {
|
||||
m.mu.RLock()
|
||||
defer m.mu.RUnlock()
|
||||
|
||||
status := map[string]interface{}{
|
||||
"enabled": m.config.Enabled,
|
||||
"mode": m.config.Mode,
|
||||
"active": m.serviceStatus,
|
||||
}
|
||||
|
||||
// Add mode-specific status
|
||||
switch m.config.Mode {
|
||||
case "primary":
|
||||
if m.primary != nil {
|
||||
// Add information about connected replicas, etc.
|
||||
status["listen_address"] = m.config.ListenAddr
|
||||
// TODO: Add more detailed primary status
|
||||
}
|
||||
case "replica":
|
||||
if m.replica != nil {
|
||||
status["primary_address"] = m.config.PrimaryAddr
|
||||
status["last_applied_sequence"] = m.lastApplied
|
||||
status["state"] = m.replica.GetCurrentState().String()
|
||||
// TODO: Add more detailed replica status
|
||||
}
|
||||
}
|
||||
|
||||
return status
|
||||
}
|
||||
|
||||
// startPrimary initializes the primary node
|
||||
func (m *Manager) startPrimary() error {
|
||||
// Access the WAL from the engine
|
||||
// This requires the engine to expose its WAL - might need interface enhancement
|
||||
wal, err := m.getWAL()
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to access WAL: %w", err)
|
||||
}
|
||||
|
||||
// Create primary replication service
|
||||
primary, err := NewPrimary(wal, m.config.PrimaryConfig)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to create primary node: %w", err)
|
||||
}
|
||||
|
||||
// Configure gRPC server options
|
||||
opts := []grpc.ServerOption{
|
||||
grpc.KeepaliveParams(keepalive.ServerParameters{
|
||||
Time: 10 * time.Second, // Send pings every 10 seconds if there is no activity
|
||||
Timeout: 5 * time.Second, // Wait 5 seconds for ping ack before assuming connection is dead
|
||||
}),
|
||||
grpc.KeepaliveEnforcementPolicy(keepalive.EnforcementPolicy{
|
||||
MinTime: 5 * time.Second, // Minimum time a client should wait before sending a ping
|
||||
PermitWithoutStream: true, // Allow pings even when there are no active streams
|
||||
}),
|
||||
grpc.MaxRecvMsgSize(16 * 1024 * 1024), // 16MB max message size
|
||||
grpc.MaxSendMsgSize(16 * 1024 * 1024), // 16MB max message size
|
||||
}
|
||||
|
||||
// Add TLS if configured
|
||||
if m.config.TLSConfig != nil {
|
||||
opts = append(opts, grpc.Creds(credentials.NewTLS(m.config.TLSConfig)))
|
||||
}
|
||||
|
||||
// Create gRPC server
|
||||
server := grpc.NewServer(opts...)
|
||||
|
||||
// Register primary service
|
||||
proto.RegisterWALReplicationServiceServer(server, primary)
|
||||
|
||||
// Start server in a separate goroutine
|
||||
go func() {
|
||||
// Start listening
|
||||
listener, err := createListener(m.config.ListenAddr)
|
||||
if err != nil {
|
||||
log.Error("Failed to create listener for primary: %v", err)
|
||||
return
|
||||
}
|
||||
|
||||
log.Info("Primary node listening on %s", m.config.ListenAddr)
|
||||
if err := server.Serve(listener); err != nil {
|
||||
log.Error("Primary gRPC server error: %v", err)
|
||||
}
|
||||
}()
|
||||
|
||||
// Store references
|
||||
m.primary = primary
|
||||
m.grpcServer = server
|
||||
m.serviceStatus = true
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// startReplica initializes the replica node
|
||||
func (m *Manager) startReplica() error {
|
||||
// Check last applied sequence (ideally from persistent storage)
|
||||
// For now, we'll start from 0
|
||||
lastApplied := uint64(0)
|
||||
|
||||
// Adjust replica config for connection
|
||||
replicaConfig := m.config.ReplicaConfig
|
||||
if replicaConfig == nil {
|
||||
replicaConfig = DefaultReplicaConfig()
|
||||
}
|
||||
|
||||
// Configure the connection to the primary
|
||||
replicaConfig.Connection.PrimaryAddress = m.config.PrimaryAddr
|
||||
replicaConfig.Connection.UseTLS = m.config.TLSConfig != nil
|
||||
|
||||
// Set TLS credentials if configured
|
||||
if m.config.TLSConfig != nil {
|
||||
replicaConfig.Connection.TLSCredentials = credentials.NewTLS(m.config.TLSConfig)
|
||||
} else {
|
||||
// Use insecure credentials if TLS is not configured
|
||||
replicaConfig.Connection.TLSCredentials = credentials.NewTLS(nil)
|
||||
}
|
||||
|
||||
// Create replica instance
|
||||
replica, err := NewReplica(lastApplied, m.walApplier, replicaConfig)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to create replica node: %w", err)
|
||||
}
|
||||
|
||||
// Start replication
|
||||
if err := replica.Start(); err != nil {
|
||||
return fmt.Errorf("failed to start replica: %w", err)
|
||||
}
|
||||
|
||||
// Set read-only mode on the engine if configured
|
||||
if m.config.ForceReadOnly {
|
||||
if err := m.setEngineReadOnly(true); err != nil {
|
||||
log.Warn("Failed to set engine to read-only mode: %v", err)
|
||||
} else {
|
||||
log.Info("Engine set to read-only mode (replica)")
|
||||
}
|
||||
}
|
||||
|
||||
// Store references
|
||||
m.replica = replica
|
||||
m.lastApplied = lastApplied
|
||||
m.serviceStatus = true
|
||||
|
||||
log.Info("Replica connected to primary at %s", m.config.PrimaryAddr)
|
||||
return nil
|
||||
}
|
||||
|
||||
// setEngineReadOnly sets the read-only mode on the engine (if supported)
|
||||
func (m *Manager) setEngineReadOnly(readOnly bool) error {
|
||||
// Try to access the SetReadOnly method if available
|
||||
// This would be engine-specific and may require interface enhancement
|
||||
// For now, we'll assume this is implemented via type assertion
|
||||
type readOnlySetter interface {
|
||||
SetReadOnly(bool)
|
||||
}
|
||||
|
||||
if setter, ok := m.engine.(readOnlySetter); ok {
|
||||
setter.SetReadOnly(readOnly)
|
||||
return nil
|
||||
}
|
||||
|
||||
return fmt.Errorf("engine does not support read-only mode setting")
|
||||
}
|
||||
|
||||
// getWAL retrieves the WAL from the engine
|
||||
func (m *Manager) getWAL() (*wal.WAL, error) {
|
||||
// This would be engine-specific and may require interface enhancement
|
||||
// For now, we'll assume this is implemented via type assertion
|
||||
type walProvider interface {
|
||||
GetWAL() *wal.WAL
|
||||
}
|
||||
|
||||
if provider, ok := m.engine.(walProvider); ok {
|
||||
wal := provider.GetWAL()
|
||||
if wal == nil {
|
||||
return nil, fmt.Errorf("engine returned nil WAL")
|
||||
}
|
||||
return wal, nil
|
||||
}
|
||||
|
||||
return nil, fmt.Errorf("engine does not provide WAL access")
|
||||
}
|
||||
|
||||
// createListener creates a network listener for the gRPC server
|
||||
func createListener(address string) (net.Listener, error) {
|
||||
return net.Listen("tcp", address)
|
||||
}
|
246
pkg/replication/manager_test.go
Normal file
246
pkg/replication/manager_test.go
Normal file
@ -0,0 +1,246 @@
|
||||
package replication
|
||||
|
||||
import (
|
||||
"testing"
|
||||
|
||||
"github.com/KevoDB/kevo/pkg/common/iterator"
|
||||
"github.com/KevoDB/kevo/pkg/engine/interfaces"
|
||||
"github.com/KevoDB/kevo/pkg/wal"
|
||||
)
|
||||
|
||||
// MockEngine implements a minimal mock engine for testing
|
||||
type MockEngine struct {
|
||||
wal *wal.WAL
|
||||
readOnly bool
|
||||
}
|
||||
|
||||
// Implement only essential methods for the test
|
||||
func (m *MockEngine) GetWAL() *wal.WAL {
|
||||
return m.wal
|
||||
}
|
||||
|
||||
func (m *MockEngine) SetReadOnly(readOnly bool) {
|
||||
m.readOnly = readOnly
|
||||
}
|
||||
|
||||
func (m *MockEngine) FlushImMemTables() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
// Implement required interface methods with minimal stubs
|
||||
func (m *MockEngine) Put(key, value []byte) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *MockEngine) Get(key []byte) ([]byte, error) {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
func (m *MockEngine) Delete(key []byte) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *MockEngine) IsDeleted(key []byte) (bool, error) {
|
||||
return false, nil
|
||||
}
|
||||
|
||||
func (m *MockEngine) GetIterator() (iterator.Iterator, error) {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
func (m *MockEngine) GetRangeIterator(startKey, endKey []byte) (iterator.Iterator, error) {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
func (m *MockEngine) ApplyBatch(entries []*wal.Entry) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *MockEngine) BeginTransaction(readOnly bool) (interfaces.Transaction, error) {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
func (m *MockEngine) TriggerCompaction() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *MockEngine) CompactRange(startKey, endKey []byte) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *MockEngine) GetStats() map[string]interface{} {
|
||||
return map[string]interface{}{}
|
||||
}
|
||||
|
||||
func (m *MockEngine) GetCompactionStats() (map[string]interface{}, error) {
|
||||
return map[string]interface{}{}, nil
|
||||
}
|
||||
|
||||
func (m *MockEngine) Close() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
// TestNewManager tests the creation of a new replication manager
|
||||
func TestNewManager(t *testing.T) {
|
||||
engine := &MockEngine{}
|
||||
|
||||
// Test with nil config
|
||||
manager, err := NewManager(engine, nil)
|
||||
if err != nil {
|
||||
t.Fatalf("Expected no error when creating manager with nil config, got: %v", err)
|
||||
}
|
||||
if manager == nil {
|
||||
t.Fatal("Expected non-nil manager")
|
||||
}
|
||||
if manager.config.Enabled {
|
||||
t.Error("Expected Enabled to be false")
|
||||
}
|
||||
if manager.config.Mode != "standalone" {
|
||||
t.Errorf("Expected Mode to be 'standalone', got '%s'", manager.config.Mode)
|
||||
}
|
||||
|
||||
// Test with custom config
|
||||
config := &ManagerConfig{
|
||||
Enabled: true,
|
||||
Mode: "primary",
|
||||
ListenAddr: ":50053",
|
||||
PrimaryAddr: "localhost:50053",
|
||||
}
|
||||
manager, err = NewManager(engine, config)
|
||||
if err != nil {
|
||||
t.Fatalf("Expected no error when creating manager with custom config, got: %v", err)
|
||||
}
|
||||
if manager == nil {
|
||||
t.Fatal("Expected non-nil manager")
|
||||
}
|
||||
if !manager.config.Enabled {
|
||||
t.Error("Expected Enabled to be true")
|
||||
}
|
||||
if manager.config.Mode != "primary" {
|
||||
t.Errorf("Expected Mode to be 'primary', got '%s'", manager.config.Mode)
|
||||
}
|
||||
}
|
||||
|
||||
// TestManagerStartStandalone tests starting the manager in standalone mode
|
||||
func TestManagerStartStandalone(t *testing.T) {
|
||||
engine := &MockEngine{}
|
||||
|
||||
config := &ManagerConfig{
|
||||
Enabled: true,
|
||||
Mode: "standalone",
|
||||
}
|
||||
|
||||
manager, err := NewManager(engine, config)
|
||||
if err != nil {
|
||||
t.Fatalf("Expected no error, got: %v", err)
|
||||
}
|
||||
|
||||
err = manager.Start()
|
||||
if err != nil {
|
||||
t.Errorf("Expected no error when starting in standalone mode, got: %v", err)
|
||||
}
|
||||
if manager.serviceStatus {
|
||||
t.Error("Expected serviceStatus to be false")
|
||||
}
|
||||
|
||||
err = manager.Stop()
|
||||
if err != nil {
|
||||
t.Errorf("Expected no error when stopping, got: %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
// TestManagerStatus tests the status reporting functionality
|
||||
func TestManagerStatus(t *testing.T) {
|
||||
engine := &MockEngine{}
|
||||
|
||||
// Test disabled mode
|
||||
config := &ManagerConfig{
|
||||
Enabled: false,
|
||||
Mode: "standalone",
|
||||
}
|
||||
|
||||
manager, _ := NewManager(engine, config)
|
||||
status := manager.Status()
|
||||
|
||||
if status["enabled"].(bool) != false {
|
||||
t.Error("Expected 'enabled' to be false")
|
||||
}
|
||||
if status["mode"].(string) != "standalone" {
|
||||
t.Errorf("Expected 'mode' to be 'standalone', got '%s'", status["mode"].(string))
|
||||
}
|
||||
if status["active"].(bool) != false {
|
||||
t.Error("Expected 'active' to be false")
|
||||
}
|
||||
|
||||
// Test primary mode
|
||||
config = &ManagerConfig{
|
||||
Enabled: true,
|
||||
Mode: "primary",
|
||||
ListenAddr: ":50057",
|
||||
}
|
||||
|
||||
manager, _ = NewManager(engine, config)
|
||||
manager.serviceStatus = true
|
||||
status = manager.Status()
|
||||
|
||||
if status["enabled"].(bool) != true {
|
||||
t.Error("Expected 'enabled' to be true")
|
||||
}
|
||||
if status["mode"].(string) != "primary" {
|
||||
t.Errorf("Expected 'mode' to be 'primary', got '%s'", status["mode"].(string))
|
||||
}
|
||||
if status["active"].(bool) != true {
|
||||
t.Error("Expected 'active' to be true")
|
||||
}
|
||||
|
||||
// There will be no listen_address in the status until the primary is actually created
|
||||
// so we skip checking that field
|
||||
}
|
||||
|
||||
// TestEngineApplier tests the engine applier implementation
|
||||
func TestEngineApplier(t *testing.T) {
|
||||
engine := &MockEngine{}
|
||||
|
||||
applier := NewEngineApplier(engine)
|
||||
|
||||
// Test Put
|
||||
entry := &wal.Entry{
|
||||
Type: wal.OpTypePut,
|
||||
Key: []byte("test-key"),
|
||||
Value: []byte("test-value"),
|
||||
}
|
||||
err := applier.Apply(entry)
|
||||
if err != nil {
|
||||
t.Errorf("Expected no error for Put, got: %v", err)
|
||||
}
|
||||
|
||||
// Test Delete
|
||||
entry = &wal.Entry{
|
||||
Type: wal.OpTypeDelete,
|
||||
Key: []byte("test-key"),
|
||||
}
|
||||
err = applier.Apply(entry)
|
||||
if err != nil {
|
||||
t.Errorf("Expected no error for Delete, got: %v", err)
|
||||
}
|
||||
|
||||
// Test Batch
|
||||
entry = &wal.Entry{
|
||||
Type: wal.OpTypeBatch,
|
||||
Key: []byte("test-key"),
|
||||
}
|
||||
err = applier.Apply(entry)
|
||||
if err != nil {
|
||||
t.Errorf("Expected no error for Batch, got: %v", err)
|
||||
}
|
||||
|
||||
// Test unsupported type
|
||||
entry = &wal.Entry{
|
||||
Type: 99, // Invalid type
|
||||
Key: []byte("test-key"),
|
||||
}
|
||||
err = applier.Apply(entry)
|
||||
if err == nil {
|
||||
t.Error("Expected error for unsupported entry type")
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue
Block a user