Browse Source

Merge branch '0.2' of https://github.com/coreos/etcd into 0.2

Xiang Li 12 years ago
parent
commit
2b6c628342
79 changed files with 1443 additions and 746 deletions
  1. 6 6
      etcd_test.go
  2. 2 2
      server/join_command.go
  3. 21 21
      server/peer_server.go
  4. 5 5
      server/peer_server_handlers.go
  5. 2 2
      server/registry.go
  6. 1 1
      server/remove_command.go
  7. 12 12
      server/server.go
  8. 4 4
      server/transporter.go
  9. 1 1
      server/v1/v1.go
  10. 1 1
      server/v2/v2.go
  11. 2 2
      store/create_command.go
  12. 2 2
      store/delete_command.go
  13. 22 16
      store/node.go
  14. 2 2
      store/stats_test.go
  15. 43 25
      store/store.go
  16. 11 11
      store/store_test.go
  17. 2 2
      store/test_and_set_command.go
  18. 2 2
      store/update_command.go
  19. 1 1
      store/watcher_test.go
  20. 1 1
      test/test.go
  21. 29 11
      third_party/bitbucket.org/kardianos/osext/osext_sysctl.go
  22. 20 26
      third_party/code.google.com/p/go.net/ipv4/gen.go
  23. 23 24
      third_party/code.google.com/p/go.net/ipv4/gentest.go
  24. 34 25
      third_party/code.google.com/p/go.net/ipv4/header.go
  25. 5 4
      third_party/code.google.com/p/go.net/ipv4/header_test.go
  26. 2 2
      third_party/code.google.com/p/go.net/ipv4/iana_test.go
  27. 20 26
      third_party/code.google.com/p/go.net/ipv6/gen.go
  28. 23 24
      third_party/code.google.com/p/go.net/ipv6/gentest.go
  29. 2 2
      third_party/code.google.com/p/go.net/ipv6/iana.go
  30. 2 2
      third_party/code.google.com/p/go.net/ipv6/iana_test.go
  31. 2 16
      third_party/code.google.com/p/go.net/websocket/client.go
  32. 5 21
      third_party/code.google.com/p/go.net/websocket/hybi.go
  33. 0 108
      third_party/code.google.com/p/go.net/websocket/hybi_test.go
  34. 0 8
      third_party/code.google.com/p/go.net/websocket/server.go
  35. 1 5
      third_party/code.google.com/p/go.net/websocket/websocket.go
  36. 14 0
      third_party/code.google.com/p/go.net/websocket/websocket_test.go
  37. 3 3
      third_party/github.com/coreos/go-etcd/README.md
  38. 17 4
      third_party/github.com/coreos/go-etcd/etcd/client.go
  39. 21 1
      third_party/github.com/coreos/go-etcd/etcd/client_test.go
  40. 2 3
      third_party/github.com/coreos/go-etcd/etcd/delete.go
  41. 1 1
      third_party/github.com/coreos/go-etcd/etcd/delete_test.go
  42. 6 7
      third_party/github.com/coreos/go-etcd/etcd/get.go
  43. 1 1
      third_party/github.com/coreos/go-etcd/etcd/get_test.go
  44. 1 1
      third_party/github.com/coreos/go-etcd/etcd/list_test.go
  45. 26 0
      third_party/github.com/coreos/go-etcd/etcd/response.go
  46. 4 5
      third_party/github.com/coreos/go-etcd/etcd/set.go
  47. 1 1
      third_party/github.com/coreos/go-etcd/etcd/set_test.go
  48. 2 3
      third_party/github.com/coreos/go-etcd/etcd/testAndSet.go
  49. 1 1
      third_party/github.com/coreos/go-etcd/etcd/testAndSet_test.go
  50. 9 5
      third_party/github.com/coreos/go-etcd/etcd/watch.go
  51. 8 6
      third_party/github.com/coreos/go-etcd/etcd/watch_test.go
  52. 3 0
      third_party/github.com/coreos/go-etcd/examples/sync-cluster/README.md
  53. 51 0
      third_party/github.com/coreos/go-etcd/examples/sync-cluster/sync-cluster.go
  54. 16 14
      third_party/github.com/coreos/go-log/log/commands.go
  55. 1 2
      third_party/github.com/coreos/go-log/log/fields.go
  56. 1 2
      third_party/github.com/coreos/go-log/log/logger.go
  57. 1 2
      third_party/github.com/coreos/go-log/log/priority.go
  58. 1 2
      third_party/github.com/coreos/go-log/log/sinks.go
  59. 1 1
      third_party/github.com/coreos/go-raft/README.md
  60. 1 1
      third_party/github.com/coreos/go-raft/command.go
  61. 9 9
      third_party/github.com/coreos/go-raft/http_transporter.go
  62. 9 9
      third_party/github.com/coreos/go-raft/http_transporter_test.go
  63. 2 2
      third_party/github.com/coreos/go-raft/join_command.go
  64. 2 2
      third_party/github.com/coreos/go-raft/leave_command.go
  65. 1 1
      third_party/github.com/coreos/go-raft/nop_command.go
  66. 2 2
      third_party/github.com/coreos/go-raft/peer.go
  67. 100 61
      third_party/github.com/coreos/go-raft/server.go
  68. 143 146
      third_party/github.com/coreos/go-raft/server_test.go
  69. 3 3
      third_party/github.com/coreos/go-raft/snapshot.go
  70. 1 1
      third_party/github.com/coreos/go-raft/snapshot_recovery_request.go
  71. 14 14
      third_party/github.com/coreos/go-raft/test.go
  72. 4 4
      third_party/github.com/coreos/go-raft/transporter.go
  73. 2 1
      third_party/github.com/coreos/go-systemd/activation/files.go
  74. 100 0
      third_party/github.com/coreos/go-systemd/dbus/dbus.go
  75. 166 0
      third_party/github.com/coreos/go-systemd/dbus/methods.go
  76. 193 0
      third_party/github.com/coreos/go-systemd/dbus/properties.go
  77. 182 0
      third_party/github.com/coreos/go-systemd/dbus/subscription.go
  78. 5 1
      third_party/github.com/coreos/go-systemd/journal/send.go
  79. 1 1
      web/web.go

