Removed RaftBindPort config. Raft bind address and port are dynamically assigned on startup instead of configured manually. Replaced memberlist-port configuration with discovery-port.

This commit is contained in:
Kelvin Mwinuka
2024-06-20 04:20:31 +08:00
parent 75baaa5c47
commit 9b0d590171
9 changed files with 129 additions and 140 deletions

View File

@@ -33,8 +33,7 @@ WORKDIR /opt/echovault/bin
CMD "./server" \ CMD "./server" \
"--bind-addr" "${BIND_ADDR}" \ "--bind-addr" "${BIND_ADDR}" \
"--port" "${PORT}" \ "--port" "${PORT}" \
"--memberlist-port" "${ML_PORT}" \ "--discovery-port" "${DISCOVERY_PORT}" \
"--raft-port" "${RAFT_PORT}" \
"--server-id" "${SERVER_ID}" \ "--server-id" "${SERVER_ID}" \
"--join-addr" "${JOIN_ADDR}" \ "--join-addr" "${JOIN_ADDR}" \
"--data-dir" "${DATA_DIR}" \ "--data-dir" "${DATA_DIR}" \

View File

@@ -33,15 +33,6 @@ func main() {
ctx := context.WithValue(context.Background(), internal.ContextServerID("ServerID"), conf.ServerID) ctx := context.WithValue(context.Background(), internal.ContextServerID("ServerID"), conf.ServerID)
// Default BindAddr if it's not specified
if conf.BindAddr == "" {
if addr, err := internal.GetIPAddress(); err != nil {
log.Fatal(err)
} else {
conf.BindAddr = addr
}
}
cancelCh := make(chan os.Signal, 1) cancelCh := make(chan os.Signal, 1)
signal.Notify(cancelCh, syscall.SIGINT, syscall.SIGTERM, os.Interrupt) signal.Notify(cancelCh, syscall.SIGINT, syscall.SIGTERM, os.Interrupt)

View File

@@ -11,9 +11,9 @@ services:
context: . context: .
dockerfile: Dockerfile.dev dockerfile: Dockerfile.dev
environment: environment:
- BIND_ADDR=0.0.0.0
- PORT=7480 - PORT=7480
- RAFT_PORT=8000 - DISCOVERY_PORT=7946
- ML_PORT=7946
- SERVER_ID=1 - SERVER_ID=1
- PLUGIN_DIR=/usr/local/lib/echovault - PLUGIN_DIR=/usr/local/lib/echovault
- DATA_DIR=/var/lib/echovault - DATA_DIR=/var/lib/echovault
@@ -44,7 +44,6 @@ services:
ports: ports:
- "7480:7480" - "7480:7480"
- "7946:7946" - "7946:7946"
- "7999:8000"
volumes: volumes:
- ./volumes/config:/etc/echovault/config - ./volumes/config:/etc/echovault/config
- ./volumes/modules:/lib/echovault/modules - ./volumes/modules:/lib/echovault/modules
@@ -58,9 +57,9 @@ services:
context: . context: .
dockerfile: Dockerfile.dev dockerfile: Dockerfile.dev
environment: environment:
- BIND_ADDR=0.0.0.0
- PORT=7480 - PORT=7480
- RAFT_PORT=8000 - DISCOVERY_PORT=7946
- ML_PORT=7946
- SERVER_ID=1 - SERVER_ID=1
- JOIN_ADDR=2/cluster_node_2:7946 - JOIN_ADDR=2/cluster_node_2:7946
- DATA_DIR=/var/lib/echovault - DATA_DIR=/var/lib/echovault
@@ -90,7 +89,6 @@ services:
ports: ports:
- "7481:7480" - "7481:7480"
- "7945:7946" - "7945:7946"
- "8000:8000"
volumes: volumes:
- ./volumes/config:/etc/echovault/config - ./volumes/config:/etc/echovault/config
- ./volumes/plugins:/lib/echovault/plugins - ./volumes/plugins:/lib/echovault/plugins
@@ -104,9 +102,9 @@ services:
context: . context: .
dockerfile: Dockerfile.dev dockerfile: Dockerfile.dev
environment: environment:
- BIND_ADDR=0.0.0.0
- PORT=7480 - PORT=7480
- RAFT_PORT=8000 - DISCOVERY_PORT=7946
- ML_PORT=7946
- SERVER_ID=2 - SERVER_ID=2
- JOIN_ADDR=3/cluster_node_3:7946 - JOIN_ADDR=3/cluster_node_3:7946
- DATA_DIR=/var/lib/echovault - DATA_DIR=/var/lib/echovault
@@ -136,7 +134,6 @@ services:
ports: ports:
- "7482:7480" - "7482:7480"
- "7947:7946" - "7947:7946"
- "8001:8000"
volumes: volumes:
- ./volumes/config:/etc/echovault/config - ./volumes/config:/etc/echovault/config
- ./volumes/plugins:/lib/echovault/plugins - ./volumes/plugins:/lib/echovault/plugins
@@ -150,9 +147,9 @@ services:
context: . context: .
dockerfile: Dockerfile.dev dockerfile: Dockerfile.dev
environment: environment:
- BIND_ADDR=0.0.0.0
- PORT=7480 - PORT=7480
- RAFT_PORT=8000 - DISCOVERY_PORT=7946
- ML_PORT=7946
- SERVER_ID=3 - SERVER_ID=3
- JOIN_ADDR=4/cluster_node_4:7946 - JOIN_ADDR=4/cluster_node_4:7946
- DATA_DIR=/var/lib/echovault - DATA_DIR=/var/lib/echovault
@@ -182,7 +179,6 @@ services:
ports: ports:
- "7483:7480" - "7483:7480"
- "7948:7946" - "7948:7946"
- "8002:8000"
volumes: volumes:
- ./volumes/config:/etc/echovault/config - ./volumes/config:/etc/echovault/config
- ./volumes/plugins:/lib/echovault/plugins - ./volumes/plugins:/lib/echovault/plugins
@@ -196,9 +192,9 @@ services:
context: . context: .
dockerfile: Dockerfile.dev dockerfile: Dockerfile.dev
environment: environment:
- BIND_ADDR=0.0.0.0
- PORT=7480 - PORT=7480
- RAFT_PORT=8000 - DISCOVERY_PORT=7946
- ML_PORT=7946
- SERVER_ID=4 - SERVER_ID=4
- JOIN_ADDR=5/cluster_node_5:7946 - JOIN_ADDR=5/cluster_node_5:7946
- DATA_DIR=/var/lib/echovault - DATA_DIR=/var/lib/echovault
@@ -228,7 +224,6 @@ services:
ports: ports:
- "7484:7480" - "7484:7480"
- "7949:7946" - "7949:7946"
- "8003:8000"
volumes: volumes:
- ./volumes/config:/etc/echovault/config - ./volumes/config:/etc/echovault/config
- ./volumes/plugins:/lib/echovault/plugins - ./volumes/plugins:/lib/echovault/plugins
@@ -242,9 +237,9 @@ services:
context: . context: .
dockerfile: Dockerfile.dev dockerfile: Dockerfile.dev
environment: environment:
- BIND_ADDR=0.0.0.0
- PORT=7480 - PORT=7480
- RAFT_PORT=8000 - DISCOVERY_PORT=7946
- ML_PORT=7946
- SERVER_ID=5 - SERVER_ID=5
- JOIN_ADDR=1/cluster_node_1:7946 - JOIN_ADDR=1/cluster_node_1:7946
- DATA_DIR=/var/lib/echovault - DATA_DIR=/var/lib/echovault
@@ -274,7 +269,6 @@ services:
ports: ports:
- "7485:7480" - "7485:7480"
- "7950:7946" - "7950:7946"
- "8004:8000"
volumes: volumes:
- ./volumes/config:/etc/echovault/config - ./volumes/config:/etc/echovault/config
- ./volumes/plugins:/lib/echovault/plugins - ./volumes/plugins:/lib/echovault/plugins

View File

@@ -41,8 +41,7 @@ type ClientServerPair struct {
serverId string serverId string
bindAddr string bindAddr string
port int port int
raftPort int discoveryPort int
mlPort int
bootstrapCluster bool bootstrapCluster bool
forwardCommand bool forwardCommand bool
joinAddr string joinAddr string
@@ -79,8 +78,7 @@ func setupServer(
bindAddr, bindAddr,
joinAddr string, joinAddr string,
port, port,
raftPort, discoveryPort int,
mlPort int,
) (*EchoVault, error) { ) (*EchoVault, error) {
conf := DefaultConfig() conf := DefaultConfig()
conf.DataDir = dataDir conf.DataDir = dataDir
@@ -89,8 +87,7 @@ func setupServer(
conf.JoinAddr = joinAddr conf.JoinAddr = joinAddr
conf.Port = uint16(port) conf.Port = uint16(port)
conf.ServerID = serverId conf.ServerID = serverId
conf.RaftBindPort = uint16(raftPort) conf.DiscoveryPort = uint16(discoveryPort)
conf.MemberListBindPort = uint16(mlPort)
conf.BootstrapCluster = bootstrapCluster conf.BootstrapCluster = bootstrapCluster
conf.EvictionPolicy = constants.NoEviction conf.EvictionPolicy = constants.NoEviction
@@ -109,8 +106,7 @@ func setupNode(node *ClientServerPair, isLeader bool, errChan *chan error) {
node.bindAddr, node.bindAddr,
node.joinAddr, node.joinAddr,
node.port, node.port,
node.raftPort, node.discoveryPort,
node.mlPort,
) )
if err != nil { if err != nil {
*errChan <- fmt.Errorf("could not start server; %v", err) *errChan <- fmt.Errorf("could not start server; %v", err)
@@ -161,17 +157,13 @@ func makeCluster(size int) ([]ClientServerPair, error) {
forwardCommand := i < len(pairs)-1 // The last node will not forward commands to the cluster leader. forwardCommand := i < len(pairs)-1 // The last node will not forward commands to the cluster leader.
joinAddr := "" joinAddr := ""
if !bootstrapCluster { if !bootstrapCluster {
joinAddr = fmt.Sprintf("%s/%s:%d", pairs[0].serverId, pairs[0].bindAddr, pairs[0].mlPort) joinAddr = fmt.Sprintf("%s/%s:%d", pairs[0].serverId, pairs[0].bindAddr, pairs[0].discoveryPort)
} }
port, err := internal.GetFreePort() port, err := internal.GetFreePort()
if err != nil { if err != nil {
return nil, fmt.Errorf("could not get free port: %v", err) return nil, fmt.Errorf("could not get free port: %v", err)
} }
raftPort, err := internal.GetFreePort() discoveryPort, err := internal.GetFreePort()
if err != nil {
return nil, fmt.Errorf("could not get free raft port: %v", err)
}
memberlistPort, err := internal.GetFreePort()
if err != nil { if err != nil {
return nil, fmt.Errorf("could not get free memberlist port: %v", err) return nil, fmt.Errorf("could not get free memberlist port: %v", err)
} }
@@ -181,8 +173,7 @@ func makeCluster(size int) ([]ClientServerPair, error) {
serverId: serverId, serverId: serverId,
bindAddr: bindAddr, bindAddr: bindAddr,
port: port, port: port,
raftPort: raftPort, discoveryPort: discoveryPort,
mlPort: memberlistPort,
bootstrapCluster: bootstrapCluster, bootstrapCluster: bootstrapCluster,
forwardCommand: forwardCommand, forwardCommand: forwardCommand,
joinAddr: joinAddr, joinAddr: joinAddr,

View File

@@ -32,32 +32,33 @@ import (
) )
type Config struct { type Config struct {
TLS bool `json:"TLS" yaml:"TLS"` TLS bool `json:"TLS" yaml:"TLS"`
MTLS bool `json:"MTLS" yaml:"MTLS"` MTLS bool `json:"MTLS" yaml:"MTLS"`
CertKeyPairs [][]string `json:"CertKeyPairs" yaml:"CertKeyPairs"` CertKeyPairs [][]string `json:"CertKeyPairs" yaml:"CertKeyPairs"`
ClientCAs []string `json:"ClientCAs" yaml:"ClientCAs"` ClientCAs []string `json:"ClientCAs" yaml:"ClientCAs"`
Port uint16 `json:"Port" yaml:"Port"` Port uint16 `json:"Port" yaml:"Port"`
ServerID string `json:"ServerId" yaml:"ServerId"` ServerID string `json:"ServerId" yaml:"ServerId"`
JoinAddr string `json:"JoinAddr" yaml:"JoinAddr"` JoinAddr string `json:"JoinAddr" yaml:"JoinAddr"`
BindAddr string `json:"BindAddr" yaml:"BindAddr"` BindAddr string `json:"BindAddr" yaml:"BindAddr"`
RaftBindPort uint16 `json:"RaftPort" yaml:"RaftPort"` DataDir string `json:"DataDir" yaml:"DataDir"`
MemberListBindPort uint16 `json:"MlPort" yaml:"MlPort"` BootstrapCluster bool `json:"BootstrapCluster" yaml:"BootstrapCluster"`
DataDir string `json:"DataDir" yaml:"DataDir"` AclConfig string `json:"AclConfig" yaml:"AclConfig"`
BootstrapCluster bool `json:"BootstrapCluster" yaml:"BootstrapCluster"` ForwardCommand bool `json:"ForwardCommand" yaml:"ForwardCommand"`
AclConfig string `json:"AclConfig" yaml:"AclConfig"` RequirePass bool `json:"RequirePass" yaml:"RequirePass"`
ForwardCommand bool `json:"ForwardCommand" yaml:"ForwardCommand"` Password string `json:"Password" yaml:"Password"`
RequirePass bool `json:"RequirePass" yaml:"RequirePass"` SnapShotThreshold uint64 `json:"SnapshotThreshold" yaml:"SnapshotThreshold"`
Password string `json:"Password" yaml:"Password"` SnapshotInterval time.Duration `json:"SnapshotInterval" yaml:"SnapshotInterval"`
SnapShotThreshold uint64 `json:"SnapshotThreshold" yaml:"SnapshotThreshold"` RestoreSnapshot bool `json:"RestoreSnapshot" yaml:"RestoreSnapshot"`
SnapshotInterval time.Duration `json:"SnapshotInterval" yaml:"SnapshotInterval"` RestoreAOF bool `json:"RestoreAOF" yaml:"RestoreAOF"`
RestoreSnapshot bool `json:"RestoreSnapshot" yaml:"RestoreSnapshot"` AOFSyncStrategy string `json:"AOFSyncStrategy" yaml:"AOFSyncStrategy"`
RestoreAOF bool `json:"RestoreAOF" yaml:"RestoreAOF"` MaxMemory uint64 `json:"MaxMemory" yaml:"MaxMemory"`
AOFSyncStrategy string `json:"AOFSyncStrategy" yaml:"AOFSyncStrategy"` EvictionPolicy string `json:"EvictionPolicy" yaml:"EvictionPolicy"`
MaxMemory uint64 `json:"MaxMemory" yaml:"MaxMemory"` EvictionSample uint `json:"EvictionSample" yaml:"EvictionSample"`
EvictionPolicy string `json:"EvictionPolicy" yaml:"EvictionPolicy"` EvictionInterval time.Duration `json:"EvictionInterval" yaml:"EvictionInterval"`
EvictionSample uint `json:"EvictionSample" yaml:"EvictionSample"` Modules []string `json:"Plugins" yaml:"Plugins"`
EvictionInterval time.Duration `json:"EvictionInterval" yaml:"EvictionInterval"` DiscoveryPort uint16
Modules []string `json:"Plugins" yaml:"Plugins"` RaftBindAddr string
RaftBindPort uint16
} }
func GetConfig() (Config, error) { func GetConfig() (Config, error) {
@@ -148,9 +149,8 @@ There is no limit by default.`, func(memory string) error {
port := flag.Int("port", 7480, "Port to use. Default is 7480") port := flag.Int("port", 7480, "Port to use. Default is 7480")
serverId := flag.String("server-id", "1", "EchoVault ID in raft cluster. Leave empty for client.") serverId := flag.String("server-id", "1", "EchoVault ID in raft cluster. Leave empty for client.")
joinAddr := flag.String("join-addr", "", "Address of cluster member in a cluster to you want to join.") joinAddr := flag.String("join-addr", "", "Address of cluster member in a cluster to you want to join.")
bindAddr := flag.String("bind-addr", "", "Address to bind the echovault to.") bindAddr := flag.String("bind-addr", "127.0.0.1", "Address to bind the echovault to.")
raftBindPort := flag.Uint("raft-port", 7481, "Port to use for intra-cluster communication. Leave on the client.") discoveryPort := flag.Uint("discovery-port", 7946, "Port to use for memberlist cluster discovery.")
mlBindPort := flag.Uint("memberlist-port", 7946, "Port to use for memberlist communication.")
dataDir := flag.String("data-dir", ".", "Directory to store snapshots and logs.") dataDir := flag.String("data-dir", ".", "Directory to store snapshots and logs.")
bootstrapCluster := flag.Bool("bootstrap-cluster", false, "Whether this instance should bootstrap a new cluster.") bootstrapCluster := flag.Bool("bootstrap-cluster", false, "Whether this instance should bootstrap a new cluster.")
aclConfig := flag.String("acl-config", "", "ACL config file path.") aclConfig := flag.String("acl-config", "", "ACL config file path.")
@@ -184,33 +184,43 @@ It is a plain text value by default but you can provide a SHA256 hash by adding
flag.Parse() flag.Parse()
raftBindAddr, e := internal.GetIPAddress()
if e != nil {
return Config{}, e
}
raftBindPort, e := internal.GetFreePort()
if e != nil {
return Config{}, e
}
conf := Config{ conf := Config{
CertKeyPairs: certKeyPairs, CertKeyPairs: certKeyPairs,
ClientCAs: clientCAs, ClientCAs: clientCAs,
TLS: *tls, TLS: *tls,
MTLS: *mtls, MTLS: *mtls,
Port: uint16(*port), Port: uint16(*port),
ServerID: *serverId, ServerID: *serverId,
JoinAddr: *joinAddr, JoinAddr: *joinAddr,
BindAddr: *bindAddr, BindAddr: *bindAddr,
RaftBindPort: uint16(*raftBindPort), DataDir: *dataDir,
MemberListBindPort: uint16(*mlBindPort), BootstrapCluster: *bootstrapCluster,
DataDir: *dataDir, AclConfig: *aclConfig,
BootstrapCluster: *bootstrapCluster, ForwardCommand: *forwardCommand,
AclConfig: *aclConfig, RequirePass: *requirePass,
ForwardCommand: *forwardCommand, Password: *password,
RequirePass: *requirePass, SnapShotThreshold: *snapshotThreshold,
Password: *password, SnapshotInterval: *snapshotInterval,
SnapShotThreshold: *snapshotThreshold, RestoreSnapshot: *restoreSnapshot,
SnapshotInterval: *snapshotInterval, RestoreAOF: *restoreAOF,
RestoreSnapshot: *restoreSnapshot, AOFSyncStrategy: aofSyncStrategy,
RestoreAOF: *restoreAOF, MaxMemory: maxMemory,
AOFSyncStrategy: aofSyncStrategy, EvictionPolicy: evictionPolicy,
MaxMemory: maxMemory, EvictionSample: *evictionSample,
EvictionPolicy: evictionPolicy, EvictionInterval: *evictionInterval,
EvictionSample: *evictionSample, Modules: modules,
EvictionInterval: *evictionInterval, DiscoveryPort: uint16(*discoveryPort),
Modules: modules, RaftBindAddr: raftBindAddr,
RaftBindPort: uint16(raftBindPort),
} }
if len(*config) > 0 { if len(*config) > 0 {

View File

@@ -1,37 +1,42 @@
package config package config
import ( import (
"github.com/echovault/echovault/internal"
"github.com/echovault/echovault/internal/constants" "github.com/echovault/echovault/internal/constants"
"time" "time"
) )
func DefaultConfig() Config { func DefaultConfig() Config {
raftBindAddr, _ := internal.GetIPAddress()
raftBindPort, _ := internal.GetFreePort()
return Config{ return Config{
TLS: false, TLS: false,
MTLS: false, MTLS: false,
CertKeyPairs: make([][]string, 0), CertKeyPairs: make([][]string, 0),
ClientCAs: make([]string, 0), ClientCAs: make([]string, 0),
Port: 7480, Port: 7480,
ServerID: "", ServerID: "",
JoinAddr: "", JoinAddr: "",
BindAddr: "localhost", BindAddr: "localhost",
RaftBindPort: 7481, RaftBindAddr: raftBindAddr,
MemberListBindPort: 7946, RaftBindPort: uint16(raftBindPort),
DataDir: ".", DiscoveryPort: 7946,
BootstrapCluster: false, DataDir: ".",
AclConfig: "", BootstrapCluster: false,
ForwardCommand: false, AclConfig: "",
RequirePass: false, ForwardCommand: false,
Password: "", RequirePass: false,
SnapShotThreshold: 1000, Password: "",
SnapshotInterval: 5 * time.Minute, SnapShotThreshold: 1000,
RestoreAOF: false, SnapshotInterval: 5 * time.Minute,
RestoreSnapshot: false, RestoreAOF: false,
AOFSyncStrategy: "everysec", RestoreSnapshot: false,
MaxMemory: 0, AOFSyncStrategy: "everysec",
EvictionPolicy: constants.NoEviction, MaxMemory: 0,
EvictionSample: 20, EvictionPolicy: constants.NoEviction,
EvictionInterval: 100 * time.Millisecond, EvictionSample: 20,
Modules: make([]string, 0), EvictionInterval: 100 * time.Millisecond,
Modules: make([]string, 0),
} }
} }

View File

@@ -50,8 +50,8 @@ func (delegate *Delegate) NodeMeta(limit int) []byte {
meta := NodeMeta{ meta := NodeMeta{
ServerID: raft.ServerID(delegate.options.config.ServerID), ServerID: raft.ServerID(delegate.options.config.ServerID),
RaftAddr: raft.ServerAddress( RaftAddr: raft.ServerAddress(
fmt.Sprintf("%s:%d", delegate.options.config.BindAddr, delegate.options.config.RaftBindPort)), fmt.Sprintf("%s:%d", delegate.options.config.RaftBindAddr, delegate.options.config.RaftBindPort)),
MemberlistAddr: fmt.Sprintf("%s:%d", delegate.options.config.BindAddr, delegate.options.config.MemberListBindPort), MemberlistAddr: fmt.Sprintf("%s:%d", delegate.options.config.BindAddr, delegate.options.config.DiscoveryPort),
} }
b, err := json.Marshal(&meta) b, err := json.Marshal(&meta)
@@ -73,7 +73,7 @@ func (delegate *Delegate) NotifyMsg(msgBytes []byte) {
switch msg.Action { switch msg.Action {
case "RaftJoin": case "RaftJoin":
// If the current node is not the cluster leader, re-broadcast the message // If the current node is not the cluster leader, re-broadcast the message.
if !delegate.options.isRaftLeader() { if !delegate.options.isRaftLeader() {
delegate.options.broadcastQueue.QueueBroadcast(&msg) delegate.options.broadcastQueue.QueueBroadcast(&msg)
return return
@@ -84,12 +84,12 @@ func (delegate *Delegate) NotifyMsg(msgBytes []byte) {
} }
case "DeleteKey": case "DeleteKey":
// If the current node is not a cluster leader, re-broadcast the message // If the current node is not a cluster leader, re-broadcast the message.
if !delegate.options.isRaftLeader() { if !delegate.options.isRaftLeader() {
delegate.options.broadcastQueue.QueueBroadcast(&msg) delegate.options.broadcastQueue.QueueBroadcast(&msg)
return return
} }
// Current node is the cluster leader, handle the key deletion // Current node is the cluster leader, handle the key deletion.
ctx := context.WithValue( ctx := context.WithValue(
context.WithValue(context.Background(), internal.ContextServerID("ServerID"), string(msg.ServerID)), context.WithValue(context.Background(), internal.ContextServerID("ServerID"), string(msg.ServerID)),
internal.ContextConnID("ConnectionID"), msg.ConnId) internal.ContextConnID("ConnectionID"), msg.ConnId)
@@ -101,7 +101,7 @@ func (delegate *Delegate) NotifyMsg(msgBytes []byte) {
} }
case "MutateData": case "MutateData":
// If the current node is not a cluster leader, re-broadcast the message // If the current node is not a cluster leader, re-broadcast the message.
if !delegate.options.isRaftLeader() { if !delegate.options.isRaftLeader() {
delegate.options.broadcastQueue.QueueBroadcast(&msg) delegate.options.broadcastQueue.QueueBroadcast(&msg)
return return

View File

@@ -64,7 +64,7 @@ func (m *MemberList) MemberListInit(ctx context.Context) {
cfg.RequireNodeNames = true cfg.RequireNodeNames = true
cfg.Name = m.options.Config.ServerID cfg.Name = m.options.Config.ServerID
cfg.BindAddr = m.options.Config.BindAddr cfg.BindAddr = m.options.Config.BindAddr
cfg.BindPort = int(m.options.Config.MemberListBindPort) cfg.BindPort = int(m.options.Config.DiscoveryPort)
cfg.Delegate = NewDelegate(DelegateOpts{ cfg.Delegate = NewDelegate(DelegateOpts{
config: m.options.Config, config: m.options.Config,
broadcastQueue: m.broadcastQueue, broadcastQueue: m.broadcastQueue,
@@ -116,7 +116,7 @@ func (m *MemberList) broadcastRaftAddress() {
NodeMeta: NodeMeta{ NodeMeta: NodeMeta{
ServerID: raft.ServerID(m.options.Config.ServerID), ServerID: raft.ServerID(m.options.Config.ServerID),
RaftAddr: raft.ServerAddress(fmt.Sprintf("%s:%d", RaftAddr: raft.ServerAddress(fmt.Sprintf("%s:%d",
m.options.Config.BindAddr, m.options.Config.RaftBindPort)), m.options.Config.RaftBindAddr, m.options.Config.RaftBindPort)),
}, },
} }
m.broadcastQueue.QueueBroadcast(&msg) m.broadcastQueue.QueueBroadcast(&msg)

View File

@@ -91,15 +91,14 @@ func (r *Raft) RaftInit(ctx context.Context) {
} }
} }
addr := fmt.Sprintf("%s:%d", conf.BindAddr, conf.RaftBindPort) bindAddr := fmt.Sprintf("%s:%d", conf.RaftBindAddr, conf.RaftBindPort)
advertiseAddr, err := net.ResolveTCPAddr("tcp", bindAddr)
advertiseAddr, err := net.ResolveTCPAddr("tcp", addr)
if err != nil { if err != nil {
log.Fatal(err) log.Fatal(err)
} }
raftTransport, err := raft.NewTCPTransport( raftTransport, err := raft.NewTCPTransport(
addr, bindAddr,
advertiseAddr, advertiseAddr,
10, 10,
5*time.Second, 5*time.Second,
@@ -142,7 +141,7 @@ func (r *Raft) RaftInit(ctx context.Context) {
{ {
Suffrage: raft.Voter, Suffrage: raft.Voter,
ID: raft.ServerID(conf.ServerID), ID: raft.ServerID(conf.ServerID),
Address: raft.ServerAddress(addr), Address: raft.ServerAddress(conf.RaftBindAddr),
}, },
}, },
}).Error() }).Error()
@@ -185,7 +184,7 @@ func (r *Raft) AddVoter(
} }
for _, s := range raftConfig.Configuration().Servers { for _, s := range raftConfig.Configuration().Servers {
// Check if a echovault already exists with the current attributes // Check if a node already exists with the current attributes.
if s.ID == id && s.Address == address { if s.ID == id && s.Address == address {
return fmt.Errorf("node with id %s and address %s already exists", id, address) return fmt.Errorf("node with id %s and address %s already exists", id, address)
} }