feat: implement replication transport layer
All checks were successful
Go Tests / Run Tests (1.24.2) (pull_request) Successful in 9m49s
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:
parent
ed991ae00d
commit
5963538bc5
610
pkg/grpc/service/replication_service.go
Normal file
610
pkg/grpc/service/replication_service.go
Normal 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
|
||||
}
|
494
pkg/grpc/transport/replication_client.go
Normal file
494
pkg/grpc/transport/replication_client.go
Normal 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
|
||||
}
|
200
pkg/grpc/transport/replication_server.go
Normal file
200
pkg/grpc/transport/replication_server.go
Normal 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
|
||||
}
|
46
pkg/replication/replicator_ext.go
Normal file
46
pkg/replication/replicator_ext.go
Normal 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]
|
||||
}
|
||||
}
|
87
pkg/replication/storage_snapshot.go
Normal file
87
pkg/replication/storage_snapshot.go
Normal 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,
|
||||
}
|
||||
}
|
401
pkg/transport/replication_test.go
Normal file
401
pkg/transport/replication_test.go
Normal 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
1464
proto/kevo/replication.pb.go
Normal file
File diff suppressed because it is too large
Load Diff
400
proto/kevo/replication_grpc.pb.go
Normal file
400
proto/kevo/replication_grpc.pb.go
Normal 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",
|
||||
}
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -4,7 +4,7 @@
|
||||
// - protoc v3.20.3
|
||||
// source: proto/kevo/service.proto
|
||||
|
||||
package proto
|
||||
package kevo
|
||||
|
||||
import (
|
||||
context "context"
|
||||
|
Loading…
Reference in New Issue
Block a user