+ 6 - 6
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()
 

+ 2 - 2
server/join_command.go

@@ -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
 	}

+ 21 - 21
server/peer_server.go

@@ -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 == "" {

+ 5 - 5
server/peer_server_handlers.go

@@ -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

+ 2 - 2
server/registry.go

@@ -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),

+ 1 - 1
server/remove_command.go

@@ -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.

+ 12 - 12
server/server.go

@@ -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
 		}()

+ 4 - 4
server/transporter.go

@@ -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)

+ 1 - 1
server/v1/v1.go

@@ -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
 }

+ 1 - 1
server/v2/v2.go

@@ -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
 }

+ 2 - 2
store/create_command.go

@@ -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())
 

+ 2 - 2
store/delete_command.go

@@ -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())
 

+ 22 - 16
store/node.go

@@ -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()
 }

+ 2 - 2
store/stats_test.go

@@ -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
 

+ 43 - 25
store/store.go

@@ -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()
 }

+ 11 - 11
store/store_test.go

@@ -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 {

+ 2 - 2
store/test_and_set_command.go

@@ -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())

+ 2 - 2
store/update_command.go

@@ -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())
 

+ 1 - 1
store/watcher_test.go

@@ -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 {

+ 1 - 1
test/test.go

@@ -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)
 

+ 29 - 11
third_party/bitbucket.org/kardianos/osext/osext_sysctl.go

@@ -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 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
 	}
-	return string(buf), nil
 }

+ 20 - 26
third_party/code.google.com/p/go.net/ipv4/gen.go

@@ -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 {

+ 23 - 24
third_party/code.google.com/p/go.net/ipv4/gentest.go

@@ -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 {

+ 34 - 25
third_party/code.google.com/p/go.net/ipv4/header.go

@@ -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])

+ 5 - 4
third_party/code.google.com/p/go.net/ipv4/header_test.go

@@ -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,

+ 2 - 2
third_party/code.google.com/p/go.net/ipv4/iana_test.go

@@ -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

+ 20 - 26
third_party/code.google.com/p/go.net/ipv6/gen.go

@@ -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 {

+ 23 - 24
third_party/code.google.com/p/go.net/ipv6/gentest.go

@@ -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 {

+ 2 - 2
third_party/code.google.com/p/go.net/ipv6/iana.go

@@ -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",

+ 2 - 2
third_party/code.google.com/p/go.net/ipv6/iana_test.go

@@ -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

+ 2 - 16
third_party/code.google.com/p/go.net/websocket/client.go

@@ -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
 }
 

+ 5 - 21
third_party/code.google.com/p/go.net/websocket/hybi.go

@@ -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

+ 0 - 108
third_party/code.google.com/p/go.net/websocket/hybi_test.go

@@ -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}

+ 0 - 8
third_party/code.google.com/p/go.net/websocket/server.go

@@ -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")

+ 1 - 5
third_party/code.google.com/p/go.net/websocket/websocket.go

@@ -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

+ 14 - 0
third_party/code.google.com/p/go.net/websocket/websocket_test.go

@@ -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.

+ 3 - 3
third_party/github.com/coreos/go-etcd/README.md

@@ -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)
 }

+ 17 - 4
third_party/github.com/coreos/go-etcd/etcd/client.go

@@ -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()
 }
 

+ 21 - 1
third_party/github.com/coreos/go-etcd/etcd/client_test.go

