Add auto-demotion after peer inactivity.
This commit is contained in:
@ -49,12 +49,12 @@ const (
|
|||||||
EcodeRaftInternal = 300
|
EcodeRaftInternal = 300
|
||||||
EcodeLeaderElect = 301
|
EcodeLeaderElect = 301
|
||||||
|
|
||||||
EcodeWatcherCleared = 400
|
EcodeWatcherCleared = 400
|
||||||
EcodeEventIndexCleared = 401
|
EcodeEventIndexCleared = 401
|
||||||
EcodeProxyInternal = 402
|
EcodeProxyInternal = 402
|
||||||
EcodeInvalidActiveSize = 403
|
EcodeInvalidActiveSize = 403
|
||||||
EcodeInvalidPromoteDelay = 404
|
EcodeInvalidPromoteDelay = 404
|
||||||
EcodePromoteError = 405
|
EcodePromoteError = 405
|
||||||
)
|
)
|
||||||
|
|
||||||
func init() {
|
func init() {
|
||||||
|
@ -27,7 +27,7 @@ type ClusterConfig struct {
|
|||||||
// NewClusterConfig returns a cluster configuration with default settings.
|
// NewClusterConfig returns a cluster configuration with default settings.
|
||||||
func NewClusterConfig() *ClusterConfig {
|
func NewClusterConfig() *ClusterConfig {
|
||||||
return &ClusterConfig{
|
return &ClusterConfig{
|
||||||
ActiveSize: DefaultActiveSize,
|
ActiveSize: DefaultActiveSize,
|
||||||
PromoteDelay: DefaultPromoteDelay,
|
PromoteDelay: DefaultPromoteDelay,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -46,4 +46,3 @@ func (c *DemoteCommand) Apply(context raft.Context) (interface{}, error) {
|
|||||||
func (c *DemoteCommand) NodeName() string {
|
func (c *DemoteCommand) NodeName() string {
|
||||||
return c.Name
|
return c.Name
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -13,21 +13,30 @@ func init() {
|
|||||||
}
|
}
|
||||||
|
|
||||||
// The JoinCommand adds a node to the cluster.
|
// 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 {
|
type JoinCommand struct {
|
||||||
MinVersion int `json:"minVersion"`
|
MinVersion int `json:"minVersion"`
|
||||||
MaxVersion int `json:"maxVersion"`
|
MaxVersion int `json:"maxVersion"`
|
||||||
Name string `json:"name"`
|
Name string `json:"name"`
|
||||||
RaftURL string `json:"raftURL"`
|
RaftURL string `json:"raftURL"`
|
||||||
EtcdURL string `json:"etcdURL"`
|
EtcdURL string `json:"etcdURL"`
|
||||||
}
|
}
|
||||||
|
|
||||||
func NewJoinCommand(minVersion int, maxVersion int, name, raftUrl, etcdUrl string) *JoinCommand {
|
func NewJoinCommand(minVersion int, maxVersion int, name, raftUrl, etcdUrl string) *JoinCommand {
|
||||||
return &JoinCommand{
|
return &JoinCommand{
|
||||||
MinVersion: minVersion,
|
MinVersion: minVersion,
|
||||||
MaxVersion: maxVersion,
|
MaxVersion: maxVersion,
|
||||||
Name: name,
|
Name: name,
|
||||||
RaftURL: raftUrl,
|
RaftURL: raftUrl,
|
||||||
EtcdURL: etcdUrl,
|
EtcdURL: etcdUrl,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1,8 +1,8 @@
|
|||||||
package server
|
package server
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"bytes"
|
|
||||||
"bufio"
|
"bufio"
|
||||||
|
"bytes"
|
||||||
"encoding/binary"
|
"encoding/binary"
|
||||||
"encoding/json"
|
"encoding/json"
|
||||||
"fmt"
|
"fmt"
|
||||||
@ -27,63 +27,64 @@ import (
|
|||||||
|
|
||||||
const ThresholdMonitorTimeout = 5 * time.Second
|
const ThresholdMonitorTimeout = 5 * time.Second
|
||||||
const ActiveMonitorTimeout = 1 * time.Second
|
const ActiveMonitorTimeout = 1 * time.Second
|
||||||
|
const PeerActivityMonitorTimeout = 1 * time.Second
|
||||||
|
|
||||||
type PeerServerConfig struct {
|
type PeerServerConfig struct {
|
||||||
Name string
|
Name string
|
||||||
Scheme string
|
Scheme string
|
||||||
URL string
|
URL string
|
||||||
SnapshotCount int
|
SnapshotCount int
|
||||||
RetryTimes int
|
RetryTimes int
|
||||||
RetryInterval float64
|
RetryInterval float64
|
||||||
}
|
}
|
||||||
|
|
||||||
type PeerServer struct {
|
type PeerServer struct {
|
||||||
Config PeerServerConfig
|
Config PeerServerConfig
|
||||||
clusterConfig *ClusterConfig
|
clusterConfig *ClusterConfig
|
||||||
raftServer raft.Server
|
raftServer raft.Server
|
||||||
server *Server
|
server *Server
|
||||||
joinIndex uint64
|
joinIndex uint64
|
||||||
followersStats *raftFollowersStats
|
followersStats *raftFollowersStats
|
||||||
serverStats *raftServerStats
|
serverStats *raftServerStats
|
||||||
registry *Registry
|
registry *Registry
|
||||||
store store.Store
|
store store.Store
|
||||||
snapConf *snapshotConf
|
snapConf *snapshotConf
|
||||||
mode Mode
|
mode Mode
|
||||||
|
|
||||||
closeChan chan bool
|
closeChan chan bool
|
||||||
timeoutThresholdChan chan interface{}
|
timeoutThresholdChan chan interface{}
|
||||||
|
|
||||||
proxyPeerURL string
|
proxyPeerURL string
|
||||||
proxyClientURL string
|
proxyClientURL string
|
||||||
|
|
||||||
metrics *metrics.Bucket
|
metrics *metrics.Bucket
|
||||||
}
|
}
|
||||||
|
|
||||||
// TODO: find a good policy to do snapshot
|
// TODO: find a good policy to do snapshot
|
||||||
type snapshotConf struct {
|
type snapshotConf struct {
|
||||||
// Etcd will check if snapshot is need every checkingInterval
|
// Etcd will check if snapshot is need every checkingInterval
|
||||||
checkingInterval time.Duration
|
checkingInterval time.Duration
|
||||||
|
|
||||||
// The index when the last snapshot happened
|
// The index when the last snapshot happened
|
||||||
lastIndex uint64
|
lastIndex uint64
|
||||||
|
|
||||||
// If the incremental number of index since the last snapshot
|
// If the incremental number of index since the last snapshot
|
||||||
// exceeds the snapshot Threshold, etcd will do a 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 {
|
func NewPeerServer(psConfig PeerServerConfig, registry *Registry, store store.Store, mb *metrics.Bucket, followersStats *raftFollowersStats, serverStats *raftServerStats) *PeerServer {
|
||||||
s := &PeerServer{
|
s := &PeerServer{
|
||||||
Config: psConfig,
|
Config: psConfig,
|
||||||
clusterConfig: NewClusterConfig(),
|
clusterConfig: NewClusterConfig(),
|
||||||
registry: registry,
|
registry: registry,
|
||||||
store: store,
|
store: store,
|
||||||
followersStats: followersStats,
|
followersStats: followersStats,
|
||||||
serverStats: serverStats,
|
serverStats: serverStats,
|
||||||
|
|
||||||
timeoutThresholdChan: make(chan interface{}, 1),
|
timeoutThresholdChan: make(chan interface{}, 1),
|
||||||
|
|
||||||
metrics: mb,
|
metrics: mb,
|
||||||
}
|
}
|
||||||
|
|
||||||
return s
|
return s
|
||||||
@ -91,10 +92,10 @@ func NewPeerServer(psConfig PeerServerConfig, registry *Registry, store store.St
|
|||||||
|
|
||||||
func (s *PeerServer) SetRaftServer(raftServer raft.Server) {
|
func (s *PeerServer) SetRaftServer(raftServer raft.Server) {
|
||||||
s.snapConf = &snapshotConf{
|
s.snapConf = &snapshotConf{
|
||||||
checkingInterval: time.Second * 3,
|
checkingInterval: time.Second * 3,
|
||||||
// this is not accurate, we will update raft to provide an api
|
// this is not accurate, we will update raft to provide an api
|
||||||
lastIndex: raftServer.CommitIndex(),
|
lastIndex: raftServer.CommitIndex(),
|
||||||
snapshotThr: uint64(s.Config.SnapshotCount),
|
snapshotThr: uint64(s.Config.SnapshotCount),
|
||||||
}
|
}
|
||||||
|
|
||||||
raftServer.AddEventListener(raft.StateChangeEventType, s.raftEventLogger)
|
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.monitorSync()
|
||||||
go s.monitorTimeoutThreshold(s.closeChan)
|
go s.monitorTimeoutThreshold(s.closeChan)
|
||||||
go s.monitorActive(s.closeChan)
|
go s.monitorActive(s.closeChan)
|
||||||
|
go s.monitorPeerActivity(s.closeChan)
|
||||||
|
|
||||||
// open the snapshot
|
// open the snapshot
|
||||||
if snapshot {
|
if snapshot {
|
||||||
@ -444,7 +446,7 @@ func (s *PeerServer) joinByPeer(server raft.Server, peer string, scheme string)
|
|||||||
if resp.StatusCode == http.StatusOK {
|
if resp.StatusCode == http.StatusOK {
|
||||||
r := bufio.NewReader(resp.Body)
|
r := bufio.NewReader(resp.Body)
|
||||||
s.joinIndex, _ = binary.ReadUvarint(r)
|
s.joinIndex, _ = binary.ReadUvarint(r)
|
||||||
|
|
||||||
// Determine whether the server joined as a proxy or peer.
|
// Determine whether the server joined as a proxy or peer.
|
||||||
var mode uint64
|
var mode uint64
|
||||||
if mode, err = binary.ReadUvarint(r); err == io.EOF {
|
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) {
|
func (s *PeerServer) monitorActive(closeChan chan bool) {
|
||||||
for {
|
for {
|
||||||
select {
|
select {
|
||||||
case <- time.After(ActiveMonitorTimeout):
|
case <-time.After(ActiveMonitorTimeout):
|
||||||
case <-closeChan:
|
case <-closeChan:
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
@ -632,18 +634,13 @@ func (s *PeerServer) monitorActive(closeChan chan bool) {
|
|||||||
peerCount := s.registry.PeerCount()
|
peerCount := s.registry.PeerCount()
|
||||||
proxies := s.registry.Proxies()
|
proxies := s.registry.Proxies()
|
||||||
peers := s.registry.Peers()
|
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 {
|
if index := sort.SearchStrings(peers, s.Config.Name); index < len(peers) && peers[index] == s.Config.Name {
|
||||||
peers = append(peers[:index], peers[index+1:]...)
|
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 we have more active nodes than we should then demote.
|
||||||
if peerCount > activeSize {
|
if peerCount > activeSize {
|
||||||
peer := peers[rand.Intn(len(peers))]
|
peer := peers[rand.Intn(len(peers))]
|
||||||
fmt.Println("active.demote»", peer)
|
|
||||||
if _, err := s.raftServer.Do(&DemoteCommand{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)
|
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 we don't have enough active nodes then try to promote a proxy.
|
||||||
if peerCount < activeSize && len(proxies) > 0 {
|
if peerCount < activeSize && len(proxies) > 0 {
|
||||||
proxy := proxies[rand.Intn(len(proxies))]
|
loop:
|
||||||
proxyPeerURL, _ := s.registry.ProxyPeerURL(proxy)
|
for _, i := range rand.Perm(len(proxies)) {
|
||||||
log.Infof("%s: promoting: %v (%s)", s.Config.Name, proxy, proxyPeerURL)
|
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.
|
// Notify proxy to promote itself.
|
||||||
client := &http.Client{
|
client := &http.Client{
|
||||||
Transport: &http.Transport{
|
Transport: &http.Transport{
|
||||||
DisableKeepAlives: false,
|
DisableKeepAlives: false,
|
||||||
ResponseHeaderTimeout: ActiveMonitorTimeout,
|
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
|
// Mode represents whether the server is an active peer or if the server is
|
||||||
// simply acting as a proxy.
|
// simply acting as a proxy.
|
||||||
@ -681,9 +714,8 @@ type Mode string
|
|||||||
|
|
||||||
const (
|
const (
|
||||||
// PeerMode is when the server is an active node in Raft.
|
// 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 is when the server is an inactive, request-forwarding node.
|
||||||
ProxyMode = Mode("proxy")
|
ProxyMode = Mode("proxy")
|
||||||
)
|
)
|
||||||
|
|
||||||
|
@ -11,8 +11,8 @@ import (
|
|||||||
"github.com/coreos/etcd/third_party/github.com/gorilla/mux"
|
"github.com/coreos/etcd/third_party/github.com/gorilla/mux"
|
||||||
|
|
||||||
etcdErr "github.com/coreos/etcd/error"
|
etcdErr "github.com/coreos/etcd/error"
|
||||||
uhttp "github.com/coreos/etcd/pkg/http"
|
|
||||||
"github.com/coreos/etcd/log"
|
"github.com/coreos/etcd/log"
|
||||||
|
uhttp "github.com/coreos/etcd/pkg/http"
|
||||||
"github.com/coreos/etcd/store"
|
"github.com/coreos/etcd/store"
|
||||||
)
|
)
|
||||||
|
|
||||||
@ -215,7 +215,7 @@ func (ps *PeerServer) getClusterConfigHttpHandler(w http.ResponseWriter, req *ht
|
|||||||
|
|
||||||
// Updates the cluster configuration.
|
// Updates the cluster configuration.
|
||||||
func (ps *PeerServer) setClusterConfigHttpHandler(w http.ResponseWriter, req *http.Request) {
|
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 {
|
if err := json.NewDecoder(req.Body).Decode(&c.Config); err != nil {
|
||||||
http.Error(w, err.Error(), http.StatusInternalServerError)
|
http.Error(w, err.Error(), http.StatusInternalServerError)
|
||||||
return
|
return
|
||||||
|
@ -7,32 +7,32 @@ import (
|
|||||||
)
|
)
|
||||||
|
|
||||||
type raftServerStats struct {
|
type raftServerStats struct {
|
||||||
Name string `json:"name"`
|
Name string `json:"name"`
|
||||||
State string `json:"state"`
|
State string `json:"state"`
|
||||||
StartTime time.Time `json:"startTime"`
|
StartTime time.Time `json:"startTime"`
|
||||||
|
|
||||||
LeaderInfo struct {
|
LeaderInfo struct {
|
||||||
Name string `json:"leader"`
|
Name string `json:"leader"`
|
||||||
Uptime string `json:"uptime"`
|
Uptime string `json:"uptime"`
|
||||||
startTime time.Time
|
startTime time.Time
|
||||||
} `json:"leaderInfo"`
|
} `json:"leaderInfo"`
|
||||||
|
|
||||||
RecvAppendRequestCnt uint64 `json:"recvAppendRequestCnt,"`
|
RecvAppendRequestCnt uint64 `json:"recvAppendRequestCnt,"`
|
||||||
RecvingPkgRate float64 `json:"recvPkgRate,omitempty"`
|
RecvingPkgRate float64 `json:"recvPkgRate,omitempty"`
|
||||||
RecvingBandwidthRate float64 `json:"recvBandwidthRate,omitempty"`
|
RecvingBandwidthRate float64 `json:"recvBandwidthRate,omitempty"`
|
||||||
|
|
||||||
SendAppendRequestCnt uint64 `json:"sendAppendRequestCnt"`
|
SendAppendRequestCnt uint64 `json:"sendAppendRequestCnt"`
|
||||||
SendingPkgRate float64 `json:"sendPkgRate,omitempty"`
|
SendingPkgRate float64 `json:"sendPkgRate,omitempty"`
|
||||||
SendingBandwidthRate float64 `json:"sendBandwidthRate,omitempty"`
|
SendingBandwidthRate float64 `json:"sendBandwidthRate,omitempty"`
|
||||||
|
|
||||||
sendRateQueue *statsQueue
|
sendRateQueue *statsQueue
|
||||||
recvRateQueue *statsQueue
|
recvRateQueue *statsQueue
|
||||||
}
|
}
|
||||||
|
|
||||||
func NewRaftServerStats(name string) *raftServerStats {
|
func NewRaftServerStats(name string) *raftServerStats {
|
||||||
return &raftServerStats{
|
return &raftServerStats{
|
||||||
Name: name,
|
Name: name,
|
||||||
StartTime: time.Now(),
|
StartTime: time.Now(),
|
||||||
sendRateQueue: &statsQueue{
|
sendRateQueue: &statsQueue{
|
||||||
back: -1,
|
back: -1,
|
||||||
},
|
},
|
||||||
|
@ -22,8 +22,8 @@ const RegistryProxyKey = "/_etcd/proxies"
|
|||||||
// The Registry stores URL information for nodes.
|
// The Registry stores URL information for nodes.
|
||||||
type Registry struct {
|
type Registry struct {
|
||||||
sync.Mutex
|
sync.Mutex
|
||||||
store store.Store
|
store store.Store
|
||||||
peers map[string]*node
|
peers map[string]*node
|
||||||
proxies map[string]*node
|
proxies map[string]*node
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -37,13 +37,13 @@ type node struct {
|
|||||||
// Creates a new Registry.
|
// Creates a new Registry.
|
||||||
func NewRegistry(s store.Store) *Registry {
|
func NewRegistry(s store.Store) *Registry {
|
||||||
return &Registry{
|
return &Registry{
|
||||||
store: s,
|
store: s,
|
||||||
peers: make(map[string]*node),
|
peers: make(map[string]*node),
|
||||||
proxies: 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 {
|
func (r *Registry) Peers() []string {
|
||||||
names := make([]string, 0, len(r.peers))
|
names := make([]string, 0, len(r.peers))
|
||||||
for name, _ := range r.peers {
|
for name, _ := range r.peers {
|
||||||
@ -53,7 +53,7 @@ func (r *Registry) Peers() []string {
|
|||||||
return names
|
return names
|
||||||
}
|
}
|
||||||
|
|
||||||
// Proxies returns a list of proxy names.
|
// Proxies returns a list of cached proxy names.
|
||||||
func (r *Registry) Proxies() []string {
|
func (r *Registry) Proxies() []string {
|
||||||
names := make([]string, 0, len(r.proxies))
|
names := make([]string, 0, len(r.proxies))
|
||||||
for name, _ := range r.proxies {
|
for name, _ := range r.proxies {
|
||||||
@ -63,7 +63,6 @@ func (r *Registry) Proxies() []string {
|
|||||||
return names
|
return names
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
// RegisterPeer adds a peer to the registry.
|
// RegisterPeer adds a peer to the registry.
|
||||||
func (r *Registry) RegisterPeer(name string, peerURL string, machURL string) error {
|
func (r *Registry) RegisterPeer(name string, peerURL string, machURL string) error {
|
||||||
// TODO(benbjohnson): Disallow peers that are already proxies.
|
// TODO(benbjohnson): Disallow peers that are already proxies.
|
||||||
@ -150,7 +149,6 @@ func (r *Registry) exists(key, name string) bool {
|
|||||||
return (e.Node != nil)
|
return (e.Node != nil)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
// Retrieves the client URL for a given node by name.
|
// Retrieves the client URL for a given node by name.
|
||||||
func (r *Registry) ClientURL(name string) (string, bool) {
|
func (r *Registry) ClientURL(name string) (string, bool) {
|
||||||
r.Lock()
|
r.Lock()
|
||||||
@ -188,7 +186,7 @@ func (r *Registry) PeerHost(name string) (string, bool) {
|
|||||||
func (r *Registry) PeerURL(name string) (string, bool) {
|
func (r *Registry) PeerURL(name string) (string, bool) {
|
||||||
r.Lock()
|
r.Lock()
|
||||||
defer r.Unlock()
|
defer r.Unlock()
|
||||||
return r.peerURL(RegistryPeerKey,name)
|
return r.peerURL(RegistryPeerKey, name)
|
||||||
}
|
}
|
||||||
|
|
||||||
func (r *Registry) peerURL(key, name string) (string, bool) {
|
func (r *Registry) peerURL(key, name string) (string, bool) {
|
||||||
|
@ -12,10 +12,10 @@ import (
|
|||||||
"github.com/coreos/etcd/third_party/github.com/gorilla/mux"
|
"github.com/coreos/etcd/third_party/github.com/gorilla/mux"
|
||||||
|
|
||||||
etcdErr "github.com/coreos/etcd/error"
|
etcdErr "github.com/coreos/etcd/error"
|
||||||
|
ehttp "github.com/coreos/etcd/http"
|
||||||
"github.com/coreos/etcd/log"
|
"github.com/coreos/etcd/log"
|
||||||
"github.com/coreos/etcd/metrics"
|
"github.com/coreos/etcd/metrics"
|
||||||
"github.com/coreos/etcd/mod"
|
"github.com/coreos/etcd/mod"
|
||||||
ehttp "github.com/coreos/etcd/http"
|
|
||||||
uhttp "github.com/coreos/etcd/pkg/http"
|
uhttp "github.com/coreos/etcd/pkg/http"
|
||||||
"github.com/coreos/etcd/server/v1"
|
"github.com/coreos/etcd/server/v1"
|
||||||
"github.com/coreos/etcd/server/v2"
|
"github.com/coreos/etcd/server/v2"
|
||||||
@ -25,26 +25,26 @@ import (
|
|||||||
|
|
||||||
// This is the default implementation of the Server interface.
|
// This is the default implementation of the Server interface.
|
||||||
type Server struct {
|
type Server struct {
|
||||||
Name string
|
Name string
|
||||||
url string
|
url string
|
||||||
handler http.Handler
|
handler http.Handler
|
||||||
peerServer *PeerServer
|
peerServer *PeerServer
|
||||||
registry *Registry
|
registry *Registry
|
||||||
store store.Store
|
store store.Store
|
||||||
metrics *metrics.Bucket
|
metrics *metrics.Bucket
|
||||||
|
|
||||||
trace bool
|
trace bool
|
||||||
}
|
}
|
||||||
|
|
||||||
// Creates a new Server.
|
// Creates a new Server.
|
||||||
func New(name, url string, peerServer *PeerServer, registry *Registry, store store.Store, mb *metrics.Bucket) *Server {
|
func New(name, url string, peerServer *PeerServer, registry *Registry, store store.Store, mb *metrics.Bucket) *Server {
|
||||||
s := &Server{
|
s := &Server{
|
||||||
Name: name,
|
Name: name,
|
||||||
url: url,
|
url: url,
|
||||||
store: store,
|
store: store,
|
||||||
registry: registry,
|
registry: registry,
|
||||||
peerServer: peerServer,
|
peerServer: peerServer,
|
||||||
metrics: mb,
|
metrics: mb,
|
||||||
}
|
}
|
||||||
|
|
||||||
return s
|
return s
|
||||||
|
@ -15,13 +15,13 @@ import (
|
|||||||
|
|
||||||
// Transporter layer for communication between raft nodes
|
// Transporter layer for communication between raft nodes
|
||||||
type transporter struct {
|
type transporter struct {
|
||||||
requestTimeout time.Duration
|
requestTimeout time.Duration
|
||||||
followersStats *raftFollowersStats
|
followersStats *raftFollowersStats
|
||||||
serverStats *raftServerStats
|
serverStats *raftServerStats
|
||||||
registry *Registry
|
registry *Registry
|
||||||
|
|
||||||
client *http.Client
|
client *http.Client
|
||||||
transport *http.Transport
|
transport *http.Transport
|
||||||
}
|
}
|
||||||
|
|
||||||
type dialer func(network, addr string) (net.Conn, error)
|
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) {
|
Dial: func(network, addr string) (net.Conn, error) {
|
||||||
return net.DialTimeout(network, addr, dialTimeout)
|
return net.DialTimeout(network, addr, dialTimeout)
|
||||||
},
|
},
|
||||||
ResponseHeaderTimeout: responseHeaderTimeout,
|
ResponseHeaderTimeout: responseHeaderTimeout,
|
||||||
}
|
}
|
||||||
|
|
||||||
t := transporter{
|
t := transporter{
|
||||||
client: &http.Client{Transport: tr},
|
client: &http.Client{Transport: tr},
|
||||||
transport: tr,
|
transport: tr,
|
||||||
requestTimeout: requestTimeout,
|
requestTimeout: requestTimeout,
|
||||||
followersStats: followersStats,
|
followersStats: followersStats,
|
||||||
serverStats: serverStats,
|
serverStats: serverStats,
|
||||||
registry: registry,
|
registry: registry,
|
||||||
}
|
}
|
||||||
|
|
||||||
return &t
|
return &t
|
||||||
@ -73,7 +73,7 @@ func (t *transporter) SendAppendEntriesRequest(server raft.Server, peer *raft.Pe
|
|||||||
|
|
||||||
thisFollowerStats, ok := t.followersStats.Followers[peer.Name]
|
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 = &raftFollowerStats{}
|
||||||
thisFollowerStats.Latency.Minimum = 1 << 63
|
thisFollowerStats.Latency.Minimum = 1 << 63
|
||||||
t.followersStats.Followers[peer.Name] = thisFollowerStats
|
t.followersStats.Followers[peer.Name] = thisFollowerStats
|
||||||
|
@ -82,3 +82,63 @@ func TestProxy(t *testing.T) {
|
|||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
assert.Equal(t, len(result.Node.Nodes), 2)
|
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")
|
||||||
|
}
|
||||||
|
}
|
||||||
|
11
third_party/github.com/coreos/raft/peer.go
vendored
11
third_party/github.com/coreos/raft/peer.go
vendored
@ -20,6 +20,7 @@ type Peer struct {
|
|||||||
mutex sync.RWMutex
|
mutex sync.RWMutex
|
||||||
stopChan chan bool
|
stopChan chan bool
|
||||||
heartbeatInterval time.Duration
|
heartbeatInterval time.Duration
|
||||||
|
lastActivity time.Time
|
||||||
}
|
}
|
||||||
|
|
||||||
//------------------------------------------------------------------------------
|
//------------------------------------------------------------------------------
|
||||||
@ -67,6 +68,11 @@ func (p *Peer) setPrevLogIndex(value uint64) {
|
|||||||
p.prevLogIndex = value
|
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
|
// Methods
|
||||||
@ -103,6 +109,7 @@ func (p *Peer) clone() *Peer {
|
|||||||
Name: p.Name,
|
Name: p.Name,
|
||||||
ConnectionString: p.ConnectionString,
|
ConnectionString: p.ConnectionString,
|
||||||
prevLogIndex: p.prevLogIndex,
|
prevLogIndex: p.prevLogIndex,
|
||||||
|
lastActivity: p.lastActivity,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -176,6 +183,7 @@ func (p *Peer) sendAppendEntriesRequest(req *AppendEntriesRequest) {
|
|||||||
|
|
||||||
// If successful then update the previous log index.
|
// If successful then update the previous log index.
|
||||||
p.mutex.Lock()
|
p.mutex.Lock()
|
||||||
|
p.lastActivity = time.Now()
|
||||||
if resp.Success() {
|
if resp.Success() {
|
||||||
if len(req.Entries) > 0 {
|
if len(req.Entries) > 0 {
|
||||||
p.prevLogIndex = req.Entries[len(req.Entries)-1].GetIndex()
|
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
|
// If successful, the peer should have been to snapshot state
|
||||||
// Send it the snapshot!
|
// Send it the snapshot!
|
||||||
|
p.lastActivity = time.Now()
|
||||||
if resp.Success {
|
if resp.Success {
|
||||||
p.sendSnapshotRecoveryRequest()
|
p.sendSnapshotRecoveryRequest()
|
||||||
} else {
|
} else {
|
||||||
@ -263,6 +272,7 @@ func (p *Peer) sendSnapshotRecoveryRequest() {
|
|||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
|
p.lastActivity = time.Now()
|
||||||
if resp.Success {
|
if resp.Success {
|
||||||
p.prevLogIndex = req.LastIndex
|
p.prevLogIndex = req.LastIndex
|
||||||
} else {
|
} else {
|
||||||
@ -283,6 +293,7 @@ func (p *Peer) sendVoteRequest(req *RequestVoteRequest, c chan *RequestVoteRespo
|
|||||||
req.peer = p
|
req.peer = p
|
||||||
if resp := p.server.Transporter().SendVoteRequest(p.server, p, req); resp != nil {
|
if resp := p.server.Transporter().SendVoteRequest(p.server, p, req); resp != nil {
|
||||||
debugln("peer.vote.recv: ", p.server.Name(), "<-", p.Name)
|
debugln("peer.vote.recv: ", p.server.Name(), "<-", p.Name)
|
||||||
|
p.lastActivity = time.Now()
|
||||||
resp.peer = p
|
resp.peer = p
|
||||||
c <- resp
|
c <- resp
|
||||||
} else {
|
} else {
|
||||||
|
Reference in New Issue
Block a user