forked from vitessio/vitess
-
Notifications
You must be signed in to change notification settings - Fork 1
/
election.go
107 lines (90 loc) · 2.98 KB
/
election.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
// Copyright 2016, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
package etcd2topo
import (
"path"
"github.com/coreos/etcd/clientv3"
log "github.com/golang/glog"
"golang.org/x/net/context"
"github.com/youtube/vitess/go/vt/topo"
)
// NewMasterParticipation is part of the topo.Server interface
func (s *Server) NewMasterParticipation(name, id string) (topo.MasterParticipation, error) {
return &etcdMasterParticipation{
s: s,
name: name,
id: id,
stop: make(chan struct{}),
done: make(chan struct{}),
}, nil
}
// etcdMasterParticipation implements topo.MasterParticipation.
//
// We use a directory (in global election path, with the name) with
// ephemeral files in it, that contains the id. The oldest revision
// wins the election.
type etcdMasterParticipation struct {
// s is our parent etcd topo Server
s *Server
// name is the name of this MasterParticipation
name string
// id is the process's current id.
id string
// stop is a channel closed when Stop is called.
stop chan struct{}
// done is a channel closed when we're done processing the Stop
done chan struct{}
}
// WaitForMastership is part of the topo.MasterParticipation interface.
func (mp *etcdMasterParticipation) WaitForMastership() (context.Context, error) {
electionPath := path.Join(mp.s.global.root, electionsPath, mp.name)
lockPath := ""
// We use a cancelable context here. If stop is closed,
// we just cancel that context.
lockCtx, lockCancel := context.WithCancel(context.Background())
go func() {
select {
case <-mp.stop:
if lockPath != "" {
if err := mp.s.unlock(context.Background(), electionPath, lockPath); err != nil {
log.Errorf("failed to delete lockPath %v for election %v: %v", lockPath, mp.name, err)
}
}
lockCancel()
close(mp.done)
}
}()
// Try to get the mastership, by getting a lock.
var err error
lockPath, err = mp.s.lock(lockCtx, electionPath, mp.id)
if err != nil {
// It can be that we were interrupted.
return nil, err
}
// We got the lock. Return the lockContext. If Stop() is called,
// it will cancel the lockCtx, and cancel the returned context.
return lockCtx, nil
}
// Stop is part of the topo.MasterParticipation interface
func (mp *etcdMasterParticipation) Stop() {
close(mp.stop)
<-mp.done
}
// GetCurrentMasterID is part of the topo.MasterParticipation interface
func (mp *etcdMasterParticipation) GetCurrentMasterID(ctx context.Context) (string, error) {
electionPath := path.Join(mp.s.global.root, electionsPath, mp.name)
// Get the keys in the directory, older first.
resp, err := mp.s.global.cli.Get(ctx, electionPath+"/",
clientv3.WithPrefix(),
clientv3.WithSort(clientv3.SortByModRevision, clientv3.SortAscend),
clientv3.WithLimit(1))
if err != nil {
return "", convertError(err)
}
if len(resp.Kvs) == 0 {
// No key starts with this prefix, means nobody is the master.
return "", nil
}
return string(resp.Kvs[0].Value), nil
}