Merge pull request #582 from benbjohnson/proxy

Proxies & Config API
release-0.4
Ben Johnson 2014-03-24 16:30:47 -06:00
commit 802aaf59db
107 changed files with 2460 additions and 1443 deletions

View File

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

View File

@ -61,7 +61,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"`
@ -91,7 +90,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
@ -248,7 +246,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.HeartbeatInterval, "peer-heartbeat-interval", c.Peer.HeartbeatInterval, "")
f.IntVar(&c.Peer.ElectionTimeout, "peer-election-timeout", c.Peer.ElectionTimeout, "")
@ -282,7 +279,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)")
f.IntVar(&c.Peer.HeartbeatInterval, "peer-heartbeat-timeout", c.Peer.HeartbeatInterval, "(deprecated)")
// END DEPRECATED FLAGS

View File

@ -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()

View File

@ -49,8 +49,12 @@ const (
EcodeRaftInternal = 300
EcodeLeaderElect = 301
EcodeWatcherCleared = 400
EcodeEventIndexCleared = 401
EcodeWatcherCleared = 400
EcodeEventIndexCleared = 401
EcodeProxyInternal = 402
EcodeInvalidActiveSize = 403
EcodeInvalidPromoteDelay = 404
EcodePromoteError = 405
)
func init() {
@ -86,6 +90,10 @@ 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"
errors[EcodePromoteError] = "Proxy promotion error"
}

15
etcd.go
View File

@ -24,7 +24,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"
@ -115,13 +115,12 @@ func main() {
// Create peer server
psConfig := server.PeerServerConfig{
Name: config.Name,
Scheme: config.PeerTLSInfo().Scheme(),
URL: config.Peer.Addr,
SnapshotCount: config.SnapshotCount,
MaxClusterSize: config.MaxClusterSize,
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)

View File

@ -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/v2/admin/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

39
server/cluster_config.go Normal file
View File

@ -0,0 +1,39 @@
package server
import (
"time"
)
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.
// 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,
}
}

70
server/demote_command.go Normal file
View File

@ -0,0 +1,70 @@
package server
import (
"fmt"
"github.com/coreos/etcd/log"
"github.com/coreos/etcd/third_party/github.com/goraft/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)
// 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)
// 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("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
}
// NodeName returns the name of the affected node.
func (c *DemoteCommand) NodeName() string {
return c.Name
}

View File

@ -2,42 +2,35 @@ package server
import (
"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"
"github.com/coreos/etcd/third_party/github.com/goraft/raft"
)
func init() {
raft.RegisterCommand(&JoinCommand{})
raft.RegisterCommand(&JoinCommandV1{})
raft.RegisterCommand(&JoinCommandV2{})
}
// The JoinCommand adds a node to the cluster.
type JoinCommand struct {
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,
}
// 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"`
RaftURL string `json:"raftURL"`
EtcdURL string `json:"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)
b := make([]byte, 8)
@ -52,13 +45,13 @@ func (c *JoinCommand) Apply(context raft.Context) (interface{}, error) {
}
// Check peer number in the cluster
if ps.registry.Count() == ps.Config.MaxClusterSize {
if ps.registry.PeerCount() >= ps.ClusterConfig().ActiveSize {
log.Debug("Reject join request from ", c.Name)
return []byte{0}, etcdErr.NewError(etcdErr.EcodeNoMorePeer, "", context.CommitIndex())
}
// 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, "")
@ -72,6 +65,76 @@ func (c *JoinCommand) Apply(context raft.Context) (interface{}, error) {
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"`
}

View File

