mirror of
https://github.com/etcd-io/etcd.git
synced 2024-09-27 06:25:44 +00:00
Merge branch '0.2' of https://github.com/coreos/etcd into 0.2
This commit is contained in:
commit
2b6c628342
12
etcd_test.go
12
etcd_test.go
@ -32,7 +32,7 @@ func TestSingleNode(t *testing.T) {
|
||||
|
||||
time.Sleep(time.Second)
|
||||
|
||||
c := etcd.NewClient()
|
||||
c := etcd.NewClient(nil)
|
||||
|
||||
c.SyncCluster()
|
||||
// Test Set
|
||||
@ -145,7 +145,7 @@ func TestSingleNodeRecovery(t *testing.T) {
|
||||
|
||||
time.Sleep(time.Second)
|
||||
|
||||
c := etcd.NewClient()
|
||||
c := etcd.NewClient(nil)
|
||||
|
||||
c.SyncCluster()
|
||||
// Test Set
|
||||
@ -205,7 +205,7 @@ func templateTestSimpleMultiNode(t *testing.T, tls bool) {
|
||||
|
||||
time.Sleep(time.Second)
|
||||
|
||||
c := etcd.NewClient()
|
||||
c := etcd.NewClient(nil)
|
||||
|
||||
c.SyncCluster()
|
||||
|
||||
@ -255,7 +255,7 @@ func TestMultiNodeKillAllAndRecovery(t *testing.T) {
|
||||
t.Fatal("cannot create cluster")
|
||||
}
|
||||
|
||||
c := etcd.NewClient()
|
||||
c := etcd.NewClient(nil)
|
||||
|
||||
c.SyncCluster()
|
||||
|
||||
@ -320,7 +320,7 @@ func TestMultiNodeKillOne(t *testing.T) {
|
||||
|
||||
time.Sleep(2 * time.Second)
|
||||
|
||||
c := etcd.NewClient()
|
||||
c := etcd.NewClient(nil)
|
||||
|
||||
c.SyncCluster()
|
||||
|
||||
@ -481,7 +481,7 @@ func TestRemoveNode(t *testing.T) {
|
||||
|
||||
time.Sleep(time.Second)
|
||||
|
||||
c := etcd.NewClient()
|
||||
c := etcd.NewClient(nil)
|
||||
|
||||
c.SyncCluster()
|
||||
|
||||
|
@ -35,7 +35,7 @@ func (c *JoinCommand) CommandName() string {
|
||||
}
|
||||
|
||||
// Join a server to the cluster
|
||||
func (c *JoinCommand) Apply(server *raft.Server) (interface{}, error) {
|
||||
func (c *JoinCommand) Apply(server raft.Server) (interface{}, error) {
|
||||
ps, _ := server.Context().(*PeerServer)
|
||||
|
||||
b := make([]byte, 8)
|
||||
@ -62,7 +62,7 @@ func (c *JoinCommand) Apply(server *raft.Server) (interface{}, error) {
|
||||
err := server.AddPeer(c.Name, "")
|
||||
|
||||
// Add peer stats
|
||||
if c.Name != ps.Name() {
|
||||
if c.Name != ps.RaftServer().Name() {
|
||||
ps.followersStats.Followers[c.Name] = &raftFollowerStats{}
|
||||
ps.followersStats.Followers[c.Name].Latency.Minimum = 1 << 63
|
||||
}
|
||||
|
@ -19,7 +19,7 @@ import (
|
||||
)
|
||||
|
||||
type PeerServer struct {
|
||||
*raft.Server
|
||||
raftServer raft.Server
|
||||
server *Server
|
||||
joinIndex uint64
|
||||
name string
|
||||
@ -30,7 +30,7 @@ type PeerServer struct {
|
||||
followersStats *raftFollowersStats
|
||||
serverStats *raftServerStats
|
||||
registry *Registry
|
||||
store *store.Store
|
||||
store store.Store
|
||||
snapConf *snapshotConf
|
||||
MaxClusterSize int
|
||||
RetryTimes int
|
||||
@ -49,7 +49,7 @@ type snapshotConf struct {
|
||||
writesThr uint64
|
||||
}
|
||||
|
||||
func NewPeerServer(name string, path string, url string, listenHost string, tlsConf *TLSConfig, tlsInfo *TLSInfo, registry *Registry, store *store.Store) *PeerServer {
|
||||
func NewPeerServer(name string, path string, url string, listenHost string, tlsConf *TLSConfig, tlsInfo *TLSInfo, registry *Registry, store store.Store) *PeerServer {
|
||||
s := &PeerServer{
|
||||
name: name,
|
||||
url: url,
|
||||
@ -78,12 +78,12 @@ func NewPeerServer(name string, path string, url string, listenHost string, tlsC
|
||||
raftTransporter := newTransporter(tlsConf.Scheme, tlsConf.Client, s)
|
||||
|
||||
// Create raft server
|
||||
server, err := raft.NewServer(name, path, raftTransporter, s.store, s, "")
|
||||
raftServer, err := raft.NewServer(name, path, raftTransporter, s.store, s, "")
|
||||
if err != nil {
|
||||
log.Fatal(err)
|
||||
}
|
||||
|
||||
s.Server = server
|
||||
s.raftServer = raftServer
|
||||
|
||||
return s
|
||||
}
|
||||
@ -92,7 +92,7 @@ func NewPeerServer(name string, path string, url string, listenHost string, tlsC
|
||||
func (s *PeerServer) ListenAndServe(snapshot bool, cluster []string) {
|
||||
// LoadSnapshot
|
||||
if snapshot {
|
||||
err := s.LoadSnapshot()
|
||||
err := s.raftServer.LoadSnapshot()
|
||||
|
||||
if err == nil {
|
||||
log.Debugf("%s finished load snapshot", s.name)
|
||||
@ -101,12 +101,12 @@ func (s *PeerServer) ListenAndServe(snapshot bool, cluster []string) {
|
||||
}
|
||||
}
|
||||
|
||||
s.SetElectionTimeout(ElectionTimeout)
|
||||
s.SetHeartbeatTimeout(HeartbeatTimeout)
|
||||
s.raftServer.SetElectionTimeout(ElectionTimeout)
|
||||
s.raftServer.SetHeartbeatTimeout(HeartbeatTimeout)
|
||||
|
||||
s.Start()
|
||||
s.raftServer.Start()
|
||||
|
||||
if s.IsLogEmpty() {
|
||||
if s.raftServer.IsLogEmpty() {
|
||||
// start as a leader in a new cluster
|
||||
if len(cluster) == 0 {
|
||||
s.startAsLeader()
|
||||
@ -116,7 +116,7 @@ func (s *PeerServer) ListenAndServe(snapshot bool, cluster []string) {
|
||||
|
||||
} else {
|
||||
// Rejoin the previous cluster
|
||||
cluster = s.registry.PeerURLs(s.Leader(), s.name)
|
||||
cluster = s.registry.PeerURLs(s.raftServer.Leader(), s.name)
|
||||
for i := 0; i < len(cluster); i++ {
|
||||
u, err := url.Parse(cluster[i])
|
||||
if err != nil {
|
||||
@ -143,8 +143,8 @@ func (s *PeerServer) ListenAndServe(snapshot bool, cluster []string) {
|
||||
}
|
||||
|
||||
// Retrieves the underlying Raft server.
|
||||
func (s *PeerServer) RaftServer() *raft.Server {
|
||||
return s.Server
|
||||
func (s *PeerServer) RaftServer() raft.Server {
|
||||
return s.raftServer
|
||||
}
|
||||
|
||||
// Associates the client server with the peer server.
|
||||
@ -155,7 +155,7 @@ func (s *PeerServer) SetServer(server *Server) {
|
||||
func (s *PeerServer) startAsLeader() {
|
||||
// leader need to join self as a peer
|
||||
for {
|
||||
_, err := s.Do(NewJoinCommand(PeerVersion, s.Name(), s.url, s.server.URL()))
|
||||
_, err := s.raftServer.Do(NewJoinCommand(PeerVersion, s.raftServer.Name(), s.url, s.server.URL()))
|
||||
if err == nil {
|
||||
break
|
||||
}
|
||||
@ -232,7 +232,7 @@ func (s *PeerServer) joinCluster(cluster []string) bool {
|
||||
continue
|
||||
}
|
||||
|
||||
err := s.joinByMachine(s.Server, machine, s.tlsConf.Scheme)
|
||||
err := s.joinByMachine(s.raftServer, machine, s.tlsConf.Scheme)
|
||||
if err == nil {
|
||||
log.Debugf("%s success join to the cluster via machine %s", s.name, machine)
|
||||
return true
|
||||
@ -249,7 +249,7 @@ func (s *PeerServer) joinCluster(cluster []string) bool {
|
||||
}
|
||||
|
||||
// Send join requests to machine.
|
||||
func (s *PeerServer) joinByMachine(server *raft.Server, machine string, scheme string) error {
|
||||
func (s *PeerServer) joinByMachine(server raft.Server, machine string, scheme string) error {
|
||||
var b bytes.Buffer
|
||||
|
||||
// t must be ok
|
||||
@ -327,7 +327,7 @@ func (s *PeerServer) Stats() []byte {
|
||||
}
|
||||
|
||||
func (s *PeerServer) PeerStats() []byte {
|
||||
if s.State() == raft.Leader {
|
||||
if s.raftServer.State() == raft.Leader {
|
||||
b, _ := json.Marshal(s.followersStats)
|
||||
return b
|
||||
}
|
||||
@ -339,15 +339,15 @@ func (s *PeerServer) monitorSnapshot() {
|
||||
time.Sleep(s.snapConf.checkingInterval)
|
||||
currentWrites := 0
|
||||
if uint64(currentWrites) > s.snapConf.writesThr {
|
||||
s.TakeSnapshot()
|
||||
s.raftServer.TakeSnapshot()
|
||||
s.snapConf.lastWrites = 0
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (s *PeerServer) dispatch(c raft.Command, w http.ResponseWriter, req *http.Request) error {
|
||||
if s.State() == raft.Leader {
|
||||
result, err := s.Do(c)
|
||||
if s.raftServer.State() == raft.Leader {
|
||||
result, err := s.raftServer.Do(c)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@ -375,7 +375,7 @@ func (s *PeerServer) dispatch(c raft.Command, w http.ResponseWriter, req *http.R
|
||||
return nil
|
||||
|
||||
} else {
|
||||
leader := s.Leader()
|
||||
leader := s.raftServer.Leader()
|
||||
|
||||
// No leader available.
|
||||
if leader == "" {
|
||||
|
@ -14,7 +14,7 @@ func (s *PeerServer) GetLogHttpHandler(w http.ResponseWriter, req *http.Request)
|
||||
log.Debugf("[recv] GET %s/log", s.url)
|
||||
w.Header().Set("Content-Type", "application/json")
|
||||
w.WriteHeader(http.StatusOK)
|
||||
json.NewEncoder(w).Encode(s.LogEntries())
|
||||
json.NewEncoder(w).Encode(s.raftServer.LogEntries())
|
||||
}
|
||||
|
||||
// Response to vote request
|
||||
@ -23,7 +23,7 @@ func (s *PeerServer) VoteHttpHandler(w http.ResponseWriter, req *http.Request) {
|
||||
err := decodeJsonRequest(req, rvreq)
|
||||
if err == nil {
|
||||
log.Debugf("[recv] POST %s/vote [%s]", s.url, rvreq.CandidateName)
|
||||
if resp := s.RequestVote(rvreq); resp != nil {
|
||||
if resp := s.raftServer.RequestVote(rvreq); resp != nil {
|
||||
w.WriteHeader(http.StatusOK)
|
||||
json.NewEncoder(w).Encode(resp)
|
||||
return
|
||||
@ -43,7 +43,7 @@ func (s *PeerServer) AppendEntriesHttpHandler(w http.ResponseWriter, req *http.R
|
||||
|
||||
s.serverStats.RecvAppendReq(aereq.LeaderName, int(req.ContentLength))
|
||||
|
||||
if resp := s.AppendEntries(aereq); resp != nil {
|
||||
if resp := s.raftServer.AppendEntries(aereq); resp != nil {
|
||||
w.WriteHeader(http.StatusOK)
|
||||
json.NewEncoder(w).Encode(resp)
|
||||
if !resp.Success {
|
||||
@ -62,7 +62,7 @@ func (s *PeerServer) SnapshotHttpHandler(w http.ResponseWriter, req *http.Reques
|
||||
err := decodeJsonRequest(req, aereq)
|
||||
if err == nil {
|
||||
log.Debugf("[recv] POST %s/snapshot/ ", s.url)
|
||||
if resp := s.RequestSnapshot(aereq); resp != nil {
|
||||
if resp := s.raftServer.RequestSnapshot(aereq); resp != nil {
|
||||
w.WriteHeader(http.StatusOK)
|
||||
json.NewEncoder(w).Encode(resp)
|
||||
return
|
||||
@ -78,7 +78,7 @@ func (s *PeerServer) SnapshotRecoveryHttpHandler(w http.ResponseWriter, req *htt
|
||||
err := decodeJsonRequest(req, aereq)
|
||||
if err == nil {
|
||||
log.Debugf("[recv] POST %s/snapshotRecovery/ ", s.url)
|
||||
if resp := s.SnapshotRecoveryRequest(aereq); resp != nil {
|
||||
if resp := s.raftServer.SnapshotRecoveryRequest(aereq); resp != nil {
|
||||
w.WriteHeader(http.StatusOK)
|
||||
json.NewEncoder(w).Encode(resp)
|
||||
return
|
||||
|
@ -18,7 +18,7 @@ const RegistryKey = "/_etcd/machines"
|
||||
// The Registry stores URL information for nodes.
|
||||
type Registry struct {
|
||||
sync.Mutex
|
||||
store *store.Store
|
||||
store store.Store
|
||||
nodes map[string]*node
|
||||
}
|
||||
|
||||
@ -30,7 +30,7 @@ type node struct {
|
||||
}
|
||||
|
||||
// Creates a new Registry.
|
||||
func NewRegistry(s *store.Store) *Registry {
|
||||
func NewRegistry(s store.Store) *Registry {
|
||||
return &Registry{
|
||||
store: s,
|
||||
nodes: make(map[string]*node),
|
||||
|
@ -23,7 +23,7 @@ func (c *RemoveCommand) CommandName() string {
|
||||
}
|
||||
|
||||
// Remove a server from the cluster
|
||||
func (c *RemoveCommand) Apply(server *raft.Server) (interface{}, error) {
|
||||
func (c *RemoveCommand) Apply(server raft.Server) (interface{}, error) {
|
||||
ps, _ := server.Context().(*PeerServer)
|
||||
|
||||
// Remove node from the shared registry.
|
||||
|
@ -21,7 +21,7 @@ type Server struct {
|
||||
http.Server
|
||||
peerServer *PeerServer
|
||||
registry *Registry
|
||||
store *store.Store
|
||||
store store.Store
|
||||
name string
|
||||
url string
|
||||
tlsConf *TLSConfig
|
||||
@ -30,7 +30,7 @@ type Server struct {
|
||||
}
|
||||
|
||||
// Creates a new Server.
|
||||
func New(name string, urlStr string, listenHost string, tlsConf *TLSConfig, tlsInfo *TLSInfo, peerServer *PeerServer, registry *Registry, store *store.Store) *Server {
|
||||
func New(name string, urlStr string, listenHost string, tlsConf *TLSConfig, tlsInfo *TLSInfo, peerServer *PeerServer, registry *Registry, store store.Store) *Server {
|
||||
s := &Server{
|
||||
Server: http.Server{
|
||||
Handler: mux.NewRouter(),
|
||||
@ -56,22 +56,22 @@ func New(name string, urlStr string, listenHost string, tlsConf *TLSConfig, tlsI
|
||||
|
||||
// The current state of the server in the cluster.
|
||||
func (s *Server) State() string {
|
||||
return s.peerServer.State()
|
||||
return s.peerServer.RaftServer().State()
|
||||
}
|
||||
|
||||
// The node name of the leader in the cluster.
|
||||
func (s *Server) Leader() string {
|
||||
return s.peerServer.Leader()
|
||||
return s.peerServer.RaftServer().Leader()
|
||||
}
|
||||
|
||||
// The current Raft committed index.
|
||||
func (s *Server) CommitIndex() uint64 {
|
||||
return s.peerServer.CommitIndex()
|
||||
return s.peerServer.RaftServer().CommitIndex()
|
||||
}
|
||||
|
||||
// The current Raft term.
|
||||
func (s *Server) Term() uint64 {
|
||||
return s.peerServer.Term()
|
||||
return s.peerServer.RaftServer().Term()
|
||||
}
|
||||
|
||||
// The server URL.
|
||||
@ -85,7 +85,7 @@ func (s *Server) PeerURL(name string) (string, bool) {
|
||||
}
|
||||
|
||||
// Returns a reference to the Store.
|
||||
func (s *Server) Store() *store.Store {
|
||||
func (s *Server) Store() store.Store {
|
||||
return s.store
|
||||
}
|
||||
|
||||
@ -201,7 +201,7 @@ func (s *Server) GetVersionHandler(w http.ResponseWriter, req *http.Request) err
|
||||
|
||||
// Handler to return the current leader's raft address
|
||||
func (s *Server) GetLeaderHandler(w http.ResponseWriter, req *http.Request) error {
|
||||
leader := s.peerServer.Leader()
|
||||
leader := s.peerServer.RaftServer().Leader()
|
||||
if leader == "" {
|
||||
return etcdErr.NewError(etcdErr.EcodeLeaderElect, "", store.UndefIndex, store.UndefTerm)
|
||||
}
|
||||
@ -213,7 +213,7 @@ func (s *Server) GetLeaderHandler(w http.ResponseWriter, req *http.Request) erro
|
||||
|
||||
// Handler to return all the known machines in the current cluster.
|
||||
func (s *Server) GetMachinesHandler(w http.ResponseWriter, req *http.Request) error {
|
||||
machines := s.registry.ClientURLs(s.peerServer.Leader(), s.name)
|
||||
machines := s.registry.ClientURLs(s.peerServer.RaftServer().Leader(), s.name)
|
||||
w.WriteHeader(http.StatusOK)
|
||||
w.Write([]byte(strings.Join(machines, ", ")))
|
||||
return nil
|
||||
@ -227,12 +227,12 @@ func (s *Server) GetStatsHandler(w http.ResponseWriter, req *http.Request) error
|
||||
|
||||
// Retrieves stats on the leader.
|
||||
func (s *Server) GetLeaderStatsHandler(w http.ResponseWriter, req *http.Request) error {
|
||||
if s.peerServer.State() == raft.Leader {
|
||||
if s.peerServer.RaftServer().State() == raft.Leader {
|
||||
w.Write(s.peerServer.PeerStats())
|
||||
return nil
|
||||
}
|
||||
|
||||
leader := s.peerServer.Leader()
|
||||
leader := s.peerServer.RaftServer().Leader()
|
||||
if leader == "" {
|
||||
return etcdErr.NewError(300, "", store.UndefIndex, store.UndefTerm)
|
||||
}
|
||||
@ -259,7 +259,7 @@ func (s *Server) SpeedTestHandler(w http.ResponseWriter, req *http.Request) erro
|
||||
Value: "bar",
|
||||
ExpireTime: time.Unix(0, 0),
|
||||
}
|
||||
s.peerServer.Do(c)
|
||||
s.peerServer.RaftServer().Do(c)
|
||||
}
|
||||
c <- true
|
||||
}()
|
||||
|
@ -62,7 +62,7 @@ func dialWithTimeout(network, addr string) (net.Conn, error) {
|
||||
}
|
||||
|
||||
// Sends AppendEntries RPCs to a peer when the server is the leader.
|
||||
func (t *transporter) SendAppendEntriesRequest(server *raft.Server, peer *raft.Peer, req *raft.AppendEntriesRequest) *raft.AppendEntriesResponse {
|
||||
func (t *transporter) SendAppendEntriesRequest(server raft.Server, peer *raft.Peer, req *raft.AppendEntriesRequest) *raft.AppendEntriesResponse {
|
||||
var aersp *raft.AppendEntriesResponse
|
||||
var b bytes.Buffer
|
||||
|
||||
@ -117,7 +117,7 @@ func (t *transporter) SendAppendEntriesRequest(server *raft.Server, peer *raft.P
|
||||
}
|
||||
|
||||
// Sends RequestVote RPCs to a peer when the server is the candidate.
|
||||
func (t *transporter) SendVoteRequest(server *raft.Server, peer *raft.Peer, req *raft.RequestVoteRequest) *raft.RequestVoteResponse {
|
||||
func (t *transporter) SendVoteRequest(server raft.Server, peer *raft.Peer, req *raft.RequestVoteRequest) *raft.RequestVoteResponse {
|
||||
var rvrsp *raft.RequestVoteResponse
|
||||
var b bytes.Buffer
|
||||
json.NewEncoder(&b).Encode(req)
|
||||
@ -146,7 +146,7 @@ func (t *transporter) SendVoteRequest(server *raft.Server, peer *raft.Peer, req
|
||||
}
|
||||
|
||||
// Sends SnapshotRequest RPCs to a peer when the server is the candidate.
|
||||
func (t *transporter) SendSnapshotRequest(server *raft.Server, peer *raft.Peer, req *raft.SnapshotRequest) *raft.SnapshotResponse {
|
||||
func (t *transporter) SendSnapshotRequest(server raft.Server, peer *raft.Peer, req *raft.SnapshotRequest) *raft.SnapshotResponse {
|
||||
var aersp *raft.SnapshotResponse
|
||||
var b bytes.Buffer
|
||||
json.NewEncoder(&b).Encode(req)
|
||||
@ -177,7 +177,7 @@ func (t *transporter) SendSnapshotRequest(server *raft.Server, peer *raft.Peer,
|
||||
}
|
||||
|
||||
// Sends SnapshotRecoveryRequest RPCs to a peer when the server is the candidate.
|
||||
func (t *transporter) SendSnapshotRecoveryRequest(server *raft.Server, peer *raft.Peer, req *raft.SnapshotRecoveryRequest) *raft.SnapshotRecoveryResponse {
|
||||
func (t *transporter) SendSnapshotRecoveryRequest(server raft.Server, peer *raft.Peer, req *raft.SnapshotRecoveryRequest) *raft.SnapshotRecoveryResponse {
|
||||
var aersp *raft.SnapshotRecoveryResponse
|
||||
var b bytes.Buffer
|
||||
json.NewEncoder(&b).Encode(req)
|
||||
|
@ -10,6 +10,6 @@ import (
|
||||
type Server interface {
|
||||
CommitIndex() uint64
|
||||
Term() uint64
|
||||
Store() *store.Store
|
||||
Store() store.Store
|
||||
Dispatch(raft.Command, http.ResponseWriter, *http.Request) error
|
||||
}
|
||||
|
@ -13,6 +13,6 @@ type Server interface {
|
||||
CommitIndex() uint64
|
||||
Term() uint64
|
||||
PeerURL(string) (string, bool)
|
||||
Store() *store.Store
|
||||
Store() store.Store
|
||||
Dispatch(raft.Command, http.ResponseWriter, *http.Request) error
|
||||
}
|
||||
|
@ -25,8 +25,8 @@ func (c *CreateCommand) CommandName() string {
|
||||
}
|
||||
|
||||
// Create node
|
||||
func (c *CreateCommand) Apply(server *raft.Server) (interface{}, error) {
|
||||
s, _ := server.StateMachine().(*Store)
|
||||
func (c *CreateCommand) Apply(server raft.Server) (interface{}, error) {
|
||||
s, _ := server.StateMachine().(Store)
|
||||
|
||||
e, err := s.Create(c.Key, c.Value, c.IncrementalSuffix, c.Force, c.ExpireTime, server.CommitIndex(), server.Term())
|
||||
|
||||
|
@ -21,8 +21,8 @@ func (c *DeleteCommand) CommandName() string {
|
||||
}
|
||||
|
||||
// Delete the key
|
||||
func (c *DeleteCommand) Apply(server *raft.Server) (interface{}, error) {
|
||||
s, _ := server.StateMachine().(*Store)
|
||||
func (c *DeleteCommand) Apply(server raft.Server) (interface{}, error) {
|
||||
s, _ := server.StateMachine().(Store)
|
||||
|
||||
e, err := s.Delete(c.Key, c.Recursive, server.CommitIndex(), server.Term())
|
||||
|
||||
|
@ -36,6 +36,9 @@ type Node struct {
|
||||
Value string // for key-value pair
|
||||
Children map[string]*Node // for directory
|
||||
|
||||
// A reference to the store this node is attached to.
|
||||
store *store
|
||||
|
||||
// a ttl node will have an expire routine associated with it.
|
||||
// we need a channel to stop that routine when the expiration changes.
|
||||
stopExpire chan bool
|
||||
@ -46,7 +49,7 @@ type Node struct {
|
||||
}
|
||||
|
||||
// newKV creates a Key-Value pair
|
||||
func newKV(nodePath string, value string, createIndex uint64,
|
||||
func newKV(store *store, nodePath string, value string, createIndex uint64,
|
||||
createTerm uint64, parent *Node, ACL string, expireTime time.Time) *Node {
|
||||
|
||||
return &Node{
|
||||
@ -57,6 +60,7 @@ func newKV(nodePath string, value string, createIndex uint64,
|
||||
ModifiedTerm: createTerm,
|
||||
Parent: parent,
|
||||
ACL: ACL,
|
||||
store: store,
|
||||
stopExpire: make(chan bool, 1),
|
||||
ExpireTime: expireTime,
|
||||
Value: value,
|
||||
@ -64,7 +68,7 @@ func newKV(nodePath string, value string, createIndex uint64,
|
||||
}
|
||||
|
||||
// newDir creates a directory
|
||||
func newDir(nodePath string, createIndex uint64, createTerm uint64,
|
||||
func newDir(store *store, nodePath string, createIndex uint64, createTerm uint64,
|
||||
parent *Node, ACL string, expireTime time.Time) *Node {
|
||||
|
||||
return &Node{
|
||||
@ -76,6 +80,7 @@ func newDir(nodePath string, createIndex uint64, createTerm uint64,
|
||||
stopExpire: make(chan bool, 1),
|
||||
ExpireTime: expireTime,
|
||||
Children: make(map[string]*Node),
|
||||
store: store,
|
||||
}
|
||||
}
|
||||
|
||||
@ -262,17 +267,17 @@ func (n *Node) internalRemove(recursive bool, callback func(path string)) {
|
||||
// if the node is already expired, delete the node and return.
|
||||
// if the node is permanent (this shouldn't happen), return at once.
|
||||
// else wait for a period time, then remove the node. and notify the watchhub.
|
||||
func (n *Node) Expire(s *Store) {
|
||||
func (n *Node) Expire() {
|
||||
expired, duration := n.IsExpired()
|
||||
|
||||
if expired { // has been expired
|
||||
// since the parent function of Expire() runs serially,
|
||||
// there is no need for lock here
|
||||
e := newEvent(Expire, n.Path, UndefIndex, UndefTerm)
|
||||
s.WatcherHub.notify(e)
|
||||
n.store.WatcherHub.notify(e)
|
||||
|
||||
n.Remove(true, nil)
|
||||
s.Stats.Inc(ExpireCount)
|
||||
n.store.Stats.Inc(ExpireCount)
|
||||
|
||||
return
|
||||
}
|
||||
@ -289,17 +294,17 @@ func (n *Node) Expire(s *Store) {
|
||||
// before expire get the lock, the expiration time
|
||||
// of the node may be updated.
|
||||
// we have to check again when get the lock
|
||||
s.worldLock.Lock()
|
||||
defer s.worldLock.Unlock()
|
||||
n.store.worldLock.Lock()
|
||||
defer n.store.worldLock.Unlock()
|
||||
|
||||
expired, _ := n.IsExpired()
|
||||
|
||||
if expired {
|
||||
e := newEvent(Expire, n.Path, UndefIndex, UndefTerm)
|
||||
s.WatcherHub.notify(e)
|
||||
n.store.WatcherHub.notify(e)
|
||||
|
||||
n.Remove(true, nil)
|
||||
s.Stats.Inc(ExpireCount)
|
||||
n.store.Stats.Inc(ExpireCount)
|
||||
}
|
||||
|
||||
return
|
||||
@ -355,7 +360,7 @@ func (n *Node) Pair(recurisive, sorted bool) KeyValuePair {
|
||||
}
|
||||
}
|
||||
|
||||
func (n *Node) UpdateTTL(expireTime time.Time, s *Store) {
|
||||
func (n *Node) UpdateTTL(expireTime time.Time) {
|
||||
if !n.IsPermanent() {
|
||||
// check if the node has been expired
|
||||
// if the node is not expired, we need to stop the go routine associated with
|
||||
@ -369,7 +374,7 @@ func (n *Node) UpdateTTL(expireTime time.Time, s *Store) {
|
||||
|
||||
if expireTime.Sub(Permanent) != 0 {
|
||||
n.ExpireTime = expireTime
|
||||
n.Expire(s)
|
||||
n.Expire()
|
||||
}
|
||||
}
|
||||
|
||||
@ -378,10 +383,10 @@ func (n *Node) UpdateTTL(expireTime time.Time, s *Store) {
|
||||
// If the node is a key-value pair, it will clone the pair.
|
||||
func (n *Node) Clone() *Node {
|
||||
if !n.IsDir() {
|
||||
return newKV(n.Path, n.Value, n.CreateIndex, n.CreateTerm, n.Parent, n.ACL, n.ExpireTime)
|
||||
return newKV(n.store, n.Path, n.Value, n.CreateIndex, n.CreateTerm, n.Parent, n.ACL, n.ExpireTime)
|
||||
}
|
||||
|
||||
clone := newDir(n.Path, n.CreateIndex, n.CreateTerm, n.Parent, n.ACL, n.ExpireTime)
|
||||
clone := newDir(n.store, n.Path, n.CreateIndex, n.CreateTerm, n.Parent, n.ACL, n.ExpireTime)
|
||||
|
||||
for key, child := range n.Children {
|
||||
clone.Children[key] = child.Clone()
|
||||
@ -397,15 +402,16 @@ func (n *Node) Clone() *Node {
|
||||
// call this function on its children.
|
||||
// We check the expire last since we need to recover the whole structure first and add all the
|
||||
// notifications into the event history.
|
||||
func (n *Node) recoverAndclean(s *Store) {
|
||||
func (n *Node) recoverAndclean() {
|
||||
if n.IsDir() {
|
||||
for _, child := range n.Children {
|
||||
child.Parent = n
|
||||
child.recoverAndclean(s)
|
||||
child.store = n.store
|
||||
child.recoverAndclean()
|
||||
}
|
||||
}
|
||||
|
||||
n.stopExpire = make(chan bool, 1)
|
||||
|
||||
n.Expire(s)
|
||||
n.Expire()
|
||||
}
|
||||
|
@ -7,7 +7,7 @@ import (
|
||||
)
|
||||
|
||||
func TestBasicStats(t *testing.T) {
|
||||
s := New()
|
||||
s := newStore()
|
||||
keys := GenKeys(rand.Intn(100), 5)
|
||||
|
||||
var i uint64
|
||||
@ -140,7 +140,7 @@ func TestBasicStats(t *testing.T) {
|
||||
t.Fatalf("TestAndSetFail [%d] != Stats.TestAndSetFail [%d]", TestAndSetFail, s.Stats.TestAndSetFail)
|
||||
}
|
||||
|
||||
s = New()
|
||||
s = newStore()
|
||||
SetSuccess = 0
|
||||
SetFail = 0
|
||||
|
||||
|
@ -13,7 +13,21 @@ import (
|
||||
etcdErr "github.com/coreos/etcd/error"
|
||||
)
|
||||
|
||||
type Store struct {
|
||||
type Store interface {
|
||||
Get(nodePath string, recursive, sorted bool, index uint64, term uint64) (*Event, error)
|
||||
Create(nodePath string, value string, incrementalSuffix bool, force bool,
|
||||
expireTime time.Time, index uint64, term uint64) (*Event, error)
|
||||
Update(nodePath string, newValue string, expireTime time.Time, index uint64, term uint64) (*Event, error)
|
||||
TestAndSet(nodePath string, prevValue string, prevIndex uint64,
|
||||
value string, expireTime time.Time, index uint64, term uint64) (*Event, error)
|
||||
Delete(nodePath string, recursive bool, index uint64, term uint64) (*Event, error)
|
||||
Watch(prefix string, recursive bool, sinceIndex uint64, index uint64, term uint64) (<-chan *Event, error)
|
||||
Save() ([]byte, error)
|
||||
Recovery(state []byte) error
|
||||
JsonStats() []byte
|
||||
}
|
||||
|
||||
type store struct {
|
||||
Root *Node
|
||||
WatcherHub *watcherHub
|
||||
Index uint64
|
||||
@ -22,9 +36,13 @@ type Store struct {
|
||||
worldLock sync.RWMutex // stop the world lock
|
||||
}
|
||||
|
||||
func New() *Store {
|
||||
s := new(Store)
|
||||
s.Root = newDir("/", UndefIndex, UndefTerm, nil, "", Permanent)
|
||||
func New() Store {
|
||||
return newStore()
|
||||
}
|
||||
|
||||
func newStore() *store {
|
||||
s := new(store)
|
||||
s.Root = newDir(s, "/", UndefIndex, UndefTerm, nil, "", Permanent)
|
||||
s.Stats = newStats()
|
||||
s.WatcherHub = newWatchHub(1000)
|
||||
|
||||
@ -34,7 +52,7 @@ func New() *Store {
|
||||
// Get function returns a get event.
|
||||
// If recursive is true, it will return all the content under the node path.
|
||||
// If sorted is true, it will sort the content by keys.
|
||||
func (s *Store) Get(nodePath string, recursive, sorted bool, index uint64, term uint64) (*Event, error) {
|
||||
func (s *store) Get(nodePath string, recursive, sorted bool, index uint64, term uint64) (*Event, error) {
|
||||
s.worldLock.RLock()
|
||||
defer s.worldLock.RUnlock()
|
||||
|
||||
@ -89,7 +107,7 @@ func (s *Store) Get(nodePath string, recursive, sorted bool, index uint64, term
|
||||
// Create function creates the Node at nodePath. Create will help to create intermediate directories with no ttl.
|
||||
// If the node has already existed, create will fail.
|
||||
// If any node on the path is a file, create will fail.
|
||||
func (s *Store) Create(nodePath string, value string, incrementalSuffix bool, force bool,
|
||||
func (s *store) Create(nodePath string, value string, incrementalSuffix bool, force bool,
|
||||
expireTime time.Time, index uint64, term uint64) (*Event, error) {
|
||||
nodePath = path.Clean(path.Join("/", nodePath))
|
||||
|
||||
@ -101,7 +119,7 @@ func (s *Store) Create(nodePath string, value string, incrementalSuffix bool, fo
|
||||
// Update function updates the value/ttl of the node.
|
||||
// If the node is a file, the value and the ttl can be updated.
|
||||
// If the node is a directory, only the ttl can be updated.
|
||||
func (s *Store) Update(nodePath string, newValue string, expireTime time.Time, index uint64, term uint64) (*Event, error) {
|
||||
func (s *store) Update(nodePath string, newValue string, expireTime time.Time, index uint64, term uint64) (*Event, error) {
|
||||
s.worldLock.Lock()
|
||||
defer s.worldLock.Unlock()
|
||||
nodePath = path.Clean(path.Join("/", nodePath))
|
||||
@ -127,7 +145,7 @@ func (s *Store) Update(nodePath string, newValue string, expireTime time.Time, i
|
||||
}
|
||||
|
||||
// update ttl
|
||||
n.UpdateTTL(expireTime, s)
|
||||
n.UpdateTTL(expireTime)
|
||||
|
||||
e.Expiration, e.TTL = n.ExpirationAndTTL()
|
||||
|
||||
@ -138,7 +156,7 @@ func (s *Store) Update(nodePath string, newValue string, expireTime time.Time, i
|
||||
return e, nil
|
||||
}
|
||||
|
||||
func (s *Store) TestAndSet(nodePath string, prevValue string, prevIndex uint64,
|
||||
func (s *store) TestAndSet(nodePath string, prevValue string, prevIndex uint64,
|
||||
value string, expireTime time.Time, index uint64, term uint64) (*Event, error) {
|
||||
|
||||
nodePath = path.Clean(path.Join("/", nodePath))
|
||||
@ -170,7 +188,7 @@ func (s *Store) TestAndSet(nodePath string, prevValue string, prevIndex uint64,
|
||||
|
||||
// if test succeed, write the value
|
||||
n.Write(value, index, term)
|
||||
n.UpdateTTL(expireTime, s)
|
||||
n.UpdateTTL(expireTime)
|
||||
|
||||
e.Value = value
|
||||
e.Expiration, e.TTL = n.ExpirationAndTTL()
|
||||
@ -187,7 +205,7 @@ func (s *Store) TestAndSet(nodePath string, prevValue string, prevIndex uint64,
|
||||
|
||||
// Delete function deletes the node at the given path.
|
||||
// If the node is a directory, recursive must be true to delete it.
|
||||
func (s *Store) Delete(nodePath string, recursive bool, index uint64, term uint64) (*Event, error) {
|
||||
func (s *store) Delete(nodePath string, recursive bool, index uint64, term uint64) (*Event, error) {
|
||||
nodePath = path.Clean(path.Join("/", nodePath))
|
||||
|
||||
s.worldLock.Lock()
|
||||
@ -226,7 +244,7 @@ func (s *Store) Delete(nodePath string, recursive bool, index uint64, term uint6
|
||||
return e, nil
|
||||
}
|
||||
|
||||
func (s *Store) Watch(prefix string, recursive bool, sinceIndex uint64, index uint64, term uint64) (<-chan *Event, error) {
|
||||
func (s *store) Watch(prefix string, recursive bool, sinceIndex uint64, index uint64, term uint64) (<-chan *Event, error) {
|
||||
prefix = path.Clean(path.Join("/", prefix))
|
||||
|
||||
s.worldLock.RLock()
|
||||
@ -254,7 +272,7 @@ func (s *Store) Watch(prefix string, recursive bool, sinceIndex uint64, index ui
|
||||
}
|
||||
|
||||
// walk function walks all the nodePath and apply the walkFunc on each directory
|
||||
func (s *Store) walk(nodePath string, walkFunc func(prev *Node, component string) (*Node, *etcdErr.Error)) (*Node, *etcdErr.Error) {
|
||||
func (s *store) walk(nodePath string, walkFunc func(prev *Node, component string) (*Node, *etcdErr.Error)) (*Node, *etcdErr.Error) {
|
||||
components := strings.Split(nodePath, "/")
|
||||
|
||||
curr := s.Root
|
||||
@ -275,7 +293,7 @@ func (s *Store) walk(nodePath string, walkFunc func(prev *Node, component string
|
||||
return curr, nil
|
||||
}
|
||||
|
||||
func (s *Store) internalCreate(nodePath string, value string, incrementalSuffix bool, force bool,
|
||||
func (s *store) internalCreate(nodePath string, value string, incrementalSuffix bool, force bool,
|
||||
expireTime time.Time, index uint64, term uint64, action string) (*Event, error) {
|
||||
|
||||
s.Index, s.Term = index, term
|
||||
@ -318,12 +336,12 @@ func (s *Store) internalCreate(nodePath string, value string, incrementalSuffix
|
||||
if len(value) != 0 { // create file
|
||||
e.Value = value
|
||||
|
||||
n = newKV(nodePath, value, index, term, d, "", expireTime)
|
||||
n = newKV(s, nodePath, value, index, term, d, "", expireTime)
|
||||
|
||||
} else { // create directory
|
||||
e.Dir = true
|
||||
|
||||
n = newDir(nodePath, index, term, d, "", expireTime)
|
||||
n = newDir(s, nodePath, index, term, d, "", expireTime)
|
||||
|
||||
}
|
||||
|
||||
@ -336,7 +354,7 @@ func (s *Store) internalCreate(nodePath string, value string, incrementalSuffix
|
||||
|
||||
// Node with TTL
|
||||
if expireTime.Sub(Permanent) != 0 {
|
||||
n.Expire(s)
|
||||
n.Expire()
|
||||
e.Expiration, e.TTL = n.ExpirationAndTTL()
|
||||
}
|
||||
|
||||
@ -346,7 +364,7 @@ func (s *Store) internalCreate(nodePath string, value string, incrementalSuffix
|
||||
}
|
||||
|
||||
// InternalGet function get the node of the given nodePath.
|
||||
func (s *Store) internalGet(nodePath string, index uint64, term uint64) (*Node, *etcdErr.Error) {
|
||||
func (s *store) internalGet(nodePath string, index uint64, term uint64) (*Node, *etcdErr.Error) {
|
||||
nodePath = path.Clean(path.Join("/", nodePath))
|
||||
|
||||
// update file system known index and term
|
||||
@ -381,7 +399,7 @@ func (s *Store) internalGet(nodePath string, index uint64, term uint64) (*Node,
|
||||
// If it is a directory, this function will return the pointer to that node.
|
||||
// If it does not exist, this function will create a new directory and return the pointer to that node.
|
||||
// If it is a file, this function will return error.
|
||||
func (s *Store) checkDir(parent *Node, dirName string) (*Node, *etcdErr.Error) {
|
||||
func (s *store) checkDir(parent *Node, dirName string) (*Node, *etcdErr.Error) {
|
||||
node, ok := parent.Children[dirName]
|
||||
|
||||
if ok {
|
||||
@ -392,7 +410,7 @@ func (s *Store) checkDir(parent *Node, dirName string) (*Node, *etcdErr.Error) {
|
||||
return nil, etcdErr.NewError(etcdErr.EcodeNotDir, parent.Path, UndefIndex, UndefTerm)
|
||||
}
|
||||
|
||||
n := newDir(path.Join(parent.Path, dirName), s.Index, s.Term, parent, parent.ACL, Permanent)
|
||||
n := newDir(s, path.Join(parent.Path, dirName), s.Index, s.Term, parent, parent.ACL, Permanent)
|
||||
|
||||
parent.Children[dirName] = n
|
||||
|
||||
@ -403,10 +421,10 @@ func (s *Store) checkDir(parent *Node, dirName string) (*Node, *etcdErr.Error) {
|
||||
// Save function will not be able to save the state of watchers.
|
||||
// Save function will not save the parent field of the node. Or there will
|
||||
// be cyclic dependencies issue for the json package.
|
||||
func (s *Store) Save() ([]byte, error) {
|
||||
func (s *store) Save() ([]byte, error) {
|
||||
s.worldLock.Lock()
|
||||
|
||||
clonedStore := New()
|
||||
clonedStore := newStore()
|
||||
clonedStore.Index = s.Index
|
||||
clonedStore.Term = s.Term
|
||||
clonedStore.Root = s.Root.Clone()
|
||||
@ -428,7 +446,7 @@ func (s *Store) Save() ([]byte, error) {
|
||||
// It needs to recovery the parent field of the nodes.
|
||||
// It needs to delete the expired nodes since the saved time and also
|
||||
// need to create monitor go routines.
|
||||
func (s *Store) Recovery(state []byte) error {
|
||||
func (s *store) Recovery(state []byte) error {
|
||||
s.worldLock.Lock()
|
||||
defer s.worldLock.Unlock()
|
||||
err := json.Unmarshal(state, s)
|
||||
@ -437,11 +455,11 @@ func (s *Store) Recovery(state []byte) error {
|
||||
return err
|
||||
}
|
||||
|
||||
s.Root.recoverAndclean(s)
|
||||
s.Root.recoverAndclean()
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *Store) JsonStats() []byte {
|
||||
func (s *store) JsonStats() []byte {
|
||||
s.Stats.Watchers = uint64(s.WatcherHub.count)
|
||||
return s.Stats.toJson()
|
||||
}
|
||||
|
@ -8,7 +8,7 @@ import (
|
||||
)
|
||||
|
||||
func TestCreateAndGet(t *testing.T) {
|
||||
s := New()
|
||||
s := newStore()
|
||||
|
||||
s.Create("/foobar", "bar", false, false, Permanent, 1, 1)
|
||||
|
||||
@ -66,7 +66,7 @@ func TestCreateAndGet(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestUpdateFile(t *testing.T) {
|
||||
s := New()
|
||||
s := newStore()
|
||||
|
||||
_, err := s.Create("/foo/bar", "bar", false, false, Permanent, 1, 1)
|
||||
|
||||
@ -161,7 +161,7 @@ func TestUpdateFile(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestListDirectory(t *testing.T) {
|
||||
s := New()
|
||||
s := newStore()
|
||||
|
||||
// create dir /foo
|
||||
// set key-value /foo/foo=bar
|
||||
@ -206,7 +206,7 @@ func TestListDirectory(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestRemove(t *testing.T) {
|
||||
s := New()
|
||||
s := newStore()
|
||||
|
||||
s.Create("/foo", "bar", false, false, Permanent, 1, 1)
|
||||
_, err := s.Delete("/foo", false, 1, 1)
|
||||
@ -245,7 +245,7 @@ func TestRemove(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestExpire(t *testing.T) {
|
||||
s := New()
|
||||
s := newStore()
|
||||
|
||||
expire := time.Now().Add(time.Second)
|
||||
|
||||
@ -287,7 +287,7 @@ func TestExpire(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestTestAndSet(t *testing.T) { // TODO prevValue == nil ?
|
||||
s := New()
|
||||
s := newStore()
|
||||
s.Create("/foo", "bar", false, false, Permanent, 1, 1)
|
||||
|
||||
// test on wrong previous value
|
||||
@ -320,7 +320,7 @@ func TestTestAndSet(t *testing.T) { // TODO prevValue == nil ?
|
||||
}
|
||||
|
||||
func TestWatch(t *testing.T) {
|
||||
s := New()
|
||||
s := newStore()
|
||||
// watch at a deeper path
|
||||
c, _ := s.Watch("/foo/foo/foo", false, 0, 0, 1)
|
||||
s.Create("/foo/foo/foo", "bar", false, false, Permanent, 1, 1)
|
||||
@ -409,7 +409,7 @@ func TestWatch(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestSort(t *testing.T) {
|
||||
s := New()
|
||||
s := newStore()
|
||||
|
||||
// simulating random creation
|
||||
keys := GenKeys(80, 4)
|
||||
@ -447,7 +447,7 @@ func TestSort(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestSaveAndRecover(t *testing.T) {
|
||||
s := New()
|
||||
s := newStore()
|
||||
|
||||
// simulating random creation
|
||||
keys := GenKeys(8, 4)
|
||||
@ -469,7 +469,7 @@ func TestSaveAndRecover(t *testing.T) {
|
||||
s.Create("/foo/foo", "bar", false, false, expire, 1, 1)
|
||||
b, err := s.Save()
|
||||
|
||||
cloneFs := New()
|
||||
cloneFs := newStore()
|
||||
time.Sleep(2 * time.Second)
|
||||
|
||||
cloneFs.Recovery(b)
|
||||
@ -521,7 +521,7 @@ func GenKeys(num int, depth int) []string {
|
||||
return keys
|
||||
}
|
||||
|
||||
func createAndGet(s *Store, path string, t *testing.T) {
|
||||
func createAndGet(s *store, path string, t *testing.T) {
|
||||
_, err := s.Create(path, "bar", false, false, Permanent, 1, 1)
|
||||
|
||||
if err != nil {
|
||||
|
@ -26,8 +26,8 @@ func (c *TestAndSetCommand) CommandName() string {
|
||||
}
|
||||
|
||||
// Set the key-value pair if the current value of the key equals to the given prevValue
|
||||
func (c *TestAndSetCommand) Apply(server *raft.Server) (interface{}, error) {
|
||||
s, _ := server.StateMachine().(*Store)
|
||||
func (c *TestAndSetCommand) Apply(server raft.Server) (interface{}, error) {
|
||||
s, _ := server.StateMachine().(Store)
|
||||
|
||||
e, err := s.TestAndSet(c.Key, c.PrevValue, c.PrevIndex,
|
||||
c.Value, c.ExpireTime, server.CommitIndex(), server.Term())
|
||||
|
@ -24,8 +24,8 @@ func (c *UpdateCommand) CommandName() string {
|
||||
}
|
||||
|
||||
// Update node
|
||||
func (c *UpdateCommand) Apply(server *raft.Server) (interface{}, error) {
|
||||
s, _ := server.StateMachine().(*Store)
|
||||
func (c *UpdateCommand) Apply(server raft.Server) (interface{}, error) {
|
||||
s, _ := server.StateMachine().(Store)
|
||||
|
||||
e, err := s.Update(c.Key, c.Value, c.ExpireTime, server.CommitIndex(), server.Term())
|
||||
|
||||
|
@ -5,7 +5,7 @@ import (
|
||||
)
|
||||
|
||||
func TestWatcher(t *testing.T) {
|
||||
s := New()
|
||||
s := newStore()
|
||||
wh := s.WatcherHub
|
||||
c, err := wh.watch("/foo", true, 1)
|
||||
if err != nil {
|
||||
|
@ -22,7 +22,7 @@ func Set(stop chan bool) {
|
||||
|
||||
stopSet := false
|
||||
i := 0
|
||||
c := etcd.NewClient()
|
||||
c := etcd.NewClient(nil)
|
||||
for {
|
||||
key := fmt.Sprintf("%s_%v", "foo", i)
|
||||
|
||||
|
@ -8,6 +8,7 @@ package osext
|
||||
|
||||
import (
|
||||
"os"
|
||||
"path/filepath"
|
||||
"runtime"
|
||||
"syscall"
|
||||
"unsafe"
|
||||
@ -47,18 +48,35 @@ func executable() (string, error) {
|
||||
break
|
||||
}
|
||||
}
|
||||
var strpath string
|
||||
if buf[0] != '/' {
|
||||
if getwdError != nil {
|
||||
return string(buf), getwdError
|
||||
} else {
|
||||
if buf[0] == '.' {
|
||||
buf = buf[1:]
|
||||
}
|
||||
if startUpcwd[len(startUpcwd)-1] != '/' {
|
||||
return startUpcwd + "/" + string(buf), nil
|
||||
}
|
||||
return startUpcwd + string(buf), nil
|
||||
var e error
|
||||
if strpath, e = getAbs(buf); e != nil {
|
||||
return strpath, e
|
||||
}
|
||||
} else {
|
||||
strpath = string(buf)
|
||||
}
|
||||
// darwin KERN_PROCARGS may return the path to a symlink rather than the
|
||||
// actual executable
|
||||
if runtime.GOOS == "darwin" {
|
||||
if strpath, err := filepath.EvalSymlinks(strpath); err != nil {
|
||||
return strpath, err
|
||||
}
|
||||
}
|
||||
return string(buf), nil
|
||||
return strpath, nil
|
||||
}
|
||||
|
||||
func getAbs(buf []byte) (string, error) {
|
||||
if getwdError != nil {
|
||||
return string(buf), getwdError
|
||||
} else {
|
||||
if buf[0] == '.' {
|
||||
buf = buf[1:]
|
||||
}
|
||||
if startUpcwd[len(startUpcwd)-1] != '/' && buf[0] != '/' {
|
||||
return startUpcwd + "/" + string(buf), nil
|
||||
}
|
||||
return startUpcwd + string(buf), nil
|
||||
}
|
||||
}
|
||||
|
46
third_party/code.google.com/p/go.net/ipv4/gen.go
vendored
46
third_party/code.google.com/p/go.net/ipv4/gen.go
vendored
@ -97,20 +97,16 @@ func parseICMPv4Parameters(w io.Writer, r io.Reader) error {
|
||||
}
|
||||
|
||||
type icmpv4Parameters struct {
|
||||
XMLName xml.Name `xml:"registry"`
|
||||
Title string `xml:"title"`
|
||||
Updated string `xml:"updated"`
|
||||
Registries []icmpv4ParamRegistry `xml:"registry"`
|
||||
}
|
||||
|
||||
type icmpv4ParamRegistry struct {
|
||||
Title string `xml:"title"`
|
||||
Records []icmpv4ParamRecord `xml:"record"`
|
||||
}
|
||||
|
||||
type icmpv4ParamRecord struct {
|
||||
Value string `xml:"value"`
|
||||
Descr string `xml:"description"`
|
||||
XMLName xml.Name `xml:"registry"`
|
||||
Title string `xml:"title"`
|
||||
Updated string `xml:"updated"`
|
||||
Registries []struct {
|
||||
Title string `xml:"title"`
|
||||
Records []struct {
|
||||
Value string `xml:"value"`
|
||||
Descr string `xml:"description"`
|
||||
} `xml:"record"`
|
||||
} `xml:"registry"`
|
||||
}
|
||||
|
||||
type canonICMPv4ParamRecord struct {
|
||||
@ -193,18 +189,16 @@ func parseProtocolNumbers(w io.Writer, r io.Reader) error {
|
||||
}
|
||||
|
||||
type protocolNumbers struct {
|
||||
XMLName xml.Name `xml:"registry"`
|
||||
Title string `xml:"title"`
|
||||
Updated string `xml:"updated"`
|
||||
RegTitle string `xml:"registry>title"`
|
||||
Note string `xml:"registry>note"`
|
||||
Records []protocolRecord `xml:"registry>record"`
|
||||
}
|
||||
|
||||
type protocolRecord struct {
|
||||
Value string `xml:"value"`
|
||||
Name string `xml:"name"`
|
||||
Descr string `xml:"description"`
|
||||
XMLName xml.Name `xml:"registry"`
|
||||
Title string `xml:"title"`
|
||||
Updated string `xml:"updated"`
|
||||
RegTitle string `xml:"registry>title"`
|
||||
Note string `xml:"registry>note"`
|
||||
Records []struct {
|
||||
Value string `xml:"value"`
|
||||
Name string `xml:"name"`
|
||||
Descr string `xml:"description"`
|
||||
} `xml:"registry>record"`
|
||||
}
|
||||
|
||||
type canonProtocolRecord struct {
|
||||
|
@ -39,7 +39,7 @@ var registries = []struct {
|
||||
|
||||
func main() {
|
||||
var bb bytes.Buffer
|
||||
fmt.Fprintf(&bb, "// go run gentv.go\n")
|
||||
fmt.Fprintf(&bb, "// go run gentest.go\n")
|
||||
fmt.Fprintf(&bb, "// GENERATED BY THE COMMAND ABOVE; DO NOT EDIT\n\n")
|
||||
fmt.Fprintf(&bb, "package ipv4_test\n\n")
|
||||
for _, r := range registries {
|
||||
@ -85,18 +85,19 @@ func parseDSCPRegistry(w io.Writer, r io.Reader) error {
|
||||
}
|
||||
|
||||
type dscpRegistry struct {
|
||||
XMLName xml.Name `xml:"registry"`
|
||||
Title string `xml:"title"`
|
||||
Updated string `xml:"updated"`
|
||||
Note string `xml:"note"`
|
||||
RegTitle string `xml:"registry>title"`
|
||||
PoolRecords []dscpRecord `xml:"registry>record"`
|
||||
Records []dscpRecord `xml:"registry>registry>record"`
|
||||
}
|
||||
|
||||
type dscpRecord struct {
|
||||
Name string `xml:"name"`
|
||||
Space string `xml:"space"`
|
||||
XMLName xml.Name `xml:"registry"`
|
||||
Title string `xml:"title"`
|
||||
Updated string `xml:"updated"`
|
||||
Note string `xml:"note"`
|
||||
RegTitle string `xml:"registry>title"`
|
||||
PoolRecords []struct {
|
||||
Name string `xml:"name"`
|
||||
Space string `xml:"space"`
|
||||
} `xml:"registry>record"`
|
||||
Records []struct {
|
||||
Name string `xml:"name"`
|
||||
Space string `xml:"space"`
|
||||
} `xml:"registry>registry>record"`
|
||||
}
|
||||
|
||||
type canonDSCPRecord struct {
|
||||
@ -145,17 +146,15 @@ func parseTOSTCByte(w io.Writer, r io.Reader) error {
|
||||
}
|
||||
|
||||
type tosTCByte struct {
|
||||
XMLName xml.Name `xml:"registry"`
|
||||
Title string `xml:"title"`
|
||||
Updated string `xml:"updated"`
|
||||
Note string `xml:"note"`
|
||||
RegTitle string `xml:"registry>title"`
|
||||
Records []tosTCByteRecord `xml:"registry>record"`
|
||||
}
|
||||
|
||||
type tosTCByteRecord struct {
|
||||
Binary string `xml:"binary"`
|
||||
Keyword string `xml:"keyword"`
|
||||
XMLName xml.Name `xml:"registry"`
|
||||
Title string `xml:"title"`
|
||||
Updated string `xml:"updated"`
|
||||
Note string `xml:"note"`
|
||||
RegTitle string `xml:"registry>title"`
|
||||
Records []struct {
|
||||
Binary string `xml:"binary"`
|
||||
Keyword string `xml:"keyword"`
|
||||
} `xml:"registry>record"`
|
||||
}
|
||||
|
||||
type canonTOSTCByteRecord struct {
|
||||
|
@ -36,41 +36,47 @@ const (
|
||||
maxHeaderLen = 60 // sensible default, revisit if later RFCs define new usage of version and header length fields
|
||||
)
|
||||
|
||||
type headerField int
|
||||
const (
|
||||
posTOS = 1 // type-of-service
|
||||
posTotalLen = 2 // packet total length
|
||||
posID = 4 // identification
|
||||
posFragOff = 6 // fragment offset
|
||||
posTTL = 8 // time-to-live
|
||||
posProtocol = 9 // next protocol
|
||||
posChecksum = 10 // checksum
|
||||
posSrc = 12 // source address
|
||||
posDst = 16 // destination address
|
||||
)
|
||||
|
||||
type HeaderFlags int
|
||||
|
||||
const (
|
||||
posTOS headerField = 1 // type-of-service
|
||||
posTotalLen = 2 // packet total length
|
||||
posID = 4 // identification
|
||||
posFragOff = 6 // fragment offset
|
||||
posTTL = 8 // time-to-live
|
||||
posProtocol = 9 // next protocol
|
||||
posChecksum = 10 // checksum
|
||||
posSrc = 12 // source address
|
||||
posDst = 16 // destination address
|
||||
MoreFragments HeaderFlags = 1 << iota // more fragments flag
|
||||
DontFragment // don't fragment flag
|
||||
)
|
||||
|
||||
// A Header represents an IPv4 header.
|
||||
type Header struct {
|
||||
Version int // protocol version
|
||||
Len int // header length
|
||||
TOS int // type-of-service
|
||||
TotalLen int // packet total length
|
||||
ID int // identification
|
||||
FragOff int // fragment offset
|
||||
TTL int // time-to-live
|
||||
Protocol int // next protocol
|
||||
Checksum int // checksum
|
||||
Src net.IP // source address
|
||||
Dst net.IP // destination address
|
||||
Options []byte // options, extension headers
|
||||
Version int // protocol version
|
||||
Len int // header length
|
||||
TOS int // type-of-service
|
||||
TotalLen int // packet total length
|
||||
ID int // identification
|
||||
Flags HeaderFlags // flags
|
||||
FragOff int // fragment offset
|
||||
TTL int // time-to-live
|
||||
Protocol int // next protocol
|
||||
Checksum int // checksum
|
||||
Src net.IP // source address
|
||||
Dst net.IP // destination address
|
||||
Options []byte // options, extension headers
|
||||
}
|
||||
|
||||
func (h *Header) String() string {
|
||||
if h == nil {
|
||||
return "<nil>"
|
||||
}
|
||||
return fmt.Sprintf("ver: %v, hdrlen: %v, tos: %#x, totallen: %v, id: %#x, fragoff: %#x, ttl: %v, proto: %v, cksum: %#x, src: %v, dst: %v", h.Version, h.Len, h.TOS, h.TotalLen, h.ID, h.FragOff, h.TTL, h.Protocol, h.Checksum, h.Src, h.Dst)
|
||||
return fmt.Sprintf("ver: %v, hdrlen: %v, tos: %#x, totallen: %v, id: %#x, flags: %#x, fragoff: %#x, ttl: %v, proto: %v, cksum: %#x, src: %v, dst: %v", h.Version, h.Len, h.TOS, h.TotalLen, h.ID, h.Flags, h.FragOff, h.TTL, h.Protocol, h.Checksum, h.Src, h.Dst)
|
||||
}
|
||||
|
||||
// Please refer to the online manual; IP(4) on Darwin, FreeBSD and
|
||||
@ -89,12 +95,13 @@ func (h *Header) Marshal() ([]byte, error) {
|
||||
b := make([]byte, hdrlen)
|
||||
b[0] = byte(Version<<4 | (hdrlen >> 2 & 0x0f))
|
||||
b[posTOS] = byte(h.TOS)
|
||||
flagsAndFragOff := (h.FragOff & 0x1fff) | int(h.Flags<<13)
|
||||
if supportsNewIPInput {
|
||||
b[posTotalLen], b[posTotalLen+1] = byte(h.TotalLen>>8), byte(h.TotalLen)
|
||||
b[posFragOff], b[posFragOff+1] = byte(h.FragOff>>8), byte(h.FragOff)
|
||||
b[posFragOff], b[posFragOff+1] = byte(flagsAndFragOff>>8), byte(flagsAndFragOff)
|
||||
} else {
|
||||
*(*uint16)(unsafe.Pointer(&b[posTotalLen : posTotalLen+1][0])) = uint16(h.TotalLen)
|
||||
*(*uint16)(unsafe.Pointer(&b[posFragOff : posFragOff+1][0])) = uint16(h.FragOff)
|
||||
*(*uint16)(unsafe.Pointer(&b[posFragOff : posFragOff+1][0])) = uint16(flagsAndFragOff)
|
||||
}
|
||||
b[posID], b[posID+1] = byte(h.ID>>8), byte(h.ID)
|
||||
b[posTTL] = byte(h.TTL)
|
||||
@ -135,6 +142,8 @@ func ParseHeader(b []byte) (*Header, error) {
|
||||
h.TotalLen += hdrlen
|
||||
h.FragOff = int(*(*uint16)(unsafe.Pointer(&b[posFragOff : posFragOff+1][0])))
|
||||
}
|
||||
h.Flags = HeaderFlags(h.FragOff&0xe000) >> 13
|
||||
h.FragOff = h.FragOff & 0x1fff
|
||||
h.ID = int(b[posID])<<8 | int(b[posID+1])
|
||||
h.TTL = int(b[posTTL])
|
||||
h.Protocol = int(b[posProtocol])
|
||||
|
@ -16,28 +16,28 @@ import (
|
||||
var (
|
||||
wireHeaderFromKernel = [ipv4.HeaderLen]byte{
|
||||
0x45, 0x01, 0xbe, 0xef,
|
||||
0xca, 0xfe, 0x05, 0xdc,
|
||||
0xca, 0xfe, 0x45, 0xdc,
|
||||
0xff, 0x01, 0xde, 0xad,
|
||||
172, 16, 254, 254,
|
||||
192, 168, 0, 1,
|
||||
}
|
||||
wireHeaderToKernel = [ipv4.HeaderLen]byte{
|
||||
0x45, 0x01, 0xbe, 0xef,
|
||||
0xca, 0xfe, 0x05, 0xdc,
|
||||
0xca, 0xfe, 0x45, 0xdc,
|
||||
0xff, 0x01, 0xde, 0xad,
|
||||
172, 16, 254, 254,
|
||||
192, 168, 0, 1,
|
||||
}
|
||||
wireHeaderFromTradBSDKernel = [ipv4.HeaderLen]byte{
|
||||
0x45, 0x01, 0xdb, 0xbe,
|
||||
0xca, 0xfe, 0xdc, 0x05,
|
||||
0xca, 0xfe, 0xdc, 0x45,
|
||||
0xff, 0x01, 0xde, 0xad,
|
||||
172, 16, 254, 254,
|
||||
192, 168, 0, 1,
|
||||
}
|
||||
wireHeaderToTradBSDKernel = [ipv4.HeaderLen]byte{
|
||||
0x45, 0x01, 0xef, 0xbe,
|
||||
0xca, 0xfe, 0xdc, 0x05,
|
||||
0xca, 0xfe, 0xdc, 0x45,
|
||||
0xff, 0x01, 0xde, 0xad,
|
||||
172, 16, 254, 254,
|
||||
192, 168, 0, 1,
|
||||
@ -51,6 +51,7 @@ var (
|
||||
TOS: 1,
|
||||
TotalLen: 0xbeef,
|
||||
ID: 0xcafe,
|
||||
Flags: ipv4.DontFragment,
|
||||
FragOff: 1500,
|
||||
TTL: 255,
|
||||
Protocol: 1,
|
||||
|
@ -1,9 +1,9 @@
|
||||
// go run gentv.go
|
||||
// go run gentest.go
|
||||
// GENERATED BY THE COMMAND ABOVE; DO NOT EDIT
|
||||
|
||||
package ipv4_test
|
||||
|
||||
// Differentiated Services Field Codepoints, Updated: 2010-05-11
|
||||
// Differentiated Services Field Codepoints (DSCP), Updated: 2013-06-25
|
||||
const (
|
||||
DiffServCS0 = 0x0 // CS0
|
||||
DiffServCS1 = 0x20 // CS1
|
||||
|
46
third_party/code.google.com/p/go.net/ipv6/gen.go
vendored
46
third_party/code.google.com/p/go.net/ipv6/gen.go
vendored
@ -97,20 +97,16 @@ func parseICMPv6Parameters(w io.Writer, r io.Reader) error {
|
||||
}
|
||||
|
||||
type icmpv6Parameters struct {
|
||||
XMLName xml.Name `xml:"registry"`
|
||||
Title string `xml:"title"`
|
||||
Updated string `xml:"updated"`
|
||||
Registries []icmpv6ParamRegistry `xml:"registry"`
|
||||
}
|
||||
|
||||
type icmpv6ParamRegistry struct {
|
||||
Title string `xml:"title"`
|
||||
Records []icmpv6ParamRecord `xml:"record"`
|
||||
}
|
||||
|
||||
type icmpv6ParamRecord struct {
|
||||
Value string `xml:"value"`
|
||||
Name string `xml:"name"`
|
||||
XMLName xml.Name `xml:"registry"`
|
||||
Title string `xml:"title"`
|
||||
Updated string `xml:"updated"`
|
||||
Registries []struct {
|
||||
Title string `xml:"title"`
|
||||
Records []struct {
|
||||
Value string `xml:"value"`
|
||||
Name string `xml:"name"`
|
||||
} `xml:"record"`
|
||||
} `xml:"registry"`
|
||||
}
|
||||
|
||||
type canonICMPv6ParamRecord struct {
|
||||
@ -188,18 +184,16 @@ func parseProtocolNumbers(w io.Writer, r io.Reader) error {
|
||||
}
|
||||
|
||||
type protocolNumbers struct {
|
||||
XMLName xml.Name `xml:"registry"`
|
||||
Title string `xml:"title"`
|
||||
Updated string `xml:"updated"`
|
||||
RegTitle string `xml:"registry>title"`
|
||||
Note string `xml:"registry>note"`
|
||||
Records []protocolRecord `xml:"registry>record"`
|
||||
}
|
||||
|
||||
type protocolRecord struct {
|
||||
Value string `xml:"value"`
|
||||
Name string `xml:"name"`
|
||||
Descr string `xml:"description"`
|
||||
XMLName xml.Name `xml:"registry"`
|
||||
Title string `xml:"title"`
|
||||
Updated string `xml:"updated"`
|
||||
RegTitle string `xml:"registry>title"`
|
||||
Note string `xml:"registry>note"`
|
||||
Records []struct {
|
||||
Value string `xml:"value"`
|
||||
Name string `xml:"name"`
|
||||
Descr string `xml:"description"`
|
||||
} `xml:"registry>record"`
|
||||
}
|
||||
|
||||
type canonProtocolRecord struct {
|
||||
|
@ -39,7 +39,7 @@ var registries = []struct {
|
||||
|
||||
func main() {
|
||||
var bb bytes.Buffer
|
||||
fmt.Fprintf(&bb, "// go run gentv.go\n")
|
||||
fmt.Fprintf(&bb, "// go run gentest.go\n")
|
||||
fmt.Fprintf(&bb, "// GENERATED BY THE COMMAND ABOVE; DO NOT EDIT\n\n")
|
||||
fmt.Fprintf(&bb, "package ipv6_test\n\n")
|
||||
for _, r := range registries {
|
||||
@ -85,18 +85,19 @@ func parseDSCPRegistry(w io.Writer, r io.Reader) error {
|
||||
}
|
||||
|
||||
type dscpRegistry struct {
|
||||
XMLName xml.Name `xml:"registry"`
|
||||
Title string `xml:"title"`
|
||||
Updated string `xml:"updated"`
|
||||
Note string `xml:"note"`
|
||||
RegTitle string `xml:"registry>title"`
|
||||
PoolRecords []dscpRecord `xml:"registry>record"`
|
||||
Records []dscpRecord `xml:"registry>registry>record"`
|
||||
}
|
||||
|
||||
type dscpRecord struct {
|
||||
Name string `xml:"name"`
|
||||
Space string `xml:"space"`
|
||||
XMLName xml.Name `xml:"registry"`
|
||||
Title string `xml:"title"`
|
||||
Updated string `xml:"updated"`
|
||||
Note string `xml:"note"`
|
||||
RegTitle string `xml:"registry>title"`
|
||||
PoolRecords []struct {
|
||||
Name string `xml:"name"`
|
||||
Space string `xml:"space"`
|
||||
} `xml:"registry>record"`
|
||||
Records []struct {
|
||||
Name string `xml:"name"`
|
||||
Space string `xml:"space"`
|
||||
} `xml:"registry>registry>record"`
|
||||
}
|
||||
|
||||
type canonDSCPRecord struct {
|
||||
@ -145,17 +146,15 @@ func parseTOSTCByte(w io.Writer, r io.Reader) error {
|
||||
}
|
||||
|
||||
type tosTCByte struct {
|
||||
XMLName xml.Name `xml:"registry"`
|
||||
Title string `xml:"title"`
|
||||
Updated string `xml:"updated"`
|
||||
Note string `xml:"note"`
|
||||
RegTitle string `xml:"registry>title"`
|
||||
Records []tosTCByteRecord `xml:"registry>record"`
|
||||
}
|
||||
|
||||
type tosTCByteRecord struct {
|
||||
Binary string `xml:"binary"`
|
||||
Keyword string `xml:"keyword"`
|
||||
XMLName xml.Name `xml:"registry"`
|
||||
Title string `xml:"title"`
|
||||
Updated string `xml:"updated"`
|
||||
Note string `xml:"note"`
|
||||
RegTitle string `xml:"registry>title"`
|
||||
Records []struct {
|
||||
Binary string `xml:"binary"`
|
||||
Keyword string `xml:"keyword"`
|
||||
} `xml:"registry>record"`
|
||||
}
|
||||
|
||||
type canonTOSTCByteRecord struct {
|
||||
|
@ -3,7 +3,7 @@
|
||||
|
||||
package ipv6
|
||||
|
||||
// Internet Control Message Protocol version 6 (ICMPv6) Parameters, Updated: 2012-11-12
|
||||
// Internet Control Message Protocol version 6 (ICMPv6) Parameters, Updated: 2013-07-03
|
||||
const (
|
||||
ICMPTypeDestinationUnreachable ICMPType = 1 // Destination Unreachable
|
||||
ICMPTypePacketTooBig ICMPType = 2 // Packet Too Big
|
||||
@ -41,7 +41,7 @@ const (
|
||||
ICMPTypeDuplicateAddressConfirmation ICMPType = 158 // Duplicate Address Confirmation
|
||||
)
|
||||
|
||||
// Internet Control Message Protocol version 6 (ICMPv6) Parameters, Updated: 2012-11-12
|
||||
// Internet Control Message Protocol version 6 (ICMPv6) Parameters, Updated: 2013-07-03
|
||||
var icmpTypes = map[ICMPType]string{
|
||||
1: "destination unreachable",
|
||||
2: "packet too big",
|
||||
|
@ -1,9 +1,9 @@
|
||||
// go run gentv.go
|
||||
// go run gentest.go
|
||||
// GENERATED BY THE COMMAND ABOVE; DO NOT EDIT
|
||||
|
||||
package ipv6_test
|
||||
|
||||
// Differentiated Services Field Codepoints, Updated: 2010-05-11
|
||||
// Differentiated Services Field Codepoints (DSCP), Updated: 2013-06-25
|
||||
const (
|
||||
DiffServCS0 = 0x0 // CS0
|
||||
DiffServCS1 = 0x20 // CS1
|
||||
|
@ -43,26 +43,12 @@ func NewConfig(server, origin string) (config *Config, err error) {
|
||||
func NewClient(config *Config, rwc io.ReadWriteCloser) (ws *Conn, err error) {
|
||||
br := bufio.NewReader(rwc)
|
||||
bw := bufio.NewWriter(rwc)
|
||||
switch config.Version {
|
||||
case ProtocolVersionHixie75:
|
||||
err = hixie75ClientHandshake(config, br, bw)
|
||||
case ProtocolVersionHixie76, ProtocolVersionHybi00:
|
||||
err = hixie76ClientHandshake(config, br, bw)
|
||||
case ProtocolVersionHybi08, ProtocolVersionHybi13:
|
||||
err = hybiClientHandshake(config, br, bw)
|
||||
default:
|
||||
err = ErrBadProtocolVersion
|
||||
}
|
||||
err = hybiClientHandshake(config, br, bw)
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
buf := bufio.NewReadWriter(br, bw)
|
||||
switch config.Version {
|
||||
case ProtocolVersionHixie75, ProtocolVersionHixie76, ProtocolVersionHybi00:
|
||||
ws = newHixieClientConn(config, buf, rwc)
|
||||
case ProtocolVersionHybi08, ProtocolVersionHybi13:
|
||||
ws = newHybiClientConn(config, buf, rwc)
|
||||
}
|
||||
ws = newHybiClientConn(config, buf, rwc)
|
||||
return
|
||||
}
|
||||
|
||||
|
@ -385,21 +385,8 @@ func getNonceAccept(nonce []byte) (expected []byte, err error) {
|
||||
return
|
||||
}
|
||||
|
||||
func isHybiVersion(version int) bool {
|
||||
switch version {
|
||||
case ProtocolVersionHybi08, ProtocolVersionHybi13:
|
||||
return true
|
||||
default:
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
// Client handshake described in draft-ietf-hybi-thewebsocket-protocol-17
|
||||
func hybiClientHandshake(config *Config, br *bufio.Reader, bw *bufio.Writer) (err error) {
|
||||
if !isHybiVersion(config.Version) {
|
||||
panic("wrong protocol version.")
|
||||
}
|
||||
|
||||
bw.WriteString("GET " + config.Location.RequestURI() + " HTTP/1.1\r\n")
|
||||
|
||||
bw.WriteString("Host: " + config.Location.Host + "\r\n")
|
||||
@ -410,11 +397,12 @@ func hybiClientHandshake(config *Config, br *bufio.Reader, bw *bufio.Writer) (er
|
||||
nonce = []byte(config.handshakeData["key"])
|
||||
}
|
||||
bw.WriteString("Sec-WebSocket-Key: " + string(nonce) + "\r\n")
|
||||
if config.Version == ProtocolVersionHybi13 {
|
||||
bw.WriteString("Origin: " + strings.ToLower(config.Origin.String()) + "\r\n")
|
||||
} else if config.Version == ProtocolVersionHybi08 {
|
||||
bw.WriteString("Sec-WebSocket-Origin: " + strings.ToLower(config.Origin.String()) + "\r\n")
|
||||
bw.WriteString("Origin: " + strings.ToLower(config.Origin.String()) + "\r\n")
|
||||
|
||||
if config.Version != ProtocolVersionHybi13 {
|
||||
return ErrBadProtocolVersion
|
||||
}
|
||||
|
||||
bw.WriteString("Sec-WebSocket-Version: " + fmt.Sprintf("%d", config.Version) + "\r\n")
|
||||
if len(config.Protocol) > 0 {
|
||||
bw.WriteString("Sec-WebSocket-Protocol: " + strings.Join(config.Protocol, ", ") + "\r\n")
|
||||
@ -500,8 +488,6 @@ func (c *hybiServerHandshaker) ReadHandshake(buf *bufio.Reader, req *http.Reques
|
||||
switch version {
|
||||
case "13":
|
||||
c.Version = ProtocolVersionHybi13
|
||||
case "8":
|
||||
c.Version = ProtocolVersionHybi08
|
||||
default:
|
||||
return http.StatusBadRequest, ErrBadWebSocketVersion
|
||||
}
|
||||
@ -536,8 +522,6 @@ func Origin(config *Config, req *http.Request) (*url.URL, error) {
|
||||
switch config.Version {
|
||||
case ProtocolVersionHybi13:
|
||||
origin = req.Header.Get("Origin")
|
||||
case ProtocolVersionHybi08:
|
||||
origin = req.Header.Get("Sec-Websocket-Origin")
|
||||
}
|
||||
if origin == "null" {
|
||||
return nil, nil
|
||||
|
@ -157,68 +157,6 @@ Sec-WebSocket-Protocol: chat
|
||||
}
|
||||
}
|
||||
|
||||
func TestHybiClientHandshakeHybi08(t *testing.T) {
|
||||
b := bytes.NewBuffer([]byte{})
|
||||
bw := bufio.NewWriter(b)
|
||||
br := bufio.NewReader(strings.NewReader(`HTTP/1.1 101 Switching Protocols
|
||||
Upgrade: websocket
|
||||
Connection: Upgrade
|
||||
Sec-WebSocket-Accept: s3pPLMBiTxaQ9kYGzzhZRbK+xOo=
|
||||
Sec-WebSocket-Protocol: chat
|
||||
|
||||
`))
|
||||
var err error
|
||||
config := new(Config)
|
||||
config.Location, err = url.ParseRequestURI("ws://server.example.com/chat")
|
||||
if err != nil {
|
||||
t.Fatal("location url", err)
|
||||
}
|
||||
config.Origin, err = url.ParseRequestURI("http://example.com")
|
||||
if err != nil {
|
||||
t.Fatal("origin url", err)
|
||||
}
|
||||
config.Protocol = append(config.Protocol, "chat")
|
||||
config.Protocol = append(config.Protocol, "superchat")
|
||||
config.Version = ProtocolVersionHybi08
|
||||
|
||||
config.handshakeData = map[string]string{
|
||||
"key": "dGhlIHNhbXBsZSBub25jZQ==",
|
||||
}
|
||||
err = hybiClientHandshake(config, br, bw)
|
||||
if err != nil {
|
||||
t.Errorf("handshake failed: %v", err)
|
||||
}
|
||||
req, err := http.ReadRequest(bufio.NewReader(b))
|
||||
if err != nil {
|
||||
t.Fatalf("read request: %v", err)
|
||||
}
|
||||
if req.Method != "GET" {
|
||||
t.Errorf("request method expected GET, but got %q", req.Method)
|
||||
}
|
||||
if req.URL.Path != "/chat" {
|
||||
t.Errorf("request path expected /demo, but got %q", req.URL.Path)
|
||||
}
|
||||
if req.Proto != "HTTP/1.1" {
|
||||
t.Errorf("request proto expected HTTP/1.1, but got %q", req.Proto)
|
||||
}
|
||||
if req.Host != "server.example.com" {
|
||||
t.Errorf("request Host expected example.com, but got %v", req.Host)
|
||||
}
|
||||
var expectedHeader = map[string]string{
|
||||
"Connection": "Upgrade",
|
||||
"Upgrade": "websocket",
|
||||
"Sec-Websocket-Key": config.handshakeData["key"],
|
||||
"Sec-Websocket-Origin": config.Origin.String(),
|
||||
"Sec-Websocket-Protocol": "chat, superchat",
|
||||
"Sec-Websocket-Version": fmt.Sprintf("%d", ProtocolVersionHybi08),
|
||||
}
|
||||
for k, v := range expectedHeader {
|
||||
if req.Header.Get(k) != v {
|
||||
t.Errorf(fmt.Sprintf("%s expected %q but got %q", k, v, req.Header.Get(k)))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestHybiServerHandshake(t *testing.T) {
|
||||
config := new(Config)
|
||||
handshaker := &hybiServerHandshaker{Config: config}
|
||||
@ -314,52 +252,6 @@ Sec-WebSocket-Version: 13
|
||||
}
|
||||
}
|
||||
|
||||
func TestHybiServerHandshakeHybi08(t *testing.T) {
|
||||
config := new(Config)
|
||||
handshaker := &hybiServerHandshaker{Config: config}
|
||||
br := bufio.NewReader(strings.NewReader(`GET /chat HTTP/1.1
|
||||
Host: server.example.com
|
||||
Upgrade: websocket
|
||||
Connection: Upgrade
|
||||
Sec-WebSocket-Key: dGhlIHNhbXBsZSBub25jZQ==
|
||||
Sec-WebSocket-Origin: http://example.com
|
||||
Sec-WebSocket-Protocol: chat, superchat
|
||||
Sec-WebSocket-Version: 8
|
||||
|
||||
`))
|
||||
req, err := http.ReadRequest(br)
|
||||
if err != nil {
|
||||
t.Fatal("request", err)
|
||||
}
|
||||
code, err := handshaker.ReadHandshake(br, req)
|
||||
if err != nil {
|
||||
t.Errorf("handshake failed: %v", err)
|
||||
}
|
||||
if code != http.StatusSwitchingProtocols {
|
||||
t.Errorf("status expected %q but got %q", http.StatusSwitchingProtocols, code)
|
||||
}
|
||||
b := bytes.NewBuffer([]byte{})
|
||||
bw := bufio.NewWriter(b)
|
||||
|
||||
config.Protocol = []string{"chat"}
|
||||
|
||||
err = handshaker.AcceptHandshake(bw)
|
||||
if err != nil {
|
||||
t.Errorf("handshake response failed: %v", err)
|
||||
}
|
||||
expectedResponse := strings.Join([]string{
|
||||
"HTTP/1.1 101 Switching Protocols",
|
||||
"Upgrade: websocket",
|
||||
"Connection: Upgrade",
|
||||
"Sec-WebSocket-Accept: s3pPLMBiTxaQ9kYGzzhZRbK+xOo=",
|
||||
"Sec-WebSocket-Protocol: chat",
|
||||
"", ""}, "\r\n")
|
||||
|
||||
if b.String() != expectedResponse {
|
||||
t.Errorf("handshake expected %q but got %q", expectedResponse, b.String())
|
||||
}
|
||||
}
|
||||
|
||||
func TestHybiServerHandshakeHybiBadVersion(t *testing.T) {
|
||||
config := new(Config)
|
||||
handshaker := &hybiServerHandshaker{Config: config}
|
||||
|
@ -22,14 +22,6 @@ func newServerConn(rwc io.ReadWriteCloser, buf *bufio.ReadWriter, req *http.Requ
|
||||
buf.Flush()
|
||||
return
|
||||
}
|
||||
if err != nil {
|
||||
hs = &hixie76ServerHandshaker{Config: config}
|
||||
code, err = hs.ReadHandshake(buf.Reader, req)
|
||||
}
|
||||
if err != nil {
|
||||
hs = &hixie75ServerHandshaker{Config: config}
|
||||
code, err = hs.ReadHandshake(buf.Reader, req)
|
||||
}
|
||||
if err != nil {
|
||||
fmt.Fprintf(buf, "HTTP/1.1 %03d %s\r\n", code, http.StatusText(code))
|
||||
buf.WriteString("\r\n")
|
||||
|
@ -21,13 +21,9 @@ import (
|
||||
)
|
||||
|
||||
const (
|
||||
ProtocolVersionHixie75 = -75
|
||||
ProtocolVersionHixie76 = -76
|
||||
ProtocolVersionHybi00 = 0
|
||||
ProtocolVersionHybi08 = 8
|
||||
ProtocolVersionHybi13 = 13
|
||||
ProtocolVersionHybi = ProtocolVersionHybi13
|
||||
SupportedProtocolVersion = "13, 8"
|
||||
SupportedProtocolVersion = "13"
|
||||
|
||||
ContinuationFrame = 0
|
||||
TextFrame = 1
|
||||
|
@ -286,6 +286,20 @@ func TestTrailingSpaces(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
func TestDialConfigBadVersion(t *testing.T) {
|
||||
once.Do(startServer)
|
||||
config := newConfig(t, "/echo")
|
||||
config.Version = 1234
|
||||
|
||||
_, err := DialConfig(config)
|
||||
|
||||
if dialerr, ok := err.(*DialError); ok {
|
||||
if dialerr.Err != ErrBadProtocolVersion {
|
||||
t.Errorf("dial expected err %q but got %q", ErrBadProtocolVersion, dialerr.Err)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestSmallBuffer(t *testing.T) {
|
||||
// http://code.google.com/p/go/issues/detail?id=1145
|
||||
// Read should be able to handle reading a fragment of a frame.
|
||||
|
@ -31,19 +31,19 @@ func main() {
|
||||
c := etcd.NewClient() // default binds to http://0.0.0.0:4001
|
||||
|
||||
// SET the value "bar" to the key "foo" with zero TTL
|
||||
// returns a: *store.Response
|
||||
// returns a: *Response
|
||||
res, _ := c.Set("foo", "bar", 0)
|
||||
fmt.Printf("set response: %+v\n", res)
|
||||
|
||||
// GET the value that is stored for the key "foo"
|
||||
// return a slice: []*store.Response
|
||||
// return a slice: []*Response
|
||||
values, _ := c.Get("foo")
|
||||
for i, res := range values { // .. and print them out
|
||||
fmt.Printf("[%d] get response: %+v\n", i, res)
|
||||
}
|
||||
|
||||
// DELETE the key "foo"
|
||||
// returns a: *store.Response
|
||||
// returns a: *Response
|
||||
res, _ = c.Delete("foo")
|
||||
fmt.Printf("delete response: %+v\n", res)
|
||||
}
|
||||
|
@ -36,12 +36,16 @@ type Client struct {
|
||||
}
|
||||
|
||||
// Setup a basic conf and cluster
|
||||
func NewClient() *Client {
|
||||
func NewClient(machines []string) *Client {
|
||||
// if an empty slice was sent in then just assume localhost
|
||||
if len(machines) == 0 {
|
||||
machines = []string{"http://127.0.0.1:4001"}
|
||||
}
|
||||
|
||||
// default leader and machines
|
||||
cluster := Cluster{
|
||||
Leader: "http://127.0.0.1:4001",
|
||||
Machines: []string{"http://127.0.0.1:4001"},
|
||||
Leader: machines[0],
|
||||
Machines: machines,
|
||||
}
|
||||
|
||||
config := Config{
|
||||
@ -107,6 +111,10 @@ func (c *Client) SetCluster(machines []string) bool {
|
||||
return success
|
||||
}
|
||||
|
||||
func (c *Client) GetCluster() []string {
|
||||
return c.cluster.Machines
|
||||
}
|
||||
|
||||
// sycn cluster information using the existing machine list
|
||||
func (c *Client) SyncCluster() bool {
|
||||
success := c.internalSyncCluster(c.cluster.Machines)
|
||||
@ -128,14 +136,16 @@ func (c *Client) internalSyncCluster(machines []string) bool {
|
||||
// try another machine in the cluster
|
||||
continue
|
||||
}
|
||||
|
||||
// update Machines List
|
||||
c.cluster.Machines = strings.Split(string(b), ",")
|
||||
c.cluster.Machines = strings.Split(string(b), ", ")
|
||||
|
||||
// update leader
|
||||
// the first one in the machine list is the leader
|
||||
logger.Debugf("update.leader[%s,%s]", c.cluster.Leader, c.cluster.Machines[0])
|
||||
c.cluster.Leader = c.cluster.Machines[0]
|
||||
|
||||
logger.Debug("sync.machines ", c.cluster.Machines)
|
||||
return true
|
||||
}
|
||||
}
|
||||
@ -146,6 +156,9 @@ func (c *Client) internalSyncCluster(machines []string) bool {
|
||||
func (c *Client) createHttpPath(serverName string, _path string) string {
|
||||
u, _ := url.Parse(serverName)
|
||||
u.Path = path.Join(u.Path, "/", _path)
|
||||
if u.Scheme == "" {
|
||||
u.Scheme = "http"
|
||||
}
|
||||
return u.String()
|
||||
}
|
||||
|
||||
|
@ -3,6 +3,8 @@ package etcd
|
||||
import (
|
||||
"fmt"
|
||||
"testing"
|
||||
"net/url"
|
||||
"net"
|
||||
)
|
||||
|
||||
// To pass this test, we need to create a cluster of 3 machines
|
||||
@ -10,13 +12,31 @@ import (
|
||||
func TestSync(t *testing.T) {
|
||||
fmt.Println("Make sure there are three nodes at 0.0.0.0:4001-4003")
|
||||
|
||||
c := NewClient()
|
||||
c := NewClient(nil)
|
||||
|
||||
success := c.SyncCluster()
|
||||
if !success {
|
||||
t.Fatal("cannot sync machines")
|
||||
}
|
||||
|
||||
for _, m := range(c.GetCluster()) {
|
||||
u, err := url.Parse(m)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
if u.Scheme != "http" {
|
||||
t.Fatal("scheme must be http")
|
||||
}
|
||||
|
||||
host, _, err := net.SplitHostPort(u.Host)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
if host != "127.0.0.1" {
|
||||
t.Fatal("Host must be 127.0.0.1")
|
||||
}
|
||||
}
|
||||
|
||||
badMachines := []string{"abc", "edef"}
|
||||
|
||||
success = c.SetCluster(badMachines)
|
||||
|
@ -2,13 +2,12 @@ package etcd
|
||||
|
||||
import (
|
||||
"encoding/json"
|
||||
"github.com/coreos/etcd/store"
|
||||
"io/ioutil"
|
||||
"net/http"
|
||||
"path"
|
||||
)
|
||||
|
||||
func (c *Client) Delete(key string) (*store.Response, error) {
|
||||
func (c *Client) Delete(key string) (*Response, error) {
|
||||
|
||||
resp, err := c.sendRequest("DELETE", path.Join("keys", key), "")
|
||||
|
||||
@ -28,7 +27,7 @@ func (c *Client) Delete(key string) (*store.Response, error) {
|
||||
return nil, handleError(b)
|
||||
}
|
||||
|
||||
var result store.Response
|
||||
var result Response
|
||||
|
||||
err = json.Unmarshal(b, &result)
|
||||
|
||||
|
@ -6,7 +6,7 @@ import (
|
||||
|
||||
func TestDelete(t *testing.T) {
|
||||
|
||||
c := NewClient()
|
||||
c := NewClient(nil)
|
||||
|
||||
c.Set("foo", "bar", 100)
|
||||
result, err := c.Delete("foo")
|
||||
|
@ -2,13 +2,12 @@ package etcd
|
||||
|
||||
import (
|
||||
"encoding/json"
|
||||
"github.com/coreos/etcd/store"
|
||||
"io/ioutil"
|
||||
"net/http"
|
||||
"path"
|
||||
)
|
||||
|
||||
func (c *Client) Get(key string) ([]*store.Response, error) {
|
||||
func (c *Client) Get(key string) ([]*Response, error) {
|
||||
logger.Debugf("get %s [%s]", key, c.cluster.Leader)
|
||||
resp, err := c.sendRequest("GET", path.Join("keys", key), "")
|
||||
|
||||
@ -36,7 +35,7 @@ func (c *Client) Get(key string) ([]*store.Response, error) {
|
||||
// GetTo gets the value of the key from a given machine address.
|
||||
// If the given machine is not available it returns an error.
|
||||
// Mainly use for testing purpose
|
||||
func (c *Client) GetFrom(key string, addr string) ([]*store.Response, error) {
|
||||
func (c *Client) GetFrom(key string, addr string) ([]*Response, error) {
|
||||
httpPath := c.createHttpPath(addr, path.Join(version, "keys", key))
|
||||
|
||||
resp, err := c.httpClient.Get(httpPath)
|
||||
@ -61,10 +60,10 @@ func (c *Client) GetFrom(key string, addr string) ([]*store.Response, error) {
|
||||
}
|
||||
|
||||
// Convert byte stream to response.
|
||||
func convertGetResponse(b []byte) ([]*store.Response, error) {
|
||||
func convertGetResponse(b []byte) ([]*Response, error) {
|
||||
|
||||
var results []*store.Response
|
||||
var result *store.Response
|
||||
var results []*Response
|
||||
var result *Response
|
||||
|
||||
err := json.Unmarshal(b, &result)
|
||||
|
||||
@ -76,7 +75,7 @@ func convertGetResponse(b []byte) ([]*store.Response, error) {
|
||||
}
|
||||
|
||||
} else {
|
||||
results = make([]*store.Response, 1)
|
||||
results = make([]*Response, 1)
|
||||
results[0] = result
|
||||
}
|
||||
return results, nil
|
||||
|
@ -7,7 +7,7 @@ import (
|
||||
|
||||
func TestGet(t *testing.T) {
|
||||
|
||||
c := NewClient()
|
||||
c := NewClient(nil)
|
||||
|
||||
c.Set("foo", "bar", 100)
|
||||
|
||||
|
@ -6,7 +6,7 @@ import (
|
||||
)
|
||||
|
||||
func TestList(t *testing.T) {
|
||||
c := NewClient()
|
||||
c := NewClient(nil)
|
||||
|
||||
c.Set("foo_list/foo", "bar", 100)
|
||||
c.Set("foo_list/fooo", "barbar", 100)
|
||||
|
26
third_party/github.com/coreos/go-etcd/etcd/response.go
vendored
Normal file
26
third_party/github.com/coreos/go-etcd/etcd/response.go
vendored
Normal file
@ -0,0 +1,26 @@
|
||||
package etcd
|
||||
|
||||
import (
|
||||
"time"
|
||||
)
|
||||
|
||||
// The response object from the server.
|
||||
type Response struct {
|
||||
Action string `json:"action"`
|
||||
Key string `json:"key"`
|
||||
Dir bool `json:"dir,omitempty"`
|
||||
PrevValue string `json:"prevValue,omitempty"`
|
||||
Value string `json:"value,omitempty"`
|
||||
|
||||
// If the key did not exist before the action,
|
||||
// this field should be set to true
|
||||
NewKey bool `json:"newKey,omitempty"`
|
||||
|
||||
Expiration *time.Time `json:"expiration,omitempty"`
|
||||
|
||||
// Time to live in second
|
||||
TTL int64 `json:"ttl,omitempty"`
|
||||
|
||||
// The command index of the raft machine when the command is executed
|
||||
Index uint64 `json:"index"`
|
||||
}
|
@ -3,14 +3,13 @@ package etcd
|
||||
import (
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"github.com/coreos/etcd/store"
|
||||
"io/ioutil"
|
||||
"net/http"
|
||||
"net/url"
|
||||
"path"
|
||||
)
|
||||
|
||||
func (c *Client) Set(key string, value string, ttl uint64) (*store.Response, error) {
|
||||
func (c *Client) Set(key string, value string, ttl uint64) (*Response, error) {
|
||||
logger.Debugf("set %s, %s, ttl: %d, [%s]", key, value, ttl, c.cluster.Leader)
|
||||
v := url.Values{}
|
||||
v.Set("value", value)
|
||||
@ -45,7 +44,7 @@ func (c *Client) Set(key string, value string, ttl uint64) (*store.Response, err
|
||||
// SetTo sets the value of the key to a given machine address.
|
||||
// If the given machine is not available or is not leader it returns an error
|
||||
// Mainly use for testing purpose.
|
||||
func (c *Client) SetTo(key string, value string, ttl uint64, addr string) (*store.Response, error) {
|
||||
func (c *Client) SetTo(key string, value string, ttl uint64, addr string) (*Response, error) {
|
||||
v := url.Values{}
|
||||
v.Set("value", value)
|
||||
|
||||
@ -77,8 +76,8 @@ func (c *Client) SetTo(key string, value string, ttl uint64, addr string) (*stor
|
||||
}
|
||||
|
||||
// Convert byte stream to response.
|
||||
func convertSetResponse(b []byte) (*store.Response, error) {
|
||||
var result store.Response
|
||||
func convertSetResponse(b []byte) (*Response, error) {
|
||||
var result Response
|
||||
|
||||
err := json.Unmarshal(b, &result)
|
||||
|
||||
|
@ -6,7 +6,7 @@ import (
|
||||
)
|
||||
|
||||
func TestSet(t *testing.T) {
|
||||
c := NewClient()
|
||||
c := NewClient(nil)
|
||||
|
||||
result, err := c.Set("foo", "bar", 100)
|
||||
|
||||
|
@ -3,14 +3,13 @@ package etcd
|
||||
import (
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"github.com/coreos/etcd/store"
|
||||
"io/ioutil"
|
||||
"net/http"
|
||||
"net/url"
|
||||
"path"
|
||||
)
|
||||
|
||||
func (c *Client) TestAndSet(key string, prevValue string, value string, ttl uint64) (*store.Response, bool, error) {
|
||||
func (c *Client) TestAndSet(key string, prevValue string, value string, ttl uint64) (*Response, bool, error) {
|
||||
logger.Debugf("set %s, %s[%s], ttl: %d, [%s]", key, value, prevValue, ttl, c.cluster.Leader)
|
||||
v := url.Values{}
|
||||
v.Set("value", value)
|
||||
@ -39,7 +38,7 @@ func (c *Client) TestAndSet(key string, prevValue string, value string, ttl uint
|
||||
return nil, false, handleError(b)
|
||||
}
|
||||
|
||||
var result store.Response
|
||||
var result Response
|
||||
|
||||
err = json.Unmarshal(b, &result)
|
||||
|
||||
|
@ -6,7 +6,7 @@ import (
|
||||
)
|
||||
|
||||
func TestTestAndSet(t *testing.T) {
|
||||
c := NewClient()
|
||||
c := NewClient(nil)
|
||||
|
||||
c.Set("foo_testAndSet", "bar", 100)
|
||||
|
||||
|
@ -4,7 +4,6 @@ import (
|
||||
"encoding/json"
|
||||
"errors"
|
||||
"fmt"
|
||||
"github.com/coreos/etcd/store"
|
||||
"io/ioutil"
|
||||
"net/http"
|
||||
"net/url"
|
||||
@ -16,6 +15,11 @@ type respAndErr struct {
|
||||
err error
|
||||
}
|
||||
|
||||
// Errors introduced by the Watch command.
|
||||
var (
|
||||
ErrWatchStoppedByUser = errors.New("Watch stopped by the user via stop channel")
|
||||
)
|
||||
|
||||
// Watch any change under the given prefix.
|
||||
// When a sinceIndex is given, watch will try to scan from that index to the last index
|
||||
// and will return any changes under the given prefix during the history
|
||||
@ -23,7 +27,7 @@ type respAndErr struct {
|
||||
// channel. And after someone receive the channel, it will go on to watch that prefix.
|
||||
// If a stop channel is given, client can close long-term watch using the stop channel
|
||||
|
||||
func (c *Client) Watch(prefix string, sinceIndex uint64, receiver chan *store.Response, stop chan bool) (*store.Response, error) {
|
||||
func (c *Client) Watch(prefix string, sinceIndex uint64, receiver chan *Response, stop chan bool) (*Response, error) {
|
||||
logger.Debugf("watch %s [%s]", prefix, c.cluster.Leader)
|
||||
if receiver == nil {
|
||||
return c.watchOnce(prefix, sinceIndex, stop)
|
||||
@ -45,7 +49,7 @@ func (c *Client) Watch(prefix string, sinceIndex uint64, receiver chan *store.Re
|
||||
|
||||
// helper func
|
||||
// return when there is change under the given prefix
|
||||
func (c *Client) watchOnce(key string, sinceIndex uint64, stop chan bool) (*store.Response, error) {
|
||||
func (c *Client) watchOnce(key string, sinceIndex uint64, stop chan bool) (*Response, error) {
|
||||
|
||||
var resp *http.Response
|
||||
var err error
|
||||
@ -66,7 +70,7 @@ func (c *Client) watchOnce(key string, sinceIndex uint64, stop chan bool) (*stor
|
||||
resp, err = res.resp, res.err
|
||||
|
||||
case <-stop:
|
||||
resp, err = nil, errors.New("User stoped watch")
|
||||
resp, err = nil, ErrWatchStoppedByUser
|
||||
}
|
||||
} else {
|
||||
resp, err = c.sendWatchRequest(key, sinceIndex)
|
||||
@ -89,7 +93,7 @@ func (c *Client) watchOnce(key string, sinceIndex uint64, stop chan bool) (*stor
|
||||
return nil, handleError(b)
|
||||
}
|
||||
|
||||
var result store.Response
|
||||
var result Response
|
||||
|
||||
err = json.Unmarshal(b, &result)
|
||||
|
||||
|
@ -2,13 +2,12 @@ package etcd
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"github.com/coreos/etcd/store"
|
||||
"testing"
|
||||
"time"
|
||||
)
|
||||
|
||||
func TestWatch(t *testing.T) {
|
||||
c := NewClient()
|
||||
c := NewClient(nil)
|
||||
|
||||
go setHelper("bar", c)
|
||||
|
||||
@ -30,14 +29,17 @@ func TestWatch(t *testing.T) {
|
||||
t.Fatalf("Watch with Index failed with %s %s %v %v", result.Key, result.Value, result.TTL, result.Index)
|
||||
}
|
||||
|
||||
ch := make(chan *store.Response, 10)
|
||||
ch := make(chan *Response, 10)
|
||||
stop := make(chan bool, 1)
|
||||
|
||||
go setLoop("bar", c)
|
||||
|
||||
go reciver(ch, stop)
|
||||
go receiver(ch, stop)
|
||||
|
||||
c.Watch("watch_foo", 0, ch, stop)
|
||||
_, err = c.Watch("watch_foo", 0, ch, stop)
|
||||
if err != ErrWatchStoppedByUser {
|
||||
t.Fatalf("Watch returned a non-user stop error")
|
||||
}
|
||||
}
|
||||
|
||||
func setHelper(value string, c *Client) {
|
||||
@ -54,7 +56,7 @@ func setLoop(value string, c *Client) {
|
||||
}
|
||||
}
|
||||
|
||||
func reciver(c chan *store.Response, stop chan bool) {
|
||||
func receiver(c chan *Response, stop chan bool) {
|
||||
for i := 0; i < 10; i++ {
|
||||
<-c
|
||||
}
|
||||
|
3
third_party/github.com/coreos/go-etcd/examples/sync-cluster/README.md
vendored
Normal file
3
third_party/github.com/coreos/go-etcd/examples/sync-cluster/README.md
vendored
Normal file
@ -0,0 +1,3 @@
|
||||
Example script from the sync-cluster bug https://github.com/coreos/go-etcd/issues/27
|
||||
|
||||
TODO: turn this into a test case
|
51
third_party/github.com/coreos/go-etcd/examples/sync-cluster/sync-cluster.go
vendored
Normal file
51
third_party/github.com/coreos/go-etcd/examples/sync-cluster/sync-cluster.go
vendored
Normal file
@ -0,0 +1,51 @@
|
||||
|
||||
package main
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"github.com/coreos/go-etcd/etcd"
|
||||
"strconv"
|
||||
"time"
|
||||
)
|
||||
|
||||
func main() {
|
||||
fmt.Println("etcd-client started")
|
||||
c := etcd.NewClient(nil)
|
||||
c.SetCluster([]string{
|
||||
"http://127.0.0.1:4001",
|
||||
"http://127.0.0.1:4002",
|
||||
"http://127.0.0.1:4003",
|
||||
})
|
||||
|
||||
ticker := time.NewTicker(time.Second * 3)
|
||||
|
||||
for {
|
||||
select {
|
||||
case d := <-ticker.C:
|
||||
n := d.Second()
|
||||
if n <= 0 {
|
||||
n = 60
|
||||
}
|
||||
|
||||
for ok := c.SyncCluster(); ok == false; {
|
||||
fmt.Println("SyncCluster failed, trying again")
|
||||
time.Sleep(100 * time.Millisecond)
|
||||
}
|
||||
|
||||
result, err := c.Set("foo", "exp_"+strconv.Itoa(n), 0)
|
||||
if err != nil {
|
||||
fmt.Println("set error", err)
|
||||
} else {
|
||||
fmt.Printf("set %+v\n", result)
|
||||
}
|
||||
|
||||
ss, err := c.Get("foo")
|
||||
if err != nil {
|
||||
fmt.Println("get error", err)
|
||||
} else {
|
||||
fmt.Println(len(ss))
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
}
|
@ -1,6 +1,5 @@
|
||||
package log
|
||||
|
||||
// Copyright 2013, David Fisher. All rights reserved.
|
||||
// Copyright 2013, CoreOS, Inc. All rights reserved.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
@ -43,6 +42,7 @@ func (logger *Logger) Logf(priority Priority, format string, v ...interface{}) {
|
||||
logger.Log(priority, fmt.Sprintf(format, v...))
|
||||
}
|
||||
|
||||
|
||||
func (logger *Logger) Emergency(v ...interface{}) {
|
||||
logger.Log(PriEmerg, v...)
|
||||
}
|
||||
@ -99,6 +99,7 @@ func (logger *Logger) Debugf(format string, v ...interface{}) {
|
||||
logger.Log(PriDebug, fmt.Sprintf(format, v...))
|
||||
}
|
||||
|
||||
|
||||
func Emergency(v ...interface{}) {
|
||||
defaultLogger.Log(PriEmerg, v...)
|
||||
}
|
||||
@ -157,56 +158,57 @@ func Debugf(format string, v ...interface{}) {
|
||||
|
||||
// Standard library log functions
|
||||
|
||||
func (logger *Logger) Fatalln(v ...interface{}) {
|
||||
func (logger *Logger)Fatalln (v ...interface{}) {
|
||||
logger.Log(PriCrit, v...)
|
||||
os.Exit(1)
|
||||
}
|
||||
func (logger *Logger) Fatalf(format string, v ...interface{}) {
|
||||
func (logger *Logger)Fatalf (format string, v ...interface{}) {
|
||||
logger.Logf(PriCrit, format, v...)
|
||||
os.Exit(1)
|
||||
}
|
||||
|
||||
func (logger *Logger) Panicln(v ...interface{}) {
|
||||
func (logger *Logger)Panicln (v ...interface{}) {
|
||||
s := fmt.Sprint(v...)
|
||||
logger.Log(PriErr, s)
|
||||
panic(s)
|
||||
}
|
||||
func (logger *Logger) Panicf(format string, v ...interface{}) {
|
||||
func (logger *Logger)Panicf (format string, v ...interface{}) {
|
||||
s := fmt.Sprintf(format, v...)
|
||||
logger.Log(PriErr, s)
|
||||
panic(s)
|
||||
}
|
||||
|
||||
func (logger *Logger) Println(v ...interface{}) {
|
||||
func (logger *Logger)Println (v ...interface{}) {
|
||||
logger.Log(PriInfo, v...)
|
||||
}
|
||||
func (logger *Logger) Printf(format string, v ...interface{}) {
|
||||
func (logger *Logger)Printf (format string, v ...interface{}) {
|
||||
logger.Logf(PriInfo, format, v...)
|
||||
}
|
||||
|
||||
func Fatalln(v ...interface{}) {
|
||||
|
||||
func Fatalln (v ...interface{}) {
|
||||
defaultLogger.Log(PriCrit, v...)
|
||||
os.Exit(1)
|
||||
}
|
||||
func Fatalf(format string, v ...interface{}) {
|
||||
func Fatalf (format string, v ...interface{}) {
|
||||
defaultLogger.Logf(PriCrit, format, v...)
|
||||
os.Exit(1)
|
||||
}
|
||||
|
||||
func Panicln(v ...interface{}) {
|
||||
func Panicln (v ...interface{}) {
|
||||
s := fmt.Sprint(v...)
|
||||
defaultLogger.Log(PriErr, s)
|
||||
panic(s)
|
||||
}
|
||||
func Panicf(format string, v ...interface{}) {
|
||||
func Panicf (format string, v ...interface{}) {
|
||||
s := fmt.Sprintf(format, v...)
|
||||
defaultLogger.Log(PriErr, s)
|
||||
panic(s)
|
||||
}
|
||||
|
||||
func Println(v ...interface{}) {
|
||||
func Println (v ...interface{}) {
|
||||
defaultLogger.Log(PriInfo, v...)
|
||||
}
|
||||
func Printf(format string, v ...interface{}) {
|
||||
func Printf (format string, v ...interface{}) {
|
||||
defaultLogger.Logf(PriInfo, format, v...)
|
||||
}
|
||||
|
@ -1,6 +1,5 @@
|
||||
package log
|
||||
|
||||
// Copyright 2013, David Fisher. All rights reserved.
|
||||
// Copyright 2013, CoreOS, Inc. All rights reserved.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
|
@ -1,6 +1,5 @@
|
||||
package log
|
||||
|
||||
// Copyright 2013, David Fisher. All rights reserved.
|
||||
// Copyright 2013, CoreOS, Inc. All rights reserved.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
|
@ -1,6 +1,5 @@
|
||||
package log
|
||||
|
||||
// Copyright 2013, David Fisher. All rights reserved.
|
||||
// Copyright 2013, CoreOS, Inc. All rights reserved.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
|
@ -1,6 +1,5 @@
|
||||
package log
|
||||
|
||||
// Copyright 2013, David Fisher. All rights reserved.
|
||||
// Copyright 2013, CoreOS, Inc. All rights reserved.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
|
@ -57,7 +57,7 @@ A distributed consensus protocol is used for maintaining a consistent state acro
|
||||
Many distributed systems are built upon the Paxos protocol but Paxos can be difficult to understand and there are many gaps between Paxos and real world implementation.
|
||||
|
||||
An alternative is the [Raft distributed consensus protocol][raft-paper] by Diego Ongaro and John Ousterhout.
|
||||
Raft is a protocol built with understandability as a primary tenant and it centers around two things:
|
||||
Raft is a protocol built with understandability as a primary tenet and it centers around two things:
|
||||
|
||||
1. Leader Election
|
||||
2. Replicated Log
|
||||
|
@ -29,7 +29,7 @@ func init() {
|
||||
// A command represents an action to be taken on the replicated state machine.
|
||||
type Command interface {
|
||||
CommandName() string
|
||||
Apply(server *Server) (interface{}, error)
|
||||
Apply(server Server) (interface{}, error)
|
||||
}
|
||||
|
||||
type CommandEncoder interface {
|
||||
|
@ -77,7 +77,7 @@ func (t *HTTPTransporter) RequestVotePath() string {
|
||||
//--------------------------------------
|
||||
|
||||
// Applies Raft routes to an HTTP router for a given server.
|
||||
func (t *HTTPTransporter) Install(server *Server, mux HTTPMuxer) {
|
||||
func (t *HTTPTransporter) Install(server Server, mux HTTPMuxer) {
|
||||
mux.HandleFunc(t.AppendEntriesPath(), t.appendEntriesHandler(server))
|
||||
mux.HandleFunc(t.RequestVotePath(), t.requestVoteHandler(server))
|
||||
}
|
||||
@ -87,14 +87,14 @@ func (t *HTTPTransporter) Install(server *Server, mux HTTPMuxer) {
|
||||
//--------------------------------------
|
||||
|
||||
// Sends an AppendEntries RPC to a peer.
|
||||
func (t *HTTPTransporter) SendAppendEntriesRequest(server *Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse {
|
||||
func (t *HTTPTransporter) SendAppendEntriesRequest(server Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse {
|
||||
var b bytes.Buffer
|
||||
if _, err := req.Encode(&b); err != nil {
|
||||
traceln("transporter.ae.encoding.error:", err)
|
||||
return nil
|
||||
}
|
||||
|
||||
url := fmt.Sprintf("http://%s%s", peer.Name, t.AppendEntriesPath())
|
||||
url := fmt.Sprintf("%s%s", peer.ConnectionString, t.AppendEntriesPath())
|
||||
traceln(server.Name(), "POST", url)
|
||||
|
||||
client := &http.Client{Transport: &http.Transport{DisableKeepAlives: t.DisableKeepAlives}}
|
||||
@ -115,14 +115,14 @@ func (t *HTTPTransporter) SendAppendEntriesRequest(server *Server, peer *Peer, r
|
||||
}
|
||||
|
||||
// Sends a RequestVote RPC to a peer.
|
||||
func (t *HTTPTransporter) SendVoteRequest(server *Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse {
|
||||
func (t *HTTPTransporter) SendVoteRequest(server Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse {
|
||||
var b bytes.Buffer
|
||||
if _, err := req.Encode(&b); err != nil {
|
||||
traceln("transporter.rv.encoding.error:", err)
|
||||
return nil
|
||||
}
|
||||
|
||||
url := fmt.Sprintf("http://%s%s", peer.Name, t.RequestVotePath())
|
||||
url := fmt.Sprintf("%s%s", peer.ConnectionString, t.RequestVotePath())
|
||||
traceln(server.Name(), "POST", url)
|
||||
|
||||
client := &http.Client{Transport: &http.Transport{DisableKeepAlives: t.DisableKeepAlives}}
|
||||
@ -143,12 +143,12 @@ func (t *HTTPTransporter) SendVoteRequest(server *Server, peer *Peer, req *Reque
|
||||
}
|
||||
|
||||
// Sends a SnapshotRequest RPC to a peer.
|
||||
func (t *HTTPTransporter) SendSnapshotRequest(server *Server, peer *Peer, req *SnapshotRequest) *SnapshotResponse {
|
||||
func (t *HTTPTransporter) SendSnapshotRequest(server Server, peer *Peer, req *SnapshotRequest) *SnapshotResponse {
|
||||
return nil
|
||||
}
|
||||
|
||||
// Sends a SnapshotRequest RPC to a peer.
|
||||
func (t *HTTPTransporter) SendSnapshotRecoveryRequest(server *Server, peer *Peer, req *SnapshotRecoveryRequest) *SnapshotRecoveryResponse {
|
||||
func (t *HTTPTransporter) SendSnapshotRecoveryRequest(server Server, peer *Peer, req *SnapshotRecoveryRequest) *SnapshotRecoveryResponse {
|
||||
return nil
|
||||
}
|
||||
|
||||
@ -157,7 +157,7 @@ func (t *HTTPTransporter) SendSnapshotRecoveryRequest(server *Server, peer *Peer
|
||||
//--------------------------------------
|
||||
|
||||
// Handles incoming AppendEntries requests.
|
||||
func (t *HTTPTransporter) appendEntriesHandler(server *Server) http.HandlerFunc {
|
||||
func (t *HTTPTransporter) appendEntriesHandler(server Server) http.HandlerFunc {
|
||||
return func(w http.ResponseWriter, r *http.Request) {
|
||||
traceln(server.Name(), "RECV /appendEntries")
|
||||
|
||||
@ -176,7 +176,7 @@ func (t *HTTPTransporter) appendEntriesHandler(server *Server) http.HandlerFunc
|
||||
}
|
||||
|
||||
// Handles incoming RequestVote requests.
|
||||
func (t *HTTPTransporter) requestVoteHandler(server *Server) http.HandlerFunc {
|
||||
func (t *HTTPTransporter) requestVoteHandler(server Server) http.HandlerFunc {
|
||||
return func(w http.ResponseWriter, r *http.Request) {
|
||||
traceln(server.Name(), "RECV /requestVote")
|
||||
|
||||
|
@ -14,8 +14,8 @@ func TestHTTPTransporter(t *testing.T) {
|
||||
transporter := NewHTTPTransporter("/raft")
|
||||
transporter.DisableKeepAlives = true
|
||||
|
||||
servers := []*Server{}
|
||||
f0 := func(server *Server, httpServer *http.Server) {
|
||||
servers := []Server{}
|
||||
f0 := func(server Server, httpServer *http.Server) {
|
||||
// Stop the leader and wait for an election.
|
||||
server.Stop()
|
||||
time.Sleep(testElectionTimeout * 2)
|
||||
@ -25,15 +25,15 @@ func TestHTTPTransporter(t *testing.T) {
|
||||
}
|
||||
server.Start()
|
||||
}
|
||||
f1 := func(server *Server, httpServer *http.Server) {
|
||||
f1 := func(server Server, httpServer *http.Server) {
|
||||
}
|
||||
f2 := func(server *Server, httpServer *http.Server) {
|
||||
f2 := func(server Server, httpServer *http.Server) {
|
||||
}
|
||||
runTestHttpServers(t, &servers, transporter, f0, f1, f2)
|
||||
}
|
||||
|
||||
// Starts multiple independent Raft servers wrapped with HTTP servers.
|
||||
func runTestHttpServers(t *testing.T, servers *[]*Server, transporter *HTTPTransporter, callbacks ...func(*Server, *http.Server)) {
|
||||
func runTestHttpServers(t *testing.T, servers *[]Server, transporter *HTTPTransporter, callbacks ...func(Server, *http.Server)) {
|
||||
var wg sync.WaitGroup
|
||||
httpServers := []*http.Server{}
|
||||
listeners := []net.Listener{}
|
||||
@ -68,7 +68,7 @@ func runTestHttpServers(t *testing.T, servers *[]*Server, transporter *HTTPTrans
|
||||
|
||||
// Setup configuration.
|
||||
for _, server := range *servers {
|
||||
if _, err := (*servers)[0].Do(&DefaultJoinCommand{Name: server.Name()}); err != nil {
|
||||
if _, err := (*servers)[0].Do(&DefaultJoinCommand{Name: server.Name(), ConnectionString: fmt.Sprintf("http://%s", server.Name())}); err != nil {
|
||||
t.Fatalf("Server %s unable to join: %v", server.Name(), err)
|
||||
}
|
||||
}
|
||||
@ -94,7 +94,7 @@ func BenchmarkSpeed(b *testing.B) {
|
||||
transporter := NewHTTPTransporter("/raft")
|
||||
transporter.DisableKeepAlives = true
|
||||
|
||||
servers := []*Server{}
|
||||
servers := []Server{}
|
||||
|
||||
for i := 0; i < 3; i++ {
|
||||
port := 9000 + i
|
||||
@ -125,7 +125,7 @@ func BenchmarkSpeed(b *testing.B) {
|
||||
|
||||
// Setup configuration.
|
||||
for _, server := range servers {
|
||||
(servers)[0].Do(&DefaultJoinCommand{Name: server.Name()})
|
||||
(servers)[0].Do(&DefaultJoinCommand{Name: server.Name(), ConnectionString: fmt.Sprintf("http://%s", server.Name())})
|
||||
}
|
||||
|
||||
c := make(chan bool)
|
||||
@ -145,7 +145,7 @@ func BenchmarkSpeed(b *testing.B) {
|
||||
}
|
||||
}
|
||||
|
||||
func send(c chan bool, s *Server) {
|
||||
func send(c chan bool, s Server) {
|
||||
for i := 0; i < 20; i++ {
|
||||
s.Do(&NOPCommand{})
|
||||
}
|
||||
|
@ -3,7 +3,7 @@ package raft
|
||||
// Join command interface
|
||||
type JoinCommand interface {
|
||||
CommandName() string
|
||||
Apply(server *Server) (interface{}, error)
|
||||
Apply(server Server) (interface{}, error)
|
||||
NodeName() string
|
||||
}
|
||||
|
||||
@ -18,7 +18,7 @@ func (c *DefaultJoinCommand) CommandName() string {
|
||||
return "raft:join"
|
||||
}
|
||||
|
||||
func (c *DefaultJoinCommand) Apply(server *Server) (interface{}, error) {
|
||||
func (c *DefaultJoinCommand) Apply(server Server) (interface{}, error) {
|
||||
err := server.AddPeer(c.Name, c.ConnectionString)
|
||||
|
||||
return []byte("join"), err
|
||||
|
@ -3,7 +3,7 @@ package raft
|
||||
// Leave command interface
|
||||
type LeaveCommand interface {
|
||||
CommandName() string
|
||||
Apply(server *Server) (interface{}, error)
|
||||
Apply(server Server) (interface{}, error)
|
||||
NodeName() string
|
||||
}
|
||||
|
||||
@ -17,7 +17,7 @@ func (c *DefaultLeaveCommand) CommandName() string {
|
||||
return "raft:leave"
|
||||
}
|
||||
|
||||
func (c *DefaultLeaveCommand) Apply(server *Server) (interface{}, error) {
|
||||
func (c *DefaultLeaveCommand) Apply(server Server) (interface{}, error) {
|
||||
err := server.RemovePeer(c.Name)
|
||||
|
||||
return []byte("leave"), err
|
||||
|
@ -13,7 +13,7 @@ func (c NOPCommand) CommandName() string {
|
||||
return "raft:nop"
|
||||
}
|
||||
|
||||
func (c NOPCommand) Apply(server *Server) (interface{}, error) {
|
||||
func (c NOPCommand) Apply(server Server) (interface{}, error) {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
|
@ -13,7 +13,7 @@ import (
|
||||
|
||||
// A peer is a reference to another server involved in the consensus protocol.
|
||||
type Peer struct {
|
||||
server *Server
|
||||
server *server
|
||||
Name string `json:"name"`
|
||||
ConnectionString string `json:"connectionString"`
|
||||
prevLogIndex uint64
|
||||
@ -29,7 +29,7 @@ type Peer struct {
|
||||
//------------------------------------------------------------------------------
|
||||
|
||||
// Creates a new peer.
|
||||
func newPeer(server *Server, name string, connectionString string, heartbeatTimeout time.Duration) *Peer {
|
||||
func newPeer(server *server, name string, connectionString string, heartbeatTimeout time.Duration) *Peer {
|
||||
return &Peer{
|
||||
server: server,
|
||||
Name: name,
|
||||
|
161
third_party/github.com/coreos/go-raft/server.go
vendored
161
third_party/github.com/coreos/go-raft/server.go
vendored
@ -57,7 +57,46 @@ var CommandTimeoutError = errors.New("raft: Command timeout")
|
||||
|
||||
// A server is involved in the consensus protocol and can act as a follower,
|
||||
// candidate or a leader.
|
||||
type Server struct {
|
||||
type Server interface {
|
||||
Name() string
|
||||
Context() interface{}
|
||||
StateMachine() StateMachine
|
||||
Leader() string
|
||||
State() string
|
||||
Path() string
|
||||
LogPath() string
|
||||
SnapshotPath(lastIndex uint64, lastTerm uint64) string
|
||||
Term() uint64
|
||||
CommitIndex() uint64
|
||||
VotedFor() string
|
||||
MemberCount() int
|
||||
QuorumSize() int
|
||||
IsLogEmpty() bool
|
||||
LogEntries() []*LogEntry
|
||||
LastCommandName() string
|
||||
GetState() string
|
||||
ElectionTimeout() time.Duration
|
||||
SetElectionTimeout(duration time.Duration)
|
||||
HeartbeatTimeout() time.Duration
|
||||
SetHeartbeatTimeout(duration time.Duration)
|
||||
Transporter() Transporter
|
||||
SetTransporter(t Transporter)
|
||||
AppendEntries(req *AppendEntriesRequest) *AppendEntriesResponse
|
||||
RequestVote(req *RequestVoteRequest) *RequestVoteResponse
|
||||
RequestSnapshot(req *SnapshotRequest) *SnapshotResponse
|
||||
SnapshotRecoveryRequest(req *SnapshotRecoveryRequest) *SnapshotRecoveryResponse
|
||||
AddPeer(name string, connectiongString string) error
|
||||
RemovePeer(name string) error
|
||||
Peers() map[string]*Peer
|
||||
Start() error
|
||||
Stop()
|
||||
Running() bool
|
||||
Do(command Command) (interface{}, error)
|
||||
TakeSnapshot() error
|
||||
LoadSnapshot() error
|
||||
}
|
||||
|
||||
type server struct {
|
||||
name string
|
||||
path string
|
||||
state string
|
||||
@ -98,7 +137,7 @@ type event struct {
|
||||
//------------------------------------------------------------------------------
|
||||
|
||||
// Creates a new server with a log at the given path.
|
||||
func NewServer(name string, path string, transporter Transporter, stateMachine StateMachine, context interface{}, connectionString string) (*Server, error) {
|
||||
func NewServer(name string, path string, transporter Transporter, stateMachine StateMachine, context interface{}, connectionString string) (Server, error) {
|
||||
if name == "" {
|
||||
return nil, errors.New("raft.Server: Name cannot be blank")
|
||||
}
|
||||
@ -106,7 +145,7 @@ func NewServer(name string, path string, transporter Transporter, stateMachine S
|
||||
panic("raft: Transporter required")
|
||||
}
|
||||
|
||||
s := &Server{
|
||||
s := &server{
|
||||
name: name,
|
||||
path: path,
|
||||
transporter: transporter,
|
||||
@ -142,22 +181,22 @@ func NewServer(name string, path string, transporter Transporter, stateMachine S
|
||||
//--------------------------------------
|
||||
|
||||
// Retrieves the name of the server.
|
||||
func (s *Server) Name() string {
|
||||
func (s *server) Name() string {
|
||||
return s.name
|
||||
}
|
||||
|
||||
// Retrieves the storage path for the server.
|
||||
func (s *Server) Path() string {
|
||||
func (s *server) Path() string {
|
||||
return s.path
|
||||
}
|
||||
|
||||
// The name of the current leader.
|
||||
func (s *Server) Leader() string {
|
||||
func (s *server) Leader() string {
|
||||
return s.leader
|
||||
}
|
||||
|
||||
// Retrieves a copy of the peer data.
|
||||
func (s *Server) Peers() map[string]*Peer {
|
||||
func (s *server) Peers() map[string]*Peer {
|
||||
s.mutex.Lock()
|
||||
defer s.mutex.Unlock()
|
||||
|
||||
@ -169,42 +208,42 @@ func (s *Server) Peers() map[string]*Peer {
|
||||
}
|
||||
|
||||
// Retrieves the object that transports requests.
|
||||
func (s *Server) Transporter() Transporter {
|
||||
func (s *server) Transporter() Transporter {
|
||||
s.mutex.RLock()
|
||||
defer s.mutex.RUnlock()
|
||||
return s.transporter
|
||||
}
|
||||
|
||||
func (s *Server) SetTransporter(t Transporter) {
|
||||
func (s *server) SetTransporter(t Transporter) {
|
||||
s.mutex.Lock()
|
||||
defer s.mutex.Unlock()
|
||||
s.transporter = t
|
||||
}
|
||||
|
||||
// Retrieves the context passed into the constructor.
|
||||
func (s *Server) Context() interface{} {
|
||||
func (s *server) Context() interface{} {
|
||||
return s.context
|
||||
}
|
||||
|
||||
// Retrieves the state machine passed into the constructor.
|
||||
func (s *Server) StateMachine() StateMachine {
|
||||
func (s *server) StateMachine() StateMachine {
|
||||
return s.stateMachine
|
||||
}
|
||||
|
||||
// Retrieves the log path for the server.
|
||||
func (s *Server) LogPath() string {
|
||||
func (s *server) LogPath() string {
|
||||
return path.Join(s.path, "log")
|
||||
}
|
||||
|
||||
// Retrieves the current state of the server.
|
||||
func (s *Server) State() string {
|
||||
func (s *server) State() string {
|
||||
s.mutex.RLock()
|
||||
defer s.mutex.RUnlock()
|
||||
return s.state
|
||||
}
|
||||
|
||||
// Sets the state of the server.
|
||||
func (s *Server) setState(state string) {
|
||||
func (s *server) setState(state string) {
|
||||
s.mutex.Lock()
|
||||
defer s.mutex.Unlock()
|
||||
s.state = state
|
||||
@ -214,44 +253,44 @@ func (s *Server) setState(state string) {
|
||||
}
|
||||
|
||||
// Retrieves the current term of the server.
|
||||
func (s *Server) Term() uint64 {
|
||||
func (s *server) Term() uint64 {
|
||||
return s.currentTerm
|
||||
}
|
||||
|
||||
// Retrieves the current commit index of the server.
|
||||
func (s *Server) CommitIndex() uint64 {
|
||||
func (s *server) CommitIndex() uint64 {
|
||||
return s.log.commitIndex
|
||||
}
|
||||
|
||||
// Retrieves the name of the candidate this server voted for in this term.
|
||||
func (s *Server) VotedFor() string {
|
||||
func (s *server) VotedFor() string {
|
||||
return s.votedFor
|
||||
}
|
||||
|
||||
// Retrieves whether the server's log has no entries.
|
||||
func (s *Server) IsLogEmpty() bool {
|
||||
func (s *server) IsLogEmpty() bool {
|
||||
return s.log.isEmpty()
|
||||
}
|
||||
|
||||
// A list of all the log entries. This should only be used for debugging purposes.
|
||||
func (s *Server) LogEntries() []*LogEntry {
|
||||
func (s *server) LogEntries() []*LogEntry {
|
||||
return s.log.entries
|
||||
}
|
||||
|
||||
// A reference to the command name of the last entry.
|
||||
func (s *Server) LastCommandName() string {
|
||||
func (s *server) LastCommandName() string {
|
||||
return s.log.lastCommandName()
|
||||
}
|
||||
|
||||
// Get the state of the server for debugging
|
||||
func (s *Server) GetState() string {
|
||||
func (s *server) GetState() string {
|
||||
s.mutex.RLock()
|
||||
defer s.mutex.RUnlock()
|
||||
return fmt.Sprintf("Name: %s, State: %s, Term: %v, CommitedIndex: %v ", s.name, s.state, s.currentTerm, s.log.commitIndex)
|
||||
}
|
||||
|
||||
// Check if the server is promotable
|
||||
func (s *Server) promotable() bool {
|
||||
func (s *server) promotable() bool {
|
||||
return s.log.currentIndex() > 0
|
||||
}
|
||||
|
||||
@ -260,14 +299,14 @@ func (s *Server) promotable() bool {
|
||||
//--------------------------------------
|
||||
|
||||
// Retrieves the number of member servers in the consensus.
|
||||
func (s *Server) MemberCount() int {
|
||||
func (s *server) MemberCount() int {
|
||||
s.mutex.Lock()
|
||||
defer s.mutex.Unlock()
|
||||
return len(s.peers) + 1
|
||||
}
|
||||
|
||||
// Retrieves the number of servers required to make a quorum.
|
||||
func (s *Server) QuorumSize() int {
|
||||
func (s *server) QuorumSize() int {
|
||||
return (s.MemberCount() / 2) + 1
|
||||
}
|
||||
|
||||
@ -276,14 +315,14 @@ func (s *Server) QuorumSize() int {
|
||||
//--------------------------------------
|
||||
|
||||
// Retrieves the election timeout.
|
||||
func (s *Server) ElectionTimeout() time.Duration {
|
||||
func (s *server) ElectionTimeout() time.Duration {
|
||||
s.mutex.RLock()
|
||||
defer s.mutex.RUnlock()
|
||||
return s.electionTimeout
|
||||
}
|
||||
|
||||
// Sets the election timeout.
|
||||
func (s *Server) SetElectionTimeout(duration time.Duration) {
|
||||
func (s *server) SetElectionTimeout(duration time.Duration) {
|
||||
s.mutex.Lock()
|
||||
defer s.mutex.Unlock()
|
||||
s.electionTimeout = duration
|
||||
@ -294,14 +333,14 @@ func (s *Server) SetElectionTimeout(duration time.Duration) {
|
||||
//--------------------------------------
|
||||
|
||||
// Retrieves the heartbeat timeout.
|
||||
func (s *Server) HeartbeatTimeout() time.Duration {
|
||||
func (s *server) HeartbeatTimeout() time.Duration {
|
||||
s.mutex.RLock()
|
||||
defer s.mutex.RUnlock()
|
||||
return s.heartbeatTimeout
|
||||
}
|
||||
|
||||
// Sets the heartbeat timeout.
|
||||
func (s *Server) SetHeartbeatTimeout(duration time.Duration) {
|
||||
func (s *server) SetHeartbeatTimeout(duration time.Duration) {
|
||||
s.mutex.Lock()
|
||||
defer s.mutex.Unlock()
|
||||
|
||||
@ -334,7 +373,7 @@ func init() {
|
||||
// If no log entries exist and a self-join command is issued then
|
||||
// immediately become leader and commit entry.
|
||||
|
||||
func (s *Server) Start() error {
|
||||
func (s *server) Start() error {
|
||||
// Exit if the server is already running.
|
||||
if s.state != Stopped {
|
||||
return errors.New("raft.Server: Server already running")
|
||||
@ -380,7 +419,7 @@ func (s *Server) Start() error {
|
||||
}
|
||||
|
||||
// Shuts down the server.
|
||||
func (s *Server) Stop() {
|
||||
func (s *server) Stop() {
|
||||
s.send(&stopValue)
|
||||
s.mutex.Lock()
|
||||
defer s.mutex.Unlock()
|
||||
@ -388,7 +427,7 @@ func (s *Server) Stop() {
|
||||
}
|
||||
|
||||
// Checks if the server is currently running.
|
||||
func (s *Server) Running() bool {
|
||||
func (s *server) Running() bool {
|
||||
s.mutex.RLock()
|
||||
defer s.mutex.RUnlock()
|
||||
return s.state != Stopped
|
||||
@ -400,7 +439,7 @@ func (s *Server) Running() bool {
|
||||
|
||||
// Sets the current term for the server. This is only used when an external
|
||||
// current term is found.
|
||||
func (s *Server) setCurrentTerm(term uint64, leaderName string, append bool) {
|
||||
func (s *server) setCurrentTerm(term uint64, leaderName string, append bool) {
|
||||
s.mutex.Lock()
|
||||
defer s.mutex.Unlock()
|
||||
|
||||
@ -439,7 +478,7 @@ func (s *Server) setCurrentTerm(term uint64, leaderName string, append bool) {
|
||||
// | new leader | |
|
||||
// |_______________________|____________________________________ |
|
||||
// The main event loop for the server
|
||||
func (s *Server) loop() {
|
||||
func (s *server) loop() {
|
||||
defer s.debugln("server.loop.end")
|
||||
|
||||
for {
|
||||
@ -467,13 +506,13 @@ func (s *Server) loop() {
|
||||
|
||||
// Sends an event to the event loop to be processed. The function will wait
|
||||
// until the event is actually processed before returning.
|
||||
func (s *Server) send(value interface{}) (interface{}, error) {
|
||||
func (s *server) send(value interface{}) (interface{}, error) {
|
||||
event := s.sendAsync(value)
|
||||
err := <-event.c
|
||||
return event.returnValue, err
|
||||
}
|
||||
|
||||
func (s *Server) sendAsync(value interface{}) *event {
|
||||
func (s *server) sendAsync(value interface{}) *event {
|
||||
event := &event{target: value, c: make(chan error, 1)}
|
||||
s.c <- event
|
||||
return event
|
||||
@ -484,7 +523,7 @@ func (s *Server) sendAsync(value interface{}) *event {
|
||||
// Converts to candidate if election timeout elapses without either:
|
||||
// 1.Receiving valid AppendEntries RPC, or
|
||||
// 2.Granting vote to candidate
|
||||
func (s *Server) followerLoop() {
|
||||
func (s *server) followerLoop() {
|
||||
|
||||
s.setState(Follower)
|
||||
timeoutChan := afterBetween(s.ElectionTimeout(), s.ElectionTimeout()*2)
|
||||
@ -547,7 +586,7 @@ func (s *Server) followerLoop() {
|
||||
}
|
||||
|
||||
// The event loop that is run when the server is in a Candidate state.
|
||||
func (s *Server) candidateLoop() {
|
||||
func (s *server) candidateLoop() {
|
||||
lastLogIndex, lastLogTerm := s.log.lastInfo()
|
||||
s.leader = ""
|
||||
|
||||
@ -630,7 +669,7 @@ func (s *Server) candidateLoop() {
|
||||
}
|
||||
|
||||
// The event loop that is run when the server is in a Leader state.
|
||||
func (s *Server) leaderLoop() {
|
||||
func (s *server) leaderLoop() {
|
||||
s.setState(Leader)
|
||||
s.syncedPeer = make(map[string]bool)
|
||||
logIndex, _ := s.log.lastInfo()
|
||||
@ -682,7 +721,7 @@ func (s *Server) leaderLoop() {
|
||||
s.syncedPeer = nil
|
||||
}
|
||||
|
||||
func (s *Server) snapshotLoop() {
|
||||
func (s *server) snapshotLoop() {
|
||||
s.setState(Snapshotting)
|
||||
|
||||
for {
|
||||
@ -721,12 +760,12 @@ func (s *Server) snapshotLoop() {
|
||||
// Attempts to execute a command and replicate it. The function will return
|
||||
// when the command has been successfully committed or an error has occurred.
|
||||
|
||||
func (s *Server) Do(command Command) (interface{}, error) {
|
||||
func (s *server) Do(command Command) (interface{}, error) {
|
||||
return s.send(command)
|
||||
}
|
||||
|
||||
// Processes a command.
|
||||
func (s *Server) processCommand(command Command, e *event) {
|
||||
func (s *server) processCommand(command Command, e *event) {
|
||||
s.debugln("server.command.process")
|
||||
|
||||
// Create an entry for the command in the log.
|
||||
@ -779,14 +818,14 @@ func (s *Server) processCommand(command Command, e *event) {
|
||||
//--------------------------------------
|
||||
|
||||
// Appends zero or more log entry from the leader to this server.
|
||||
func (s *Server) AppendEntries(req *AppendEntriesRequest) *AppendEntriesResponse {
|
||||
func (s *server) AppendEntries(req *AppendEntriesRequest) *AppendEntriesResponse {
|
||||
ret, _ := s.send(req)
|
||||
resp, _ := ret.(*AppendEntriesResponse)
|
||||
return resp
|
||||
}
|
||||
|
||||
// Processes the "append entries" request.
|
||||
func (s *Server) processAppendEntriesRequest(req *AppendEntriesRequest) (*AppendEntriesResponse, bool) {
|
||||
func (s *server) processAppendEntriesRequest(req *AppendEntriesRequest) (*AppendEntriesResponse, bool) {
|
||||
|
||||
s.traceln("server.ae.process")
|
||||
|
||||
@ -824,7 +863,7 @@ func (s *Server) processAppendEntriesRequest(req *AppendEntriesRequest) (*Append
|
||||
// Processes the "append entries" response from the peer. This is only
|
||||
// processed when the server is a leader. Responses received during other
|
||||
// states are dropped.
|
||||
func (s *Server) processAppendEntriesResponse(resp *AppendEntriesResponse) {
|
||||
func (s *server) processAppendEntriesResponse(resp *AppendEntriesResponse) {
|
||||
|
||||
// If we find a higher term then change to a follower and exit.
|
||||
if resp.Term > s.currentTerm {
|
||||
@ -854,7 +893,7 @@ func (s *Server) processAppendEntriesResponse(resp *AppendEntriesResponse) {
|
||||
for _, peer := range s.peers {
|
||||
indices = append(indices, peer.getPrevLogIndex())
|
||||
}
|
||||
sort.Sort(uint64Slice(indices))
|
||||
sort.Sort(sort.Reverse(uint64Slice(indices)))
|
||||
|
||||
// We can commit up to the index which the majority of the members have appended.
|
||||
commitIndex := indices[s.QuorumSize()-1]
|
||||
@ -888,14 +927,14 @@ func (s *Server) processAppendEntriesResponse(resp *AppendEntriesResponse) {
|
||||
// Requests a vote from a server. A vote can be obtained if the vote's term is
|
||||
// at the server's current term and the server has not made a vote yet. A vote
|
||||
// can also be obtained if the term is greater than the server's current term.
|
||||
func (s *Server) RequestVote(req *RequestVoteRequest) *RequestVoteResponse {
|
||||
func (s *server) RequestVote(req *RequestVoteRequest) *RequestVoteResponse {
|
||||
ret, _ := s.send(req)
|
||||
resp, _ := ret.(*RequestVoteResponse)
|
||||
return resp
|
||||
}
|
||||
|
||||
// Processes a "request vote" request.
|
||||
func (s *Server) processRequestVoteRequest(req *RequestVoteRequest) (*RequestVoteResponse, bool) {
|
||||
func (s *server) processRequestVoteRequest(req *RequestVoteRequest) (*RequestVoteResponse, bool) {
|
||||
|
||||
// If the request is coming from an old term then reject it.
|
||||
if req.Term < s.currentTerm {
|
||||
@ -933,7 +972,7 @@ func (s *Server) processRequestVoteRequest(req *RequestVoteRequest) (*RequestVot
|
||||
//--------------------------------------
|
||||
|
||||
// Adds a peer to the server.
|
||||
func (s *Server) AddPeer(name string, connectiongString string) error {
|
||||
func (s *server) AddPeer(name string, connectiongString string) error {
|
||||
s.debugln("server.peer.add: ", name, len(s.peers))
|
||||
|
||||
// Do not allow peers to be added twice.
|
||||
@ -959,7 +998,7 @@ func (s *Server) AddPeer(name string, connectiongString string) error {
|
||||
}
|
||||
|
||||
// Removes a peer from the server.
|
||||
func (s *Server) RemovePeer(name string) error {
|
||||
func (s *server) RemovePeer(name string) error {
|
||||
s.debugln("server.peer.remove: ", name, len(s.peers))
|
||||
|
||||
// Skip the Peer if it has the same name as the Server
|
||||
@ -988,7 +1027,7 @@ func (s *Server) RemovePeer(name string) error {
|
||||
// Log compaction
|
||||
//--------------------------------------
|
||||
|
||||
func (s *Server) TakeSnapshot() error {
|
||||
func (s *server) TakeSnapshot() error {
|
||||
//TODO put a snapshot mutex
|
||||
s.debugln("take Snapshot")
|
||||
if s.currentSnapshot != nil {
|
||||
@ -1047,7 +1086,7 @@ func (s *Server) TakeSnapshot() error {
|
||||
}
|
||||
|
||||
// Retrieves the log path for the server.
|
||||
func (s *Server) saveSnapshot() error {
|
||||
func (s *server) saveSnapshot() error {
|
||||
|
||||
if s.currentSnapshot == nil {
|
||||
return errors.New("no snapshot to save")
|
||||
@ -1071,17 +1110,17 @@ func (s *Server) saveSnapshot() error {
|
||||
}
|
||||
|
||||
// Retrieves the log path for the server.
|
||||
func (s *Server) SnapshotPath(lastIndex uint64, lastTerm uint64) string {
|
||||
func (s *server) SnapshotPath(lastIndex uint64, lastTerm uint64) string {
|
||||
return path.Join(s.path, "snapshot", fmt.Sprintf("%v_%v.ss", lastTerm, lastIndex))
|
||||
}
|
||||
|
||||
func (s *Server) RequestSnapshot(req *SnapshotRequest) *SnapshotResponse {
|
||||
func (s *server) RequestSnapshot(req *SnapshotRequest) *SnapshotResponse {
|
||||
ret, _ := s.send(req)
|
||||
resp, _ := ret.(*SnapshotResponse)
|
||||
return resp
|
||||
}
|
||||
|
||||
func (s *Server) processSnapshotRequest(req *SnapshotRequest) *SnapshotResponse {
|
||||
func (s *server) processSnapshotRequest(req *SnapshotRequest) *SnapshotResponse {
|
||||
|
||||
// If the follower’s log contains an entry at the snapshot’s last index with a term
|
||||
// that matches the snapshot’s last term
|
||||
@ -1099,13 +1138,13 @@ func (s *Server) processSnapshotRequest(req *SnapshotRequest) *SnapshotResponse
|
||||
return newSnapshotResponse(true)
|
||||
}
|
||||
|
||||
func (s *Server) SnapshotRecoveryRequest(req *SnapshotRecoveryRequest) *SnapshotRecoveryResponse {
|
||||
func (s *server) SnapshotRecoveryRequest(req *SnapshotRecoveryRequest) *SnapshotRecoveryResponse {
|
||||
ret, _ := s.send(req)
|
||||
resp, _ := ret.(*SnapshotRecoveryResponse)
|
||||
return resp
|
||||
}
|
||||
|
||||
func (s *Server) processSnapshotRecoveryRequest(req *SnapshotRecoveryRequest) *SnapshotRecoveryResponse {
|
||||
func (s *server) processSnapshotRecoveryRequest(req *SnapshotRecoveryRequest) *SnapshotRecoveryResponse {
|
||||
|
||||
s.stateMachine.Recovery(req.State)
|
||||
|
||||
@ -1136,7 +1175,7 @@ func (s *Server) processSnapshotRecoveryRequest(req *SnapshotRecoveryRequest) *S
|
||||
}
|
||||
|
||||
// Load a snapshot at restart
|
||||
func (s *Server) LoadSnapshot() error {
|
||||
func (s *server) LoadSnapshot() error {
|
||||
dir, err := os.OpenFile(path.Join(s.path, "snapshot"), os.O_RDONLY, 0)
|
||||
if err != nil {
|
||||
|
||||
@ -1221,7 +1260,7 @@ func (s *Server) LoadSnapshot() error {
|
||||
// Config File
|
||||
//--------------------------------------
|
||||
|
||||
func (s *Server) writeConf() {
|
||||
func (s *server) writeConf() {
|
||||
|
||||
peers := make([]*Peer, len(s.peers))
|
||||
|
||||
@ -1251,7 +1290,7 @@ func (s *Server) writeConf() {
|
||||
}
|
||||
|
||||
// Read the configuration for the server.
|
||||
func (s *Server) readConf() error {
|
||||
func (s *server) readConf() error {
|
||||
confPath := path.Join(s.path, "conf")
|
||||
s.debugln("readConf.open ", confPath)
|
||||
|
||||
@ -1277,10 +1316,10 @@ func (s *Server) readConf() error {
|
||||
// Debugging
|
||||
//--------------------------------------
|
||||
|
||||
func (s *Server) debugln(v ...interface{}) {
|
||||
func (s *server) debugln(v ...interface{}) {
|
||||
debugf("[%s Term:%d] %s", s.name, s.currentTerm, fmt.Sprintln(v...))
|
||||
}
|
||||
|
||||
func (s *Server) traceln(v ...interface{}) {
|
||||
func (s *server) traceln(v ...interface{}) {
|
||||
tracef("[%s] %s", s.name, fmt.Sprintln(v...))
|
||||
}
|
||||
|
289
third_party/github.com/coreos/go-raft/server_test.go
vendored
289
third_party/github.com/coreos/go-raft/server_test.go
vendored
@ -37,40 +37,40 @@ func TestServerRequestVote(t *testing.T) {
|
||||
|
||||
// // Ensure that a vote request is denied if it comes from an old term.
|
||||
func TestServerRequestVoteDeniedForStaleTerm(t *testing.T) {
|
||||
server := newTestServer("1", &testTransporter{})
|
||||
s := newTestServer("1", &testTransporter{})
|
||||
|
||||
server.Start()
|
||||
if _, err := server.Do(&DefaultJoinCommand{Name: server.Name()}); err != nil {
|
||||
t.Fatalf("Server %s unable to join: %v", server.Name(), err)
|
||||
s.Start()
|
||||
if _, err := s.Do(&DefaultJoinCommand{Name: s.Name()}); err != nil {
|
||||
t.Fatalf("Server %s unable to join: %v", s.Name(), err)
|
||||
}
|
||||
|
||||
server.currentTerm = 2
|
||||
defer server.Stop()
|
||||
resp := server.RequestVote(newRequestVoteRequest(1, "foo", 1, 0))
|
||||
s.(*server).currentTerm = 2
|
||||
defer s.Stop()
|
||||
resp := s.RequestVote(newRequestVoteRequest(1, "foo", 1, 0))
|
||||
if resp.Term != 2 || resp.VoteGranted {
|
||||
t.Fatalf("Invalid request vote response: %v/%v", resp.Term, resp.VoteGranted)
|
||||
}
|
||||
if server.currentTerm != 2 && server.State() != Follower {
|
||||
t.Fatalf("Server did not update term and demote: %v / %v", server.currentTerm, server.State())
|
||||
if s.Term() != 2 && s.State() != Follower {
|
||||
t.Fatalf("Server did not update term and demote: %v / %v", s.Term(), s.State())
|
||||
}
|
||||
}
|
||||
|
||||
// Ensure that a vote request is denied if we've already voted for a different candidate.
|
||||
func TestServerRequestVoteDeniedIfAlreadyVoted(t *testing.T) {
|
||||
server := newTestServer("1", &testTransporter{})
|
||||
s := newTestServer("1", &testTransporter{})
|
||||
|
||||
server.Start()
|
||||
if _, err := server.Do(&DefaultJoinCommand{Name: server.Name()}); err != nil {
|
||||
t.Fatalf("Server %s unable to join: %v", server.Name(), err)
|
||||
s.Start()
|
||||
if _, err := s.Do(&DefaultJoinCommand{Name: s.Name()}); err != nil {
|
||||
t.Fatalf("Server %s unable to join: %v", s.Name(), err)
|
||||
}
|
||||
|
||||
server.currentTerm = 2
|
||||
defer server.Stop()
|
||||
resp := server.RequestVote(newRequestVoteRequest(2, "foo", 1, 0))
|
||||
s.(*server).currentTerm = 2
|
||||
defer s.Stop()
|
||||
resp := s.RequestVote(newRequestVoteRequest(2, "foo", 1, 0))
|
||||
if resp.Term != 2 || !resp.VoteGranted {
|
||||
t.Fatalf("First vote should not have been denied")
|
||||
}
|
||||
resp = server.RequestVote(newRequestVoteRequest(2, "bar", 1, 0))
|
||||
resp = s.RequestVote(newRequestVoteRequest(2, "bar", 1, 0))
|
||||
if resp.Term != 2 || resp.VoteGranted {
|
||||
t.Fatalf("Second vote should have been denied")
|
||||
}
|
||||
@ -78,24 +78,24 @@ func TestServerRequestVoteDeniedIfAlreadyVoted(t *testing.T) {
|
||||
|
||||
// Ensure that a vote request is approved if vote occurs in a new term.
|
||||
func TestServerRequestVoteApprovedIfAlreadyVotedInOlderTerm(t *testing.T) {
|
||||
server := newTestServer("1", &testTransporter{})
|
||||
s := newTestServer("1", &testTransporter{})
|
||||
|
||||
server.Start()
|
||||
if _, err := server.Do(&DefaultJoinCommand{Name: server.Name()}); err != nil {
|
||||
t.Fatalf("Server %s unable to join: %v", server.Name(), err)
|
||||
s.Start()
|
||||
if _, err := s.Do(&DefaultJoinCommand{Name: s.Name()}); err != nil {
|
||||
t.Fatalf("Server %s unable to join: %v", s.Name(), err)
|
||||
}
|
||||
|
||||
time.Sleep(time.Millisecond * 100)
|
||||
|
||||
server.currentTerm = 2
|
||||
defer server.Stop()
|
||||
resp := server.RequestVote(newRequestVoteRequest(2, "foo", 2, 1))
|
||||
if resp.Term != 2 || !resp.VoteGranted || server.VotedFor() != "foo" {
|
||||
s.(*server).currentTerm = 2
|
||||
defer s.Stop()
|
||||
resp := s.RequestVote(newRequestVoteRequest(2, "foo", 2, 1))
|
||||
if resp.Term != 2 || !resp.VoteGranted || s.VotedFor() != "foo" {
|
||||
t.Fatalf("First vote should not have been denied")
|
||||
}
|
||||
resp = server.RequestVote(newRequestVoteRequest(3, "bar", 2, 1))
|
||||
resp = s.RequestVote(newRequestVoteRequest(3, "bar", 2, 1))
|
||||
|
||||
if resp.Term != 3 || !resp.VoteGranted || server.VotedFor() != "bar" {
|
||||
if resp.Term != 3 || !resp.VoteGranted || s.VotedFor() != "bar" {
|
||||
t.Fatalf("Second vote should have been approved")
|
||||
}
|
||||
}
|
||||
@ -106,33 +106,32 @@ func TestServerRequestVoteDenyIfCandidateLogIsBehind(t *testing.T) {
|
||||
e0, _ := newLogEntry(tmpLog, 1, 1, &testCommand1{Val: "foo", I: 20})
|
||||
e1, _ := newLogEntry(tmpLog, 2, 1, &testCommand2{X: 100})
|
||||
e2, _ := newLogEntry(tmpLog, 3, 2, &testCommand1{Val: "bar", I: 0})
|
||||
server := newTestServerWithLog("1", &testTransporter{}, []*LogEntry{e0, e1, e2})
|
||||
s := newTestServerWithLog("1", &testTransporter{}, []*LogEntry{e0, e1, e2})
|
||||
|
||||
// start as a follower with term 2 and index 3
|
||||
server.Start()
|
||||
|
||||
defer server.Stop()
|
||||
s.Start()
|
||||
defer s.Stop()
|
||||
|
||||
// request vote from term 3 with last log entry 2, 2
|
||||
resp := server.RequestVote(newRequestVoteRequest(3, "foo", 2, 2))
|
||||
resp := s.RequestVote(newRequestVoteRequest(3, "foo", 2, 2))
|
||||
if resp.Term != 3 || resp.VoteGranted {
|
||||
t.Fatalf("Stale index vote should have been denied [%v/%v]", resp.Term, resp.VoteGranted)
|
||||
}
|
||||
|
||||
// request vote from term 2 with last log entry 2, 3
|
||||
resp = server.RequestVote(newRequestVoteRequest(2, "foo", 3, 2))
|
||||
resp = s.RequestVote(newRequestVoteRequest(2, "foo", 3, 2))
|
||||
if resp.Term != 3 || resp.VoteGranted {
|
||||
t.Fatalf("Stale term vote should have been denied [%v/%v]", resp.Term, resp.VoteGranted)
|
||||
}
|
||||
|
||||
// request vote from term 3 with last log entry 2, 3
|
||||
resp = server.RequestVote(newRequestVoteRequest(3, "foo", 3, 2))
|
||||
resp = s.RequestVote(newRequestVoteRequest(3, "foo", 3, 2))
|
||||
if resp.Term != 3 || !resp.VoteGranted {
|
||||
t.Fatalf("Matching log vote should have been granted")
|
||||
}
|
||||
|
||||
// request vote from term 3 with last log entry 2, 4
|
||||
resp = server.RequestVote(newRequestVoteRequest(3, "foo", 4, 2))
|
||||
resp = s.RequestVote(newRequestVoteRequest(3, "foo", 4, 2))
|
||||
if resp.Term != 3 || !resp.VoteGranted {
|
||||
t.Fatalf("Ahead-of-log vote should have been granted")
|
||||
}
|
||||
@ -145,28 +144,27 @@ func TestServerRequestVoteDenyIfCandidateLogIsBehind(t *testing.T) {
|
||||
// // Ensure that we can self-promote a server to candidate, obtain votes and become a fearless leader.
|
||||
func TestServerPromoteSelf(t *testing.T) {
|
||||
e0, _ := newLogEntry(newLog(), 1, 1, &testCommand1{Val: "foo", I: 20})
|
||||
server := newTestServerWithLog("1", &testTransporter{}, []*LogEntry{e0})
|
||||
s := newTestServerWithLog("1", &testTransporter{}, []*LogEntry{e0})
|
||||
|
||||
// start as a follower
|
||||
server.Start()
|
||||
|
||||
defer server.Stop()
|
||||
s.Start()
|
||||
defer s.Stop()
|
||||
|
||||
time.Sleep(2 * testElectionTimeout)
|
||||
|
||||
if server.State() != Leader {
|
||||
t.Fatalf("Server self-promotion failed: %v", server.State())
|
||||
if s.State() != Leader {
|
||||
t.Fatalf("Server self-promotion failed: %v", s.State())
|
||||
}
|
||||
}
|
||||
|
||||
//Ensure that we can promote a server within a cluster to a leader.
|
||||
func TestServerPromote(t *testing.T) {
|
||||
lookup := map[string]*Server{}
|
||||
lookup := map[string]Server{}
|
||||
transporter := &testTransporter{}
|
||||
transporter.sendVoteRequestFunc = func(server *Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse {
|
||||
transporter.sendVoteRequestFunc = func(s Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse {
|
||||
return lookup[peer.Name].RequestVote(req)
|
||||
}
|
||||
transporter.sendAppendEntriesRequestFunc = func(server *Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse {
|
||||
transporter.sendAppendEntriesRequestFunc = func(s Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse {
|
||||
return lookup[peer.Name].AppendEntries(req)
|
||||
}
|
||||
servers := newTestCluster([]string{"1", "2", "3"}, transporter, lookup)
|
||||
@ -180,8 +178,8 @@ func TestServerPromote(t *testing.T) {
|
||||
if servers[0].State() != Leader && servers[1].State() != Leader && servers[2].State() != Leader {
|
||||
t.Fatalf("No leader elected: (%s, %s, %s)", servers[0].State(), servers[1].State(), servers[2].State())
|
||||
}
|
||||
for _, server := range servers {
|
||||
server.Stop()
|
||||
for _, s := range servers {
|
||||
s.Stop()
|
||||
}
|
||||
}
|
||||
|
||||
@ -191,20 +189,20 @@ func TestServerPromote(t *testing.T) {
|
||||
|
||||
// Ensure we can append entries to a server.
|
||||
func TestServerAppendEntries(t *testing.T) {
|
||||
server := newTestServer("1", &testTransporter{})
|
||||
s := newTestServer("1", &testTransporter{})
|
||||
|
||||
server.SetHeartbeatTimeout(time.Second * 10)
|
||||
server.Start()
|
||||
defer server.Stop()
|
||||
s.SetHeartbeatTimeout(time.Second * 10)
|
||||
s.Start()
|
||||
defer s.Stop()
|
||||
|
||||
// Append single entry.
|
||||
e, _ := newLogEntry(nil, 1, 1, &testCommand1{Val: "foo", I: 10})
|
||||
entries := []*LogEntry{e}
|
||||
resp := server.AppendEntries(newAppendEntriesRequest(1, 0, 0, 0, "ldr", entries))
|
||||
resp := s.AppendEntries(newAppendEntriesRequest(1, 0, 0, 0, "ldr", entries))
|
||||
if resp.Term != 1 || !resp.Success {
|
||||
t.Fatalf("AppendEntries failed: %v/%v", resp.Term, resp.Success)
|
||||
}
|
||||
if index, term := server.log.commitInfo(); index != 0 || term != 0 {
|
||||
if index, term := s.(*server).log.commitInfo(); index != 0 || term != 0 {
|
||||
t.Fatalf("Invalid commit info [IDX=%v, TERM=%v]", index, term)
|
||||
}
|
||||
|
||||
@ -212,57 +210,56 @@ func TestServerAppendEntries(t *testing.T) {
|
||||
e1, _ := newLogEntry(nil, 2, 1, &testCommand1{Val: "bar", I: 20})
|
||||
e2, _ := newLogEntry(nil, 3, 1, &testCommand1{Val: "baz", I: 30})
|
||||
entries = []*LogEntry{e1, e2}
|
||||
resp = server.AppendEntries(newAppendEntriesRequest(1, 1, 1, 1, "ldr", entries))
|
||||
resp = s.AppendEntries(newAppendEntriesRequest(1, 1, 1, 1, "ldr", entries))
|
||||
if resp.Term != 1 || !resp.Success {
|
||||
t.Fatalf("AppendEntries failed: %v/%v", resp.Term, resp.Success)
|
||||
}
|
||||
if index, term := server.log.commitInfo(); index != 1 || term != 1 {
|
||||
if index, term := s.(*server).log.commitInfo(); index != 1 || term != 1 {
|
||||
t.Fatalf("Invalid commit info [IDX=%v, TERM=%v]", index, term)
|
||||
}
|
||||
|
||||
// Send zero entries and commit everything.
|
||||
resp = server.AppendEntries(newAppendEntriesRequest(2, 3, 1, 3, "ldr", []*LogEntry{}))
|
||||
resp = s.AppendEntries(newAppendEntriesRequest(2, 3, 1, 3, "ldr", []*LogEntry{}))
|
||||
if resp.Term != 2 || !resp.Success {
|
||||
t.Fatalf("AppendEntries failed: %v/%v", resp.Term, resp.Success)
|
||||
}
|
||||
if index, term := server.log.commitInfo(); index != 3 || term != 1 {
|
||||
if index, term := s.(*server).log.commitInfo(); index != 3 || term != 1 {
|
||||
t.Fatalf("Invalid commit info [IDX=%v, TERM=%v]", index, term)
|
||||
}
|
||||
}
|
||||
|
||||
//Ensure that entries with stale terms are rejected.
|
||||
func TestServerAppendEntriesWithStaleTermsAreRejected(t *testing.T) {
|
||||
server := newTestServer("1", &testTransporter{})
|
||||
s := newTestServer("1", &testTransporter{})
|
||||
|
||||
server.Start()
|
||||
s.Start()
|
||||
|
||||
defer server.Stop()
|
||||
server.currentTerm = 2
|
||||
defer s.Stop()
|
||||
s.(*server).currentTerm = 2
|
||||
|
||||
// Append single entry.
|
||||
e, _ := newLogEntry(nil, 1, 1, &testCommand1{Val: "foo", I: 10})
|
||||
entries := []*LogEntry{e}
|
||||
resp := server.AppendEntries(newAppendEntriesRequest(1, 0, 0, 0, "ldr", entries))
|
||||
resp := s.AppendEntries(newAppendEntriesRequest(1, 0, 0, 0, "ldr", entries))
|
||||
if resp.Term != 2 || resp.Success {
|
||||
t.Fatalf("AppendEntries should have failed: %v/%v", resp.Term, resp.Success)
|
||||
}
|
||||
if index, term := server.log.commitInfo(); index != 0 || term != 0 {
|
||||
if index, term := s.(*server).log.commitInfo(); index != 0 || term != 0 {
|
||||
t.Fatalf("Invalid commit info [IDX=%v, TERM=%v]", index, term)
|
||||
}
|
||||
}
|
||||
|
||||
// Ensure that we reject entries if the commit log is different.
|
||||
func TestServerAppendEntriesRejectedIfAlreadyCommitted(t *testing.T) {
|
||||
server := newTestServer("1", &testTransporter{})
|
||||
server.Start()
|
||||
|
||||
defer server.Stop()
|
||||
s := newTestServer("1", &testTransporter{})
|
||||
s.Start()
|
||||
defer s.Stop()
|
||||
|
||||
// Append single entry + commit.
|
||||
e1, _ := newLogEntry(nil, 1, 1, &testCommand1{Val: "foo", I: 10})
|
||||
e2, _ := newLogEntry(nil, 2, 1, &testCommand1{Val: "foo", I: 15})
|
||||
entries := []*LogEntry{e1, e2}
|
||||
resp := server.AppendEntries(newAppendEntriesRequest(1, 0, 0, 2, "ldr", entries))
|
||||
resp := s.AppendEntries(newAppendEntriesRequest(1, 0, 0, 2, "ldr", entries))
|
||||
if resp.Term != 1 || !resp.Success {
|
||||
t.Fatalf("AppendEntries failed: %v/%v", resp.Term, resp.Success)
|
||||
}
|
||||
@ -270,7 +267,7 @@ func TestServerAppendEntriesRejectedIfAlreadyCommitted(t *testing.T) {
|
||||
// Append entry again (post-commit).
|
||||
e, _ := newLogEntry(nil, 2, 1, &testCommand1{Val: "bar", I: 20})
|
||||
entries = []*LogEntry{e}
|
||||
resp = server.AppendEntries(newAppendEntriesRequest(1, 2, 1, 1, "ldr", entries))
|
||||
resp = s.AppendEntries(newAppendEntriesRequest(1, 2, 1, 1, "ldr", entries))
|
||||
if resp.Term != 1 || resp.Success {
|
||||
t.Fatalf("AppendEntries should have failed: %v/%v", resp.Term, resp.Success)
|
||||
}
|
||||
@ -278,9 +275,9 @@ func TestServerAppendEntriesRejectedIfAlreadyCommitted(t *testing.T) {
|
||||
|
||||
// Ensure that we uncommitted entries are rolled back if new entries overwrite them.
|
||||
func TestServerAppendEntriesOverwritesUncommittedEntries(t *testing.T) {
|
||||
server := newTestServer("1", &testTransporter{})
|
||||
server.Start()
|
||||
defer server.Stop()
|
||||
s := newTestServer("1", &testTransporter{})
|
||||
s.Start()
|
||||
defer s.Stop()
|
||||
|
||||
entry1, _ := newLogEntry(nil, 1, 1, &testCommand1{Val: "foo", I: 10})
|
||||
entry2, _ := newLogEntry(nil, 2, 1, &testCommand1{Val: "foo", I: 15})
|
||||
@ -288,15 +285,15 @@ func TestServerAppendEntriesOverwritesUncommittedEntries(t *testing.T) {
|
||||
|
||||
// Append single entry + commit.
|
||||
entries := []*LogEntry{entry1, entry2}
|
||||
resp := server.AppendEntries(newAppendEntriesRequest(1, 0, 0, 1, "ldr", entries))
|
||||
if resp.Term != 1 || !resp.Success || server.log.commitIndex != 1 || !reflect.DeepEqual(server.log.entries, []*LogEntry{entry1, entry2}) {
|
||||
resp := s.AppendEntries(newAppendEntriesRequest(1, 0, 0, 1, "ldr", entries))
|
||||
if resp.Term != 1 || !resp.Success || s.(*server).log.commitIndex != 1 || !reflect.DeepEqual(s.(*server).log.entries, []*LogEntry{entry1, entry2}) {
|
||||
t.Fatalf("AppendEntries failed: %v/%v", resp.Term, resp.Success)
|
||||
}
|
||||
|
||||
// Append entry that overwrites the second (uncommitted) entry.
|
||||
entries = []*LogEntry{entry3}
|
||||
resp = server.AppendEntries(newAppendEntriesRequest(2, 1, 1, 2, "ldr", entries))
|
||||
if resp.Term != 2 || !resp.Success || server.log.commitIndex != 2 || !reflect.DeepEqual(server.log.entries, []*LogEntry{entry1, entry3}) {
|
||||
resp = s.AppendEntries(newAppendEntriesRequest(2, 1, 1, 2, "ldr", entries))
|
||||
if resp.Term != 2 || !resp.Success || s.(*server).log.commitIndex != 2 || !reflect.DeepEqual(s.(*server).log.entries, []*LogEntry{entry1, entry3}) {
|
||||
t.Fatalf("AppendEntries should have succeeded: %v/%v", resp.Term, resp.Success)
|
||||
}
|
||||
}
|
||||
@ -307,11 +304,11 @@ func TestServerAppendEntriesOverwritesUncommittedEntries(t *testing.T) {
|
||||
|
||||
// Ensure that a follower cannot execute a command.
|
||||
func TestServerDenyCommandExecutionWhenFollower(t *testing.T) {
|
||||
server := newTestServer("1", &testTransporter{})
|
||||
server.Start()
|
||||
defer server.Stop()
|
||||
s := newTestServer("1", &testTransporter{})
|
||||
s.Start()
|
||||
defer s.Stop()
|
||||
var err error
|
||||
if _, err = server.Do(&testCommand1{Val: "foo", I: 10}); err != NotLeaderError {
|
||||
if _, err = s.Do(&testCommand1{Val: "foo", I: 10}); err != NotLeaderError {
|
||||
t.Fatalf("Expected error: %v, got: %v", NotLeaderError, err)
|
||||
}
|
||||
}
|
||||
@ -324,27 +321,27 @@ func TestServerDenyCommandExecutionWhenFollower(t *testing.T) {
|
||||
func TestServerRecoverFromPreviousLogAndConf(t *testing.T) {
|
||||
// Initialize the servers.
|
||||
var mutex sync.RWMutex
|
||||
servers := map[string]*Server{}
|
||||
servers := map[string]Server{}
|
||||
|
||||
transporter := &testTransporter{}
|
||||
transporter.sendVoteRequestFunc = func(server *Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse {
|
||||
transporter.sendVoteRequestFunc = func(s Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse {
|
||||
mutex.RLock()
|
||||
s := servers[peer.Name]
|
||||
target := servers[peer.Name]
|
||||
mutex.RUnlock()
|
||||
return s.RequestVote(req)
|
||||
return target.RequestVote(req)
|
||||
}
|
||||
transporter.sendAppendEntriesRequestFunc = func(server *Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse {
|
||||
transporter.sendAppendEntriesRequestFunc = func(s Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse {
|
||||
mutex.RLock()
|
||||
s := servers[peer.Name]
|
||||
target := servers[peer.Name]
|
||||
mutex.RUnlock()
|
||||
return s.AppendEntries(req)
|
||||
return target.AppendEntries(req)
|
||||
}
|
||||
|
||||
disTransporter := &testTransporter{}
|
||||
disTransporter.sendVoteRequestFunc = func(server *Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse {
|
||||
disTransporter.sendVoteRequestFunc = func(s Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse {
|
||||
return nil
|
||||
}
|
||||
disTransporter.sendAppendEntriesRequestFunc = func(server *Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse {
|
||||
disTransporter.sendAppendEntriesRequestFunc = func(s Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse {
|
||||
return nil
|
||||
}
|
||||
|
||||
@ -358,22 +355,22 @@ func TestServerRecoverFromPreviousLogAndConf(t *testing.T) {
|
||||
names = append(names, strconv.Itoa(i))
|
||||
}
|
||||
|
||||
var leader *Server
|
||||
var leader Server
|
||||
for _, name := range names {
|
||||
server := newTestServer(name, transporter)
|
||||
s := newTestServer(name, transporter)
|
||||
|
||||
servers[name] = server
|
||||
paths[name] = server.Path()
|
||||
servers[name] = s
|
||||
paths[name] = s.Path()
|
||||
|
||||
if name == "1" {
|
||||
leader = server
|
||||
server.SetHeartbeatTimeout(testHeartbeatTimeout)
|
||||
server.Start()
|
||||
leader = s
|
||||
s.SetHeartbeatTimeout(testHeartbeatTimeout)
|
||||
s.Start()
|
||||
time.Sleep(testHeartbeatTimeout)
|
||||
} else {
|
||||
server.SetElectionTimeout(testElectionTimeout)
|
||||
server.SetHeartbeatTimeout(testHeartbeatTimeout)
|
||||
server.Start()
|
||||
s.SetElectionTimeout(testElectionTimeout)
|
||||
s.SetHeartbeatTimeout(testHeartbeatTimeout)
|
||||
s.Start()
|
||||
time.Sleep(testHeartbeatTimeout)
|
||||
}
|
||||
if _, err := leader.Do(&DefaultJoinCommand{Name: name}); err != nil {
|
||||
@ -385,35 +382,35 @@ func TestServerRecoverFromPreviousLogAndConf(t *testing.T) {
|
||||
// commit some commands
|
||||
for i := 0; i < 10; i++ {
|
||||
if _, err := leader.Do(&testCommand2{X: 1}); err != nil {
|
||||
t.Fatalf("cannot commit command:", err.Error())
|
||||
t.Fatalf("cannot commit command: %s", err.Error())
|
||||
}
|
||||
}
|
||||
|
||||
time.Sleep(2 * testHeartbeatTimeout)
|
||||
|
||||
for _, name := range names {
|
||||
server := servers[name]
|
||||
if server.CommitIndex() != 16 {
|
||||
t.Fatalf("%s commitIndex is invalid [%d/%d]", name, server.CommitIndex(), 16)
|
||||
s := servers[name]
|
||||
if s.CommitIndex() != 16 {
|
||||
t.Fatalf("%s commitIndex is invalid [%d/%d]", name, s.CommitIndex(), 16)
|
||||
}
|
||||
server.Stop()
|
||||
s.Stop()
|
||||
}
|
||||
|
||||
for _, name := range names {
|
||||
// with old path and disable transportation
|
||||
server := newTestServerWithPath(name, disTransporter, paths[name])
|
||||
servers[name] = server
|
||||
s := newTestServerWithPath(name, disTransporter, paths[name])
|
||||
servers[name] = s
|
||||
|
||||
server.Start()
|
||||
s.Start()
|
||||
|
||||
// should only commit to the last join command
|
||||
if server.CommitIndex() != 6 {
|
||||
t.Fatalf("%s recover phase 1 commitIndex is invalid [%d/%d]", name, server.CommitIndex(), 6)
|
||||
if s.CommitIndex() != 6 {
|
||||
t.Fatalf("%s recover phase 1 commitIndex is invalid [%d/%d]", name, s.CommitIndex(), 6)
|
||||
}
|
||||
|
||||
// peer conf should be recovered
|
||||
if len(server.Peers()) != 4 {
|
||||
t.Fatalf("%s recover phase 1 peer failed! [%d/%d]", name, len(server.Peers()), 4)
|
||||
if len(s.Peers()) != 4 {
|
||||
t.Fatalf("%s recover phase 1 peer failed! [%d/%d]", name, len(s.Peers()), 4)
|
||||
}
|
||||
}
|
||||
|
||||
@ -426,11 +423,11 @@ func TestServerRecoverFromPreviousLogAndConf(t *testing.T) {
|
||||
|
||||
// should commit to the previous index + 1(nop command when new leader elected)
|
||||
for _, name := range names {
|
||||
server := servers[name]
|
||||
if server.CommitIndex() != 17 {
|
||||
t.Fatalf("%s commitIndex is invalid [%d/%d]", name, server.CommitIndex(), 17)
|
||||
s := servers[name]
|
||||
if s.CommitIndex() != 17 {
|
||||
t.Fatalf("%s commitIndex is invalid [%d/%d]", name, s.CommitIndex(), 17)
|
||||
}
|
||||
server.Stop()
|
||||
s.Stop()
|
||||
}
|
||||
}
|
||||
|
||||
@ -440,29 +437,29 @@ func TestServerRecoverFromPreviousLogAndConf(t *testing.T) {
|
||||
|
||||
// Ensure that we can start a single server and append to its log.
|
||||
func TestServerSingleNode(t *testing.T) {
|
||||
server := newTestServer("1", &testTransporter{})
|
||||
if server.State() != Stopped {
|
||||
t.Fatalf("Unexpected server state: %v", server.State())
|
||||
s := newTestServer("1", &testTransporter{})
|
||||
if s.State() != Stopped {
|
||||
t.Fatalf("Unexpected server state: %v", s.State())
|
||||
}
|
||||
|
||||
server.Start()
|
||||
s.Start()
|
||||
|
||||
time.Sleep(testHeartbeatTimeout)
|
||||
|
||||
// Join the server to itself.
|
||||
if _, err := server.Do(&DefaultJoinCommand{Name: "1"}); err != nil {
|
||||
if _, err := s.Do(&DefaultJoinCommand{Name: "1"}); err != nil {
|
||||
t.Fatalf("Unable to join: %v", err)
|
||||
}
|
||||
debugln("finish command")
|
||||
|
||||
if server.State() != Leader {
|
||||
t.Fatalf("Unexpected server state: %v", server.State())
|
||||
if s.State() != Leader {
|
||||
t.Fatalf("Unexpected server state: %v", s.State())
|
||||
}
|
||||
|
||||
server.Stop()
|
||||
s.Stop()
|
||||
|
||||
if server.State() != Stopped {
|
||||
t.Fatalf("Unexpected server state: %v", server.State())
|
||||
if s.State() != Stopped {
|
||||
t.Fatalf("Unexpected server state: %v", s.State())
|
||||
}
|
||||
}
|
||||
|
||||
@ -470,27 +467,27 @@ func TestServerSingleNode(t *testing.T) {
|
||||
func TestServerMultiNode(t *testing.T) {
|
||||
// Initialize the servers.
|
||||
var mutex sync.RWMutex
|
||||
servers := map[string]*Server{}
|
||||
servers := map[string]Server{}
|
||||
|
||||
transporter := &testTransporter{}
|
||||
transporter.sendVoteRequestFunc = func(server *Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse {
|
||||
transporter.sendVoteRequestFunc = func(s Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse {
|
||||
mutex.RLock()
|
||||
s := servers[peer.Name]
|
||||
target := servers[peer.Name]
|
||||
mutex.RUnlock()
|
||||
return s.RequestVote(req)
|
||||
return target.RequestVote(req)
|
||||
}
|
||||
transporter.sendAppendEntriesRequestFunc = func(server *Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse {
|
||||
transporter.sendAppendEntriesRequestFunc = func(s Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse {
|
||||
mutex.RLock()
|
||||
s := servers[peer.Name]
|
||||
target := servers[peer.Name]
|
||||
mutex.RUnlock()
|
||||
return s.AppendEntries(req)
|
||||
return target.AppendEntries(req)
|
||||
}
|
||||
|
||||
disTransporter := &testTransporter{}
|
||||
disTransporter.sendVoteRequestFunc = func(server *Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse {
|
||||
disTransporter.sendVoteRequestFunc = func(s Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse {
|
||||
return nil
|
||||
}
|
||||
disTransporter.sendAppendEntriesRequestFunc = func(server *Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse {
|
||||
disTransporter.sendAppendEntriesRequestFunc = func(s Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse {
|
||||
return nil
|
||||
}
|
||||
|
||||
@ -503,24 +500,24 @@ func TestServerMultiNode(t *testing.T) {
|
||||
names = append(names, strconv.Itoa(i))
|
||||
}
|
||||
|
||||
var leader *Server
|
||||
var leader Server
|
||||
for _, name := range names {
|
||||
server := newTestServer(name, transporter)
|
||||
defer server.Stop()
|
||||
s := newTestServer(name, transporter)
|
||||
defer s.Stop()
|
||||
|
||||
mutex.Lock()
|
||||
servers[name] = server
|
||||
servers[name] = s
|
||||
mutex.Unlock()
|
||||
|
||||
if name == "1" {
|
||||
leader = server
|
||||
server.SetHeartbeatTimeout(testHeartbeatTimeout)
|
||||
server.Start()
|
||||
leader = s
|
||||
s.SetHeartbeatTimeout(testHeartbeatTimeout)
|
||||
s.Start()
|
||||
time.Sleep(testHeartbeatTimeout)
|
||||
} else {
|
||||
server.SetElectionTimeout(testElectionTimeout)
|
||||
server.SetHeartbeatTimeout(testHeartbeatTimeout)
|
||||
server.Start()
|
||||
s.SetElectionTimeout(testElectionTimeout)
|
||||
s.SetHeartbeatTimeout(testHeartbeatTimeout)
|
||||
s.Start()
|
||||
time.Sleep(testHeartbeatTimeout)
|
||||
}
|
||||
if _, err := leader.Do(&DefaultJoinCommand{Name: name}); err != nil {
|
||||
@ -536,7 +533,7 @@ func TestServerMultiNode(t *testing.T) {
|
||||
t.Fatalf("Expected member count to be %v, got %v", n, leader.MemberCount())
|
||||
}
|
||||
if servers["2"].State() == Leader || servers["3"].State() == Leader {
|
||||
t.Fatalf("Expected leader should be 1: 2=%v, 3=%v\n", servers["2"].state, servers["3"].state)
|
||||
t.Fatalf("Expected leader should be 1: 2=%v, 3=%v\n", servers["2"].State(), servers["3"].State())
|
||||
}
|
||||
mutex.RUnlock()
|
||||
|
||||
@ -573,7 +570,7 @@ func TestServerMultiNode(t *testing.T) {
|
||||
}
|
||||
debugln("[Test] Done")
|
||||
}
|
||||
debugln("Leader is ", value.Name(), " Index ", value.log.commitIndex)
|
||||
debugln("Leader is ", value.Name(), " Index ", value.(*server).log.commitIndex)
|
||||
}
|
||||
debugln("Not Found leader")
|
||||
}
|
||||
@ -584,7 +581,7 @@ func TestServerMultiNode(t *testing.T) {
|
||||
if value.State() == Leader {
|
||||
leader++
|
||||
}
|
||||
debugln(value.Name(), " ", value.currentTerm, " ", value.state)
|
||||
debugln(value.Name(), " ", value.(*server).Term(), " ", value.State())
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -20,9 +20,9 @@ type Snapshot struct {
|
||||
LastIndex uint64 `json:"lastIndex"`
|
||||
LastTerm uint64 `json:"lastTerm"`
|
||||
// cluster configuration.
|
||||
Peers []*Peer `json: "peers"`
|
||||
State []byte `json: "state"`
|
||||
Path string `json: "path"`
|
||||
Peers []*Peer `json:"peers"`
|
||||
State []byte `json:"state"`
|
||||
Path string `json:"path"`
|
||||
}
|
||||
|
||||
// Save the snapshot to a file
|
||||
|
@ -80,7 +80,7 @@ func (req *SnapshotRecoveryRequest) Decode(r io.Reader) (int, error) {
|
||||
req.LeaderName = pb.GetLeaderName()
|
||||
req.LastIndex = pb.GetLastIndex()
|
||||
req.LastTerm = pb.GetLastTerm()
|
||||
req.State = req.State
|
||||
req.State = pb.GetState()
|
||||
|
||||
req.Peers = make([]*Peer, len(pb.Peers))
|
||||
|
||||
|
28
third_party/github.com/coreos/go-raft/test.go
vendored
28
third_party/github.com/coreos/go-raft/test.go
vendored
@ -60,7 +60,7 @@ func setupLog(entries []*LogEntry) (*Log, string) {
|
||||
// Servers
|
||||
//--------------------------------------
|
||||
|
||||
func newTestServer(name string, transporter Transporter) *Server {
|
||||
func newTestServer(name string, transporter Transporter) Server {
|
||||
p, _ := ioutil.TempDir("", "raft-server-")
|
||||
if err := os.MkdirAll(p, 0644); err != nil {
|
||||
panic(err.Error())
|
||||
@ -69,12 +69,12 @@ func newTestServer(name string, transporter Transporter) *Server {
|
||||
return server
|
||||
}
|
||||
|
||||
func newTestServerWithPath(name string, transporter Transporter, p string) *Server {
|
||||
func newTestServerWithPath(name string, transporter Transporter, p string) Server {
|
||||
server, _ := NewServer(name, p, transporter, nil, nil, "")
|
||||
return server
|
||||
}
|
||||
|
||||
func newTestServerWithLog(name string, transporter Transporter, entries []*LogEntry) *Server {
|
||||
func newTestServerWithLog(name string, transporter Transporter, entries []*LogEntry) Server {
|
||||
server := newTestServer(name, transporter)
|
||||
f, err := os.Create(server.LogPath())
|
||||
if err != nil {
|
||||
@ -88,8 +88,8 @@ func newTestServerWithLog(name string, transporter Transporter, entries []*LogEn
|
||||
return server
|
||||
}
|
||||
|
||||
func newTestCluster(names []string, transporter Transporter, lookup map[string]*Server) []*Server {
|
||||
servers := []*Server{}
|
||||
func newTestCluster(names []string, transporter Transporter, lookup map[string]Server) []Server {
|
||||
servers := []Server{}
|
||||
e0, _ := newLogEntry(newLog(), 1, 1, &testCommand1{Val: "foo", I: 20})
|
||||
|
||||
for _, name := range names {
|
||||
@ -116,24 +116,24 @@ func newTestCluster(names []string, transporter Transporter, lookup map[string]*
|
||||
//--------------------------------------
|
||||
|
||||
type testTransporter struct {
|
||||
sendVoteRequestFunc func(server *Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse
|
||||
sendAppendEntriesRequestFunc func(server *Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse
|
||||
sendSnapshotRequestFunc func(server *Server, peer *Peer, req *SnapshotRequest) *SnapshotResponse
|
||||
sendVoteRequestFunc func(server Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse
|
||||
sendAppendEntriesRequestFunc func(server Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse
|
||||
sendSnapshotRequestFunc func(server Server, peer *Peer, req *SnapshotRequest) *SnapshotResponse
|
||||
}
|
||||
|
||||
func (t *testTransporter) SendVoteRequest(server *Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse {
|
||||
func (t *testTransporter) SendVoteRequest(server Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse {
|
||||
return t.sendVoteRequestFunc(server, peer, req)
|
||||
}
|
||||
|
||||
func (t *testTransporter) SendAppendEntriesRequest(server *Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse {
|
||||
func (t *testTransporter) SendAppendEntriesRequest(server Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse {
|
||||
return t.sendAppendEntriesRequestFunc(server, peer, req)
|
||||
}
|
||||
|
||||
func (t *testTransporter) SendSnapshotRequest(server *Server, peer *Peer, req *SnapshotRequest) *SnapshotResponse {
|
||||
func (t *testTransporter) SendSnapshotRequest(server Server, peer *Peer, req *SnapshotRequest) *SnapshotResponse {
|
||||
return t.sendSnapshotRequestFunc(server, peer, req)
|
||||
}
|
||||
|
||||
func (t *testTransporter) SendSnapshotRecoveryRequest(server *Server, peer *Peer, req *SnapshotRecoveryRequest) *SnapshotRecoveryResponse {
|
||||
func (t *testTransporter) SendSnapshotRecoveryRequest(server Server, peer *Peer, req *SnapshotRecoveryRequest) *SnapshotRecoveryResponse {
|
||||
return t.SendSnapshotRecoveryRequest(server, peer, req)
|
||||
}
|
||||
|
||||
@ -163,7 +163,7 @@ func (c *testCommand1) CommandName() string {
|
||||
return "cmd_1"
|
||||
}
|
||||
|
||||
func (c *testCommand1) Apply(server *Server) (interface{}, error) {
|
||||
func (c *testCommand1) Apply(server Server) (interface{}, error) {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
@ -179,6 +179,6 @@ func (c *testCommand2) CommandName() string {
|
||||
return "cmd_2"
|
||||
}
|
||||
|
||||
func (c *testCommand2) Apply(server *Server) (interface{}, error) {
|
||||
func (c *testCommand2) Apply(server Server) (interface{}, error) {
|
||||
return nil, nil
|
||||
}
|
||||
|
@ -9,8 +9,8 @@ package raft
|
||||
// Transporter is the interface for allowing the host application to transport
|
||||
// requests to other nodes.
|
||||
type Transporter interface {
|
||||
SendVoteRequest(server *Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse
|
||||
SendAppendEntriesRequest(server *Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse
|
||||
SendSnapshotRequest(server *Server, peer *Peer, req *SnapshotRequest) *SnapshotResponse
|
||||
SendSnapshotRecoveryRequest(server *Server, peer *Peer, req *SnapshotRecoveryRequest) *SnapshotRecoveryResponse
|
||||
SendVoteRequest(server Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse
|
||||
SendAppendEntriesRequest(server Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse
|
||||
SendSnapshotRequest(server Server, peer *Peer, req *SnapshotRequest) *SnapshotResponse
|
||||
SendSnapshotRecoveryRequest(server Server, peer *Peer, req *SnapshotRecoveryRequest) *SnapshotRecoveryResponse
|
||||
}
|
||||
|
@ -1,3 +1,4 @@
|
||||
// Package activation implements primitives for systemd socket activation.
|
||||
package activation
|
||||
|
||||
import (
|
||||
@ -23,7 +24,7 @@ func Files() []*os.File {
|
||||
files := []*os.File(nil)
|
||||
for fd := listenFdsStart; fd < listenFdsStart+nfds; fd++ {
|
||||
syscall.CloseOnExec(fd)
|
||||
files = append(files, os.NewFile(uintptr(fd), "LISTEN_FD_"+strconv.Itoa(fd)))
|
||||
files = append(files, os.NewFile(uintptr(fd), "LISTEN_FD_" + strconv.Itoa(fd)))
|
||||
}
|
||||
return files
|
||||
}
|
||||
|
100
third_party/github.com/coreos/go-systemd/dbus/dbus.go
vendored
Normal file
100
third_party/github.com/coreos/go-systemd/dbus/dbus.go
vendored
Normal file
@ -0,0 +1,100 @@
|
||||
// Integration with the systemd D-Bus API. See http://www.freedesktop.org/wiki/Software/systemd/dbus/
|
||||
package dbus
|
||||
|
||||
import (
|
||||
"github.com/guelfey/go.dbus"
|
||||
"sync"
|
||||
)
|
||||
|
||||
const signalBuffer = 100
|
||||
|
||||
type Conn struct {
|
||||
sysconn *dbus.Conn
|
||||
sysobj *dbus.Object
|
||||
jobListener struct {
|
||||
jobs map[dbus.ObjectPath]chan string
|
||||
sync.Mutex
|
||||
}
|
||||
subscriber struct {
|
||||
updateCh chan<- *SubStateUpdate
|
||||
errCh chan<- error
|
||||
sync.Mutex
|
||||
ignore map[dbus.ObjectPath]int64
|
||||
cleanIgnore int64
|
||||
}
|
||||
dispatch map[string]func(dbus.Signal)
|
||||
}
|
||||
|
||||
func New() *Conn {
|
||||
c := new(Conn)
|
||||
c.initConnection()
|
||||
c.initJobs()
|
||||
c.initSubscription()
|
||||
c.initDispatch()
|
||||
return c
|
||||
}
|
||||
|
||||
func (c *Conn) initConnection() {
|
||||
var err error
|
||||
c.sysconn, err = dbus.SystemBusPrivate()
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
|
||||
err = c.sysconn.Auth(nil)
|
||||
if err != nil {
|
||||
c.sysconn.Close()
|
||||
return
|
||||
}
|
||||
|
||||
err = c.sysconn.Hello()
|
||||
if err != nil {
|
||||
c.sysconn.Close()
|
||||
return
|
||||
}
|
||||
|
||||
c.sysobj = c.sysconn.Object("org.freedesktop.systemd1", dbus.ObjectPath("/org/freedesktop/systemd1"))
|
||||
|
||||
c.sysconn.BusObject().Call("org.freedesktop.DBus.AddMatch", 0,
|
||||
"type='signal',interface='org.freedesktop.systemd1.Manager',member='JobRemoved'")
|
||||
c.sysconn.BusObject().Call("org.freedesktop.DBus.AddMatch", 0,
|
||||
"type='signal',interface='org.freedesktop.systemd1.Manager',member='UnitNew'")
|
||||
c.sysconn.BusObject().Call("org.freedesktop.DBus.AddMatch", 0,
|
||||
"type='signal',interface='org.freedesktop.DBus.Properties',member='PropertiesChanged'")
|
||||
|
||||
err = c.sysobj.Call("org.freedesktop.systemd1.Manager.Subscribe", 0).Store()
|
||||
if err != nil {
|
||||
c.sysconn.Close()
|
||||
return
|
||||
}
|
||||
}
|
||||
|
||||
func (c *Conn) initDispatch() {
|
||||
ch := make(chan *dbus.Signal, signalBuffer)
|
||||
|
||||
c.sysconn.Signal(ch)
|
||||
|
||||
go func() {
|
||||
for {
|
||||
signal := <-ch
|
||||
switch signal.Name {
|
||||
case "org.freedesktop.systemd1.Manager.JobRemoved":
|
||||
c.jobComplete(signal)
|
||||
|
||||
unitName := signal.Body[2].(string)
|
||||
var unitPath dbus.ObjectPath
|
||||
c.sysobj.Call("GetUnit", 0, unitName).Store(&unitPath)
|
||||
if unitPath != dbus.ObjectPath("") {
|
||||
c.sendSubStateUpdate(unitPath)
|
||||
}
|
||||
case "org.freedesktop.systemd1.Manager.UnitNew":
|
||||
c.sendSubStateUpdate(signal.Body[1].(dbus.ObjectPath))
|
||||
case "org.freedesktop.DBus.Properties.PropertiesChanged":
|
||||
if signal.Body[0].(string) == "org.freedesktop.systemd1.Unit" {
|
||||
// we only care about SubState updates, which are a Unit property
|
||||
c.sendSubStateUpdate(signal.Path)
|
||||
}
|
||||
}
|
||||
}
|
||||
}()
|
||||
}
|
166
third_party/github.com/coreos/go-systemd/dbus/methods.go
vendored
Normal file
166
third_party/github.com/coreos/go-systemd/dbus/methods.go
vendored
Normal file
@ -0,0 +1,166 @@
|
||||
package dbus
|
||||
|
||||
import (
|
||||
"github.com/guelfey/go.dbus"
|
||||
)
|
||||
|
||||
func (c *Conn) initJobs() {
|
||||
c.jobListener.jobs = make(map[dbus.ObjectPath]chan string)
|
||||
}
|
||||
|
||||
func (c *Conn) jobComplete(signal *dbus.Signal) {
|
||||
var id uint32
|
||||
var job dbus.ObjectPath
|
||||
var unit string
|
||||
var result string
|
||||
dbus.Store(signal.Body, &id, &job, &unit, &result)
|
||||
c.jobListener.Lock()
|
||||
out, ok := c.jobListener.jobs[job]
|
||||
if ok {
|
||||
out <- result
|
||||
}
|
||||
c.jobListener.Unlock()
|
||||
}
|
||||
|
||||
func (c *Conn) startJob(job string, args ...interface{}) (<-chan string, error) {
|
||||
c.jobListener.Lock()
|
||||
defer c.jobListener.Unlock()
|
||||
|
||||
ch := make(chan string, 1)
|
||||
var path dbus.ObjectPath
|
||||
err := c.sysobj.Call(job, 0, args...).Store(&path)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
c.jobListener.jobs[path] = ch
|
||||
return ch, nil
|
||||
}
|
||||
|
||||
func (c *Conn) runJob(job string, args ...interface{}) (string, error) {
|
||||
respCh, err := c.startJob(job, args...)
|
||||
if err != nil {
|
||||
return "", err
|
||||
}
|
||||
return <-respCh, nil
|
||||
}
|
||||
|
||||
// StartUnit enqeues a start job and depending jobs, if any (unless otherwise
|
||||
// specified by the mode string).
|
||||
//
|
||||
// Takes the unit to activate, plus a mode string. The mode needs to be one of
|
||||
// replace, fail, isolate, ignore-dependencies, ignore-requirements. If
|
||||
// "replace" the call will start the unit and its dependencies, possibly
|
||||
// replacing already queued jobs that conflict with this. If "fail" the call
|
||||
// will start the unit and its dependencies, but will fail if this would change
|
||||
// an already queued job. If "isolate" the call will start the unit in question
|
||||
// and terminate all units that aren't dependencies of it. If
|
||||
// "ignore-dependencies" it will start a unit but ignore all its dependencies.
|
||||
// If "ignore-requirements" it will start a unit but only ignore the
|
||||
// requirement dependencies. It is not recommended to make use of the latter
|
||||
// two options.
|
||||
//
|
||||
// Result string: one of done, canceled, timeout, failed, dependency, skipped.
|
||||
// done indicates successful execution of a job. canceled indicates that a job
|
||||
// has been canceled before it finished execution. timeout indicates that the
|
||||
// job timeout was reached. failed indicates that the job failed. dependency
|
||||
// indicates that a job this job has been depending on failed and the job hence
|
||||
// has been removed too. skipped indicates that a job was skipped because it
|
||||
// didn't apply to the units current state.
|
||||
func (c *Conn) StartUnit(name string, mode string) (string, error) {
|
||||
return c.runJob("StartUnit", name, mode)
|
||||
}
|
||||
|
||||
// StopUnit is similar to StartUnit but stops the specified unit rather
|
||||
// than starting it.
|
||||
func (c *Conn) StopUnit(name string, mode string) (string, error) {
|
||||
return c.runJob("StopUnit", name, mode)
|
||||
}
|
||||
|
||||
// ReloadUnit reloads a unit. Reloading is done only if the unit is already running and fails otherwise.
|
||||
func (c *Conn) ReloadUnit(name string, mode string) (string, error) {
|
||||
return c.runJob("ReloadUnit", name, mode)
|
||||
}
|
||||
|
||||
// RestartUnit restarts a service. If a service is restarted that isn't
|
||||
// running it will be started.
|
||||
func (c *Conn) RestartUnit(name string, mode string) (string, error) {
|
||||
return c.runJob("RestartUnit", name, mode)
|
||||
}
|
||||
|
||||
// TryRestartUnit is like RestartUnit, except that a service that isn't running
|
||||
// is not affected by the restart.
|
||||
func (c *Conn) TryRestartUnit(name string, mode string) (string, error) {
|
||||
return c.runJob("TryRestartUnit", name, mode)
|
||||
}
|
||||
|
||||
// ReloadOrRestart attempts a reload if the unit supports it and use a restart
|
||||
// otherwise.
|
||||
func (c *Conn) ReloadOrRestartUnit(name string, mode string) (string, error) {
|
||||
return c.runJob("ReloadOrRestartUnit", name, mode)
|
||||
}
|
||||
|
||||
// ReloadOrTryRestart attempts a reload if the unit supports it and use a "Try"
|
||||
// flavored restart otherwise.
|
||||
func (c *Conn) ReloadOrTryRestartUnit(name string, mode string) (string, error) {
|
||||
return c.runJob("ReloadOrTryRestartUnit", name, mode)
|
||||
}
|
||||
|
||||
// StartTransientUnit() may be used to create and start a transient unit, which
|
||||
// will be released as soon as it is not running or referenced anymore or the
|
||||
// system is rebooted. name is the unit name including suffix, and must be
|
||||
// unique. mode is the same as in StartUnit(), properties contains properties
|
||||
// of the unit.
|
||||
func (c *Conn) StartTransientUnit(name string, mode string, properties ...Property) (string, error) {
|
||||
// the dbus interface for this method does not use the last argument and
|
||||
// should simply be given an empty list. We use a concrete type here
|
||||
// (instead of the more appropriate interface{}) to satisfy the dbus library.
|
||||
return c.runJob("StartTransientUnit", name, mode, properties, make([]string, 0))
|
||||
}
|
||||
|
||||
// KillUnit takes the unit name and a UNIX signal number to send. All of the unit's
|
||||
// processes are killed.
|
||||
func (c *Conn) KillUnit(name string, signal int32) {
|
||||
c.sysobj.Call("KillUnit", 0, name, "all", signal).Store()
|
||||
}
|
||||
|
||||
// ListUnits returns an array with all currently loaded units. Note that
|
||||
// units may be known by multiple names at the same time, and hence there might
|
||||
// be more unit names loaded than actual units behind them.
|
||||
func (c *Conn) ListUnits() ([]UnitStatus, error) {
|
||||
result := make([][]interface{}, 0)
|
||||
err := c.sysobj.Call("ListUnits", 0).Store(&result)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
resultInterface := make([]interface{}, len(result))
|
||||
for i := range result {
|
||||
resultInterface[i] = result[i]
|
||||
}
|
||||
|
||||
status := make([]UnitStatus, len(result))
|
||||
statusInterface := make([]interface{}, len(status))
|
||||
for i := range status {
|
||||
statusInterface[i] = &status[i]
|
||||
}
|
||||
|
||||
err = dbus.Store(resultInterface, statusInterface...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return status, nil
|
||||
}
|
||||
|
||||
type UnitStatus struct {
|
||||
Name string // The primary unit name as string
|
||||
Description string // The human readable description string
|
||||
LoadState string // The load state (i.e. whether the unit file has been loaded successfully)
|
||||
ActiveState string // The active state (i.e. whether the unit is currently started or not)
|
||||
SubState string // The sub state (a more fine-grained version of the active state that is specific to the unit type, which the active state is not)
|
||||
Followed string // A unit that is being followed in its state by this unit, if there is any, otherwise the empty string.
|
||||
Path dbus.ObjectPath // The unit object path
|
||||
JobId uint32 // If there is a job queued for the job unit the numeric job id, 0 otherwise
|
||||
JobType string // The job type as string
|
||||
JobPath dbus.ObjectPath // The job object path
|
||||
}
|
193
third_party/github.com/coreos/go-systemd/dbus/properties.go
vendored
Normal file
193
third_party/github.com/coreos/go-systemd/dbus/properties.go
vendored
Normal file
@ -0,0 +1,193 @@
|
||||
package dbus
|
||||
|
||||
import (
|
||||
"github.com/guelfey/go.dbus"
|
||||
)
|
||||
|
||||
// From the systemd docs:
|
||||
//
|
||||
// The properties array of StartTransientUnit() may take many of the settings
|
||||
// that may also be configured in unit files. Not all parameters are currently
|
||||
// accepted though, but we plan to cover more properties with future release.
|
||||
// Currently you may set the Description, Slice and all dependency types of
|
||||
// units, as well as RemainAfterExit, ExecStart for service units,
|
||||
// TimeoutStopUSec and PIDs for scope units, and CPUAccounting, CPUShares,
|
||||
// BlockIOAccounting, BlockIOWeight, BlockIOReadBandwidth,
|
||||
// BlockIOWriteBandwidth, BlockIODeviceWeight, MemoryAccounting, MemoryLimit,
|
||||
// DevicePolicy, DeviceAllow for services/scopes/slices. These fields map
|
||||
// directly to their counterparts in unit files and as normal D-Bus object
|
||||
// properties. The exception here is the PIDs field of scope units which is
|
||||
// used for construction of the scope only and specifies the initial PIDs to
|
||||
// add to the scope object.
|
||||
|
||||
type Property property
|
||||
|
||||
type property struct {
|
||||
Name string
|
||||
Value dbus.Variant
|
||||
}
|
||||
|
||||
type execStart struct {
|
||||
Path string // the binary path to execute
|
||||
Args []string // an array with all arguments to pass to the executed command, starting with argument 0
|
||||
UncleanIsFailure bool // a boolean whether it should be considered a failure if the process exits uncleanly
|
||||
}
|
||||
|
||||
// PropExecStart sets the ExecStart service property. The first argument is a
|
||||
// slice with the binary path to execute followed by the arguments to pass to
|
||||
// the executed command. See
|
||||
// http://www.freedesktop.org/software/systemd/man/systemd.service.html#ExecStart=
|
||||
func PropExecStart(command []string, uncleanIsFailure bool) Property {
|
||||
return Property(
|
||||
property{
|
||||
Name: "ExecStart",
|
||||
Value: dbus.MakeVariant(
|
||||
[]execStart{
|
||||
execStart{
|
||||
Path: command[0],
|
||||
Args: command,
|
||||
UncleanIsFailure: uncleanIsFailure,
|
||||
}})})
|
||||
}
|
||||
|
||||
// PropRemainAfterExit sets the RemainAfterExit service property. See
|
||||
// http://www.freedesktop.org/software/systemd/man/systemd.service.html#RemainAfterExit=
|
||||
func PropRemainAfterExit(b bool) Property {
|
||||
return Property(
|
||||
property{
|
||||
Name: "RemainAfterExit",
|
||||
Value: dbus.MakeVariant(b),
|
||||
})
|
||||
}
|
||||
|
||||
// PropDescription sets the Description unit property. See
|
||||
// http://www.freedesktop.org/software/systemd/man/systemd.unit#Description=
|
||||
func PropDescription(desc string) Property {
|
||||
return Property(
|
||||
property{
|
||||
Name: "Description",
|
||||
Value: dbus.MakeVariant(desc),
|
||||
})
|
||||
}
|
||||
|
||||
func propDependency(name string, units []string) Property {
|
||||
return Property(
|
||||
property{
|
||||
Name: name,
|
||||
Value: dbus.MakeVariant(units),
|
||||
})
|
||||
}
|
||||
|
||||
// PropRequires sets the Requires unit property. See
|
||||
// http://www.freedesktop.org/software/systemd/main/systemd.unit.html#Requires=
|
||||
func PropRequires(units ...string) Property {
|
||||
return propDependency("Requires", units)
|
||||
}
|
||||
|
||||
// PropRequiresOverridable sets the RequiresOverridable unit property. See
|
||||
// http://www.freedesktop.org/software/systemd/main/systemd.unit.html#RequiresOverridable=
|
||||
func PropRequiresOverridable(units ...string) Property {
|
||||
return propDependency("RequiresOverridable", units)
|
||||
}
|
||||
|
||||
// PropRequisite sets the Requisite unit property. See
|
||||
// http://www.freedesktop.org/software/systemd/main/systemd.unit.html#Requisite=
|
||||
func PropRequisite(units ...string) Property {
|
||||
return propDependency("Requisite", units)
|
||||
}
|
||||
|
||||
// PropRequisiteOverridable sets the RequisiteOverridable unit property. See
|
||||
// http://www.freedesktop.org/software/systemd/main/systemd.unit.html#RequisiteOverridable=
|
||||
func PropRequisiteOverridable(units ...string) Property {
|
||||
return propDependency("RequisiteOverridable", units)
|
||||
}
|
||||
|
||||
// PropWants sets the Wants unit property. See
|
||||
// http://www.freedesktop.org/software/systemd/main/systemd.unit.html#Wants=
|
||||
func PropWants(units ...string) Property {
|
||||
return propDependency("Wants", units)
|
||||
}
|
||||
|
||||
// PropBindsTo sets the BindsTo unit property. See
|
||||
// http://www.freedesktop.org/software/systemd/main/systemd.unit.html#BindsTo=
|
||||
func PropBindsTo(units ...string) Property {
|
||||
return propDependency("BindsTo", units)
|
||||
}
|
||||
|
||||
// PropRequiredBy sets the RequiredBy unit property. See
|
||||
// http://www.freedesktop.org/software/systemd/main/systemd.unit.html#RequiredBy=
|
||||
func PropRequiredBy(units ...string) Property {
|
||||
return propDependency("RequiredBy", units)
|
||||
}
|
||||
|
||||
// PropRequiredByOverridable sets the RequiredByOverridable unit property. See
|
||||
// http://www.freedesktop.org/software/systemd/main/systemd.unit.html#RequiredByOverridable=
|
||||
func PropRequiredByOverridable(units ...string) Property {
|
||||
return propDependency("RequiredByOverridable", units)
|
||||
}
|
||||
|
||||
// PropWantedBy sets the WantedBy unit property. See
|
||||
// http://www.freedesktop.org/software/systemd/main/systemd.unit.html#WantedBy=
|
||||
func PropWantedBy(units ...string) Property {
|
||||
return propDependency("WantedBy", units)
|
||||
}
|
||||
|
||||
// PropBoundBy sets the BoundBy unit property. See
|
||||
// http://www.freedesktop.org/software/systemd/main/systemd.unit.html#BoundBy=
|
||||
func PropBoundBy(units ...string) Property {
|
||||
return propDependency("BoundBy", units)
|
||||
}
|
||||
|
||||
// PropConflicts sets the Conflicts unit property. See
|
||||
// http://www.freedesktop.org/software/systemd/main/systemd.unit.html#Conflicts=
|
||||
func PropConflicts(units ...string) Property {
|
||||
return propDependency("Conflicts", units)
|
||||
}
|
||||
|
||||
// PropConflictedBy sets the ConflictedBy unit property. See
|
||||
// http://www.freedesktop.org/software/systemd/main/systemd.unit.html#ConflictedBy=
|
||||
func PropConflictedBy(units ...string) Property {
|
||||
return propDependency("ConflictedBy", units)
|
||||
}
|
||||
|
||||
// PropBefore sets the Before unit property. See
|
||||
// http://www.freedesktop.org/software/systemd/main/systemd.unit.html#Before=
|
||||
func PropBefore(units ...string) Property {
|
||||
return propDependency("Before", units)
|
||||
}
|
||||
|
||||
// PropAfter sets the After unit property. See
|
||||
// http://www.freedesktop.org/software/systemd/main/systemd.unit.html#After=
|
||||
func PropAfter(units ...string) Property {
|
||||
return propDependency("After", units)
|
||||
}
|
||||
|
||||
// PropOnFailure sets the OnFailure unit property. See
|
||||
// http://www.freedesktop.org/software/systemd/main/systemd.unit.html#OnFailure=
|
||||
func PropOnFailure(units ...string) Property {
|
||||
return propDependency("OnFailure", units)
|
||||
}
|
||||
|
||||
// PropTriggers sets the Triggers unit property. See
|
||||
// http://www.freedesktop.org/software/systemd/main/systemd.unit.html#Triggers=
|
||||
func PropTriggers(units ...string) Property {
|
||||
return propDependency("Triggers", units)
|
||||
}
|
||||
|
||||
// PropTriggeredBy sets the TriggeredBy unit property. See
|
||||
// http://www.freedesktop.org/software/systemd/main/systemd.unit.html#TriggeredBy=
|
||||
func PropTriggeredBy(units ...string) Property {
|
||||
return propDependency("TriggeredBy", units)
|
||||
}
|
||||
|
||||
// PropPropagatesReloadTo sets the PropagatesReloadTo unit property. See
|
||||
// http://www.freedesktop.org/software/systemd/main/systemd.unit.html#PropagatesReloadTo=
|
||||
func PropPropagatesReloadTo(units ...string) Property {
|
||||
return propDependency("PropagatesReloadTo", units)
|
||||
}
|
||||
|
||||
// PropRequiresMountsFor sets the RequiresMountsFor unit property. See
|
||||
// http://www.freedesktop.org/software/systemd/main/systemd.unit.html#RequiresMountsFor=
|
||||
func PropRequiresMountsFor(units ...string) Property {
|
||||
return propDependency("RequiresMountsFor", units)
|
||||
}
|
182
third_party/github.com/coreos/go-systemd/dbus/subscription.go
vendored
Normal file
182
third_party/github.com/coreos/go-systemd/dbus/subscription.go
vendored
Normal file
@ -0,0 +1,182 @@
|
||||
package dbus
|
||||
|
||||
import (
|
||||
"github.com/guelfey/go.dbus"
|
||||
"time"
|
||||
)
|
||||
|
||||
const (
|
||||
cleanIgnoreInterval = int64(10 * time.Second)
|
||||
ignoreInterval = int64(30 * time.Millisecond)
|
||||
)
|
||||
|
||||
func (c *Conn) initSubscription() {
|
||||
c.subscriber.ignore = make(map[dbus.ObjectPath]int64)
|
||||
}
|
||||
|
||||
// Returns two unbuffered channels which will receive all changed units every
|
||||
// @interval@ seconds. Deleted units are sent as nil.
|
||||
func (c *Conn) SubscribeUnits(interval time.Duration) (<-chan map[string]*UnitStatus, <-chan error) {
|
||||
return c.SubscribeUnitsCustom(interval, 0, func(u1, u2 *UnitStatus) bool { return *u1 != *u2 })
|
||||
}
|
||||
|
||||
// SubscribeUnitsCustom is like SubscribeUnits but lets you specify the buffer
|
||||
// size of the channels and the comparison function for detecting changes.
|
||||
func (c *Conn) SubscribeUnitsCustom(interval time.Duration, buffer int, isChanged func(*UnitStatus, *UnitStatus) bool) (<-chan map[string]*UnitStatus, <-chan error) {
|
||||
old := make(map[string]*UnitStatus)
|
||||
statusChan := make(chan map[string]*UnitStatus, buffer)
|
||||
errChan := make(chan error, buffer)
|
||||
|
||||
go func() {
|
||||
for {
|
||||
timerChan := time.After(interval)
|
||||
|
||||
units, err := c.ListUnits()
|
||||
if err == nil {
|
||||
cur := make(map[string]*UnitStatus)
|
||||
for i := range units {
|
||||
cur[units[i].Name] = &units[i]
|
||||
}
|
||||
|
||||
// add all new or changed units
|
||||
changed := make(map[string]*UnitStatus)
|
||||
for n, u := range cur {
|
||||
if oldU, ok := old[n]; !ok || isChanged(oldU, u) {
|
||||
changed[n] = u
|
||||
}
|
||||
delete(old, n)
|
||||
}
|
||||
|
||||
// add all deleted units
|
||||
for oldN := range old {
|
||||
changed[oldN] = nil
|
||||
}
|
||||
|
||||
old = cur
|
||||
|
||||
statusChan <- changed
|
||||
} else {
|
||||
errChan <- err
|
||||
}
|
||||
|
||||
<-timerChan
|
||||
}
|
||||
}()
|
||||
|
||||
return statusChan, errChan
|
||||
}
|
||||
|
||||
type SubStateUpdate struct {
|
||||
UnitName string
|
||||
SubState string
|
||||
}
|
||||
|
||||
type Error string
|
||||
|
||||
func (e Error) Error() string {
|
||||
return string(e)
|
||||
}
|
||||
|
||||
// SetSubStateSubscriber writes to updateCh when any unit's substate changes.
|
||||
// Althrough this writes to updateCh on every state change, the reported state
|
||||
// may be more recent than the change that generated it (due to an unavoidable
|
||||
// race in the systemd dbus interface). That is, this method provides a good
|
||||
// way to keep a current view of all units' states, but is not guaranteed to
|
||||
// show every state transition they go through. Furthermore, state changes
|
||||
// will only be written to the channel with non-blocking writes. If updateCh
|
||||
// is full, it attempts to write an error to errCh; if errCh is full, the error
|
||||
// passes silently.
|
||||
func (c *Conn) SetSubStateSubscriber(updateCh chan<- *SubStateUpdate, errCh chan<- error) {
|
||||
c.subscriber.Lock()
|
||||
defer c.subscriber.Unlock()
|
||||
c.subscriber.updateCh = updateCh
|
||||
c.subscriber.errCh = errCh
|
||||
}
|
||||
|
||||
func (c *Conn) sendSubStateUpdate(path dbus.ObjectPath) {
|
||||
c.subscriber.Lock()
|
||||
defer c.subscriber.Unlock()
|
||||
if c.subscriber.updateCh == nil {
|
||||
return
|
||||
}
|
||||
|
||||
if c.shouldIgnore(path) {
|
||||
return
|
||||
}
|
||||
|
||||
info, err := c.getUnitInfo(path)
|
||||
if err != nil {
|
||||
select {
|
||||
case c.subscriber.errCh <- err:
|
||||
default:
|
||||
}
|
||||
}
|
||||
|
||||
name := info["Id"].Value().(string)
|
||||
substate := info["SubState"].Value().(string)
|
||||
|
||||
update := &SubStateUpdate{name, substate}
|
||||
select {
|
||||
case c.subscriber.updateCh <- update:
|
||||
default:
|
||||
select {
|
||||
case c.subscriber.errCh <- Error("update channel full!"):
|
||||
default:
|
||||
}
|
||||
}
|
||||
|
||||
c.updateIgnore(path, info)
|
||||
}
|
||||
|
||||
func (c *Conn) getUnitInfo(path dbus.ObjectPath) (map[string]dbus.Variant, error) {
|
||||
var err error
|
||||
var props map[string]dbus.Variant
|
||||
obj := c.sysconn.Object("org.freedesktop.systemd1", path)
|
||||
err = obj.Call("GetAll", 0, "org.freedesktop.systemd1.Unit").Store(&props)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return props, nil
|
||||
}
|
||||
|
||||
// The ignore functions work around a wart in the systemd dbus interface.
|
||||
// Requesting the properties of an unloaded unit will cause systemd to send a
|
||||
// pair of UnitNew/UnitRemoved signals. Because we need to get a unit's
|
||||
// properties on UnitNew (as that's the only indication of a new unit coming up
|
||||
// for the first time), we would enter an infinite loop if we did not attempt
|
||||
// to detect and ignore these spurious signals. The signal themselves are
|
||||
// indistinguishable from relevant ones, so we (somewhat hackishly) ignore an
|
||||
// unloaded unit's signals for a short time after requesting its properties.
|
||||
// This means that we will miss e.g. a transient unit being restarted
|
||||
// *immediately* upon failure and also a transient unit being started
|
||||
// immediately after requesting its status (with systemctl status, for example,
|
||||
// because this causes a UnitNew signal to be sent which then causes us to fetch
|
||||
// the properties).
|
||||
|
||||
func (c *Conn) shouldIgnore(path dbus.ObjectPath) bool {
|
||||
t, ok := c.subscriber.ignore[path]
|
||||
return ok && t >= time.Now().UnixNano()
|
||||
}
|
||||
|
||||
func (c *Conn) updateIgnore(path dbus.ObjectPath, info map[string]dbus.Variant) {
|
||||
c.cleanIgnore()
|
||||
|
||||
// unit is unloaded - it will trigger bad systemd dbus behavior
|
||||
if info["LoadState"].Value().(string) == "not-found" {
|
||||
c.subscriber.ignore[path] = time.Now().UnixNano() + ignoreInterval
|
||||
}
|
||||
}
|
||||
|
||||
// without this, ignore would grow unboundedly over time
|
||||
func (c *Conn) cleanIgnore() {
|
||||
now := time.Now().UnixNano()
|
||||
if c.subscriber.cleanIgnore < now {
|
||||
c.subscriber.cleanIgnore = now + cleanIgnoreInterval
|
||||
|
||||
for p, t := range c.subscriber.ignore {
|
||||
if t < now {
|
||||
delete(c.subscriber.ignore, p)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
@ -32,7 +32,11 @@ const (
|
||||
var conn net.Conn
|
||||
|
||||
func init() {
|
||||
conn, _ = net.Dial("unixgram", "/run/systemd/journal/socket")
|
||||
var err error
|
||||
conn, err = net.Dial("unixgram", "/run/systemd/journal/socket")
|
||||
if err != nil {
|
||||
conn = nil
|
||||
}
|
||||
}
|
||||
|
||||
// Enabled returns true iff the systemd journal is available for logging
|
||||
|
@ -23,7 +23,7 @@ func mainHandler(c http.ResponseWriter, req *http.Request) {
|
||||
mainTempl.Execute(c, p)
|
||||
}
|
||||
|
||||
func Start(raftServer *raft.Server, webURL string) {
|
||||
func Start(raftServer raft.Server, webURL string) {
|
||||
u, _ := url.Parse(webURL)
|
||||
|
||||
webMux := http.NewServeMux()
|
||||
|
Loading…
x
Reference in New Issue
Block a user