Skip to content

Commit c94c475

Browse files
authored
feat(raft): Implement Raft-based Consistent Hash State Management (#636)
* feat: Add Raft consensus for consistent hashing This commit introduces Raft consensus to maintain consistency of hash-to-proxy mappings across multiple GatewayD instances. Key changes include: - Add new Raft package implementing consensus protocol using HashiCorp's Raft - Integrate Raft with consistent hashing load balancer - Store proxy mappings in distributed state machine - Add configuration options for Raft cluster setup - Implement leadership monitoring and peer management - Add FSM snapshot and restore capabilities The implementation ensures that hash-to-proxy mappings remain consistent across cluster nodes, improving reliability for consistent hash-based load balancing. * refactor: update consistent hash implementation with block-based proxy mapping - Replace proxy ID with block name for consistent hash mapping - Remove direct raft node dependency from ConsistentHash struct - Add ProxyByBlock map to Server for block-based proxy lookups - Include group name in hash key generation for better distribution - Add proxy initialization during server startup - Update FSM to use consistent naming for hash map storage This change improves the consistent hashing mechanism by using block names instead of proxy IDs, making it more aligned with the block-based architecture while maintaining backwards compatibility with the original load balancing strategy. * refactor: remove proxy ID and related functionality - Remove unused UUID-based ID field from Proxy struct - Remove GetID() method from IProxy interface and Proxy implementation - Remove GetProxyByID() method from Server struct - Remove uuid package dependency The proxy ID was not being used meaningfully in the codebase, so removing it simplifies the proxy implementation. * feat(raft): Add Raft integration tests and consistent hash improvements This commit introduces comprehensive Raft testing infrastructure and enhances the consistent hash implementation with distributed state management. Key changes: - Add new test cases for Raft leadership, follower behavior, and FSM operations - Integrate Raft with consistent hash load balancer for distributed state - Add TestRaftHelper utility for simplified Raft testing setup - Update consistent hash tests to use Raft for state persistence - Add GetState method to RaftNode for state inspection - Improve test coverage for concurrent operations The changes ensure that proxy mappings are consistently maintained across the cluster using Raft consensus, making the load balancer more reliable in distributed environments. * feat(raft): add configurable directory and improve test stability - Add Directory field to Raft config to make raft storage location configurable - Use t.TempDir() in tests to ensure proper cleanup of test directories - Rename HashMapCommand to ConsistentHashCommand for better clarity - Update command type constants and map names to be more descriptive - Fix test flakiness by using unique node IDs and random available ports - Remove manual directory cleanup in favor of t.TempDir() cleanup - Update configuration files with raft directory settings This change improves test stability and makes the raft storage location configurable while cleaning up naming conventions throughout the raft package. * feat(config): add default Raft configuration values Add default configuration values for Raft consensus implementation: - RaftAddress: 127.0.0.1:2223 - RaftNodeID: node1 - RaftLeaderID: node1 - RaftDirectory: raft This change initializes the default Raft configuration in the config loader. * refactor(raft): improve error handling and code organization - Enhance error handling with wrapped errors and detailed messages - Add meaningful constants for timeouts and configuration values - Rename RaftNode to Node for better clarity - Fix JSON field names to match Raft convention (nodeId, leaderId) - Add missing error checks in critical paths - Improve documentation and code comments - Update golangci linter settings to include raft package * Add temporary directory for Raft in Test_pluginScaffoldCmd - Introduced a temporary directory for Raft using t.TempDir() in the Test_pluginScaffoldCmd test case. - Set the GATEWAYD_RAFT_DIRECTORY environment variable to the new temporary directory. - This change ensures that Raft operations during testing are isolated and do not interfere with other tests or system directories. * feat(config): add JSON parsing for raft peers env variable - Replace loadEnvVars with loadEnvVarsWithTransform to handle complex env values - Add special handling for raft.peers to parse JSON array into RaftPeer structs - Update GlobalKoanf and PluginKoanf to use new transformer function This change allows proper parsing of list-type environment variables, specifically for raft peer configurations. * Add GRPC to raft Add gRPC support to the Raft implementation to enable proper request forwarding between nodes. Changes include: - Add protobuf definitions for Raft service with ForwardApply RPC - Add gRPC server and client implementations for Raft nodes - Update Raft configuration to include gRPC addresses - Implement request forwarding logic for non-leader nodes - Update node configuration to handle gRPC connections - Add proper cleanup of gRPC resources during shutdown The changes enable proper forwarding of apply requests from follower nodes to the leader, improving the distributed consensus mechanism. * feat: add Docker Compose configuration for Raft cluster setup Add docker-compose-raft.yaml that configures a 3-node GatewayD cluster using Raft consensus protocol. The setup includes: - 3 GatewayD nodes with Raft configuration - Separate read/write PostgreSQL instances - Redis for caching - Observability stack (Prometheus, Tempo, Grafana) - Plugin installation service This configuration enables high availability and leader election through Raft consensus. * refactor(raft): improve error handling and code clarity - Improve variable naming in loadEnvVarsWithTransform for better readability - Clean up error handling in forwardToLeader and ForwardApply - Add proper error propagation in RPC responses - Fix string type conversions for peer IDs and addresses - Organize imports and add missing error package - Remove unused convertPeers function - Add clarifying comments for Apply methods This commit focuses on code quality improvements and better error handling in the Raft implementation without changing core functionality. * Add unit tests for Raft RPC server and client - Implement `TestRPCServer_ForwardApply` to test the `ForwardApply` method of the RPC server, ensuring correct handling of apply requests with various configurations. - Implement `TestRPCClient` to verify the creation and management of RPC clients, including client retrieval and connection closure. - Utilize `setupGRPCServer` to create a gRPC server for testing purposes. - Ensure proper setup and teardown of test nodes and gRPC connections to maintain test isolation and reliability. * Update Raft configuration in gatewayd.yaml - Change `nodeId` and `leaderId` from `node2` to `node1`. - Add `grpcAddress` with value `127.0.0.1:50051`. - Update `peers` to an empty list instead of an empty dictionary. These changes adjust the Raft configuration to reflect the new node setup and include a gRPC address for communication. * Convert RaftPeer slice to string for environment variable compatibility The function `v1.NewStruct(args)` only accepts `NewValue`, which requires converting certain types to strings. This change adds support for converting a slice of `config.RaftPeer` to a comma-separated string format. Each peer is formatted as "ID:Address:GRPCAddress". This conversion is necessary to overwrite the peers as an environment variable. * Update checksum in gatewayd_plugins.yaml - Updated the checksum value for the plugin configuration to ensure integrity and consistency with the latest changes. * Refactor Raft configuration to use `IsBootstrap` flag - Replaced `LeaderID` with `IsBootstrap` in Raft configuration across multiple files. - Updated YAML configuration files (`gatewayd.yaml`, `docker-compose-raft.yaml`) to reflect the new `IsBootstrap` flag. - Modified Go source files (`config.go`, `constants.go`, `types.go`, `raft.go`) to use `IsBootstrap` instead of `LeaderID`. - Adjusted test cases in `raft_test.go`, `rpc_test.go`, and `raft_helpers.go` to accommodate the new `IsBootstrap` flag. - Ensured that the `IsBootstrap` flag is correctly set for nodes intended to bootstrap the Raft cluster. * Increase the sleep time to pass the test case on the local machine. * fix: resolve lint issues in rpc_test.go - Added `t.Helper()` to `setupGRPCServer` and `setupNodes` functions to improve test helper identification. - Corrected variable naming in `TestRPCServer_ForwardApply` for clarity and consistency. - Ensured comments end with a period for consistency. - Updated assertions to use `GetSuccess()` method for better readability. * feat: Improve code readability with comments and updates - Updated Docker image references in `docker-compose-raft.yaml` to use `gatewaydio/gatewayd:latest` and added `pull_policy: always` for consistent image updates. - Changed server and API addresses in `gatewayd.yaml` for better port management. - Enhanced logging in `raft.go` by switching from `Info` to `Debug` for certain messages to reduce verbosity. - Added detailed comments in `raft.go` and `rpc.go` to explain the purpose and functionality of key methods, improving code readability and maintainability. - Introduced new helper functions with comments to clarify their roles in the Raft and RPC processes. * Improve Redis container setup and async test handling - Updated `createTestRedis` in `act_helpers_test.go` to use `wait.ForAll` for better reliability by ensuring both log readiness and port listening. - Enhanced `Test_Run_Async_Redis` in `registry_test.go` by adding a context with a timeout to the consumer subscription for improved test robustness. - Simplified the sleep duration in `Test_Run_Async_Redis` to reduce unnecessary wait time. * Handle Fatal Error on Raft Node Initialization Failure - Added error handling to record and log errors when Raft node initialization fails. - Ensured the application exits with a specific error code if the Raft node cannot be started. - Updated tests to set environment variables for Raft node configuration. - Added a new error code for Raft node startup failure in the error definitions. This change ensures that if the Raft node cannot be configured and started, the application will terminate gracefully, preventing further execution with an invalid state. * Update test configuration in gatewayd.yaml - Changed the raft address from 127.0.0.1:2223 to 127.0.0.1:2222. - Updated the nodeID from node2 to node1. These updates are made to the test data configuration to align with the current test case requirements. * Update comment to accurately describe Raft configuration constants The comment above the constants was misleading, suggesting they were only command types. Updated the comment to reflect that these constants are related to Raft operations. * Simplify leader check in monitorLeadership function - Removed the unnecessary `isLeader` variable in the `monitorLeadership` function. - Directly checked the node's state against `raft.Leader` in the if condition. * Fix: Gracefully handle ErrRaftShutdown during Node shutdown Updated the `Shutdown` method in `raft.go` to gracefully handle the `ErrRaftShutdown` error. This change ensures that if the Raft node is already shut down, the error is ignored, preventing unnecessary error handling.
1 parent cb993b2 commit c94c475

29 files changed

+1925
-76
lines changed

.golangci.yaml

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -70,6 +70,7 @@ linters-settings:
7070
- "github.com/testcontainers/testcontainers-go"
7171
- "github.com/stretchr/testify/require"
7272
- "github.com/docker/go-connections/nat"
73+
- "github.com/hashicorp/raft"
7374
test:
7475
files:
7576
- $test
@@ -92,6 +93,7 @@ linters-settings:
9293
- "github.com/redis/go-redis/v9"
9394
- "github.com/docker/go-connections/nat"
9495
- "github.com/codingsince1985/checksum"
96+
- "github.com/hashicorp/raft"
9597
tagalign:
9698
align: false
9799
sort: false

act/act_helpers_test.go

Lines changed: 4 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -62,7 +62,10 @@ func createTestRedis(t *testing.T) string {
6262
req := testcontainers.ContainerRequest{
6363
Image: "redis:6",
6464
ExposedPorts: []string{"6379/tcp"},
65-
WaitingFor: wait.ForLog("Ready to accept connections"),
65+
WaitingFor: wait.ForAll(
66+
wait.ForLog("Ready to accept connections"),
67+
wait.ForListeningPort("6379/tcp"),
68+
),
6669
}
6770
redisContainer, err := testcontainers.GenericContainer(
6871
ctx, testcontainers.GenericContainerRequest{

act/registry_test.go

Lines changed: 6 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -747,10 +747,12 @@ func Test_Run_Async_Redis(t *testing.T) {
747747
consumer, err := sdkAct.NewConsumer(hclogger, rdb, 5, "test-async-chan")
748748
require.NoError(t, err)
749749

750-
require.NoError(t, consumer.Subscribe(context.Background(), func(ctx context.Context, task []byte) error {
751-
err := actRegistry.runAsyncActionFn(ctx, task)
752-
waitGroup.Done()
753-
return err
750+
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
751+
defer cancel()
752+
753+
require.NoError(t, consumer.Subscribe(ctx, func(ctx context.Context, task []byte) error {
754+
defer waitGroup.Done()
755+
return actRegistry.runAsyncActionFn(ctx, task)
754756
}))
755757

756758
outputs := actRegistry.Apply([]sdkAct.Signal{

cmd/plugin_scaffold_test.go

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -28,6 +28,9 @@ func Test_pluginScaffoldCmd(t *testing.T) {
2828
postgresAddress2 := postgresHostIP2 + ":" + postgresMappedPort2.Port()
2929
t.Setenv("GATEWAYD_CLIENTS_TEST_WRITE_ADDRESS", postgresAddress2)
3030

31+
raftTempDir := t.TempDir()
32+
t.Setenv("GATEWAYD_RAFT_DIRECTORY", raftTempDir)
33+
3134
globalTestConfigFile := filepath.Join("testdata", "gatewayd.yaml")
3235
plugin.IsPluginTemplateEmbedded()
3336
pluginTestScaffoldInputFile := "./testdata/scaffold_input.yaml"

cmd/run.go

Lines changed: 13 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -29,6 +29,7 @@ import (
2929
"github.com/gatewayd-io/gatewayd/network"
3030
"github.com/gatewayd-io/gatewayd/plugin"
3131
"github.com/gatewayd-io/gatewayd/pool"
32+
"github.com/gatewayd-io/gatewayd/raft"
3233
"github.com/gatewayd-io/gatewayd/tracing"
3334
usage "github.com/gatewayd-io/gatewayd/usagereport/v1"
3435
"github.com/getsentry/sentry-go"
@@ -910,6 +911,17 @@ var runCmd = &cobra.Command{
910911

911912
span.End()
912913

914+
_, span = otel.Tracer(config.TracerName).Start(runCtx, "Create Raft Node")
915+
defer span.End()
916+
917+
raftNode, originalErr := raft.NewRaftNode(logger, conf.Global.Raft)
918+
if originalErr != nil {
919+
logger.Error().Err(originalErr).Msg("Failed to start raft node")
920+
span.RecordError(originalErr)
921+
pluginRegistry.Shutdown()
922+
os.Exit(gerr.FailedToStartRaftNode)
923+
}
924+
913925
_, span = otel.Tracer(config.TracerName).Start(runCtx, "Create servers")
914926
// Create and initialize servers.
915927
for name, cfg := range conf.Global.Servers {
@@ -946,6 +958,7 @@ var runCmd = &cobra.Command{
946958
LoadbalancerStrategyName: cfg.LoadBalancer.Strategy,
947959
LoadbalancerRules: cfg.LoadBalancer.LoadBalancingRules,
948960
LoadbalancerConsistentHash: cfg.LoadBalancer.ConsistentHash,
961+
RaftNode: raftNode,
949962
},
950963
)
951964

cmd/run_test.go

Lines changed: 20 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -23,6 +23,11 @@ func Test_runCmd(t *testing.T) {
2323
postgresAddress := postgresHostIP + ":" + postgresMappedPort.Port()
2424
t.Setenv("GATEWAYD_CLIENTS_DEFAULT_WRITES_ADDRESS", postgresAddress)
2525

26+
tempDir := t.TempDir()
27+
t.Setenv("GATEWAYD_RAFT_DIRECTORY", tempDir)
28+
t.Setenv("GATEWAYD_RAFT_ADDRESS", "127.0.0.1:0")
29+
t.Setenv("GATEWAYD_RAFT_GRPCADDRESS", "127.0.0.1:0")
30+
2631
globalTestConfigFile := "./test_global_runCmd.yaml"
2732
pluginTestConfigFile := "./test_plugins_runCmd.yaml"
2833
// Create a test plugins config file.
@@ -87,6 +92,11 @@ func Test_runCmdWithTLS(t *testing.T) {
8792
postgresAddress := postgresHostIP + ":" + postgresMappedPort.Port()
8893
t.Setenv("GATEWAYD_CLIENTS_DEFAULT_WRITES_ADDRESS", postgresAddress)
8994

95+
tempDir := t.TempDir()
96+
t.Setenv("GATEWAYD_RAFT_DIRECTORY", tempDir)
97+
t.Setenv("GATEWAYD_RAFT_ADDRESS", "127.0.0.1:0")
98+
t.Setenv("GATEWAYD_RAFT_GRPCADDRESS", "127.0.0.1:0")
99+
90100
globalTLSTestConfigFile := "./testdata/gatewayd_tls.yaml"
91101
pluginTestConfigFile := "./test_plugins_runCmdWithTLS.yaml"
92102
// Create a test plugins config file.
@@ -150,6 +160,11 @@ func Test_runCmdWithMultiTenancy(t *testing.T) {
150160
postgresAddress2 := postgresHostIP2 + ":" + postgresMappedPort2.Port()
151161
t.Setenv("GATEWAYD_CLIENTS_TEST_WRITE_ADDRESS", postgresAddress2)
152162

163+
tempDir := t.TempDir()
164+
t.Setenv("GATEWAYD_RAFT_DIRECTORY", tempDir)
165+
t.Setenv("GATEWAYD_RAFT_ADDRESS", "127.0.0.1:0")
166+
t.Setenv("GATEWAYD_RAFT_GRPCADDRESS", "127.0.0.1:0")
167+
153168
globalTestConfigFile := "./testdata/gatewayd.yaml"
154169
pluginTestConfigFile := "./test_plugins_runCmdWithMultiTenancy.yaml"
155170
// Create a test plugins config file.
@@ -211,6 +226,11 @@ func Test_runCmdWithCachePlugin(t *testing.T) {
211226
postgresAddress := postgresHostIP + ":" + postgresMappedPort.Port()
212227
t.Setenv("GATEWAYD_CLIENTS_DEFAULT_WRITES_ADDRESS", postgresAddress)
213228

229+
tempDir := t.TempDir()
230+
t.Setenv("GATEWAYD_RAFT_DIRECTORY", tempDir)
231+
t.Setenv("GATEWAYD_RAFT_ADDRESS", "127.0.0.1:0")
232+
t.Setenv("GATEWAYD_RAFT_GRPCADDRESS", "127.0.0.1:0")
233+
214234
globalTestConfigFile := "./test_global_runCmdWithCachePlugin.yaml"
215235
pluginTestConfigFile := "./test_plugins_runCmdWithCachePlugin.yaml"
216236
// TODO: Remove this once these global variables are removed from cmd/run.go.

cmd/testdata/gatewayd.yaml

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -73,3 +73,9 @@ servers:
7373

7474
api:
7575
enabled: True
76+
77+
raft:
78+
address: 127.0.0.1:2222
79+
nodeID: node1
80+
isBootstrap: true
81+
peers: {}

config/config.go

Lines changed: 50 additions & 32 deletions
Original file line numberDiff line numberDiff line change
@@ -180,6 +180,13 @@ func (c *Config) LoadDefaults(ctx context.Context) *gerr.GatewayDError {
180180
GRPCNetwork: DefaultGRPCAPINetwork,
181181
GRPCAddress: DefaultGRPCAPIAddress,
182182
},
183+
Raft: Raft{
184+
Address: DefaultRaftAddress,
185+
NodeID: DefaultRaftNodeID,
186+
IsBootstrap: DefaultRaftIsBootstrap,
187+
Directory: DefaultRaftDirectory,
188+
GRPCAddress: DefaultRaftGRPCAddress,
189+
},
183190
}
184191

185192
//nolint:nestif
@@ -201,7 +208,7 @@ func (c *Config) LoadDefaults(ctx context.Context) *gerr.GatewayDError {
201208
return gerr.ErrConfigParseError.Wrap(err)
202209
}
203210

204-
if configObject == "api" {
211+
if configObject == "api" || configObject == "raft" {
205212
// Handle API configuration separately
206213
// TODO: Add support for multiple API config groups.
207214
continue
@@ -309,7 +316,7 @@ func (c *Config) LoadDefaults(ctx context.Context) *gerr.GatewayDError {
309316
func (c *Config) LoadGlobalEnvVars(ctx context.Context) *gerr.GatewayDError {
310317
_, span := otel.Tracer(TracerName).Start(ctx, "Load global environment variables")
311318

312-
if err := c.GlobalKoanf.Load(loadEnvVars(), nil); err != nil {
319+
if err := c.GlobalKoanf.Load(loadEnvVarsWithTransform(), nil); err != nil {
313320
span.RecordError(err)
314321
span.End()
315322
return gerr.ErrConfigParseError.Wrap(
@@ -326,7 +333,7 @@ func (c *Config) LoadGlobalEnvVars(ctx context.Context) *gerr.GatewayDError {
326333
func (c *Config) LoadPluginEnvVars(ctx context.Context) *gerr.GatewayDError {
327334
_, span := otel.Tracer(TracerName).Start(ctx, "Load plugin environment variables")
328335

329-
if err := c.PluginKoanf.Load(loadEnvVars(), nil); err != nil {
336+
if err := c.PluginKoanf.Load(loadEnvVarsWithTransform(), nil); err != nil {
330337
span.RecordError(err)
331338
span.End()
332339
return gerr.ErrConfigParseError.Wrap(
@@ -338,41 +345,52 @@ func (c *Config) LoadPluginEnvVars(ctx context.Context) *gerr.GatewayDError {
338345
return nil
339346
}
340347

341-
func loadEnvVars() *env.Env {
342-
return env.Provider(EnvPrefix, ".", transformEnvVariable)
343-
}
344-
345-
// transformEnvVariable transforms the environment variable name to a format based on JSON tags.
346-
func transformEnvVariable(envVar string) string {
347-
structs := []any{
348-
&API{},
349-
&Logger{},
350-
&Pool{},
351-
&Proxy{},
352-
&Server{},
353-
&Metrics{},
354-
&PluginConfig{},
355-
}
356-
tagMapping := make(map[string]string)
357-
generateTagMapping(structs, tagMapping)
348+
func loadEnvVarsWithTransform() *env.Env {
349+
// Use ProviderWithValue to transform both key and value
350+
return env.ProviderWithValue(EnvPrefix, ".", func(envKey string, value string) (string, interface{}) {
351+
// Transform the key
352+
key := strings.ToLower(strings.TrimPrefix(envKey, EnvPrefix))
353+
354+
structs := []any{
355+
&API{},
356+
&Logger{},
357+
&Pool{},
358+
&Proxy{},
359+
&Server{},
360+
&Metrics{},
361+
&PluginConfig{},
362+
&Raft{},
363+
}
364+
tagMapping := make(map[string]string)
365+
generateTagMapping(structs, tagMapping)
358366

359-
lowerEnvVar := strings.ToLower(strings.TrimPrefix(envVar, EnvPrefix))
360-
parts := strings.Split(lowerEnvVar, "_")
367+
parts := strings.Split(key, "_")
361368

362-
var transformedParts strings.Builder
369+
var transformedParts strings.Builder
363370

364-
for i, part := range parts {
365-
if i > 0 {
366-
transformedParts.WriteString(".")
371+
for i, part := range parts {
372+
if i > 0 {
373+
transformedParts.WriteString(".")
374+
}
375+
if mappedValue, exists := tagMapping[part]; exists {
376+
transformedParts.WriteString(mappedValue)
377+
} else {
378+
transformedParts.WriteString(part)
379+
}
367380
}
368-
if mappedValue, exists := tagMapping[part]; exists {
369-
transformedParts.WriteString(mappedValue)
370-
} else {
371-
transformedParts.WriteString(part)
381+
382+
// Check if the key is "peers" and transform the value using JSON unmarshal
383+
if transformedParts.String() == "raft.peers" {
384+
var raftPeers []RaftPeer
385+
if err := json.Unmarshal([]byte(value), &raftPeers); err != nil {
386+
return transformedParts.String(), fmt.Errorf("failed to unmarshal peers: %w", err)
387+
}
388+
return transformedParts.String(), raftPeers
372389
}
373-
}
374390

375-
return transformedParts.String()
391+
// Return the key and value as is if no transformation is needed
392+
return transformedParts.String(), value
393+
})
376394
}
377395

378396
// LoadGlobalConfigFile loads the plugin configuration file.

config/constants.go

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -126,6 +126,13 @@ const (
126126
DefaultActionRedisEnabled = false
127127
DefaultRedisAddress = "localhost:6379"
128128
DefaultRedisChannel = "gatewayd-actions"
129+
130+
// Raft constants.
131+
DefaultRaftAddress = "127.0.0.1:2223"
132+
DefaultRaftNodeID = "node1"
133+
DefaultRaftIsBootstrap = true
134+
DefaultRaftDirectory = "raft"
135+
DefaultRaftGRPCAddress = "127.0.0.1:50051"
129136
)
130137

131138
// Load balancing strategies.

config/types.go

Lines changed: 16 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -138,6 +138,21 @@ type API struct {
138138
GRPCNetwork string `json:"grpcNetwork" jsonschema:"enum=tcp,enum=udp,enum=unix"`
139139
}
140140

141+
type Raft struct {
142+
Address string `json:"address"`
143+
NodeID string `json:"nodeId"`
144+
IsBootstrap bool `json:"isBootstrap"`
145+
Peers []RaftPeer `json:"peers"`
146+
Directory string `json:"directory" jsonschema:"default=raft"`
147+
GRPCAddress string `json:"grpcAddress"`
148+
}
149+
150+
type RaftPeer struct {
151+
ID string `json:"id"`
152+
Address string `json:"address"`
153+
GRPCAddress string `json:"grpcAddress"`
154+
}
155+
141156
type GlobalConfig struct {
142157
API API `json:"api"`
143158
Loggers map[string]*Logger `json:"loggers"`
@@ -146,4 +161,5 @@ type GlobalConfig struct {
146161
Proxies map[string]map[string]*Proxy `json:"proxies"`
147162
Servers map[string]*Server `json:"servers"`
148163
Metrics map[string]*Metrics `json:"metrics"`
164+
Raft Raft `json:"raft"`
149165
}

0 commit comments

Comments
 (0)