Machine join/remove v2 API.

This commit is contained in:
Ben Johnson
2014-03-18 11:46:48 -07:00
parent e9a1ac15d9
commit 7d4fda550d
13 changed files with 335 additions and 100 deletions

View File

@ -0,0 +1,74 @@
## Proxies
Adding peers in an etcd cluster adds network, CPU, and disk overhead to the leader since each one requires replication.
Peers primarily provide resiliency in the event of a leader failure but the benefit of more failover nodes decreases as the cluster size increases.
A lightweight alternative is the proxy.
Proxies are a way for an etcd node to forward requests along to the cluster but the proxies are not part of the Raft cluster themselves.
This provides an easier API for local applications while reducing the overhead required by a regular peer node.
Proxies also act as standby nodes in the event that a peer node in the cluster has not recovered after a long duration.
## Configuration Parameters
Proxies require two additional configuration parameters: active size & promotion delay.
The active size specifies a target size for the number of peers in the cluster.
If there are not enough peers to meet the active size then proxies are promoted to peers until the peer count is equal to the active size.
If there are more peers than the target active size then peers are demoted to proxies.
The promotion delay specifies how long the cluster should wait before removing a dead peer and promoting a proxy.
By default this is 30 minutes.
If a peer is inactive for 30 minutes then the peer is removed and a live proxy is found to take its place.
## Logical Workflow
Start a etcd machine and join the cluster:
```
If peer count less than active size:
If machine already exists as a proxy:
Remove machine from proxy list
Join as peer
If peer count greater than or equal to active size:
Join as proxy
```
Remove an existing etcd machine from the cluster:
```
If machine exists in peer list:
Remove from peer list
If machine exists in proxy list:
Remove from proxy list
```
Leader's active size monitor:
```
Loop:
Sleep 5 seconds
If peer count less than active size:
If proxy count greater than zero:
Request a random proxy to rejoin
Goto Loop
If peer count greater than active size:
Demote randomly selected peer
Goto Loop
```
Leader's peer activity monitor:
```
Loop:
Sleep 5 seconds
For each peer:
If peer last activity time greater than promote delay:
Demote peer
Goto Loop
```

View File

@ -31,7 +31,7 @@ done
tmux new-window -t $SESSION:2 -n 'proxy' tmux new-window -t $SESSION:2 -n 'proxy'
tmux split-window -h tmux split-window -h
tmux select-pane -t 0 tmux select-pane -t 0
tmux send-keys "curl -XPUT -H \"Content-Type: application/json\" -d '{\"activeSize\":3, \"promoteDelay\":30}' http://127.0.0.1:7001/config" C-m tmux send-keys "curl -XPUT -H \"Content-Type: application/json\" -d '{\"activeSize\":3, \"promoteDelay\":30}' http://127.0.0.1:7001/v2/admin/config" C-m
for i in 4 5 6; do for i in 4 5 6; do
tmux select-pane -t 0 tmux select-pane -t 0

View File