@ -2,16 +2,17 @@ package server
import (
"bytes"
"encoding/binary"
"encoding/json"
"fmt"
"io/ioutil"
"math/rand"
"net/http"
"net/url"
"sort"
"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"
@ -21,59 +22,82 @@ import (
"github.com/coreos/etcd/store"
)
const ThresholdMonitorTimeout = 5 * 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
Scheme string
URL string
SnapshotCount int
MaxClusterSize int
RetryTimes int
RetryInterval float64
Name string
Scheme string
URL string
SnapshotCount int
RetryTimes int
RetryInterval float64
}
type PeerServer struct {
Config PeerServerConfig
raftServer raft.Server
server *Server
joinIndex uint64
followersStats *raftFollowersStats
serverStats *raftServerStats
registry *Registry
store store.Store
snapConf *snapshotConf
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{}
metrics *metrics.Bucket
proxyPeerURL string
proxyClientURL string
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,
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
@ -81,10 +105,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)
@ -100,6 +124,49 @@ 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 cause the PeerServer to demote peers or
// promote proxies to match the new size.
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
}
// 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)
@ -226,6 +293,8 @@ func (s *PeerServer) Start(snapshot bool, discoverURL string, peers []string) er
go s.monitorSync()
go s.monitorTimeoutThreshold(s.closeChan)
go s.monitorActiveSize(s.closeChan)
go s.monitorPeerActivity(s.closeChan)
// open the snapshot
if snapshot {
@ -252,6 +321,7 @@ 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("/vote", s.VoteHttpHandler)
router.HandleFunc("/log", s.GetLogHttpHandler)
@ -260,6 +330,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
}
@ -276,7 +353,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
}
@ -364,8 +448,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)
@ -379,13 +461,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()))
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: "/join"}
joinURL := url.URL{Host: peer, Scheme: scheme, Path: "/v2/admin/machines/" + server.Name()}
log.Infof("Send Join Request to %s", joinURL.String())
log.Debugf("Send Join Request to %s", joinURL.String())
resp, _, err := t.Post(joinURL.String(), &b)
req, _ := http.NewRequest("PUT", joinURL.String(), &b)
resp, err := t.client.Do(req)
for {
if err != nil {
@ -394,15 +484,34 @@ func (s *PeerServer) joinByPeer(server raft.Server, peer string, scheme string)
if resp != nil {
defer resp.Body.Close()
log.Infof("»»»» %d", resp.StatusCode)
if resp.StatusCode == http.StatusOK {
b, _ := ioutil.ReadAll(resp.Body)
s.joinIndex, _ = binary.Uvarint(b)
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)
if msg.Mode == ProxyMode {
s.proxyClientURL = resp.Header.Get("X-Leader-Client-URL")
s.proxyPeerURL = resp.Header.Get("X-Leader-Peer-URL")
}
return nil
}
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, _, err = t.Post(address, &b)
} else if resp.StatusCode == http.StatusBadRequest {
@ -542,3 +651,110 @@ func (s *PeerServer) monitorTimeoutThreshold(closeChan chan bool) {
time.Sleep(ThresholdMonitorTimeout)
}
}
// 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):
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))]
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)
}
continue
}
// If we don't have enough active nodes then try to promote a proxy.
if peerCount < activeSize && len(proxies) > 0 {
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,
},
}
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
}
}
}
}
// monitorPeerActivity has the leader periodically 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.
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")
)

View File

@ -3,15 +3,16 @@ package server
import (
"encoding/json"
"net/http"
"net/url"
"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"
uhttp "github.com/coreos/etcd/pkg/http"
"github.com/coreos/etcd/log"
uhttp "github.com/coreos/etcd/pkg/http"
"github.com/coreos/etcd/store"
)
@ -149,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 {
@ -171,6 +170,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" {
@ -179,7 +197,7 @@ func (ps *PeerServer) RemoveHttpHandler(w http.ResponseWriter, req *http.Request
}
vars := mux.Vars(req)
command := &RemoveCommand{
command := &RemoveCommandV1{
Name: vars["name"],
}
@ -188,6 +206,111 @@ 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) {
// 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)
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
}
// 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)
@ -233,3 +356,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"`
}

View File

@ -3,36 +3,36 @@ 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 {
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,
},

View File

@ -5,6 +5,7 @@ import (
"net/url"
"path"
"path/filepath"
"sort"
"strings"
"sync"
@ -13,13 +14,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
store store.Store
peers map[string]*node
proxies map[string]*node
}
// The internal storage format of the registry.
@ -32,62 +37,135 @@ type node struct {
// Creates a new Registry.
func NewRegistry(s store.Store) *Registry {
return &Registry{
store: s,
nodes: make(map[string]*node),
store: s,
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 cached peer names.
func (r *Registry) Peers() []string {
names := make([]string, 0, len(r.peers))
for name, _ := range r.peers {
names = append(names, name)
}
sort.Sort(sort.StringSlice(names))
return 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 {
names = append(names, name)
}
sort.Sort(sort.StringSlice(names))
return names
}
// RegisterPeer adds a peer to the registry.
func (r *Registry) RegisterPeer(name string, peerURL string, machURL string) error {
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 {
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 {
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 {
if node := r.load(key, name); node != nil {
r.peers[name] = node
}
}
if node := r.nodes[name]; node != nil {
if node := r.peers[name]; node != nil {
return node.url, true
}
@ -110,73 +188,114 @@ 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 {
if node := r.load(key, name); node != nil {
r.peers[name] = node
}
}
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 {
if node := r.load(key, name); node != nil {
r.proxies[name] = node
}
}
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 {
if node := r.load(key, name); node != nil {
r.proxies[name] = node
}
}
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 (%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(key, name), false, false)
if err != nil {
return
return nil
}
// Parse as a query string.
@ -186,7 +305,7 @@ func (r *Registry) load(name string) {
}
// Create node.
r.nodes[name] = &node{
return &node{
url: m["etcd"][0],
peerURL: m["raft"][0],
}

View File

@ -2,32 +2,39 @@ package server
import (
"encoding/binary"
"encoding/json"
"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() {
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.
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)
@ -65,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"`
}

View File

@ -8,14 +8,14 @@ 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"
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
@ -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,26 @@ 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 *JoinCommandV1, *RemoveCommandV1:
url, _ = ps.registry.PeerURL(leader)
case *JoinCommandV2, *RemoveCommandV2:
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.

View File

@ -0,0 +1,26 @@
package server
import (
"github.com/coreos/etcd/third_party/github.com/goraft/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)
ps.SetClusterConfig(c.Config)
return nil, nil
}

View File

@ -9,10 +9,9 @@ import (
"net/http"
"time"
"github.com/coreos/etcd/third_party/github.com/coreos/raft"
httpclient "github.com/coreos/etcd/third_party/github.com/mreiferson/go-httpclient"
"github.com/coreos/etcd/log"
"github.com/coreos/etcd/third_party/github.com/goraft/raft"
httpclient "github.com/coreos/etcd/third_party/github.com/mreiferson/go-httpclient"
)
const (

View File

@ -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"
)

View File

@ -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"
)

View File

@ -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"
)
@ -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)
}

View File

@ -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"
)

View File

@ -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"
)

View File

@ -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.

View File

@ -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,
}
}

