From 1d961b8e56c939dfb4c5b27fa19775a5649f190a Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Tue, 18 Feb 2014 13:29:18 -0700 Subject: [PATCH 01/15] Add proxy mode. --- error/error.go | 7 ++ server/cluster_config.go | 33 +++++ server/join_command.go | 22 ++-- server/peer_server.go | 134 +++++++++++++++++++- server/peer_server_handlers.go | 17 +++ server/promote_command.go | 64 ++++++++++ server/registry.go | 180 +++++++++++++++++++++------ server/remove_command.go | 7 +- server/server.go | 49 +++++--- server/set_cluster_config_command.go | 25 ++++ server/v2/get_handler.go | 6 + tests/functional/proxy_test.go | 45 +++++++ tests/functional/util.go | 2 +- 13 files changed, 522 insertions(+), 69 deletions(-) create mode 100644 server/cluster_config.go create mode 100644 server/promote_command.go create mode 100644 server/set_cluster_config_command.go create mode 100644 tests/functional/proxy_test.go diff --git a/error/error.go b/error/error.go index cc86c9117..447a5f7c3 100644 --- a/error/error.go +++ b/error/error.go @@ -51,6 +51,10 @@ const ( EcodeWatcherCleared = 400 EcodeEventIndexCleared = 401 + EcodeProxyInternal = 402 + EcodeInvalidActiveSize = 403 + EcodeInvalidPromoteDelay = 404 + EcodePromoteError = 405 ) func init() { @@ -86,6 +90,9 @@ func init() { // etcd related errors errors[EcodeWatcherCleared] = "watcher is cleared due to etcd recovery" errors[EcodeEventIndexCleared] = "The event in requested index is outdated and cleared" + errors[EcodeProxyInternal] = "Proxy Internal Error" + errors[EcodeInvalidActiveSize] = "Invalid active size" + errors[EcodeInvalidPromoteDelay] = "Proxy promote delay" } diff --git a/server/cluster_config.go b/server/cluster_config.go new file mode 100644 index 000000000..807cfa7dd --- /dev/null +++ b/server/cluster_config.go @@ -0,0 +1,33 @@ +package server + +import ( + "time" +) + +const ( + // DefaultActiveSize is the default number of active followers allowed. + DefaultActiveSize = 9 + + // DefaultPromoteDelay is the default elapsed time before promotion. + DefaultPromoteDelay = int((30 * time.Minute) / time.Second) +) + +// ClusterConfig represents cluster-wide configuration settings. +// These settings can only be changed through Raft. +type ClusterConfig struct { + // ActiveSize is the maximum number of node that can join as Raft followers. + // Nodes that join the cluster after the limit is reached are proxies. + ActiveSize int `json:"activeSize"` + + // PromoteDelay is the amount of time, in seconds, after a node is + // unreachable that it will be swapped out for a proxy node, if available. + PromoteDelay int `json:"PromoteDelay"` +} + +// NewClusterConfig returns a cluster configuration with default settings. +func NewClusterConfig() *ClusterConfig { + return &ClusterConfig{ + ActiveSize: DefaultActiveSize, + PromoteDelay: DefaultPromoteDelay, + } +} diff --git a/server/join_command.go b/server/join_command.go index de65db9eb..567e6e182 100644 --- a/server/join_command.go +++ b/server/join_command.go @@ -1,9 +1,9 @@ package server import ( + "bytes" "encoding/binary" - etcdErr "github.com/coreos/etcd/error" "github.com/coreos/etcd/log" "github.com/coreos/etcd/third_party/github.com/coreos/raft" ) @@ -40,25 +40,30 @@ func (c *JoinCommand) CommandName() string { func (c *JoinCommand) Apply(context raft.Context) (interface{}, error) { ps, _ := context.Server().Context().(*PeerServer) + var buf bytes.Buffer b := make([]byte, 8) - binary.PutUvarint(b, context.CommitIndex()) + n := binary.PutUvarint(b, context.CommitIndex()) + buf.Write(b[:n]) // 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 b, nil + binary.Write(&buf, binary.BigEndian, uint8(0)) // Mark as peer. + return buf.Bytes(), nil } // Check peer number in the cluster - if ps.registry.Count() == ps.Config.MaxClusterSize { - log.Debug("Reject join request from ", c.Name) - return []byte{0}, etcdErr.NewError(etcdErr.EcodeNoMorePeer, "", context.CommitIndex()) + if ps.registry.PeerCount() >= ps.ClusterConfig().ActiveSize { + log.Debug("Join as proxy ", c.Name) + ps.registry.RegisterProxy(c.Name, c.RaftURL, c.EtcdURL) + binary.Write(&buf, binary.BigEndian, uint8(1)) // Mark as proxy. + return buf.Bytes(), nil } // Add to shared peer registry. - ps.registry.Register(c.Name, c.RaftURL, c.EtcdURL) + ps.registry.RegisterPeer(c.Name, c.RaftURL, c.EtcdURL) // Add peer in raft err := context.Server().AddPeer(c.Name, "") @@ -69,7 +74,8 @@ func (c *JoinCommand) Apply(context raft.Context) (interface{}, error) { ps.followersStats.Followers[c.Name].Latency.Minimum = 1 << 63 } - return b, err + binary.Write(&buf, binary.BigEndian, uint8(0)) // Mark as peer. + return buf.Bytes(), err } func (c *JoinCommand) NodeName() string { diff --git a/server/peer_server.go b/server/peer_server.go index 422733c3b..c0783e9d5 100644 --- a/server/peer_server.go +++ b/server/peer_server.go @@ -2,12 +2,16 @@ package server import ( "bytes" + "bufio" "encoding/binary" "encoding/json" "fmt" + "io" "io/ioutil" + "math/rand" "net/http" "net/url" + "sort" "strconv" "time" @@ -22,19 +26,20 @@ import ( ) const ThresholdMonitorTimeout = 5 * time.Second +const ActiveMonitorTimeout = 5 * time.Second type PeerServerConfig struct { Name string Scheme string URL string SnapshotCount int - MaxClusterSize int RetryTimes int RetryInterval float64 } type PeerServer struct { Config PeerServerConfig + clusterConfig *ClusterConfig raftServer raft.Server server *Server joinIndex uint64 @@ -43,10 +48,14 @@ type PeerServer struct { registry *Registry store store.Store snapConf *snapshotConf + mode Mode closeChan chan bool timeoutThresholdChan chan interface{} + proxyPeerURL string + proxyClientURL string + metrics *metrics.Bucket } @@ -66,6 +75,7 @@ type snapshotConf struct { func NewPeerServer(psConfig PeerServerConfig, registry *Registry, store store.Store, mb *metrics.Bucket, followersStats *raftFollowersStats, serverStats *raftServerStats) *PeerServer { s := &PeerServer{ Config: psConfig, + clusterConfig: NewClusterConfig(), registry: registry, store: store, followersStats: followersStats, @@ -100,6 +110,50 @@ func (s *PeerServer) SetRaftServer(raftServer raft.Server) { s.raftServer = raftServer } +// Mode retrieves the current mode of the server. +func (s *PeerServer) Mode() Mode { + return s.mode +} + +// SetMode updates the current mode of the server. +// Switching to a peer mode will start the Raft server. +// Switching to a proxy mode will stop the Raft server. +func (s *PeerServer) SetMode(mode Mode) { + s.mode = mode + + switch mode { + case PeerMode: + if s.raftServer.Running() { + s.raftServer.Start() + } + case ProxyMode: + if !s.raftServer.Running() { + s.raftServer.Stop() + } + } +} + +// ClusterConfig retrieves the current cluster configuration. +func (s *PeerServer) ClusterConfig() *ClusterConfig { + return s.clusterConfig +} + +// SetClusterConfig updates the current cluster configuration. +// Adjusting the active size will +func (s *PeerServer) SetClusterConfig(c *ClusterConfig) error { + prevActiveSize := s.clusterConfig.ActiveSize + s.clusterConfig = c + + // Validate configuration. + if c.ActiveSize < 1 { + return etcdErr.NewError(etcdErr.EcodeInvalidActiveSize, "Post", 0) + } else if c.PromoteDelay < 0 { + return etcdErr.NewError(etcdErr.EcodeInvalidPromoteDelay, "Post", 0) + } + + return nil +} + // Helper function to do discovery and return results in expected format func (s *PeerServer) handleDiscovery(discoverURL string) (peers []string, err error) { peers, err = discovery.Do(discoverURL, s.Config.Name, s.Config.URL) @@ -213,6 +267,7 @@ func (s *PeerServer) Start(snapshot bool, discoverURL string, peers []string) er go s.monitorSync() go s.monitorTimeoutThreshold(s.closeChan) + go s.monitorActive(s.closeChan) // open the snapshot if snapshot { @@ -240,6 +295,8 @@ func (s *PeerServer) HTTPHandler() http.Handler { router.HandleFunc("/upgrade", s.UpgradeHttpHandler) router.HandleFunc("/join", s.JoinHttpHandler) router.HandleFunc("/remove/{name:.+}", s.RemoveHttpHandler) + router.HandleFunc("/config", s.getClusterConfigHttpHandler).Methods("GET") + router.HandleFunc("/config", s.setClusterConfigHttpHandler).Methods("POST") router.HandleFunc("/vote", s.VoteHttpHandler) router.HandleFunc("/log", s.GetLogHttpHandler) router.HandleFunc("/log/append", s.AppendEntriesHttpHandler) @@ -385,8 +442,30 @@ func (s *PeerServer) joinByPeer(server raft.Server, peer string, scheme string) t.CancelWhenTimeout(req) if resp.StatusCode == http.StatusOK { - b, _ := ioutil.ReadAll(resp.Body) - s.joinIndex, _ = binary.Uvarint(b) + r := bufio.NewReader(resp.Body) + s.joinIndex, _ = binary.ReadUvarint(r) + + // Determine whether the server joined as a proxy or peer. + var mode uint64 + if mode, err = binary.ReadUvarint(r); err == io.EOF { + mode = 0 + } else if err != nil { + log.Debugf("Error reading join mode: %v", err) + return err + } + + switch mode { + case 0: + s.SetMode(PeerMode) + case 1: + s.SetMode(ProxyMode) + s.proxyClientURL = resp.Header.Get("X-Leader-Client-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 } if resp.StatusCode == http.StatusTemporaryRedirect { @@ -532,3 +611,52 @@ func (s *PeerServer) monitorTimeoutThreshold(closeChan chan bool) { time.Sleep(ThresholdMonitorTimeout) } } + +// monitorActive periodically checks the status of cluster nodes and swaps them +// out for proxies as needed. +func (s *PeerServer) monitorActive(closeChan chan bool) { + for { + select { + case <- time.After(ActiveMonitorTimeout): + case <-closeChan: + return + } + + // Ignore while this peer is not a leader. + if s.raftServer.State() != raft.Leader { + continue + } + + // Retrieve target active size and actual active size. + activeSize := s.ClusterConfig().ActiveSize + peerCount := s.registry.PeerCount() + proxies := s.registry.Proxies() + peers := s.registry.Peers() + if index := sort.SearchStrings(peers, s.Config.Name); index < len(peers) && peers[index] == s.Config.Name { + peers = append(peers[:index], peers[index+1:]...) + } + + // If we have more active nodes than we should then demote. + if peerCount > activeSize { + peer := peers[rand.Intn(len(peers))] + if _, err := s.raftServer.Do(&RemoveCommand{Name: peer}); err != nil { + log.Infof("%s: warning: demotion error: %v", s.Config.Name, err) + } + continue + } + } +} + + +// Mode represents whether the server is an active peer or if the server is +// simply acting as a proxy. +type Mode string + +const ( + // PeerMode is when the server is an active node in Raft. + PeerMode = Mode("peer") + + // ProxyMode is when the server is an inactive, request-forwarding node. + ProxyMode = Mode("proxy") +) + diff --git a/server/peer_server_handlers.go b/server/peer_server_handlers.go index 6b60e2d5e..96abbd0be 100644 --- a/server/peer_server_handlers.go +++ b/server/peer_server_handlers.go @@ -188,6 +188,23 @@ func (ps *PeerServer) RemoveHttpHandler(w http.ResponseWriter, req *http.Request ps.server.Dispatch(command, w, req) } +// Returns a JSON-encoded cluster configuration. +func (ps *PeerServer) getClusterConfigHttpHandler(w http.ResponseWriter, req *http.Request) { + json.NewEncoder(w).Encode(&ps.clusterConfig) +} + +// Updates the cluster configuration. +func (ps *PeerServer) setClusterConfigHttpHandler(w http.ResponseWriter, req *http.Request) { + c := &SetClusterConfigCommand{Config:&ClusterConfig{}} + if err := json.NewDecoder(req.Body).Decode(&c.Config); err != nil { + http.Error(w, err.Error(), http.StatusInternalServerError) + return + } + + log.Debugf("[recv] Update Cluster Config Request") + ps.server.Dispatch(c, w, req) +} + // Response to the name request func (ps *PeerServer) NameHttpHandler(w http.ResponseWriter, req *http.Request) { log.Debugf("[recv] Get %s/name/ ", ps.Config.URL) diff --git a/server/promote_command.go b/server/promote_command.go new file mode 100644 index 000000000..9558abcdb --- /dev/null +++ b/server/promote_command.go @@ -0,0 +1,64 @@ +package server + +import ( + "github.com/coreos/etcd/log" + "github.com/coreos/etcd/third_party/github.com/coreos/raft" +) + +func init() { + raft.RegisterCommand(&PromoteCommand{}) +} + +// PromoteCommand represents a Raft command for converting a proxy to a peer. +type PromoteCommand struct { + Name string `json:"name"` +} + +// CommandName returns the name of the command. +func (c *PromoteCommand) CommandName() string { + return "etcd:promote" +} + +// Apply promotes a named proxy to a peer. +func (c *PromoteCommand) Apply(context raft.Context) (interface{}, error) { + ps, _ := context.Server().Context().(*PeerServer) + config := ps.ClusterConfig() + + // If cluster size is larger than max cluster size then return an error. + if ps.registry.PeerCount() >= config.ActiveSize { + return etcdErr.NewError(etcdErr.EcodePromoteError, "", 0) + } + + // If proxy doesn't exist then return an error. + if !ps.registry.ProxyExists(c.Name) { + return etcdErr.NewError(etcdErr.EcodePromoteError, "", 0) + } + + // Retrieve proxy settings. + proxyClientURL := ps.registry.ProxyClientURL() + proxyPeerURL := ps.registry.ProxyPeerURL() + + // Remove from registry as a proxy. + if err := ps.registry.UnregisterProxy(c.Name); err != nil { + log.Info("Cannot remove proxy: ", c.Name) + return nil, err + } + + // Add to shared peer registry. + ps.registry.RegisterPeer(c.Name, c.RaftURL, c.EtcdURL) + + // Add peer in raft + err := context.Server().AddPeer(c.Name, "") + + // 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 nil, err +} + +func (c *JoinCommand) NodeName() string { + return c.Name +} diff --git a/server/registry.go b/server/registry.go index e1e99edee..1d6413cfe 100644 --- a/server/registry.go +++ b/server/registry.go @@ -13,13 +13,17 @@ import ( ) // The location of the peer URL data. -const RegistryKey = "/_etcd/machines" +const RegistryPeerKey = "/_etcd/machines" + +// The location of the proxy URL data. +const RegistryProxyKey = "/_etcd/proxies" // The Registry stores URL information for nodes. type Registry struct { sync.Mutex store store.Store - nodes map[string]*node + peers map[string]*node + proxies map[string]*node } // The internal storage format of the registry. @@ -33,61 +37,126 @@ type node struct { func NewRegistry(s store.Store) *Registry { return &Registry{ store: s, - nodes: make(map[string]*node), + peers: make(map[string]*node), + proxies: make(map[string]*node), } } -// Adds a node to the registry. -func (r *Registry) Register(name string, peerURL string, machURL string) error { +// Peers returns a list of peer names. +func (r *Registry) Peers() []string { + names := make([]string, 0, len(r.peers)) + for name, _ := range r.peers { + names = append(names, name) + } + return names +} + +// Proxies returns a list of proxy names. +func (r *Registry) Proxies() []string { + names := make([]string, 0, len(r.proxies)) + for name, _ := range r.proxies { + names = append(names, name) + } + return names +} + + +// RegisterPeer adds a peer to the registry. +func (r *Registry) RegisterPeer(name string, peerURL string, machURL string) error { + // TODO(benbjohnson): Disallow peers that are already proxies. + return r.register(RegistryPeerKey, name, peerURL, machURL) +} + +// RegisterProxy adds a proxy to the registry. +func (r *Registry) RegisterProxy(name string, peerURL string, machURL string) error { + // TODO(benbjohnson): Disallow proxies that are already peers. + return r.register(RegistryProxyKey, name, peerURL, machURL) +} + +func (r *Registry) register(key, name string, peerURL string, machURL string) error { r.Lock() defer r.Unlock() // Write data to store. - key := path.Join(RegistryKey, name) v := url.Values{} v.Set("raft", peerURL) v.Set("etcd", machURL) - _, err := r.store.Create(key, false, v.Encode(), false, store.Permanent) + _, err := r.store.Create(path.Join(key, name), false, v.Encode(), false, store.Permanent) log.Debugf("Register: %s", name) return err } -// Removes a node from the registry. -func (r *Registry) Unregister(name string) error { +// UnregisterPeer removes a peer from the registry. +func (r *Registry) UnregisterPeer(name string) error { + return r.unregister(RegistryPeerKey, name) +} + +// UnregisterProxy removes a proxy from the registry. +func (r *Registry) UnregisterProxy(name string) error { + return r.unregister(RegistryProxyKey, name) +} + +func (r *Registry) unregister(key, name string) error { r.Lock() defer r.Unlock() - // Remove from cache. - // delete(r.nodes, name) - // Remove the key from the store. - _, err := r.store.Delete(path.Join(RegistryKey, name), false, false) + _, err := r.store.Delete(path.Join(key, name), false, false) log.Debugf("Unregister: %s", name) return err } +// PeerCount returns the number of peers in the cluster. +func (r *Registry) PeerCount() int { + return r.count(RegistryPeerKey) +} + +// ProxyCount returns the number of proxies in the cluster. +func (r *Registry) ProxyCount() int { + return r.count(RegistryProxyKey) +} + // Returns the number of nodes in the cluster. -func (r *Registry) Count() int { - e, err := r.store.Get(RegistryKey, false, false) +func (r *Registry) count(key string) int { + e, err := r.store.Get(key, false, false) if err != nil { return 0 } return len(e.Node.Nodes) } +// PeerExists checks if a peer with the given name exists. +func (r *Registry) PeerExists(name string) bool { + return r.exists(RegistryPeerKey, name) +} + +// ProxyExists checks if a proxy with the given name exists. +func (r *Registry) ProxyExists(name string) bool { + return r.exists(RegistryProxyKey, name) +} + +func (r *Registry) exists(key, name string) bool { + e, err := r.store.Get(path.Join(key, name), false, false) + if err != nil { + return false + } + return (e.Node != nil) +} + + // Retrieves the client URL for a given node by name. func (r *Registry) ClientURL(name string) (string, bool) { r.Lock() defer r.Unlock() - return r.clientURL(name) + return r.clientURL(RegistryPeerKey, name) } -func (r *Registry) clientURL(name string) (string, bool) { - if r.nodes[name] == nil { - r.load(name) +func (r *Registry) clientURL(key, name string) (string, bool) { + if r.peers[name] == nil { + r.peers[name] = r.load(key, name) } - if node := r.nodes[name]; node != nil { + if node := r.peers[name]; node != nil { return node.url, true } @@ -110,73 +179,108 @@ func (r *Registry) PeerHost(name string) (string, bool) { func (r *Registry) PeerURL(name string) (string, bool) { r.Lock() defer r.Unlock() - return r.peerURL(name) + return r.peerURL(RegistryPeerKey,name) } -func (r *Registry) peerURL(name string) (string, bool) { - if r.nodes[name] == nil { - r.load(name) +func (r *Registry) peerURL(key, name string) (string, bool) { + if r.peers[name] == nil { + r.peers[name] = r.load(key, name) } - if node := r.nodes[name]; node != nil { + if node := r.peers[name]; node != nil { return node.peerURL, true } return "", false } +// Retrieves the client URL for a given proxy by name. +func (r *Registry) ProxyClientURL(name string) (string, bool) { + r.Lock() + defer r.Unlock() + return r.proxyClientURL(RegistryProxyKey, name) +} + +func (r *Registry) proxyClientURL(key, name string) (string, bool) { + if r.proxies[name] == nil { + r.proxies[name] = r.load(key, name) + } + if node := r.proxies[name]; node != nil { + return node.url, true + } + return "", false +} + +// Retrieves the peer URL for a given proxy by name. +func (r *Registry) ProxyPeerURL(name string) (string, bool) { + r.Lock() + defer r.Unlock() + return r.proxyPeerURL(RegistryProxyKey,name) +} + +func (r *Registry) proxyPeerURL(key, name string) (string, bool) { + if r.proxies[name] == nil { + r.proxies[name] = r.load(key, name) + } + if node := r.proxies[name]; node != nil { + return node.peerURL, true + } + return "", false +} + // Retrieves the Client URLs for all nodes. func (r *Registry) ClientURLs(leaderName, selfName string) []string { - return r.urls(leaderName, selfName, r.clientURL) + return r.urls(RegistryPeerKey, leaderName, selfName, r.clientURL) } // Retrieves the Peer URLs for all nodes. func (r *Registry) PeerURLs(leaderName, selfName string) []string { - return r.urls(leaderName, selfName, r.peerURL) + return r.urls(RegistryPeerKey, leaderName, selfName, r.peerURL) } // Retrieves the URLs for all nodes using url function. -func (r *Registry) urls(leaderName, selfName string, url func(name string) (string, bool)) []string { +func (r *Registry) urls(key, leaderName, selfName string, url func(key, name string) (string, bool)) []string { r.Lock() defer r.Unlock() // Build list including the leader and self. urls := make([]string, 0) - if url, _ := url(leaderName); len(url) > 0 { + if url, _ := url(key, leaderName); len(url) > 0 { urls = append(urls, url) } // Retrieve a list of all nodes. - if e, _ := r.store.Get(RegistryKey, false, false); e != nil { + if e, _ := r.store.Get(key, false, false); e != nil { // Lookup the URL for each one. for _, pair := range e.Node.Nodes { _, name := filepath.Split(pair.Key) - if url, _ := url(name); len(url) > 0 && name != leaderName { + if url, _ := url(key, name); len(url) > 0 && name != leaderName { urls = append(urls, url) } } } - log.Infof("URLs: %s / %s (%s)", leaderName, selfName, strings.Join(urls, ",")) + log.Infof("URLs: %s / %s (%s)", key, leaderName, selfName, strings.Join(urls, ",")) return urls } // Removes a node from the cache. func (r *Registry) Invalidate(name string) { - delete(r.nodes, name) + delete(r.peers, name) + delete(r.proxies, name) } // Loads the given node by name from the store into the cache. -func (r *Registry) load(name string) { +func (r *Registry) load(key, name string) *node { if name == "" { - return + return nil } // Retrieve from store. - e, err := r.store.Get(path.Join(RegistryKey, name), false, false) + e, err := r.store.Get(path.Join(RegistryPeerKey, name), false, false) if err != nil { - return + return nil } // Parse as a query string. @@ -186,7 +290,7 @@ func (r *Registry) load(name string) { } // Create node. - r.nodes[name] = &node{ + return &node{ url: m["etcd"][0], peerURL: m["raft"][0], } diff --git a/server/remove_command.go b/server/remove_command.go index 4cbf98a25..3019f9d9d 100644 --- a/server/remove_command.go +++ b/server/remove_command.go @@ -26,8 +26,13 @@ func (c *RemoveCommand) CommandName() string { func (c *RemoveCommand) Apply(context raft.Context) (interface{}, error) { ps, _ := context.Server().Context().(*PeerServer) + // If this is a proxy then remove it and exit. + if ps.registry.ProxyExists(c.Name) { + return []byte{0}, ps.registry.UnregisterProxy(c.Name) + } + // Remove node from the shared registry. - err := ps.registry.Unregister(c.Name) + err := ps.registry.UnregisterPeer(c.Name) // Delete from stats delete(ps.followersStats.Followers, c.Name) diff --git a/server/server.go b/server/server.go index 033765164..f51972b85 100644 --- a/server/server.go +++ b/server/server.go @@ -164,6 +164,17 @@ func (s *Server) handleFunc(r *mux.Router, path string, f func(http.ResponseWrit // Log request. log.Debugf("[recv] %s %s %s [%s]", req.Method, s.URL(), req.URL.Path, req.RemoteAddr) + // Forward request along if the server is a proxy. + if s.peerServer.Mode() == ProxyMode { + if s.peerServer.proxyClientURL == "" { + w.Header().Set("Content-Type", "application/json") + etcdErr.NewError(402, "", 0).Write(w) + return + } + uhttp.Redirect(s.peerServer.proxyClientURL, w, req) + return + } + // Execute handler function and return error if necessary. if err := f(w, req); err != nil { if etcdErr, ok := err.(*etcdErr.Error); ok { @@ -206,6 +217,9 @@ func (s *Server) Dispatch(c raft.Command, w http.ResponseWriter, req *http.Reque return etcdErr.NewError(300, "Empty result from raft", s.Store().Index()) } + w.Header().Set("X-Leader-Client-URL", s.url) + w.Header().Set("X-Leader-Peer-URL", ps.Config.URL) + // response for raft related commands[join/remove] if b, ok := result.([]byte); ok { w.WriteHeader(http.StatusOK) @@ -239,25 +253,24 @@ func (s *Server) Dispatch(c raft.Command, w http.ResponseWriter, req *http.Reque return nil - } else { - leader := ps.raftServer.Leader() - - // No leader available. - if leader == "" { - return etcdErr.NewError(300, "", s.Store().Index()) - } - - var url string - switch c.(type) { - case *JoinCommand, *RemoveCommand: - url, _ = ps.registry.PeerURL(leader) - default: - url, _ = ps.registry.ClientURL(leader) - } - uhttp.Redirect(url, w, req) - - return nil } + + leader := ps.raftServer.Leader() + if leader == "" { + return etcdErr.NewError(300, "", s.Store().Index()) + } + + var url string + switch c.(type) { + case *JoinCommand, *RemoveCommand: + url, _ = ps.registry.PeerURL(leader) + default: + url, _ = ps.registry.ClientURL(leader) + } + + uhttp.Redirect(url, w, req) + + return nil } // Handler to return the current version of etcd. diff --git a/server/set_cluster_config_command.go b/server/set_cluster_config_command.go new file mode 100644 index 000000000..e5954a5ec --- /dev/null +++ b/server/set_cluster_config_command.go @@ -0,0 +1,25 @@ +package server + +import ( + "github.com/coreos/etcd/third_party/github.com/coreos/raft" +) + +func init() { + raft.RegisterCommand(&SetClusterConfigCommand{}) +} + +// SetClusterConfigCommand sets the cluster-level configuration. +type SetClusterConfigCommand struct { + Config *ClusterConfig `json:"config"` +} + +// CommandName returns the name of the command. +func (c *SetClusterConfigCommand) CommandName() string { + return "etcd:setClusterConfig" +} + +// Apply updates the cluster configuration. +func (c *SetClusterConfigCommand) Apply(context raft.Context) (interface{}, error) { + ps, _ := context.Server().Context().(*PeerServer) + return nil, ps.SetClusterConfig(c.Config) +} diff --git a/server/v2/get_handler.go b/server/v2/get_handler.go index ee55a4ca6..019e18cdc 100644 --- a/server/v2/get_handler.go +++ b/server/v2/get_handler.go @@ -122,5 +122,11 @@ func writeHeaders(w http.ResponseWriter, s Server) { w.Header().Add("X-Etcd-Index", fmt.Sprint(s.Store().Index())) w.Header().Add("X-Raft-Index", fmt.Sprint(s.CommitIndex())) w.Header().Add("X-Raft-Term", fmt.Sprint(s.Term())) + if url, ok := s.ClientURL(s.Leader()); ok { + w.Header().Set("X-Leader-Client-URL", url) + } + if url, ok := s.PeerURL(s.Leader()); ok { + w.Header().Set("X-Leader-Peer-URL", url) + } w.WriteHeader(http.StatusOK) } diff --git a/tests/functional/proxy_test.go b/tests/functional/proxy_test.go new file mode 100644 index 000000000..eaa48439c --- /dev/null +++ b/tests/functional/proxy_test.go @@ -0,0 +1,45 @@ +package test + +import ( + "fmt" + "os" + "testing" + "time" + + "github.com/coreos/etcd/tests" + "github.com/coreos/etcd/third_party/github.com/coreos/go-etcd/etcd" + "github.com/coreos/etcd/third_party/github.com/stretchr/testify/assert" +) + +// Create a full cluster and then add extra an extra proxy node. +func TestProxy(t *testing.T) { + clusterSize := 10 // MaxClusterSize + 1 + _, etcds, err := CreateCluster(clusterSize, &os.ProcAttr{Files: []*os.File{nil, os.Stdout, os.Stderr}}, false) + assert.NoError(t, err) + defer DestroyCluster(etcds) + + if err != nil { + t.Fatal("cannot create cluster") + } + + c := etcd.NewClient(nil) + c.SyncCluster() + + // Set key. + time.Sleep(time.Second) + if _, err := c.Set("foo", "bar", 0); err != nil { + panic(err) + } + time.Sleep(time.Second) + + // Check that all peers and proxies have the value. + for i, _ := range etcds { + resp, err := tests.Get(fmt.Sprintf("http://localhost:%d/v2/keys/foo", 4000 + (i+1))) + if assert.NoError(t, err) { + body := tests.ReadBodyJSON(resp) + if node, _ := body["node"].(map[string]interface{}); assert.NotNil(t, node) { + assert.Equal(t, node["value"], "bar") + } + } + } +} diff --git a/tests/functional/util.go b/tests/functional/util.go index 7544e3fbc..9e5284cc2 100644 --- a/tests/functional/util.go +++ b/tests/functional/util.go @@ -109,7 +109,7 @@ func CreateCluster(size int, procAttr *os.ProcAttr, ssl bool) ([][]string, []*os } } else { strI := strconv.Itoa(i + 1) - argGroup[i] = []string{"etcd", "-name=node" + strI, "-addr=127.0.0.1:400" + strI, "-peer-addr=127.0.0.1:700" + strI, "-data-dir=/tmp/node" + strI, "-peers=127.0.0.1:7001"} + argGroup[i] = []string{"etcd", "-name=node" + strI, fmt.Sprintf("-addr=127.0.0.1:%d", 4001 + i), fmt.Sprintf("-peer-addr=127.0.0.1:%d", 7001 + i), "-data-dir=/tmp/node" + strI, "-peers=127.0.0.1:7001"} if ssl { argGroup[i] = append(argGroup[i], sslServer2...) } From f5698d3566419b75c98eb015ed089d2fde76ee7f Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Mon, 24 Feb 2014 17:01:04 -0700 Subject: [PATCH 02/15] Proxy promotion. --- config/config.go | 4 -- config/config_test.go | 37 -------------- etcd.go | 1 - server/cluster_config.go | 2 +- server/join_command.go | 5 ++ server/peer_server.go | 31 +++++++++++- server/peer_server_handlers.go | 20 ++++++++ server/promote_command.go | 64 ------------------------- server/registry.go | 29 ++++++++--- tests/functional/cluster_config_test.go | 29 +++++++++++ tests/functional/proxy_test.go | 18 ++++++- tests/server_utils.go | 1 - 12 files changed, 123 insertions(+), 118 deletions(-) delete mode 100644 server/promote_command.go create mode 100644 tests/functional/cluster_config_test.go diff --git a/config/config.go b/config/config.go index 14baaf6f9..de9889f6c 100644 --- a/config/config.go +++ b/config/config.go @@ -60,7 +60,6 @@ type Config struct { KeyFile string `toml:"key_file" env:"ETCD_KEY_FILE"` Peers []string `toml:"peers" env:"ETCD_PEERS"` PeersFile string `toml:"peers_file" env:"ETCD_PEERS_FILE"` - MaxClusterSize int `toml:"max_cluster_size" env:"ETCD_MAX_CLUSTER_SIZE"` MaxResultBuffer int `toml:"max_result_buffer" env:"ETCD_MAX_RESULT_BUFFER"` MaxRetryAttempts int `toml:"max_retry_attempts" env:"ETCD_MAX_RETRY_ATTEMPTS"` RetryInterval float64 `toml:"retry_interval" env:"ETCD_RETRY_INTERVAL"` @@ -90,7 +89,6 @@ func New() *Config { c := new(Config) c.SystemPath = DefaultSystemConfigPath c.Addr = "127.0.0.1:4001" - c.MaxClusterSize = 9 c.MaxResultBuffer = 1024 c.MaxRetryAttempts = 3 c.RetryInterval = 10.0 @@ -247,7 +245,6 @@ func (c *Config) LoadFlags(arguments []string) error { f.IntVar(&c.MaxResultBuffer, "max-result-buffer", c.MaxResultBuffer, "") f.IntVar(&c.MaxRetryAttempts, "max-retry-attempts", c.MaxRetryAttempts, "") f.Float64Var(&c.RetryInterval, "retry-interval", c.RetryInterval, "") - f.IntVar(&c.MaxClusterSize, "max-cluster-size", c.MaxClusterSize, "") f.IntVar(&c.Peer.HeartbeatTimeout, "peer-heartbeat-timeout", c.Peer.HeartbeatTimeout, "") f.IntVar(&c.Peer.ElectionTimeout, "peer-election-timeout", c.Peer.ElectionTimeout, "") @@ -281,7 +278,6 @@ func (c *Config) LoadFlags(arguments []string) error { f.StringVar(&c.DataDir, "d", c.DataDir, "(deprecated)") f.IntVar(&c.MaxResultBuffer, "m", c.MaxResultBuffer, "(deprecated)") f.IntVar(&c.MaxRetryAttempts, "r", c.MaxRetryAttempts, "(deprecated)") - f.IntVar(&c.MaxClusterSize, "maxsize", c.MaxClusterSize, "(deprecated)") f.IntVar(&c.SnapshotCount, "snapshotCount", c.SnapshotCount, "(deprecated)") // END DEPRECATED FLAGS diff --git a/config/config_test.go b/config/config_test.go index d006e4d48..bc33cd212 100644 --- a/config/config_test.go +++ b/config/config_test.go @@ -51,7 +51,6 @@ func TestConfigTOML(t *testing.T) { assert.Equal(t, c.BindAddr, "127.0.0.1:4003", "") assert.Equal(t, c.Peers, []string{"coreos.com:4001", "coreos.com:4002"}, "") assert.Equal(t, c.PeersFile, "/tmp/peers", "") - assert.Equal(t, c.MaxClusterSize, 10, "") assert.Equal(t, c.MaxResultBuffer, 512, "") assert.Equal(t, c.MaxRetryAttempts, 5, "") assert.Equal(t, c.Name, "test-name", "") @@ -101,7 +100,6 @@ func TestConfigEnv(t *testing.T) { assert.Equal(t, c.BindAddr, "127.0.0.1:4003", "") assert.Equal(t, c.Peers, []string{"coreos.com:4001", "coreos.com:4002"}, "") assert.Equal(t, c.PeersFile, "/tmp/peers", "") - assert.Equal(t, c.MaxClusterSize, 10, "") assert.Equal(t, c.MaxResultBuffer, 512, "") assert.Equal(t, c.MaxRetryAttempts, 5, "") assert.Equal(t, c.Name, "test-name", "") @@ -281,21 +279,6 @@ func TestConfigPeersFileFlag(t *testing.T) { assert.Equal(t, c.PeersFile, "/tmp/peers", "") } -// Ensures that the Max Cluster Size can be parsed from the environment. -func TestConfigMaxClusterSizeEnv(t *testing.T) { - withEnv("ETCD_MAX_CLUSTER_SIZE", "5", func(c *Config) { - assert.Nil(t, c.LoadEnv(), "") - assert.Equal(t, c.MaxClusterSize, 5, "") - }) -} - -// Ensures that a the Max Cluster Size flag can be parsed. -func TestConfigMaxClusterSizeFlag(t *testing.T) { - c := New() - assert.Nil(t, c.LoadFlags([]string{"-max-cluster-size", "5"}), "") - assert.Equal(t, c.MaxClusterSize, 5, "") -} - // Ensures that the Max Result Buffer can be parsed from the environment. func TestConfigMaxResultBufferEnv(t *testing.T) { withEnv("ETCD_MAX_RESULT_BUFFER", "512", func(c *Config) { @@ -600,26 +583,6 @@ func TestConfigDeprecatedPeersFileFlag(t *testing.T) { assert.Equal(t, stderr, "[deprecated] use -peers-file, not -CF\n", "") } -func TestConfigDeprecatedMaxClusterSizeFlag(t *testing.T) { - _, stderr := capture(func() { - c := New() - err := c.LoadFlags([]string{"-maxsize", "5"}) - assert.NoError(t, err) - assert.Equal(t, c.MaxClusterSize, 5, "") - }) - assert.Equal(t, stderr, "[deprecated] use -max-cluster-size, not -maxsize\n", "") -} - -func TestConfigDeprecatedMaxResultBufferFlag(t *testing.T) { - _, stderr := capture(func() { - c := New() - err := c.LoadFlags([]string{"-m", "512"}) - assert.NoError(t, err) - assert.Equal(t, c.MaxResultBuffer, 512, "") - }) - assert.Equal(t, stderr, "[deprecated] use -max-result-buffer, not -m\n", "") -} - func TestConfigDeprecatedMaxRetryAttemptsFlag(t *testing.T) { _, stderr := capture(func() { c := New() diff --git a/etcd.go b/etcd.go index 6984b6320..c06e14313 100644 --- a/etcd.go +++ b/etcd.go @@ -120,7 +120,6 @@ func main() { Scheme: config.PeerTLSInfo().Scheme(), URL: config.Peer.Addr, SnapshotCount: config.SnapshotCount, - MaxClusterSize: config.MaxClusterSize, RetryTimes: config.MaxRetryAttempts, RetryInterval: config.RetryInterval, } diff --git a/server/cluster_config.go b/server/cluster_config.go index 807cfa7dd..bdb1ff231 100644 --- a/server/cluster_config.go +++ b/server/cluster_config.go @@ -21,7 +21,7 @@ type ClusterConfig struct { // PromoteDelay is the amount of time, in seconds, after a node is // unreachable that it will be swapped out for a proxy node, if available. - PromoteDelay int `json:"PromoteDelay"` + PromoteDelay int `json:"promoteDelay"` } // NewClusterConfig returns a cluster configuration with default settings. diff --git a/server/join_command.go b/server/join_command.go index 567e6e182..e247efa3f 100644 --- a/server/join_command.go +++ b/server/join_command.go @@ -62,6 +62,11 @@ func (c *JoinCommand) Apply(context raft.Context) (interface{}, error) { 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. ps.registry.RegisterPeer(c.Name, c.RaftURL, c.EtcdURL) diff --git a/server/peer_server.go b/server/peer_server.go index c0783e9d5..ab6eddbe8 100644 --- a/server/peer_server.go +++ b/server/peer_server.go @@ -141,7 +141,6 @@ func (s *PeerServer) ClusterConfig() *ClusterConfig { // SetClusterConfig updates the current cluster configuration. // Adjusting the active size will func (s *PeerServer) SetClusterConfig(c *ClusterConfig) error { - prevActiveSize := s.clusterConfig.ActiveSize s.clusterConfig = c // Validate configuration. @@ -294,9 +293,10 @@ func (s *PeerServer) HTTPHandler() http.Handler { router.HandleFunc("/version/{version:[0-9]+}/check", s.VersionCheckHttpHandler) router.HandleFunc("/upgrade", s.UpgradeHttpHandler) router.HandleFunc("/join", s.JoinHttpHandler) + router.HandleFunc("/promote", s.PromoteHttpHandler).Methods("POST") router.HandleFunc("/remove/{name:.+}", s.RemoveHttpHandler) router.HandleFunc("/config", s.getClusterConfigHttpHandler).Methods("GET") - router.HandleFunc("/config", s.setClusterConfigHttpHandler).Methods("POST") + router.HandleFunc("/config", s.setClusterConfigHttpHandler).Methods("PUT") router.HandleFunc("/vote", s.VoteHttpHandler) router.HandleFunc("/log", s.GetLogHttpHandler) router.HandleFunc("/log/append", s.AppendEntriesHttpHandler) @@ -632,18 +632,45 @@ func (s *PeerServer) monitorActive(closeChan chan bool) { peerCount := s.registry.PeerCount() proxies := s.registry.Proxies() peers := s.registry.Peers() + fmt.Println("active.3»", peers) if index := sort.SearchStrings(peers, s.Config.Name); index < len(peers) && peers[index] == s.Config.Name { peers = append(peers[:index], peers[index+1:]...) } + fmt.Println("active.1»", activeSize, peerCount) + fmt.Println("active.2»", proxies) + // If we have more active nodes than we should then demote. if peerCount > activeSize { peer := peers[rand.Intn(len(peers))] + fmt.Println("active.demote»", peer) if _, err := s.raftServer.Do(&RemoveCommand{Name: peer}); err != nil { log.Infof("%s: warning: demotion error: %v", s.Config.Name, err) } continue } + + // If we don't have enough active nodes then try to promote a proxy. + if peerCount < activeSize && len(proxies) > 0 { + proxy := proxies[rand.Intn(len(proxies))] + proxyPeerURL, _ := s.registry.ProxyPeerURL(proxy) + log.Infof("%s: promoting: %v (%s)", s.Config.Name, proxy, proxyPeerURL) + + // Notify proxy to promote itself. + client := &http.Client{ + Transport: &http.Transport{ + DisableKeepAlives: false, + ResponseHeaderTimeout: ActiveMonitorTimeout, + }, + } + resp, err := client.Post(fmt.Sprintf("%s/promote", proxyPeerURL), "application/json", nil) + if err != nil { + log.Infof("%s: warning: promotion error: %v", s.Config.Name, err) + } else if resp.StatusCode != http.StatusOK { + log.Infof("%s: warning: promotion failure: %v", s.Config.Name, resp.StatusCode) + } + continue + } } } diff --git a/server/peer_server_handlers.go b/server/peer_server_handlers.go index 96abbd0be..50603cee2 100644 --- a/server/peer_server_handlers.go +++ b/server/peer_server_handlers.go @@ -3,6 +3,7 @@ package server import ( "encoding/json" "net/http" + "net/url" "strconv" "time" @@ -171,6 +172,25 @@ func (ps *PeerServer) JoinHttpHandler(w http.ResponseWriter, req *http.Request) } } +// Attempt to rejoin the cluster as a peer. +func (ps *PeerServer) PromoteHttpHandler(w http.ResponseWriter, req *http.Request) { + log.Infof("%s attempting to promote in cluster: %s", ps.Config.Name, ps.proxyPeerURL) + url, err := url.Parse(ps.proxyPeerURL) + if err != nil { + w.WriteHeader(http.StatusInternalServerError) + return + } + + err = ps.joinByPeer(ps.raftServer, url.Host, ps.Config.Scheme) + if err != nil { + log.Infof("%s error while promoting: %v", ps.Config.Name, err) + w.WriteHeader(http.StatusInternalServerError) + return + } + log.Infof("%s promoted in the cluster", ps.Config.Name) + w.WriteHeader(http.StatusOK) +} + // Response to remove request func (ps *PeerServer) RemoveHttpHandler(w http.ResponseWriter, req *http.Request) { if req.Method != "DELETE" { diff --git a/server/promote_command.go b/server/promote_command.go deleted file mode 100644 index 9558abcdb..000000000 --- a/server/promote_command.go +++ /dev/null @@ -1,64 +0,0 @@ -package server - -import ( - "github.com/coreos/etcd/log" - "github.com/coreos/etcd/third_party/github.com/coreos/raft" -) - -func init() { - raft.RegisterCommand(&PromoteCommand{}) -} - -// PromoteCommand represents a Raft command for converting a proxy to a peer. -type PromoteCommand struct { - Name string `json:"name"` -} - -// CommandName returns the name of the command. -func (c *PromoteCommand) CommandName() string { - return "etcd:promote" -} - -// Apply promotes a named proxy to a peer. -func (c *PromoteCommand) Apply(context raft.Context) (interface{}, error) { - ps, _ := context.Server().Context().(*PeerServer) - config := ps.ClusterConfig() - - // If cluster size is larger than max cluster size then return an error. - if ps.registry.PeerCount() >= config.ActiveSize { - return etcdErr.NewError(etcdErr.EcodePromoteError, "", 0) - } - - // If proxy doesn't exist then return an error. - if !ps.registry.ProxyExists(c.Name) { - return etcdErr.NewError(etcdErr.EcodePromoteError, "", 0) - } - - // Retrieve proxy settings. - proxyClientURL := ps.registry.ProxyClientURL() - proxyPeerURL := ps.registry.ProxyPeerURL() - - // Remove from registry as a proxy. - if err := ps.registry.UnregisterProxy(c.Name); err != nil { - log.Info("Cannot remove proxy: ", c.Name) - return nil, err - } - - // Add to shared peer registry. - ps.registry.RegisterPeer(c.Name, c.RaftURL, c.EtcdURL) - - // Add peer in raft - err := context.Server().AddPeer(c.Name, "") - - // 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 nil, err -} - -func (c *JoinCommand) NodeName() string { - return c.Name -} diff --git a/server/registry.go b/server/registry.go index 1d6413cfe..3a2015309 100644 --- a/server/registry.go +++ b/server/registry.go @@ -5,6 +5,7 @@ import ( "net/url" "path" "path/filepath" + "sort" "strings" "sync" @@ -48,6 +49,7 @@ func (r *Registry) Peers() []string { for name, _ := range r.peers { names = append(names, name) } + sort.Sort(sort.StringSlice(names)) return names } @@ -57,6 +59,7 @@ func (r *Registry) Proxies() []string { for name, _ := range r.proxies { names = append(names, name) } + sort.Sort(sort.StringSlice(names)) return names } @@ -70,7 +73,11 @@ func (r *Registry) RegisterPeer(name string, peerURL string, machURL string) err // RegisterProxy adds a proxy to the registry. func (r *Registry) RegisterProxy(name string, peerURL string, machURL string) error { // TODO(benbjohnson): Disallow proxies that are already peers. - return r.register(RegistryProxyKey, name, peerURL, machURL) + if err := r.register(RegistryProxyKey, name, peerURL, machURL); err != nil { + return err + } + r.proxies[name] = r.load(RegistryProxyKey, name) + return nil } func (r *Registry) register(key, name string, peerURL string, machURL string) error { @@ -153,7 +160,9 @@ func (r *Registry) ClientURL(name string) (string, bool) { func (r *Registry) clientURL(key, name string) (string, bool) { if r.peers[name] == nil { - r.peers[name] = r.load(key, name) + if node := r.load(key, name); node != nil { + r.peers[name] = node + } } if node := r.peers[name]; node != nil { @@ -184,7 +193,9 @@ func (r *Registry) PeerURL(name string) (string, bool) { func (r *Registry) peerURL(key, name string) (string, bool) { if r.peers[name] == nil { - r.peers[name] = r.load(key, name) + if node := r.load(key, name); node != nil { + r.peers[name] = node + } } if node := r.peers[name]; node != nil { @@ -203,7 +214,9 @@ func (r *Registry) ProxyClientURL(name string) (string, bool) { func (r *Registry) proxyClientURL(key, name string) (string, bool) { if r.proxies[name] == nil { - r.proxies[name] = r.load(key, name) + if node := r.load(key, name); node != nil { + r.proxies[name] = node + } } if node := r.proxies[name]; node != nil { return node.url, true @@ -215,12 +228,14 @@ func (r *Registry) proxyClientURL(key, name string) (string, bool) { func (r *Registry) ProxyPeerURL(name string) (string, bool) { r.Lock() defer r.Unlock() - return r.proxyPeerURL(RegistryProxyKey,name) + return r.proxyPeerURL(RegistryProxyKey, name) } func (r *Registry) proxyPeerURL(key, name string) (string, bool) { if r.proxies[name] == nil { - r.proxies[name] = r.load(key, name) + if node := r.load(key, name); node != nil { + r.proxies[name] = node + } } if node := r.proxies[name]; node != nil { return node.peerURL, true @@ -278,7 +293,7 @@ func (r *Registry) load(key, name string) *node { } // Retrieve from store. - e, err := r.store.Get(path.Join(RegistryPeerKey, name), false, false) + e, err := r.store.Get(path.Join(key, name), false, false) if err != nil { return nil } diff --git a/tests/functional/cluster_config_test.go b/tests/functional/cluster_config_test.go new file mode 100644 index 000000000..8f8b667f0 --- /dev/null +++ b/tests/functional/cluster_config_test.go @@ -0,0 +1,29 @@ +package test + +import ( + "bytes" + "os" + "testing" + "time" + + "github.com/coreos/etcd/tests" + "github.com/coreos/etcd/third_party/github.com/stretchr/testify/assert" +) + +// Ensure that the cluster configuration can be updated. +func TestClusterConfig(t *testing.T) { + _, etcds, err := CreateCluster(3, &os.ProcAttr{Files: []*os.File{nil, os.Stdout, os.Stderr}}, false) + assert.NoError(t, err) + defer DestroyCluster(etcds) + + resp, _ := tests.Put("http://localhost:7001/config", "application/json", bytes.NewBufferString(`{"activeSize":3, "promoteDelay":60}`)) + assert.Equal(t, resp.StatusCode, 200) + + time.Sleep(1 * time.Second) + + resp, _ = tests.Get("http://localhost:7002/config") + body := tests.ReadBodyJSON(resp) + assert.Equal(t, resp.StatusCode, 200) + assert.Equal(t, body["activeSize"], 3) + assert.Equal(t, body["promoteDelay"], 60) +} diff --git a/tests/functional/proxy_test.go b/tests/functional/proxy_test.go index eaa48439c..5c162eb93 100644 --- a/tests/functional/proxy_test.go +++ b/tests/functional/proxy_test.go @@ -1,11 +1,13 @@ package test import ( + "bytes" "fmt" "os" "testing" "time" + "github.com/coreos/etcd/server" "github.com/coreos/etcd/tests" "github.com/coreos/etcd/third_party/github.com/coreos/go-etcd/etcd" "github.com/coreos/etcd/third_party/github.com/stretchr/testify/assert" @@ -13,7 +15,7 @@ import ( // Create a full cluster and then add extra an extra proxy node. func TestProxy(t *testing.T) { - clusterSize := 10 // MaxClusterSize + 1 + clusterSize := 10 // DefaultActiveSize + 1 _, etcds, err := CreateCluster(clusterSize, &os.ProcAttr{Files: []*os.File{nil, os.Stdout, os.Stderr}}, false) assert.NoError(t, err) defer DestroyCluster(etcds) @@ -42,4 +44,18 @@ func TestProxy(t *testing.T) { } } } + + time.Sleep(server.ActiveMonitorTimeout + (1 * time.Second)) + + // 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}`)) + if !assert.Equal(t, resp.StatusCode, 200) { + t.FailNow() + } + + time.Sleep(server.ActiveMonitorTimeout + (1 * time.Second)) + + // Verify that the proxy node is now a peer. + fmt.Println("CHECK!") + time.Sleep(30 * time.Second) } diff --git a/tests/server_utils.go b/tests/server_utils.go index eefe1782d..1a86170c4 100644 --- a/tests/server_utils.go +++ b/tests/server_utils.go @@ -39,7 +39,6 @@ func RunServer(f func(*server.Server)) { URL: "http://" + testRaftURL, Scheme: "http", SnapshotCount: testSnapshotCount, - MaxClusterSize: 9, } mb := metrics.NewBucket("") From fddbf35df27d3b6f7fceb0a03542b99bfead23b7 Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Tue, 25 Feb 2014 10:02:01 -0700 Subject: [PATCH 03/15] Add automatic node promotion / demotion. --- server/demote_command.go | 49 ++++++++++++++++++++++++++++++++++ server/peer_server.go | 4 +-- tests/functional/proxy_test.go | 29 +++++++++++++++++--- 3 files changed, 77 insertions(+), 5 deletions(-) create mode 100644 server/demote_command.go diff --git a/server/demote_command.go b/server/demote_command.go new file mode 100644 index 000000000..0e832da11 --- /dev/null +++ b/server/demote_command.go @@ -0,0 +1,49 @@ +package server + +import ( + "github.com/coreos/etcd/log" + "github.com/coreos/etcd/third_party/github.com/coreos/raft" +) + +func init() { + raft.RegisterCommand(&DemoteCommand{}) +} + +// DemoteCommand represents a command to change a peer to a proxy. +type DemoteCommand struct { + Name string `json:"name"` +} + +// CommandName returns the name of the command. +func (c *DemoteCommand) CommandName() string { + return "etcd:demote" +} + +// Apply executes the command. +func (c *DemoteCommand) Apply(context raft.Context) (interface{}, error) { + ps, _ := context.Server().Context().(*PeerServer) + + // Save URLs. + clientURL, _ := ps.registry.ClientURL(c.Name) + peerURL, _ := ps.registry.PeerURL(c.Name) + + // Perform a removal. + (&RemoveCommand{Name: c.Name}).Apply(context) + + // Register node as a proxy. + ps.registry.RegisterProxy(c.Name, peerURL, clientURL) + + // Update mode if this change applies to this server. + if c.Name == ps.Config.Name { + log.Infof("Set mode after demotion: %s", c.Name) + ps.SetMode(ProxyMode) + } + + return nil, nil +} + +// NodeName returns the name of the affected node. +func (c *DemoteCommand) NodeName() string { + return c.Name +} + diff --git a/server/peer_server.go b/server/peer_server.go index ab6eddbe8..c99ff2fba 100644 --- a/server/peer_server.go +++ b/server/peer_server.go @@ -26,7 +26,7 @@ import ( ) const ThresholdMonitorTimeout = 5 * time.Second -const ActiveMonitorTimeout = 5 * time.Second +const ActiveMonitorTimeout = 1 * time.Second type PeerServerConfig struct { Name string @@ -644,7 +644,7 @@ func (s *PeerServer) monitorActive(closeChan chan bool) { if peerCount > activeSize { peer := peers[rand.Intn(len(peers))] fmt.Println("active.demote»", peer) - if _, err := s.raftServer.Do(&RemoveCommand{Name: peer}); err != nil { + if _, err := s.raftServer.Do(&DemoteCommand{Name: peer}); err != nil { log.Infof("%s: warning: demotion error: %v", s.Config.Name, err) } continue diff --git a/tests/functional/proxy_test.go b/tests/functional/proxy_test.go index 5c162eb93..d7a0b66b6 100644 --- a/tests/functional/proxy_test.go +++ b/tests/functional/proxy_test.go @@ -45,7 +45,10 @@ func TestProxy(t *testing.T) { } } - time.Sleep(server.ActiveMonitorTimeout + (1 * time.Second)) + // Verify that we have one proxy. + result, err := c.Get("_etcd/proxies", false, true) + assert.NoError(t, err) + assert.Equal(t, len(result.Node.Nodes), 1) // 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}`)) @@ -56,6 +59,26 @@ func TestProxy(t *testing.T) { time.Sleep(server.ActiveMonitorTimeout + (1 * time.Second)) // Verify that the proxy node is now a peer. - fmt.Println("CHECK!") - time.Sleep(30 * time.Second) + result, err = c.Get("_etcd/proxies", false, true) + assert.NoError(t, err) + assert.Equal(t, len(result.Node.Nodes), 0) + + // Reconfigure with a smaller active size (8 nodes). + resp, _ = tests.Put("http://localhost:7001/config", "application/json", bytes.NewBufferString(`{"activeSize":8, "promoteDelay":1800}`)) + if !assert.Equal(t, resp.StatusCode, 200) { + t.FailNow() + } + + // Wait for two monitor cycles before checking for demotion. + time.Sleep((2 * server.ActiveMonitorTimeout) + (1 * time.Second)) + + // Verify that we now have eight peers. + result, err = c.Get("_etcd/machines", false, true) + assert.NoError(t, err) + assert.Equal(t, len(result.Node.Nodes), 8) + + // Verify that we now have two proxies. + result, err = c.Get("_etcd/proxies", false, true) + assert.NoError(t, err) + assert.Equal(t, len(result.Node.Nodes), 2) } From c8d6b26dfdf150f96d06e6a0b27730776646b10e Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Fri, 28 Feb 2014 09:17:02 -0700 Subject: [PATCH 04/15] Add auto-demotion after peer inactivity. --- error/error.go | 10 +- server/cluster_config.go | 2 +- server/demote_command.go | 1 - server/join_command.go | 29 ++-- server/peer_server.go | 154 +++++++++++++-------- server/peer_server_handlers.go | 4 +- server/raft_server_stats.go | 36 ++--- server/registry.go | 16 +-- server/server.go | 30 ++-- server/transporter.go | 28 ++-- tests/functional/proxy_test.go | 60 ++++++++ third_party/github.com/coreos/raft/peer.go | 11 ++ 12 files changed, 245 insertions(+), 136 deletions(-) diff --git a/error/error.go b/error/error.go index 447a5f7c3..6ec4f95c1 100644 --- a/error/error.go +++ b/error/error.go @@ -49,12 +49,12 @@ const ( EcodeRaftInternal = 300 EcodeLeaderElect = 301 - EcodeWatcherCleared = 400 - EcodeEventIndexCleared = 401 - EcodeProxyInternal = 402 - EcodeInvalidActiveSize = 403 + EcodeWatcherCleared = 400 + EcodeEventIndexCleared = 401 + EcodeProxyInternal = 402 + EcodeInvalidActiveSize = 403 EcodeInvalidPromoteDelay = 404 - EcodePromoteError = 405 + EcodePromoteError = 405 ) func init() { diff --git a/server/cluster_config.go b/server/cluster_config.go index bdb1ff231..b47739c5d 100644 --- a/server/cluster_config.go +++ b/server/cluster_config.go @@ -27,7 +27,7 @@ type ClusterConfig struct { // NewClusterConfig returns a cluster configuration with default settings. func NewClusterConfig() *ClusterConfig { return &ClusterConfig{ - ActiveSize: DefaultActiveSize, + ActiveSize: DefaultActiveSize, PromoteDelay: DefaultPromoteDelay, } } diff --git a/server/demote_command.go b/server/demote_command.go index 0e832da11..f48ef1fe1 100644 --- a/server/demote_command.go +++ b/server/demote_command.go @@ -46,4 +46,3 @@ func (c *DemoteCommand) Apply(context raft.Context) (interface{}, error) { func (c *DemoteCommand) NodeName() string { return c.Name } - diff --git a/server/join_command.go b/server/join_command.go index e247efa3f..919ee7848 100644 --- a/server/join_command.go +++ b/server/join_command.go @@ -13,21 +13,30 @@ func init() { } // The JoinCommand adds a node to the cluster. +// +// The command returns two values back to binary format. +// The first value is a Uvarint representing the the join_index. +// The second value is a single byte flag representing whether the joining +// node is a peer (0) or a proxy (1). +// +// 8 bytes | 1 byte +// join_index | join_mode +// type JoinCommand struct { - MinVersion int `json:"minVersion"` - MaxVersion int `json:"maxVersion"` - Name string `json:"name"` - RaftURL string `json:"raftURL"` - EtcdURL string `json:"etcdURL"` + MinVersion int `json:"minVersion"` + MaxVersion int `json:"maxVersion"` + Name string `json:"name"` + RaftURL string `json:"raftURL"` + 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, + MinVersion: minVersion, + MaxVersion: maxVersion, + Name: name, + RaftURL: raftUrl, + EtcdURL: etcdUrl, } } diff --git a/server/peer_server.go b/server/peer_server.go index c99ff2fba..7a10cd25b 100644 --- a/server/peer_server.go +++ b/server/peer_server.go @@ -1,8 +1,8 @@ package server import ( - "bytes" "bufio" + "bytes" "encoding/binary" "encoding/json" "fmt" @@ -27,63 +27,64 @@ import ( const ThresholdMonitorTimeout = 5 * time.Second const ActiveMonitorTimeout = 1 * time.Second +const PeerActivityMonitorTimeout = 1 * time.Second type PeerServerConfig struct { - Name string - Scheme string - URL string - SnapshotCount int - RetryTimes int - RetryInterval float64 + Name string + Scheme string + URL string + SnapshotCount int + RetryTimes int + RetryInterval float64 } type PeerServer struct { - Config PeerServerConfig - clusterConfig *ClusterConfig - raftServer raft.Server - server *Server - joinIndex uint64 - followersStats *raftFollowersStats - serverStats *raftServerStats - registry *Registry - store store.Store - snapConf *snapshotConf - mode Mode + Config PeerServerConfig + clusterConfig *ClusterConfig + raftServer raft.Server + server *Server + joinIndex uint64 + followersStats *raftFollowersStats + serverStats *raftServerStats + registry *Registry + store store.Store + snapConf *snapshotConf + mode Mode - closeChan chan bool - timeoutThresholdChan chan interface{} + closeChan chan bool + timeoutThresholdChan chan interface{} - proxyPeerURL string + proxyPeerURL string proxyClientURL string - metrics *metrics.Bucket + metrics *metrics.Bucket } // TODO: find a good policy to do snapshot type snapshotConf struct { // Etcd will check if snapshot is need every checkingInterval - checkingInterval time.Duration + checkingInterval time.Duration // The index when the last snapshot happened - lastIndex uint64 + lastIndex uint64 // If the incremental number of index since the last snapshot // exceeds the snapshot Threshold, etcd will do a snapshot - snapshotThr uint64 + snapshotThr uint64 } func NewPeerServer(psConfig PeerServerConfig, registry *Registry, store store.Store, mb *metrics.Bucket, followersStats *raftFollowersStats, serverStats *raftServerStats) *PeerServer { s := &PeerServer{ - Config: psConfig, - clusterConfig: NewClusterConfig(), - registry: registry, - store: store, - followersStats: followersStats, - serverStats: serverStats, + Config: psConfig, + clusterConfig: NewClusterConfig(), + registry: registry, + store: store, + followersStats: followersStats, + serverStats: serverStats, - timeoutThresholdChan: make(chan interface{}, 1), + timeoutThresholdChan: make(chan interface{}, 1), - metrics: mb, + metrics: mb, } return s @@ -91,10 +92,10 @@ func NewPeerServer(psConfig PeerServerConfig, registry *Registry, store store.St func (s *PeerServer) SetRaftServer(raftServer raft.Server) { s.snapConf = &snapshotConf{ - checkingInterval: time.Second * 3, + checkingInterval: time.Second * 3, // this is not accurate, we will update raft to provide an api - lastIndex: raftServer.CommitIndex(), - snapshotThr: uint64(s.Config.SnapshotCount), + lastIndex: raftServer.CommitIndex(), + snapshotThr: uint64(s.Config.SnapshotCount), } raftServer.AddEventListener(raft.StateChangeEventType, s.raftEventLogger) @@ -267,6 +268,7 @@ func (s *PeerServer) Start(snapshot bool, discoverURL string, peers []string) er go s.monitorSync() go s.monitorTimeoutThreshold(s.closeChan) go s.monitorActive(s.closeChan) + go s.monitorPeerActivity(s.closeChan) // open the snapshot if snapshot { @@ -444,7 +446,7 @@ func (s *PeerServer) joinByPeer(server raft.Server, peer string, scheme string) if resp.StatusCode == http.StatusOK { r := bufio.NewReader(resp.Body) s.joinIndex, _ = binary.ReadUvarint(r) - + // Determine whether the server joined as a proxy or peer. var mode uint64 if mode, err = binary.ReadUvarint(r); err == io.EOF { @@ -617,7 +619,7 @@ func (s *PeerServer) monitorTimeoutThreshold(closeChan chan bool) { func (s *PeerServer) monitorActive(closeChan chan bool) { for { select { - case <- time.After(ActiveMonitorTimeout): + case <-time.After(ActiveMonitorTimeout): case <-closeChan: return } @@ -632,18 +634,13 @@ func (s *PeerServer) monitorActive(closeChan chan bool) { peerCount := s.registry.PeerCount() proxies := s.registry.Proxies() peers := s.registry.Peers() - fmt.Println("active.3»", peers) if index := sort.SearchStrings(peers, s.Config.Name); index < len(peers) && peers[index] == s.Config.Name { peers = append(peers[:index], peers[index+1:]...) } - fmt.Println("active.1»", activeSize, peerCount) - fmt.Println("active.2»", proxies) - // If we have more active nodes than we should then demote. if peerCount > activeSize { peer := peers[rand.Intn(len(peers))] - fmt.Println("active.demote»", peer) if _, err := s.raftServer.Do(&DemoteCommand{Name: peer}); err != nil { log.Infof("%s: warning: demotion error: %v", s.Config.Name, err) } @@ -652,28 +649,64 @@ func (s *PeerServer) monitorActive(closeChan chan bool) { // If we don't have enough active nodes then try to promote a proxy. if peerCount < activeSize && len(proxies) > 0 { - proxy := proxies[rand.Intn(len(proxies))] - proxyPeerURL, _ := s.registry.ProxyPeerURL(proxy) - log.Infof("%s: promoting: %v (%s)", s.Config.Name, proxy, proxyPeerURL) + loop: + for _, i := range rand.Perm(len(proxies)) { + proxy := proxies[i] + proxyPeerURL, _ := s.registry.ProxyPeerURL(proxy) + log.Infof("%s: attempting to promote: %v (%s)", s.Config.Name, proxy, proxyPeerURL) - // Notify proxy to promote itself. - client := &http.Client{ - Transport: &http.Transport{ - DisableKeepAlives: false, - ResponseHeaderTimeout: ActiveMonitorTimeout, - }, + // Notify proxy to promote itself. + client := &http.Client{ + Transport: &http.Transport{ + DisableKeepAlives: false, + ResponseHeaderTimeout: ActiveMonitorTimeout, + }, + } + resp, err := client.Post(fmt.Sprintf("%s/promote", proxyPeerURL), "application/json", nil) + if err != nil { + log.Infof("%s: warning: promotion error: %v", s.Config.Name, err) + continue + } else if resp.StatusCode != http.StatusOK { + log.Infof("%s: warning: promotion failure: %v", s.Config.Name, resp.StatusCode) + continue + } + break loop } - resp, err := client.Post(fmt.Sprintf("%s/promote", proxyPeerURL), "application/json", nil) - if err != nil { - log.Infof("%s: warning: promotion error: %v", s.Config.Name, err) - } else if resp.StatusCode != http.StatusOK { - log.Infof("%s: warning: promotion failure: %v", s.Config.Name, resp.StatusCode) - } - continue } } } +// monitorPeerActivity periodically checks for dead nodes and demotes them. +func (s *PeerServer) monitorPeerActivity(closeChan chan bool) { + for { + select { + case <-time.After(PeerActivityMonitorTimeout): + case <-closeChan: + return + } + + // Ignore while this peer is not a leader. + if s.raftServer.State() != raft.Leader { + continue + } + + // Check last activity for all peers. + now := time.Now() + promoteDelay := time.Duration(s.ClusterConfig().PromoteDelay) * time.Second + peers := s.raftServer.Peers() + for _, peer := range peers { + // If the last response from the peer is longer than the promote delay + // then automatically demote the peer. + if !peer.LastActivity().IsZero() && now.Sub(peer.LastActivity()) > promoteDelay { + log.Infof("%s: demoting node: %v; last activity %v ago", s.Config.Name, peer.Name, now.Sub(peer.LastActivity())) + if _, err := s.raftServer.Do(&DemoteCommand{Name: peer.Name}); err != nil { + log.Infof("%s: warning: autodemotion error: %v", s.Config.Name, err) + } + continue + } + } + } +} // Mode represents whether the server is an active peer or if the server is // simply acting as a proxy. @@ -681,9 +714,8 @@ type Mode string const ( // PeerMode is when the server is an active node in Raft. - PeerMode = Mode("peer") + PeerMode = Mode("peer") // ProxyMode is when the server is an inactive, request-forwarding node. ProxyMode = Mode("proxy") ) - diff --git a/server/peer_server_handlers.go b/server/peer_server_handlers.go index 50603cee2..9fe7d06b0 100644 --- a/server/peer_server_handlers.go +++ b/server/peer_server_handlers.go @@ -11,8 +11,8 @@ import ( "github.com/coreos/etcd/third_party/github.com/gorilla/mux" etcdErr "github.com/coreos/etcd/error" - uhttp "github.com/coreos/etcd/pkg/http" "github.com/coreos/etcd/log" + uhttp "github.com/coreos/etcd/pkg/http" "github.com/coreos/etcd/store" ) @@ -215,7 +215,7 @@ func (ps *PeerServer) getClusterConfigHttpHandler(w http.ResponseWriter, req *ht // Updates the cluster configuration. func (ps *PeerServer) setClusterConfigHttpHandler(w http.ResponseWriter, req *http.Request) { - c := &SetClusterConfigCommand{Config:&ClusterConfig{}} + c := &SetClusterConfigCommand{Config: &ClusterConfig{}} if err := json.NewDecoder(req.Body).Decode(&c.Config); err != nil { http.Error(w, err.Error(), http.StatusInternalServerError) return diff --git a/server/raft_server_stats.go b/server/raft_server_stats.go index 245a76573..a3c2fd108 100644 --- a/server/raft_server_stats.go +++ b/server/raft_server_stats.go @@ -7,32 +7,32 @@ import ( ) type raftServerStats struct { - Name string `json:"name"` - State string `json:"state"` - StartTime time.Time `json:"startTime"` + Name string `json:"name"` + State string `json:"state"` + StartTime time.Time `json:"startTime"` - LeaderInfo struct { - Name string `json:"leader"` - Uptime string `json:"uptime"` - startTime time.Time - } `json:"leaderInfo"` + LeaderInfo struct { + Name string `json:"leader"` + Uptime string `json:"uptime"` + startTime time.Time + } `json:"leaderInfo"` - RecvAppendRequestCnt uint64 `json:"recvAppendRequestCnt,"` - RecvingPkgRate float64 `json:"recvPkgRate,omitempty"` - RecvingBandwidthRate float64 `json:"recvBandwidthRate,omitempty"` + RecvAppendRequestCnt uint64 `json:"recvAppendRequestCnt,"` + RecvingPkgRate float64 `json:"recvPkgRate,omitempty"` + RecvingBandwidthRate float64 `json:"recvBandwidthRate,omitempty"` - SendAppendRequestCnt uint64 `json:"sendAppendRequestCnt"` - SendingPkgRate float64 `json:"sendPkgRate,omitempty"` - SendingBandwidthRate float64 `json:"sendBandwidthRate,omitempty"` + SendAppendRequestCnt uint64 `json:"sendAppendRequestCnt"` + SendingPkgRate float64 `json:"sendPkgRate,omitempty"` + SendingBandwidthRate float64 `json:"sendBandwidthRate,omitempty"` - sendRateQueue *statsQueue - recvRateQueue *statsQueue + sendRateQueue *statsQueue + recvRateQueue *statsQueue } func NewRaftServerStats(name string) *raftServerStats { return &raftServerStats{ - Name: name, - StartTime: time.Now(), + Name: name, + StartTime: time.Now(), sendRateQueue: &statsQueue{ back: -1, }, diff --git a/server/registry.go b/server/registry.go index 3a2015309..1447b40db 100644 --- a/server/registry.go +++ b/server/registry.go @@ -22,8 +22,8 @@ const RegistryProxyKey = "/_etcd/proxies" // The Registry stores URL information for nodes. type Registry struct { sync.Mutex - store store.Store - peers map[string]*node + store store.Store + peers map[string]*node proxies map[string]*node } @@ -37,13 +37,13 @@ type node struct { // Creates a new Registry. func NewRegistry(s store.Store) *Registry { return &Registry{ - store: s, - peers: make(map[string]*node), + store: s, + peers: make(map[string]*node), proxies: make(map[string]*node), } } -// Peers returns a list of peer names. +// Peers returns a list of cached peer names. func (r *Registry) Peers() []string { names := make([]string, 0, len(r.peers)) for name, _ := range r.peers { @@ -53,7 +53,7 @@ func (r *Registry) Peers() []string { return names } -// Proxies returns a list of proxy names. +// Proxies returns a list of cached proxy names. func (r *Registry) Proxies() []string { names := make([]string, 0, len(r.proxies)) for name, _ := range r.proxies { @@ -63,7 +63,6 @@ func (r *Registry) Proxies() []string { return names } - // RegisterPeer adds a peer to the registry. func (r *Registry) RegisterPeer(name string, peerURL string, machURL string) error { // TODO(benbjohnson): Disallow peers that are already proxies. @@ -150,7 +149,6 @@ func (r *Registry) exists(key, name string) bool { return (e.Node != nil) } - // Retrieves the client URL for a given node by name. func (r *Registry) ClientURL(name string) (string, bool) { r.Lock() @@ -188,7 +186,7 @@ func (r *Registry) PeerHost(name string) (string, bool) { func (r *Registry) PeerURL(name string) (string, bool) { r.Lock() defer r.Unlock() - return r.peerURL(RegistryPeerKey,name) + return r.peerURL(RegistryPeerKey, name) } func (r *Registry) peerURL(key, name string) (string, bool) { diff --git a/server/server.go b/server/server.go index f51972b85..51c0a17a0 100644 --- a/server/server.go +++ b/server/server.go @@ -12,10 +12,10 @@ import ( "github.com/coreos/etcd/third_party/github.com/gorilla/mux" etcdErr "github.com/coreos/etcd/error" + ehttp "github.com/coreos/etcd/http" "github.com/coreos/etcd/log" "github.com/coreos/etcd/metrics" "github.com/coreos/etcd/mod" - ehttp "github.com/coreos/etcd/http" uhttp "github.com/coreos/etcd/pkg/http" "github.com/coreos/etcd/server/v1" "github.com/coreos/etcd/server/v2" @@ -25,26 +25,26 @@ import ( // This is the default implementation of the Server interface. type Server struct { - Name string - url string - handler http.Handler - peerServer *PeerServer - registry *Registry - store store.Store - metrics *metrics.Bucket + Name string + url string + handler http.Handler + peerServer *PeerServer + registry *Registry + store store.Store + metrics *metrics.Bucket - trace bool + trace bool } // Creates a new Server. func New(name, url string, peerServer *PeerServer, registry *Registry, store store.Store, mb *metrics.Bucket) *Server { s := &Server{ - Name: name, - url: url, - store: store, - registry: registry, - peerServer: peerServer, - metrics: mb, + Name: name, + url: url, + store: store, + registry: registry, + peerServer: peerServer, + metrics: mb, } return s diff --git a/server/transporter.go b/server/transporter.go index 4415d91a1..06954d4f7 100644 --- a/server/transporter.go +++ b/server/transporter.go @@ -15,13 +15,13 @@ import ( // Transporter layer for communication between raft nodes type transporter struct { - requestTimeout time.Duration - followersStats *raftFollowersStats - serverStats *raftServerStats - registry *Registry + requestTimeout time.Duration + followersStats *raftFollowersStats + serverStats *raftServerStats + registry *Registry - client *http.Client - transport *http.Transport + client *http.Client + transport *http.Transport } type dialer func(network, addr string) (net.Conn, error) @@ -34,16 +34,16 @@ func NewTransporter(followersStats *raftFollowersStats, serverStats *raftServerS Dial: func(network, addr string) (net.Conn, error) { return net.DialTimeout(network, addr, dialTimeout) }, - ResponseHeaderTimeout: responseHeaderTimeout, + ResponseHeaderTimeout: responseHeaderTimeout, } t := transporter{ - client: &http.Client{Transport: tr}, - transport: tr, - requestTimeout: requestTimeout, - followersStats: followersStats, - serverStats: serverStats, - registry: registry, + client: &http.Client{Transport: tr}, + transport: tr, + requestTimeout: requestTimeout, + followersStats: followersStats, + serverStats: serverStats, + registry: registry, } return &t @@ -73,7 +73,7 @@ func (t *transporter) SendAppendEntriesRequest(server raft.Server, peer *raft.Pe thisFollowerStats, ok := t.followersStats.Followers[peer.Name] - if !ok { //this is the first time this follower has been seen + if !ok { //this is the first time this follower has been seen thisFollowerStats = &raftFollowerStats{} thisFollowerStats.Latency.Minimum = 1 << 63 t.followersStats.Followers[peer.Name] = thisFollowerStats diff --git a/tests/functional/proxy_test.go b/tests/functional/proxy_test.go index d7a0b66b6..03de0120d 100644 --- a/tests/functional/proxy_test.go +++ b/tests/functional/proxy_test.go @@ -82,3 +82,63 @@ func TestProxy(t *testing.T) { assert.NoError(t, err) assert.Equal(t, len(result.Node.Nodes), 2) } + +// Create a full cluster, disconnect a peer, wait for autodemotion, wait for autopromotion. +func TestProxyAutoPromote(t *testing.T) { + clusterSize := 10 // DefaultActiveSize + 1 + _, etcds, err := CreateCluster(clusterSize, &os.ProcAttr{Files: []*os.File{nil, os.Stdout, os.Stderr}}, false) + if err != nil { + t.Fatal("cannot create cluster") + } + defer func() { + // Wrap this in a closure so that it picks up the updated version of + // the "etcds" variable. + DestroyCluster(etcds) + }() + + c := etcd.NewClient(nil) + c.SyncCluster() + + time.Sleep(1 * time.Second) + + // Verify that we have one proxy. + result, err := c.Get("_etcd/proxies", false, true) + assert.NoError(t, err) + assert.Equal(t, len(result.Node.Nodes), 1) + + // Reconfigure with a short promote delay (1 second). + resp, _ := tests.Put("http://localhost:7001/config", "application/json", bytes.NewBufferString(`{"activeSize":9, "promoteDelay":1}`)) + if !assert.Equal(t, resp.StatusCode, 200) { + t.FailNow() + } + + // Remove peer. + etcd := etcds[1] + etcds = append(etcds[:1], etcds[2:]...) + if err := etcd.Kill(); err != nil { + panic(err.Error()) + } + etcd.Release() + + // Wait for it to get dropped. + time.Sleep(server.PeerActivityMonitorTimeout + (1 * time.Second)) + + // Wait for the proxy to be promoted. + time.Sleep(server.ActiveMonitorTimeout + (1 * time.Second)) + + // Verify that we have 9 peers. + result, err = c.Get("_etcd/machines", true, true) + assert.NoError(t, err) + assert.Equal(t, len(result.Node.Nodes), 9) + + // Verify that node10 is one of those peers. + result, err = c.Get("_etcd/machines/node10", false, false) + assert.NoError(t, err) + + // Verify that there are no more proxies. + result, err = c.Get("_etcd/proxies", false, true) + assert.NoError(t, err) + if assert.Equal(t, len(result.Node.Nodes), 1) { + assert.Equal(t, result.Node.Nodes[0].Key, "/_etcd/proxies/node2") + } +} diff --git a/third_party/github.com/coreos/raft/peer.go b/third_party/github.com/coreos/raft/peer.go index b7ab84feb..e9101364c 100644 --- a/third_party/github.com/coreos/raft/peer.go +++ b/third_party/github.com/coreos/raft/peer.go @@ -20,6 +20,7 @@ type Peer struct { mutex sync.RWMutex stopChan chan bool heartbeatInterval time.Duration + lastActivity time.Time } //------------------------------------------------------------------------------ @@ -67,6 +68,11 @@ func (p *Peer) setPrevLogIndex(value uint64) { p.prevLogIndex = value } +// LastActivity returns the last time any response was received from the peer. +func (p *Peer) LastActivity() time.Time { + return p.lastActivity +} + //------------------------------------------------------------------------------ // // Methods @@ -103,6 +109,7 @@ func (p *Peer) clone() *Peer { Name: p.Name, ConnectionString: p.ConnectionString, prevLogIndex: p.prevLogIndex, + lastActivity: p.lastActivity, } } @@ -176,6 +183,7 @@ func (p *Peer) sendAppendEntriesRequest(req *AppendEntriesRequest) { // If successful then update the previous log index. p.mutex.Lock() + p.lastActivity = time.Now() if resp.Success() { if len(req.Entries) > 0 { p.prevLogIndex = req.Entries[len(req.Entries)-1].GetIndex() @@ -243,6 +251,7 @@ func (p *Peer) sendSnapshotRequest(req *SnapshotRequest) { // If successful, the peer should have been to snapshot state // Send it the snapshot! + p.lastActivity = time.Now() if resp.Success { p.sendSnapshotRecoveryRequest() } else { @@ -263,6 +272,7 @@ func (p *Peer) sendSnapshotRecoveryRequest() { return } + p.lastActivity = time.Now() if resp.Success { p.prevLogIndex = req.LastIndex } else { @@ -283,6 +293,7 @@ func (p *Peer) sendVoteRequest(req *RequestVoteRequest, c chan *RequestVoteRespo req.peer = p if resp := p.server.Transporter().SendVoteRequest(p.server, p, req); resp != nil { debugln("peer.vote.recv: ", p.server.Name(), "<-", p.Name) + p.lastActivity = time.Now() resp.peer = p c <- resp } else { From 063f3b574c0e673009aba9ddcb99b387dddb5408 Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Tue, 4 Mar 2014 08:53:13 -0700 Subject: [PATCH 05/15] Check for peer before demotion. --- server/demote_command.go | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/server/demote_command.go b/server/demote_command.go index f48ef1fe1..a224b7baf 100644 --- a/server/demote_command.go +++ b/server/demote_command.go @@ -1,6 +1,8 @@ package server import ( + "fmt" + "github.com/coreos/etcd/log" "github.com/coreos/etcd/third_party/github.com/coreos/raft" ) @@ -23,6 +25,11 @@ func (c *DemoteCommand) CommandName() string { func (c *DemoteCommand) Apply(context raft.Context) (interface{}, error) { ps, _ := context.Server().Context().(*PeerServer) + // Ignore this command if there is no peer. + if !ps.registry.PeerExists(c.Name) { + return nil, fmt.Errorf("peer does not exist: %s", c.Name) + } + // Save URLs. clientURL, _ := ps.registry.ClientURL(c.Name) peerURL, _ := ps.registry.PeerURL(c.Name) From c3b4afc13969e263c3919d7e145f7584ce1f6d2e Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Tue, 4 Mar 2014 08:56:10 -0700 Subject: [PATCH 06/15] Fix JoinCommand comment. --- server/join_command.go | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/server/join_command.go b/server/join_command.go index 919ee7848..871faedf8 100644 --- a/server/join_command.go +++ b/server/join_command.go @@ -14,12 +14,10 @@ func init() { // The JoinCommand adds a node to the cluster. // -// The command returns two values back to binary format. -// The first value is a Uvarint representing the the join_index. -// The second value is a single byte flag representing whether the joining -// node is a peer (0) or a proxy (1). +// The command returns the join_index (Uvarint) and peer flag (peer=0, proxy=1) +// in following binary format: // -// 8 bytes | 1 byte +// 8 bytes | 1 byte // join_index | join_mode // type JoinCommand struct { From fe4dee03ab0933c90b314d46aa59e872452254f2 Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Tue, 4 Mar 2014 09:15:15 -0700 Subject: [PATCH 07/15] Minor fixes. --- error/error.go | 1 + server/demote_command.go | 2 +- server/peer_server.go | 26 ++++++++++++++------------ 3 files changed, 16 insertions(+), 13 deletions(-) diff --git a/error/error.go b/error/error.go index 6ec4f95c1..a8ca72bc1 100644 --- a/error/error.go +++ b/error/error.go @@ -93,6 +93,7 @@ func init() { errors[EcodeProxyInternal] = "Proxy Internal Error" errors[EcodeInvalidActiveSize] = "Invalid active size" errors[EcodeInvalidPromoteDelay] = "Proxy promote delay" + errors[EcodePromoteError] = "Proxy promotion error" } diff --git a/server/demote_command.go b/server/demote_command.go index a224b7baf..c5e9add1e 100644 --- a/server/demote_command.go +++ b/server/demote_command.go @@ -43,7 +43,7 @@ func (c *DemoteCommand) Apply(context raft.Context) (interface{}, error) { // Update mode if this change applies to this server. if c.Name == ps.Config.Name { log.Infof("Set mode after demotion: %s", c.Name) - ps.SetMode(ProxyMode) + ps.setMode(ProxyMode) } return nil, nil diff --git a/server/peer_server.go b/server/peer_server.go index 7a10cd25b..3b98c2bb1 100644 --- a/server/peer_server.go +++ b/server/peer_server.go @@ -119,16 +119,16 @@ func (s *PeerServer) Mode() Mode { // SetMode updates the current mode of the server. // Switching to a peer mode will start the Raft server. // Switching to a proxy mode will stop the Raft server. -func (s *PeerServer) SetMode(mode Mode) { +func (s *PeerServer) setMode(mode Mode) { s.mode = mode switch mode { case PeerMode: - if s.raftServer.Running() { + if !s.raftServer.Running() { s.raftServer.Start() } case ProxyMode: - if !s.raftServer.Running() { + if s.raftServer.Running() { s.raftServer.Stop() } } @@ -140,10 +140,9 @@ func (s *PeerServer) ClusterConfig() *ClusterConfig { } // SetClusterConfig updates the current cluster configuration. -// Adjusting the active size will +// Adjusting the active size will cause the PeerServer to demote peers or +// promote proxies to match the new size. func (s *PeerServer) SetClusterConfig(c *ClusterConfig) error { - s.clusterConfig = c - // Validate configuration. if c.ActiveSize < 1 { return etcdErr.NewError(etcdErr.EcodeInvalidActiveSize, "Post", 0) @@ -151,6 +150,8 @@ func (s *PeerServer) SetClusterConfig(c *ClusterConfig) error { return etcdErr.NewError(etcdErr.EcodeInvalidPromoteDelay, "Post", 0) } + s.clusterConfig = c + return nil } @@ -458,9 +459,9 @@ func (s *PeerServer) joinByPeer(server raft.Server, peer string, scheme string) switch mode { case 0: - s.SetMode(PeerMode) + s.setMode(PeerMode) case 1: - s.SetMode(ProxyMode) + s.setMode(ProxyMode) s.proxyClientURL = resp.Header.Get("X-Leader-Client-URL") s.proxyPeerURL = resp.Header.Get("X-Leader-Peer-URL") default: @@ -614,8 +615,8 @@ func (s *PeerServer) monitorTimeoutThreshold(closeChan chan bool) { } } -// monitorActive periodically checks the status of cluster nodes and swaps them -// out for proxies as needed. +// monitorActive has the leader periodically check the status of cluster nodes +// and swaps them out for proxies as needed. func (s *PeerServer) monitorActive(closeChan chan bool) { for { select { @@ -641,6 +642,7 @@ func (s *PeerServer) monitorActive(closeChan chan bool) { // If we have more active nodes than we should then demote. if peerCount > activeSize { peer := peers[rand.Intn(len(peers))] + log.Infof("%s: demoting: %v", s.Config.Name, peer) if _, err := s.raftServer.Do(&DemoteCommand{Name: peer}); err != nil { log.Infof("%s: warning: demotion error: %v", s.Config.Name, err) } @@ -676,7 +678,7 @@ func (s *PeerServer) monitorActive(closeChan chan bool) { } } -// monitorPeerActivity periodically checks for dead nodes and demotes them. +// monitorPeerActivity has the leader periodically for dead nodes and demotes them. func (s *PeerServer) monitorPeerActivity(closeChan chan bool) { for { select { @@ -708,7 +710,7 @@ func (s *PeerServer) monitorPeerActivity(closeChan chan bool) { } } -// Mode represents whether the server is an active peer or if the server is +// Mode represents whether the server is an active peer or if the server is // simply acting as a proxy. type Mode string From ddce52c44eace0c0d83c477280dae313ebca9263 Mon Sep 17 00:00:00 2001 From: Brandon Philips Date: Tue, 4 Mar 2014 18:58:39 -0800 Subject: [PATCH 08/15] feat(scripts/test-cluster): create three proxy nodes This uses some sleep hackery to make sure that window 1 has followers and window 2 has proxy machines. Seems to work! --- scripts/test-cluster | 18 +++++++++++++++--- 1 file changed, 15 insertions(+), 3 deletions(-) diff --git a/scripts/test-cluster b/scripts/test-cluster index 7239a2149..f2c11fcf1 100755 --- a/scripts/test-cluster +++ b/scripts/test-cluster @@ -1,10 +1,11 @@ -#!/bin/bash +#!/bin/bash -x SESSION=etcd-cluster DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" ulimit -n unlimited tmux new-session -d -s $SESSION +mkdir test-cluster peer_args= if [ -n "${DISCOVERY_URL}" ]; then @@ -15,7 +16,7 @@ fi tmux new-window -t $SESSION:1 -n 'peers' tmux split-window -h tmux select-pane -t 0 -tmux send-keys "${DIR}/../bin/etcd -peer-addr 127.0.0.1:7001 -addr 127.0.0.1:4001 -data-dir peer1 -name peer1 ${peer_args}" C-m +tmux send-keys "${DIR}/../bin/etcd -peer-addr 127.0.0.1:7001 -addr 127.0.0.1:4001 -data-dir test-cluster/peer1 -name peer1 ${peer_args}" C-m if [ -z "${peer_args}" ]; then peer_args="-peers 127.0.0.1:7001" @@ -24,7 +25,18 @@ fi for i in 2 3; do tmux select-pane -t 0 tmux split-window -v - tmux send-keys "${DIR}/../bin/etcd -cors='*' -peer-addr 127.0.0.1:700${i} -addr 127.0.0.1:400${i} -data-dir peer${i} -name peer${i} ${peer_args}" C-m + tmux send-keys "sleep 2; ${DIR}/../bin/etcd -cors='*' -peer-addr 127.0.0.1:700${i} -addr 127.0.0.1:400${i} -data-dir test-cluster/peer${i} -name peer${i} ${peer_args}" C-m +done + +tmux new-window -t $SESSION:2 -n 'proxy' +tmux split-window -h +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 + +for i in 4 5 6; do + tmux select-pane -t 0 + tmux split-window -v + tmux send-keys "sleep 5; ${DIR}/../bin/etcd -cors='*' -peer-addr 127.0.0.1:700${i} -addr 127.0.0.1:400${i} -data-dir test-cluster/peer${i} -name peer${i} ${peer_args}" C-m done # Attach to session From 3cd83c61c434949c4b6efc6b6dcbeff3b92af8e3 Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Thu, 6 Mar 2014 13:35:46 -0700 Subject: [PATCH 09/15] Fix PUT /config response and log errors. --- server/peer_server_handlers.go | 2 ++ server/registry.go | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/server/peer_server_handlers.go b/server/peer_server_handlers.go index 9fe7d06b0..e748d6a60 100644 --- a/server/peer_server_handlers.go +++ b/server/peer_server_handlers.go @@ -223,6 +223,8 @@ func (ps *PeerServer) setClusterConfigHttpHandler(w http.ResponseWriter, req *ht log.Debugf("[recv] Update Cluster Config Request") ps.server.Dispatch(c, w, req) + + json.NewEncoder(w).Encode(&ps.clusterConfig) } // Response to the name request diff --git a/server/registry.go b/server/registry.go index 1447b40db..9c5f02570 100644 --- a/server/registry.go +++ b/server/registry.go @@ -273,7 +273,7 @@ func (r *Registry) urls(key, leaderName, selfName string, url func(key, name str } } - log.Infof("URLs: %s / %s (%s)", key, leaderName, selfName, strings.Join(urls, ",")) + log.Infof("URLs: %s: %s / %s (%s)", key, leaderName, selfName, strings.Join(urls, ",")) return urls } From 3fff1a8dcd9762bb53c79dcde9eb7bb8e42d2385 Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Thu, 6 Mar 2014 15:11:31 -0700 Subject: [PATCH 10/15] Add /machines and /machines/:name endpoints. --- server/peer_server.go | 2 ++ server/peer_server_handlers.go | 52 ++++++++++++++++++++++++++++++++++ tests/functional/proxy_test.go | 2 +- 3 files changed, 55 insertions(+), 1 deletion(-) diff --git a/server/peer_server.go b/server/peer_server.go index 3b98c2bb1..949151c3e 100644 --- a/server/peer_server.go +++ b/server/peer_server.go @@ -300,6 +300,8 @@ func (s *PeerServer) HTTPHandler() http.Handler { 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("/log", s.GetLogHttpHandler) router.HandleFunc("/log/append", s.AppendEntriesHttpHandler) diff --git a/server/peer_server_handlers.go b/server/peer_server_handlers.go index e748d6a60..374c78173 100644 --- a/server/peer_server_handlers.go +++ b/server/peer_server_handlers.go @@ -227,6 +227,50 @@ func (ps *PeerServer) setClusterConfigHttpHandler(w http.ResponseWriter, req *ht json.NewEncoder(w).Encode(&ps.clusterConfig) } +// Retrieves a list of peers and proxies. +func (ps *PeerServer) getMachinesHttpHandler(w http.ResponseWriter, req *http.Request) { + machines := make([]*machineMessage, 0) + for _, name := range ps.registry.Peers() { + machines = append(machines, ps.getMachineMessage(name)) + } + for _, name := range ps.registry.Proxies() { + machines = append(machines, ps.getMachineMessage(name)) + } + json.NewEncoder(w).Encode(&machines) +} + +// Retrieve single peer or proxy. +func (ps *PeerServer) getMachineHttpHandler(w http.ResponseWriter, req *http.Request) { + vars := mux.Vars(req) + json.NewEncoder(w).Encode(ps.getMachineMessage(vars["name"])) +} + +func (ps *PeerServer) getMachineMessage(name string) *machineMessage { + if ps.registry.PeerExists(name) { + clientURL, _ := ps.registry.ClientURL(name) + peerURL, _ := ps.registry.PeerURL(name) + return &machineMessage{ + Name: name, + Mode: PeerMode, + ClientURL: clientURL, + PeerURL: peerURL, + } + } + + if ps.registry.ProxyExists(name) { + clientURL, _ := ps.registry.ProxyClientURL(name) + peerURL, _ := ps.registry.ProxyPeerURL(name) + return &machineMessage{ + Name: name, + Mode: ProxyMode, + ClientURL: clientURL, + PeerURL: peerURL, + } + } + + return nil +} + // Response to the name request func (ps *PeerServer) NameHttpHandler(w http.ResponseWriter, req *http.Request) { log.Debugf("[recv] Get %s/name/ ", ps.Config.URL) @@ -272,3 +316,11 @@ func (ps *PeerServer) UpgradeHttpHandler(w http.ResponseWriter, req *http.Reques w.WriteHeader(http.StatusOK) } + +// machineMessage represents information about a peer or proxy in the registry. +type machineMessage struct { + Name string `json:"name"` + Mode Mode `json:"mode"` + ClientURL string `json:"clientURL"` + PeerURL string `json:"peerURL"` +} diff --git a/tests/functional/proxy_test.go b/tests/functional/proxy_test.go index 03de0120d..e3290c688 100644 --- a/tests/functional/proxy_test.go +++ b/tests/functional/proxy_test.go @@ -36,7 +36,7 @@ func TestProxy(t *testing.T) { // Check that all peers and proxies have the value. for i, _ := range etcds { - resp, err := tests.Get(fmt.Sprintf("http://localhost:%d/v2/keys/foo", 4000 + (i+1))) + resp, err := tests.Get(fmt.Sprintf("http://localhost:%d/v2/keys/foo", 4000+(i+1))) if assert.NoError(t, err) { body := tests.ReadBodyJSON(resp) if node, _ := body["node"].(map[string]interface{}); assert.NotNil(t, node) { From c91688315a9abcd8ab84a63ab58185d680a5fabb Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Fri, 7 Mar 2014 07:38:40 -0700 Subject: [PATCH 11/15] Minor fixes to proxies. --- server/join_command.go | 7 ++++--- server/peer_server.go | 36 ++++++++++++++++++++++++++---------- 2 files changed, 30 insertions(+), 13 deletions(-) diff --git a/server/join_command.go b/server/join_command.go index 871faedf8..44ffc23ac 100644 --- a/server/join_command.go +++ b/server/join_command.go @@ -20,6 +20,7 @@ func init() { // 8 bytes | 1 byte // join_index | join_mode // +// This binary protocol is for backward compatibility. type JoinCommand struct { MinVersion int `json:"minVersion"` MaxVersion int `json:"maxVersion"` @@ -57,7 +58,7 @@ func (c *JoinCommand) Apply(context raft.Context) (interface{}, error) { // Check if the join command is from a previous peer, who lost all its previous log. if _, ok := ps.registry.ClientURL(c.Name); ok { - binary.Write(&buf, binary.BigEndian, uint8(0)) // Mark as peer. + binary.Write(&buf, binary.BigEndian, uint8(peerModeFlag)) // Mark as peer. return buf.Bytes(), nil } @@ -65,7 +66,7 @@ func (c *JoinCommand) Apply(context raft.Context) (interface{}, error) { if ps.registry.PeerCount() >= ps.ClusterConfig().ActiveSize { log.Debug("Join as proxy ", c.Name) ps.registry.RegisterProxy(c.Name, c.RaftURL, c.EtcdURL) - binary.Write(&buf, binary.BigEndian, uint8(1)) // Mark as proxy. + binary.Write(&buf, binary.BigEndian, uint8(proxyModeFlag)) // Mark as proxy. return buf.Bytes(), nil } @@ -86,7 +87,7 @@ func (c *JoinCommand) Apply(context raft.Context) (interface{}, error) { ps.followersStats.Followers[c.Name].Latency.Minimum = 1 << 63 } - binary.Write(&buf, binary.BigEndian, uint8(0)) // Mark as peer. + binary.Write(&buf, binary.BigEndian, uint8(peerModeFlag)) // Mark as peer. return buf.Bytes(), err } diff --git a/server/peer_server.go b/server/peer_server.go index 949151c3e..5cff2cdfa 100644 --- a/server/peer_server.go +++ b/server/peer_server.go @@ -25,9 +25,25 @@ import ( "github.com/coreos/etcd/store" ) -const ThresholdMonitorTimeout = 5 * time.Second -const ActiveMonitorTimeout = 1 * time.Second -const PeerActivityMonitorTimeout = 1 * time.Second +const ( + // ThresholdMonitorTimeout is the time between log notifications that the + // Raft heartbeat is too close to the election timeout. + ThresholdMonitorTimeout = 5 * time.Second + + // ActiveMonitorTimeout is the time between checks on the active size of + // the cluster. If the active size is different than the actual size then + // etcd attempts to promote/demote to bring it to the correct number. + ActiveMonitorTimeout = 1 * time.Second + + // PeerActivityMonitorTimeout is the time between checks for dead nodes in + // the cluster. + PeerActivityMonitorTimeout = 1 * time.Second +) + +const ( + peerModeFlag = 0 + proxyModeFlag = 1 +) type PeerServerConfig struct { Name string @@ -268,7 +284,7 @@ func (s *PeerServer) Start(snapshot bool, discoverURL string, peers []string) er go s.monitorSync() go s.monitorTimeoutThreshold(s.closeChan) - go s.monitorActive(s.closeChan) + go s.monitorActiveSize(s.closeChan) go s.monitorPeerActivity(s.closeChan) // open the snapshot @@ -453,16 +469,16 @@ func (s *PeerServer) joinByPeer(server raft.Server, peer string, scheme string) // Determine whether the server joined as a proxy or peer. var mode uint64 if mode, err = binary.ReadUvarint(r); err == io.EOF { - mode = 0 + mode = peerModeFlag } else if err != nil { log.Debugf("Error reading join mode: %v", err) return err } switch mode { - case 0: + case peerModeFlag: s.setMode(PeerMode) - case 1: + case proxyModeFlag: s.setMode(ProxyMode) s.proxyClientURL = resp.Header.Get("X-Leader-Client-URL") s.proxyPeerURL = resp.Header.Get("X-Leader-Peer-URL") @@ -617,9 +633,9 @@ func (s *PeerServer) monitorTimeoutThreshold(closeChan chan bool) { } } -// monitorActive has the leader periodically check the status of cluster nodes -// and swaps them out for proxies as needed. -func (s *PeerServer) monitorActive(closeChan chan bool) { +// monitorActiveSize has the leader periodically check the status of cluster +// nodes and swaps them out for proxies as needed. +func (s *PeerServer) monitorActiveSize(closeChan chan bool) { for { select { case <-time.After(ActiveMonitorTimeout): From c0a59b3a273c3cdffb24abe6b99cb73fb2793a2c Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Mon, 10 Mar 2014 14:44:04 -0600 Subject: [PATCH 12/15] Add minimum active size and promote delay. --- server/cluster_config.go | 6 ++++++ server/peer_server.go | 15 +++++++-------- server/set_cluster_config_command.go | 3 ++- tests/functional/proxy_test.go | 8 ++++---- 4 files changed, 19 insertions(+), 13 deletions(-) diff --git a/server/cluster_config.go b/server/cluster_config.go index b47739c5d..5f0cc0fdf 100644 --- a/server/cluster_config.go +++ b/server/cluster_config.go @@ -8,8 +8,14 @@ const ( // DefaultActiveSize is the default number of active followers allowed. DefaultActiveSize = 9 + // MinActiveSize is the minimum active size allowed. + MinActiveSize = 3 + // DefaultPromoteDelay is the default elapsed time before promotion. DefaultPromoteDelay = int((30 * time.Minute) / time.Second) + + // MinPromoteDelay is the minimum promote delay allowed. + MinPromoteDelay = int((2 * time.Second) / time.Second) ) // ClusterConfig represents cluster-wide configuration settings. diff --git a/server/peer_server.go b/server/peer_server.go index 5cff2cdfa..82e39defa 100644 --- a/server/peer_server.go +++ b/server/peer_server.go @@ -158,17 +158,16 @@ func (s *PeerServer) ClusterConfig() *ClusterConfig { // SetClusterConfig updates the current cluster configuration. // Adjusting the active size will cause the PeerServer to demote peers or // promote proxies to match the new size. -func (s *PeerServer) SetClusterConfig(c *ClusterConfig) error { - // Validate configuration. - if c.ActiveSize < 1 { - return etcdErr.NewError(etcdErr.EcodeInvalidActiveSize, "Post", 0) - } else if c.PromoteDelay < 0 { - return etcdErr.NewError(etcdErr.EcodeInvalidPromoteDelay, "Post", 0) +func (s *PeerServer) SetClusterConfig(c *ClusterConfig) { + // Set minimums. + if c.ActiveSize < MinActiveSize { + c.ActiveSize = MinActiveSize + } + if c.PromoteDelay < MinPromoteDelay { + c.PromoteDelay = MinPromoteDelay } s.clusterConfig = c - - return nil } // Helper function to do discovery and return results in expected format diff --git a/server/set_cluster_config_command.go b/server/set_cluster_config_command.go index e5954a5ec..0db2a3e81 100644 --- a/server/set_cluster_config_command.go +++ b/server/set_cluster_config_command.go @@ -21,5 +21,6 @@ func (c *SetClusterConfigCommand) CommandName() string { // Apply updates the cluster configuration. func (c *SetClusterConfigCommand) Apply(context raft.Context) (interface{}, error) { ps, _ := context.Server().Context().(*PeerServer) - return nil, ps.SetClusterConfig(c.Config) + ps.SetClusterConfig(c.Config) + return nil, nil } diff --git a/tests/functional/proxy_test.go b/tests/functional/proxy_test.go index e3290c688..6c9d52e0b 100644 --- a/tests/functional/proxy_test.go +++ b/tests/functional/proxy_test.go @@ -106,8 +106,8 @@ func TestProxyAutoPromote(t *testing.T) { assert.NoError(t, err) assert.Equal(t, len(result.Node.Nodes), 1) - // Reconfigure with a short promote delay (1 second). - resp, _ := tests.Put("http://localhost:7001/config", "application/json", bytes.NewBufferString(`{"activeSize":9, "promoteDelay":1}`)) + // Reconfigure with a short promote delay (2 second). + resp, _ := tests.Put("http://localhost:7001/config", "application/json", bytes.NewBufferString(`{"activeSize":9, "promoteDelay":2}`)) if !assert.Equal(t, resp.StatusCode, 200) { t.FailNow() } @@ -121,10 +121,10 @@ func TestProxyAutoPromote(t *testing.T) { etcd.Release() // Wait for it to get dropped. - time.Sleep(server.PeerActivityMonitorTimeout + (1 * time.Second)) + time.Sleep(server.PeerActivityMonitorTimeout + (2 * time.Second)) // Wait for the proxy to be promoted. - time.Sleep(server.ActiveMonitorTimeout + (1 * time.Second)) + time.Sleep(server.ActiveMonitorTimeout + (2 * time.Second)) // Verify that we have 9 peers. result, err = c.Get("_etcd/machines", true, true) From e9a1ac15d9002751ee6d8f47c7981cc56070526e Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Mon, 10 Mar 2014 16:05:03 -0700 Subject: [PATCH 13/15] Allow partial cluster config updates. --- server/peer_server_handlers.go | 19 +++++++++++++++++-- 1 file changed, 17 insertions(+), 2 deletions(-) diff --git a/server/peer_server_handlers.go b/server/peer_server_handlers.go index 374c78173..bde6f50fd 100644 --- a/server/peer_server_handlers.go +++ b/server/peer_server_handlers.go @@ -215,12 +215,27 @@ func (ps *PeerServer) getClusterConfigHttpHandler(w http.ResponseWriter, req *ht // Updates the cluster configuration. func (ps *PeerServer) setClusterConfigHttpHandler(w http.ResponseWriter, req *http.Request) { - c := &SetClusterConfigCommand{Config: &ClusterConfig{}} - if err := json.NewDecoder(req.Body).Decode(&c.Config); err != nil { + // Decode map. + m := make(map[string]interface{}) + if err := json.NewDecoder(req.Body).Decode(&m); err != nil { http.Error(w, err.Error(), http.StatusInternalServerError) return } + // Copy config and update fields passed in. + config := &ClusterConfig{ + ActiveSize: ps.clusterConfig.ActiveSize, + PromoteDelay: ps.clusterConfig.PromoteDelay, + } + if activeSize, ok := m["activeSize"].(float64); ok { + config.ActiveSize = int(activeSize) + } + if promoteDelay, ok := m["promoteDelay"].(float64); ok { + config.PromoteDelay = int(promoteDelay) + } + + // Issue command to update. + c := &SetClusterConfigCommand{Config: config} log.Debugf("[recv] Update Cluster Config Request") ps.server.Dispatch(c, w, req) From 7d4fda550dbb58eaf12836b9c8fbe834782e5029 Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Tue, 18 Mar 2014 11:46:48 -0700 Subject: [PATCH 14/15] Machine join/remove v2 API. --- Documentation/design/proxies.md | 74 +++++++++++ scripts/test-cluster | 2 +- server/demote_command.go | 23 +++- server/join_command.go | 126 +++++++++++++------ server/peer_server.go | 74 ++++++----- server/peer_server_handlers.go | 37 +++++- server/registry.go | 8 +- server/remove_command.go | 74 ++++++++++- server/server.go | 4 +- tests/functional/cluster_config_test.go | 4 +- tests/functional/proxy_test.go | 6 +- tests/functional/remove_node_test.go | 2 +- third_party/github.com/coreos/raft/server.go | 1 + 13 files changed, 335 insertions(+), 100 deletions(-) create mode 100644 Documentation/design/proxies.md diff --git a/Documentation/design/proxies.md b/Documentation/design/proxies.md new file mode 100644 index 000000000..46b52bab8 --- /dev/null +++ b/Documentation/design/proxies.md @@ -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 +``` diff --git a/scripts/test-cluster b/scripts/test-cluster index f2c11fcf1..a0cd26935 100755 --- a/scripts/test-cluster +++ b/scripts/test-cluster @@ -31,7 +31,7 @@ done tmux new-window -t $SESSION:2 -n 'proxy' tmux split-window -h 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 tmux select-pane -t 0 diff --git a/server/demote_command.go b/server/demote_command.go index c5e9add1e..d13b4dc11 100644 --- a/server/demote_command.go +++ b/server/demote_command.go @@ -34,16 +34,31 @@ func (c *DemoteCommand) Apply(context raft.Context) (interface{}, error) { clientURL, _ := ps.registry.ClientURL(c.Name) peerURL, _ := ps.registry.PeerURL(c.Name) - // Perform a removal. - (&RemoveCommand{Name: c.Name}).Apply(context) + // Remove node from the shared registry. + 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. ps.registry.RegisterProxy(c.Name, peerURL, clientURL) // Update mode if this change applies to this server. if c.Name == ps.Config.Name { - log.Infof("Set mode after demotion: %s", c.Name) - ps.setMode(ProxyMode) + log.Infof("Demote peer %s: Set mode to proxy with %s", c.Name, ps.server.Leader()) + ps.proxyPeerURL, _ = ps.registry.PeerURL(ps.server.Leader()) + go ps.setMode(ProxyMode) } return nil, nil diff --git a/server/join_command.go b/server/join_command.go index 44ffc23ac..37be0d132 100644 --- a/server/join_command.go +++ b/server/join_command.go @@ -1,27 +1,22 @@ package server import ( - "bytes" "encoding/binary" + "encoding/json" + etcdErr "github.com/coreos/etcd/error" "github.com/coreos/etcd/log" "github.com/coreos/etcd/third_party/github.com/coreos/raft" ) func init() { - raft.RegisterCommand(&JoinCommand{}) + raft.RegisterCommand(&JoinCommandV1{}) + raft.RegisterCommand(&JoinCommandV2{}) } -// The JoinCommand adds a node to the cluster. -// -// The command returns the join_index (Uvarint) and peer flag (peer=0, proxy=1) -// in following binary format: -// -// 8 bytes | 1 byte -// join_index | join_mode -// -// This binary protocol is for backward compatibility. -type JoinCommand struct { +// JoinCommandV1 represents a request to join the cluster. +// The command returns the join_index (Uvarint). +type JoinCommandV1 struct { MinVersion int `json:"minVersion"` MaxVersion int `json:"maxVersion"` Name string `json:"name"` @@ -29,50 +24,30 @@ type JoinCommand struct { 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 -func (c *JoinCommand) CommandName() string { +func (c *JoinCommandV1) CommandName() string { return "etcd:join" } // 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) - var buf bytes.Buffer b := make([]byte, 8) - n := binary.PutUvarint(b, context.CommitIndex()) - buf.Write(b[:n]) + binary.PutUvarint(b, 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 { - binary.Write(&buf, binary.BigEndian, uint8(peerModeFlag)) // Mark as peer. - return buf.Bytes(), nil + return b, nil } // 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.RaftURL, c.EtcdURL) - 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) + log.Debug("Reject join request from ", c.Name) + return []byte{0}, etcdErr.NewError(etcdErr.EcodeNoMorePeer, "", context.CommitIndex()) } // 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 } - binary.Write(&buf, binary.BigEndian, uint8(peerModeFlag)) // Mark as peer. - return buf.Bytes(), err + return b, err } -func (c *JoinCommand) NodeName() string { +func (c *JoinCommandV1) NodeName() string { 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"` +} diff --git a/server/peer_server.go b/server/peer_server.go index 82e39defa..449ca52fc 100644 --- a/server/peer_server.go +++ b/server/peer_server.go @@ -1,12 +1,9 @@ package server import ( - "bufio" "bytes" - "encoding/binary" "encoding/json" "fmt" - "io" "io/ioutil" "math/rand" "net/http" @@ -313,10 +310,6 @@ func (s *PeerServer) HTTPHandler() http.Handler { router.HandleFunc("/join", s.JoinHttpHandler) router.HandleFunc("/promote", s.PromoteHttpHandler).Methods("POST") 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("/log", s.GetLogHttpHandler) router.HandleFunc("/log/append", s.AppendEntriesHttpHandler) @@ -324,6 +317,13 @@ func (s *PeerServer) HTTPHandler() http.Handler { router.HandleFunc("/snapshotRecovery", s.SnapshotRecoveryHttpHandler) 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 } @@ -340,7 +340,14 @@ func (s *PeerServer) SetServer(server *Server) { func (s *PeerServer) startAsLeader() { // leader need to join self as a peer 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 { break } @@ -429,8 +436,6 @@ func (s *PeerServer) joinCluster(cluster []string) bool { // Send join requests to peer. func (s *PeerServer) joinByPeer(server raft.Server, peer string, scheme string) error { - var b bytes.Buffer - // t must be ok 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()) } - json.NewEncoder(&b).Encode(NewJoinCommand(store.MinVersion(), store.MaxVersion(), server.Name(), s.Config.URL, s.server.URL())) - - joinURL := url.URL{Host: peer, Scheme: scheme, Path: "/join"} + var b bytes.Buffer + c := &JoinCommandV2{ + 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()) - resp, req, err := t.Post(joinURL.String(), &b) - + req, _ := http.NewRequest("PUT", joinURL.String(), &b) + resp, err := t.client.Do(req) for { if err != nil { 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) if resp.StatusCode == http.StatusOK { - r := bufio.NewReader(resp.Body) - s.joinIndex, _ = binary.ReadUvarint(r) - - // 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) + var msg joinMessageV2 + if err := json.NewDecoder(resp.Body).Decode(&msg); err != nil { + log.Debugf("Error reading join response: %v", err) return err } + s.joinIndex = msg.CommitIndex + s.setMode(msg.Mode) - switch mode { - case peerModeFlag: - s.setMode(PeerMode) - case proxyModeFlag: - s.setMode(ProxyMode) + if msg.Mode == ProxyMode { s.proxyClientURL = resp.Header.Get("X-Leader-Client-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 @@ -491,7 +492,14 @@ func (s *PeerServer) joinByPeer(server raft.Server, peer string, scheme string) if resp.StatusCode == http.StatusTemporaryRedirect { address := resp.Header.Get("Location") 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) } else if resp.StatusCode == http.StatusBadRequest { diff --git a/server/peer_server_handlers.go b/server/peer_server_handlers.go index bde6f50fd..7932a1a83 100644 --- a/server/peer_server_handlers.go +++ b/server/peer_server_handlers.go @@ -150,16 +150,14 @@ func (ps *PeerServer) EtcdURLHttpHandler(w http.ResponseWriter, req *http.Reques // Response to the join request func (ps *PeerServer) JoinHttpHandler(w http.ResponseWriter, req *http.Request) { - command := &JoinCommand{} - - err := uhttp.DecodeJsonRequest(req, command) - if err != nil { + command := &JoinCommandV1{} + if err := uhttp.DecodeJsonRequest(req, command); err != nil { w.WriteHeader(http.StatusInternalServerError) return } 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. if err != nil { @@ -199,7 +197,7 @@ func (ps *PeerServer) RemoveHttpHandler(w http.ResponseWriter, req *http.Request } vars := mux.Vars(req) - command := &RemoveCommand{ + command := &RemoveCommandV1{ Name: vars["name"], } @@ -286,6 +284,33 @@ func (ps *PeerServer) getMachineMessage(name string) *machineMessage { 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 func (ps *PeerServer) NameHttpHandler(w http.ResponseWriter, req *http.Request) { log.Debugf("[recv] Get %s/name/ ", ps.Config.URL) diff --git a/server/registry.go b/server/registry.go index 9c5f02570..54fd884f7 100644 --- a/server/registry.go +++ b/server/registry.go @@ -65,13 +65,15 @@ func (r *Registry) Proxies() []string { // RegisterPeer adds a peer to the registry. func (r *Registry) RegisterPeer(name string, peerURL string, machURL string) error { - // TODO(benbjohnson): Disallow peers that are already proxies. - return r.register(RegistryPeerKey, name, peerURL, machURL) + if err := r.register(RegistryPeerKey, name, peerURL, machURL); err != nil { + return err + } + r.peers[name] = r.load(RegistryPeerKey, name) + return nil } // RegisterProxy adds a proxy to the registry. 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 { return err } diff --git a/server/remove_command.go b/server/remove_command.go index 3019f9d9d..6f4ecea5a 100644 --- a/server/remove_command.go +++ b/server/remove_command.go @@ -2,6 +2,7 @@ package server import ( "encoding/binary" + "encoding/json" "os" "github.com/coreos/etcd/log" @@ -9,21 +10,22 @@ import ( ) func init() { - raft.RegisterCommand(&RemoveCommand{}) + raft.RegisterCommand(&RemoveCommandV1{}) + raft.RegisterCommand(&RemoveCommandV2{}) } -// The RemoveCommand removes a server from the cluster. -type RemoveCommand struct { +// The RemoveCommandV1 removes a server from the cluster. +type RemoveCommandV1 struct { Name string `json:"name"` } // The name of the remove command in the log -func (c *RemoveCommand) CommandName() string { +func (c *RemoveCommandV1) CommandName() string { return "etcd:remove" } // 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) // 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 } + +// 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"` +} diff --git a/server/server.go b/server/server.go index 51c0a17a0..83c133bad 100644 --- a/server/server.go +++ b/server/server.go @@ -262,7 +262,9 @@ func (s *Server) Dispatch(c raft.Command, w http.ResponseWriter, req *http.Reque var url string switch c.(type) { - case *JoinCommand, *RemoveCommand: + case *JoinCommandV1, *RemoveCommandV1: + url, _ = ps.registry.PeerURL(leader) + case *JoinCommandV2, *RemoveCommandV2: url, _ = ps.registry.PeerURL(leader) default: url, _ = ps.registry.ClientURL(leader) diff --git a/tests/functional/cluster_config_test.go b/tests/functional/cluster_config_test.go index 8f8b667f0..c75ce1de4 100644 --- a/tests/functional/cluster_config_test.go +++ b/tests/functional/cluster_config_test.go @@ -16,12 +16,12 @@ func TestClusterConfig(t *testing.T) { assert.NoError(t, err) 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) 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) assert.Equal(t, resp.StatusCode, 200) assert.Equal(t, body["activeSize"], 3) diff --git a/tests/functional/proxy_test.go b/tests/functional/proxy_test.go index 6c9d52e0b..665862aad 100644 --- a/tests/functional/proxy_test.go +++ b/tests/functional/proxy_test.go @@ -51,7 +51,7 @@ func TestProxy(t *testing.T) { assert.Equal(t, len(result.Node.Nodes), 1) // 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) { t.FailNow() } @@ -64,7 +64,7 @@ func TestProxy(t *testing.T) { assert.Equal(t, len(result.Node.Nodes), 0) // 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) { t.FailNow() } @@ -107,7 +107,7 @@ func TestProxyAutoPromote(t *testing.T) { assert.Equal(t, len(result.Node.Nodes), 1) // 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) { t.FailNow() } diff --git a/tests/functional/remove_node_test.go b/tests/functional/remove_node_test.go index 273577007..5e22ba08e 100644 --- a/tests/functional/remove_node_test.go +++ b/tests/functional/remove_node_test.go @@ -25,7 +25,7 @@ func TestRemoveNode(t *testing.T) { 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{} for i := 0; i < 2; i++ { diff --git a/third_party/github.com/coreos/raft/server.go b/third_party/github.com/coreos/raft/server.go index 5fc3205d9..8cef1341a 100644 --- a/third_party/github.com/coreos/raft/server.go +++ b/third_party/github.com/coreos/raft/server.go @@ -477,6 +477,7 @@ func (s *server) Stop() { // make sure the server has stopped before we close the log <-s.stopped s.log.close() + s.state = Stopped } // Checks if the server is currently running. From 174b9ff343ad482250c78285b5abf80770c4d0d3 Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Mon, 24 Mar 2014 15:09:47 -0700 Subject: [PATCH 15/15] bump(github.com/goraft/raft): 6bf34b9 Move from coreos/raft to goraft/raft and update to latest. --- etcd.go | 14 +- server/demote_command.go | 2 +- server/join_command.go | 2 +- server/peer_server.go | 2 +- server/peer_server_handlers.go | 2 +- server/raft_server_stats.go | 2 +- server/remove_command.go | 2 +- server/server.go | 2 +- server/set_cluster_config_command.go | 2 +- server/transporter.go | 2 +- server/v1/set_key_handler.go | 2 +- server/v1/v1.go | 2 +- server/v2/get_handler.go | 2 +- server/v2/put_handler.go | 2 +- server/v2/v2.go | 2 +- store/command_factory.go | 2 +- store/v2/command_factory.go | 48 +- store/v2/compare_and_delete_command.go | 2 +- store/v2/compare_and_swap_command.go | 12 +- store/v2/create_command.go | 12 +- store/v2/delete_command.go | 8 +- store/v2/set_command.go | 10 +- store/v2/sync_command.go | 2 +- store/v2/update_command.go | 8 +- tests/functional/v1_migration_test.go | 2 +- tests/mock/server_v2.go | 4 +- tests/server_utils.go | 40 +- .../p/gogoprotobuf/proto/lib.go | 4 +- .../proto/testdata/test.pb.go.golden | 2 +- .../github.com/coreos/raft/.travis.yml | 10 - .../coreos/raft/append_entries_request.go | 82 --- .../coreos/raft/append_entries_response.go | 73 --- .../raft/append_entries_response_test.go | 34 -- .../github.com/coreos/raft/join_command.go | 28 - .../github.com/coreos/raft/leave_command.go | 26 - .../github.com/coreos/raft/nop_command.go | 26 - .../coreos/raft/request_vote_request.go | 69 --- .../coreos/raft/request_vote_response.go | 62 -- .../github.com/coreos/raft/snapshot.go | 61 -- .../coreos/raft/snapshot_recovery_request.go | 90 --- .../coreos/raft/snapshot_recovery_response.go | 63 -- .../coreos/raft/snapshot_request.go | 65 --- .../coreos/raft/snapshot_response.go | 56 -- third_party/github.com/coreos/raft/sort.go | 23 - third_party/github.com/coreos/raft/time.go | 17 - third_party/github.com/coreos/raft/util.go | 31 - .../{coreos => goraft}/raft/.gitignore | 0 .../{coreos => goraft}/raft/LICENSE | 0 .../{coreos => goraft}/raft/Makefile | 0 .../{coreos => goraft}/raft/README.md | 1 + .../github.com/goraft/raft/append_entries.go | 146 +++++ .../raft/append_entries_test.go} | 28 + .../{coreos => goraft}/raft/command.go | 0 .../github.com/goraft/raft/commands.go | 78 +++ .../{coreos => goraft}/raft/config.go | 0 .../{coreos => goraft}/raft/context.go | 0 .../{coreos => goraft}/raft/debug.go | 0 .../{coreos => goraft}/raft/event.go | 0 .../raft/event_dispatcher.go | 0 .../raft/event_dispatcher_test.go | 0 .../raft/http_transporter.go | 0 .../raft/http_transporter_test.go | 0 .../github.com/{coreos => goraft}/raft/log.go | 2 +- .../{coreos => goraft}/raft/log_entry.go | 26 +- .../{coreos => goraft}/raft/log_test.go | 0 .../{coreos => goraft}/raft/peer.go | 14 +- .../protobuf/append_entries_request.pb.go | 18 +- .../protobuf/append_entries_request.proto | 0 .../protobuf/append_entries_responses.pb.go | 14 +- .../protobuf/append_entries_responses.proto | 0 .../raft/protobuf/log_entry.pb.go | 14 +- .../raft/protobuf/log_entry.proto | 0 .../raft/protobuf/request_vote_request.pb.go | 14 +- .../raft/protobuf/request_vote_request.proto | 0 .../protobuf/request_vote_responses.pb.go | 10 +- .../protobuf/request_vote_responses.proto | 0 .../protobuf/snapshot_recovery_request.pb.go | 26 +- .../protobuf/snapshot_recovery_request.proto | 0 .../protobuf/snapshot_recovery_response.pb.go | 12 +- .../protobuf/snapshot_recovery_response.proto | 0 .../raft/protobuf/snapshot_request.pb.go | 12 +- .../raft/protobuf/snapshot_request.proto | 0 .../raft/protobuf/snapshot_response.pb.go | 8 +- .../raft/protobuf/snapshot_response.proto | 0 .../github.com/goraft/raft/request_vote.go | 122 ++++ .../{coreos => goraft}/raft/server.go | 541 ++++++++++-------- .../{coreos => goraft}/raft/server_test.go | 41 ++ .../github.com/goraft/raft/snapshot.go | 304 ++++++++++ .../{coreos => goraft}/raft/snapshot_test.go | 4 +- .../{coreos => goraft}/raft/statemachine.go | 0 .../raft/statemachine_test.go | 0 .../{coreos => goraft}/raft/test.go | 0 .../{coreos => goraft}/raft/transporter.go | 0 third_party/github.com/goraft/raft/util.go | 61 ++ .../{coreos => goraft}/raft/z_test.go | 0 95 files changed, 1269 insertions(+), 1241 deletions(-) delete mode 100644 third_party/github.com/coreos/raft/.travis.yml delete mode 100644 third_party/github.com/coreos/raft/append_entries_request.go delete mode 100644 third_party/github.com/coreos/raft/append_entries_response.go delete mode 100644 third_party/github.com/coreos/raft/append_entries_response_test.go delete mode 100644 third_party/github.com/coreos/raft/join_command.go delete mode 100644 third_party/github.com/coreos/raft/leave_command.go delete mode 100644 third_party/github.com/coreos/raft/nop_command.go delete mode 100644 third_party/github.com/coreos/raft/request_vote_request.go delete mode 100644 third_party/github.com/coreos/raft/request_vote_response.go delete mode 100644 third_party/github.com/coreos/raft/snapshot.go delete mode 100644 third_party/github.com/coreos/raft/snapshot_recovery_request.go delete mode 100644 third_party/github.com/coreos/raft/snapshot_recovery_response.go delete mode 100644 third_party/github.com/coreos/raft/snapshot_request.go delete mode 100644 third_party/github.com/coreos/raft/snapshot_response.go delete mode 100644 third_party/github.com/coreos/raft/sort.go delete mode 100644 third_party/github.com/coreos/raft/time.go delete mode 100644 third_party/github.com/coreos/raft/util.go rename third_party/github.com/{coreos => goraft}/raft/.gitignore (100%) rename third_party/github.com/{coreos => goraft}/raft/LICENSE (100%) rename third_party/github.com/{coreos => goraft}/raft/Makefile (100%) rename third_party/github.com/{coreos => goraft}/raft/README.md (98%) create mode 100644 third_party/github.com/goraft/raft/append_entries.go rename third_party/github.com/{coreos/raft/append_entries_request_test.go => goraft/raft/append_entries_test.go} (58%) rename third_party/github.com/{coreos => goraft}/raft/command.go (100%) create mode 100644 third_party/github.com/goraft/raft/commands.go rename third_party/github.com/{coreos => goraft}/raft/config.go (100%) rename third_party/github.com/{coreos => goraft}/raft/context.go (100%) rename third_party/github.com/{coreos => goraft}/raft/debug.go (100%) rename third_party/github.com/{coreos => goraft}/raft/event.go (100%) rename third_party/github.com/{coreos => goraft}/raft/event_dispatcher.go (100%) rename third_party/github.com/{coreos => goraft}/raft/event_dispatcher_test.go (100%) rename third_party/github.com/{coreos => goraft}/raft/http_transporter.go (100%) rename third_party/github.com/{coreos => goraft}/raft/http_transporter_test.go (100%) rename third_party/github.com/{coreos => goraft}/raft/log.go (99%) rename third_party/github.com/{coreos => goraft}/raft/log_entry.go (82%) rename third_party/github.com/{coreos => goraft}/raft/log_test.go (100%) rename third_party/github.com/{coreos => goraft}/raft/peer.go (98%) rename third_party/github.com/{coreos => goraft}/raft/protobuf/append_entries_request.pb.go (96%) rename third_party/github.com/{coreos => goraft}/raft/protobuf/append_entries_request.proto (100%) rename third_party/github.com/{coreos => goraft}/raft/protobuf/append_entries_responses.pb.go (96%) rename third_party/github.com/{coreos => goraft}/raft/protobuf/append_entries_responses.proto (100%) rename third_party/github.com/{coreos => goraft}/raft/protobuf/log_entry.pb.go (96%) rename third_party/github.com/{coreos => goraft}/raft/protobuf/log_entry.proto (100%) rename third_party/github.com/{coreos => goraft}/raft/protobuf/request_vote_request.pb.go (96%) rename third_party/github.com/{coreos => goraft}/raft/protobuf/request_vote_request.proto (100%) rename third_party/github.com/{coreos => goraft}/raft/protobuf/request_vote_responses.pb.go (97%) rename third_party/github.com/{coreos => goraft}/raft/protobuf/request_vote_responses.proto (100%) rename third_party/github.com/{coreos => goraft}/raft/protobuf/snapshot_recovery_request.pb.go (95%) rename third_party/github.com/{coreos => goraft}/raft/protobuf/snapshot_recovery_request.proto (100%) rename third_party/github.com/{coreos => goraft}/raft/protobuf/snapshot_recovery_response.pb.go (97%) rename third_party/github.com/{coreos => goraft}/raft/protobuf/snapshot_recovery_response.proto (100%) rename third_party/github.com/{coreos => goraft}/raft/protobuf/snapshot_request.pb.go (97%) rename third_party/github.com/{coreos => goraft}/raft/protobuf/snapshot_request.proto (100%) rename third_party/github.com/{coreos => goraft}/raft/protobuf/snapshot_response.pb.go (97%) rename third_party/github.com/{coreos => goraft}/raft/protobuf/snapshot_response.proto (100%) create mode 100644 third_party/github.com/goraft/raft/request_vote.go rename third_party/github.com/{coreos => goraft}/raft/server.go (76%) rename third_party/github.com/{coreos => goraft}/raft/server_test.go (94%) create mode 100644 third_party/github.com/goraft/raft/snapshot.go rename third_party/github.com/{coreos => goraft}/raft/snapshot_test.go (93%) rename third_party/github.com/{coreos => goraft}/raft/statemachine.go (100%) rename third_party/github.com/{coreos => goraft}/raft/statemachine_test.go (100%) rename third_party/github.com/{coreos => goraft}/raft/test.go (100%) rename third_party/github.com/{coreos => goraft}/raft/transporter.go (100%) create mode 100644 third_party/github.com/goraft/raft/util.go rename third_party/github.com/{coreos => goraft}/raft/z_test.go (100%) diff --git a/etcd.go b/etcd.go index c06e14313..24ad37894 100644 --- a/etcd.go +++ b/etcd.go @@ -25,7 +25,7 @@ import ( "runtime" "time" - "github.com/coreos/etcd/third_party/github.com/coreos/raft" + "github.com/coreos/etcd/third_party/github.com/goraft/raft" "github.com/coreos/etcd/config" ehttp "github.com/coreos/etcd/http" @@ -116,12 +116,12 @@ func main() { // Create peer server psConfig := server.PeerServerConfig{ - Name: config.Name, - Scheme: config.PeerTLSInfo().Scheme(), - URL: config.Peer.Addr, - SnapshotCount: config.SnapshotCount, - RetryTimes: config.MaxRetryAttempts, - RetryInterval: config.RetryInterval, + Name: config.Name, + Scheme: config.PeerTLSInfo().Scheme(), + URL: config.Peer.Addr, + SnapshotCount: config.SnapshotCount, + RetryTimes: config.MaxRetryAttempts, + RetryInterval: config.RetryInterval, } ps := server.NewPeerServer(psConfig, registry, store, &mb, followersStats, serverStats) diff --git a/server/demote_command.go b/server/demote_command.go index d13b4dc11..f9ef87f91 100644 --- a/server/demote_command.go +++ b/server/demote_command.go @@ -4,7 +4,7 @@ import ( "fmt" "github.com/coreos/etcd/log" - "github.com/coreos/etcd/third_party/github.com/coreos/raft" + "github.com/coreos/etcd/third_party/github.com/goraft/raft" ) func init() { diff --git a/server/join_command.go b/server/join_command.go index 37be0d132..c2664402c 100644 --- a/server/join_command.go +++ b/server/join_command.go @@ -6,7 +6,7 @@ import ( etcdErr "github.com/coreos/etcd/error" "github.com/coreos/etcd/log" - "github.com/coreos/etcd/third_party/github.com/coreos/raft" + "github.com/coreos/etcd/third_party/github.com/goraft/raft" ) func init() { diff --git a/server/peer_server.go b/server/peer_server.go index 449ca52fc..876735ff9 100644 --- a/server/peer_server.go +++ b/server/peer_server.go @@ -12,7 +12,7 @@ import ( "strconv" "time" - "github.com/coreos/etcd/third_party/github.com/coreos/raft" + "github.com/coreos/etcd/third_party/github.com/goraft/raft" "github.com/coreos/etcd/third_party/github.com/gorilla/mux" "github.com/coreos/etcd/discovery" diff --git a/server/peer_server_handlers.go b/server/peer_server_handlers.go index 7932a1a83..0fe1e8b21 100644 --- a/server/peer_server_handlers.go +++ b/server/peer_server_handlers.go @@ -7,7 +7,7 @@ import ( "strconv" "time" - "github.com/coreos/etcd/third_party/github.com/coreos/raft" + "github.com/coreos/etcd/third_party/github.com/goraft/raft" "github.com/coreos/etcd/third_party/github.com/gorilla/mux" etcdErr "github.com/coreos/etcd/error" diff --git a/server/raft_server_stats.go b/server/raft_server_stats.go index a3c2fd108..5db3886e2 100644 --- a/server/raft_server_stats.go +++ b/server/raft_server_stats.go @@ -3,7 +3,7 @@ package server import ( "time" - "github.com/coreos/etcd/third_party/github.com/coreos/raft" + "github.com/coreos/etcd/third_party/github.com/goraft/raft" ) type raftServerStats struct { diff --git a/server/remove_command.go b/server/remove_command.go index 6f4ecea5a..83a73e520 100644 --- a/server/remove_command.go +++ b/server/remove_command.go @@ -6,7 +6,7 @@ import ( "os" "github.com/coreos/etcd/log" - "github.com/coreos/etcd/third_party/github.com/coreos/raft" + "github.com/coreos/etcd/third_party/github.com/goraft/raft" ) func init() { diff --git a/server/server.go b/server/server.go index 83c133bad..29fcb21ef 100644 --- a/server/server.go +++ b/server/server.go @@ -8,7 +8,7 @@ import ( "strings" "time" - "github.com/coreos/etcd/third_party/github.com/coreos/raft" + "github.com/coreos/etcd/third_party/github.com/goraft/raft" "github.com/coreos/etcd/third_party/github.com/gorilla/mux" etcdErr "github.com/coreos/etcd/error" diff --git a/server/set_cluster_config_command.go b/server/set_cluster_config_command.go index 0db2a3e81..bc9b14277 100644 --- a/server/set_cluster_config_command.go +++ b/server/set_cluster_config_command.go @@ -1,7 +1,7 @@ package server import ( - "github.com/coreos/etcd/third_party/github.com/coreos/raft" + "github.com/coreos/etcd/third_party/github.com/goraft/raft" ) func init() { diff --git a/server/transporter.go b/server/transporter.go index 06954d4f7..f38c0d0d8 100644 --- a/server/transporter.go +++ b/server/transporter.go @@ -10,7 +10,7 @@ import ( "time" "github.com/coreos/etcd/log" - "github.com/coreos/etcd/third_party/github.com/coreos/raft" + "github.com/coreos/etcd/third_party/github.com/goraft/raft" ) // Transporter layer for communication between raft nodes diff --git a/server/v1/set_key_handler.go b/server/v1/set_key_handler.go index 7fea19d6f..fa27db2a1 100644 --- a/server/v1/set_key_handler.go +++ b/server/v1/set_key_handler.go @@ -5,7 +5,7 @@ import ( etcdErr "github.com/coreos/etcd/error" "github.com/coreos/etcd/store" - "github.com/coreos/etcd/third_party/github.com/coreos/raft" + "github.com/coreos/etcd/third_party/github.com/goraft/raft" "github.com/coreos/etcd/third_party/github.com/gorilla/mux" ) diff --git a/server/v1/v1.go b/server/v1/v1.go index 769af5d73..6fed2c6e6 100644 --- a/server/v1/v1.go +++ b/server/v1/v1.go @@ -2,7 +2,7 @@ package v1 import ( "github.com/coreos/etcd/store" - "github.com/coreos/etcd/third_party/github.com/coreos/raft" + "github.com/coreos/etcd/third_party/github.com/goraft/raft" "net/http" ) diff --git a/server/v2/get_handler.go b/server/v2/get_handler.go index 019e18cdc..93147729d 100644 --- a/server/v2/get_handler.go +++ b/server/v2/get_handler.go @@ -9,7 +9,7 @@ import ( etcdErr "github.com/coreos/etcd/error" "github.com/coreos/etcd/log" - "github.com/coreos/etcd/third_party/github.com/coreos/raft" + "github.com/coreos/etcd/third_party/github.com/goraft/raft" "github.com/coreos/etcd/third_party/github.com/gorilla/mux" ) diff --git a/server/v2/put_handler.go b/server/v2/put_handler.go index 0abb5dd50..aa912a295 100644 --- a/server/v2/put_handler.go +++ b/server/v2/put_handler.go @@ -7,7 +7,7 @@ import ( etcdErr "github.com/coreos/etcd/error" "github.com/coreos/etcd/store" - "github.com/coreos/etcd/third_party/github.com/coreos/raft" + "github.com/coreos/etcd/third_party/github.com/goraft/raft" "github.com/coreos/etcd/third_party/github.com/gorilla/mux" ) diff --git a/server/v2/v2.go b/server/v2/v2.go index 876f04de0..0a441741e 100644 --- a/server/v2/v2.go +++ b/server/v2/v2.go @@ -2,7 +2,7 @@ package v2 import ( "github.com/coreos/etcd/store" - "github.com/coreos/etcd/third_party/github.com/coreos/raft" + "github.com/coreos/etcd/third_party/github.com/goraft/raft" "net/http" ) diff --git a/store/command_factory.go b/store/command_factory.go index 74ff346e9..4ef77d2f5 100644 --- a/store/command_factory.go +++ b/store/command_factory.go @@ -4,7 +4,7 @@ import ( "fmt" "time" - "github.com/coreos/etcd/third_party/github.com/coreos/raft" + "github.com/coreos/etcd/third_party/github.com/goraft/raft" ) // A lookup of factories by version. diff --git a/store/v2/command_factory.go b/store/v2/command_factory.go index 6979bab3e..2ba00029b 100644 --- a/store/v2/command_factory.go +++ b/store/v2/command_factory.go @@ -4,7 +4,7 @@ import ( "time" "github.com/coreos/etcd/store" - "github.com/coreos/etcd/third_party/github.com/coreos/raft" + "github.com/coreos/etcd/third_party/github.com/goraft/raft" ) func init() { @@ -28,59 +28,59 @@ func (f *CommandFactory) CreateUpgradeCommand() raft.Command { // CreateSetCommand creates a version 2 command to set a key to a given value in the store. func (f *CommandFactory) CreateSetCommand(key string, dir bool, value string, expireTime time.Time) raft.Command { return &SetCommand{ - Key: key, - Value: value, - ExpireTime: expireTime, - Dir: dir, + Key: key, + Value: value, + ExpireTime: expireTime, + Dir: dir, } } // CreateCreateCommand creates a version 2 command to create a new key in the store. func (f *CommandFactory) CreateCreateCommand(key string, dir bool, value string, expireTime time.Time, unique bool) raft.Command { return &CreateCommand{ - Key: key, - Value: value, - ExpireTime: expireTime, - Unique: unique, - Dir: dir, + Key: key, + Value: value, + ExpireTime: expireTime, + Unique: unique, + Dir: dir, } } // CreateUpdateCommand creates a version 2 command to update a key to a given value in the store. func (f *CommandFactory) CreateUpdateCommand(key string, value string, expireTime time.Time) raft.Command { return &UpdateCommand{ - Key: key, - Value: value, - ExpireTime: expireTime, + Key: key, + Value: value, + ExpireTime: expireTime, } } // CreateDeleteCommand creates a version 2 command to delete a key from the store. func (f *CommandFactory) CreateDeleteCommand(key string, dir, recursive bool) raft.Command { return &DeleteCommand{ - Key: key, - Recursive: recursive, - Dir: dir, + Key: key, + Recursive: recursive, + Dir: dir, } } // CreateCompareAndSwapCommand creates a version 2 command to conditionally set a key in the store. func (f *CommandFactory) CreateCompareAndSwapCommand(key string, value string, prevValue string, prevIndex uint64, expireTime time.Time) raft.Command { return &CompareAndSwapCommand{ - Key: key, - Value: value, - PrevValue: prevValue, - PrevIndex: prevIndex, - ExpireTime: expireTime, + Key: key, + Value: value, + PrevValue: prevValue, + PrevIndex: prevIndex, + ExpireTime: expireTime, } } // CreateCompareAndDeleteCommand creates a version 2 command to conditionally delete a key from the store. func (f *CommandFactory) CreateCompareAndDeleteCommand(key string, prevValue string, prevIndex uint64) raft.Command { return &CompareAndDeleteCommand{ - Key: key, - PrevValue: prevValue, - PrevIndex: prevIndex, + Key: key, + PrevValue: prevValue, + PrevIndex: prevIndex, } } diff --git a/store/v2/compare_and_delete_command.go b/store/v2/compare_and_delete_command.go index 4dac1c301..a8e13c7c1 100644 --- a/store/v2/compare_and_delete_command.go +++ b/store/v2/compare_and_delete_command.go @@ -3,7 +3,7 @@ package v2 import ( "github.com/coreos/etcd/log" "github.com/coreos/etcd/store" - "github.com/coreos/etcd/third_party/github.com/coreos/raft" + "github.com/coreos/etcd/third_party/github.com/goraft/raft" ) func init() { diff --git a/store/v2/compare_and_swap_command.go b/store/v2/compare_and_swap_command.go index ec77495da..4af2f65f8 100644 --- a/store/v2/compare_and_swap_command.go +++ b/store/v2/compare_and_swap_command.go @@ -5,7 +5,7 @@ import ( "github.com/coreos/etcd/log" "github.com/coreos/etcd/store" - "github.com/coreos/etcd/third_party/github.com/coreos/raft" + "github.com/coreos/etcd/third_party/github.com/goraft/raft" ) func init() { @@ -14,11 +14,11 @@ func init() { // The CompareAndSwap performs a conditional update on a key in the store. type CompareAndSwapCommand struct { - Key string `json:"key"` - Value string `json:"value"` - ExpireTime time.Time `json:"expireTime"` - PrevValue string `json:"prevValue"` - PrevIndex uint64 `json:"prevIndex"` + Key string `json:"key"` + Value string `json:"value"` + ExpireTime time.Time `json:"expireTime"` + PrevValue string `json:"prevValue"` + PrevIndex uint64 `json:"prevIndex"` } // The name of the testAndSet command in the log diff --git a/store/v2/create_command.go b/store/v2/create_command.go index 3302d9c0c..c8cb7d78b 100644 --- a/store/v2/create_command.go +++ b/store/v2/create_command.go @@ -5,7 +5,7 @@ import ( "github.com/coreos/etcd/log" "github.com/coreos/etcd/store" - "github.com/coreos/etcd/third_party/github.com/coreos/raft" + "github.com/coreos/etcd/third_party/github.com/goraft/raft" ) func init() { @@ -14,11 +14,11 @@ func init() { // Create command type CreateCommand struct { - Key string `json:"key"` - Value string `json:"value"` - ExpireTime time.Time `json:"expireTime"` - Unique bool `json:"unique"` - Dir bool `json:"dir"` + Key string `json:"key"` + Value string `json:"value"` + ExpireTime time.Time `json:"expireTime"` + Unique bool `json:"unique"` + Dir bool `json:"dir"` } // The name of the create command in the log diff --git a/store/v2/delete_command.go b/store/v2/delete_command.go index cb31fe9fb..effdfe06d 100644 --- a/store/v2/delete_command.go +++ b/store/v2/delete_command.go @@ -3,7 +3,7 @@ package v2 import ( "github.com/coreos/etcd/log" "github.com/coreos/etcd/store" - "github.com/coreos/etcd/third_party/github.com/coreos/raft" + "github.com/coreos/etcd/third_party/github.com/goraft/raft" ) func init() { @@ -12,9 +12,9 @@ func init() { // The DeleteCommand removes a key from the Store. type DeleteCommand struct { - Key string `json:"key"` - Recursive bool `json:"recursive"` - Dir bool `json:"dir"` + Key string `json:"key"` + Recursive bool `json:"recursive"` + Dir bool `json:"dir"` } // The name of the delete command in the log diff --git a/store/v2/set_command.go b/store/v2/set_command.go index 8a5394fb1..8651f50ca 100644 --- a/store/v2/set_command.go +++ b/store/v2/set_command.go @@ -5,7 +5,7 @@ import ( "github.com/coreos/etcd/log" "github.com/coreos/etcd/store" - "github.com/coreos/etcd/third_party/github.com/coreos/raft" + "github.com/coreos/etcd/third_party/github.com/goraft/raft" ) func init() { @@ -14,10 +14,10 @@ func init() { // Create command type SetCommand struct { - Key string `json:"key"` - Value string `json:"value"` - ExpireTime time.Time `json:"expireTime"` - Dir bool `json:"dir"` + Key string `json:"key"` + Value string `json:"value"` + ExpireTime time.Time `json:"expireTime"` + Dir bool `json:"dir"` } // The name of the create command in the log diff --git a/store/v2/sync_command.go b/store/v2/sync_command.go index f9a23e48a..aab0ab8b0 100644 --- a/store/v2/sync_command.go +++ b/store/v2/sync_command.go @@ -4,7 +4,7 @@ import ( "time" "github.com/coreos/etcd/store" - "github.com/coreos/etcd/third_party/github.com/coreos/raft" + "github.com/coreos/etcd/third_party/github.com/goraft/raft" ) func init() { diff --git a/store/v2/update_command.go b/store/v2/update_command.go index b1a80b7be..13efc1281 100644 --- a/store/v2/update_command.go +++ b/store/v2/update_command.go @@ -3,7 +3,7 @@ package v2 import ( "github.com/coreos/etcd/log" "github.com/coreos/etcd/store" - "github.com/coreos/etcd/third_party/github.com/coreos/raft" + "github.com/coreos/etcd/third_party/github.com/goraft/raft" "time" ) @@ -13,9 +13,9 @@ func init() { // Update command type UpdateCommand struct { - Key string `json:"key"` - Value string `json:"value"` - ExpireTime time.Time `json:"expireTime"` + Key string `json:"key"` + Value string `json:"value"` + ExpireTime time.Time `json:"expireTime"` } // The name of the update command in the log diff --git a/tests/functional/v1_migration_test.go b/tests/functional/v1_migration_test.go index b698fb103..75aaebb3f 100644 --- a/tests/functional/v1_migration_test.go +++ b/tests/functional/v1_migration_test.go @@ -37,7 +37,7 @@ func TestV1SoloMigration(t *testing.T) { args := []string{"etcd", fmt.Sprintf("-data-dir=%s", nodepath)} args = append(args, "-addr", "127.0.0.1:4001") args = append(args, "-peer-addr", "127.0.0.1:7001") - args = append(args, "-name", "v1") + args = append(args, "-name", "node0") process, err := os.StartProcess(EtcdBinPath, args, procAttr) if err != nil { t.Fatal("start process failed:" + err.Error()) diff --git a/tests/mock/server_v2.go b/tests/mock/server_v2.go index 8701b5271..ab471b22d 100644 --- a/tests/mock/server_v2.go +++ b/tests/mock/server_v2.go @@ -4,14 +4,14 @@ import ( "net/http" "github.com/coreos/etcd/store" - "github.com/coreos/etcd/third_party/github.com/coreos/raft" + "github.com/coreos/etcd/third_party/github.com/goraft/raft" "github.com/stretchr/testify/mock" ) // A mock Server for the v2 handlers. type ServerV2 struct { mock.Mock - store store.Store + store store.Store } func NewServerV2(store store.Store) *ServerV2 { diff --git a/tests/server_utils.go b/tests/server_utils.go index 1a86170c4..9fa35ddc4 100644 --- a/tests/server_utils.go +++ b/tests/server_utils.go @@ -7,7 +7,7 @@ import ( "sync" "time" - "github.com/coreos/etcd/third_party/github.com/coreos/raft" + "github.com/coreos/etcd/third_party/github.com/goraft/raft" "github.com/coreos/etcd/metrics" "github.com/coreos/etcd/server" @@ -15,12 +15,12 @@ import ( ) const ( - testName = "ETCDTEST" - testClientURL = "localhost:4401" - testRaftURL = "localhost:7701" - testSnapshotCount = 10000 - testHeartbeatTimeout = time.Duration(50) * time.Millisecond - testElectionTimeout = time.Duration(200) * time.Millisecond + testName = "ETCDTEST" + testClientURL = "localhost:4401" + testRaftURL = "localhost:7701" + testSnapshotCount = 10000 + testHeartbeatTimeout = time.Duration(50) * time.Millisecond + testElectionTimeout = time.Duration(200) * time.Millisecond ) // Starts a server in a temporary directory. @@ -35,10 +35,10 @@ func RunServer(f func(*server.Server)) { followersStats := server.NewRaftFollowersStats(testName) psConfig := server.PeerServerConfig{ - Name: testName, - URL: "http://" + testRaftURL, - Scheme: "http", - SnapshotCount: testSnapshotCount, + Name: testName, + URL: "http://" + testRaftURL, + Scheme: "http", + SnapshotCount: testSnapshotCount, } mb := metrics.NewBucket("") @@ -103,16 +103,16 @@ func RunServer(f func(*server.Server)) { } type waitHandler struct { - wg *sync.WaitGroup - handler http.Handler + wg *sync.WaitGroup + handler http.Handler } -func (h *waitHandler) ServeHTTP(w http.ResponseWriter, r *http.Request){ - h.wg.Add(1) - defer h.wg.Done() - h.handler.ServeHTTP(w, r) +func (h *waitHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { + h.wg.Add(1) + defer h.wg.Done() + h.handler.ServeHTTP(w, r) - //important to flush before decrementing the wait group. - //we won't get a chance to once main() ends. - w.(http.Flusher).Flush() + //important to flush before decrementing the wait group. + //we won't get a chance to once main() ends. + w.(http.Flusher).Flush() } diff --git a/third_party/code.google.com/p/gogoprotobuf/proto/lib.go b/third_party/code.google.com/p/gogoprotobuf/proto/lib.go index 91ba6dd8c..09a854aed 100644 --- a/third_party/code.google.com/p/gogoprotobuf/proto/lib.go +++ b/third_party/code.google.com/p/gogoprotobuf/proto/lib.go @@ -89,7 +89,7 @@ package example - import "code.google.com/p/gogoprotobuf/proto" + import "github.com/coreos/etcd/third_party/code.google.com/p/gogoprotobuf/proto" type FOO int32 const ( @@ -168,7 +168,7 @@ import ( "log" - "code.google.com/p/gogoprotobuf/proto" + "github.com/coreos/etcd/third_party/code.google.com/p/gogoprotobuf/proto" "./example.pb" ) diff --git a/third_party/code.google.com/p/gogoprotobuf/proto/testdata/test.pb.go.golden b/third_party/code.google.com/p/gogoprotobuf/proto/testdata/test.pb.go.golden index b79ce68e1..8fd41537b 100644 --- a/third_party/code.google.com/p/gogoprotobuf/proto/testdata/test.pb.go.golden +++ b/third_party/code.google.com/p/gogoprotobuf/proto/testdata/test.pb.go.golden @@ -4,7 +4,7 @@ package testdata -import proto "code.google.com/p/gogoprotobuf/proto" +import proto "github.com/coreos/etcd/third_party/code.google.com/p/gogoprotobuf/proto" import json "encoding/json" import math "math" diff --git a/third_party/github.com/coreos/raft/.travis.yml b/third_party/github.com/coreos/raft/.travis.yml deleted file mode 100644 index 4e10f5ffe..000000000 --- a/third_party/github.com/coreos/raft/.travis.yml +++ /dev/null @@ -1,10 +0,0 @@ -language: go - -go: - - 1.1.2 - - 1.2 - -install: - - go get github.com/stretchr/testify/assert - - make dependencies - diff --git a/third_party/github.com/coreos/raft/append_entries_request.go b/third_party/github.com/coreos/raft/append_entries_request.go deleted file mode 100644 index 24f880fa9..000000000 --- a/third_party/github.com/coreos/raft/append_entries_request.go +++ /dev/null @@ -1,82 +0,0 @@ -package raft - -import ( - "io" - "io/ioutil" - - "github.com/coreos/etcd/third_party/code.google.com/p/gogoprotobuf/proto" - "github.com/coreos/etcd/third_party/github.com/coreos/raft/protobuf" -) - -// The request sent to a server to append entries to the log. -type AppendEntriesRequest struct { - Term uint64 - PrevLogIndex uint64 - PrevLogTerm uint64 - CommitIndex uint64 - LeaderName string - Entries []*protobuf.LogEntry -} - -// Creates a new AppendEntries request. -func newAppendEntriesRequest(term uint64, prevLogIndex uint64, prevLogTerm uint64, - commitIndex uint64, leaderName string, entries []*LogEntry) *AppendEntriesRequest { - pbEntries := make([]*protobuf.LogEntry, len(entries)) - - for i := range entries { - pbEntries[i] = entries[i].pb - } - - return &AppendEntriesRequest{ - Term: term, - PrevLogIndex: prevLogIndex, - PrevLogTerm: prevLogTerm, - CommitIndex: commitIndex, - LeaderName: leaderName, - Entries: pbEntries, - } -} - -// Encodes the AppendEntriesRequest to a buffer. Returns the number of bytes -// written and any error that may have occurred. -func (req *AppendEntriesRequest) Encode(w io.Writer) (int, error) { - pb := &protobuf.AppendEntriesRequest{ - Term: proto.Uint64(req.Term), - PrevLogIndex: proto.Uint64(req.PrevLogIndex), - PrevLogTerm: proto.Uint64(req.PrevLogTerm), - CommitIndex: proto.Uint64(req.CommitIndex), - LeaderName: proto.String(req.LeaderName), - Entries: req.Entries, - } - - p, err := proto.Marshal(pb) - if err != nil { - return -1, err - } - - return w.Write(p) -} - -// Decodes the AppendEntriesRequest from a buffer. Returns the number of bytes read and -// any error that occurs. -func (req *AppendEntriesRequest) Decode(r io.Reader) (int, error) { - data, err := ioutil.ReadAll(r) - - if err != nil { - return -1, err - } - - pb := new(protobuf.AppendEntriesRequest) - if err := proto.Unmarshal(data, pb); err != nil { - return -1, err - } - - req.Term = pb.GetTerm() - req.PrevLogIndex = pb.GetPrevLogIndex() - req.PrevLogTerm = pb.GetPrevLogTerm() - req.CommitIndex = pb.GetCommitIndex() - req.LeaderName = pb.GetLeaderName() - req.Entries = pb.GetEntries() - - return len(data), nil -} diff --git a/third_party/github.com/coreos/raft/append_entries_response.go b/third_party/github.com/coreos/raft/append_entries_response.go deleted file mode 100644 index 004c8a865..000000000 --- a/third_party/github.com/coreos/raft/append_entries_response.go +++ /dev/null @@ -1,73 +0,0 @@ -package raft - -import ( - "io" - "io/ioutil" - - "github.com/coreos/etcd/third_party/code.google.com/p/gogoprotobuf/proto" - "github.com/coreos/etcd/third_party/github.com/coreos/raft/protobuf" -) - -// The response returned from a server appending entries to the log. -type AppendEntriesResponse struct { - pb *protobuf.AppendEntriesResponse - peer string - append bool -} - -// Creates a new AppendEntries response. -func newAppendEntriesResponse(term uint64, success bool, index uint64, commitIndex uint64) *AppendEntriesResponse { - pb := &protobuf.AppendEntriesResponse{ - Term: proto.Uint64(term), - Index: proto.Uint64(index), - Success: proto.Bool(success), - CommitIndex: proto.Uint64(commitIndex), - } - - return &AppendEntriesResponse{ - pb: pb, - } -} - -func (aer *AppendEntriesResponse) Index() uint64 { - return aer.pb.GetIndex() -} - -func (aer *AppendEntriesResponse) CommitIndex() uint64 { - return aer.pb.GetCommitIndex() -} - -func (aer *AppendEntriesResponse) Term() uint64 { - return aer.pb.GetTerm() -} - -func (aer *AppendEntriesResponse) Success() bool { - return aer.pb.GetSuccess() -} - -// Encodes the AppendEntriesResponse to a buffer. Returns the number of bytes -// written and any error that may have occurred. -func (resp *AppendEntriesResponse) Encode(w io.Writer) (int, error) { - b, err := proto.Marshal(resp.pb) - if err != nil { - return -1, err - } - - return w.Write(b) -} - -// Decodes the AppendEntriesResponse from a buffer. Returns the number of bytes read and -// any error that occurs. -func (resp *AppendEntriesResponse) Decode(r io.Reader) (int, error) { - data, err := ioutil.ReadAll(r) - if err != nil { - return -1, err - } - - resp.pb = new(protobuf.AppendEntriesResponse) - if err := proto.Unmarshal(data, resp.pb); err != nil { - return -1, err - } - - return len(data), nil -} diff --git a/third_party/github.com/coreos/raft/append_entries_response_test.go b/third_party/github.com/coreos/raft/append_entries_response_test.go deleted file mode 100644 index f51ead1f8..000000000 --- a/third_party/github.com/coreos/raft/append_entries_response_test.go +++ /dev/null @@ -1,34 +0,0 @@ -package raft - -import ( - "bytes" - "testing" -) - -func BenchmarkAppendEntriesResponseEncoding(b *testing.B) { - req, tmp := createTestAppendEntriesResponse(2000) - b.ResetTimer() - for i := 0; i < b.N; i++ { - var buf bytes.Buffer - req.Encode(&buf) - } - b.SetBytes(int64(len(tmp))) -} - -func BenchmarkAppendEntriesResponseDecoding(b *testing.B) { - req, buf := createTestAppendEntriesResponse(2000) - b.ResetTimer() - for i := 0; i < b.N; i++ { - req.Decode(bytes.NewReader(buf)) - } - b.SetBytes(int64(len(buf))) -} - -func createTestAppendEntriesResponse(entryCount int) (*AppendEntriesResponse, []byte) { - resp := newAppendEntriesResponse(1, true, 1, 1) - - var buf bytes.Buffer - resp.Encode(&buf) - - return resp, buf.Bytes() -} diff --git a/third_party/github.com/coreos/raft/join_command.go b/third_party/github.com/coreos/raft/join_command.go deleted file mode 100644 index aa761b473..000000000 --- a/third_party/github.com/coreos/raft/join_command.go +++ /dev/null @@ -1,28 +0,0 @@ -package raft - -// Join command interface -type JoinCommand interface { - Command - NodeName() string -} - -// Join command -type DefaultJoinCommand struct { - Name string `json:"name"` - ConnectionString string `json:"connectionString"` -} - -// The name of the Join command in the log -func (c *DefaultJoinCommand) CommandName() string { - return "raft:join" -} - -func (c *DefaultJoinCommand) Apply(server Server) (interface{}, error) { - err := server.AddPeer(c.Name, c.ConnectionString) - - return []byte("join"), err -} - -func (c *DefaultJoinCommand) NodeName() string { - return c.Name -} diff --git a/third_party/github.com/coreos/raft/leave_command.go b/third_party/github.com/coreos/raft/leave_command.go deleted file mode 100644 index 37ec46395..000000000 --- a/third_party/github.com/coreos/raft/leave_command.go +++ /dev/null @@ -1,26 +0,0 @@ -package raft - -// Leave command interface -type LeaveCommand interface { - Command - NodeName() string -} - -// Leave command -type DefaultLeaveCommand struct { - Name string `json:"name"` -} - -// The name of the Leave command in the log -func (c *DefaultLeaveCommand) CommandName() string { - return "raft:leave" -} - -func (c *DefaultLeaveCommand) Apply(server Server) (interface{}, error) { - err := server.RemovePeer(c.Name) - - return []byte("leave"), err -} -func (c *DefaultLeaveCommand) NodeName() string { - return c.Name -} diff --git a/third_party/github.com/coreos/raft/nop_command.go b/third_party/github.com/coreos/raft/nop_command.go deleted file mode 100644 index 1e5671428..000000000 --- a/third_party/github.com/coreos/raft/nop_command.go +++ /dev/null @@ -1,26 +0,0 @@ -package raft - -import ( - "io" -) - -// NOP command -type NOPCommand struct { -} - -// The name of the NOP command in the log -func (c NOPCommand) CommandName() string { - return "raft:nop" -} - -func (c NOPCommand) Apply(server Server) (interface{}, error) { - return nil, nil -} - -func (c NOPCommand) Encode(w io.Writer) error { - return nil -} - -func (c NOPCommand) Decode(r io.Reader) error { - return nil -} diff --git a/third_party/github.com/coreos/raft/request_vote_request.go b/third_party/github.com/coreos/raft/request_vote_request.go deleted file mode 100644 index 48f39a70e..000000000 --- a/third_party/github.com/coreos/raft/request_vote_request.go +++ /dev/null @@ -1,69 +0,0 @@ -package raft - -import ( - "io" - "io/ioutil" - - "github.com/coreos/etcd/third_party/code.google.com/p/gogoprotobuf/proto" - "github.com/coreos/etcd/third_party/github.com/coreos/raft/protobuf" -) - -// The request sent to a server to vote for a candidate to become a leader. -type RequestVoteRequest struct { - peer *Peer - Term uint64 - LastLogIndex uint64 - LastLogTerm uint64 - CandidateName string -} - -// Creates a new RequestVote request. -func newRequestVoteRequest(term uint64, candidateName string, lastLogIndex uint64, lastLogTerm uint64) *RequestVoteRequest { - return &RequestVoteRequest{ - Term: term, - LastLogIndex: lastLogIndex, - LastLogTerm: lastLogTerm, - CandidateName: candidateName, - } -} - -// Encodes the RequestVoteRequest to a buffer. Returns the number of bytes -// written and any error that may have occurred. -func (req *RequestVoteRequest) Encode(w io.Writer) (int, error) { - pb := &protobuf.RequestVoteRequest{ - Term: proto.Uint64(req.Term), - LastLogIndex: proto.Uint64(req.LastLogIndex), - LastLogTerm: proto.Uint64(req.LastLogTerm), - CandidateName: proto.String(req.CandidateName), - } - p, err := proto.Marshal(pb) - if err != nil { - return -1, err - } - - return w.Write(p) -} - -// Decodes the RequestVoteRequest from a buffer. Returns the number of bytes read and -// any error that occurs. -func (req *RequestVoteRequest) Decode(r io.Reader) (int, error) { - data, err := ioutil.ReadAll(r) - - if err != nil { - return -1, err - } - - totalBytes := len(data) - - pb := &protobuf.RequestVoteRequest{} - if err = proto.Unmarshal(data, pb); err != nil { - return -1, err - } - - req.Term = pb.GetTerm() - req.LastLogIndex = pb.GetLastLogIndex() - req.LastLogTerm = pb.GetLastLogTerm() - req.CandidateName = pb.GetCandidateName() - - return totalBytes, nil -} diff --git a/third_party/github.com/coreos/raft/request_vote_response.go b/third_party/github.com/coreos/raft/request_vote_response.go deleted file mode 100644 index 0be426917..000000000 --- a/third_party/github.com/coreos/raft/request_vote_response.go +++ /dev/null @@ -1,62 +0,0 @@ -package raft - -import ( - "io" - "io/ioutil" - - "github.com/coreos/etcd/third_party/code.google.com/p/gogoprotobuf/proto" - "github.com/coreos/etcd/third_party/github.com/coreos/raft/protobuf" -) - -// The response returned from a server after a vote for a candidate to become a leader. -type RequestVoteResponse struct { - peer *Peer - Term uint64 - VoteGranted bool -} - -// Creates a new RequestVote response. -func newRequestVoteResponse(term uint64, voteGranted bool) *RequestVoteResponse { - return &RequestVoteResponse{ - Term: term, - VoteGranted: voteGranted, - } -} - -// Encodes the RequestVoteResponse to a buffer. Returns the number of bytes -// written and any error that may have occurred. -func (resp *RequestVoteResponse) Encode(w io.Writer) (int, error) { - pb := &protobuf.RequestVoteResponse{ - Term: proto.Uint64(resp.Term), - VoteGranted: proto.Bool(resp.VoteGranted), - } - - p, err := proto.Marshal(pb) - if err != nil { - return -1, err - } - - return w.Write(p) -} - -// Decodes the RequestVoteResponse from a buffer. Returns the number of bytes read and -// any error that occurs. -func (resp *RequestVoteResponse) Decode(r io.Reader) (int, error) { - data, err := ioutil.ReadAll(r) - - if err != nil { - return 0, err - } - - totalBytes := len(data) - - pb := &protobuf.RequestVoteResponse{} - if err = proto.Unmarshal(data, pb); err != nil { - return -1, err - } - - resp.Term = pb.GetTerm() - resp.VoteGranted = pb.GetVoteGranted() - - return totalBytes, nil -} diff --git a/third_party/github.com/coreos/raft/snapshot.go b/third_party/github.com/coreos/raft/snapshot.go deleted file mode 100644 index 083a003e9..000000000 --- a/third_party/github.com/coreos/raft/snapshot.go +++ /dev/null @@ -1,61 +0,0 @@ -package raft - -import ( - "encoding/json" - "fmt" - "hash/crc32" - "os" -) - -// Snapshot represents an in-memory representation of the current state of the system. -type Snapshot struct { - LastIndex uint64 `json:"lastIndex"` - LastTerm uint64 `json:"lastTerm"` - - // Cluster configuration. - Peers []*Peer `json:"peers"` - State []byte `json:"state"` - Path string `json:"path"` -} - -// save writes the snapshot to file. -func (ss *Snapshot) save() error { - // Open the file for writing. - file, err := os.OpenFile(ss.Path, os.O_CREATE|os.O_WRONLY, 0600) - if err != nil { - return err - } - defer file.Close() - - // Serialize to JSON. - b, err := json.Marshal(ss) - if err != nil { - return err - } - - // Generate checksum and write it to disk. - checksum := crc32.ChecksumIEEE(b) - if _, err = fmt.Fprintf(file, "%08x\n", checksum); err != nil { - return err - } - - // Write the snapshot to disk. - if _, err = file.Write(b); err != nil { - return err - } - - // Ensure that the snapshot has been flushed to disk before continuing. - if err := file.Sync(); err != nil { - return err - } - - return nil -} - -// remove deletes the snapshot file. -func (ss *Snapshot) remove() error { - if err := os.Remove(ss.Path); err != nil { - return err - } - return nil -} diff --git a/third_party/github.com/coreos/raft/snapshot_recovery_request.go b/third_party/github.com/coreos/raft/snapshot_recovery_request.go deleted file mode 100644 index a8e0d6888..000000000 --- a/third_party/github.com/coreos/raft/snapshot_recovery_request.go +++ /dev/null @@ -1,90 +0,0 @@ -package raft - -import ( - "io" - "io/ioutil" - - "github.com/coreos/etcd/third_party/code.google.com/p/gogoprotobuf/proto" - "github.com/coreos/etcd/third_party/github.com/coreos/raft/protobuf" -) - -// The request sent to a server to start from the snapshot. -type SnapshotRecoveryRequest struct { - LeaderName string - LastIndex uint64 - LastTerm uint64 - Peers []*Peer - State []byte -} - -// Creates a new Snapshot request. -func newSnapshotRecoveryRequest(leaderName string, snapshot *Snapshot) *SnapshotRecoveryRequest { - return &SnapshotRecoveryRequest{ - LeaderName: leaderName, - LastIndex: snapshot.LastIndex, - LastTerm: snapshot.LastTerm, - Peers: snapshot.Peers, - State: snapshot.State, - } -} - -// Encodes the SnapshotRecoveryRequest to a buffer. Returns the number of bytes -// written and any error that may have occurred. -func (req *SnapshotRecoveryRequest) Encode(w io.Writer) (int, error) { - - protoPeers := make([]*protobuf.SnapshotRecoveryRequest_Peer, len(req.Peers)) - - for i, peer := range req.Peers { - protoPeers[i] = &protobuf.SnapshotRecoveryRequest_Peer{ - Name: proto.String(peer.Name), - ConnectionString: proto.String(peer.ConnectionString), - } - } - - pb := &protobuf.SnapshotRecoveryRequest{ - LeaderName: proto.String(req.LeaderName), - LastIndex: proto.Uint64(req.LastIndex), - LastTerm: proto.Uint64(req.LastTerm), - Peers: protoPeers, - State: req.State, - } - p, err := proto.Marshal(pb) - if err != nil { - return -1, err - } - - return w.Write(p) -} - -// Decodes the SnapshotRecoveryRequest from a buffer. Returns the number of bytes read and -// any error that occurs. -func (req *SnapshotRecoveryRequest) Decode(r io.Reader) (int, error) { - data, err := ioutil.ReadAll(r) - - if err != nil { - return 0, err - } - - totalBytes := len(data) - - pb := &protobuf.SnapshotRecoveryRequest{} - if err = proto.Unmarshal(data, pb); err != nil { - return -1, err - } - - req.LeaderName = pb.GetLeaderName() - req.LastIndex = pb.GetLastIndex() - req.LastTerm = pb.GetLastTerm() - req.State = pb.GetState() - - req.Peers = make([]*Peer, len(pb.Peers)) - - for i, peer := range pb.Peers { - req.Peers[i] = &Peer{ - Name: peer.GetName(), - ConnectionString: peer.GetConnectionString(), - } - } - - return totalBytes, nil -} diff --git a/third_party/github.com/coreos/raft/snapshot_recovery_response.go b/third_party/github.com/coreos/raft/snapshot_recovery_response.go deleted file mode 100644 index 3895139c1..000000000 --- a/third_party/github.com/coreos/raft/snapshot_recovery_response.go +++ /dev/null @@ -1,63 +0,0 @@ -package raft - -import ( - "io" - "io/ioutil" - - "github.com/coreos/etcd/third_party/code.google.com/p/gogoprotobuf/proto" - "github.com/coreos/etcd/third_party/github.com/coreos/raft/protobuf" -) - -// The response returned from a server appending entries to the log. -type SnapshotRecoveryResponse struct { - Term uint64 - Success bool - CommitIndex uint64 -} - -// Creates a new Snapshot response. -func newSnapshotRecoveryResponse(term uint64, success bool, commitIndex uint64) *SnapshotRecoveryResponse { - return &SnapshotRecoveryResponse{ - Term: term, - Success: success, - CommitIndex: commitIndex, - } -} - -// Encode writes the response to a writer. -// Returns the number of bytes written and any error that occurs. -func (req *SnapshotRecoveryResponse) Encode(w io.Writer) (int, error) { - pb := &protobuf.SnapshotRecoveryResponse{ - Term: proto.Uint64(req.Term), - Success: proto.Bool(req.Success), - CommitIndex: proto.Uint64(req.CommitIndex), - } - p, err := proto.Marshal(pb) - if err != nil { - return -1, err - } - - return w.Write(p) -} - -// Decodes the SnapshotRecoveryResponse from a buffer. -func (req *SnapshotRecoveryResponse) Decode(r io.Reader) (int, error) { - data, err := ioutil.ReadAll(r) - - if err != nil { - return 0, err - } - - totalBytes := len(data) - - pb := &protobuf.SnapshotRecoveryResponse{} - if err := proto.Unmarshal(data, pb); err != nil { - return -1, err - } - - req.Term = pb.GetTerm() - req.Success = pb.GetSuccess() - req.CommitIndex = pb.GetCommitIndex() - - return totalBytes, nil -} diff --git a/third_party/github.com/coreos/raft/snapshot_request.go b/third_party/github.com/coreos/raft/snapshot_request.go deleted file mode 100644 index 22228e84d..000000000 --- a/third_party/github.com/coreos/raft/snapshot_request.go +++ /dev/null @@ -1,65 +0,0 @@ -package raft - -import ( - "io" - "io/ioutil" - - "github.com/coreos/etcd/third_party/code.google.com/p/gogoprotobuf/proto" - "github.com/coreos/etcd/third_party/github.com/coreos/raft/protobuf" -) - -// The request sent to a server to start from the snapshot. -type SnapshotRequest struct { - LeaderName string - LastIndex uint64 - LastTerm uint64 -} - -// Creates a new Snapshot request. -func newSnapshotRequest(leaderName string, snapshot *Snapshot) *SnapshotRequest { - return &SnapshotRequest{ - LeaderName: leaderName, - LastIndex: snapshot.LastIndex, - LastTerm: snapshot.LastTerm, - } -} - -// Encodes the SnapshotRequest to a buffer. Returns the number of bytes -// written and any error that may have occurred. -func (req *SnapshotRequest) Encode(w io.Writer) (int, error) { - pb := &protobuf.SnapshotRequest{ - LeaderName: proto.String(req.LeaderName), - LastIndex: proto.Uint64(req.LastIndex), - LastTerm: proto.Uint64(req.LastTerm), - } - p, err := proto.Marshal(pb) - if err != nil { - return -1, err - } - - return w.Write(p) -} - -// Decodes the SnapshotRequest from a buffer. Returns the number of bytes read and -// any error that occurs. -func (req *SnapshotRequest) Decode(r io.Reader) (int, error) { - data, err := ioutil.ReadAll(r) - - if err != nil { - return 0, err - } - - totalBytes := len(data) - - pb := &protobuf.SnapshotRequest{} - - if err := proto.Unmarshal(data, pb); err != nil { - return -1, err - } - - req.LeaderName = pb.GetLeaderName() - req.LastIndex = pb.GetLastIndex() - req.LastTerm = pb.GetLastTerm() - - return totalBytes, nil -} diff --git a/third_party/github.com/coreos/raft/snapshot_response.go b/third_party/github.com/coreos/raft/snapshot_response.go deleted file mode 100644 index bc2d4adf5..000000000 --- a/third_party/github.com/coreos/raft/snapshot_response.go +++ /dev/null @@ -1,56 +0,0 @@ -package raft - -import ( - "io" - "io/ioutil" - - "github.com/coreos/etcd/third_party/code.google.com/p/gogoprotobuf/proto" - "github.com/coreos/etcd/third_party/github.com/coreos/raft/protobuf" -) - -// The response returned if the follower entered snapshot state -type SnapshotResponse struct { - Success bool `json:"success"` -} - -// Creates a new Snapshot response. -func newSnapshotResponse(success bool) *SnapshotResponse { - return &SnapshotResponse{ - Success: success, - } -} - -// Encodes the SnapshotResponse to a buffer. Returns the number of bytes -// written and any error that may have occurred. -func (resp *SnapshotResponse) Encode(w io.Writer) (int, error) { - pb := &protobuf.SnapshotResponse{ - Success: proto.Bool(resp.Success), - } - p, err := proto.Marshal(pb) - if err != nil { - return -1, err - } - - return w.Write(p) -} - -// Decodes the SnapshotResponse from a buffer. Returns the number of bytes read and -// any error that occurs. -func (resp *SnapshotResponse) Decode(r io.Reader) (int, error) { - data, err := ioutil.ReadAll(r) - - if err != nil { - return 0, err - } - - totalBytes := len(data) - - pb := &protobuf.SnapshotResponse{} - if err := proto.Unmarshal(data, pb); err != nil { - return -1, err - } - - resp.Success = pb.GetSuccess() - - return totalBytes, nil -} diff --git a/third_party/github.com/coreos/raft/sort.go b/third_party/github.com/coreos/raft/sort.go deleted file mode 100644 index bf4c303af..000000000 --- a/third_party/github.com/coreos/raft/sort.go +++ /dev/null @@ -1,23 +0,0 @@ -package raft - -//------------------------------------------------------------------------------ -// -// Typedefs -// -//------------------------------------------------------------------------------ - -type uint64Slice []uint64 - -//------------------------------------------------------------------------------ -// -// Functions -// -//------------------------------------------------------------------------------ - -//-------------------------------------- -// uint64 -//-------------------------------------- - -func (p uint64Slice) Len() int { return len(p) } -func (p uint64Slice) Less(i, j int) bool { return p[i] < p[j] } -func (p uint64Slice) Swap(i, j int) { p[i], p[j] = p[j], p[i] } diff --git a/third_party/github.com/coreos/raft/time.go b/third_party/github.com/coreos/raft/time.go deleted file mode 100644 index cae863ccf..000000000 --- a/third_party/github.com/coreos/raft/time.go +++ /dev/null @@ -1,17 +0,0 @@ -package raft - -import ( - "math/rand" - "time" -) - -// Waits for a random time between two durations and sends the current time on -// the returned channel. -func afterBetween(min time.Duration, max time.Duration) <-chan time.Time { - rand := rand.New(rand.NewSource(time.Now().UnixNano())) - d, delta := min, (max - min) - if delta > 0 { - d += time.Duration(rand.Int63n(int64(delta))) - } - return time.After(d) -} diff --git a/third_party/github.com/coreos/raft/util.go b/third_party/github.com/coreos/raft/util.go deleted file mode 100644 index ff3d62f0c..000000000 --- a/third_party/github.com/coreos/raft/util.go +++ /dev/null @@ -1,31 +0,0 @@ -package raft - -import ( - "io" - "os" -) - -// WriteFile writes data to a file named by filename. -// If the file does not exist, WriteFile creates it with permissions perm; -// otherwise WriteFile truncates it before writing. -// This is copied from ioutil.WriteFile with the addition of a Sync call to -// ensure the data reaches the disk. -func writeFileSynced(filename string, data []byte, perm os.FileMode) error { - f, err := os.OpenFile(filename, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, perm) - if err != nil { - return err - } - - n, err := f.Write(data) - if n < len(data) { - f.Close() - return io.ErrShortWrite - } - - err = f.Sync() - if err != nil { - return err - } - - return f.Close() -} diff --git a/third_party/github.com/coreos/raft/.gitignore b/third_party/github.com/goraft/raft/.gitignore similarity index 100% rename from third_party/github.com/coreos/raft/.gitignore rename to third_party/github.com/goraft/raft/.gitignore diff --git a/third_party/github.com/coreos/raft/LICENSE b/third_party/github.com/goraft/raft/LICENSE similarity index 100% rename from third_party/github.com/coreos/raft/LICENSE rename to third_party/github.com/goraft/raft/LICENSE diff --git a/third_party/github.com/coreos/raft/Makefile b/third_party/github.com/goraft/raft/Makefile similarity index 100% rename from third_party/github.com/coreos/raft/Makefile rename to third_party/github.com/goraft/raft/Makefile diff --git a/third_party/github.com/coreos/raft/README.md b/third_party/github.com/goraft/raft/README.md similarity index 98% rename from third_party/github.com/coreos/raft/README.md rename to third_party/github.com/goraft/raft/README.md index 13e0c8404..c3111601b 100644 --- a/third_party/github.com/coreos/raft/README.md +++ b/third_party/github.com/goraft/raft/README.md @@ -36,6 +36,7 @@ These projects are built on go-raft: - [goraft/raftd](https://github.com/goraft/raftd) - A reference implementation for using the go-raft library for distributed consensus. - [skynetservices/skydns](https://github.com/skynetservices/skydns) - DNS for skynet or any other service discovery. - [influxdb/influxdb](https://github.com/influxdb/influxdb) - An open-source, distributed, time series, events, and metrics database. +- [Weed File System](https://weed-fs.googlecode.com) - A scalable distributed key-to-file system with O(1) disk access for each read. If you have a project that you're using go-raft in, please add it to this README so others can see implementation examples. diff --git a/third_party/github.com/goraft/raft/append_entries.go b/third_party/github.com/goraft/raft/append_entries.go new file mode 100644 index 000000000..92ac77459 --- /dev/null +++ b/third_party/github.com/goraft/raft/append_entries.go @@ -0,0 +1,146 @@ +package raft + +import ( + "io" + "io/ioutil" + + "github.com/coreos/etcd/third_party/code.google.com/p/gogoprotobuf/proto" + "github.com/coreos/etcd/third_party/github.com/goraft/raft/protobuf" +) + +// The request sent to a server to append entries to the log. +type AppendEntriesRequest struct { + Term uint64 + PrevLogIndex uint64 + PrevLogTerm uint64 + CommitIndex uint64 + LeaderName string + Entries []*protobuf.LogEntry +} + +// The response returned from a server appending entries to the log. +type AppendEntriesResponse struct { + pb *protobuf.AppendEntriesResponse + peer string + append bool +} + +// Creates a new AppendEntries request. +func newAppendEntriesRequest(term uint64, prevLogIndex uint64, prevLogTerm uint64, + commitIndex uint64, leaderName string, entries []*LogEntry) *AppendEntriesRequest { + pbEntries := make([]*protobuf.LogEntry, len(entries)) + + for i := range entries { + pbEntries[i] = entries[i].pb + } + + return &AppendEntriesRequest{ + Term: term, + PrevLogIndex: prevLogIndex, + PrevLogTerm: prevLogTerm, + CommitIndex: commitIndex, + LeaderName: leaderName, + Entries: pbEntries, + } +} + +// Encodes the AppendEntriesRequest to a buffer. Returns the number of bytes +// written and any error that may have occurred. +func (req *AppendEntriesRequest) Encode(w io.Writer) (int, error) { + pb := &protobuf.AppendEntriesRequest{ + Term: proto.Uint64(req.Term), + PrevLogIndex: proto.Uint64(req.PrevLogIndex), + PrevLogTerm: proto.Uint64(req.PrevLogTerm), + CommitIndex: proto.Uint64(req.CommitIndex), + LeaderName: proto.String(req.LeaderName), + Entries: req.Entries, + } + + p, err := proto.Marshal(pb) + if err != nil { + return -1, err + } + + return w.Write(p) +} + +// Decodes the AppendEntriesRequest from a buffer. Returns the number of bytes read and +// any error that occurs. +func (req *AppendEntriesRequest) Decode(r io.Reader) (int, error) { + data, err := ioutil.ReadAll(r) + + if err != nil { + return -1, err + } + + pb := new(protobuf.AppendEntriesRequest) + if err := proto.Unmarshal(data, pb); err != nil { + return -1, err + } + + req.Term = pb.GetTerm() + req.PrevLogIndex = pb.GetPrevLogIndex() + req.PrevLogTerm = pb.GetPrevLogTerm() + req.CommitIndex = pb.GetCommitIndex() + req.LeaderName = pb.GetLeaderName() + req.Entries = pb.GetEntries() + + return len(data), nil +} + +// Creates a new AppendEntries response. +func newAppendEntriesResponse(term uint64, success bool, index uint64, commitIndex uint64) *AppendEntriesResponse { + pb := &protobuf.AppendEntriesResponse{ + Term: proto.Uint64(term), + Index: proto.Uint64(index), + Success: proto.Bool(success), + CommitIndex: proto.Uint64(commitIndex), + } + + return &AppendEntriesResponse{ + pb: pb, + } +} + +func (aer *AppendEntriesResponse) Index() uint64 { + return aer.pb.GetIndex() +} + +func (aer *AppendEntriesResponse) CommitIndex() uint64 { + return aer.pb.GetCommitIndex() +} + +func (aer *AppendEntriesResponse) Term() uint64 { + return aer.pb.GetTerm() +} + +func (aer *AppendEntriesResponse) Success() bool { + return aer.pb.GetSuccess() +} + +// Encodes the AppendEntriesResponse to a buffer. Returns the number of bytes +// written and any error that may have occurred. +func (resp *AppendEntriesResponse) Encode(w io.Writer) (int, error) { + b, err := proto.Marshal(resp.pb) + if err != nil { + return -1, err + } + + return w.Write(b) +} + +// Decodes the AppendEntriesResponse from a buffer. Returns the number of bytes read and +// any error that occurs. +func (resp *AppendEntriesResponse) Decode(r io.Reader) (int, error) { + data, err := ioutil.ReadAll(r) + if err != nil { + return -1, err + } + + resp.pb = new(protobuf.AppendEntriesResponse) + if err := proto.Unmarshal(data, resp.pb); err != nil { + return -1, err + } + + return len(data), nil +} diff --git a/third_party/github.com/coreos/raft/append_entries_request_test.go b/third_party/github.com/goraft/raft/append_entries_test.go similarity index 58% rename from third_party/github.com/coreos/raft/append_entries_request_test.go rename to third_party/github.com/goraft/raft/append_entries_test.go index 34a827f67..80295127e 100644 --- a/third_party/github.com/coreos/raft/append_entries_request_test.go +++ b/third_party/github.com/goraft/raft/append_entries_test.go @@ -24,6 +24,25 @@ func BenchmarkAppendEntriesRequestDecoding(b *testing.B) { b.SetBytes(int64(len(buf))) } +func BenchmarkAppendEntriesResponseEncoding(b *testing.B) { + req, tmp := createTestAppendEntriesResponse(2000) + b.ResetTimer() + for i := 0; i < b.N; i++ { + var buf bytes.Buffer + req.Encode(&buf) + } + b.SetBytes(int64(len(tmp))) +} + +func BenchmarkAppendEntriesResponseDecoding(b *testing.B) { + req, buf := createTestAppendEntriesResponse(2000) + b.ResetTimer() + for i := 0; i < b.N; i++ { + req.Decode(bytes.NewReader(buf)) + } + b.SetBytes(int64(len(buf))) +} + func createTestAppendEntriesRequest(entryCount int) (*AppendEntriesRequest, []byte) { entries := make([]*LogEntry, 0) for i := 0; i < entryCount; i++ { @@ -38,3 +57,12 @@ func createTestAppendEntriesRequest(entryCount int) (*AppendEntriesRequest, []by return req, buf.Bytes() } + +func createTestAppendEntriesResponse(entryCount int) (*AppendEntriesResponse, []byte) { + resp := newAppendEntriesResponse(1, true, 1, 1) + + var buf bytes.Buffer + resp.Encode(&buf) + + return resp, buf.Bytes() +} diff --git a/third_party/github.com/coreos/raft/command.go b/third_party/github.com/goraft/raft/command.go similarity index 100% rename from third_party/github.com/coreos/raft/command.go rename to third_party/github.com/goraft/raft/command.go diff --git a/third_party/github.com/goraft/raft/commands.go b/third_party/github.com/goraft/raft/commands.go new file mode 100644 index 000000000..17f802daf --- /dev/null +++ b/third_party/github.com/goraft/raft/commands.go @@ -0,0 +1,78 @@ +package raft + +import ( + "io" +) + +// Join command interface +type JoinCommand interface { + Command + NodeName() string +} + +// Join command +type DefaultJoinCommand struct { + Name string `json:"name"` + ConnectionString string `json:"connectionString"` +} + +// Leave command interface +type LeaveCommand interface { + Command + NodeName() string +} + +// Leave command +type DefaultLeaveCommand struct { + Name string `json:"name"` +} + +// NOP command +type NOPCommand struct { +} + +// The name of the Join command in the log +func (c *DefaultJoinCommand) CommandName() string { + return "raft:join" +} + +func (c *DefaultJoinCommand) Apply(server Server) (interface{}, error) { + err := server.AddPeer(c.Name, c.ConnectionString) + + return []byte("join"), err +} + +func (c *DefaultJoinCommand) NodeName() string { + return c.Name +} + +// The name of the Leave command in the log +func (c *DefaultLeaveCommand) CommandName() string { + return "raft:leave" +} + +func (c *DefaultLeaveCommand) Apply(server Server) (interface{}, error) { + err := server.RemovePeer(c.Name) + + return []byte("leave"), err +} +func (c *DefaultLeaveCommand) NodeName() string { + return c.Name +} + +// The name of the NOP command in the log +func (c NOPCommand) CommandName() string { + return "raft:nop" +} + +func (c NOPCommand) Apply(server Server) (interface{}, error) { + return nil, nil +} + +func (c NOPCommand) Encode(w io.Writer) error { + return nil +} + +func (c NOPCommand) Decode(r io.Reader) error { + return nil +} diff --git a/third_party/github.com/coreos/raft/config.go b/third_party/github.com/goraft/raft/config.go similarity index 100% rename from third_party/github.com/coreos/raft/config.go rename to third_party/github.com/goraft/raft/config.go diff --git a/third_party/github.com/coreos/raft/context.go b/third_party/github.com/goraft/raft/context.go similarity index 100% rename from third_party/github.com/coreos/raft/context.go rename to third_party/github.com/goraft/raft/context.go diff --git a/third_party/github.com/coreos/raft/debug.go b/third_party/github.com/goraft/raft/debug.go similarity index 100% rename from third_party/github.com/coreos/raft/debug.go rename to third_party/github.com/goraft/raft/debug.go diff --git a/third_party/github.com/coreos/raft/event.go b/third_party/github.com/goraft/raft/event.go similarity index 100% rename from third_party/github.com/coreos/raft/event.go rename to third_party/github.com/goraft/raft/event.go diff --git a/third_party/github.com/coreos/raft/event_dispatcher.go b/third_party/github.com/goraft/raft/event_dispatcher.go similarity index 100% rename from third_party/github.com/coreos/raft/event_dispatcher.go rename to third_party/github.com/goraft/raft/event_dispatcher.go diff --git a/third_party/github.com/coreos/raft/event_dispatcher_test.go b/third_party/github.com/goraft/raft/event_dispatcher_test.go similarity index 100% rename from third_party/github.com/coreos/raft/event_dispatcher_test.go rename to third_party/github.com/goraft/raft/event_dispatcher_test.go diff --git a/third_party/github.com/coreos/raft/http_transporter.go b/third_party/github.com/goraft/raft/http_transporter.go similarity index 100% rename from third_party/github.com/coreos/raft/http_transporter.go rename to third_party/github.com/goraft/raft/http_transporter.go diff --git a/third_party/github.com/coreos/raft/http_transporter_test.go b/third_party/github.com/goraft/raft/http_transporter_test.go similarity index 100% rename from third_party/github.com/coreos/raft/http_transporter_test.go rename to third_party/github.com/goraft/raft/http_transporter_test.go diff --git a/third_party/github.com/coreos/raft/log.go b/third_party/github.com/goraft/raft/log.go similarity index 99% rename from third_party/github.com/coreos/raft/log.go rename to third_party/github.com/goraft/raft/log.go index 018086021..bd4e4afea 100644 --- a/third_party/github.com/coreos/raft/log.go +++ b/third_party/github.com/goraft/raft/log.go @@ -8,7 +8,7 @@ import ( "os" "sync" - "github.com/coreos/etcd/third_party/github.com/coreos/raft/protobuf" + "github.com/coreos/etcd/third_party/github.com/goraft/raft/protobuf" ) //------------------------------------------------------------------------------ diff --git a/third_party/github.com/coreos/raft/log_entry.go b/third_party/github.com/goraft/raft/log_entry.go similarity index 82% rename from third_party/github.com/coreos/raft/log_entry.go rename to third_party/github.com/goraft/raft/log_entry.go index 8c06eae44..f186b5724 100644 --- a/third_party/github.com/coreos/raft/log_entry.go +++ b/third_party/github.com/goraft/raft/log_entry.go @@ -7,15 +7,15 @@ import ( "io" "github.com/coreos/etcd/third_party/code.google.com/p/gogoprotobuf/proto" - "github.com/coreos/etcd/third_party/github.com/coreos/raft/protobuf" + "github.com/coreos/etcd/third_party/github.com/goraft/raft/protobuf" ) // A log entry stores a single item in the log. type LogEntry struct { - pb *protobuf.LogEntry - Position int64 // position in the log file - log *Log - event *ev + pb *protobuf.LogEntry + Position int64 // position in the log file + log *Log + event *ev } // Creates a new log entry associated with a log. @@ -34,16 +34,16 @@ func newLogEntry(log *Log, event *ev, index uint64, term uint64, command Command } pb := &protobuf.LogEntry{ - Index: proto.Uint64(index), - Term: proto.Uint64(term), - CommandName: proto.String(commandName), - Command: buf.Bytes(), + Index: proto.Uint64(index), + Term: proto.Uint64(term), + CommandName: proto.String(commandName), + Command: buf.Bytes(), } e := &LogEntry{ - pb: pb, - log: log, - event: event, + pb: pb, + log: log, + event: event, } return e, nil @@ -91,7 +91,7 @@ func (e *LogEntry) Decode(r io.Reader) (int, error) { } data := make([]byte, length) - _, err = r.Read(data) + _, err = io.ReadFull(r, data) if err != nil { return -1, err diff --git a/third_party/github.com/coreos/raft/log_test.go b/third_party/github.com/goraft/raft/log_test.go similarity index 100% rename from third_party/github.com/coreos/raft/log_test.go rename to third_party/github.com/goraft/raft/log_test.go diff --git a/third_party/github.com/coreos/raft/peer.go b/third_party/github.com/goraft/raft/peer.go similarity index 98% rename from third_party/github.com/coreos/raft/peer.go rename to third_party/github.com/goraft/raft/peer.go index e9101364c..83ecc683d 100644 --- a/third_party/github.com/coreos/raft/peer.go +++ b/third_party/github.com/goraft/raft/peer.go @@ -68,11 +68,6 @@ func (p *Peer) setPrevLogIndex(value uint64) { p.prevLogIndex = value } -// LastActivity returns the last time any response was received from the peer. -func (p *Peer) LastActivity() time.Time { - return p.lastActivity -} - //------------------------------------------------------------------------------ // // Methods @@ -96,6 +91,11 @@ func (p *Peer) stopHeartbeat(flush bool) { p.stopChan <- flush } +// LastActivity returns the last time any response was received from the peer. +func (p *Peer) LastActivity() time.Time { + return p.lastActivity +} + //-------------------------------------- // Copying //-------------------------------------- @@ -160,7 +160,7 @@ func (p *Peer) flush() { if entries != nil { p.sendAppendEntriesRequest(newAppendEntriesRequest(term, prevLogIndex, prevLogTerm, p.server.log.CommitIndex(), p.server.name, entries)) } else { - p.sendSnapshotRequest(newSnapshotRequest(p.server.name, p.server.lastSnapshot)) + p.sendSnapshotRequest(newSnapshotRequest(p.server.name, p.server.snapshot)) } } @@ -263,7 +263,7 @@ func (p *Peer) sendSnapshotRequest(req *SnapshotRequest) { // Sends an Snapshot Recovery request to the peer through the transport. func (p *Peer) sendSnapshotRecoveryRequest() { - req := newSnapshotRecoveryRequest(p.server.name, p.server.lastSnapshot) + req := newSnapshotRecoveryRequest(p.server.name, p.server.snapshot) debugln("peer.snap.recovery.send: ", p.Name) resp := p.server.Transporter().SendSnapshotRecoveryRequest(p.server, p, req) diff --git a/third_party/github.com/coreos/raft/protobuf/append_entries_request.pb.go b/third_party/github.com/goraft/raft/protobuf/append_entries_request.pb.go similarity index 96% rename from third_party/github.com/coreos/raft/protobuf/append_entries_request.pb.go rename to third_party/github.com/goraft/raft/protobuf/append_entries_request.pb.go index a3a58bbc1..491af090c 100644 --- a/third_party/github.com/coreos/raft/protobuf/append_entries_request.pb.go +++ b/third_party/github.com/goraft/raft/protobuf/append_entries_request.pb.go @@ -33,17 +33,17 @@ var _ = &json.SyntaxError{} var _ = math.Inf type AppendEntriesRequest struct { - Term *uint64 `protobuf:"varint,1,req" json:"Term,omitempty"` - PrevLogIndex *uint64 `protobuf:"varint,2,req" json:"PrevLogIndex,omitempty"` - PrevLogTerm *uint64 `protobuf:"varint,3,req" json:"PrevLogTerm,omitempty"` - CommitIndex *uint64 `protobuf:"varint,4,req" json:"CommitIndex,omitempty"` - LeaderName *string `protobuf:"bytes,5,req" json:"LeaderName,omitempty"` - Entries []*LogEntry `protobuf:"bytes,6,rep" json:"Entries,omitempty"` - XXX_unrecognized []byte `json:"-"` + Term *uint64 `protobuf:"varint,1,req" json:"Term,omitempty"` + PrevLogIndex *uint64 `protobuf:"varint,2,req" json:"PrevLogIndex,omitempty"` + PrevLogTerm *uint64 `protobuf:"varint,3,req" json:"PrevLogTerm,omitempty"` + CommitIndex *uint64 `protobuf:"varint,4,req" json:"CommitIndex,omitempty"` + LeaderName *string `protobuf:"bytes,5,req" json:"LeaderName,omitempty"` + Entries []*LogEntry `protobuf:"bytes,6,rep" json:"Entries,omitempty"` + XXX_unrecognized []byte `json:"-"` } -func (m *AppendEntriesRequest) Reset() { *m = AppendEntriesRequest{} } -func (*AppendEntriesRequest) ProtoMessage() {} +func (m *AppendEntriesRequest) Reset() { *m = AppendEntriesRequest{} } +func (*AppendEntriesRequest) ProtoMessage() {} func (m *AppendEntriesRequest) GetTerm() uint64 { if m != nil && m.Term != nil { diff --git a/third_party/github.com/coreos/raft/protobuf/append_entries_request.proto b/third_party/github.com/goraft/raft/protobuf/append_entries_request.proto similarity index 100% rename from third_party/github.com/coreos/raft/protobuf/append_entries_request.proto rename to third_party/github.com/goraft/raft/protobuf/append_entries_request.proto diff --git a/third_party/github.com/coreos/raft/protobuf/append_entries_responses.pb.go b/third_party/github.com/goraft/raft/protobuf/append_entries_responses.pb.go similarity index 96% rename from third_party/github.com/coreos/raft/protobuf/append_entries_responses.pb.go rename to third_party/github.com/goraft/raft/protobuf/append_entries_responses.pb.go index 6d753a423..e71b4b462 100644 --- a/third_party/github.com/coreos/raft/protobuf/append_entries_responses.pb.go +++ b/third_party/github.com/goraft/raft/protobuf/append_entries_responses.pb.go @@ -33,15 +33,15 @@ var _ = &json.SyntaxError{} var _ = math.Inf type AppendEntriesResponse struct { - Term *uint64 `protobuf:"varint,1,req" json:"Term,omitempty"` - Index *uint64 `protobuf:"varint,2,req" json:"Index,omitempty"` - CommitIndex *uint64 `protobuf:"varint,3,req" json:"CommitIndex,omitempty"` - Success *bool `protobuf:"varint,4,req" json:"Success,omitempty"` - XXX_unrecognized []byte `json:"-"` + Term *uint64 `protobuf:"varint,1,req" json:"Term,omitempty"` + Index *uint64 `protobuf:"varint,2,req" json:"Index,omitempty"` + CommitIndex *uint64 `protobuf:"varint,3,req" json:"CommitIndex,omitempty"` + Success *bool `protobuf:"varint,4,req" json:"Success,omitempty"` + XXX_unrecognized []byte `json:"-"` } -func (m *AppendEntriesResponse) Reset() { *m = AppendEntriesResponse{} } -func (*AppendEntriesResponse) ProtoMessage() {} +func (m *AppendEntriesResponse) Reset() { *m = AppendEntriesResponse{} } +func (*AppendEntriesResponse) ProtoMessage() {} func (m *AppendEntriesResponse) GetTerm() uint64 { if m != nil && m.Term != nil { diff --git a/third_party/github.com/coreos/raft/protobuf/append_entries_responses.proto b/third_party/github.com/goraft/raft/protobuf/append_entries_responses.proto similarity index 100% rename from third_party/github.com/coreos/raft/protobuf/append_entries_responses.proto rename to third_party/github.com/goraft/raft/protobuf/append_entries_responses.proto diff --git a/third_party/github.com/coreos/raft/protobuf/log_entry.pb.go b/third_party/github.com/goraft/raft/protobuf/log_entry.pb.go similarity index 96% rename from third_party/github.com/coreos/raft/protobuf/log_entry.pb.go rename to third_party/github.com/goraft/raft/protobuf/log_entry.pb.go index 6fd14cd57..65394561e 100644 --- a/third_party/github.com/coreos/raft/protobuf/log_entry.pb.go +++ b/third_party/github.com/goraft/raft/protobuf/log_entry.pb.go @@ -33,15 +33,15 @@ var _ = &json.SyntaxError{} var _ = math.Inf type LogEntry struct { - Index *uint64 `protobuf:"varint,1,req" json:"Index,omitempty"` - Term *uint64 `protobuf:"varint,2,req" json:"Term,omitempty"` - CommandName *string `protobuf:"bytes,3,req" json:"CommandName,omitempty"` - Command []byte `protobuf:"bytes,4,opt" json:"Command,omitempty"` - XXX_unrecognized []byte `json:"-"` + Index *uint64 `protobuf:"varint,1,req" json:"Index,omitempty"` + Term *uint64 `protobuf:"varint,2,req" json:"Term,omitempty"` + CommandName *string `protobuf:"bytes,3,req" json:"CommandName,omitempty"` + Command []byte `protobuf:"bytes,4,opt" json:"Command,omitempty"` + XXX_unrecognized []byte `json:"-"` } -func (m *LogEntry) Reset() { *m = LogEntry{} } -func (*LogEntry) ProtoMessage() {} +func (m *LogEntry) Reset() { *m = LogEntry{} } +func (*LogEntry) ProtoMessage() {} func (m *LogEntry) GetIndex() uint64 { if m != nil && m.Index != nil { diff --git a/third_party/github.com/coreos/raft/protobuf/log_entry.proto b/third_party/github.com/goraft/raft/protobuf/log_entry.proto similarity index 100% rename from third_party/github.com/coreos/raft/protobuf/log_entry.proto rename to third_party/github.com/goraft/raft/protobuf/log_entry.proto diff --git a/third_party/github.com/coreos/raft/protobuf/request_vote_request.pb.go b/third_party/github.com/goraft/raft/protobuf/request_vote_request.pb.go similarity index 96% rename from third_party/github.com/coreos/raft/protobuf/request_vote_request.pb.go rename to third_party/github.com/goraft/raft/protobuf/request_vote_request.pb.go index f7dd587f2..dd057480d 100644 --- a/third_party/github.com/coreos/raft/protobuf/request_vote_request.pb.go +++ b/third_party/github.com/goraft/raft/protobuf/request_vote_request.pb.go @@ -33,15 +33,15 @@ var _ = &json.SyntaxError{} var _ = math.Inf type RequestVoteRequest struct { - Term *uint64 `protobuf:"varint,1,req" json:"Term,omitempty"` - LastLogIndex *uint64 `protobuf:"varint,2,req" json:"LastLogIndex,omitempty"` - LastLogTerm *uint64 `protobuf:"varint,3,req" json:"LastLogTerm,omitempty"` - CandidateName *string `protobuf:"bytes,4,req" json:"CandidateName,omitempty"` - XXX_unrecognized []byte `json:"-"` + Term *uint64 `protobuf:"varint,1,req" json:"Term,omitempty"` + LastLogIndex *uint64 `protobuf:"varint,2,req" json:"LastLogIndex,omitempty"` + LastLogTerm *uint64 `protobuf:"varint,3,req" json:"LastLogTerm,omitempty"` + CandidateName *string `protobuf:"bytes,4,req" json:"CandidateName,omitempty"` + XXX_unrecognized []byte `json:"-"` } -func (m *RequestVoteRequest) Reset() { *m = RequestVoteRequest{} } -func (*RequestVoteRequest) ProtoMessage() {} +func (m *RequestVoteRequest) Reset() { *m = RequestVoteRequest{} } +func (*RequestVoteRequest) ProtoMessage() {} func (m *RequestVoteRequest) GetTerm() uint64 { if m != nil && m.Term != nil { diff --git a/third_party/github.com/coreos/raft/protobuf/request_vote_request.proto b/third_party/github.com/goraft/raft/protobuf/request_vote_request.proto similarity index 100% rename from third_party/github.com/coreos/raft/protobuf/request_vote_request.proto rename to third_party/github.com/goraft/raft/protobuf/request_vote_request.proto diff --git a/third_party/github.com/coreos/raft/protobuf/request_vote_responses.pb.go b/third_party/github.com/goraft/raft/protobuf/request_vote_responses.pb.go similarity index 97% rename from third_party/github.com/coreos/raft/protobuf/request_vote_responses.pb.go rename to third_party/github.com/goraft/raft/protobuf/request_vote_responses.pb.go index 0cefd0536..337f10563 100644 --- a/third_party/github.com/coreos/raft/protobuf/request_vote_responses.pb.go +++ b/third_party/github.com/goraft/raft/protobuf/request_vote_responses.pb.go @@ -33,13 +33,13 @@ var _ = &json.SyntaxError{} var _ = math.Inf type RequestVoteResponse struct { - Term *uint64 `protobuf:"varint,1,req" json:"Term,omitempty"` - VoteGranted *bool `protobuf:"varint,2,req" json:"VoteGranted,omitempty"` - XXX_unrecognized []byte `json:"-"` + Term *uint64 `protobuf:"varint,1,req" json:"Term,omitempty"` + VoteGranted *bool `protobuf:"varint,2,req" json:"VoteGranted,omitempty"` + XXX_unrecognized []byte `json:"-"` } -func (m *RequestVoteResponse) Reset() { *m = RequestVoteResponse{} } -func (*RequestVoteResponse) ProtoMessage() {} +func (m *RequestVoteResponse) Reset() { *m = RequestVoteResponse{} } +func (*RequestVoteResponse) ProtoMessage() {} func (m *RequestVoteResponse) GetTerm() uint64 { if m != nil && m.Term != nil { diff --git a/third_party/github.com/coreos/raft/protobuf/request_vote_responses.proto b/third_party/github.com/goraft/raft/protobuf/request_vote_responses.proto similarity index 100% rename from third_party/github.com/coreos/raft/protobuf/request_vote_responses.proto rename to third_party/github.com/goraft/raft/protobuf/request_vote_responses.proto diff --git a/third_party/github.com/coreos/raft/protobuf/snapshot_recovery_request.pb.go b/third_party/github.com/goraft/raft/protobuf/snapshot_recovery_request.pb.go similarity index 95% rename from third_party/github.com/coreos/raft/protobuf/snapshot_recovery_request.pb.go rename to third_party/github.com/goraft/raft/protobuf/snapshot_recovery_request.pb.go index 1ebbf19bd..88a4e668b 100644 --- a/third_party/github.com/coreos/raft/protobuf/snapshot_recovery_request.pb.go +++ b/third_party/github.com/goraft/raft/protobuf/snapshot_recovery_request.pb.go @@ -33,16 +33,16 @@ var _ = &json.SyntaxError{} var _ = math.Inf type SnapshotRecoveryRequest struct { - LeaderName *string `protobuf:"bytes,1,req" json:"LeaderName,omitempty"` - LastIndex *uint64 `protobuf:"varint,2,req" json:"LastIndex,omitempty"` - LastTerm *uint64 `protobuf:"varint,3,req" json:"LastTerm,omitempty"` - Peers []*SnapshotRecoveryRequest_Peer `protobuf:"bytes,4,rep" json:"Peers,omitempty"` - State []byte `protobuf:"bytes,5,req" json:"State,omitempty"` - XXX_unrecognized []byte `json:"-"` + LeaderName *string `protobuf:"bytes,1,req" json:"LeaderName,omitempty"` + LastIndex *uint64 `protobuf:"varint,2,req" json:"LastIndex,omitempty"` + LastTerm *uint64 `protobuf:"varint,3,req" json:"LastTerm,omitempty"` + Peers []*SnapshotRecoveryRequest_Peer `protobuf:"bytes,4,rep" json:"Peers,omitempty"` + State []byte `protobuf:"bytes,5,req" json:"State,omitempty"` + XXX_unrecognized []byte `json:"-"` } -func (m *SnapshotRecoveryRequest) Reset() { *m = SnapshotRecoveryRequest{} } -func (*SnapshotRecoveryRequest) ProtoMessage() {} +func (m *SnapshotRecoveryRequest) Reset() { *m = SnapshotRecoveryRequest{} } +func (*SnapshotRecoveryRequest) ProtoMessage() {} func (m *SnapshotRecoveryRequest) GetLeaderName() string { if m != nil && m.LeaderName != nil { @@ -80,13 +80,13 @@ func (m *SnapshotRecoveryRequest) GetState() []byte { } type SnapshotRecoveryRequest_Peer struct { - Name *string `protobuf:"bytes,1,req" json:"Name,omitempty"` - ConnectionString *string `protobuf:"bytes,2,req" json:"ConnectionString,omitempty"` - XXX_unrecognized []byte `json:"-"` + Name *string `protobuf:"bytes,1,req" json:"Name,omitempty"` + ConnectionString *string `protobuf:"bytes,2,req" json:"ConnectionString,omitempty"` + XXX_unrecognized []byte `json:"-"` } -func (m *SnapshotRecoveryRequest_Peer) Reset() { *m = SnapshotRecoveryRequest_Peer{} } -func (*SnapshotRecoveryRequest_Peer) ProtoMessage() {} +func (m *SnapshotRecoveryRequest_Peer) Reset() { *m = SnapshotRecoveryRequest_Peer{} } +func (*SnapshotRecoveryRequest_Peer) ProtoMessage() {} func (m *SnapshotRecoveryRequest_Peer) GetName() string { if m != nil && m.Name != nil { diff --git a/third_party/github.com/coreos/raft/protobuf/snapshot_recovery_request.proto b/third_party/github.com/goraft/raft/protobuf/snapshot_recovery_request.proto similarity index 100% rename from third_party/github.com/coreos/raft/protobuf/snapshot_recovery_request.proto rename to third_party/github.com/goraft/raft/protobuf/snapshot_recovery_request.proto diff --git a/third_party/github.com/coreos/raft/protobuf/snapshot_recovery_response.pb.go b/third_party/github.com/goraft/raft/protobuf/snapshot_recovery_response.pb.go similarity index 97% rename from third_party/github.com/coreos/raft/protobuf/snapshot_recovery_response.pb.go rename to third_party/github.com/goraft/raft/protobuf/snapshot_recovery_response.pb.go index 345f09e4c..ec012c533 100644 --- a/third_party/github.com/coreos/raft/protobuf/snapshot_recovery_response.pb.go +++ b/third_party/github.com/goraft/raft/protobuf/snapshot_recovery_response.pb.go @@ -33,14 +33,14 @@ var _ = &json.SyntaxError{} var _ = math.Inf type SnapshotRecoveryResponse struct { - Term *uint64 `protobuf:"varint,1,req" json:"Term,omitempty"` - Success *bool `protobuf:"varint,2,req" json:"Success,omitempty"` - CommitIndex *uint64 `protobuf:"varint,3,req" json:"CommitIndex,omitempty"` - XXX_unrecognized []byte `json:"-"` + Term *uint64 `protobuf:"varint,1,req" json:"Term,omitempty"` + Success *bool `protobuf:"varint,2,req" json:"Success,omitempty"` + CommitIndex *uint64 `protobuf:"varint,3,req" json:"CommitIndex,omitempty"` + XXX_unrecognized []byte `json:"-"` } -func (m *SnapshotRecoveryResponse) Reset() { *m = SnapshotRecoveryResponse{} } -func (*SnapshotRecoveryResponse) ProtoMessage() {} +func (m *SnapshotRecoveryResponse) Reset() { *m = SnapshotRecoveryResponse{} } +func (*SnapshotRecoveryResponse) ProtoMessage() {} func (m *SnapshotRecoveryResponse) GetTerm() uint64 { if m != nil && m.Term != nil { diff --git a/third_party/github.com/coreos/raft/protobuf/snapshot_recovery_response.proto b/third_party/github.com/goraft/raft/protobuf/snapshot_recovery_response.proto similarity index 100% rename from third_party/github.com/coreos/raft/protobuf/snapshot_recovery_response.proto rename to third_party/github.com/goraft/raft/protobuf/snapshot_recovery_response.proto diff --git a/third_party/github.com/coreos/raft/protobuf/snapshot_request.pb.go b/third_party/github.com/goraft/raft/protobuf/snapshot_request.pb.go similarity index 97% rename from third_party/github.com/coreos/raft/protobuf/snapshot_request.pb.go rename to third_party/github.com/goraft/raft/protobuf/snapshot_request.pb.go index e5cf4a695..2af28c838 100644 --- a/third_party/github.com/coreos/raft/protobuf/snapshot_request.pb.go +++ b/third_party/github.com/goraft/raft/protobuf/snapshot_request.pb.go @@ -33,14 +33,14 @@ var _ = &json.SyntaxError{} var _ = math.Inf type SnapshotRequest struct { - LeaderName *string `protobuf:"bytes,1,req" json:"LeaderName,omitempty"` - LastIndex *uint64 `protobuf:"varint,2,req" json:"LastIndex,omitempty"` - LastTerm *uint64 `protobuf:"varint,3,req" json:"LastTerm,omitempty"` - XXX_unrecognized []byte `json:"-"` + LeaderName *string `protobuf:"bytes,1,req" json:"LeaderName,omitempty"` + LastIndex *uint64 `protobuf:"varint,2,req" json:"LastIndex,omitempty"` + LastTerm *uint64 `protobuf:"varint,3,req" json:"LastTerm,omitempty"` + XXX_unrecognized []byte `json:"-"` } -func (m *SnapshotRequest) Reset() { *m = SnapshotRequest{} } -func (*SnapshotRequest) ProtoMessage() {} +func (m *SnapshotRequest) Reset() { *m = SnapshotRequest{} } +func (*SnapshotRequest) ProtoMessage() {} func (m *SnapshotRequest) GetLeaderName() string { if m != nil && m.LeaderName != nil { diff --git a/third_party/github.com/coreos/raft/protobuf/snapshot_request.proto b/third_party/github.com/goraft/raft/protobuf/snapshot_request.proto similarity index 100% rename from third_party/github.com/coreos/raft/protobuf/snapshot_request.proto rename to third_party/github.com/goraft/raft/protobuf/snapshot_request.proto diff --git a/third_party/github.com/coreos/raft/protobuf/snapshot_response.pb.go b/third_party/github.com/goraft/raft/protobuf/snapshot_response.pb.go similarity index 97% rename from third_party/github.com/coreos/raft/protobuf/snapshot_response.pb.go rename to third_party/github.com/goraft/raft/protobuf/snapshot_response.pb.go index 9d009256e..9b4193187 100644 --- a/third_party/github.com/coreos/raft/protobuf/snapshot_response.pb.go +++ b/third_party/github.com/goraft/raft/protobuf/snapshot_response.pb.go @@ -33,12 +33,12 @@ var _ = &json.SyntaxError{} var _ = math.Inf type SnapshotResponse struct { - Success *bool `protobuf:"varint,1,req" json:"Success,omitempty"` - XXX_unrecognized []byte `json:"-"` + Success *bool `protobuf:"varint,1,req" json:"Success,omitempty"` + XXX_unrecognized []byte `json:"-"` } -func (m *SnapshotResponse) Reset() { *m = SnapshotResponse{} } -func (*SnapshotResponse) ProtoMessage() {} +func (m *SnapshotResponse) Reset() { *m = SnapshotResponse{} } +func (*SnapshotResponse) ProtoMessage() {} func (m *SnapshotResponse) GetSuccess() bool { if m != nil && m.Success != nil { diff --git a/third_party/github.com/coreos/raft/protobuf/snapshot_response.proto b/third_party/github.com/goraft/raft/protobuf/snapshot_response.proto similarity index 100% rename from third_party/github.com/coreos/raft/protobuf/snapshot_response.proto rename to third_party/github.com/goraft/raft/protobuf/snapshot_response.proto diff --git a/third_party/github.com/goraft/raft/request_vote.go b/third_party/github.com/goraft/raft/request_vote.go new file mode 100644 index 000000000..1f7f576eb --- /dev/null +++ b/third_party/github.com/goraft/raft/request_vote.go @@ -0,0 +1,122 @@ +package raft + +import ( + "io" + "io/ioutil" + + "github.com/coreos/etcd/third_party/code.google.com/p/gogoprotobuf/proto" + "github.com/coreos/etcd/third_party/github.com/goraft/raft/protobuf" +) + +// The request sent to a server to vote for a candidate to become a leader. +type RequestVoteRequest struct { + peer *Peer + Term uint64 + LastLogIndex uint64 + LastLogTerm uint64 + CandidateName string +} + +// The response returned from a server after a vote for a candidate to become a leader. +type RequestVoteResponse struct { + peer *Peer + Term uint64 + VoteGranted bool +} + +// Creates a new RequestVote request. +func newRequestVoteRequest(term uint64, candidateName string, lastLogIndex uint64, lastLogTerm uint64) *RequestVoteRequest { + return &RequestVoteRequest{ + Term: term, + LastLogIndex: lastLogIndex, + LastLogTerm: lastLogTerm, + CandidateName: candidateName, + } +} + +// Encodes the RequestVoteRequest to a buffer. Returns the number of bytes +// written and any error that may have occurred. +func (req *RequestVoteRequest) Encode(w io.Writer) (int, error) { + pb := &protobuf.RequestVoteRequest{ + Term: proto.Uint64(req.Term), + LastLogIndex: proto.Uint64(req.LastLogIndex), + LastLogTerm: proto.Uint64(req.LastLogTerm), + CandidateName: proto.String(req.CandidateName), + } + p, err := proto.Marshal(pb) + if err != nil { + return -1, err + } + + return w.Write(p) +} + +// Decodes the RequestVoteRequest from a buffer. Returns the number of bytes read and +// any error that occurs. +func (req *RequestVoteRequest) Decode(r io.Reader) (int, error) { + data, err := ioutil.ReadAll(r) + + if err != nil { + return -1, err + } + + totalBytes := len(data) + + pb := &protobuf.RequestVoteRequest{} + if err = proto.Unmarshal(data, pb); err != nil { + return -1, err + } + + req.Term = pb.GetTerm() + req.LastLogIndex = pb.GetLastLogIndex() + req.LastLogTerm = pb.GetLastLogTerm() + req.CandidateName = pb.GetCandidateName() + + return totalBytes, nil +} + +// Creates a new RequestVote response. +func newRequestVoteResponse(term uint64, voteGranted bool) *RequestVoteResponse { + return &RequestVoteResponse{ + Term: term, + VoteGranted: voteGranted, + } +} + +// Encodes the RequestVoteResponse to a buffer. Returns the number of bytes +// written and any error that may have occurred. +func (resp *RequestVoteResponse) Encode(w io.Writer) (int, error) { + pb := &protobuf.RequestVoteResponse{ + Term: proto.Uint64(resp.Term), + VoteGranted: proto.Bool(resp.VoteGranted), + } + + p, err := proto.Marshal(pb) + if err != nil { + return -1, err + } + + return w.Write(p) +} + +// Decodes the RequestVoteResponse from a buffer. Returns the number of bytes read and +// any error that occurs. +func (resp *RequestVoteResponse) Decode(r io.Reader) (int, error) { + data, err := ioutil.ReadAll(r) + + if err != nil { + return 0, err + } + + totalBytes := len(data) + + pb := &protobuf.RequestVoteResponse{} + if err = proto.Unmarshal(data, pb); err != nil { + return -1, err + } + + resp.Term = pb.GetTerm() + resp.VoteGranted = pb.GetVoteGranted() + + return totalBytes, nil +} diff --git a/third_party/github.com/coreos/raft/server.go b/third_party/github.com/goraft/raft/server.go similarity index 76% rename from third_party/github.com/coreos/raft/server.go rename to third_party/github.com/goraft/raft/server.go index 8cef1341a..3f9b653a4 100644 --- a/third_party/github.com/coreos/raft/server.go +++ b/third_party/github.com/goraft/raft/server.go @@ -21,6 +21,7 @@ import ( const ( Stopped = "stopped" + Initialized = "initialized" Follower = "follower" Candidate = "candidate" Leader = "leader" @@ -45,8 +46,6 @@ const ( // election timeout. const ElectionTimeoutThresholdPercent = 0.8 -var stopValue interface{} - //------------------------------------------------------------------------------ // // Errors @@ -96,6 +95,7 @@ type Server interface { AddPeer(name string, connectiongString string) error RemovePeer(name string) error Peers() map[string]*Peer + Init() error Start() error Stop() Running() bool @@ -103,6 +103,7 @@ type Server interface { TakeSnapshot() error LoadSnapshot() error AddEventListener(string, EventListener) + FlushCommitIndex() } type server struct { @@ -122,13 +123,19 @@ type server struct { mutex sync.RWMutex syncedPeer map[string]bool - stopped chan bool + stopped chan chan bool c chan *ev electionTimeout time.Duration heartbeatInterval time.Duration - currentSnapshot *Snapshot - lastSnapshot *Snapshot + snapshot *Snapshot + + // PendingSnapshot is an unfinished snapshot. + // After the pendingSnapshot is saved to disk, + // it will be set to snapshot and also will be + // set to nil. + pendingSnapshot *Snapshot + stateMachine StateMachine maxLogEntriesPerRequest uint64 @@ -170,7 +177,7 @@ func NewServer(name string, path string, transporter Transporter, stateMachine S state: Stopped, peers: make(map[string]*Peer), log: newLog(), - stopped: make(chan bool), + stopped: make(chan chan bool), c: make(chan *ev, 256), electionTimeout: DefaultElectionTimeout, heartbeatInterval: DefaultHeartbeatInterval, @@ -292,9 +299,8 @@ func (s *server) setState(state string) { } // Dispatch state and leader change events. - if prevState != state { - s.DispatchEvent(newEvent(StateChangeEventType, s.state, prevState)) - } + s.DispatchEvent(newEvent(StateChangeEventType, s.state, prevState)) + if prevLeader != s.leader { s.DispatchEvent(newEvent(LeaderChangeEventType, s.leader, prevLeader)) } @@ -419,35 +425,21 @@ func init() { RegisterCommand(&DefaultLeaveCommand{}) } -// Start as follow +// Start the raft server // If log entries exist then allow promotion to candidate if no AEs received. // If no log entries exist then wait for AEs from another node. // If no log entries exist and a self-join command is issued then // immediately become leader and commit entry. - func (s *server) Start() error { // Exit if the server is already running. - if s.State() != Stopped { - return errors.New("raft.Server: Server already running") + if s.Running() { + return fmt.Errorf("raft.Server: Server already running[%v]", s.state) } - // Create snapshot directory if not exist - os.Mkdir(path.Join(s.path, "snapshot"), 0700) - - if err := s.readConf(); err != nil { - s.debugln("raft: Conf file error: ", err) - return fmt.Errorf("raft: Initialization error: %s", err) + if err := s.Init(); err != nil { + return err } - // Initialize the log and load it up. - if err := s.log.open(s.LogPath()); err != nil { - s.debugln("raft: Log error: ", err) - return fmt.Errorf("raft: Initialization error: %s", err) - } - - // Update the term to the last term in the log. - _, s.currentTerm = s.log.lastInfo() - s.setState(Follower) // If no log entries exist then @@ -470,69 +462,102 @@ func (s *server) Start() error { return nil } +// Init initializes the raft server +func (s *server) Init() error { + if s.Running() { + return fmt.Errorf("raft.Server: Server already running[%v]", s.state) + } + + // server has been initialized or server was stopped after initialized + if s.state == Initialized || !s.log.isEmpty() { + s.state = Initialized + return nil + } + + // Create snapshot directory if it does not exist + err := os.Mkdir(path.Join(s.path, "snapshot"), 0700) + if err != nil && !os.IsExist(err) { + s.debugln("raft: Snapshot dir error: ", err) + return fmt.Errorf("raft: Initialization error: %s", err) + } + + if err := s.readConf(); err != nil { + s.debugln("raft: Conf file error: ", err) + return fmt.Errorf("raft: Initialization error: %s", err) + } + + // Initialize the log and load it up. + if err := s.log.open(s.LogPath()); err != nil { + s.debugln("raft: Log error: ", err) + return fmt.Errorf("raft: Initialization error: %s", err) + } + + // Update the term to the last term in the log. + _, s.currentTerm = s.log.lastInfo() + + s.state = Initialized + return nil +} + // Shuts down the server. func (s *server) Stop() { - s.send(&stopValue) + stop := make(chan bool) + s.stopped <- stop + s.state = Stopped // make sure the server has stopped before we close the log - <-s.stopped + <-stop s.log.close() - s.state = Stopped } // Checks if the server is currently running. func (s *server) Running() bool { s.mutex.RLock() defer s.mutex.RUnlock() - return s.state != Stopped + return (s.state != Stopped && s.state != Initialized) } //-------------------------------------- // Term //-------------------------------------- -// Sets the current term for the server. This is only used when an external -// current term is found. -func (s *server) setCurrentTerm(term uint64, leaderName string, append bool) { +// updates the current term for the server. This is only used when a larger +// external term is found. +func (s *server) updateCurrentTerm(term uint64, leaderName string) { + _assert(term > s.currentTerm, + "upadteCurrentTerm: update is called when term is not larger than currentTerm") + s.mutex.Lock() defer s.mutex.Unlock() - // Store previous values temporarily. - prevState := s.state prevTerm := s.currentTerm prevLeader := s.leader - if term > s.currentTerm { - // stop heartbeats before step-down - if s.state == Leader { - s.mutex.Unlock() - for _, peer := range s.peers { - peer.stopHeartbeat(false) - } - s.mutex.Lock() + // set currentTerm = T, convert to follower (§5.1) + // stop heartbeats before step-down + if s.state == Leader { + s.mutex.Unlock() + for _, peer := range s.peers { + peer.stopHeartbeat(false) } - // update the term and clear vote for - s.state = Follower - s.currentTerm = term - s.leader = leaderName - s.votedFor = "" - } else if term == s.currentTerm && s.state != Leader && append { - // discover new leader when candidate - // save leader name when follower - s.state = Follower - s.leader = leaderName + s.mutex.Lock() } + // update the term and clear vote for + if s.state != Follower { + s.mutex.Unlock() + s.setState(Follower) + s.mutex.Lock() + } + s.currentTerm = term + s.leader = leaderName + s.votedFor = "" // Dispatch change events. - if prevState != s.state { - s.DispatchEvent(newEvent(StateChangeEventType, s.state, prevState)) - } + s.DispatchEvent(newEvent(TermChangeEventType, s.currentTerm, prevTerm)) + if prevLeader != s.leader { s.DispatchEvent(newEvent(LeaderChangeEventType, s.leader, prevLeader)) } - if prevTerm != s.currentTerm { - s.DispatchEvent(newEvent(TermChangeEventType, s.currentTerm, prevTerm)) - } } //-------------------------------------- @@ -555,26 +580,19 @@ func (s *server) setCurrentTerm(term uint64, leaderName string, append bool) { func (s *server) loop() { defer s.debugln("server.loop.end") - for { + for s.state != Stopped { state := s.State() s.debugln("server.loop.run ", state) switch state { case Follower: s.followerLoop() - case Candidate: s.candidateLoop() - case Leader: s.leaderLoop() - case Snapshotting: s.snapshotLoop() - - case Stopped: - s.stopped <- true - return } } } @@ -610,7 +628,6 @@ func (s *server) sendAsync(value interface{}) { // 1.Receiving valid AppendEntries RPC, or // 2.Granting vote to candidate func (s *server) followerLoop() { - s.setState(Follower) since := time.Now() electionTimeout := s.ElectionTimeout() timeoutChan := afterBetween(s.ElectionTimeout(), s.ElectionTimeout()*2) @@ -619,42 +636,41 @@ func (s *server) followerLoop() { var err error update := false select { + case stop := <-s.stopped: + s.setState(Stopped) + stop <- true + return + case e := <-s.c: - if e.target == &stopValue { - s.setState(Stopped) - } else { - switch req := e.target.(type) { - case JoinCommand: - //If no log entries exist and a self-join command is issued - //then immediately become leader and commit entry. - if s.log.currentIndex() == 0 && req.NodeName() == s.Name() { - s.debugln("selfjoin and promote to leader") - s.setState(Leader) - s.processCommand(req, e) - } else { - err = NotLeaderError - } - case *AppendEntriesRequest: - // If heartbeats get too close to the election timeout then send an event. - elapsedTime := time.Now().Sub(since) - if elapsedTime > time.Duration(float64(electionTimeout)*ElectionTimeoutThresholdPercent) { - s.DispatchEvent(newEvent(ElectionTimeoutThresholdEventType, elapsedTime, nil)) - } - e.returnValue, update = s.processAppendEntriesRequest(req) - case *RequestVoteRequest: - e.returnValue, update = s.processRequestVoteRequest(req) - case *SnapshotRequest: - e.returnValue = s.processSnapshotRequest(req) - default: + switch req := e.target.(type) { + case JoinCommand: + //If no log entries exist and a self-join command is issued + //then immediately become leader and commit entry. + if s.log.currentIndex() == 0 && req.NodeName() == s.Name() { + s.debugln("selfjoin and promote to leader") + s.setState(Leader) + s.processCommand(req, e) + } else { err = NotLeaderError } + case *AppendEntriesRequest: + // If heartbeats get too close to the election timeout then send an event. + elapsedTime := time.Now().Sub(since) + if elapsedTime > time.Duration(float64(electionTimeout)*ElectionTimeoutThresholdPercent) { + s.DispatchEvent(newEvent(ElectionTimeoutThresholdEventType, elapsedTime, nil)) + } + e.returnValue, update = s.processAppendEntriesRequest(req) + case *RequestVoteRequest: + e.returnValue, update = s.processRequestVoteRequest(req) + case *SnapshotRequest: + e.returnValue = s.processSnapshotRequest(req) + default: + err = NotLeaderError } - // Callback to event. e.c <- err case <-timeoutChan: - // only allow synced follower to promote to candidate if s.promotable() { s.setState(Candidate) @@ -675,8 +691,6 @@ func (s *server) followerLoop() { // The event loop that is run when the server is in a Candidate state. func (s *server) candidateLoop() { - lastLogIndex, lastLogTerm := s.log.lastInfo() - // Clear leader value. prevLeader := s.leader s.leader = "" @@ -684,81 +698,77 @@ func (s *server) candidateLoop() { s.DispatchEvent(newEvent(LeaderChangeEventType, s.leader, prevLeader)) } + lastLogIndex, lastLogTerm := s.log.lastInfo() + doVote := true + votesGranted := 0 + var timeoutChan <-chan time.Time + var respChan chan *RequestVoteResponse + for s.State() == Candidate { - // Increment current term, vote for self. - s.currentTerm++ - s.votedFor = s.name + if doVote { + // Increment current term, vote for self. + s.currentTerm++ + s.votedFor = s.name - // Send RequestVote RPCs to all other servers. - respChan := make(chan *RequestVoteResponse, len(s.peers)) - for _, peer := range s.peers { - go peer.sendVoteRequest(newRequestVoteRequest(s.currentTerm, s.name, lastLogIndex, lastLogTerm), respChan) + // Send RequestVote RPCs to all other servers. + respChan = make(chan *RequestVoteResponse, len(s.peers)) + for _, peer := range s.peers { + go peer.sendVoteRequest(newRequestVoteRequest(s.currentTerm, s.name, lastLogIndex, lastLogTerm), respChan) + } + + // Wait for either: + // * Votes received from majority of servers: become leader + // * AppendEntries RPC received from new leader: step down. + // * Election timeout elapses without election resolution: increment term, start new election + // * Discover higher term: step down (§5.1) + votesGranted = 1 + timeoutChan = afterBetween(s.ElectionTimeout(), s.ElectionTimeout()*2) + doVote = false } - // Wait for either: - // * Votes received from majority of servers: become leader - // * AppendEntries RPC received from new leader: step down. - // * Election timeout elapses without election resolution: increment term, start new election - // * Discover higher term: step down (§5.1) - votesGranted := 1 - timeoutChan := afterBetween(s.ElectionTimeout(), s.ElectionTimeout()*2) - timeout := false - - for { - // If we received enough votes then stop waiting for more votes. - s.debugln("server.candidate.votes: ", votesGranted, " quorum:", s.QuorumSize()) - if votesGranted >= s.QuorumSize() { - s.setState(Leader) - break - } - - // Collect votes from peers. - select { - case resp := <-respChan: - if resp.VoteGranted { - s.debugln("server.candidate.vote.granted: ", votesGranted) - votesGranted++ - } else if resp.Term > s.currentTerm { - s.debugln("server.candidate.vote.failed") - s.setCurrentTerm(resp.Term, "", false) - } else { - s.debugln("server.candidate.vote: denied") - } - - case e := <-s.c: - var err error - if e.target == &stopValue { - s.setState(Stopped) - } else { - switch req := e.target.(type) { - case Command: - err = NotLeaderError - case *AppendEntriesRequest: - e.returnValue, _ = s.processAppendEntriesRequest(req) - case *RequestVoteRequest: - e.returnValue, _ = s.processRequestVoteRequest(req) - } - } - // Callback to event. - e.c <- err - - case <-timeoutChan: - timeout = true - } - - // both process AER and RVR can make the server to follower - // also break when timeout happens - if s.State() != Candidate || timeout { - break - } + // If we received enough votes then stop waiting for more votes. + // And return from the candidate loop + if votesGranted == s.QuorumSize() { + s.debugln("server.candidate.recv.enough.votes") + s.setState(Leader) + return + } + + // Collect votes from peers. + select { + case stop := <-s.stopped: + s.setState(Stopped) + stop <- true + return + + case resp := <-respChan: + if success := s.processVoteResponse(resp); success { + s.debugln("server.candidate.vote.granted: ", votesGranted) + votesGranted++ + } + + case e := <-s.c: + var err error + switch req := e.target.(type) { + case Command: + err = NotLeaderError + case *AppendEntriesRequest: + e.returnValue, _ = s.processAppendEntriesRequest(req) + case *RequestVoteRequest: + e.returnValue, _ = s.processRequestVoteRequest(req) + } + + // Callback to event. + e.c <- err + + case <-timeoutChan: + doVote = true } - // continue when timeout happened } } // The event loop that is run when the server is in a Leader state. func (s *server) leaderLoop() { - s.setState(Leader) logIndex, _ := s.log.lastInfo() // Update the peers prevLogIndex to leader's lastLogIndex and start heartbeat. @@ -778,25 +788,26 @@ func (s *server) leaderLoop() { for s.State() == Leader { var err error select { + case stop := <-s.stopped: + // Stop all peers before stop + for _, peer := range s.peers { + peer.stopHeartbeat(false) + } + s.setState(Stopped) + stop <- true + return + case e := <-s.c: - if e.target == &stopValue { - // Stop all peers before stop - for _, peer := range s.peers { - peer.stopHeartbeat(false) - } - s.setState(Stopped) - } else { - switch req := e.target.(type) { - case Command: - s.processCommand(req, e) - continue - case *AppendEntriesRequest: - e.returnValue, _ = s.processAppendEntriesRequest(req) - case *AppendEntriesResponse: - s.processAppendEntriesResponse(req) - case *RequestVoteRequest: - e.returnValue, _ = s.processRequestVoteRequest(req) - } + switch req := e.target.(type) { + case Command: + s.processCommand(req, e) + continue + case *AppendEntriesRequest: + e.returnValue, _ = s.processAppendEntriesRequest(req) + case *AppendEntriesResponse: + s.processAppendEntriesResponse(req) + case *RequestVoteRequest: + e.returnValue, _ = s.processRequestVoteRequest(req) } // Callback to event. @@ -808,16 +819,15 @@ func (s *server) leaderLoop() { } func (s *server) snapshotLoop() { - s.setState(Snapshotting) - for s.State() == Snapshotting { var err error - - e := <-s.c - - if e.target == &stopValue { + select { + case stop := <-s.stopped: s.setState(Stopped) - } else { + stop <- true + return + + case e := <-s.c: switch req := e.target.(type) { case Command: err = NotLeaderError @@ -828,9 +838,9 @@ func (s *server) snapshotLoop() { case *SnapshotRecoveryRequest: e.returnValue = s.processSnapshotRecoveryRequest(req) } + // Callback to event. + e.c <- err } - // Callback to event. - e.c <- err } } @@ -892,8 +902,17 @@ func (s *server) processAppendEntriesRequest(req *AppendEntriesRequest) (*Append return newAppendEntriesResponse(s.currentTerm, false, s.log.currentIndex(), s.log.CommitIndex()), false } - // Update term and leader. - s.setCurrentTerm(req.Term, req.LeaderName, true) + if req.Term == s.currentTerm { + _assert(s.state != Leader, "leader.elected.at.same.term.%d\n", s.currentTerm) + // change state to follower + s.state = Follower + // discover new leader when candidate + // save leader name when follower + s.leader = req.LeaderName + } else { + // Update term and leader. + s.updateCurrentTerm(req.Term, req.LeaderName) + } // Reject if log doesn't contain a matching previous entry. if err := s.log.truncate(req.PrevLogIndex, req.PrevLogTerm); err != nil { @@ -924,7 +943,7 @@ func (s *server) processAppendEntriesRequest(req *AppendEntriesRequest) (*Append func (s *server) processAppendEntriesResponse(resp *AppendEntriesResponse) { // If we find a higher term then change to a follower and exit. if resp.Term() > s.Term() { - s.setCurrentTerm(resp.Term(), "", false) + s.updateCurrentTerm(resp.Term(), "") return } @@ -964,6 +983,25 @@ func (s *server) processAppendEntriesResponse(resp *AppendEntriesResponse) { } } +// processVoteReponse processes a vote request: +// 1. if the vote is granted for the current term of the candidate, return true +// 2. if the vote is denied due to smaller term, update the term of this server +// which will also cause the candidate to step-down, and return false. +// 3. if the vote is for a smaller term, ignore it and return false. +func (s *server) processVoteResponse(resp *RequestVoteResponse) bool { + if resp.VoteGranted && resp.Term == s.currentTerm { + return true + } + + if resp.Term > s.currentTerm { + s.debugln("server.candidate.vote.failed") + s.updateCurrentTerm(resp.Term, "") + } else { + s.debugln("server.candidate.vote: denied") + } + return false +} + //-------------------------------------- // Request Vote //-------------------------------------- @@ -986,10 +1024,12 @@ func (s *server) processRequestVoteRequest(req *RequestVoteRequest) (*RequestVot return newRequestVoteResponse(s.currentTerm, false), false } - s.setCurrentTerm(req.Term, "", false) - - // If we've already voted for a different candidate then don't vote for this candidate. - if s.votedFor != "" && s.votedFor != req.CandidateName { + // If the term of the request peer is larger than this node, update the term + // If the term is equal and we've already voted for a different candidate then + // don't vote for this candidate. + if req.Term > s.Term() { + s.updateCurrentTerm(req.Term, "") + } else if s.votedFor != "" && s.votedFor != req.CandidateName { s.debugln("server.deny.vote: cause duplicate vote: ", req.CandidateName, " already vote for ", s.votedFor) return newRequestVoteResponse(s.currentTerm, false), false @@ -1057,7 +1097,15 @@ func (s *server) RemovePeer(name string) error { // Stop peer and remove it. if s.State() == Leader { - peer.stopHeartbeat(true) + // We create a go routine here to avoid potential deadlock. + // We are holding log write lock when reach this line of code. + // Peer.stopHeartbeat can be blocked without go routine, if the + // target go routine (which we want to stop) is calling + // log.getEntriesAfter and waiting for log read lock. + // So we might be holding log lock and waiting for log lock, + // which lead to a deadlock. + // TODO(xiangli) refactor log lock + go peer.stopHeartbeat(true) } delete(s.peers, name) @@ -1076,30 +1124,35 @@ func (s *server) RemovePeer(name string) error { //-------------------------------------- func (s *server) TakeSnapshot() error { - // TODO: put a snapshot mutex - s.debugln("take Snapshot") + if s.stateMachine == nil { + return errors.New("Snapshot: Cannot create snapshot. Missing state machine.") + } + // Shortcut without lock // Exit if the server is currently creating a snapshot. - if s.currentSnapshot != nil { - return errors.New("handling snapshot") + if s.pendingSnapshot != nil { + return errors.New("Snapshot: Last snapshot is not finished.") } - // Exit if there are no logs yet in the system. + // TODO: acquire the lock and no more committed is allowed + // This will be done after finishing refactoring heartbeat + s.debugln("take.snapshot") + lastIndex, lastTerm := s.log.commitInfo() - path := s.SnapshotPath(lastIndex, lastTerm) - if lastIndex == 0 { - return errors.New("No logs") + + // check if there is log has been committed since the + // last snapshot. + if lastIndex == s.log.startIndex { + return nil } - var state []byte - var err error - if s.stateMachine != nil { - state, err = s.stateMachine.Save() - if err != nil { - return err - } - } else { - state = []byte{0} + path := s.SnapshotPath(lastIndex, lastTerm) + // Attach snapshot to pending snapshot and save it to disk. + s.pendingSnapshot = &Snapshot{lastIndex, lastTerm, nil, nil, path} + + state, err := s.stateMachine.Save() + if err != nil { + return err } // Clone the list of peers. @@ -1109,8 +1162,9 @@ func (s *server) TakeSnapshot() error { } peers = append(peers, &Peer{Name: s.Name(), ConnectionString: s.connectionString}) - // Attach current snapshot and save it to disk. - s.currentSnapshot = &Snapshot{lastIndex, lastTerm, peers, state, path} + // Attach snapshot to pending snapshot and save it to disk. + s.pendingSnapshot.Peers = peers + s.pendingSnapshot.State = state s.saveSnapshot() // We keep some log entries after the snapshot. @@ -1126,24 +1180,24 @@ func (s *server) TakeSnapshot() error { // Retrieves the log path for the server. func (s *server) saveSnapshot() error { - if s.currentSnapshot == nil { - return errors.New("no snapshot to save") + if s.pendingSnapshot == nil { + return errors.New("pendingSnapshot.is.nil") } // Write snapshot to disk. - if err := s.currentSnapshot.save(); err != nil { + if err := s.pendingSnapshot.save(); err != nil { return err } // Swap the current and last snapshots. - tmp := s.lastSnapshot - s.lastSnapshot = s.currentSnapshot + tmp := s.snapshot + s.snapshot = s.pendingSnapshot // Delete the previous snapshot if there is any change - if tmp != nil && !(tmp.LastIndex == s.lastSnapshot.LastIndex && tmp.LastTerm == s.lastSnapshot.LastTerm) { + if tmp != nil && !(tmp.LastIndex == s.snapshot.LastIndex && tmp.LastTerm == s.snapshot.LastTerm) { tmp.remove() } - s.currentSnapshot = nil + s.pendingSnapshot = nil return nil } @@ -1184,7 +1238,7 @@ func (s *server) SnapshotRecoveryRequest(req *SnapshotRecoveryRequest) *Snapshot func (s *server) processSnapshotRecoveryRequest(req *SnapshotRecoveryRequest) *SnapshotRecoveryResponse { // Recover state sent from request. if err := s.stateMachine.Recovery(req.State); err != nil { - return newSnapshotRecoveryResponse(req.LastTerm, false, req.LastIndex) + panic("cannot recover from previous state") } // Recover the cluster configuration. @@ -1198,14 +1252,13 @@ func (s *server) processSnapshotRecoveryRequest(req *SnapshotRecoveryRequest) *S s.log.updateCommitIndex(req.LastIndex) // Create local snapshot. - s.currentSnapshot = &Snapshot{req.LastIndex, req.LastTerm, req.Peers, req.State, s.SnapshotPath(req.LastIndex, req.LastTerm)} + s.pendingSnapshot = &Snapshot{req.LastIndex, req.LastTerm, req.Peers, req.State, s.SnapshotPath(req.LastIndex, req.LastTerm)} s.saveSnapshot() // Clear the previous log entries. s.log.compact(req.LastIndex, req.LastTerm) return newSnapshotRecoveryResponse(req.LastTerm, true, req.LastIndex) - } // Load a snapshot at restart @@ -1213,6 +1266,7 @@ func (s *server) LoadSnapshot() error { // Open snapshot/ directory. dir, err := os.OpenFile(path.Join(s.path, "snapshot"), os.O_RDONLY, 0) if err != nil { + s.debugln("cannot.open.snapshot: ", err) return err } @@ -1225,7 +1279,8 @@ func (s *server) LoadSnapshot() error { dir.Close() if len(filenames) == 0 { - return errors.New("no snapshot") + s.debugln("no.snapshot.to.load") + return nil } // Grab the latest snapshot. @@ -1245,7 +1300,7 @@ func (s *server) LoadSnapshot() error { if err != nil { return err } else if n != 1 { - return errors.New("Bad snapshot file") + return errors.New("checksum.err: bad.snapshot.file") } // Load remaining snapshot contents. @@ -1262,26 +1317,26 @@ func (s *server) LoadSnapshot() error { } // Decode snapshot. - if err = json.Unmarshal(b, &s.lastSnapshot); err != nil { - s.debugln("unmarshal error: ", err) + if err = json.Unmarshal(b, &s.snapshot); err != nil { + s.debugln("unmarshal.snapshot.error: ", err) return err } // Recover snapshot into state machine. - if err = s.stateMachine.Recovery(s.lastSnapshot.State); err != nil { - s.debugln("recovery error: ", err) + if err = s.stateMachine.Recovery(s.snapshot.State); err != nil { + s.debugln("recovery.snapshot.error: ", err) return err } // Recover cluster configuration. - for _, peer := range s.lastSnapshot.Peers { + for _, peer := range s.snapshot.Peers { s.AddPeer(peer.Name, peer.ConnectionString) } // Update log state. - s.log.startTerm = s.lastSnapshot.LastTerm - s.log.startIndex = s.lastSnapshot.LastIndex - s.log.updateCommitIndex(s.lastSnapshot.LastIndex) + s.log.startTerm = s.snapshot.LastTerm + s.log.startIndex = s.snapshot.LastIndex + s.log.updateCommitIndex(s.snapshot.LastIndex) return err } @@ -1290,6 +1345,14 @@ func (s *server) LoadSnapshot() error { // Config File //-------------------------------------- +// Flushes commit index to the disk. +// So when the raft server restarts, it will commit upto the flushed commitIndex. +func (s *server) FlushCommitIndex() { + s.debugln("server.conf.update") + // Write the configuration to file. + s.writeConf() +} + func (s *server) writeConf() { peers := make([]*Peer, len(s.peers)) diff --git a/third_party/github.com/coreos/raft/server_test.go b/third_party/github.com/goraft/raft/server_test.go similarity index 94% rename from third_party/github.com/coreos/raft/server_test.go rename to third_party/github.com/goraft/raft/server_test.go index edfe48f7f..9b53c9a9b 100644 --- a/third_party/github.com/coreos/raft/server_test.go +++ b/third_party/github.com/goraft/raft/server_test.go @@ -139,6 +139,47 @@ func TestServerRequestVoteDenyIfCandidateLogIsBehind(t *testing.T) { } } +func TestProcessVoteResponse(t *testing.T) { + // server Term: 0, status: Leader + // response Term : 1, granted + // Expectation: not success + // Server Term 1 status:Leader + server := &server{} + server.eventDispatcher = newEventDispatcher(server) + server.currentTerm = 0 + server.state = Leader + response := &RequestVoteResponse{ + VoteGranted: true, + Term: 1, + } + if success := server.processVoteResponse(response); success { + t.Fatal("Process should fail if the resp's term is larger than server's") + } + if server.state != Follower { + t.Fatal("Server should stepdown") + } + + // server Term: 1, status: Follower + // response Term: 2, granted + // Expectation: not success + response.Term = 2 + if success := server.processVoteResponse(response); success { + t.Fatal("Process should fail if the resp's term is larger than server's") + } + if server.state != Follower { + t.Fatal("Server should still be Follower") + } + + server.currentTerm = 2 + // server Term: 2, status: Follower + // response Term: 2 + // Expectation: success + if success := server.processVoteResponse(response); !success { + t.Fatal("Process should success if the server's term is larger than resp's") + } + +} + // //-------------------------------------- // // Promotion // //-------------------------------------- diff --git a/third_party/github.com/goraft/raft/snapshot.go b/third_party/github.com/goraft/raft/snapshot.go new file mode 100644 index 000000000..9156e7d08 --- /dev/null +++ b/third_party/github.com/goraft/raft/snapshot.go @@ -0,0 +1,304 @@ +package raft + +import ( + "encoding/json" + "fmt" + "hash/crc32" + "io" + "io/ioutil" + "os" + + "github.com/coreos/etcd/third_party/code.google.com/p/gogoprotobuf/proto" + "github.com/coreos/etcd/third_party/github.com/goraft/raft/protobuf" +) + +// Snapshot represents an in-memory representation of the current state of the system. +type Snapshot struct { + LastIndex uint64 `json:"lastIndex"` + LastTerm uint64 `json:"lastTerm"` + + // Cluster configuration. + Peers []*Peer `json:"peers"` + State []byte `json:"state"` + Path string `json:"path"` +} + +// The request sent to a server to start from the snapshot. +type SnapshotRecoveryRequest struct { + LeaderName string + LastIndex uint64 + LastTerm uint64 + Peers []*Peer + State []byte +} + +// The response returned from a server appending entries to the log. +type SnapshotRecoveryResponse struct { + Term uint64 + Success bool + CommitIndex uint64 +} + +// The request sent to a server to start from the snapshot. +type SnapshotRequest struct { + LeaderName string + LastIndex uint64 + LastTerm uint64 +} + +// The response returned if the follower entered snapshot state +type SnapshotResponse struct { + Success bool `json:"success"` +} + +// save writes the snapshot to file. +func (ss *Snapshot) save() error { + // Open the file for writing. + file, err := os.OpenFile(ss.Path, os.O_CREATE|os.O_WRONLY, 0600) + if err != nil { + return err + } + defer file.Close() + + // Serialize to JSON. + b, err := json.Marshal(ss) + if err != nil { + return err + } + + // Generate checksum and write it to disk. + checksum := crc32.ChecksumIEEE(b) + if _, err = fmt.Fprintf(file, "%08x\n", checksum); err != nil { + return err + } + + // Write the snapshot to disk. + if _, err = file.Write(b); err != nil { + return err + } + + // Ensure that the snapshot has been flushed to disk before continuing. + if err := file.Sync(); err != nil { + return err + } + + return nil +} + +// remove deletes the snapshot file. +func (ss *Snapshot) remove() error { + if err := os.Remove(ss.Path); err != nil { + return err + } + return nil +} + +// Creates a new Snapshot request. +func newSnapshotRecoveryRequest(leaderName string, snapshot *Snapshot) *SnapshotRecoveryRequest { + return &SnapshotRecoveryRequest{ + LeaderName: leaderName, + LastIndex: snapshot.LastIndex, + LastTerm: snapshot.LastTerm, + Peers: snapshot.Peers, + State: snapshot.State, + } +} + +// Encodes the SnapshotRecoveryRequest to a buffer. Returns the number of bytes +// written and any error that may have occurred. +func (req *SnapshotRecoveryRequest) Encode(w io.Writer) (int, error) { + + protoPeers := make([]*protobuf.SnapshotRecoveryRequest_Peer, len(req.Peers)) + + for i, peer := range req.Peers { + protoPeers[i] = &protobuf.SnapshotRecoveryRequest_Peer{ + Name: proto.String(peer.Name), + ConnectionString: proto.String(peer.ConnectionString), + } + } + + pb := &protobuf.SnapshotRecoveryRequest{ + LeaderName: proto.String(req.LeaderName), + LastIndex: proto.Uint64(req.LastIndex), + LastTerm: proto.Uint64(req.LastTerm), + Peers: protoPeers, + State: req.State, + } + p, err := proto.Marshal(pb) + if err != nil { + return -1, err + } + + return w.Write(p) +} + +// Decodes the SnapshotRecoveryRequest from a buffer. Returns the number of bytes read and +// any error that occurs. +func (req *SnapshotRecoveryRequest) Decode(r io.Reader) (int, error) { + data, err := ioutil.ReadAll(r) + + if err != nil { + return 0, err + } + + totalBytes := len(data) + + pb := &protobuf.SnapshotRecoveryRequest{} + if err = proto.Unmarshal(data, pb); err != nil { + return -1, err + } + + req.LeaderName = pb.GetLeaderName() + req.LastIndex = pb.GetLastIndex() + req.LastTerm = pb.GetLastTerm() + req.State = pb.GetState() + + req.Peers = make([]*Peer, len(pb.Peers)) + + for i, peer := range pb.Peers { + req.Peers[i] = &Peer{ + Name: peer.GetName(), + ConnectionString: peer.GetConnectionString(), + } + } + + return totalBytes, nil +} + +// Creates a new Snapshot response. +func newSnapshotRecoveryResponse(term uint64, success bool, commitIndex uint64) *SnapshotRecoveryResponse { + return &SnapshotRecoveryResponse{ + Term: term, + Success: success, + CommitIndex: commitIndex, + } +} + +// Encode writes the response to a writer. +// Returns the number of bytes written and any error that occurs. +func (req *SnapshotRecoveryResponse) Encode(w io.Writer) (int, error) { + pb := &protobuf.SnapshotRecoveryResponse{ + Term: proto.Uint64(req.Term), + Success: proto.Bool(req.Success), + CommitIndex: proto.Uint64(req.CommitIndex), + } + p, err := proto.Marshal(pb) + if err != nil { + return -1, err + } + + return w.Write(p) +} + +// Decodes the SnapshotRecoveryResponse from a buffer. +func (req *SnapshotRecoveryResponse) Decode(r io.Reader) (int, error) { + data, err := ioutil.ReadAll(r) + + if err != nil { + return 0, err + } + + totalBytes := len(data) + + pb := &protobuf.SnapshotRecoveryResponse{} + if err := proto.Unmarshal(data, pb); err != nil { + return -1, err + } + + req.Term = pb.GetTerm() + req.Success = pb.GetSuccess() + req.CommitIndex = pb.GetCommitIndex() + + return totalBytes, nil +} + +// Creates a new Snapshot request. +func newSnapshotRequest(leaderName string, snapshot *Snapshot) *SnapshotRequest { + return &SnapshotRequest{ + LeaderName: leaderName, + LastIndex: snapshot.LastIndex, + LastTerm: snapshot.LastTerm, + } +} + +// Encodes the SnapshotRequest to a buffer. Returns the number of bytes +// written and any error that may have occurred. +func (req *SnapshotRequest) Encode(w io.Writer) (int, error) { + pb := &protobuf.SnapshotRequest{ + LeaderName: proto.String(req.LeaderName), + LastIndex: proto.Uint64(req.LastIndex), + LastTerm: proto.Uint64(req.LastTerm), + } + p, err := proto.Marshal(pb) + if err != nil { + return -1, err + } + + return w.Write(p) +} + +// Decodes the SnapshotRequest from a buffer. Returns the number of bytes read and +// any error that occurs. +func (req *SnapshotRequest) Decode(r io.Reader) (int, error) { + data, err := ioutil.ReadAll(r) + + if err != nil { + return 0, err + } + + totalBytes := len(data) + + pb := &protobuf.SnapshotRequest{} + + if err := proto.Unmarshal(data, pb); err != nil { + return -1, err + } + + req.LeaderName = pb.GetLeaderName() + req.LastIndex = pb.GetLastIndex() + req.LastTerm = pb.GetLastTerm() + + return totalBytes, nil +} + +// Creates a new Snapshot response. +func newSnapshotResponse(success bool) *SnapshotResponse { + return &SnapshotResponse{ + Success: success, + } +} + +// Encodes the SnapshotResponse to a buffer. Returns the number of bytes +// written and any error that may have occurred. +func (resp *SnapshotResponse) Encode(w io.Writer) (int, error) { + pb := &protobuf.SnapshotResponse{ + Success: proto.Bool(resp.Success), + } + p, err := proto.Marshal(pb) + if err != nil { + return -1, err + } + + return w.Write(p) +} + +// Decodes the SnapshotResponse from a buffer. Returns the number of bytes read and +// any error that occurs. +func (resp *SnapshotResponse) Decode(r io.Reader) (int, error) { + data, err := ioutil.ReadAll(r) + + if err != nil { + return 0, err + } + + totalBytes := len(data) + + pb := &protobuf.SnapshotResponse{} + if err := proto.Unmarshal(data, pb); err != nil { + return -1, err + } + + resp.Success = pb.GetSuccess() + + return totalBytes, nil +} diff --git a/third_party/github.com/coreos/raft/snapshot_test.go b/third_party/github.com/goraft/raft/snapshot_test.go similarity index 93% rename from third_party/github.com/coreos/raft/snapshot_test.go rename to third_party/github.com/goraft/raft/snapshot_test.go index 6cc03efab..d650aa975 100644 --- a/third_party/github.com/coreos/raft/snapshot_test.go +++ b/third_party/github.com/goraft/raft/snapshot_test.go @@ -16,13 +16,13 @@ func TestSnapshot(t *testing.T) { s.Do(&testCommand1{}) err := s.TakeSnapshot() assert.NoError(t, err) - assert.Equal(t, s.(*server).lastSnapshot.LastIndex, uint64(2)) + assert.Equal(t, s.(*server).snapshot.LastIndex, uint64(2)) // Repeat to make sure new snapshot gets created. s.Do(&testCommand1{}) err = s.TakeSnapshot() assert.NoError(t, err) - assert.Equal(t, s.(*server).lastSnapshot.LastIndex, uint64(4)) + assert.Equal(t, s.(*server).snapshot.LastIndex, uint64(4)) // Restart server. s.Stop() diff --git a/third_party/github.com/coreos/raft/statemachine.go b/third_party/github.com/goraft/raft/statemachine.go similarity index 100% rename from third_party/github.com/coreos/raft/statemachine.go rename to third_party/github.com/goraft/raft/statemachine.go diff --git a/third_party/github.com/coreos/raft/statemachine_test.go b/third_party/github.com/goraft/raft/statemachine_test.go similarity index 100% rename from third_party/github.com/coreos/raft/statemachine_test.go rename to third_party/github.com/goraft/raft/statemachine_test.go diff --git a/third_party/github.com/coreos/raft/test.go b/third_party/github.com/goraft/raft/test.go similarity index 100% rename from third_party/github.com/coreos/raft/test.go rename to third_party/github.com/goraft/raft/test.go diff --git a/third_party/github.com/coreos/raft/transporter.go b/third_party/github.com/goraft/raft/transporter.go similarity index 100% rename from third_party/github.com/coreos/raft/transporter.go rename to third_party/github.com/goraft/raft/transporter.go diff --git a/third_party/github.com/goraft/raft/util.go b/third_party/github.com/goraft/raft/util.go new file mode 100644 index 000000000..5fa2c41a8 --- /dev/null +++ b/third_party/github.com/goraft/raft/util.go @@ -0,0 +1,61 @@ +package raft + +import ( + "fmt" + "io" + "math/rand" + "os" + "time" +) + +// uint64Slice implements sort interface +type uint64Slice []uint64 + +func (p uint64Slice) Len() int { return len(p) } +func (p uint64Slice) Less(i, j int) bool { return p[i] < p[j] } +func (p uint64Slice) Swap(i, j int) { p[i], p[j] = p[j], p[i] } + +// WriteFile writes data to a file named by filename. +// If the file does not exist, WriteFile creates it with permissions perm; +// otherwise WriteFile truncates it before writing. +// This is copied from ioutil.WriteFile with the addition of a Sync call to +// ensure the data reaches the disk. +func writeFileSynced(filename string, data []byte, perm os.FileMode) error { + f, err := os.OpenFile(filename, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, perm) + if err != nil { + return err + } + + n, err := f.Write(data) + if n < len(data) { + f.Close() + return io.ErrShortWrite + } + + err = f.Sync() + if err != nil { + return err + } + + return f.Close() +} + +// Waits for a random time between two durations and sends the current time on +// the returned channel. +func afterBetween(min time.Duration, max time.Duration) <-chan time.Time { + rand := rand.New(rand.NewSource(time.Now().UnixNano())) + d, delta := min, (max - min) + if delta > 0 { + d += time.Duration(rand.Int63n(int64(delta))) + } + return time.After(d) +} + +// TODO(xiangli): Remove assertions when we reach version 1.0 + +// _assert will panic with a given formatted message if the given condition is false. +func _assert(condition bool, msg string, v ...interface{}) { + if !condition { + panic(fmt.Sprintf("assertion failed: "+msg, v...)) + } +} diff --git a/third_party/github.com/coreos/raft/z_test.go b/third_party/github.com/goraft/raft/z_test.go similarity index 100% rename from third_party/github.com/coreos/raft/z_test.go rename to third_party/github.com/goraft/raft/z_test.go