@ -34,16 +34,31 @@ func (c *DemoteCommand) Apply(context raft.Context) (interface{}, error) {
clientURL, _ := ps.registry.ClientURL(c.Name) clientURL, _ := ps.registry.ClientURL(c.Name)
peerURL, _ := ps.registry.PeerURL(c.Name) peerURL, _ := ps.registry.PeerURL(c.Name)
// Perform a removal. // Remove node from the shared registry.
(&RemoveCommand{Name: c.Name}).Apply(context) err := ps.registry.UnregisterPeer(c.Name)
if err != nil {
log.Debugf("Demote peer %s: Error while unregistering (%v)", c.Name, err)
return nil, err
}
// Delete from stats
delete(ps.followersStats.Followers, c.Name)
// Remove peer in raft
err = context.Server().RemovePeer(c.Name)
if err != nil {
log.Debugf("Demote peer %s: (%v)", c.Name, err)
return nil, err
}
// Register node as a proxy. // Register node as a proxy.
ps.registry.RegisterProxy(c.Name, peerURL, clientURL) ps.registry.RegisterProxy(c.Name, peerURL, clientURL)
// Update mode if this change applies to this server. // Update mode if this change applies to this server.
if c.Name == ps.Config.Name { if c.Name == ps.Config.Name {
log.Infof("Set mode after demotion: %s", c.Name) log.Infof("Demote peer %s: Set mode to proxy with %s", c.Name, ps.server.Leader())
ps.setMode(ProxyMode) ps.proxyPeerURL, _ = ps.registry.PeerURL(ps.server.Leader())
go ps.setMode(ProxyMode)
} }
return nil, nil return nil, nil

View File

@ -1,27 +1,22 @@
package server package server
import ( import (
"bytes"
"encoding/binary" "encoding/binary"
"encoding/json"
etcdErr "github.com/coreos/etcd/error"
"github.com/coreos/etcd/log" "github.com/coreos/etcd/log"
"github.com/coreos/etcd/third_party/github.com/coreos/raft" "github.com/coreos/etcd/third_party/github.com/coreos/raft"
) )
func init() { func init() {
raft.RegisterCommand(&JoinCommand{}) raft.RegisterCommand(&JoinCommandV1{})
raft.RegisterCommand(&JoinCommandV2{})
} }
// The JoinCommand adds a node to the cluster. // JoinCommandV1 represents a request to join the cluster.
// // The command returns the join_index (Uvarint).
// The command returns the join_index (Uvarint) and peer flag (peer=0, proxy=1) type JoinCommandV1 struct {
// in following binary format:
//
// 8 bytes | 1 byte
// join_index | join_mode
//
// This binary protocol is for backward compatibility.
type JoinCommand struct {
MinVersion int `json:"minVersion"` MinVersion int `json:"minVersion"`
MaxVersion int `json:"maxVersion"` MaxVersion int `json:"maxVersion"`
Name string `json:"name"` Name string `json:"name"`
@ -29,50 +24,30 @@ type JoinCommand struct {
EtcdURL string `json:"etcdURL"` EtcdURL string `json:"etcdURL"`
} }
func NewJoinCommand(minVersion int, maxVersion int, name, raftUrl, etcdUrl string) *JoinCommand {
return &JoinCommand{
MinVersion: minVersion,
MaxVersion: maxVersion,
Name: name,
RaftURL: raftUrl,
EtcdURL: etcdUrl,
}
}
// The name of the join command in the log // The name of the join command in the log
func (c *JoinCommand) CommandName() string { func (c *JoinCommandV1) CommandName() string {
return "etcd:join" return "etcd:join"
} }
// Join a server to the cluster // Join a server to the cluster
func (c *JoinCommand) Apply(context raft.Context) (interface{}, error) { func (c *JoinCommandV1) Apply(context raft.Context) (interface{}, error) {
ps, _ := context.Server().Context().(*PeerServer) ps, _ := context.Server().Context().(*PeerServer)
var buf bytes.Buffer
b := make([]byte, 8) b := make([]byte, 8)
n := binary.PutUvarint(b, context.CommitIndex()) binary.PutUvarint(b, context.CommitIndex())
buf.Write(b[:n])
// Make sure we're not getting a cached value from the registry. // Make sure we're not getting a cached value from the registry.
ps.registry.Invalidate(c.Name) ps.registry.Invalidate(c.Name)
// Check if the join command is from a previous peer, who lost all its previous log. // Check if the join command is from a previous peer, who lost all its previous log.
if _, ok := ps.registry.ClientURL(c.Name); ok { if _, ok := ps.registry.ClientURL(c.Name); ok {
binary.Write(&buf, binary.BigEndian, uint8(peerModeFlag)) // Mark as peer. return b, nil
return buf.Bytes(), nil
} }
// Check peer number in the cluster // Check peer number in the cluster
if ps.registry.PeerCount() >= ps.ClusterConfig().ActiveSize { if ps.registry.PeerCount() >= ps.ClusterConfig().ActiveSize {
log.Debug("Join as proxy ", c.Name) log.Debug("Reject join request from ", c.Name)
ps.registry.RegisterProxy(c.Name, c.RaftURL, c.EtcdURL) return []byte{0}, etcdErr.NewError(etcdErr.EcodeNoMorePeer, "", context.CommitIndex())
binary.Write(&buf, binary.BigEndian, uint8(proxyModeFlag)) // Mark as proxy.
return buf.Bytes(), nil
}
// Remove it as a proxy if it is one.
if ps.registry.ProxyExists(c.Name) {
ps.registry.UnregisterProxy(c.Name)
} }
// Add to shared peer registry. // Add to shared peer registry.
@ -87,10 +62,79 @@ func (c *JoinCommand) Apply(context raft.Context) (interface{}, error) {
ps.followersStats.Followers[c.Name].Latency.Minimum = 1 << 63 ps.followersStats.Followers[c.Name].Latency.Minimum = 1 << 63
} }
binary.Write(&buf, binary.BigEndian, uint8(peerModeFlag)) // Mark as peer. return b, err
return buf.Bytes(), err
} }
func (c *JoinCommand) NodeName() string { func (c *JoinCommandV1) NodeName() string {
return c.Name return c.Name
} }
// JoinCommandV2 represents a request to join the cluster.
type JoinCommandV2 struct {
MinVersion int `json:"minVersion"`
MaxVersion int `json:"maxVersion"`
Name string `json:"name"`
PeerURL string `json:"peerURL"`
ClientURL string `json:"clientURL"`
}
// CommandName returns the name of the command in the Raft log.
func (c *JoinCommandV2) CommandName() string {
return "etcd:v2:join"
}
// Apply attempts to join a machine to the cluster.
func (c *JoinCommandV2) Apply(context raft.Context) (interface{}, error) {
ps, _ := context.Server().Context().(*PeerServer)
var msg = joinMessageV2{
Mode: PeerMode,
CommitIndex: context.CommitIndex(),
}
// Make sure we're not getting a cached value from the registry.
ps.registry.Invalidate(c.Name)
// Check if the join command is from a previous peer, who lost all its previous log.
if _, ok := ps.registry.ClientURL(c.Name); ok {
return json.Marshal(msg)
}
// Check peer number in the cluster.
if ps.registry.PeerCount() >= ps.ClusterConfig().ActiveSize {
log.Debug("Join as proxy ", c.Name)
ps.registry.RegisterProxy(c.Name, c.PeerURL, c.ClientURL)
msg.Mode = ProxyMode
return json.Marshal(msg)
}
// Remove it as a proxy if it is one.
if ps.registry.ProxyExists(c.Name) {
ps.registry.UnregisterProxy(c.Name)
}
// Add to shared peer registry.
ps.registry.RegisterPeer(c.Name, c.PeerURL, c.ClientURL)
// Add peer in raft
if err := context.Server().AddPeer(c.Name, ""); err != nil {
b, _ := json.Marshal(msg)
return b, err
}
// Add peer stats
if c.Name != ps.RaftServer().Name() {
ps.followersStats.Followers[c.Name] = &raftFollowerStats{}
ps.followersStats.Followers[c.Name].Latency.Minimum = 1 << 63
}
return json.Marshal(msg)
}
func (c *JoinCommandV2) NodeName() string {
return c.Name
}
type joinMessageV2 struct {
CommitIndex uint64 `json:"commitIndex"`
Mode Mode `json:"mode"`
}

