mirror of
https://github.com/etcd-io/etcd.git
synced 2024-09-27 06:25:44 +00:00
Merge pull request #786 from unihorn/91
feat(standby_server): write cluster info to disk
This commit is contained in:
commit
1084e51320
@ -33,6 +33,9 @@ After each interval, standbys synchronize information with cluster.
|
||||
#### Main logic
|
||||
|
||||
```
|
||||
If find existing standby cluster info:
|
||||
Goto standby loop
|
||||
|
||||
Find cluster as required
|
||||
If determine to start peer server:
|
||||
Goto peer loop
|
||||
@ -74,7 +77,6 @@ return true
|
||||
**Note**
|
||||
1. [TODO] The running mode cannot be determined by log, because the log may be outdated. But the log could be used to estimate its state.
|
||||
2. Even if sync cluster fails, it will restart still for recovery from full outage.
|
||||
3. [BUG] Possible peers from discover URL, peers flag and data dir could be outdated because standby machine doesn't record log. This could make reconnect fail if the whole cluster migrates to new address.
|
||||
|
||||
|
||||
#### Peer mode start logic
|
||||
@ -100,12 +102,12 @@ When removed from the cluster:
|
||||
Loop:
|
||||
Sleep for some time
|
||||
|
||||
Sync cluster
|
||||
Sync cluster, and write cluster info into disk
|
||||
|
||||
If peer count < active size:
|
||||
Send join request
|
||||
If succeed:
|
||||
Return
|
||||
Check active size and send join request if needed
|
||||
If succeed:
|
||||
Clear cluster info from disk
|
||||
Return
|
||||
```
|
||||
|
||||
|
||||
@ -192,9 +194,6 @@ Machines in peer mode recover heartbeat between each other.
|
||||
|
||||
Machines in standby mode always sync the cluster. If sync fails, it uses the first address from data log as redirect target.
|
||||
|
||||
**Note**
|
||||
1. [TODO] Machine which runs in standby mode and has no log data cannot be recovered. But it could join the cluster finally if it is restarted always.
|
||||
|
||||
|
||||
### Kill one peer machine
|
||||
|
||||
|
@ -366,6 +366,9 @@ func (c *Config) Reset() error {
|
||||
if err := os.RemoveAll(filepath.Join(c.DataDir, "snapshot")); err != nil {
|
||||
return err
|
||||
}
|
||||
if err := os.RemoveAll(filepath.Join(c.DataDir, "standby_info")); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
19
etcd/etcd.go
19
etcd/etcd.go
@ -229,6 +229,7 @@ func (e *Etcd) Run() {
|
||||
PeerScheme: e.Config.PeerTLSInfo().Scheme(),
|
||||
PeerURL: e.Config.Peer.Addr,
|
||||
ClientURL: e.Config.Addr,
|
||||
DataDir: e.Config.DataDir,
|
||||
}
|
||||
e.StandbyServer = server.NewStandbyServer(ssConfig, client)
|
||||
|
||||
@ -237,14 +238,18 @@ func (e *Etcd) Run() {
|
||||
peerTLSConfig := server.TLSServerConfig(e.Config.PeerTLSInfo())
|
||||
etcdTLSConfig := server.TLSServerConfig(e.Config.EtcdTLSInfo())
|
||||
|
||||
startPeerServer, possiblePeers, err := e.PeerServer.FindCluster(e.Config.Discovery, e.Config.Peers)
|
||||
if err != nil {
|
||||
log.Fatal(err)
|
||||
}
|
||||
if startPeerServer {
|
||||
e.setMode(PeerMode)
|
||||
if !e.StandbyServer.IsRunning() {
|
||||
startPeerServer, possiblePeers, err := e.PeerServer.FindCluster(e.Config.Discovery, e.Config.Peers)
|
||||
if err != nil {
|
||||
log.Fatal(err)
|
||||
}
|
||||
if startPeerServer {
|
||||
e.setMode(PeerMode)
|
||||
} else {
|
||||
e.StandbyServer.SyncCluster(possiblePeers)
|
||||
e.setMode(StandbyMode)
|
||||
}
|
||||
} else {
|
||||
e.StandbyServer.SyncCluster(possiblePeers)
|
||||
e.setMode(StandbyMode)
|
||||
}
|
||||
|
||||
|
@ -1,9 +1,13 @@
|
||||
package server
|
||||
|
||||
import (
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"io/ioutil"
|
||||
"net/http"
|
||||
"net/url"
|
||||
"os"
|
||||
"path/filepath"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
@ -15,20 +19,28 @@ import (
|
||||
"github.com/coreos/etcd/store"
|
||||
)
|
||||
|
||||
const standbyInfoName = "standby_info"
|
||||
|
||||
type StandbyServerConfig struct {
|
||||
Name string
|
||||
PeerScheme string
|
||||
PeerURL string
|
||||
ClientURL string
|
||||
DataDir string
|
||||
}
|
||||
|
||||
type standbyInfo struct {
|
||||
Running bool
|
||||
Cluster []*machineMessage
|
||||
SyncInterval float64
|
||||
}
|
||||
|
||||
type StandbyServer struct {
|
||||
Config StandbyServerConfig
|
||||
client *Client
|
||||
|
||||
cluster []*machineMessage
|
||||
syncInterval float64
|
||||
joinIndex uint64
|
||||
standbyInfo
|
||||
joinIndex uint64
|
||||
|
||||
removeNotify chan bool
|
||||
started bool
|
||||
@ -39,11 +51,15 @@ type StandbyServer struct {
|
||||
}
|
||||
|
||||
func NewStandbyServer(config StandbyServerConfig, client *Client) *StandbyServer {
|
||||
return &StandbyServer{
|
||||
Config: config,
|
||||
client: client,
|
||||
syncInterval: DefaultSyncInterval,
|
||||
s := &StandbyServer{
|
||||
Config: config,
|
||||
client: client,
|
||||
standbyInfo: standbyInfo{SyncInterval: DefaultSyncInterval},
|
||||
}
|
||||
if err := s.loadInfo(); err != nil {
|
||||
log.Warnf("error load standby info file: %v", err)
|
||||
}
|
||||
return s
|
||||
}
|
||||
|
||||
func (s *StandbyServer) Start() {
|
||||
@ -62,6 +78,7 @@ func (s *StandbyServer) Start() {
|
||||
defer s.routineGroup.Done()
|
||||
s.monitorCluster()
|
||||
}()
|
||||
s.Running = true
|
||||
}
|
||||
|
||||
// Stop stops the server gracefully.
|
||||
@ -75,6 +92,11 @@ func (s *StandbyServer) Stop() {
|
||||
|
||||
close(s.closeChan)
|
||||
s.routineGroup.Wait()
|
||||
|
||||
if err := s.saveInfo(); err != nil {
|
||||
log.Warnf("error saving cluster info for standby")
|
||||
}
|
||||
s.Running = false
|
||||
}
|
||||
|
||||
// RemoveNotify notifies the server is removed from standby mode and ready
|
||||
@ -87,20 +109,24 @@ func (s *StandbyServer) ClientHTTPHandler() http.Handler {
|
||||
return http.HandlerFunc(s.redirectRequests)
|
||||
}
|
||||
|
||||
func (s *StandbyServer) Cluster() []string {
|
||||
func (s *StandbyServer) IsRunning() bool {
|
||||
return s.Running
|
||||
}
|
||||
|
||||
func (s *StandbyServer) ClusterURLs() []string {
|
||||
peerURLs := make([]string, 0)
|
||||
for _, peer := range s.cluster {
|
||||
for _, peer := range s.Cluster {
|
||||
peerURLs = append(peerURLs, peer.PeerURL)
|
||||
}
|
||||
return peerURLs
|
||||
}
|
||||
|
||||
func (s *StandbyServer) ClusterSize() int {
|
||||
return len(s.cluster)
|
||||
return len(s.Cluster)
|
||||
}
|
||||
|
||||
func (s *StandbyServer) setCluster(cluster []*machineMessage) {
|
||||
s.cluster = cluster
|
||||
s.Cluster = cluster
|
||||
}
|
||||
|
||||
func (s *StandbyServer) SyncCluster(peers []string) error {
|
||||
@ -109,20 +135,20 @@ func (s *StandbyServer) SyncCluster(peers []string) error {
|
||||
}
|
||||
|
||||
if err := s.syncCluster(peers); err != nil {
|
||||
log.Infof("fail syncing cluster(%v): %v", s.Cluster(), err)
|
||||
log.Infof("fail syncing cluster(%v): %v", s.ClusterURLs(), err)
|
||||
return err
|
||||
}
|
||||
|
||||
log.Infof("set cluster(%v) for standby server", s.Cluster())
|
||||
log.Infof("set cluster(%v) for standby server", s.ClusterURLs())
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *StandbyServer) SetSyncInterval(second float64) {
|
||||
s.syncInterval = second
|
||||
s.SyncInterval = second
|
||||
}
|
||||
|
||||
func (s *StandbyServer) ClusterLeader() *machineMessage {
|
||||
for _, machine := range s.cluster {
|
||||
for _, machine := range s.Cluster {
|
||||
if machine.State == raft.Leader {
|
||||
return machine
|
||||
}
|
||||
@ -144,9 +170,11 @@ func (s *StandbyServer) redirectRequests(w http.ResponseWriter, r *http.Request)
|
||||
uhttp.Redirect(leader.ClientURL, w, r)
|
||||
}
|
||||
|
||||
// monitorCluster assumes that the machine has tried to join the cluster and
|
||||
// failed, so it waits for the interval at the beginning.
|
||||
func (s *StandbyServer) monitorCluster() {
|
||||
for {
|
||||
timer := time.NewTimer(time.Duration(int64(s.syncInterval * float64(time.Second))))
|
||||
timer := time.NewTimer(time.Duration(int64(s.SyncInterval * float64(time.Second))))
|
||||
defer timer.Stop()
|
||||
select {
|
||||
case <-s.closeChan:
|
||||
@ -155,13 +183,13 @@ func (s *StandbyServer) monitorCluster() {
|
||||
}
|
||||
|
||||
if err := s.syncCluster(nil); err != nil {
|
||||
log.Warnf("fail syncing cluster(%v): %v", s.Cluster(), err)
|
||||
log.Warnf("fail syncing cluster(%v): %v", s.ClusterURLs(), err)
|
||||
continue
|
||||
}
|
||||
|
||||
leader := s.ClusterLeader()
|
||||
if leader == nil {
|
||||
log.Warnf("fail getting leader from cluster(%v)", s.Cluster())
|
||||
log.Warnf("fail getting leader from cluster(%v)", s.ClusterURLs())
|
||||
continue
|
||||
}
|
||||
|
||||
@ -180,7 +208,7 @@ func (s *StandbyServer) monitorCluster() {
|
||||
}
|
||||
|
||||
func (s *StandbyServer) syncCluster(peerURLs []string) error {
|
||||
peerURLs = append(s.Cluster(), peerURLs...)
|
||||
peerURLs = append(s.ClusterURLs(), peerURLs...)
|
||||
|
||||
for _, peerURL := range peerURLs {
|
||||
// Fetch current peer list
|
||||
@ -198,6 +226,9 @@ func (s *StandbyServer) syncCluster(peerURLs []string) error {
|
||||
|
||||
s.setCluster(machines)
|
||||
s.SetSyncInterval(config.SyncInterval)
|
||||
if err := s.saveInfo(); err != nil {
|
||||
log.Warnf("fail saving cluster info into disk: %v", err)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
return fmt.Errorf("unreachable cluster")
|
||||
@ -221,8 +252,8 @@ func (s *StandbyServer) join(peer string) error {
|
||||
log.Debugf("error getting cluster config")
|
||||
return err
|
||||
}
|
||||
if clusterConfig.ActiveSize <= len(s.Cluster()) {
|
||||
log.Debugf("stop joining because the cluster is full with %d nodes", len(s.Cluster()))
|
||||
if clusterConfig.ActiveSize <= len(s.Cluster) {
|
||||
log.Debugf("stop joining because the cluster is full with %d nodes", len(s.Cluster))
|
||||
return fmt.Errorf("out of quota")
|
||||
}
|
||||
|
||||
@ -252,3 +283,41 @@ func (s *StandbyServer) fullPeerURL(urlStr string) string {
|
||||
u.Scheme = s.Config.PeerScheme
|
||||
return u.String()
|
||||
}
|
||||
|
||||
func (s *StandbyServer) loadInfo() error {
|
||||
var info standbyInfo
|
||||
|
||||
path := filepath.Join(s.Config.DataDir, standbyInfoName)
|
||||
file, err := os.OpenFile(path, os.O_RDONLY, 0600)
|
||||
if err != nil {
|
||||
if os.IsNotExist(err) {
|
||||
return nil
|
||||
}
|
||||
return err
|
||||
}
|
||||
defer file.Close()
|
||||
if err = json.NewDecoder(file).Decode(&info); err != nil {
|
||||
return err
|
||||
}
|
||||
s.standbyInfo = info
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *StandbyServer) saveInfo() error {
|
||||
tmpFile, err := ioutil.TempFile(s.Config.DataDir, standbyInfoName)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if err = json.NewEncoder(tmpFile).Encode(s.standbyInfo); err != nil {
|
||||
tmpFile.Close()
|
||||
os.Remove(tmpFile.Name())
|
||||
return err
|
||||
}
|
||||
tmpFile.Close()
|
||||
|
||||
path := filepath.Join(s.Config.DataDir, standbyInfoName)
|
||||
if err = os.Rename(tmpFile.Name(), path); err != nil {
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
@ -167,7 +167,7 @@ func TestMultiNodeKillAllAndRecoveryWithStandbys(t *testing.T) {
|
||||
leaderChan := make(chan string, 1)
|
||||
all := make(chan bool, 1)
|
||||
|
||||
clusterSize := 5
|
||||
clusterSize := 15
|
||||
argGroup, etcds, err := CreateCluster(clusterSize, procAttr, false)
|
||||
defer DestroyCluster(etcds)
|
||||
|
||||
@ -184,8 +184,8 @@ func TestMultiNodeKillAllAndRecoveryWithStandbys(t *testing.T) {
|
||||
|
||||
c.SyncCluster()
|
||||
|
||||
// Reconfigure with smaller active size (3 nodes) and wait for demotion.
|
||||
resp, _ := tests.Put("http://localhost:7001/v2/admin/config", "application/json", bytes.NewBufferString(`{"activeSize":3}`))
|
||||
// Reconfigure with smaller active size (7 nodes) and wait for remove.
|
||||
resp, _ := tests.Put("http://localhost:7001/v2/admin/config", "application/json", bytes.NewBufferString(`{"activeSize":7}`))
|
||||
if !assert.Equal(t, resp.StatusCode, 200) {
|
||||
t.FailNow()
|
||||
}
|
||||
@ -195,10 +195,10 @@ func TestMultiNodeKillAllAndRecoveryWithStandbys(t *testing.T) {
|
||||
// Verify that there is three machines in peer mode.
|
||||
result, err := c.Get("_etcd/machines", false, true)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, len(result.Node.Nodes), 3)
|
||||
assert.Equal(t, len(result.Node.Nodes), 7)
|
||||
|
||||
// send 10 commands
|
||||
for i := 0; i < 10; i++ {
|
||||
// send set commands
|
||||
for i := 0; i < 2*clusterSize; i++ {
|
||||
// Test Set
|
||||
_, err := c.Set("foo", "bar", 0)
|
||||
if err != nil {
|
||||
@ -220,13 +220,13 @@ func TestMultiNodeKillAllAndRecoveryWithStandbys(t *testing.T) {
|
||||
time.Sleep(time.Second)
|
||||
|
||||
for i := 0; i < clusterSize; i++ {
|
||||
etcds[i], err = os.StartProcess(EtcdBinPath, argGroup[i], procAttr)
|
||||
etcds[i], err = os.StartProcess(EtcdBinPath, append(argGroup[i], "-peers="), procAttr)
|
||||
}
|
||||
|
||||
time.Sleep(2 * time.Second)
|
||||
|
||||
// send 10 commands
|
||||
for i := 0; i < 10; i++ {
|
||||
// send set commands
|
||||
for i := 0; i < 2*clusterSize; i++ {
|
||||
// Test Set
|
||||
_, err := c.Set("foo", "bar", 0)
|
||||
if err != nil {
|
||||
@ -234,8 +234,8 @@ func TestMultiNodeKillAllAndRecoveryWithStandbys(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
// Verify that we have three machines.
|
||||
// Verify that we have seven machines.
|
||||
result, err = c.Get("_etcd/machines", false, true)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, len(result.Node.Nodes), 3)
|
||||
assert.Equal(t, len(result.Node.Nodes), 7)
|
||||
}
|
||||
|
@ -19,7 +19,7 @@ func TestRemoveNode(t *testing.T) {
|
||||
procAttr := new(os.ProcAttr)
|
||||
procAttr.Files = []*os.File{nil, os.Stdout, os.Stderr}
|
||||
|
||||
clusterSize := 3
|
||||
clusterSize := 4
|
||||
argGroup, etcds, _ := CreateCluster(clusterSize, procAttr, false)
|
||||
defer DestroyCluster(etcds)
|
||||
|
||||
@ -29,7 +29,7 @@ func TestRemoveNode(t *testing.T) {
|
||||
|
||||
c.SyncCluster()
|
||||
|
||||
resp, _ := tests.Put("http://localhost:7001/v2/admin/config", "application/json", bytes.NewBufferString(`{"syncInterval":1}`))
|
||||
resp, _ := tests.Put("http://localhost:7001/v2/admin/config", "application/json", bytes.NewBufferString(`{"activeSize":4, "syncInterval":1}`))
|
||||
if !assert.Equal(t, resp.StatusCode, 200) {
|
||||
t.FailNow()
|
||||
}
|
||||
@ -39,6 +39,11 @@ func TestRemoveNode(t *testing.T) {
|
||||
client := &http.Client{}
|
||||
for i := 0; i < 2; i++ {
|
||||
for i := 0; i < 2; i++ {
|
||||
r, _ := tests.Put("http://localhost:7001/v2/admin/config", "application/json", bytes.NewBufferString(`{"activeSize":3}`))
|
||||
if !assert.Equal(t, r.StatusCode, 200) {
|
||||
t.FailNow()
|
||||
}
|
||||
|
||||
client.Do(rmReq)
|
||||
|
||||
fmt.Println("send remove to node3 and wait for its exiting")
|
||||
@ -50,7 +55,7 @@ func TestRemoveNode(t *testing.T) {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
if len(resp.Node.Nodes) != 2 {
|
||||
if len(resp.Node.Nodes) != 3 {
|
||||
t.Fatal("cannot remove peer")
|
||||
}
|
||||
|
||||
@ -69,6 +74,11 @@ func TestRemoveNode(t *testing.T) {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
r, _ = tests.Put("http://localhost:7001/v2/admin/config", "application/json", bytes.NewBufferString(`{"activeSize":4}`))
|
||||
if !assert.Equal(t, r.StatusCode, 200) {
|
||||
t.FailNow()
|
||||
}
|
||||
|
||||
time.Sleep(time.Second + time.Second)
|
||||
|
||||
resp, err = c.Get("_etcd/machines", false, false)
|
||||
@ -77,26 +87,33 @@ func TestRemoveNode(t *testing.T) {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
if len(resp.Node.Nodes) != 3 {
|
||||
t.Fatalf("add peer fails #1 (%d != 3)", len(resp.Node.Nodes))
|
||||
if len(resp.Node.Nodes) != 4 {
|
||||
t.Fatalf("add peer fails #1 (%d != 4)", len(resp.Node.Nodes))
|
||||
}
|
||||
}
|
||||
|
||||
// first kill the node, then remove it, then add it back
|
||||
for i := 0; i < 2; i++ {
|
||||
r, _ := tests.Put("http://localhost:7001/v2/admin/config", "application/json", bytes.NewBufferString(`{"activeSize":3}`))
|
||||
if !assert.Equal(t, r.StatusCode, 200) {
|
||||
t.FailNow()
|
||||
}
|
||||
|
||||
etcds[2].Kill()
|
||||
fmt.Println("kill node3 and wait for its exiting")
|
||||
etcds[2].Wait()
|
||||
|
||||
client.Do(rmReq)
|
||||
|
||||
time.Sleep(100 * time.Millisecond)
|
||||
|
||||
resp, err := c.Get("_etcd/machines", false, false)
|
||||
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
if len(resp.Node.Nodes) != 2 {
|
||||
if len(resp.Node.Nodes) != 3 {
|
||||
t.Fatal("cannot remove peer")
|
||||
}
|
||||
|
||||
@ -112,7 +129,12 @@ func TestRemoveNode(t *testing.T) {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
time.Sleep(time.Second)
|
||||
r, _ = tests.Put("http://localhost:7001/v2/admin/config", "application/json", bytes.NewBufferString(`{"activeSize":4}`))
|
||||
if !assert.Equal(t, r.StatusCode, 200) {
|
||||
t.FailNow()
|
||||
}
|
||||
|
||||
time.Sleep(time.Second + time.Second)
|
||||
|
||||
resp, err = c.Get("_etcd/machines", false, false)
|
||||
|
||||
@ -120,8 +142,8 @@ func TestRemoveNode(t *testing.T) {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
if len(resp.Node.Nodes) != 3 {
|
||||
t.Fatalf("add peer fails #2 (%d != 3)", len(resp.Node.Nodes))
|
||||
if len(resp.Node.Nodes) != 4 {
|
||||
t.Fatalf("add peer fails #2 (%d != 4)", len(resp.Node.Nodes))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -169,7 +169,7 @@ func DestroyCluster(etcds []*os.Process) error {
|
||||
//
|
||||
func Monitor(size int, allowDeadNum int, leaderChan chan string, all chan bool, stop chan bool) {
|
||||
leaderMap := make(map[int]string)
|
||||
baseAddrFormat := "http://0.0.0.0:400%d"
|
||||
baseAddrFormat := "http://0.0.0.0:%d"
|
||||
|
||||
for {
|
||||
knownLeader := "unknown"
|
||||
@ -177,7 +177,7 @@ func Monitor(size int, allowDeadNum int, leaderChan chan string, all chan bool,
|
||||
var i int
|
||||
|
||||
for i = 0; i < size; i++ {
|
||||
leader, err := getLeader(fmt.Sprintf(baseAddrFormat, i+1))
|
||||
leader, err := getLeader(fmt.Sprintf(baseAddrFormat, i+4001))
|
||||
|
||||
if err == nil {
|
||||
leaderMap[i] = leader
|
||||
|
Loading…
x
Reference in New Issue
Block a user