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:
parent
4429836929
commit
f9e332096c
@ -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
|
||||
}
|
||||
|
@ -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))
|
||||
}
|
||||
}
|
||||
|
@ -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"
|
||||
}
|
||||
|
132
pkg/client/replication_test.go
Normal file
132
pkg/client/replication_test.go
Normal 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")
|
||||
}
|
||||
}
|
@ -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
|
||||
|
@ -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
|
||||
}
|
||||
|
||||
|
@ -9,6 +9,6 @@ import (
|
||||
func (m *Manager) GetWAL() *wal.WAL {
|
||||
m.mu.RLock()
|
||||
defer m.mu.RUnlock()
|
||||
|
||||
|
||||
return m.wal
|
||||
}
|
||||
}
|
||||
|
@ -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
|
||||
}
|
||||
|
56
pkg/replication/info_provider.go
Normal file
56
pkg/replication/info_provider.go
Normal 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()
|
||||
}
|
@ -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
|
||||
}
|
||||
|
35
pkg/replication/primary_info.go
Normal file
35
pkg/replication/primary_info.go
Normal 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
|
||||
}
|
@ -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,
|
||||
},
|
||||
|
@ -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
|
||||
}
|
@ -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{
|
||||
{
|
||||
|
Loading…
Reference in New Issue
Block a user