etcd/etcdserver/server.go

703 lines
19 KiB
Go
Raw Normal View History

/*
Copyright 2014 CoreOS, Inc.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/
2014-08-26 05:39:02 +04:00
package etcdserver
import (
2014-09-27 01:18:59 +04:00
"encoding/json"
"errors"
2014-09-16 04:35:02 +04:00
"log"
2014-09-18 01:38:10 +04:00
"math/rand"
"os"
"strconv"
2014-09-25 05:05:33 +04:00
"sync/atomic"
"time"
2014-10-14 11:35:28 +04:00
"github.com/coreos/etcd/Godeps/_workspace/src/code.google.com/p/go.net/context"
2014-10-04 16:20:45 +04:00
"github.com/coreos/etcd/discovery"
2014-09-03 08:36:14 +04:00
pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
"github.com/coreos/etcd/etcdserver/stats"
2014-10-11 16:41:00 +04:00
"github.com/coreos/etcd/pkg/pbutil"
2014-08-26 05:39:02 +04:00
"github.com/coreos/etcd/raft"
2014-08-28 05:53:18 +04:00
"github.com/coreos/etcd/raft/raftpb"
"github.com/coreos/etcd/snap"
"github.com/coreos/etcd/store"
2014-08-26 05:39:02 +04:00
"github.com/coreos/etcd/wait"
"github.com/coreos/etcd/wal"
2014-08-26 05:39:02 +04:00
)
2014-09-17 05:18:45 +04:00
const (
// owner can make/remove files inside the directory
privateDirMode = 0700
2014-09-17 05:18:45 +04:00
defaultSyncTimeout = time.Second
DefaultSnapCount = 10000
// TODO: calculate based on heartbeat interval
defaultPublishRetryInterval = 5 * time.Second
2014-09-17 05:18:45 +04:00
)
2014-09-16 04:35:02 +04:00
2014-08-29 03:41:42 +04:00
var (
ErrUnknownMethod = errors.New("etcdserver: unknown method")
ErrStopped = errors.New("etcdserver: server stopped")
ErrIDRemoved = errors.New("etcdserver: ID removed")
ErrIDExists = errors.New("etcdserver: ID exists")
ErrIDNotFound = errors.New("etcdserver: ID not found")
2014-08-29 03:41:42 +04:00
)
2014-09-18 01:38:10 +04:00
func init() {
rand.Seed(time.Now().UnixNano())
}
type sendFunc func(m []raftpb.Message)
2014-08-28 00:37:22 +04:00
2014-08-26 05:39:02 +04:00
type Response struct {
2014-08-31 01:19:03 +04:00
Event *store.Event
2014-09-09 03:56:10 +04:00
Watcher store.Watcher
2014-08-31 02:27:43 +04:00
err error
2014-08-26 05:39:02 +04:00
}
2014-09-17 05:18:45 +04:00
type Storage interface {
// Save function saves ents and state to the underlying stable storage.
// Save MUST block until st and ents are on stable storage.
Save(st raftpb.HardState, ents []raftpb.Entry)
// SaveSnap function saves snapshot to the underlying stable storage.
SaveSnap(snap raftpb.Snapshot)
// TODO: WAL should be able to control cut itself. After implement self-controled cut,
// remove it in this interface.
// Cut cuts out a new wal file for saving new state and entries.
Cut() error
}
type Server interface {
// Start performs any initialization of the Server necessary for it to
// begin serving requests. It must be called before Do or Process.
// Start must be non-blocking; any long-running server functionality
// should be implemented in goroutines.
2014-09-30 04:14:22 +04:00
Start()
// Stop terminates the Server and performs any necessary finalization.
// Do and Process cannot be called after Stop has been invoked.
Stop()
// Do takes a request and attempts to fulfil it, returning a Response.
Do(ctx context.Context, r pb.Request) (Response, error)
// Process takes a raft message and applies it to the server's raft state
// machine, respecting any timeout of the given context.
Process(ctx context.Context, m raftpb.Message) error
// AddMember attempts to add a member into the cluster. It will return
// ErrIDRemoved if member ID is removed from the cluster, or return
// ErrIDExists if member ID exists in the cluster.
AddMember(ctx context.Context, memb Member) error
// RemoveMember attempts to remove a member from the cluster. It will
// return ErrIDRemoved if member ID is removed from the cluster, or return
// ErrIDNotFound if member ID is not in the cluster.
RemoveMember(ctx context.Context, id uint64) error
}
type Stats interface {
// SelfStats returns the struct representing statistics of this server
SelfStats() []byte
// LeaderStats returns the statistics of all followers in the cluster
// if this server is leader. Otherwise, nil is returned.
LeaderStats() []byte
// StoreStats returns statistics of the store backing this EtcdServer
StoreStats() []byte
// UpdateRecvApp updates the underlying statistics in response to a receiving an Append request
UpdateRecvApp(from uint64, length int64)
}
2014-09-25 05:05:33 +04:00
type RaftTimer interface {
2014-10-08 15:58:53 +04:00
Index() uint64
Term() uint64
2014-09-25 05:05:33 +04:00
}
2014-10-11 17:03:03 +04:00
// EtcdServer is the production implementation of the Server interface
type EtcdServer struct {
w wait.Wait
done chan struct{}
2014-10-17 08:37:10 +04:00
stopped chan struct{}
2014-10-11 17:03:03 +04:00
id uint64
2014-10-11 15:20:14 +04:00
clusterID uint64
2014-10-11 17:03:03 +04:00
attributes Attributes
ClusterStore ClusterStore
node raft.Node
store store.Store
stats *stats.ServerStats
lstats *stats.LeaderStats
2014-10-11 17:03:03 +04:00
// send specifies the send function for sending msgs to members. send
// MUST NOT block. It is okay to drop messages, since clients should
// timeout and reissue their messages. If send is nil, server will
// panic.
send sendFunc
storage Storage
2014-10-17 00:35:59 +04:00
Ticker <-chan time.Time
SyncTicker <-chan time.Time
2014-10-11 17:03:03 +04:00
snapCount uint64 // number of entries to trigger a snapshot
// Cache of the latest raft index and raft term the server has seen
raftIndex uint64
raftTerm uint64
}
// NewServer creates a new EtcdServer from the supplied configuration. The
// configuration is considered static for the lifetime of the EtcdServer.
func NewServer(cfg *ServerConfig) *EtcdServer {
2014-10-11 17:03:03 +04:00
if err := os.MkdirAll(cfg.SnapDir(), privateDirMode); err != nil {
log.Fatalf("etcdserver: cannot create snapshot directory: %v", err)
}
2014-10-11 17:03:03 +04:00
ss := snap.New(cfg.SnapDir())
st := store.New()
var w *wal.WAL
var n raft.Node
2014-10-11 15:20:14 +04:00
var id, cid uint64
2014-10-11 17:03:03 +04:00
if !wal.Exist(cfg.WALDir()) {
if err := cfg.VerifyBootstrapConfig(); err != nil {
log.Fatalf("etcdserver: %v", err)
2014-10-11 17:03:03 +04:00
}
2014-10-11 17:47:44 +04:00
if cfg.ShouldDiscover() {
2014-10-11 17:03:03 +04:00
d, err := discovery.New(cfg.DiscoveryURL, cfg.ID(), cfg.Cluster.String())
2014-10-04 16:20:45 +04:00
if err != nil {
2014-10-15 09:05:24 +04:00
log.Fatalf("etcdserver: cannot init discovery %v", err)
2014-10-04 16:20:45 +04:00
}
s, err := d.Discover()
if err != nil {
2014-10-15 09:05:24 +04:00
log.Fatalf("etcdserver: %v", err)
2014-10-04 16:20:45 +04:00
}
if err = cfg.Cluster.Set(s); err != nil {
2014-10-15 09:05:24 +04:00
log.Fatalf("etcdserver: %v", err)
2014-10-04 16:20:45 +04:00
}
}
2014-10-11 15:20:14 +04:00
id, cid, n, w = startNode(cfg)
} else {
2014-10-11 17:47:44 +04:00
if cfg.ShouldDiscover() {
2014-10-15 09:05:24 +04:00
log.Printf("etcdserver: warn: ignoring discovery: etcd has already been initialized and has a valid log in %q", cfg.WALDir())
}
2014-10-08 15:58:53 +04:00
var index uint64
snapshot, err := ss.Load()
if err != nil && err != snap.ErrNoSnapshot {
log.Fatal(err)
}
if snapshot != nil {
2014-10-15 09:05:24 +04:00
log.Printf("etcdserver: recovering from snapshot at index %d", snapshot.Index)
st.Recovery(snapshot.Data)
index = snapshot.Index
}
2014-10-11 15:20:14 +04:00
id, cid, n, w = restartNode(cfg, index, snapshot)
}
2014-10-21 21:41:32 +04:00
cls := &clusterStore{Store: st, id: cid}
sstats := &stats.ServerStats{
Name: cfg.Name,
ID: idAsHex(cfg.ID()),
}
lstats := stats.NewLeaderStats(idAsHex(cfg.ID()))
s := &EtcdServer{
store: st,
node: n,
2014-10-11 15:20:14 +04:00
id: id,
clusterID: cid,
attributes: Attributes{Name: cfg.Name, ClientURLs: cfg.ClientURLs.StringSlice()},
storage: struct {
*wal.WAL
*snap.Snapshotter
}{w, ss},
stats: sstats,
lstats: lstats,
send: Sender(cfg.Transport, cls, sstats, lstats),
2014-10-17 00:35:59 +04:00
Ticker: time.Tick(100 * time.Millisecond),
SyncTicker: time.Tick(500 * time.Millisecond),
snapCount: cfg.SnapCount,
ClusterStore: cls,
}
return s
}
2014-09-01 05:48:18 +04:00
// Start prepares and starts server in a new goroutine. It is no longer safe to
// modify a server's fields after it has been sent to Start.
// It also starts a goroutine to publish its server information.
2014-09-30 04:14:22 +04:00
func (s *EtcdServer) Start() {
s.start()
go s.publish(defaultPublishRetryInterval)
}
// start prepares and starts server in a new goroutine. It is no longer safe to
// modify a server's fields after it has been sent to Start.
// This function is just used for testing.
func (s *EtcdServer) start() {
if s.snapCount == 0 {
2014-09-17 05:18:45 +04:00
log.Printf("etcdserver: set snapshot count to default %d", DefaultSnapCount)
s.snapCount = DefaultSnapCount
2014-09-17 05:18:45 +04:00
}
2014-08-29 03:41:42 +04:00
s.w = wait.New()
s.done = make(chan struct{})
2014-10-17 08:37:10 +04:00
s.stopped = make(chan struct{})
s.stats.Initialize()
// TODO: if this is an empty log, writes all peer infos
// into the first entry
2014-08-29 03:41:42 +04:00
go s.run()
}
2014-08-28 00:37:22 +04:00
func (s *EtcdServer) Process(ctx context.Context, m raftpb.Message) error {
return s.node.Step(ctx, m)
}
func (s *EtcdServer) run() {
2014-09-16 04:35:02 +04:00
var syncC <-chan time.Time
2014-09-17 05:18:45 +04:00
// snapi indicates the index of the last submitted snapshot request
2014-10-08 15:58:53 +04:00
var snapi, appliedi uint64
var nodes []uint64
2014-08-26 05:39:02 +04:00
for {
select {
2014-10-17 00:35:59 +04:00
case <-s.Ticker:
s.node.Tick()
case rd := <-s.node.Ready():
if rd.SoftState != nil {
nodes = rd.SoftState.Nodes
if rd.RaftState == raft.StateLeader {
2014-10-17 00:35:59 +04:00
syncC = s.SyncTicker
} else {
syncC = nil
}
}
s.storage.Save(rd.HardState, rd.Entries)
s.storage.SaveSnap(rd.Snapshot)
s.send(rd.Messages)
// TODO(bmizerany): do this in the background, but take
// care to apply entries in a single goroutine, and not
// race them.
// TODO: apply configuration change into ClusterStore.
if len(rd.CommittedEntries) != 0 {
appliedi = s.apply(rd.CommittedEntries, nodes)
}
2014-09-17 05:18:45 +04:00
if rd.Snapshot.Index > snapi {
snapi = rd.Snapshot.Index
}
// recover from snapshot if it is more updated than current applied
if rd.Snapshot.Index > appliedi {
if err := s.store.Recovery(rd.Snapshot.Data); err != nil {
2014-09-17 05:18:45 +04:00
panic("TODO: this is bad, what do we do about it?")
}
appliedi = rd.Snapshot.Index
}
if appliedi-snapi > s.snapCount {
s.snapshot(appliedi, nodes)
2014-09-17 05:18:45 +04:00
snapi = appliedi
}
2014-09-16 04:35:02 +04:00
case <-syncC:
s.sync(defaultSyncTimeout)
2014-08-29 03:41:42 +04:00
case <-s.done:
2014-10-17 08:37:10 +04:00
close(s.stopped)
2014-08-26 22:37:07 +04:00
return
2014-08-26 05:39:02 +04:00
}
}
}
2014-10-17 08:37:10 +04:00
// Stop stops the server gracefully, and shuts down the running goroutine.
// Stop should be called after a Start(s), otherwise it will block forever.
func (s *EtcdServer) Stop() {
s.node.Stop()
close(s.done)
2014-10-17 08:37:10 +04:00
<-s.stopped
2014-08-30 02:32:41 +04:00
}
2014-08-29 03:41:42 +04:00
// Do interprets r and performs an operation on s.store according to r.Method
2014-09-16 04:35:02 +04:00
// and other fields. If r.Method is "POST", "PUT", "DELETE", or a "GET" with
2014-09-01 05:48:18 +04:00
// Quorum == true, r will be sent through consensus before performing its
// respective operation. Do will block until an action is performed or there is
// an error.
func (s *EtcdServer) Do(ctx context.Context, r pb.Request) (Response, error) {
if r.ID == 0 {
2014-10-12 04:29:39 +04:00
panic("r.ID cannot be 0")
2014-08-26 05:39:02 +04:00
}
2014-08-31 01:35:27 +04:00
if r.Method == "GET" && r.Quorum {
r.Method = "QGET"
}
switch r.Method {
2014-08-31 01:35:27 +04:00
case "POST", "PUT", "DELETE", "QGET":
data, err := r.Marshal()
if err != nil {
return Response{}, err
}
ch := s.w.Register(r.ID)
s.node.Propose(ctx, data)
select {
case x := <-ch:
resp := x.(Response)
return resp, resp.err
case <-ctx.Done():
s.w.Trigger(r.ID, nil) // GC wait
return Response{}, ctx.Err()
2014-08-29 03:41:42 +04:00
case <-s.done:
return Response{}, ErrStopped
}
case "GET":
switch {
case r.Wait:
wc, err := s.store.Watch(r.Path, r.Recursive, r.Stream, r.Since)
if err != nil {
return Response{}, err
}
return Response{Watcher: wc}, nil
default:
ev, err := s.store.Get(r.Path, r.Recursive, r.Sorted)
if err != nil {
return Response{}, err
}
return Response{Event: ev}, nil
}
default:
return Response{}, ErrUnknownMethod
2014-08-26 05:39:02 +04:00
}
}
2014-08-26 22:37:07 +04:00
func (s *EtcdServer) SelfStats() []byte {
return s.stats.JSON()
}
func (s *EtcdServer) LeaderStats() []byte {
// TODO(jonboulle): need to lock access to lstats, set it to nil when not leader, ...
return s.lstats.JSON()
}
func (s *EtcdServer) StoreStats() []byte {
return s.store.JsonStats()
}
func (s *EtcdServer) UpdateRecvApp(from uint64, length int64) {
s.stats.RecvAppendReq(idAsHex(from), int(length))
}
func (s *EtcdServer) AddMember(ctx context.Context, memb Member) error {
// TODO: move Member to protobuf type
b, err := json.Marshal(memb)
if err != nil {
return err
}
2014-09-23 23:02:44 +04:00
cc := raftpb.ConfChange{
2014-09-21 05:56:40 +04:00
ID: GenID(),
2014-09-23 23:02:44 +04:00
Type: raftpb.ConfChangeAddNode,
NodeID: memb.ID,
Context: b,
}
return s.configure(ctx, cc)
}
2014-10-08 15:58:53 +04:00
func (s *EtcdServer) RemoveMember(ctx context.Context, id uint64) error {
2014-09-23 23:02:44 +04:00
cc := raftpb.ConfChange{
2014-09-21 05:56:40 +04:00
ID: GenID(),
2014-09-23 23:02:44 +04:00
Type: raftpb.ConfChangeRemoveNode,
NodeID: id,
}
return s.configure(ctx, cc)
}
2014-09-25 05:05:33 +04:00
// Implement the RaftTimer interface
2014-10-08 15:58:53 +04:00
func (s *EtcdServer) Index() uint64 {
return atomic.LoadUint64(&s.raftIndex)
2014-09-25 05:05:33 +04:00
}
2014-10-08 15:58:53 +04:00
func (s *EtcdServer) Term() uint64 {
return atomic.LoadUint64(&s.raftTerm)
2014-09-25 05:05:33 +04:00
}
// configure sends configuration change through consensus then performs it.
// It will block until the change is performed or there is an error.
2014-09-23 23:02:44 +04:00
func (s *EtcdServer) configure(ctx context.Context, cc raftpb.ConfChange) error {
2014-10-12 04:35:58 +04:00
ch := s.w.Register(cc.ID)
if err := s.node.ProposeConfChange(ctx, cc); err != nil {
log.Printf("configure error: %v", err)
2014-10-12 04:35:58 +04:00
s.w.Trigger(cc.ID, nil)
return err
}
select {
case x := <-ch:
if err, ok := x.(error); ok {
return err
}
if x != nil {
log.Panicf("unexpected return type")
}
return nil
case <-ctx.Done():
2014-10-12 04:35:58 +04:00
s.w.Trigger(cc.ID, nil) // GC wait
return ctx.Err()
case <-s.done:
return ErrStopped
}
}
2014-09-16 04:35:02 +04:00
// sync proposes a SYNC request and is non-blocking.
// This makes no guarantee that the request will be proposed or performed.
// The request will be cancelled after the given timeout.
func (s *EtcdServer) sync(timeout time.Duration) {
ctx, cancel := context.WithTimeout(context.Background(), timeout)
req := pb.Request{
Method: "SYNC",
2014-10-12 04:35:58 +04:00
ID: GenID(),
2014-09-16 04:35:02 +04:00
Time: time.Now().UnixNano(),
}
2014-10-11 16:41:00 +04:00
data := pbutil.MustMarshal(&req)
2014-09-16 04:35:02 +04:00
// There is no promise that node has leader when do SYNC request,
// so it uses goroutine to propose.
go func() {
s.node.Propose(ctx, data)
2014-09-16 04:35:02 +04:00
cancel()
}()
}
2014-09-27 01:18:59 +04:00
// publish registers server information into the cluster. The information
// is the JSON representation of this server's member struct, updated with the
// static clientURLs of the server.
2014-09-27 01:18:59 +04:00
// The function keeps attempting to register until it succeeds,
// or its server is stopped.
func (s *EtcdServer) publish(retryInterval time.Duration) {
b, err := json.Marshal(s.attributes)
2014-09-27 01:18:59 +04:00
if err != nil {
log.Printf("etcdserver: json marshal error: %v", err)
return
}
req := pb.Request{
2014-10-12 04:35:58 +04:00
ID: GenID(),
2014-09-27 01:18:59 +04:00
Method: "PUT",
Path: Member{ID: s.id}.storeKey() + attributesSuffix,
2014-09-27 01:18:59 +04:00
Val: string(b),
}
for {
ctx, cancel := context.WithTimeout(context.Background(), retryInterval)
_, err := s.Do(ctx, req)
cancel()
switch err {
case nil:
log.Printf("etcdserver: published %+v to the cluster", s.attributes)
2014-09-27 01:18:59 +04:00
return
case ErrStopped:
log.Printf("etcdserver: aborting publish because server is stopped")
return
default:
log.Printf("etcdserver: publish error: %v", err)
}
}
}
func getExpirationTime(r *pb.Request) time.Time {
var t time.Time
if r.Expiration != 0 {
t = time.Unix(0, r.Expiration)
}
return t
}
func (s *EtcdServer) apply(es []raftpb.Entry, nodes []uint64) uint64 {
2014-10-11 17:58:39 +04:00
var applied uint64
for i := range es {
e := es[i]
switch e.Type {
case raftpb.EntryNormal:
var r pb.Request
2014-10-11 16:41:00 +04:00
pbutil.MustUnmarshal(&r, e.Data)
2014-10-11 17:58:39 +04:00
s.w.Trigger(r.ID, s.applyRequest(r))
case raftpb.EntryConfChange:
var cc raftpb.ConfChange
2014-10-11 16:41:00 +04:00
pbutil.MustUnmarshal(&cc, e.Data)
s.w.Trigger(cc.ID, s.applyConfChange(cc, nodes))
2014-10-11 17:58:39 +04:00
default:
panic("unexpected entry type")
}
atomic.StoreUint64(&s.raftIndex, e.Index)
atomic.StoreUint64(&s.raftTerm, e.Term)
applied = e.Index
}
return applied
}
// applyRequest interprets r as a call to store.X and returns a Response interpreted
// from store.Event
func (s *EtcdServer) applyRequest(r pb.Request) Response {
2014-08-31 02:27:43 +04:00
f := func(ev *store.Event, err error) Response {
return Response{Event: ev, err: err}
}
expr := getExpirationTime(&r)
switch r.Method {
case "POST":
return f(s.store.Create(r.Path, r.Dir, r.Val, true, expr))
case "PUT":
2014-09-19 08:22:10 +04:00
exists, existsSet := getBool(r.PrevExist)
2014-08-27 03:39:26 +04:00
switch {
2014-08-27 23:04:00 +04:00
case existsSet:
2014-08-27 22:54:29 +04:00
if exists {
return f(s.store.Update(r.Path, r.Val, expr))
2014-08-27 22:54:29 +04:00
}
return f(s.store.Create(r.Path, r.Dir, r.Val, false, expr))
case r.PrevIndex > 0 || r.PrevValue != "":
return f(s.store.CompareAndSwap(r.Path, r.PrevValue, r.PrevIndex, r.Val, expr))
2014-08-27 22:29:33 +04:00
default:
return f(s.store.Set(r.Path, r.Dir, r.Val, expr))
2014-08-27 03:39:26 +04:00
}
case "DELETE":
2014-08-27 22:34:42 +04:00
switch {
case r.PrevIndex > 0 || r.PrevValue != "":
return f(s.store.CompareAndDelete(r.Path, r.PrevValue, r.PrevIndex))
2014-08-27 22:34:42 +04:00
default:
return f(s.store.Delete(r.Path, r.Dir, r.Recursive))
2014-08-27 22:34:42 +04:00
}
2014-08-31 01:35:27 +04:00
case "QGET":
return f(s.store.Get(r.Path, r.Recursive, r.Sorted))
2014-09-16 04:35:02 +04:00
case "SYNC":
s.store.DeleteExpiredKeys(time.Unix(0, r.Time))
2014-09-16 04:35:02 +04:00
return Response{}
default:
2014-08-28 00:37:22 +04:00
// This should never be reached, but just in case:
2014-08-31 02:27:43 +04:00
return Response{err: ErrUnknownMethod}
}
2014-08-26 22:37:07 +04:00
}
2014-08-27 22:54:29 +04:00
func (s *EtcdServer) applyConfChange(cc raftpb.ConfChange, nodes []uint64) error {
if err := checkConfChange(cc, nodes); err != nil {
cc.NodeID = raft.None
s.node.ApplyConfChange(cc)
return err
}
s.node.ApplyConfChange(cc)
switch cc.Type {
case raftpb.ConfChangeAddNode:
var m Member
if err := json.Unmarshal(cc.Context, &m); err != nil {
panic("unexpected unmarshal error")
}
2014-10-04 02:21:10 +04:00
if cc.NodeID != m.ID {
2014-10-07 21:59:02 +04:00
panic("unexpected nodeID mismatch")
2014-10-04 02:21:10 +04:00
}
s.ClusterStore.Add(m)
case raftpb.ConfChangeRemoveNode:
s.ClusterStore.Remove(cc.NodeID)
}
return nil
}
func checkConfChange(cc raftpb.ConfChange, nodes []uint64) error {
switch cc.Type {
case raftpb.ConfChangeAddNode:
if containsUint64(nodes, cc.NodeID) {
return ErrIDExists
}
case raftpb.ConfChangeRemoveNode:
if !containsUint64(nodes, cc.NodeID) {
return ErrIDNotFound
}
default:
panic("unexpected ConfChange type")
}
return nil
}
2014-09-17 05:18:45 +04:00
// TODO: non-blocking snapshot
2014-10-08 15:58:53 +04:00
func (s *EtcdServer) snapshot(snapi uint64, snapnodes []uint64) {
d, err := s.store.Save()
2014-09-17 05:18:45 +04:00
// TODO: current store will never fail to do a snapshot
// what should we do if the store might fail?
if err != nil {
panic("TODO: this is bad, what do we do about it?")
}
s.node.Compact(snapi, snapnodes, d)
s.storage.Cut()
2014-09-17 05:18:45 +04:00
}
2014-10-11 15:20:14 +04:00
func startNode(cfg *ServerConfig) (id, cid uint64, n raft.Node, w *wal.WAL) {
2014-10-11 16:41:00 +04:00
var err error
// TODO: remove the discoveryURL when it becomes part of the source for
// generating nodeID.
2014-10-11 15:20:14 +04:00
cfg.Cluster.GenID([]byte(cfg.DiscoveryURL))
metadata := pbutil.MustMarshal(&pb.Metadata{NodeID: cfg.ID(), ClusterID: cfg.Cluster.ID()})
2014-10-11 16:41:00 +04:00
if w, err = wal.Create(cfg.WALDir(), metadata); err != nil {
2014-10-11 17:03:03 +04:00
log.Fatal(err)
}
2014-10-11 15:20:14 +04:00
ids := cfg.Cluster.MemberIDs()
2014-10-11 17:03:03 +04:00
peers := make([]raft.Peer, len(ids))
for i, id := range ids {
2014-10-11 15:20:14 +04:00
ctx, err := json.Marshal((*cfg.Cluster).FindID(id))
2014-10-11 17:03:03 +04:00
if err != nil {
log.Fatal(err)
}
peers[i] = raft.Peer{ID: id, Context: ctx}
}
2014-10-11 15:20:14 +04:00
id, cid = cfg.ID(), cfg.Cluster.ID()
2014-10-21 03:35:41 +04:00
log.Printf("etcdserver: start node %d in cluster %d", id, cid)
2014-10-11 17:03:03 +04:00
n = raft.StartNode(cfg.ID(), peers, 10, 1)
return
}
2014-10-11 15:20:14 +04:00
func restartNode(cfg *ServerConfig, index uint64, snapshot *raftpb.Snapshot) (id, cid uint64, n raft.Node, w *wal.WAL) {
2014-10-11 17:03:03 +04:00
var err error
// restart a node from previous wal
if w, err = wal.OpenAtIndex(cfg.WALDir(), index); err != nil {
log.Fatal(err)
}
2014-10-11 16:41:00 +04:00
wmetadata, st, ents, err := w.ReadAll()
2014-10-11 17:03:03 +04:00
if err != nil {
log.Fatal(err)
}
2014-10-11 16:41:00 +04:00
var metadata pb.Metadata
pbutil.MustUnmarshal(&metadata, wmetadata)
2014-10-11 15:20:14 +04:00
id, cid = metadata.NodeID, metadata.ClusterID
2014-10-21 03:35:41 +04:00
log.Printf("etcdserver: restart node %d in cluster %d at commit index %d", id, cid, st.Commit)
2014-10-11 15:20:14 +04:00
n = raft.RestartNode(id, 10, 1, snapshot, st, ents)
2014-10-11 17:03:03 +04:00
return
}
2014-09-16 04:35:02 +04:00
// TODO: move the function to /id pkg maybe?
// GenID generates a random id that is not equal to 0.
2014-10-08 15:58:53 +04:00
func GenID() (n uint64) {
2014-09-18 01:38:10 +04:00
for n == 0 {
2014-10-08 15:58:53 +04:00
n = uint64(rand.Int63())
2014-09-16 04:35:02 +04:00
}
2014-09-18 01:38:10 +04:00
return
2014-09-16 04:35:02 +04:00
}
2014-08-27 22:54:29 +04:00
func getBool(v *bool) (vv bool, set bool) {
if v == nil {
return false, false
}
return *v, true
}
func containsUint64(a []uint64, x uint64) bool {
for _, v := range a {
if v == x {
return true
}
}
return false
}
func idAsHex(id uint64) string {
return strconv.FormatUint(id, 16)
}