View File

@ -1,12 +1,9 @@
package server package server
import ( import (
"bufio"
"bytes" "bytes"
"encoding/binary"
"encoding/json" "encoding/json"
"fmt" "fmt"
"io"
"io/ioutil" "io/ioutil"
"math/rand" "math/rand"
"net/http" "net/http"
@ -313,10 +310,6 @@ func (s *PeerServer) HTTPHandler() http.Handler {
router.HandleFunc("/join", s.JoinHttpHandler) router.HandleFunc("/join", s.JoinHttpHandler)
router.HandleFunc("/promote", s.PromoteHttpHandler).Methods("POST") router.HandleFunc("/promote", s.PromoteHttpHandler).Methods("POST")
router.HandleFunc("/remove/{name:.+}", s.RemoveHttpHandler) router.HandleFunc("/remove/{name:.+}", s.RemoveHttpHandler)
router.HandleFunc("/config", s.getClusterConfigHttpHandler).Methods("GET")
router.HandleFunc("/config", s.setClusterConfigHttpHandler).Methods("PUT")
router.HandleFunc("/machines", s.getMachinesHttpHandler).Methods("GET")
router.HandleFunc("/machines/{name}", s.getMachineHttpHandler).Methods("GET")
router.HandleFunc("/vote", s.VoteHttpHandler) router.HandleFunc("/vote", s.VoteHttpHandler)
router.HandleFunc("/log", s.GetLogHttpHandler) router.HandleFunc("/log", s.GetLogHttpHandler)
router.HandleFunc("/log/append", s.AppendEntriesHttpHandler) router.HandleFunc("/log/append", s.AppendEntriesHttpHandler)
@ -324,6 +317,13 @@ func (s *PeerServer) HTTPHandler() http.Handler {
router.HandleFunc("/snapshotRecovery", s.SnapshotRecoveryHttpHandler) router.HandleFunc("/snapshotRecovery", s.SnapshotRecoveryHttpHandler)
router.HandleFunc("/etcdURL", s.EtcdURLHttpHandler) router.HandleFunc("/etcdURL", s.EtcdURLHttpHandler)
router.HandleFunc("/v2/admin/config", s.getClusterConfigHttpHandler).Methods("GET")
router.HandleFunc("/v2/admin/config", s.setClusterConfigHttpHandler).Methods("PUT")
router.HandleFunc("/v2/admin/machines", s.getMachinesHttpHandler).Methods("GET")
router.HandleFunc("/v2/admin/machines/{name}", s.getMachineHttpHandler).Methods("GET")
router.HandleFunc("/v2/admin/machines/{name}", s.addMachineHttpHandler).Methods("PUT")
router.HandleFunc("/v2/admin/machines/{name}", s.removeMachineHttpHandler).Methods("DELETE")
return router return router
} }
@ -340,7 +340,14 @@ func (s *PeerServer) SetServer(server *Server) {
func (s *PeerServer) startAsLeader() { func (s *PeerServer) startAsLeader() {
// leader need to join self as a peer // leader need to join self as a peer
for { for {
_, err := s.raftServer.Do(NewJoinCommand(store.MinVersion(), store.MaxVersion(), s.raftServer.Name(), s.Config.URL, s.server.URL())) c := &JoinCommandV1{
MinVersion: store.MinVersion(),
MaxVersion: store.MaxVersion(),
Name: s.raftServer.Name(),
RaftURL: s.Config.URL,
EtcdURL: s.server.URL(),
}
_, err := s.raftServer.Do(c)
if err == nil { if err == nil {
break break
} }
@ -429,8 +436,6 @@ func (s *PeerServer) joinCluster(cluster []string) bool {
// Send join requests to peer. // Send join requests to peer.
func (s *PeerServer) joinByPeer(server raft.Server, peer string, scheme string) error { func (s *PeerServer) joinByPeer(server raft.Server, peer string, scheme string) error {
var b bytes.Buffer
// t must be ok // t must be ok
t, _ := server.Transporter().(*transporter) t, _ := server.Transporter().(*transporter)
@ -444,14 +449,21 @@ func (s *PeerServer) joinByPeer(server raft.Server, peer string, scheme string)
return fmt.Errorf("Unable to join: cluster version is %d; version compatibility is %d - %d", version, store.MinVersion(), store.MaxVersion()) return fmt.Errorf("Unable to join: cluster version is %d; version compatibility is %d - %d", version, store.MinVersion(), store.MaxVersion())
} }
json.NewEncoder(&b).Encode(NewJoinCommand(store.MinVersion(), store.MaxVersion(), server.Name(), s.Config.URL, s.server.URL())) var b bytes.Buffer
c := &JoinCommandV2{
joinURL := url.URL{Host: peer, Scheme: scheme, Path: "/join"} MinVersion: store.MinVersion(),
MaxVersion: store.MaxVersion(),
Name: server.Name(),
PeerURL: s.Config.URL,
ClientURL: s.server.URL(),
}
json.NewEncoder(&b).Encode(c)
joinURL := url.URL{Host: peer, Scheme: scheme, Path: "/v2/admin/machines/" + server.Name()}
log.Debugf("Send Join Request to %s", joinURL.String()) log.Debugf("Send Join Request to %s", joinURL.String())
resp, req, err := t.Post(joinURL.String(), &b) req, _ := http.NewRequest("PUT", joinURL.String(), &b)
resp, err := t.client.Do(req)
for { for {
if err != nil { if err != nil {
return fmt.Errorf("Unable to join: %v", err) return fmt.Errorf("Unable to join: %v", err)
@ -462,28 +474,17 @@ func (s *PeerServer) joinByPeer(server raft.Server, peer string, scheme string)
t.CancelWhenTimeout(req) t.CancelWhenTimeout(req)
if resp.StatusCode == http.StatusOK { if resp.StatusCode == http.StatusOK {
r := bufio.NewReader(resp.Body) var msg joinMessageV2
s.joinIndex, _ = binary.ReadUvarint(r) if err := json.NewDecoder(resp.Body).Decode(&msg); err != nil {
log.Debugf("Error reading join response: %v", err)
// Determine whether the server joined as a proxy or peer.
var mode uint64
if mode, err = binary.ReadUvarint(r); err == io.EOF {
mode = peerModeFlag
} else if err != nil {
log.Debugf("Error reading join mode: %v", err)
return err return err
} }
s.joinIndex = msg.CommitIndex
s.setMode(msg.Mode)
switch mode { if msg.Mode == ProxyMode {
case peerModeFlag:
s.setMode(PeerMode)
case proxyModeFlag:
s.setMode(ProxyMode)
s.proxyClientURL = resp.Header.Get("X-Leader-Client-URL") s.proxyClientURL = resp.Header.Get("X-Leader-Client-URL")
s.proxyPeerURL = resp.Header.Get("X-Leader-Peer-URL") s.proxyPeerURL = resp.Header.Get("X-Leader-Peer-URL")
default:
log.Debugf("Invalid join mode: %v", err)
return fmt.Errorf("Invalid join mode (%d): %v", mode, err)
} }
return nil return nil
@ -491,7 +492,14 @@ func (s *PeerServer) joinByPeer(server raft.Server, peer string, scheme string)
if resp.StatusCode == http.StatusTemporaryRedirect { if resp.StatusCode == http.StatusTemporaryRedirect {
address := resp.Header.Get("Location") address := resp.Header.Get("Location")
log.Debugf("Send Join Request to %s", address) log.Debugf("Send Join Request to %s", address)
json.NewEncoder(&b).Encode(NewJoinCommand(store.MinVersion(), store.MaxVersion(), server.Name(), s.Config.URL, s.server.URL())) c := &JoinCommandV1{
MinVersion: store.MinVersion(),
MaxVersion: store.MaxVersion(),
Name: server.Name(),
RaftURL: s.Config.URL,
EtcdURL: s.server.URL(),
}
json.NewEncoder(&b).Encode(c)
resp, req, err = t.Post(address, &b) resp, req, err = t.Post(address, &b)
} else if resp.StatusCode == http.StatusBadRequest { } else if resp.StatusCode == http.StatusBadRequest {

View File

@ -150,16 +150,14 @@ func (ps *PeerServer) EtcdURLHttpHandler(w http.ResponseWriter, req *http.Reques
// Response to the join request // Response to the join request
func (ps *PeerServer) JoinHttpHandler(w http.ResponseWriter, req *http.Request) { func (ps *PeerServer) JoinHttpHandler(w http.ResponseWriter, req *http.Request) {
command := &JoinCommand{} command := &JoinCommandV1{}
if err := uhttp.DecodeJsonRequest(req, command); err != nil {
err := uhttp.DecodeJsonRequest(req, command)
if err != nil {
w.WriteHeader(http.StatusInternalServerError) w.WriteHeader(http.StatusInternalServerError)
return return
} }
log.Debugf("Receive Join Request from %s", command.Name) log.Debugf("Receive Join Request from %s", command.Name)
err = ps.server.Dispatch(command, w, req) err := ps.server.Dispatch(command, w, req)
// Return status. // Return status.
if err != nil { if err != nil {
@ -199,7 +197,7 @@ func (ps *PeerServer) RemoveHttpHandler(w http.ResponseWriter, req *http.Request
} }
vars := mux.Vars(req) vars := mux.Vars(req)
command := &RemoveCommand{ command := &RemoveCommandV1{
Name: vars["name"], Name: vars["name"],
} }
@ -286,6 +284,33 @@ func (ps *PeerServer) getMachineMessage(name string) *machineMessage {
return nil return nil
} }
// Adds a machine to the cluster.
func (ps *PeerServer) addMachineHttpHandler(w http.ResponseWriter, req *http.Request) {
c := &JoinCommandV2{}
if err := uhttp.DecodeJsonRequest(req, c); err != nil {
w.WriteHeader(http.StatusInternalServerError)
return
}
log.Debugf("Receive Join Request (v2) from %s", c.Name)
if err := ps.server.Dispatch(c, w, req); err != nil {
if etcdErr, ok := err.(*etcdErr.Error); ok {
log.Debug("Return error: ", (*etcdErr).Error())
etcdErr.Write(w)
} else {
http.Error(w, err.Error(), http.StatusInternalServerError)
}
}
}
// Removes a machine from the cluster.
func (ps *PeerServer) removeMachineHttpHandler(w http.ResponseWriter, req *http.Request) {
vars := mux.Vars(req)
c := &RemoveCommandV2{Name: vars["name"]}
log.Debugf("[recv] Remove Request [%s]", c.Name)
ps.server.Dispatch(c, w, req)
}
// Response to the name request // Response to the name request
func (ps *PeerServer) NameHttpHandler(w http.ResponseWriter, req *http.Request) { func (ps *PeerServer) NameHttpHandler(w http.ResponseWriter, req *http.Request) {
log.Debugf("[recv] Get %s/name/ ", ps.Config.URL) log.Debugf("[recv] Get %s/name/ ", ps.Config.URL)

View File

@ -65,13 +65,15 @@ func (r *Registry) Proxies() []string {
// RegisterPeer adds a peer to the registry. // RegisterPeer adds a peer to the registry.
func (r *Registry) RegisterPeer(name string, peerURL string, machURL string) error { func (r *Registry) RegisterPeer(name string, peerURL string, machURL string) error {
// TODO(benbjohnson): Disallow peers that are already proxies. if err := r.register(RegistryPeerKey, name, peerURL, machURL); err != nil {
return r.register(RegistryPeerKey, name, peerURL, machURL) return err
}
r.peers[name] = r.load(RegistryPeerKey, name)
return nil
} }
// RegisterProxy adds a proxy to the registry. // RegisterProxy adds a proxy to the registry.
func (r *Registry) RegisterProxy(name string, peerURL string, machURL string) error { func (r *Registry) RegisterProxy(name string, peerURL string, machURL string) error {
// TODO(benbjohnson): Disallow proxies that are already peers.
if err := r.register(RegistryProxyKey, name, peerURL, machURL); err != nil { if err := r.register(RegistryProxyKey, name, peerURL, machURL); err != nil {
return err return err
} }

View File

@ -2,6 +2,7 @@ package server
import ( import (
"encoding/binary" "encoding/binary"
"encoding/json"
"os" "os"
"github.com/coreos/etcd/log" "github.com/coreos/etcd/log"
@ -9,21 +10,22 @@ import (
) )
func init() { func init() {
raft.RegisterCommand(&RemoveCommand{}) raft.RegisterCommand(&RemoveCommandV1{})
raft.RegisterCommand(&RemoveCommandV2{})
} }
// The RemoveCommand removes a server from the cluster. // The RemoveCommandV1 removes a server from the cluster.
type RemoveCommand struct { type RemoveCommandV1 struct {
Name string `json:"name"` Name string `json:"name"`
} }
// The name of the remove command in the log // The name of the remove command in the log
func (c *RemoveCommand) CommandName() string { func (c *RemoveCommandV1) CommandName() string {
return "etcd:remove" return "etcd:remove"
} }
// Remove a server from the cluster // Remove a server from the cluster
func (c *RemoveCommand) Apply(context raft.Context) (interface{}, error) { func (c *RemoveCommandV1) Apply(context raft.Context) (interface{}, error) {
ps, _ := context.Server().Context().(*PeerServer) ps, _ := context.Server().Context().(*PeerServer)
// If this is a proxy then remove it and exit. // If this is a proxy then remove it and exit.
@ -70,3 +72,65 @@ func (c *RemoveCommand) Apply(context raft.Context) (interface{}, error) {
return b, err return b, err
} }
// RemoveCommandV2 represents a command to remove a machine from the server.
type RemoveCommandV2 struct {
Name string `json:"name"`
}
// CommandName returns the name of the command.
func (c *RemoveCommandV2) CommandName() string {
return "etcd:v2:remove"
}
// Apply removes the given machine from the cluster.
func (c *RemoveCommandV2) Apply(context raft.Context) (interface{}, error) {
ps, _ := context.Server().Context().(*PeerServer)
ret, _ := json.Marshal(removeMessageV2{CommitIndex: context.CommitIndex()})
// If this is a proxy then remove it and exit.
if ps.registry.ProxyExists(c.Name) {
if err := ps.registry.UnregisterProxy(c.Name); err != nil {
return nil, err
}
return ret, nil
}
// Remove node from the shared registry.
err := ps.registry.UnregisterPeer(c.Name)
// Delete from stats
delete(ps.followersStats.Followers, c.Name)
if err != nil {
log.Debugf("Error while unregistering: %s (%v)", c.Name, err)
return nil, err
}
// Remove peer in raft
if err := context.Server().RemovePeer(c.Name); err != nil {
log.Debugf("Unable to remove peer: %s (%v)", c.Name, err)
return nil, err
}
if c.Name == context.Server().Name() {
// the removed node is this node
// if the node is not replaying the previous logs
// and the node has sent out a join request in this
// start. It is sure that this node received a new remove
// command and need to be removed
if context.CommitIndex() > ps.joinIndex && ps.joinIndex != 0 {
log.Debugf("server [%s] is removed", context.Server().Name())
os.Exit(0)
} else {
// else ignore remove
log.Debugf("ignore previous remove command.")
}
}
return ret, nil
}
type removeMessageV2 struct {
CommitIndex uint64 `json:"commitIndex"`
}

View File

@ -262,7 +262,9 @@ func (s *Server) Dispatch(c raft.Command, w http.ResponseWriter, req *http.Reque
var url string var url string
switch c.(type) { switch c.(type) {
case *JoinCommand, *RemoveCommand: case *JoinCommandV1, *RemoveCommandV1:
url, _ = ps.registry.PeerURL(leader)
case *JoinCommandV2, *RemoveCommandV2:
url, _ = ps.registry.PeerURL(leader) url, _ = ps.registry.PeerURL(leader)
default: default:
url, _ = ps.registry.ClientURL(leader) url, _ = ps.registry.ClientURL(leader)

View File

@ -16,12 +16,12 @@ func TestClusterConfig(t *testing.T) {
assert.NoError(t, err) assert.NoError(t, err)
defer DestroyCluster(etcds) defer DestroyCluster(etcds)
resp, _ := tests.Put("http://localhost:7001/config", "application/json", bytes.NewBufferString(`{"activeSize":3, "promoteDelay":60}`)) resp, _ := tests.Put("http://localhost:7001/v2/admin/config", "application/json", bytes.NewBufferString(`{"activeSize":3, "promoteDelay":60}`))
assert.Equal(t, resp.StatusCode, 200) assert.Equal(t, resp.StatusCode, 200)
time.Sleep(1 * time.Second) time.Sleep(1 * time.Second)
resp, _ = tests.Get("http://localhost:7002/config") resp, _ = tests.Get("http://localhost:7002/v2/admin/config")
body := tests.ReadBodyJSON(resp) body := tests.ReadBodyJSON(resp)
assert.Equal(t, resp.StatusCode, 200) assert.Equal(t, resp.StatusCode, 200)
assert.Equal(t, body["activeSize"], 3) assert.Equal(t, body["activeSize"], 3)

View File

@ -51,7 +51,7 @@ func TestProxy(t *testing.T) {
assert.Equal(t, len(result.Node.Nodes), 1) assert.Equal(t, len(result.Node.Nodes), 1)
// Reconfigure with larger active size (10 nodes) and wait for promotion. // Reconfigure with larger active size (10 nodes) and wait for promotion.
resp, _ := tests.Put("http://localhost:7001/config", "application/json", bytes.NewBufferString(`{"activeSize":10, "promoteDelay":1800}`)) resp, _ := tests.Put("http://localhost:7001/v2/admin/config", "application/json", bytes.NewBufferString(`{"activeSize":10, "promoteDelay":1800}`))
if !assert.Equal(t, resp.StatusCode, 200) { if !assert.Equal(t, resp.StatusCode, 200) {
t.FailNow() t.FailNow()
} }
@ -64,7 +64,7 @@ func TestProxy(t *testing.T) {
assert.Equal(t, len(result.Node.Nodes), 0) assert.Equal(t, len(result.Node.Nodes), 0)
// Reconfigure with a smaller active size (8 nodes). // Reconfigure with a smaller active size (8 nodes).
resp, _ = tests.Put("http://localhost:7001/config", "application/json", bytes.NewBufferString(`{"activeSize":8, "promoteDelay":1800}`)) resp, _ = tests.Put("http://localhost:7001/v2/admin/config", "application/json", bytes.NewBufferString(`{"activeSize":8, "promoteDelay":1800}`))
if !assert.Equal(t, resp.StatusCode, 200) { if !assert.Equal(t, resp.StatusCode, 200) {
t.FailNow() t.FailNow()
} }
@ -107,7 +107,7 @@ func TestProxyAutoPromote(t *testing.T) {
assert.Equal(t, len(result.Node.Nodes), 1) assert.Equal(t, len(result.Node.Nodes), 1)
// Reconfigure with a short promote delay (2 second). // Reconfigure with a short promote delay (2 second).
resp, _ := tests.Put("http://localhost:7001/config", "application/json", bytes.NewBufferString(`{"activeSize":9, "promoteDelay":2}`)) resp, _ := tests.Put("http://localhost:7001/v2/admin/config", "application/json", bytes.NewBufferString(`{"activeSize":9, "promoteDelay":2}`))
if !assert.Equal(t, resp.StatusCode, 200) { if !assert.Equal(t, resp.StatusCode, 200) {
t.FailNow() t.FailNow()
} }

View File

@ -25,7 +25,7 @@ func TestRemoveNode(t *testing.T) {
c.SyncCluster() c.SyncCluster()
rmReq, _ := http.NewRequest("DELETE", "http://127.0.0.1:7001/remove/node3", nil) rmReq, _ := http.NewRequest("DELETE", "http://127.0.0.1:7001/v2/admin/machines/node3", nil)
client := &http.Client{} client := &http.Client{}
for i := 0; i < 2; i++ { for i := 0; i < 2; i++ {

View File

@ -477,6 +477,7 @@ func (s *server) Stop() {
// make sure the server has stopped before we close the log // make sure the server has stopped before we close the log
<-s.stopped <-s.stopped
s.log.close() s.log.close()
s.state = Stopped
} }
// Checks if the server is currently running. // Checks if the server is currently running.