@@ -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 - 3
third_party/github.com/coreos/go-etcd/etcd/delete.go

@@ -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)
 

+ 1 - 1
third_party/github.com/coreos/go-etcd/etcd/delete_test.go

@@ -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")

+ 6 - 7
third_party/github.com/coreos/go-etcd/etcd/get.go

@@ -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

+ 1 - 1
third_party/github.com/coreos/go-etcd/etcd/get_test.go

@@ -7,7 +7,7 @@ import (
 
 func TestGet(t *testing.T) {
 
-	c := NewClient()
+	c := NewClient(nil)
 
 	c.Set("foo", "bar", 100)
 

+ 1 - 1
third_party/github.com/coreos/go-etcd/etcd/list_test.go

@@ -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 - 0
third_party/github.com/coreos/go-etcd/etcd/response.go

@@ -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"`
+}

+ 4 - 5
third_party/github.com/coreos/go-etcd/etcd/set.go

@@ -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)
 

+ 1 - 1
third_party/github.com/coreos/go-etcd/etcd/set_test.go

@@ -6,7 +6,7 @@ import (
 )
 
 func TestSet(t *testing.T) {
-	c := NewClient()
+	c := NewClient(nil)
 
 	result, err := c.Set("foo", "bar", 100)
 

+ 2 - 3
third_party/github.com/coreos/go-etcd/etcd/testAndSet.go

@@ -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)
 

+ 1 - 1
third_party/github.com/coreos/go-etcd/etcd/testAndSet_test.go

@@ -6,7 +6,7 @@ import (
 )
 
 func TestTestAndSet(t *testing.T) {
-	c := NewClient()
+	c := NewClient(nil)
 
 	c.Set("foo_testAndSet", "bar", 100)
 

+ 9 - 5
third_party/github.com/coreos/go-etcd/etcd/watch.go

@@ -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)
 

+ 8 - 6
third_party/github.com/coreos/go-etcd/etcd/watch_test.go

@@ -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 - 0
third_party/github.com/coreos/go-etcd/examples/sync-cluster/README.md

@@ -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 - 0
third_party/github.com/coreos/go-etcd/examples/sync-cluster/sync-cluster.go

@@ -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))
+			}
+
+		}
+	}
+}

+ 16 - 14
third_party/github.com/coreos/go-log/log/commands.go

@@ -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 - 2
third_party/github.com/coreos/go-log/log/fields.go

@@ -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 - 2
third_party/github.com/coreos/go-log/log/logger.go

@@ -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 - 2
third_party/github.com/coreos/go-log/log/priority.go

@@ -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 - 2
third_party/github.com/coreos/go-log/log/sinks.go

@@ -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 - 1
third_party/github.com/coreos/go-raft/README.md

@@ -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

+ 1 - 1
third_party/github.com/coreos/go-raft/command.go

@@ -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 {

+ 9 - 9
third_party/github.com/coreos/go-raft/http_transporter.go

@@ -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")
 

+ 9 - 9
third_party/github.com/coreos/go-raft/http_transporter_test.go

@@ -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{})
 	}

+ 2 - 2
third_party/github.com/coreos/go-raft/join_command.go

@@ -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

+ 2 - 2
third_party/github.com/coreos/go-raft/leave_command.go

@@ -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

+ 1 - 1
third_party/github.com/coreos/go-raft/nop_command.go

@@ -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
 }
 

+ 2 - 2
third_party/github.com/coreos/go-raft/peer.go

@@ -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,

+ 100 - 61
third_party/github.com/coreos/go-raft/server.go

@@ -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...))
 }

+ 143 - 146
third_party/github.com/coreos/go-raft/server_test.go

@@ -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())
 				}
 			}
 

+ 3 - 3
third_party/github.com/coreos/go-raft/snapshot.go

@@ -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

+ 1 - 1
third_party/github.com/coreos/go-raft/snapshot_recovery_request.go

@@ -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))
 

+ 14 - 14
third_party/github.com/coreos/go-raft/test.go

@@ -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
 }

+ 4 - 4
third_party/github.com/coreos/go-raft/transporter.go

@@ -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
 }

+ 2 - 1
third_party/github.com/coreos/go-systemd/activation/files.go

@@ -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 - 0
third_party/github.com/coreos/go-systemd/dbus/dbus.go

@@ -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 - 0
third_party/github.com/coreos/go-systemd/dbus/methods.go

@@ -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 - 0
third_party/github.com/coreos/go-systemd/dbus/properties.go

@@ -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 - 0
third_party/github.com/coreos/go-systemd/dbus/subscription.go

@@ -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)
+			}
+		}
+	}
+}

+ 5 - 1
third_party/github.com/coreos/go-systemd/journal/send.go

@@ -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

+ 1 - 1
web/web.go

@@ -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()