feat: Update client sdk (Go) with smart connection logic

- Client SDK will connect to a node, get node information and decide if
  it needs to connect to a primary for writes, or pick a replica to
  connect to for reads
- Updated service with a GetNodeInfo rpc call which returns information
  about the node to enable the smart selection code in the sdks
This commit is contained in:
Jeremy Tregunna 2025-04-28 00:31:49 -06:00
parent 4429836929
commit f9e332096c
14 changed files with 1192 additions and 143 deletions

View File

@ -93,10 +93,6 @@ func (s *Server) Start() error {
// Create gRPC server with options
s.grpcServer = grpc.NewServer(serverOpts...)
// Create and register the Kevo service implementation
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
@ -128,6 +124,10 @@ func (s *Server) Start() error {
}
}
// Create and register the Kevo service implementation
s.kevoService = grpcservice.NewKevoServiceServer(s.eng, s.txRegistry, s.replicationManager)
pb.RegisterKevoServiceServer(s.grpcServer, s.kevoService)
fmt.Println("gRPC server initialized")
return nil
}

View File

@ -236,11 +236,11 @@ func runWriteBenchmark(e *engine.EngineFacade) string {
}
// Handle WAL rotation errors more gracefully
if strings.Contains(err.Error(), "WAL is rotating") ||
strings.Contains(err.Error(), "WAL is closed") {
if strings.Contains(err.Error(), "WAL is rotating") ||
strings.Contains(err.Error(), "WAL is closed") {
// These are expected during WAL rotation, just retry after a short delay
walRotationCount++
if walRotationCount % 100 == 0 {
if walRotationCount%100 == 0 {
fmt.Printf("Retrying due to WAL rotation (%d retries so far)...\n", walRotationCount)
}
time.Sleep(20 * time.Millisecond)
@ -334,10 +334,10 @@ func runRandomWriteBenchmark(e *engine.EngineFacade) string {
}
// Handle WAL rotation errors
if strings.Contains(err.Error(), "WAL is rotating") ||
strings.Contains(err.Error(), "WAL is closed") {
if strings.Contains(err.Error(), "WAL is rotating") ||
strings.Contains(err.Error(), "WAL is closed") {
walRotationCount++
if walRotationCount % 100 == 0 {
if walRotationCount%100 == 0 {
fmt.Printf("Retrying due to WAL rotation (%d retries so far)...\n", walRotationCount)
}
time.Sleep(20 * time.Millisecond)
@ -430,10 +430,10 @@ func runSequentialWriteBenchmark(e *engine.EngineFacade) string {
}
// Handle WAL rotation errors
if strings.Contains(err.Error(), "WAL is rotating") ||
strings.Contains(err.Error(), "WAL is closed") {
if strings.Contains(err.Error(), "WAL is rotating") ||
strings.Contains(err.Error(), "WAL is closed") {
walRotationCount++
if walRotationCount % 100 == 0 {
if walRotationCount%100 == 0 {
fmt.Printf("Retrying due to WAL rotation (%d retries so far)...\n", walRotationCount)
}
time.Sleep(20 * time.Millisecond)
@ -586,9 +586,9 @@ func runRandomReadBenchmark(e *engine.EngineFacade) string {
// Write the test data with random keys
for i := 0; i < actualNumKeys; i++ {
keys[i] = []byte(fmt.Sprintf("rand-key-%s-%06d",
keys[i] = []byte(fmt.Sprintf("rand-key-%s-%06d",
strconv.FormatUint(r.Uint64(), 16), i))
if err := e.Put(keys[i], value); err != nil {
if err == engine.ErrEngineClosed {
fmt.Fprintf(os.Stderr, "Engine closed during preparation\n")
@ -644,7 +644,7 @@ benchmarkEnd:
result := fmt.Sprintf("\nRandom Read Benchmark Results:")
result += fmt.Sprintf("\n Operations: %d", opsCount)
result += fmt.Sprintf("\n Hit Rate: %.2f%%", hitRate)
result += fmt.Sprintf("\n Hit Rate: %.2f%%", hitRate)
result += fmt.Sprintf("\n Time: %.2f seconds", elapsed.Seconds())
result += fmt.Sprintf("\n Throughput: %.2f ops/sec", opsPerSecond)
result += fmt.Sprintf("\n Latency: %.3f µs/op", 1000000.0/opsPerSecond)
@ -770,18 +770,18 @@ func runRangeScanBenchmark(e *engine.EngineFacade) string {
// Keys will be organized into buckets for realistic scanning
const BUCKETS = 100
keysPerBucket := actualNumKeys / BUCKETS
value := make([]byte, *valueSize)
for i := range value {
value[i] = byte(i % 256)
}
fmt.Printf("Creating %d buckets with approximately %d keys each...\n",
fmt.Printf("Creating %d buckets with approximately %d keys each...\n",
BUCKETS, keysPerBucket)
for bucket := 0; bucket < BUCKETS; bucket++ {
bucketPrefix := fmt.Sprintf("bucket-%03d:", bucket)
// Create keys within this bucket
for i := 0; i < keysPerBucket; i++ {
key := []byte(fmt.Sprintf("%s%06d", bucketPrefix, i))
@ -811,7 +811,7 @@ func runRangeScanBenchmark(e *engine.EngineFacade) string {
var opsCount, entriesScanned int
r := rand.New(rand.NewSource(time.Now().UnixNano()))
// Use configured scan size or default to 100
scanSize := *scanSize
@ -819,10 +819,10 @@ func runRangeScanBenchmark(e *engine.EngineFacade) string {
// Pick a random bucket to scan
bucket := r.Intn(BUCKETS)
bucketPrefix := fmt.Sprintf("bucket-%03d:", bucket)
// Determine scan range - either full bucket or partial depending on scan size
var startKey, endKey []byte
if scanSize >= keysPerBucket {
// Scan whole bucket
startKey = []byte(fmt.Sprintf("%s%06d", bucketPrefix, 0))
@ -993,4 +993,4 @@ func generateKey(counter int) []byte {
// Random key with counter to ensure uniqueness
return []byte(fmt.Sprintf("key-%s-%010d",
strconv.FormatUint(rand.Uint64(), 16), counter))
}
}

View File

@ -5,6 +5,7 @@ import (
"encoding/json"
"errors"
"fmt"
"sync"
"time"
"github.com/KevoDB/kevo/pkg/transport"
@ -66,10 +67,32 @@ func DefaultClientOptions() ClientOptions {
}
}
// ReplicaInfo represents information about a replica node
type ReplicaInfo struct {
Address string // Host:port of the replica
LastSequence uint64 // Last applied sequence number
Available bool // Whether the replica is available
Region string // Optional region information
Meta map[string]string // Additional metadata
}
// NodeInfo contains information about the server node and topology
type NodeInfo struct {
Role string // "primary", "replica", or "standalone"
PrimaryAddr string // Address of the primary node
Replicas []ReplicaInfo // Available replica nodes
LastSequence uint64 // Last applied sequence number
ReadOnly bool // Whether the node is in read-only mode
}
// Client represents a connection to a Kevo database server
type Client struct {
options ClientOptions
client transport.Client
options ClientOptions
client transport.Client
primaryConn transport.Client // Connection to primary (when connected to replica)
replicaConn []transport.Client // Connections to replicas (when connected to primary)
nodeInfo *NodeInfo // Information about the current node and topology
connMutex sync.RWMutex // Protects connections
}
// NewClient creates a new Kevo client with the given options
@ -107,26 +130,223 @@ func NewClient(options ClientOptions) (*Client, error) {
}
// Connect establishes a connection to the server
// and discovers the replication topology if available
func (c *Client) Connect(ctx context.Context) error {
return c.client.Connect(ctx)
// First connect to the primary endpoint
if err := c.client.Connect(ctx); err != nil {
return err
}
// Query node information to discover the topology
return c.discoverTopology(ctx)
}
// Close closes the connection to the server
// discoverTopology queries the node for replication information
// and establishes additional connections if needed
func (c *Client) discoverTopology(ctx context.Context) error {
// Get node info from the connected server
nodeInfo, err := c.getNodeInfo(ctx)
if err != nil {
// If GetNodeInfo isn't supported, assume it's standalone
// This ensures backward compatibility with older servers
nodeInfo = &NodeInfo{
Role: "standalone",
ReadOnly: false,
}
}
c.connMutex.Lock()
defer c.connMutex.Unlock()
// Store the node info
c.nodeInfo = nodeInfo
// Based on the role, establish additional connections as needed
switch nodeInfo.Role {
case "replica":
// If connected to a replica and a primary is available, connect to it
if nodeInfo.PrimaryAddr != "" && nodeInfo.PrimaryAddr != c.options.Endpoint {
primaryOptions := c.options
primaryOptions.Endpoint = nodeInfo.PrimaryAddr
// Create client connection to primary
primaryClient, err := transport.GetClient(
primaryOptions.TransportType,
primaryOptions.Endpoint,
c.createTransportOptions(primaryOptions),
)
if err == nil {
// Try to connect to primary
if err := primaryClient.Connect(ctx); err == nil {
c.primaryConn = primaryClient
}
}
}
case "primary":
// If connected to a primary and replicas are available, connect to some of them
c.replicaConn = make([]transport.Client, 0, len(nodeInfo.Replicas))
// Connect to up to 2 replicas (to avoid too many connections)
for i, replica := range nodeInfo.Replicas {
if i >= 2 || !replica.Available {
continue
}
replicaOptions := c.options
replicaOptions.Endpoint = replica.Address
// Create client connection to replica
replicaClient, err := transport.GetClient(
replicaOptions.TransportType,
replicaOptions.Endpoint,
c.createTransportOptions(replicaOptions),
)
if err == nil {
// Try to connect to replica
if err := replicaClient.Connect(ctx); err == nil {
c.replicaConn = append(c.replicaConn, replicaClient)
}
}
}
}
return nil
}
// createTransportOptions converts client options to transport options
func (c *Client) createTransportOptions(options ClientOptions) transport.TransportOptions {
return transport.TransportOptions{
Timeout: options.ConnectTimeout,
MaxMessageSize: options.MaxMessageSize,
Compression: options.Compression,
TLSEnabled: options.TLSEnabled,
CertFile: options.CertFile,
KeyFile: options.KeyFile,
CAFile: options.CAFile,
RetryPolicy: transport.RetryPolicy{
MaxRetries: options.MaxRetries,
InitialBackoff: options.InitialBackoff,
MaxBackoff: options.MaxBackoff,
BackoffFactor: options.BackoffFactor,
Jitter: options.RetryJitter,
},
}
}
// Close closes all connections to servers
func (c *Client) Close() error {
c.connMutex.Lock()
defer c.connMutex.Unlock()
// Close primary connection
if c.primaryConn != nil {
c.primaryConn.Close()
c.primaryConn = nil
}
// Close replica connections
for _, replica := range c.replicaConn {
replica.Close()
}
c.replicaConn = nil
// Close main connection
return c.client.Close()
}
// getNodeInfo retrieves node information from the server
func (c *Client) getNodeInfo(ctx context.Context) (*NodeInfo, error) {
// Create a request to the GetNodeInfo endpoint
req := transport.NewRequest("GetNodeInfo", nil)
// Send the request
timeoutCtx, cancel := context.WithTimeout(ctx, c.options.RequestTimeout)
defer cancel()
resp, err := c.client.Send(timeoutCtx, req)
if err != nil {
return nil, fmt.Errorf("failed to get node info: %w", err)
}
// Parse the response
var nodeInfoResp struct {
NodeRole int `json:"node_role"`
PrimaryAddress string `json:"primary_address"`
Replicas []json.RawMessage `json:"replicas"`
LastSequence uint64 `json:"last_sequence"`
ReadOnly bool `json:"read_only"`
}
if err := json.Unmarshal(resp.Payload(), &nodeInfoResp); err != nil {
return nil, fmt.Errorf("failed to unmarshal node info response: %w", err)
}
// Convert role from int to string
var role string
switch nodeInfoResp.NodeRole {
case 0:
role = "standalone"
case 1:
role = "primary"
case 2:
role = "replica"
default:
role = "unknown"
}
// Parse replica information
replicas := make([]ReplicaInfo, 0, len(nodeInfoResp.Replicas))
for _, rawReplica := range nodeInfoResp.Replicas {
var replica struct {
Address string `json:"address"`
LastSequence uint64 `json:"last_sequence"`
Available bool `json:"available"`
Region string `json:"region"`
Meta map[string]string `json:"meta"`
}
if err := json.Unmarshal(rawReplica, &replica); err != nil {
continue // Skip replicas that can't be parsed
}
replicas = append(replicas, ReplicaInfo{
Address: replica.Address,
LastSequence: replica.LastSequence,
Available: replica.Available,
Region: replica.Region,
Meta: replica.Meta,
})
}
return &NodeInfo{
Role: role,
PrimaryAddr: nodeInfoResp.PrimaryAddress,
Replicas: replicas,
LastSequence: nodeInfoResp.LastSequence,
ReadOnly: nodeInfoResp.ReadOnly,
}, nil
}
// IsConnected returns whether the client is connected to the server
func (c *Client) IsConnected() bool {
return c.client != nil && c.client.IsConnected()
}
// Get retrieves a value by key
// If connected to a primary with replicas, it will route reads to a replica
func (c *Client) Get(ctx context.Context, key []byte) ([]byte, bool, error) {
if !c.IsConnected() {
return nil, false, errors.New("not connected to server")
}
// Check if we should route to replica
c.connMutex.RLock()
shouldUseReplica := c.nodeInfo != nil &&
c.nodeInfo.Role == "primary" &&
len(c.replicaConn) > 0
c.connMutex.RUnlock()
req := struct {
Key []byte `json:"key"`
}{
@ -141,9 +361,29 @@ func (c *Client) Get(ctx context.Context, key []byte) ([]byte, bool, error) {
timeoutCtx, cancel := context.WithTimeout(ctx, c.options.RequestTimeout)
defer cancel()
resp, err := c.client.Send(timeoutCtx, transport.NewRequest(transport.TypeGet, reqData))
if err != nil {
return nil, false, fmt.Errorf("failed to send request: %w", err)
var resp transport.Response
var sendErr error
if shouldUseReplica {
// Select a replica for reading
c.connMutex.RLock()
selectedReplica := c.replicaConn[0] // Simple selection: always use first replica
c.connMutex.RUnlock()
// Try the replica first
resp, sendErr = selectedReplica.Send(timeoutCtx, transport.NewRequest(transport.TypeGet, reqData))
// If replica fails, fall back to primary
if sendErr != nil {
resp, sendErr = c.client.Send(timeoutCtx, transport.NewRequest(transport.TypeGet, reqData))
}
} else {
// Use default connection
resp, sendErr = c.client.Send(timeoutCtx, transport.NewRequest(transport.TypeGet, reqData))
}
if sendErr != nil {
return nil, false, fmt.Errorf("failed to send request: %w", sendErr)
}
var getResp struct {
@ -159,11 +399,19 @@ func (c *Client) Get(ctx context.Context, key []byte) ([]byte, bool, error) {
}
// Put stores a key-value pair
// If connected to a replica, it will automatically route the write to the primary
func (c *Client) Put(ctx context.Context, key, value []byte, sync bool) (bool, error) {
if !c.IsConnected() {
return false, errors.New("not connected to server")
}
// Check if we should route to primary
c.connMutex.RLock()
shouldUsePrimary := c.nodeInfo != nil &&
c.nodeInfo.Role == "replica" &&
c.primaryConn != nil
c.connMutex.RUnlock()
req := struct {
Key []byte `json:"key"`
Value []byte `json:"value"`
@ -182,9 +430,42 @@ func (c *Client) Put(ctx context.Context, key, value []byte, sync bool) (bool, e
timeoutCtx, cancel := context.WithTimeout(ctx, c.options.RequestTimeout)
defer cancel()
resp, err := c.client.Send(timeoutCtx, transport.NewRequest(transport.TypePut, reqData))
if err != nil {
return false, fmt.Errorf("failed to send request: %w", err)
var resp transport.Response
var sendErr error
if shouldUsePrimary {
// Use primary connection for writes when connected to replica
c.connMutex.RLock()
primaryConn := c.primaryConn
c.connMutex.RUnlock()
resp, sendErr = primaryConn.Send(timeoutCtx, transport.NewRequest(transport.TypePut, reqData))
} else {
// Use default connection
resp, sendErr = c.client.Send(timeoutCtx, transport.NewRequest(transport.TypePut, reqData))
// If we get a read-only error and we have node info, try to extract primary address
if sendErr != nil && c.nodeInfo == nil {
// Try to discover topology to get primary address
if discoverErr := c.discoverTopology(ctx); discoverErr == nil {
// Check again if we now have a primary connection
c.connMutex.RLock()
primaryAvailable := c.nodeInfo != nil &&
c.nodeInfo.Role == "replica" &&
c.primaryConn != nil
primaryConn := c.primaryConn
c.connMutex.RUnlock()
// If we now have a primary connection, retry the write
if primaryAvailable && primaryConn != nil {
resp, sendErr = primaryConn.Send(timeoutCtx, transport.NewRequest(transport.TypePut, reqData))
}
}
}
}
if sendErr != nil {
return false, fmt.Errorf("failed to send request: %w", sendErr)
}
var putResp struct {
@ -199,11 +480,19 @@ func (c *Client) Put(ctx context.Context, key, value []byte, sync bool) (bool, e
}
// Delete removes a key-value pair
// If connected to a replica, it will automatically route the delete to the primary
func (c *Client) Delete(ctx context.Context, key []byte, sync bool) (bool, error) {
if !c.IsConnected() {
return false, errors.New("not connected to server")
}
// Check if we should route to primary
c.connMutex.RLock()
shouldUsePrimary := c.nodeInfo != nil &&
c.nodeInfo.Role == "replica" &&
c.primaryConn != nil
c.connMutex.RUnlock()
req := struct {
Key []byte `json:"key"`
Sync bool `json:"sync"`
@ -220,9 +509,42 @@ func (c *Client) Delete(ctx context.Context, key []byte, sync bool) (bool, error
timeoutCtx, cancel := context.WithTimeout(ctx, c.options.RequestTimeout)
defer cancel()
resp, err := c.client.Send(timeoutCtx, transport.NewRequest(transport.TypeDelete, reqData))
if err != nil {
return false, fmt.Errorf("failed to send request: %w", err)
var resp transport.Response
var sendErr error
if shouldUsePrimary {
// Use primary connection for writes when connected to replica
c.connMutex.RLock()
primaryConn := c.primaryConn
c.connMutex.RUnlock()
resp, sendErr = primaryConn.Send(timeoutCtx, transport.NewRequest(transport.TypeDelete, reqData))
} else {
// Use default connection
resp, sendErr = c.client.Send(timeoutCtx, transport.NewRequest(transport.TypeDelete, reqData))
// If we get a read-only error and we have node info, try to extract primary address
if sendErr != nil && c.nodeInfo == nil {
// Try to discover topology to get primary address
if discoverErr := c.discoverTopology(ctx); discoverErr == nil {
// Check again if we now have a primary connection
c.connMutex.RLock()
primaryAvailable := c.nodeInfo != nil &&
c.nodeInfo.Role == "replica" &&
c.primaryConn != nil
primaryConn := c.primaryConn
c.connMutex.RUnlock()
// If we now have a primary connection, retry the delete
if primaryAvailable && primaryConn != nil {
resp, sendErr = primaryConn.Send(timeoutCtx, transport.NewRequest(transport.TypeDelete, reqData))
}
}
}
}
if sendErr != nil {
return false, fmt.Errorf("failed to send request: %w", sendErr)
}
var deleteResp struct {
@ -244,11 +566,19 @@ type BatchOperation struct {
}
// BatchWrite performs multiple operations in a single atomic batch
// If connected to a replica, it will automatically route the batch to the primary
func (c *Client) BatchWrite(ctx context.Context, operations []BatchOperation, sync bool) (bool, error) {
if !c.IsConnected() {
return false, errors.New("not connected to server")
}
// Check if we should route to primary
c.connMutex.RLock()
shouldUsePrimary := c.nodeInfo != nil &&
c.nodeInfo.Role == "replica" &&
c.primaryConn != nil
c.connMutex.RUnlock()
req := struct {
Operations []struct {
Type string `json:"type"`
@ -280,9 +610,42 @@ func (c *Client) BatchWrite(ctx context.Context, operations []BatchOperation, sy
timeoutCtx, cancel := context.WithTimeout(ctx, c.options.RequestTimeout)
defer cancel()
resp, err := c.client.Send(timeoutCtx, transport.NewRequest(transport.TypeBatchWrite, reqData))
if err != nil {
return false, fmt.Errorf("failed to send request: %w", err)
var resp transport.Response
var sendErr error
if shouldUsePrimary {
// Use primary connection for writes when connected to replica
c.connMutex.RLock()
primaryConn := c.primaryConn
c.connMutex.RUnlock()
resp, sendErr = primaryConn.Send(timeoutCtx, transport.NewRequest(transport.TypeBatchWrite, reqData))
} else {
// Use default connection
resp, sendErr = c.client.Send(timeoutCtx, transport.NewRequest(transport.TypeBatchWrite, reqData))
// If we get a read-only error and we have node info, try to extract primary address
if sendErr != nil && c.nodeInfo == nil {
// Try to discover topology to get primary address
if discoverErr := c.discoverTopology(ctx); discoverErr == nil {
// Check again if we now have a primary connection
c.connMutex.RLock()
primaryAvailable := c.nodeInfo != nil &&
c.nodeInfo.Role == "replica" &&
c.primaryConn != nil
primaryConn := c.primaryConn
c.connMutex.RUnlock()
// If we now have a primary connection, retry the batch
if primaryAvailable && primaryConn != nil {
resp, sendErr = primaryConn.Send(timeoutCtx, transport.NewRequest(transport.TypeBatchWrite, reqData))
}
}
}
}
if sendErr != nil {
return false, fmt.Errorf("failed to send request: %w", sendErr)
}
var batchResp struct {
@ -379,3 +742,51 @@ type Stats struct {
WriteAmplification float64
ReadAmplification float64
}
// GetNodeInfo returns information about the current node and replication topology
func (c *Client) GetReplicationInfo() (*NodeInfo, error) {
c.connMutex.RLock()
defer c.connMutex.RUnlock()
if c.nodeInfo == nil {
return nil, errors.New("replication information not available")
}
// Return a copy to avoid concurrent access issues
return &NodeInfo{
Role: c.nodeInfo.Role,
PrimaryAddr: c.nodeInfo.PrimaryAddr,
Replicas: c.nodeInfo.Replicas,
LastSequence: c.nodeInfo.LastSequence,
ReadOnly: c.nodeInfo.ReadOnly,
}, nil
}
// RefreshTopology updates the replication topology information
func (c *Client) RefreshTopology(ctx context.Context) error {
return c.discoverTopology(ctx)
}
// IsPrimary returns true if the connected node is a primary
func (c *Client) IsPrimary() bool {
c.connMutex.RLock()
defer c.connMutex.RUnlock()
return c.nodeInfo != nil && c.nodeInfo.Role == "primary"
}
// IsReplica returns true if the connected node is a replica
func (c *Client) IsReplica() bool {
c.connMutex.RLock()
defer c.connMutex.RUnlock()
return c.nodeInfo != nil && c.nodeInfo.Role == "replica"
}
// IsStandalone returns true if the connected node is standalone (not part of replication)
func (c *Client) IsStandalone() bool {
c.connMutex.RLock()
defer c.connMutex.RUnlock()
return c.nodeInfo == nil || c.nodeInfo.Role == "standalone"
}

View File

@ -0,0 +1,132 @@
package client
import (
"context"
"testing"
)
// Renamed from TestClientConnectWithTopology to avoid duplicate function name
func TestClientConnectWithReplicationTopology(t *testing.T) {
// Create mock client
mock := newMockClient()
mock.setResponse("GetNodeInfo", []byte(`{
"node_role": 0,
"primary_address": "",
"replicas": [],
"last_sequence": 0,
"read_only": false
}`))
// Create and override client
options := DefaultClientOptions()
options.TransportType = "mock"
client, err := NewClient(options)
if err != nil {
t.Fatalf("Failed to create client: %v", err)
}
// Replace the transport with our manually configured mock
client.client = mock
// Connect and discover topology
err = client.Connect(context.Background())
if err != nil {
t.Fatalf("Connect failed: %v", err)
}
// Verify node info was collected correctly
if client.nodeInfo == nil {
t.Fatal("Expected nodeInfo to be set")
}
if client.nodeInfo.Role != "standalone" {
t.Errorf("Expected role to be standalone, got %s", client.nodeInfo.Role)
}
}
// Test simple replica check
func TestIsReplicaMethod(t *testing.T) {
// Setup client with replica node info
client := &Client{
options: DefaultClientOptions(),
nodeInfo: &NodeInfo{
Role: "replica",
PrimaryAddr: "primary:50051",
},
}
// Verify IsReplica returns true
if !client.IsReplica() {
t.Error("Expected IsReplica() to return true for a replica node")
}
// Verify IsPrimary returns false
if client.IsPrimary() {
t.Error("Expected IsPrimary() to return false for a replica node")
}
// Verify IsStandalone returns false
if client.IsStandalone() {
t.Error("Expected IsStandalone() to return false for a replica node")
}
}
// Test simple primary check
func TestIsPrimaryMethod(t *testing.T) {
// Setup client with primary node info
client := &Client{
options: DefaultClientOptions(),
nodeInfo: &NodeInfo{
Role: "primary",
},
}
// Verify IsPrimary returns true
if !client.IsPrimary() {
t.Error("Expected IsPrimary() to return true for a primary node")
}
// Verify IsReplica returns false
if client.IsReplica() {
t.Error("Expected IsReplica() to return false for a primary node")
}
// Verify IsStandalone returns false
if client.IsStandalone() {
t.Error("Expected IsStandalone() to return false for a primary node")
}
}
// Test simple standalone check
func TestIsStandaloneMethod(t *testing.T) {
// Setup client with standalone node info
client := &Client{
options: DefaultClientOptions(),
nodeInfo: &NodeInfo{
Role: "standalone",
},
}
// Verify IsStandalone returns true
if !client.IsStandalone() {
t.Error("Expected IsStandalone() to return true for a standalone node")
}
// Verify IsPrimary returns false
if client.IsPrimary() {
t.Error("Expected IsPrimary() to return false for a standalone node")
}
// Verify IsReplica returns false
if client.IsReplica() {
t.Error("Expected IsReplica() to return false for a standalone node")
}
// Test with nil nodeInfo should also return true for standalone
client = &Client{
options: DefaultClientOptions(),
nodeInfo: nil,
}
if !client.IsStandalone() {
t.Error("Expected IsStandalone() to return true when nodeInfo is nil")
}
}

View File

@ -38,6 +38,9 @@ type Engine interface {
// Lifecycle management
Close() error
// Read-only mode?
IsReadOnly() bool
}
// Components is a struct containing all the components needed by the engine

View File

@ -536,10 +536,10 @@ func (m *Manager) rotateWAL() error {
// Store the old WAL for proper closure
oldWAL := m.wal
// Atomically update the WAL reference
m.wal = newWAL
// Now close the old WAL after the new one is in place
if err := oldWAL.Close(); err != nil {
// Just log the error but don't fail the rotation
@ -547,7 +547,7 @@ func (m *Manager) rotateWAL() error {
m.stats.TrackError("wal_close_error")
fmt.Printf("Warning: error closing old WAL: %v\n", err)
}
return nil
}

View File

@ -9,6 +9,6 @@ import (
func (m *Manager) GetWAL() *wal.WAL {
m.mu.RLock()
defer m.mu.RUnlock()
return m.wal
}
}

View File

@ -7,6 +7,7 @@ import (
"github.com/KevoDB/kevo/pkg/common/iterator"
"github.com/KevoDB/kevo/pkg/engine/interfaces"
"github.com/KevoDB/kevo/pkg/replication"
pb "github.com/KevoDB/kevo/proto/kevo"
)
@ -21,17 +22,18 @@ type TxRegistry interface {
// KevoServiceServer implements the gRPC KevoService interface
type KevoServiceServer struct {
pb.UnimplementedKevoServiceServer
engine interfaces.Engine
txRegistry TxRegistry
activeTx sync.Map // map[string]interfaces.Transaction
txMu sync.Mutex
compactionSem chan struct{} // Semaphore for limiting concurrent compactions
maxKeySize int // Maximum allowed key size
maxValueSize int // Maximum allowed value size
maxBatchSize int // Maximum number of operations in a batch
maxTransactions int // Maximum number of concurrent transactions
transactionTTL int64 // Maximum time in seconds a transaction can be idle
activeTransCount int32 // Count of active transactions
engine interfaces.Engine
txRegistry TxRegistry
activeTx sync.Map // map[string]interfaces.Transaction
txMu sync.Mutex
compactionSem chan struct{} // Semaphore for limiting concurrent compactions
maxKeySize int // Maximum allowed key size
maxValueSize int // Maximum allowed value size
maxBatchSize int // Maximum number of operations in a batch
maxTransactions int // Maximum number of concurrent transactions
transactionTTL int64 // Maximum time in seconds a transaction can be idle
activeTransCount int32 // Count of active transactions
replicationManager ReplicationInfoProvider // Interface to the replication manager
}
// CleanupConnection implements the ConnectionCleanup interface
@ -42,17 +44,29 @@ func (s *KevoServiceServer) CleanupConnection(connectionID string) {
}
}
// ReplicationInfoProvider defines an interface for accessing replication topology information
type ReplicationInfoProvider interface {
// GetNodeInfo returns information about the replication topology
// Returns: nodeRole, primaryAddr, replicas, lastSequence, readOnly
GetNodeInfo() (string, string, []ReplicaInfo, uint64, bool)
}
// ReplicaInfo contains information about a replica node
// This should mirror the structure in pkg/replication/info_provider.go
type ReplicaInfo = replication.ReplicationNodeInfo
// NewKevoServiceServer creates a new KevoServiceServer
func NewKevoServiceServer(engine interfaces.Engine, txRegistry TxRegistry) *KevoServiceServer {
func NewKevoServiceServer(engine interfaces.Engine, txRegistry TxRegistry, replicationManager ReplicationInfoProvider) *KevoServiceServer {
return &KevoServiceServer{
engine: engine,
txRegistry: txRegistry,
compactionSem: make(chan struct{}, 1), // Allow only one compaction at a time
maxKeySize: 4096, // 4KB
maxValueSize: 10 * 1024 * 1024, // 10MB
maxBatchSize: 1000,
maxTransactions: 1000,
transactionTTL: 300, // 5 minutes
engine: engine,
txRegistry: txRegistry,
replicationManager: replicationManager,
compactionSem: make(chan struct{}, 1), // Allow only one compaction at a time
maxKeySize: 4096, // 4KB
maxValueSize: 10 * 1024 * 1024, // 10MB
maxBatchSize: 1000,
maxTransactions: 1000,
transactionTTL: 300, // 5 minutes
}
}
@ -790,3 +804,50 @@ func (s *KevoServiceServer) Compact(ctx context.Context, req *pb.CompactRequest)
return &pb.CompactResponse{Success: true}, nil
}
// GetNodeInfo returns information about this node and the replication topology
func (s *KevoServiceServer) GetNodeInfo(ctx context.Context, req *pb.GetNodeInfoRequest) (*pb.GetNodeInfoResponse, error) {
response := &pb.GetNodeInfoResponse{
NodeRole: pb.GetNodeInfoResponse_STANDALONE, // Default to standalone
ReadOnly: false,
PrimaryAddress: "",
Replicas: nil,
}
// Check if we can access replication information
if s.replicationManager != nil {
// Get node role and replication info from the manager
nodeRole, primaryAddr, replicas, lastSeq, readOnly := s.replicationManager.GetNodeInfo()
// Set node role
switch nodeRole {
case "primary":
response.NodeRole = pb.GetNodeInfoResponse_PRIMARY
case "replica":
response.NodeRole = pb.GetNodeInfoResponse_REPLICA
default:
response.NodeRole = pb.GetNodeInfoResponse_STANDALONE
}
// Set primary address if available
response.PrimaryAddress = primaryAddr
// Set replicas information
for _, replica := range replicas {
replicaInfo := &pb.ReplicaInfo{
Address: replica.Address,
LastSequence: replica.LastSequence,
Available: replica.Available,
Region: replica.Region,
Meta: replica.Meta,
}
response.Replicas = append(response.Replicas, replicaInfo)
}
// Set sequence and read-only status
response.LastSequence = lastSeq
response.ReadOnly = readOnly
}
return response, nil
}

View File

@ -0,0 +1,56 @@
package replication
// ReplicationNodeInfo contains information about a node in the replication topology
type ReplicationNodeInfo struct {
Address string // Host:port of the node
LastSequence uint64 // Last applied sequence number
Available bool // Whether the node is available
Region string // Optional region information
Meta map[string]string // Additional metadata
}
// GetNodeInfo exposes replication topology information to the client service
func (m *Manager) GetNodeInfo() (string, string, []ReplicationNodeInfo, uint64, bool) {
// Return information about the current node and replication topology
var role string
var primaryAddr string
var replicas []ReplicationNodeInfo
var lastSequence uint64
m.mu.RLock()
defer m.mu.RUnlock()
// Set role
role = m.config.Mode
// Set primary address
if role == "replica" {
primaryAddr = m.config.PrimaryAddr
} else if role == "primary" {
primaryAddr = m.config.ListenAddr
}
// Set last sequence
if role == "primary" && m.primary != nil {
lastSequence = m.primary.GetLastSequence()
} else if role == "replica" && m.replica != nil {
lastSequence = m.replica.GetLastAppliedSequence()
}
// Gather replica information
if role == "primary" && m.primary != nil {
// Get replica sessions from primary
replicas = m.primary.GetReplicaInfo()
} else if role == "replica" {
// Add self as a replica
replicas = append(replicas, ReplicationNodeInfo{
Address: m.config.ListenAddr,
LastSequence: lastSequence,
Available: true,
Region: "",
Meta: map[string]string{},
})
}
return role, primaryAddr, replicas, lastSequence, m.engine.IsReadOnly()
}

View File

@ -23,6 +23,10 @@ func (m *MockEngine) SetReadOnly(readOnly bool) {
m.readOnly = readOnly
}
func (m *MockEngine) IsReadOnly() bool {
return m.readOnly
}
func (m *MockEngine) FlushImMemTables() error {
return nil
}

View File

@ -0,0 +1,35 @@
package replication
// GetReplicaInfo returns information about all connected replicas
func (p *Primary) GetReplicaInfo() []ReplicationNodeInfo {
p.mu.RLock()
defer p.mu.RUnlock()
var replicas []ReplicationNodeInfo
// Convert replica sessions to ReplicationNodeInfo
for _, session := range p.sessions {
if !session.Connected {
continue
}
replica := ReplicationNodeInfo{
Address: session.ID, // We don't have actual address, so use ID
LastSequence: session.LastAckSequence,
Available: session.Active,
Region: "",
Meta: map[string]string{},
}
replicas = append(replicas, replica)
}
return replicas
}
// GetLastSequence returns the highest sequence number that has been synced to disk
func (p *Primary) GetLastSequence() uint64 {
p.mu.RLock()
defer p.mu.RUnlock()
return p.lastSyncedSeq
}

View File

@ -67,6 +67,56 @@ func (Operation_Type) EnumDescriptor() ([]byte, []int) {
return file_proto_kevo_service_proto_rawDescGZIP(), []int{7, 0}
}
// Node role information
type GetNodeInfoResponse_NodeRole int32
const (
GetNodeInfoResponse_STANDALONE GetNodeInfoResponse_NodeRole = 0
GetNodeInfoResponse_PRIMARY GetNodeInfoResponse_NodeRole = 1
GetNodeInfoResponse_REPLICA GetNodeInfoResponse_NodeRole = 2
)
// Enum value maps for GetNodeInfoResponse_NodeRole.
var (
GetNodeInfoResponse_NodeRole_name = map[int32]string{
0: "STANDALONE",
1: "PRIMARY",
2: "REPLICA",
}
GetNodeInfoResponse_NodeRole_value = map[string]int32{
"STANDALONE": 0,
"PRIMARY": 1,
"REPLICA": 2,
}
)
func (x GetNodeInfoResponse_NodeRole) Enum() *GetNodeInfoResponse_NodeRole {
p := new(GetNodeInfoResponse_NodeRole)
*p = x
return p
}
func (x GetNodeInfoResponse_NodeRole) String() string {
return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
}
func (GetNodeInfoResponse_NodeRole) Descriptor() protoreflect.EnumDescriptor {
return file_proto_kevo_service_proto_enumTypes[1].Descriptor()
}
func (GetNodeInfoResponse_NodeRole) Type() protoreflect.EnumType {
return &file_proto_kevo_service_proto_enumTypes[1]
}
func (x GetNodeInfoResponse_NodeRole) Number() protoreflect.EnumNumber {
return protoreflect.EnumNumber(x)
}
// Deprecated: Use GetNodeInfoResponse_NodeRole.Descriptor instead.
func (GetNodeInfoResponse_NodeRole) EnumDescriptor() ([]byte, []int) {
return file_proto_kevo_service_proto_rawDescGZIP(), []int{32, 0}
}
// Basic message types
type GetRequest struct {
state protoimpl.MessageState `protogen:"open.v1"`
@ -1769,6 +1819,197 @@ func (x *CompactResponse) GetSuccess() bool {
return false
}
// Node information and topology
type GetNodeInfoRequest struct {
state protoimpl.MessageState `protogen:"open.v1"`
unknownFields protoimpl.UnknownFields
sizeCache protoimpl.SizeCache
}
func (x *GetNodeInfoRequest) Reset() {
*x = GetNodeInfoRequest{}
mi := &file_proto_kevo_service_proto_msgTypes[31]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
func (x *GetNodeInfoRequest) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*GetNodeInfoRequest) ProtoMessage() {}
func (x *GetNodeInfoRequest) ProtoReflect() protoreflect.Message {
mi := &file_proto_kevo_service_proto_msgTypes[31]
if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use GetNodeInfoRequest.ProtoReflect.Descriptor instead.
func (*GetNodeInfoRequest) Descriptor() ([]byte, []int) {
return file_proto_kevo_service_proto_rawDescGZIP(), []int{31}
}
type GetNodeInfoResponse struct {
state protoimpl.MessageState `protogen:"open.v1"`
NodeRole GetNodeInfoResponse_NodeRole `protobuf:"varint,1,opt,name=node_role,json=nodeRole,proto3,enum=kevo.GetNodeInfoResponse_NodeRole" json:"node_role,omitempty"`
// Connection information
PrimaryAddress string `protobuf:"bytes,2,opt,name=primary_address,json=primaryAddress,proto3" json:"primary_address,omitempty"` // Empty if standalone
Replicas []*ReplicaInfo `protobuf:"bytes,3,rep,name=replicas,proto3" json:"replicas,omitempty"` // Empty if standalone
// Node status
LastSequence uint64 `protobuf:"varint,4,opt,name=last_sequence,json=lastSequence,proto3" json:"last_sequence,omitempty"` // Last applied sequence number
ReadOnly bool `protobuf:"varint,5,opt,name=read_only,json=readOnly,proto3" json:"read_only,omitempty"` // Whether the node is in read-only mode
unknownFields protoimpl.UnknownFields
sizeCache protoimpl.SizeCache
}
func (x *GetNodeInfoResponse) Reset() {
*x = GetNodeInfoResponse{}
mi := &file_proto_kevo_service_proto_msgTypes[32]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
func (x *GetNodeInfoResponse) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*GetNodeInfoResponse) ProtoMessage() {}
func (x *GetNodeInfoResponse) ProtoReflect() protoreflect.Message {
mi := &file_proto_kevo_service_proto_msgTypes[32]
if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use GetNodeInfoResponse.ProtoReflect.Descriptor instead.
func (*GetNodeInfoResponse) Descriptor() ([]byte, []int) {
return file_proto_kevo_service_proto_rawDescGZIP(), []int{32}
}
func (x *GetNodeInfoResponse) GetNodeRole() GetNodeInfoResponse_NodeRole {
if x != nil {
return x.NodeRole
}
return GetNodeInfoResponse_STANDALONE
}
func (x *GetNodeInfoResponse) GetPrimaryAddress() string {
if x != nil {
return x.PrimaryAddress
}
return ""
}
func (x *GetNodeInfoResponse) GetReplicas() []*ReplicaInfo {
if x != nil {
return x.Replicas
}
return nil
}
func (x *GetNodeInfoResponse) GetLastSequence() uint64 {
if x != nil {
return x.LastSequence
}
return 0
}
func (x *GetNodeInfoResponse) GetReadOnly() bool {
if x != nil {
return x.ReadOnly
}
return false
}
type ReplicaInfo struct {
state protoimpl.MessageState `protogen:"open.v1"`
Address string `protobuf:"bytes,1,opt,name=address,proto3" json:"address,omitempty"` // Host:port of the replica
LastSequence uint64 `protobuf:"varint,2,opt,name=last_sequence,json=lastSequence,proto3" json:"last_sequence,omitempty"` // Last applied sequence number
Available bool `protobuf:"varint,3,opt,name=available,proto3" json:"available,omitempty"` // Whether the replica is available
Region string `protobuf:"bytes,4,opt,name=region,proto3" json:"region,omitempty"` // Optional region information
Meta map[string]string `protobuf:"bytes,5,rep,name=meta,proto3" json:"meta,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` // Additional metadata
unknownFields protoimpl.UnknownFields
sizeCache protoimpl.SizeCache
}
func (x *ReplicaInfo) Reset() {
*x = ReplicaInfo{}
mi := &file_proto_kevo_service_proto_msgTypes[33]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
func (x *ReplicaInfo) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*ReplicaInfo) ProtoMessage() {}
func (x *ReplicaInfo) ProtoReflect() protoreflect.Message {
mi := &file_proto_kevo_service_proto_msgTypes[33]
if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use ReplicaInfo.ProtoReflect.Descriptor instead.
func (*ReplicaInfo) Descriptor() ([]byte, []int) {
return file_proto_kevo_service_proto_rawDescGZIP(), []int{33}
}
func (x *ReplicaInfo) GetAddress() string {
if x != nil {
return x.Address
}
return ""
}
func (x *ReplicaInfo) GetLastSequence() uint64 {
if x != nil {
return x.LastSequence
}
return 0
}
func (x *ReplicaInfo) GetAvailable() bool {
if x != nil {
return x.Available
}
return false
}
func (x *ReplicaInfo) GetRegion() string {
if x != nil {
return x.Region
}
return ""
}
func (x *ReplicaInfo) GetMeta() map[string]string {
if x != nil {
return x.Meta
}
return nil
}
var File_proto_kevo_service_proto protoreflect.FileDescriptor
const file_proto_kevo_service_proto_rawDesc = "" +
@ -1895,7 +2136,28 @@ const file_proto_kevo_service_proto_rawDesc = "" +
"\x0eCompactRequest\x12\x14\n" +
"\x05force\x18\x01 \x01(\bR\x05force\"+\n" +
"\x0fCompactResponse\x12\x18\n" +
"\asuccess\x18\x01 \x01(\bR\asuccess2\xda\x06\n" +
"\asuccess\x18\x01 \x01(\bR\asuccess\"\x14\n" +
"\x12GetNodeInfoRequest\"\xa6\x02\n" +
"\x13GetNodeInfoResponse\x12?\n" +
"\tnode_role\x18\x01 \x01(\x0e2\".kevo.GetNodeInfoResponse.NodeRoleR\bnodeRole\x12'\n" +
"\x0fprimary_address\x18\x02 \x01(\tR\x0eprimaryAddress\x12-\n" +
"\breplicas\x18\x03 \x03(\v2\x11.kevo.ReplicaInfoR\breplicas\x12#\n" +
"\rlast_sequence\x18\x04 \x01(\x04R\flastSequence\x12\x1b\n" +
"\tread_only\x18\x05 \x01(\bR\breadOnly\"4\n" +
"\bNodeRole\x12\x0e\n" +
"\n" +
"STANDALONE\x10\x00\x12\v\n" +
"\aPRIMARY\x10\x01\x12\v\n" +
"\aREPLICA\x10\x02\"\xec\x01\n" +
"\vReplicaInfo\x12\x18\n" +
"\aaddress\x18\x01 \x01(\tR\aaddress\x12#\n" +
"\rlast_sequence\x18\x02 \x01(\x04R\flastSequence\x12\x1c\n" +
"\tavailable\x18\x03 \x01(\bR\tavailable\x12\x16\n" +
"\x06region\x18\x04 \x01(\tR\x06region\x12/\n" +
"\x04meta\x18\x05 \x03(\v2\x1b.kevo.ReplicaInfo.MetaEntryR\x04meta\x1a7\n" +
"\tMetaEntry\x12\x10\n" +
"\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n" +
"\x05value\x18\x02 \x01(\tR\x05value:\x028\x012\x9e\a\n" +
"\vKevoService\x12*\n" +
"\x03Get\x12\x10.kevo.GetRequest\x1a\x11.kevo.GetResponse\x12*\n" +
"\x03Put\x12\x10.kevo.PutRequest\x1a\x11.kevo.PutResponse\x123\n" +
@ -1911,7 +2173,8 @@ 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.CompactResponse\x12B\n" +
"\vGetNodeInfo\x12\x18.kevo.GetNodeInfoRequest\x1a\x19.kevo.GetNodeInfoResponseB5Z3github.com/jeremytregunna/kevo/pkg/grpc/proto;protob\x06proto3"
var (
file_proto_kevo_service_proto_rawDescOnce sync.Once
@ -1925,86 +2188,96 @@ func file_proto_kevo_service_proto_rawDescGZIP() []byte {
return file_proto_kevo_service_proto_rawDescData
}
var file_proto_kevo_service_proto_enumTypes = make([]protoimpl.EnumInfo, 1)
var file_proto_kevo_service_proto_msgTypes = make([]protoimpl.MessageInfo, 34)
var file_proto_kevo_service_proto_enumTypes = make([]protoimpl.EnumInfo, 2)
var file_proto_kevo_service_proto_msgTypes = make([]protoimpl.MessageInfo, 38)
var file_proto_kevo_service_proto_goTypes = []any{
(Operation_Type)(0), // 0: kevo.Operation.Type
(*GetRequest)(nil), // 1: kevo.GetRequest
(*GetResponse)(nil), // 2: kevo.GetResponse
(*PutRequest)(nil), // 3: kevo.PutRequest
(*PutResponse)(nil), // 4: kevo.PutResponse
(*DeleteRequest)(nil), // 5: kevo.DeleteRequest
(*DeleteResponse)(nil), // 6: kevo.DeleteResponse
(*BatchWriteRequest)(nil), // 7: kevo.BatchWriteRequest
(*Operation)(nil), // 8: kevo.Operation
(*BatchWriteResponse)(nil), // 9: kevo.BatchWriteResponse
(*ScanRequest)(nil), // 10: kevo.ScanRequest
(*ScanResponse)(nil), // 11: kevo.ScanResponse
(*BeginTransactionRequest)(nil), // 12: kevo.BeginTransactionRequest
(*BeginTransactionResponse)(nil), // 13: kevo.BeginTransactionResponse
(*CommitTransactionRequest)(nil), // 14: kevo.CommitTransactionRequest
(*CommitTransactionResponse)(nil), // 15: kevo.CommitTransactionResponse
(*RollbackTransactionRequest)(nil), // 16: kevo.RollbackTransactionRequest
(*RollbackTransactionResponse)(nil), // 17: kevo.RollbackTransactionResponse
(*TxGetRequest)(nil), // 18: kevo.TxGetRequest
(*TxGetResponse)(nil), // 19: kevo.TxGetResponse
(*TxPutRequest)(nil), // 20: kevo.TxPutRequest
(*TxPutResponse)(nil), // 21: kevo.TxPutResponse
(*TxDeleteRequest)(nil), // 22: kevo.TxDeleteRequest
(*TxDeleteResponse)(nil), // 23: kevo.TxDeleteResponse
(*TxScanRequest)(nil), // 24: kevo.TxScanRequest
(*TxScanResponse)(nil), // 25: kevo.TxScanResponse
(*GetStatsRequest)(nil), // 26: kevo.GetStatsRequest
(*GetStatsResponse)(nil), // 27: kevo.GetStatsResponse
(*LatencyStats)(nil), // 28: kevo.LatencyStats
(*RecoveryStats)(nil), // 29: kevo.RecoveryStats
(*CompactRequest)(nil), // 30: kevo.CompactRequest
(*CompactResponse)(nil), // 31: kevo.CompactResponse
nil, // 32: kevo.GetStatsResponse.OperationCountsEntry
nil, // 33: kevo.GetStatsResponse.LatencyStatsEntry
nil, // 34: kevo.GetStatsResponse.ErrorCountsEntry
(GetNodeInfoResponse_NodeRole)(0), // 1: kevo.GetNodeInfoResponse.NodeRole
(*GetRequest)(nil), // 2: kevo.GetRequest
(*GetResponse)(nil), // 3: kevo.GetResponse
(*PutRequest)(nil), // 4: kevo.PutRequest
(*PutResponse)(nil), // 5: kevo.PutResponse
(*DeleteRequest)(nil), // 6: kevo.DeleteRequest
(*DeleteResponse)(nil), // 7: kevo.DeleteResponse
(*BatchWriteRequest)(nil), // 8: kevo.BatchWriteRequest
(*Operation)(nil), // 9: kevo.Operation
(*BatchWriteResponse)(nil), // 10: kevo.BatchWriteResponse
(*ScanRequest)(nil), // 11: kevo.ScanRequest
(*ScanResponse)(nil), // 12: kevo.ScanResponse
(*BeginTransactionRequest)(nil), // 13: kevo.BeginTransactionRequest
(*BeginTransactionResponse)(nil), // 14: kevo.BeginTransactionResponse
(*CommitTransactionRequest)(nil), // 15: kevo.CommitTransactionRequest
(*CommitTransactionResponse)(nil), // 16: kevo.CommitTransactionResponse
(*RollbackTransactionRequest)(nil), // 17: kevo.RollbackTransactionRequest
(*RollbackTransactionResponse)(nil), // 18: kevo.RollbackTransactionResponse
(*TxGetRequest)(nil), // 19: kevo.TxGetRequest
(*TxGetResponse)(nil), // 20: kevo.TxGetResponse
(*TxPutRequest)(nil), // 21: kevo.TxPutRequest
(*TxPutResponse)(nil), // 22: kevo.TxPutResponse
(*TxDeleteRequest)(nil), // 23: kevo.TxDeleteRequest
(*TxDeleteResponse)(nil), // 24: kevo.TxDeleteResponse
(*TxScanRequest)(nil), // 25: kevo.TxScanRequest
(*TxScanResponse)(nil), // 26: kevo.TxScanResponse
(*GetStatsRequest)(nil), // 27: kevo.GetStatsRequest
(*GetStatsResponse)(nil), // 28: kevo.GetStatsResponse
(*LatencyStats)(nil), // 29: kevo.LatencyStats
(*RecoveryStats)(nil), // 30: kevo.RecoveryStats
(*CompactRequest)(nil), // 31: kevo.CompactRequest
(*CompactResponse)(nil), // 32: kevo.CompactResponse
(*GetNodeInfoRequest)(nil), // 33: kevo.GetNodeInfoRequest
(*GetNodeInfoResponse)(nil), // 34: kevo.GetNodeInfoResponse
(*ReplicaInfo)(nil), // 35: kevo.ReplicaInfo
nil, // 36: kevo.GetStatsResponse.OperationCountsEntry
nil, // 37: kevo.GetStatsResponse.LatencyStatsEntry
nil, // 38: kevo.GetStatsResponse.ErrorCountsEntry
nil, // 39: kevo.ReplicaInfo.MetaEntry
}
var file_proto_kevo_service_proto_depIdxs = []int32{
8, // 0: kevo.BatchWriteRequest.operations:type_name -> kevo.Operation
9, // 0: kevo.BatchWriteRequest.operations:type_name -> kevo.Operation
0, // 1: kevo.Operation.type:type_name -> kevo.Operation.Type
32, // 2: kevo.GetStatsResponse.operation_counts:type_name -> kevo.GetStatsResponse.OperationCountsEntry
33, // 3: kevo.GetStatsResponse.latency_stats:type_name -> kevo.GetStatsResponse.LatencyStatsEntry
34, // 4: kevo.GetStatsResponse.error_counts:type_name -> kevo.GetStatsResponse.ErrorCountsEntry
29, // 5: kevo.GetStatsResponse.recovery_stats:type_name -> kevo.RecoveryStats
28, // 6: kevo.GetStatsResponse.LatencyStatsEntry.value:type_name -> kevo.LatencyStats
1, // 7: kevo.KevoService.Get:input_type -> kevo.GetRequest
3, // 8: kevo.KevoService.Put:input_type -> kevo.PutRequest
5, // 9: kevo.KevoService.Delete:input_type -> kevo.DeleteRequest
7, // 10: kevo.KevoService.BatchWrite:input_type -> kevo.BatchWriteRequest
10, // 11: kevo.KevoService.Scan:input_type -> kevo.ScanRequest
12, // 12: kevo.KevoService.BeginTransaction:input_type -> kevo.BeginTransactionRequest
14, // 13: kevo.KevoService.CommitTransaction:input_type -> kevo.CommitTransactionRequest
16, // 14: kevo.KevoService.RollbackTransaction:input_type -> kevo.RollbackTransactionRequest
18, // 15: kevo.KevoService.TxGet:input_type -> kevo.TxGetRequest
20, // 16: kevo.KevoService.TxPut:input_type -> kevo.TxPutRequest
22, // 17: kevo.KevoService.TxDelete:input_type -> kevo.TxDeleteRequest
24, // 18: kevo.KevoService.TxScan:input_type -> kevo.TxScanRequest
26, // 19: kevo.KevoService.GetStats:input_type -> kevo.GetStatsRequest
30, // 20: kevo.KevoService.Compact:input_type -> kevo.CompactRequest
2, // 21: kevo.KevoService.Get:output_type -> kevo.GetResponse
4, // 22: kevo.KevoService.Put:output_type -> kevo.PutResponse
6, // 23: kevo.KevoService.Delete:output_type -> kevo.DeleteResponse
9, // 24: kevo.KevoService.BatchWrite:output_type -> kevo.BatchWriteResponse
11, // 25: kevo.KevoService.Scan:output_type -> kevo.ScanResponse
13, // 26: kevo.KevoService.BeginTransaction:output_type -> kevo.BeginTransactionResponse
15, // 27: kevo.KevoService.CommitTransaction:output_type -> kevo.CommitTransactionResponse
17, // 28: kevo.KevoService.RollbackTransaction:output_type -> kevo.RollbackTransactionResponse
19, // 29: kevo.KevoService.TxGet:output_type -> kevo.TxGetResponse
21, // 30: kevo.KevoService.TxPut:output_type -> kevo.TxPutResponse
23, // 31: kevo.KevoService.TxDelete:output_type -> kevo.TxDeleteResponse
25, // 32: kevo.KevoService.TxScan:output_type -> kevo.TxScanResponse
27, // 33: kevo.KevoService.GetStats:output_type -> kevo.GetStatsResponse
31, // 34: kevo.KevoService.Compact:output_type -> kevo.CompactResponse
21, // [21:35] is the sub-list for method output_type
7, // [7:21] is the sub-list for method input_type
7, // [7:7] is the sub-list for extension type_name
7, // [7:7] is the sub-list for extension extendee
0, // [0:7] is the sub-list for field type_name
36, // 2: kevo.GetStatsResponse.operation_counts:type_name -> kevo.GetStatsResponse.OperationCountsEntry
37, // 3: kevo.GetStatsResponse.latency_stats:type_name -> kevo.GetStatsResponse.LatencyStatsEntry
38, // 4: kevo.GetStatsResponse.error_counts:type_name -> kevo.GetStatsResponse.ErrorCountsEntry
30, // 5: kevo.GetStatsResponse.recovery_stats:type_name -> kevo.RecoveryStats
1, // 6: kevo.GetNodeInfoResponse.node_role:type_name -> kevo.GetNodeInfoResponse.NodeRole
35, // 7: kevo.GetNodeInfoResponse.replicas:type_name -> kevo.ReplicaInfo
39, // 8: kevo.ReplicaInfo.meta:type_name -> kevo.ReplicaInfo.MetaEntry
29, // 9: kevo.GetStatsResponse.LatencyStatsEntry.value:type_name -> kevo.LatencyStats
2, // 10: kevo.KevoService.Get:input_type -> kevo.GetRequest
4, // 11: kevo.KevoService.Put:input_type -> kevo.PutRequest
6, // 12: kevo.KevoService.Delete:input_type -> kevo.DeleteRequest
8, // 13: kevo.KevoService.BatchWrite:input_type -> kevo.BatchWriteRequest
11, // 14: kevo.KevoService.Scan:input_type -> kevo.ScanRequest
13, // 15: kevo.KevoService.BeginTransaction:input_type -> kevo.BeginTransactionRequest
15, // 16: kevo.KevoService.CommitTransaction:input_type -> kevo.CommitTransactionRequest
17, // 17: kevo.KevoService.RollbackTransaction:input_type -> kevo.RollbackTransactionRequest
19, // 18: kevo.KevoService.TxGet:input_type -> kevo.TxGetRequest
21, // 19: kevo.KevoService.TxPut:input_type -> kevo.TxPutRequest
23, // 20: kevo.KevoService.TxDelete:input_type -> kevo.TxDeleteRequest
25, // 21: kevo.KevoService.TxScan:input_type -> kevo.TxScanRequest
27, // 22: kevo.KevoService.GetStats:input_type -> kevo.GetStatsRequest
31, // 23: kevo.KevoService.Compact:input_type -> kevo.CompactRequest
33, // 24: kevo.KevoService.GetNodeInfo:input_type -> kevo.GetNodeInfoRequest
3, // 25: kevo.KevoService.Get:output_type -> kevo.GetResponse
5, // 26: kevo.KevoService.Put:output_type -> kevo.PutResponse
7, // 27: kevo.KevoService.Delete:output_type -> kevo.DeleteResponse
10, // 28: kevo.KevoService.BatchWrite:output_type -> kevo.BatchWriteResponse
12, // 29: kevo.KevoService.Scan:output_type -> kevo.ScanResponse
14, // 30: kevo.KevoService.BeginTransaction:output_type -> kevo.BeginTransactionResponse
16, // 31: kevo.KevoService.CommitTransaction:output_type -> kevo.CommitTransactionResponse
18, // 32: kevo.KevoService.RollbackTransaction:output_type -> kevo.RollbackTransactionResponse
20, // 33: kevo.KevoService.TxGet:output_type -> kevo.TxGetResponse
22, // 34: kevo.KevoService.TxPut:output_type -> kevo.TxPutResponse
24, // 35: kevo.KevoService.TxDelete:output_type -> kevo.TxDeleteResponse
26, // 36: kevo.KevoService.TxScan:output_type -> kevo.TxScanResponse
28, // 37: kevo.KevoService.GetStats:output_type -> kevo.GetStatsResponse
32, // 38: kevo.KevoService.Compact:output_type -> kevo.CompactResponse
34, // 39: kevo.KevoService.GetNodeInfo:output_type -> kevo.GetNodeInfoResponse
25, // [25:40] is the sub-list for method output_type
10, // [10:25] is the sub-list for method input_type
10, // [10:10] is the sub-list for extension type_name
10, // [10:10] is the sub-list for extension extendee
0, // [0:10] is the sub-list for field type_name
}
func init() { file_proto_kevo_service_proto_init() }
@ -2017,8 +2290,8 @@ func file_proto_kevo_service_proto_init() {
File: protoimpl.DescBuilder{
GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
RawDescriptor: unsafe.Slice(unsafe.StringData(file_proto_kevo_service_proto_rawDesc), len(file_proto_kevo_service_proto_rawDesc)),
NumEnums: 1,
NumMessages: 34,
NumEnums: 2,
NumMessages: 38,
NumExtensions: 0,
NumServices: 1,
},

View File

@ -30,6 +30,9 @@ service KevoService {
// Administrative Operations
rpc GetStats(GetStatsRequest) returns (GetStatsResponse);
rpc Compact(CompactRequest) returns (CompactResponse);
// Replication and Topology Operations
rpc GetNodeInfo(GetNodeInfoRequest) returns (GetNodeInfoResponse);
}
// Basic message types
@ -209,4 +212,35 @@ message CompactRequest {
message CompactResponse {
bool success = 1;
}
// Node information and topology
message GetNodeInfoRequest {
// No parameters needed for now
}
message GetNodeInfoResponse {
// Node role information
enum NodeRole {
STANDALONE = 0;
PRIMARY = 1;
REPLICA = 2;
}
NodeRole node_role = 1;
// Connection information
string primary_address = 2; // Empty if standalone
repeated ReplicaInfo replicas = 3; // Empty if standalone
// Node status
uint64 last_sequence = 4; // Last applied sequence number
bool read_only = 5; // Whether the node is in read-only mode
}
message ReplicaInfo {
string address = 1; // Host:port of the replica
uint64 last_sequence = 2; // Last applied sequence number
bool available = 3; // Whether the replica is available
string region = 4; // Optional region information
map<string, string> meta = 5; // Additional metadata
}

View File

@ -33,6 +33,7 @@ const (
KevoService_TxScan_FullMethodName = "/kevo.KevoService/TxScan"
KevoService_GetStats_FullMethodName = "/kevo.KevoService/GetStats"
KevoService_Compact_FullMethodName = "/kevo.KevoService/Compact"
KevoService_GetNodeInfo_FullMethodName = "/kevo.KevoService/GetNodeInfo"
)
// KevoServiceClient is the client API for KevoService service.
@ -59,6 +60,8 @@ type KevoServiceClient interface {
// Administrative Operations
GetStats(ctx context.Context, in *GetStatsRequest, opts ...grpc.CallOption) (*GetStatsResponse, error)
Compact(ctx context.Context, in *CompactRequest, opts ...grpc.CallOption) (*CompactResponse, error)
// Replication and Topology Operations
GetNodeInfo(ctx context.Context, in *GetNodeInfoRequest, opts ...grpc.CallOption) (*GetNodeInfoResponse, error)
}
type kevoServiceClient struct {
@ -227,6 +230,16 @@ func (c *kevoServiceClient) Compact(ctx context.Context, in *CompactRequest, opt
return out, nil
}
func (c *kevoServiceClient) GetNodeInfo(ctx context.Context, in *GetNodeInfoRequest, opts ...grpc.CallOption) (*GetNodeInfoResponse, error) {
cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...)
out := new(GetNodeInfoResponse)
err := c.cc.Invoke(ctx, KevoService_GetNodeInfo_FullMethodName, in, out, cOpts...)
if err != nil {
return nil, err
}
return out, nil
}
// KevoServiceServer is the server API for KevoService service.
// All implementations must embed UnimplementedKevoServiceServer
// for forward compatibility.
@ -251,6 +264,8 @@ type KevoServiceServer interface {
// Administrative Operations
GetStats(context.Context, *GetStatsRequest) (*GetStatsResponse, error)
Compact(context.Context, *CompactRequest) (*CompactResponse, error)
// Replication and Topology Operations
GetNodeInfo(context.Context, *GetNodeInfoRequest) (*GetNodeInfoResponse, error)
mustEmbedUnimplementedKevoServiceServer()
}
@ -303,6 +318,9 @@ func (UnimplementedKevoServiceServer) GetStats(context.Context, *GetStatsRequest
func (UnimplementedKevoServiceServer) Compact(context.Context, *CompactRequest) (*CompactResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method Compact not implemented")
}
func (UnimplementedKevoServiceServer) GetNodeInfo(context.Context, *GetNodeInfoRequest) (*GetNodeInfoResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method GetNodeInfo not implemented")
}
func (UnimplementedKevoServiceServer) mustEmbedUnimplementedKevoServiceServer() {}
func (UnimplementedKevoServiceServer) testEmbeddedByValue() {}
@ -562,6 +580,24 @@ func _KevoService_Compact_Handler(srv interface{}, ctx context.Context, dec func
return interceptor(ctx, in, info, handler)
}
func _KevoService_GetNodeInfo_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(GetNodeInfoRequest)
if err := dec(in); err != nil {
return nil, err
}
if interceptor == nil {
return srv.(KevoServiceServer).GetNodeInfo(ctx, in)
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: KevoService_GetNodeInfo_FullMethodName,
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(KevoServiceServer).GetNodeInfo(ctx, req.(*GetNodeInfoRequest))
}
return interceptor(ctx, in, info, handler)
}
// KevoService_ServiceDesc is the grpc.ServiceDesc for KevoService service.
// It's only intended for direct use with grpc.RegisterService,
// and not to be introspected or modified (even as a copy)
@ -617,6 +653,10 @@ var KevoService_ServiceDesc = grpc.ServiceDesc{
MethodName: "Compact",
Handler: _KevoService_Compact_Handler,
},
{
MethodName: "GetNodeInfo",
Handler: _KevoService_GetNodeInfo_Handler,
},
},
Streams: []grpc.StreamDesc{
{