|
@@ -91,6 +91,7 @@ type Sender interface {
|
|
|
Remove(id types.ID)
|
|
Remove(id types.ID)
|
|
|
Update(m *Member)
|
|
Update(m *Member)
|
|
|
Stop()
|
|
Stop()
|
|
|
|
|
+ ShouldStopNotify() <-chan struct{}
|
|
|
}
|
|
}
|
|
|
|
|
|
|
|
type Storage interface {
|
|
type Storage interface {
|
|
@@ -157,7 +158,7 @@ type RaftTimer interface {
|
|
|
type EtcdServer struct {
|
|
type EtcdServer struct {
|
|
|
w wait.Wait
|
|
w wait.Wait
|
|
|
done chan struct{}
|
|
done chan struct{}
|
|
|
- stopped chan struct{}
|
|
|
|
|
|
|
+ stop chan struct{}
|
|
|
id types.ID
|
|
id types.ID
|
|
|
attributes Attributes
|
|
attributes Attributes
|
|
|
|
|
|
|
@@ -186,6 +187,8 @@ type EtcdServer struct {
|
|
|
// Cache of the latest raft index and raft term the server has seen
|
|
// Cache of the latest raft index and raft term the server has seen
|
|
|
raftIndex uint64
|
|
raftIndex uint64
|
|
|
raftTerm uint64
|
|
raftTerm uint64
|
|
|
|
|
+
|
|
|
|
|
+ raftLead uint64
|
|
|
}
|
|
}
|
|
|
|
|
|
|
|
// NewServer creates a new EtcdServer from the supplied configuration. The
|
|
// NewServer creates a new EtcdServer from the supplied configuration. The
|
|
@@ -204,20 +207,24 @@ func NewServer(cfg *ServerConfig) (*EtcdServer, error) {
|
|
|
switch {
|
|
switch {
|
|
|
case !haveWAL && !cfg.NewCluster:
|
|
case !haveWAL && !cfg.NewCluster:
|
|
|
us := getOtherPeerURLs(cfg.Cluster, cfg.Name)
|
|
us := getOtherPeerURLs(cfg.Cluster, cfg.Name)
|
|
|
- cl, err := GetClusterFromPeers(us)
|
|
|
|
|
|
|
+ existingCluster, err := GetClusterFromPeers(us)
|
|
|
if err != nil {
|
|
if err != nil {
|
|
|
return nil, fmt.Errorf("cannot fetch cluster info from peer urls: %v", err)
|
|
return nil, fmt.Errorf("cannot fetch cluster info from peer urls: %v", err)
|
|
|
}
|
|
}
|
|
|
- if err := cfg.Cluster.ValidateAndAssignIDs(cl.Members()); err != nil {
|
|
|
|
|
- return nil, fmt.Errorf("error validating IDs from cluster %s: %v", cl, err)
|
|
|
|
|
|
|
+ if err := ValidateClusterAndAssignIDs(cfg.Cluster, existingCluster); err != nil {
|
|
|
|
|
+ return nil, fmt.Errorf("error validating peerURLs %s: %v", existingCluster, err)
|
|
|
}
|
|
}
|
|
|
- cfg.Cluster.SetID(cl.id)
|
|
|
|
|
|
|
+ cfg.Cluster.SetID(existingCluster.id)
|
|
|
cfg.Cluster.SetStore(st)
|
|
cfg.Cluster.SetStore(st)
|
|
|
|
|
+ cfg.Print()
|
|
|
id, n, s, w = startNode(cfg, nil)
|
|
id, n, s, w = startNode(cfg, nil)
|
|
|
case !haveWAL && cfg.NewCluster:
|
|
case !haveWAL && cfg.NewCluster:
|
|
|
if err := cfg.VerifyBootstrapConfig(); err != nil {
|
|
if err := cfg.VerifyBootstrapConfig(); err != nil {
|
|
|
return nil, err
|
|
return nil, err
|
|
|
}
|
|
}
|
|
|
|
|
+ if err := checkClientURLsEmptyFromPeers(cfg.Cluster, cfg.Name); err != nil {
|
|
|
|
|
+ return nil, err
|
|
|
|
|
+ }
|
|
|
m := cfg.Cluster.MemberByName(cfg.Name)
|
|
m := cfg.Cluster.MemberByName(cfg.Name)
|
|
|
if cfg.ShouldDiscover() {
|
|
if cfg.ShouldDiscover() {
|
|
|
s, err := discovery.JoinCluster(cfg.DiscoveryURL, cfg.DiscoveryProxy, m.ID, cfg.Cluster.String())
|
|
s, err := discovery.JoinCluster(cfg.DiscoveryURL, cfg.DiscoveryProxy, m.ID, cfg.Cluster.String())
|
|
@@ -229,7 +236,7 @@ func NewServer(cfg *ServerConfig) (*EtcdServer, error) {
|
|
|
}
|
|
}
|
|
|
}
|
|
}
|
|
|
cfg.Cluster.SetStore(st)
|
|
cfg.Cluster.SetStore(st)
|
|
|
- log.Printf("etcdserver: initial cluster members: %s", cfg.Cluster)
|
|
|
|
|
|
|
+ cfg.PrintWithInitial()
|
|
|
id, n, s, w = startNode(cfg, cfg.Cluster.MemberIDs())
|
|
id, n, s, w = startNode(cfg, cfg.Cluster.MemberIDs())
|
|
|
case haveWAL:
|
|
case haveWAL:
|
|
|
if cfg.ShouldDiscover() {
|
|
if cfg.ShouldDiscover() {
|
|
@@ -246,6 +253,7 @@ func NewServer(cfg *ServerConfig) (*EtcdServer, error) {
|
|
|
index = snapshot.Index
|
|
index = snapshot.Index
|
|
|
}
|
|
}
|
|
|
cfg.Cluster = NewClusterFromStore(cfg.Cluster.token, st)
|
|
cfg.Cluster = NewClusterFromStore(cfg.Cluster.token, st)
|
|
|
|
|
+ cfg.Print()
|
|
|
if snapshot != nil {
|
|
if snapshot != nil {
|
|
|
log.Printf("etcdserver: loaded peers from snapshot: %s", cfg.Cluster)
|
|
log.Printf("etcdserver: loaded peers from snapshot: %s", cfg.Cluster)
|
|
|
}
|
|
}
|
|
@@ -304,7 +312,7 @@ func (s *EtcdServer) start() {
|
|
|
}
|
|
}
|
|
|
s.w = wait.New()
|
|
s.w = wait.New()
|
|
|
s.done = make(chan struct{})
|
|
s.done = make(chan struct{})
|
|
|
- s.stopped = make(chan struct{})
|
|
|
|
|
|
|
+ s.stop = make(chan struct{})
|
|
|
s.stats.Initialize()
|
|
s.stats.Initialize()
|
|
|
// TODO: if this is an empty log, writes all peer infos
|
|
// TODO: if this is an empty log, writes all peer infos
|
|
|
// into the first entry
|
|
// into the first entry
|
|
@@ -325,12 +333,21 @@ func (s *EtcdServer) run() {
|
|
|
// snapi indicates the index of the last submitted snapshot request
|
|
// snapi indicates the index of the last submitted snapshot request
|
|
|
var snapi, appliedi uint64
|
|
var snapi, appliedi uint64
|
|
|
var nodes []uint64
|
|
var nodes []uint64
|
|
|
|
|
+ var shouldstop bool
|
|
|
|
|
+ shouldstopC := s.sender.ShouldStopNotify()
|
|
|
|
|
+
|
|
|
|
|
+ defer func() {
|
|
|
|
|
+ s.node.Stop()
|
|
|
|
|
+ s.sender.Stop()
|
|
|
|
|
+ close(s.done)
|
|
|
|
|
+ }()
|
|
|
for {
|
|
for {
|
|
|
select {
|
|
select {
|
|
|
case <-s.Ticker:
|
|
case <-s.Ticker:
|
|
|
s.node.Tick()
|
|
s.node.Tick()
|
|
|
case rd := <-s.node.Ready():
|
|
case rd := <-s.node.Ready():
|
|
|
if rd.SoftState != nil {
|
|
if rd.SoftState != nil {
|
|
|
|
|
+ atomic.StoreUint64(&s.raftLead, rd.SoftState.Lead)
|
|
|
nodes = rd.SoftState.Nodes
|
|
nodes = rd.SoftState.Nodes
|
|
|
if rd.RaftState == raft.StateLeader {
|
|
if rd.RaftState == raft.StateLeader {
|
|
|
syncC = s.SyncTicker
|
|
syncC = s.SyncTicker
|
|
@@ -348,35 +365,48 @@ func (s *EtcdServer) run() {
|
|
|
}
|
|
}
|
|
|
s.sender.Send(rd.Messages)
|
|
s.sender.Send(rd.Messages)
|
|
|
|
|
|
|
|
- // TODO(bmizerany): do this in the background, but take
|
|
|
|
|
- // care to apply entries in a single goroutine, and not
|
|
|
|
|
- // race them.
|
|
|
|
|
- if len(rd.CommittedEntries) != 0 {
|
|
|
|
|
- appliedi = s.apply(rd.CommittedEntries)
|
|
|
|
|
- }
|
|
|
|
|
-
|
|
|
|
|
- if rd.Snapshot.Index > snapi {
|
|
|
|
|
- snapi = rd.Snapshot.Index
|
|
|
|
|
- }
|
|
|
|
|
-
|
|
|
|
|
// recover from snapshot if it is more updated than current applied
|
|
// recover from snapshot if it is more updated than current applied
|
|
|
if rd.Snapshot.Index > appliedi {
|
|
if rd.Snapshot.Index > appliedi {
|
|
|
if err := s.store.Recovery(rd.Snapshot.Data); err != nil {
|
|
if err := s.store.Recovery(rd.Snapshot.Data); err != nil {
|
|
|
log.Panicf("recovery store error: %v", err)
|
|
log.Panicf("recovery store error: %v", err)
|
|
|
}
|
|
}
|
|
|
|
|
+ s.Cluster.Recover()
|
|
|
appliedi = rd.Snapshot.Index
|
|
appliedi = rd.Snapshot.Index
|
|
|
}
|
|
}
|
|
|
|
|
+ // TODO(bmizerany): do this in the background, but take
|
|
|
|
|
+ // care to apply entries in a single goroutine, and not
|
|
|
|
|
+ // race them.
|
|
|
|
|
+ if len(rd.CommittedEntries) != 0 {
|
|
|
|
|
+ firsti := rd.CommittedEntries[0].Index
|
|
|
|
|
+ if appliedi == 0 {
|
|
|
|
|
+ appliedi = firsti - 1
|
|
|
|
|
+ }
|
|
|
|
|
+ if firsti > appliedi+1 {
|
|
|
|
|
+ log.Panicf("etcdserver: first index of committed entry[%d] should <= appliedi[%d] + 1", firsti, appliedi)
|
|
|
|
|
+ }
|
|
|
|
|
+ var ents []raftpb.Entry
|
|
|
|
|
+ if appliedi+1-firsti < uint64(len(rd.CommittedEntries)) {
|
|
|
|
|
+ ents = rd.CommittedEntries[appliedi+1-firsti:]
|
|
|
|
|
+ }
|
|
|
|
|
+ if appliedi, shouldstop = s.apply(ents); shouldstop {
|
|
|
|
|
+ return
|
|
|
|
|
+ }
|
|
|
|
|
+ }
|
|
|
|
|
|
|
|
s.node.Advance()
|
|
s.node.Advance()
|
|
|
|
|
|
|
|
|
|
+ if rd.Snapshot.Index > snapi {
|
|
|
|
|
+ snapi = rd.Snapshot.Index
|
|
|
|
|
+ }
|
|
|
if appliedi-snapi > s.snapCount {
|
|
if appliedi-snapi > s.snapCount {
|
|
|
s.snapshot(appliedi, nodes)
|
|
s.snapshot(appliedi, nodes)
|
|
|
snapi = appliedi
|
|
snapi = appliedi
|
|
|
}
|
|
}
|
|
|
case <-syncC:
|
|
case <-syncC:
|
|
|
s.sync(defaultSyncTimeout)
|
|
s.sync(defaultSyncTimeout)
|
|
|
- case <-s.done:
|
|
|
|
|
- close(s.stopped)
|
|
|
|
|
|
|
+ case <-shouldstopC:
|
|
|
|
|
+ return
|
|
|
|
|
+ case <-s.stop:
|
|
|
return
|
|
return
|
|
|
}
|
|
}
|
|
|
}
|
|
}
|
|
@@ -385,12 +415,18 @@ func (s *EtcdServer) run() {
|
|
|
// Stop stops the server gracefully, and shuts down the running goroutine.
|
|
// Stop stops the server gracefully, and shuts down the running goroutine.
|
|
|
// Stop should be called after a Start(s), otherwise it will block forever.
|
|
// Stop should be called after a Start(s), otherwise it will block forever.
|
|
|
func (s *EtcdServer) Stop() {
|
|
func (s *EtcdServer) Stop() {
|
|
|
- s.node.Stop()
|
|
|
|
|
- close(s.done)
|
|
|
|
|
- <-s.stopped
|
|
|
|
|
- s.sender.Stop()
|
|
|
|
|
|
|
+ select {
|
|
|
|
|
+ case s.stop <- struct{}{}:
|
|
|
|
|
+ case <-s.done:
|
|
|
|
|
+ return
|
|
|
|
|
+ }
|
|
|
|
|
+ <-s.done
|
|
|
}
|
|
}
|
|
|
|
|
|
|
|
|
|
+// StopNotify returns a channel that receives a empty struct
|
|
|
|
|
+// when the server is stopped.
|
|
|
|
|
+func (s *EtcdServer) StopNotify() <-chan struct{} { return s.done }
|
|
|
|
|
+
|
|
|
// Do interprets r and performs an operation on s.store according to r.Method
|
|
// Do interprets r and performs an operation on s.store according to r.Method
|
|
|
// and other fields. If r.Method is "POST", "PUT", "DELETE", or a "GET" with
|
|
// and other fields. If r.Method is "POST", "PUT", "DELETE", or a "GET" with
|
|
|
// Quorum == true, r will be sent through consensus before performing its
|
|
// Quorum == true, r will be sent through consensus before performing its
|
|
@@ -447,18 +483,14 @@ func (s *EtcdServer) Do(ctx context.Context, r pb.Request) (Response, error) {
|
|
|
}
|
|
}
|
|
|
}
|
|
}
|
|
|
|
|
|
|
|
-func (s *EtcdServer) SelfStats() []byte {
|
|
|
|
|
- return s.stats.JSON()
|
|
|
|
|
-}
|
|
|
|
|
|
|
+func (s *EtcdServer) SelfStats() []byte { return s.stats.JSON() }
|
|
|
|
|
|
|
|
func (s *EtcdServer) LeaderStats() []byte {
|
|
func (s *EtcdServer) LeaderStats() []byte {
|
|
|
// TODO(jonboulle): need to lock access to lstats, set it to nil when not leader, ...
|
|
// TODO(jonboulle): need to lock access to lstats, set it to nil when not leader, ...
|
|
|
return s.lstats.JSON()
|
|
return s.lstats.JSON()
|
|
|
}
|
|
}
|
|
|
|
|
|
|
|
-func (s *EtcdServer) StoreStats() []byte {
|
|
|
|
|
- return s.store.JsonStats()
|
|
|
|
|
-}
|
|
|
|
|
|
|
+func (s *EtcdServer) StoreStats() []byte { return s.store.JsonStats() }
|
|
|
|
|
|
|
|
func (s *EtcdServer) UpdateRecvApp(from types.ID, length int64) {
|
|
func (s *EtcdServer) UpdateRecvApp(from types.ID, length int64) {
|
|
|
s.stats.RecvAppendReq(from.String(), int(length))
|
|
s.stats.RecvAppendReq(from.String(), int(length))
|
|
@@ -503,13 +535,14 @@ func (s *EtcdServer) UpdateMember(ctx context.Context, memb Member) error {
|
|
|
}
|
|
}
|
|
|
|
|
|
|
|
// Implement the RaftTimer interface
|
|
// Implement the RaftTimer interface
|
|
|
-func (s *EtcdServer) Index() uint64 {
|
|
|
|
|
- return atomic.LoadUint64(&s.raftIndex)
|
|
|
|
|
-}
|
|
|
|
|
|
|
+func (s *EtcdServer) Index() uint64 { return atomic.LoadUint64(&s.raftIndex) }
|
|
|
|
|
|
|
|
-func (s *EtcdServer) Term() uint64 {
|
|
|
|
|
- return atomic.LoadUint64(&s.raftTerm)
|
|
|
|
|
-}
|
|
|
|
|
|
|
+func (s *EtcdServer) Term() uint64 { return atomic.LoadUint64(&s.raftTerm) }
|
|
|
|
|
+
|
|
|
|
|
+// Only for testing purpose
|
|
|
|
|
+// TODO: add Raft server interface to expose raft related info:
|
|
|
|
|
+// Index, Term, Lead, Committed, Applied, LastIndex, etc.
|
|
|
|
|
+func (s *EtcdServer) Lead() uint64 { return atomic.LoadUint64(&s.raftLead) }
|
|
|
|
|
|
|
|
// configure sends a configuration change through consensus and
|
|
// configure sends a configuration change through consensus and
|
|
|
// then waits for it to be applied to the server. It
|
|
// then waits for it to be applied to the server. It
|
|
@@ -570,7 +603,7 @@ func (s *EtcdServer) publish(retryInterval time.Duration) {
|
|
|
req := pb.Request{
|
|
req := pb.Request{
|
|
|
ID: GenID(),
|
|
ID: GenID(),
|
|
|
Method: "PUT",
|
|
Method: "PUT",
|
|
|
- Path: path.Join(memberStoreKey(s.id), attributesSuffix),
|
|
|
|
|
|
|
+ Path: MemberAttributesStorePath(s.id),
|
|
|
Val: string(b),
|
|
Val: string(b),
|
|
|
}
|
|
}
|
|
|
|
|
|
|
@@ -601,7 +634,7 @@ func getExpirationTime(r *pb.Request) time.Time {
|
|
|
|
|
|
|
|
// apply takes an Entry received from Raft (after it has been committed) and
|
|
// apply takes an Entry received from Raft (after it has been committed) and
|
|
|
// applies it to the current state of the EtcdServer
|
|
// applies it to the current state of the EtcdServer
|
|
|
-func (s *EtcdServer) apply(es []raftpb.Entry) uint64 {
|
|
|
|
|
|
|
+func (s *EtcdServer) apply(es []raftpb.Entry) (uint64, bool) {
|
|
|
var applied uint64
|
|
var applied uint64
|
|
|
for i := range es {
|
|
for i := range es {
|
|
|
e := es[i]
|
|
e := es[i]
|
|
@@ -613,7 +646,11 @@ func (s *EtcdServer) apply(es []raftpb.Entry) uint64 {
|
|
|
case raftpb.EntryConfChange:
|
|
case raftpb.EntryConfChange:
|
|
|
var cc raftpb.ConfChange
|
|
var cc raftpb.ConfChange
|
|
|
pbutil.MustUnmarshal(&cc, e.Data)
|
|
pbutil.MustUnmarshal(&cc, e.Data)
|
|
|
- s.w.Trigger(cc.ID, s.applyConfChange(cc))
|
|
|
|
|
|
|
+ shouldstop, err := s.applyConfChange(cc)
|
|
|
|
|
+ s.w.Trigger(cc.ID, err)
|
|
|
|
|
+ if shouldstop {
|
|
|
|
|
+ return applied, true
|
|
|
|
|
+ }
|
|
|
default:
|
|
default:
|
|
|
log.Panicf("entry type should be either EntryNormal or EntryConfChange")
|
|
log.Panicf("entry type should be either EntryNormal or EntryConfChange")
|
|
|
}
|
|
}
|
|
@@ -621,7 +658,7 @@ func (s *EtcdServer) apply(es []raftpb.Entry) uint64 {
|
|
|
atomic.StoreUint64(&s.raftTerm, e.Term)
|
|
atomic.StoreUint64(&s.raftTerm, e.Term)
|
|
|
applied = e.Index
|
|
applied = e.Index
|
|
|
}
|
|
}
|
|
|
- return applied
|
|
|
|
|
|
|
+ return applied, false
|
|
|
}
|
|
}
|
|
|
|
|
|
|
|
// applyRequest interprets r as a call to store.X and returns a Response interpreted
|
|
// applyRequest interprets r as a call to store.X and returns a Response interpreted
|
|
@@ -675,11 +712,11 @@ func (s *EtcdServer) applyRequest(r pb.Request) Response {
|
|
|
|
|
|
|
|
// applyConfChange applies a ConfChange to the server. It is only
|
|
// applyConfChange applies a ConfChange to the server. It is only
|
|
|
// invoked with a ConfChange that has already passed through Raft
|
|
// invoked with a ConfChange that has already passed through Raft
|
|
|
-func (s *EtcdServer) applyConfChange(cc raftpb.ConfChange) error {
|
|
|
|
|
|
|
+func (s *EtcdServer) applyConfChange(cc raftpb.ConfChange) (bool, error) {
|
|
|
if err := s.Cluster.ValidateConfigurationChange(cc); err != nil {
|
|
if err := s.Cluster.ValidateConfigurationChange(cc); err != nil {
|
|
|
cc.NodeID = raft.None
|
|
cc.NodeID = raft.None
|
|
|
s.node.ApplyConfChange(cc)
|
|
s.node.ApplyConfChange(cc)
|
|
|
- return err
|
|
|
|
|
|
|
+ return false, err
|
|
|
}
|
|
}
|
|
|
s.node.ApplyConfChange(cc)
|
|
s.node.ApplyConfChange(cc)
|
|
|
switch cc.Type {
|
|
switch cc.Type {
|
|
@@ -703,6 +740,8 @@ func (s *EtcdServer) applyConfChange(cc raftpb.ConfChange) error {
|
|
|
s.Cluster.RemoveMember(id)
|
|
s.Cluster.RemoveMember(id)
|
|
|
if id == s.id {
|
|
if id == s.id {
|
|
|
log.Printf("etcdserver: removed local member %s from cluster %s", id, s.Cluster.ID())
|
|
log.Printf("etcdserver: removed local member %s from cluster %s", id, s.Cluster.ID())
|
|
|
|
|
+ log.Println("etcdserver: the data-dir used by this member must be removed so that this host can be re-added with a new member ID")
|
|
|
|
|
+ return true, nil
|
|
|
} else {
|
|
} else {
|
|
|
s.sender.Remove(id)
|
|
s.sender.Remove(id)
|
|
|
log.Printf("etcdserver: removed member %s from cluster %s", id, s.Cluster.ID())
|
|
log.Printf("etcdserver: removed member %s from cluster %s", id, s.Cluster.ID())
|
|
@@ -723,7 +762,7 @@ func (s *EtcdServer) applyConfChange(cc raftpb.ConfChange) error {
|
|
|
log.Printf("etcdserver: update member %s %v in cluster %s", m.ID, m.PeerURLs, s.Cluster.ID())
|
|
log.Printf("etcdserver: update member %s %v in cluster %s", m.ID, m.PeerURLs, s.Cluster.ID())
|
|
|
}
|
|
}
|
|
|
}
|
|
}
|
|
|
- return nil
|
|
|
|
|
|
|
+ return false, nil
|
|
|
}
|
|
}
|
|
|
|
|
|
|
|
// TODO: non-blocking snapshot
|
|
// TODO: non-blocking snapshot
|
|
@@ -740,31 +779,70 @@ func (s *EtcdServer) snapshot(snapi uint64, snapnodes []uint64) {
|
|
|
}
|
|
}
|
|
|
}
|
|
}
|
|
|
|
|
|
|
|
|
|
+// checkClientURLsEmptyFromPeers does its best to get the cluster from peers,
|
|
|
|
|
+// and if this succeeds, checks that the member of the given id exists in the
|
|
|
|
|
+// cluster, and its ClientURLs is empty.
|
|
|
|
|
+func checkClientURLsEmptyFromPeers(cl *Cluster, name string) error {
|
|
|
|
|
+ us := getOtherPeerURLs(cl, name)
|
|
|
|
|
+ rcl, err := getClusterFromPeers(us, false)
|
|
|
|
|
+ if err != nil {
|
|
|
|
|
+ return nil
|
|
|
|
|
+ }
|
|
|
|
|
+ id := cl.MemberByName(name).ID
|
|
|
|
|
+ m := rcl.Member(id)
|
|
|
|
|
+ if m == nil {
|
|
|
|
|
+ return nil
|
|
|
|
|
+ }
|
|
|
|
|
+ if len(m.ClientURLs) > 0 {
|
|
|
|
|
+ return fmt.Errorf("etcdserver: member with id %s has started and registered its client urls", id)
|
|
|
|
|
+ }
|
|
|
|
|
+ return nil
|
|
|
|
|
+}
|
|
|
|
|
+
|
|
|
// GetClusterFromPeers takes a set of URLs representing etcd peers, and
|
|
// GetClusterFromPeers takes a set of URLs representing etcd peers, and
|
|
|
// attempts to construct a Cluster by accessing the members endpoint on one of
|
|
// attempts to construct a Cluster by accessing the members endpoint on one of
|
|
|
// these URLs. The first URL to provide a response is used. If no URLs provide
|
|
// these URLs. The first URL to provide a response is used. If no URLs provide
|
|
|
// a response, or a Cluster cannot be successfully created from a received
|
|
// a response, or a Cluster cannot be successfully created from a received
|
|
|
// response, an error is returned.
|
|
// response, an error is returned.
|
|
|
func GetClusterFromPeers(urls []string) (*Cluster, error) {
|
|
func GetClusterFromPeers(urls []string) (*Cluster, error) {
|
|
|
|
|
+ return getClusterFromPeers(urls, true)
|
|
|
|
|
+}
|
|
|
|
|
+
|
|
|
|
|
+// If logerr is true, it prints out more error messages.
|
|
|
|
|
+func getClusterFromPeers(urls []string, logerr bool) (*Cluster, error) {
|
|
|
|
|
+ cc := &http.Client{
|
|
|
|
|
+ Transport: &http.Transport{
|
|
|
|
|
+ ResponseHeaderTimeout: 500 * time.Millisecond,
|
|
|
|
|
+ },
|
|
|
|
|
+ Timeout: time.Second,
|
|
|
|
|
+ }
|
|
|
for _, u := range urls {
|
|
for _, u := range urls {
|
|
|
- resp, err := http.Get(u + "/members")
|
|
|
|
|
|
|
+ resp, err := cc.Get(u + "/members")
|
|
|
if err != nil {
|
|
if err != nil {
|
|
|
- log.Printf("etcdserver: could not get cluster response from %s: %v", u, err)
|
|
|
|
|
|
|
+ if logerr {
|
|
|
|
|
+ log.Printf("etcdserver: could not get cluster response from %s: %v", u, err)
|
|
|
|
|
+ }
|
|
|
continue
|
|
continue
|
|
|
}
|
|
}
|
|
|
b, err := ioutil.ReadAll(resp.Body)
|
|
b, err := ioutil.ReadAll(resp.Body)
|
|
|
if err != nil {
|
|
if err != nil {
|
|
|
- log.Printf("etcdserver: could not read the body of cluster response: %v", err)
|
|
|
|
|
|
|
+ if logerr {
|
|
|
|
|
+ log.Printf("etcdserver: could not read the body of cluster response: %v", err)
|
|
|
|
|
+ }
|
|
|
continue
|
|
continue
|
|
|
}
|
|
}
|
|
|
var membs []*Member
|
|
var membs []*Member
|
|
|
if err := json.Unmarshal(b, &membs); err != nil {
|
|
if err := json.Unmarshal(b, &membs); err != nil {
|
|
|
- log.Printf("etcdserver: could not unmarshal cluster response: %v", err)
|
|
|
|
|
|
|
+ if logerr {
|
|
|
|
|
+ log.Printf("etcdserver: could not unmarshal cluster response: %v", err)
|
|
|
|
|
+ }
|
|
|
continue
|
|
continue
|
|
|
}
|
|
}
|
|
|
id, err := types.IDFromString(resp.Header.Get("X-Etcd-Cluster-ID"))
|
|
id, err := types.IDFromString(resp.Header.Get("X-Etcd-Cluster-ID"))
|
|
|
if err != nil {
|
|
if err != nil {
|
|
|
- log.Printf("etcdserver: could not parse the cluster ID from cluster res: %v", err)
|
|
|
|
|
|
|
+ if logerr {
|
|
|
|
|
+ log.Printf("etcdserver: could not parse the cluster ID from cluster res: %v", err)
|
|
|
|
|
+ }
|
|
|
continue
|
|
continue
|
|
|
}
|
|
}
|
|
|
return NewClusterFromMembers("", id, membs), nil
|
|
return NewClusterFromMembers("", id, membs), nil
|