Skip to content

Commit

Permalink
Merge pull request etcd-io#582 from benbjohnson/proxy
Browse files Browse the repository at this point in the history
Proxies & Config API
  • Loading branch information
benbjohnson committed Mar 24, 2014
2 parents aeb9e06 + 62b89a1 commit 802aaf5
Show file tree
Hide file tree
Showing 107 changed files with 2,456 additions and 1,439 deletions.
74 changes: 74 additions & 0 deletions Documentation/design/proxies.md
Original file line number Diff line number Diff line change
@@ -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
```
4 changes: 0 additions & 4 deletions config/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"`
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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, "")

Expand Down Expand Up @@ -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
Expand Down
37 changes: 0 additions & 37 deletions config/config_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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", "")
Expand Down Expand Up @@ -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", "")
Expand Down Expand Up @@ -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) {
Expand Down Expand Up @@ -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()
Expand Down
12 changes: 10 additions & 2 deletions error/error.go
Original file line number Diff line number Diff line change
Expand Up @@ -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() {
Expand Down Expand Up @@ -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"

}

Expand Down
15 changes: 7 additions & 8 deletions etcd.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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)

Expand Down
18 changes: 15 additions & 3 deletions scripts/test-cluster
Original file line number Diff line number Diff line change
@@ -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
Expand All @@ -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"
Expand All @@ -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
Expand Down
39 changes: 39 additions & 0 deletions server/cluster_config.go
Original file line number Diff line number Diff line change
@@ -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 changes: 70 additions & 0 deletions server/demote_command.go
Original file line number Diff line number Diff line change
@@ -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
}
Loading

0 comments on commit 802aaf5

Please sign in to comment.