View File

@ -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() {

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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() {

View File

@ -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

View File

@ -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/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/v2/admin/config")
body := tests.ReadBodyJSON(resp)
assert.Equal(t, resp.StatusCode, 200)
assert.Equal(t, body["activeSize"], 3)
assert.Equal(t, body["promoteDelay"], 60)
}

View File

@ -0,0 +1,144 @@
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"
)
// Create a full cluster and then add extra an extra proxy node.
func TestProxy(t *testing.T) {
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)
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")
}
}
}
// 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/v2/admin/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.
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/v2/admin/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)
}
// 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 (2 second).
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()
}
// 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 + (2 * time.Second))
// Wait for the proxy to be promoted.
time.Sleep(server.ActiveMonitorTimeout + (2 * 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")
}
}

View File

@ -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++ {

View File

@ -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...)
}

View File

@ -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())

View File

@ -30,7 +30,7 @@ func ReadBodyJSON(resp *http.Response) map[string]interface{} {
m := make(map[string]interface{})
b := ReadBody(resp)
if err := json.Unmarshal(b, &m); err != nil {
panic(fmt.Sprintf("HTTP body JSON parse error: %v", err))
panic(fmt.Sprintf("HTTP body JSON parse error: %v: %s", err, string(b)))
}
return m
}

View File

@ -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 {

View File

@ -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"
@ -35,11 +35,10 @@ func RunServer(f func(*server.Server)) {
followersStats := server.NewRaftFollowersStats(testName)
psConfig := server.PeerServerConfig{
Name: testName,
URL: "http://" + testRaftURL,
Scheme: "http",
SnapshotCount: testSnapshotCount,
MaxClusterSize: 9,
Name: testName,
URL: "http://" + testRaftURL,
Scheme: "http",
SnapshotCount: testSnapshotCount,
}
mb := metrics.NewBucket("")

View File

@ -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"
)

View File

@ -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"

View File

@ -1,10 +0,0 @@
language: go
go:
- 1.1.2
- 1.2
install:
- go get github.com/stretchr/testify/assert
- make dependencies

View File

@ -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
}

View File

@ -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
}

View File

@ -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()
}

View File

@ -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
}

View File

@ -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
}

View File

@ -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
}

View File

@ -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
}

View File

@ -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
}

View File

@ -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
}

View File

@ -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
}

View File

@ -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
}

View File

@ -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
}

View File

@ -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
}

View File

@ -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] }

View File

@ -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)
}

View File

@ -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()
}

View File

@ -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.

View File

@ -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
}

View File

@ -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()
}

View File

@ -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
}

View File

@ -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"
)
//------------------------------------------------------------------------------

View File

@ -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

View File

@ -20,6 +20,7 @@ type Peer struct {
mutex sync.RWMutex
stopChan chan bool
heartbeatInterval time.Duration
lastActivity time.Time
}
//------------------------------------------------------------------------------
@ -90,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
//--------------------------------------
@ -103,6 +109,7 @@ func (p *Peer) clone() *Peer {
Name: p.Name,
ConnectionString: p.ConnectionString,
prevLogIndex: p.prevLogIndex,
lastActivity: p.lastActivity,
}
}
@ -153,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))
}
}
@ -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 {
@ -254,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)
@ -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 {

View File

@ -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 {

View File

@ -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 {

View File

@ -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 {

View File

@ -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 {

View File

@ -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 {

View File

@ -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 {

View File

@ -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 {

View File

@ -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 {

View File

@ -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 {

View File

@ -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
}

View File

@ -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,12 +462,51 @@ 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()
}
@ -483,55 +514,50 @@ func (s *server) Stop() {
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))
}
}
//--------------------------------------
@ -554,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
}
}
}
@ -609,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)
@ -618,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)
@ -674,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 = ""
@ -683,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.
@ -777,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.
@ -807,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
@ -827,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
}
}
@ -891,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 {
@ -923,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
}
@ -963,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
//--------------------------------------
@ -985,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
@ -1056,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)
@ -1075,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.
@ -1108,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.
@ -1125,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
}
@ -1183,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.
@ -1197,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
@ -1212,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
}
@ -1224,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.
@ -1244,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.
@ -1261,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
}
@ -1289,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))

View File

@ -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
// //--------------------------------------

View File

@ -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
}

Some files were not shown because too many files have changed in this diff Show More