Browse Source

Merge pull request #1285 from jonboulle/stats_leader_followers

etcdserver: add StoreStats, ServerStats and LeaderStats
Jonathan Boulle 11 years ago
parent
commit
6a30d3ba04

+ 33 - 6
etcdserver/cluster_store.go

@@ -6,8 +6,12 @@ import (
 	"fmt"
 	"fmt"
 	"log"
 	"log"
 	"net/http"
 	"net/http"
+	"strconv"
+	"time"
 
 
 	etcdErr "github.com/coreos/etcd/error"
 	etcdErr "github.com/coreos/etcd/error"
+
+	"github.com/coreos/etcd/etcdserver/stats"
 	"github.com/coreos/etcd/raft/raftpb"
 	"github.com/coreos/etcd/raft/raftpb"
 	"github.com/coreos/etcd/store"
 	"github.com/coreos/etcd/store"
 )
 )
@@ -103,19 +107,25 @@ func (s *clusterStore) Remove(id uint64) {
 	}
 	}
 }
 }
 
 
-func Sender(t *http.Transport, cls ClusterStore) func(msgs []raftpb.Message) {
+// Sender creates the default production sender used to transport raft messages
+// in the cluster. The returned sender will update the given ServerStats and
+// LeaderStats appropriately.
+func Sender(t *http.Transport, cls ClusterStore, ss *stats.ServerStats, ls *stats.LeaderStats) func(msgs []raftpb.Message) {
 	c := &http.Client{Transport: t}
 	c := &http.Client{Transport: t}
 
 
 	return func(msgs []raftpb.Message) {
 	return func(msgs []raftpb.Message) {
 		for _, m := range msgs {
 		for _, m := range msgs {
 			// TODO: reuse go routines
 			// TODO: reuse go routines
 			// limit the number of outgoing connections for the same receiver
 			// limit the number of outgoing connections for the same receiver
-			go send(c, cls, m)
+			go send(c, cls, m, ss, ls)
 		}
 		}
 	}
 	}
 }
 }
 
 
-func send(c *http.Client, cls ClusterStore, m raftpb.Message) {
+// send uses the given client to send a message to a member in the given
+// ClusterStore, retrying up to 3 times for each message. The given
+// ServerStats and LeaderStats are updated appropriately
+func send(c *http.Client, cls ClusterStore, m raftpb.Message, ss *stats.ServerStats, ls *stats.LeaderStats) {
 	// TODO (xiangli): reasonable retry logic
 	// TODO (xiangli): reasonable retry logic
 	for i := 0; i < 3; i++ {
 	for i := 0; i < 3; i++ {
 		u := cls.Get().Pick(m.To)
 		u := cls.Get().Pick(m.To)
@@ -126,7 +136,6 @@ func send(c *http.Client, cls ClusterStore, m raftpb.Message) {
 			log.Printf("etcdhttp: no addr for %d", m.To)
 			log.Printf("etcdhttp: no addr for %d", m.To)
 			return
 			return
 		}
 		}
-
 		u = fmt.Sprintf("%s%s", u, raftPrefix)
 		u = fmt.Sprintf("%s%s", u, raftPrefix)
 
 
 		// TODO: don't block. we should be able to have 1000s
 		// TODO: don't block. we should be able to have 1000s
@@ -136,13 +145,31 @@ func send(c *http.Client, cls ClusterStore, m raftpb.Message) {
 			log.Println("etcdhttp: dropping message:", err)
 			log.Println("etcdhttp: dropping message:", err)
 			return // drop bad message
 			return // drop bad message
 		}
 		}
-		if httpPost(c, u, data) {
-			return // success
+		if m.Type == raftpb.MsgApp {
+			ss.SendAppendReq(len(data))
+		}
+		to := strconv.FormatUint(m.To, 16)
+		fs, ok := ls.Followers[to]
+		if !ok {
+			fs = &stats.FollowerStats{}
+			fs.Latency.Minimum = 1 << 63
+			ls.Followers[to] = fs
+		}
+
+		start := time.Now()
+		sent := httpPost(c, u, data)
+		end := time.Now()
+		if sent {
+			fs.Succ(end.Sub(start))
+			return
 		}
 		}
+		fs.Fail()
 		// TODO: backoff
 		// TODO: backoff
 	}
 	}
 }
 }
 
 
+// httpPost POSTs a data payload to a url using the given client. Returns true
+// if the POST succeeds, false on any failure.
 func httpPost(c *http.Client, url string, data []byte) bool {
 func httpPost(c *http.Client, url string, data []byte) bool {
 	resp, err := c.Post(url, "application/protobuf", bytes.NewBuffer(data))
 	resp, err := c.Post(url, "application/protobuf", bytes.NewBuffer(data))
 	if err != nil {
 	if err != nil {

+ 49 - 0
etcdserver/etcdhttp/http.go

@@ -25,6 +25,7 @@ const (
 	deprecatedMachinesPrefix = "/v2/machines"
 	deprecatedMachinesPrefix = "/v2/machines"
 	adminMembersPrefix       = "/v2/admin/members/"
 	adminMembersPrefix       = "/v2/admin/members/"
 	raftPrefix               = "/raft"
 	raftPrefix               = "/raft"
+	statsPrefix              = "/v2/stats"
 
 
 	// time to wait for response from EtcdServer requests
 	// time to wait for response from EtcdServer requests
 	defaultServerTimeout = 500 * time.Millisecond
 	defaultServerTimeout = 500 * time.Millisecond
@@ -40,12 +41,16 @@ func NewClientHandler(server *etcdserver.EtcdServer) http.Handler {
 	sh := &serverHandler{
 	sh := &serverHandler{
 		server:       server,
 		server:       server,
 		clusterStore: server.ClusterStore,
 		clusterStore: server.ClusterStore,
+		stats:        server,
 		timer:        server,
 		timer:        server,
 		timeout:      defaultServerTimeout,
 		timeout:      defaultServerTimeout,
 	}
 	}
 	mux := http.NewServeMux()
 	mux := http.NewServeMux()
 	mux.HandleFunc(keysPrefix, sh.serveKeys)
 	mux.HandleFunc(keysPrefix, sh.serveKeys)
 	mux.HandleFunc(keysPrefix+"/", sh.serveKeys)
 	mux.HandleFunc(keysPrefix+"/", sh.serveKeys)
+	mux.HandleFunc(statsPrefix+"/store", sh.serveStoreStats)
+	mux.HandleFunc(statsPrefix+"/self", sh.serveSelfStats)
+	mux.HandleFunc(statsPrefix+"/leader", sh.serveLeaderStats)
 	// TODO: dynamic configuration may make this outdated. take care of it.
 	// TODO: dynamic configuration may make this outdated. take care of it.
 	// TODO: dynamic configuration may introduce race also.
 	// TODO: dynamic configuration may introduce race also.
 	// TODO: add serveMembers
 	// TODO: add serveMembers
@@ -70,6 +75,8 @@ func NewPeerHandler(server etcdserver.Server) http.Handler {
 type serverHandler struct {
 type serverHandler struct {
 	timeout      time.Duration
 	timeout      time.Duration
 	server       etcdserver.Server
 	server       etcdserver.Server
+	stats        etcdserver.ServerStats
+	storestats   etcdserver.StoreStats
 	timer        etcdserver.RaftTimer
 	timer        etcdserver.RaftTimer
 	clusterStore etcdserver.ClusterStore
 	clusterStore etcdserver.ClusterStore
 }
 }
@@ -162,6 +169,44 @@ func (h serverHandler) serveAdminMembers(w http.ResponseWriter, r *http.Request)
 	}
 	}
 }
 }
 
 
+func (h serverHandler) serveStoreStats(w http.ResponseWriter, r *http.Request) {
+	if !allowMethod(w, r.Method, "GET") {
+		return
+	}
+	w.Header().Set("Content-Type", "application/json")
+	w.Write(h.storestats.JSON())
+}
+
+func (h serverHandler) serveSelfStats(w http.ResponseWriter, r *http.Request) {
+	if !allowMethod(w, r.Method, "GET") {
+		return
+	}
+	s := h.stats.SelfStats()
+	b, err := json.Marshal(s)
+	if err != nil {
+		log.Printf("error marshalling stats: %v\n", err)
+		http.Error(w, "Internal Server Error", http.StatusInternalServerError)
+		return
+	}
+	w.Header().Set("Content-Type", "application/json")
+	w.Write(b)
+}
+
+func (h serverHandler) serveLeaderStats(w http.ResponseWriter, r *http.Request) {
+	if !allowMethod(w, r.Method, "GET") {
+		return
+	}
+	s := h.stats.LeaderStats()
+	b, err := json.Marshal(s)
+	if err != nil {
+		log.Printf("error marshalling stats: %v\n", err)
+		http.Error(w, "Internal Server Error", http.StatusInternalServerError)
+		return
+	}
+	w.Header().Set("Content-Type", "application/json")
+	w.Write(b)
+}
+
 func (h serverHandler) serveRaft(w http.ResponseWriter, r *http.Request) {
 func (h serverHandler) serveRaft(w http.ResponseWriter, r *http.Request) {
 	if !allowMethod(w, r.Method, "POST") {
 	if !allowMethod(w, r.Method, "POST") {
 		return
 		return
@@ -180,6 +225,10 @@ func (h serverHandler) serveRaft(w http.ResponseWriter, r *http.Request) {
 		return
 		return
 	}
 	}
 	log.Printf("etcdhttp: raft recv message from %#x: %+v", m.From, m)
 	log.Printf("etcdhttp: raft recv message from %#x: %+v", m.From, m)
+	if m.Type == raftpb.MsgApp {
+		// TODO(jonboulle): standardize id uint-->string process: always base 16?
+		h.stats.SelfStats().RecvAppendReq(strconv.FormatUint(m.From, 16), int(r.ContentLength))
+	}
 	if err := h.server.Process(context.TODO(), m); err != nil {
 	if err := h.server.Process(context.TODO(), m); err != nil {
 		log.Println("etcdhttp: error processing raft message:", err)
 		log.Println("etcdhttp: error processing raft message:", err)
 		writeError(w, err)
 		writeError(w, err)

+ 123 - 1
etcdserver/etcdhttp/http_test.go

@@ -19,6 +19,7 @@ import (
 	etcdErr "github.com/coreos/etcd/error"
 	etcdErr "github.com/coreos/etcd/error"
 	"github.com/coreos/etcd/etcdserver"
 	"github.com/coreos/etcd/etcdserver"
 	"github.com/coreos/etcd/etcdserver/etcdserverpb"
 	"github.com/coreos/etcd/etcdserver/etcdserverpb"
+	"github.com/coreos/etcd/etcdserver/stats"
 	"github.com/coreos/etcd/raft/raftpb"
 	"github.com/coreos/etcd/raft/raftpb"
 	"github.com/coreos/etcd/store"
 	"github.com/coreos/etcd/store"
 )
 )
@@ -633,10 +634,131 @@ func TestServeMachines(t *testing.T) {
 		t.Errorf("body = %s, want %s", g, w)
 		t.Errorf("body = %s, want %s", g, w)
 	}
 	}
 	if writer.Code != http.StatusOK {
 	if writer.Code != http.StatusOK {
-		t.Errorf("header = %d, want %d", writer.Code, http.StatusOK)
+		t.Errorf("code = %d, want %d", writer.Code, http.StatusOK)
 	}
 	}
 }
 }
 
 
+type dummyServerStats struct {
+	ss *stats.ServerStats
+	ls *stats.LeaderStats
+}
+
+func (dss *dummyServerStats) SelfStats() *stats.ServerStats   { return dss.ss }
+func (dss *dummyServerStats) LeaderStats() *stats.LeaderStats { return dss.ls }
+
+func TestServeSelfStats(t *testing.T) {
+	ss := &stats.ServerStats{
+		Name:           "foobar",
+		RecvingPkgRate: 123.4,
+	}
+	w, err := json.Marshal(ss)
+	if err != nil {
+		t.Fatal("error marshaling: %v", err)
+	}
+	sh := &serverHandler{
+		stats: &dummyServerStats{
+			ss: ss,
+		},
+	}
+	rw := httptest.NewRecorder()
+	sh.serveSelfStats(rw, &http.Request{Method: "GET"})
+	if rw.Code != http.StatusOK {
+		t.Errorf("code = %d, want %d", rw.Code, http.StatusOK)
+	}
+	wct := "application/json"
+	if gct := rw.Header().Get("Content-Type"); gct != wct {
+		t.Errorf("Content-Type = %q, want %q", gct, wct)
+	}
+	if g := rw.Body.String(); g != string(w) {
+		t.Errorf("body = %s, want %s", g, string(w))
+	}
+}
+
+func TestSelfServeStatsBad(t *testing.T) {
+	for _, m := range []string{"PUT", "POST", "DELETE"} {
+		sh := &serverHandler{}
+		rw := httptest.NewRecorder()
+		sh.serveSelfStats(
+			rw,
+			&http.Request{
+				Method: m,
+			},
+		)
+		if rw.Code != http.StatusMethodNotAllowed {
+			t.Errorf("method %s: code=%d, want %d", m, http.StatusMethodNotAllowed)
+		}
+	}
+}
+
+func TestLeaderServeStatsBad(t *testing.T) {
+	for _, m := range []string{"PUT", "POST", "DELETE"} {
+		sh := &serverHandler{}
+		rw := httptest.NewRecorder()
+		sh.serveLeaderStats(
+			rw,
+			&http.Request{
+				Method: m,
+			},
+		)
+		if rw.Code != http.StatusMethodNotAllowed {
+			t.Errorf("method %s: code=%d, want %d", m, http.StatusMethodNotAllowed)
+		}
+	}
+}
+
+func TestServeLeaderStats(t *testing.T) {
+	ls := &stats.LeaderStats{
+		Leader: "foobar",
+	}
+	w, err := json.Marshal(ls)
+	if err != nil {
+		t.Fatal("error marshaling: %v", err)
+	}
+	sh := &serverHandler{
+		stats: &dummyServerStats{
+			ls: ls,
+		},
+	}
+	rw := httptest.NewRecorder()
+	sh.serveLeaderStats(rw, &http.Request{Method: "GET"})
+	if rw.Code != http.StatusOK {
+		t.Errorf("code = %d, want %d", rw.Code, http.StatusOK)
+	}
+	wct := "application/json"
+	if gct := rw.Header().Get("Content-Type"); gct != wct {
+		t.Errorf("Content-Type = %q, want %q", gct, wct)
+	}
+	if g := rw.Body.String(); g != string(w) {
+		t.Errorf("body = %s, want %s", g, string(w))
+	}
+}
+
+type dummyStoreStats struct {
+	data []byte
+}
+
+func (dss *dummyStoreStats) JSON() []byte { return dss.data }
+
+func TestServeStoreStats(t *testing.T) {
+	w := "foobarbaz"
+	sh := &serverHandler{
+		storestats: &dummyStoreStats{data: []byte(w)},
+	}
+	rw := httptest.NewRecorder()
+	sh.serveStoreStats(rw, &http.Request{Method: "GET"})
+	if rw.Code != http.StatusOK {
+		t.Errorf("code = %d, want %d", rw.Code, http.StatusOK)
+	}
+	wct := "application/json"
+	if gct := rw.Header().Get("Content-Type"); gct != wct {
+		t.Errorf("Content-Type = %q, want %q", gct, wct)
+	}
+	if g := rw.Body.String(); g != w {
+		t.Errorf("body = %s, want %s", g, w)
+	}
+
+}
+
 func TestAllowMethod(t *testing.T) {
 func TestAllowMethod(t *testing.T) {
 	tests := []struct {
 	tests := []struct {
 		m  string
 		m  string

+ 46 - 1
etcdserver/server.go

@@ -6,12 +6,14 @@ import (
 	"log"
 	"log"
 	"math/rand"
 	"math/rand"
 	"os"
 	"os"
+	"strconv"
 	"sync/atomic"
 	"sync/atomic"
 	"time"
 	"time"
 
 
 	"github.com/coreos/etcd/Godeps/_workspace/src/code.google.com/p/go.net/context"
 	"github.com/coreos/etcd/Godeps/_workspace/src/code.google.com/p/go.net/context"
 	"github.com/coreos/etcd/discovery"
 	"github.com/coreos/etcd/discovery"
 	pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
 	pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
+	"github.com/coreos/etcd/etcdserver/stats"
 	"github.com/coreos/etcd/pkg/pbutil"
 	"github.com/coreos/etcd/pkg/pbutil"
 	"github.com/coreos/etcd/raft"
 	"github.com/coreos/etcd/raft"
 	"github.com/coreos/etcd/raft/raftpb"
 	"github.com/coreos/etcd/raft/raftpb"
@@ -88,6 +90,20 @@ type Server interface {
 	RemoveMember(ctx context.Context, id uint64) error
 	RemoveMember(ctx context.Context, id uint64) error
 }
 }
 
 
+type ServerStats interface {
+	// SelfStats returns the statistics of this server
+	SelfStats() *stats.ServerStats
+	// LeaderStats returns the statistics of all followers in the cluster
+	// if this server is leader. Otherwise, nil is returned.
+	LeaderStats() *stats.LeaderStats
+}
+
+type StoreStats interface {
+	// JSON returns statistics of the underlying Store used by the
+	// EtcdServer, in JSON format
+	JSON() []byte
+}
+
 type RaftTimer interface {
 type RaftTimer interface {
 	Index() uint64
 	Index() uint64
 	Term() uint64
 	Term() uint64
@@ -105,6 +121,9 @@ type EtcdServer struct {
 	node  raft.Node
 	node  raft.Node
 	store store.Store
 	store store.Store
 
 
+	stats  *stats.ServerStats
+	lstats *stats.LeaderStats
+
 	// send specifies the send function for sending msgs to members. send
 	// send specifies the send function for sending msgs to members. send
 	// MUST NOT block. It is okay to drop messages, since clients should
 	// MUST NOT block. It is okay to drop messages, since clients should
 	// timeout and reissue their messages.  If send is nil, server will
 	// timeout and reissue their messages.  If send is nil, server will
@@ -172,6 +191,13 @@ func NewServer(cfg *ServerConfig) *EtcdServer {
 	}
 	}
 
 
 	cls := &clusterStore{Store: st}
 	cls := &clusterStore{Store: st}
+
+	sstats := &stats.ServerStats{
+		Name: cfg.Name,
+		ID:   strconv.FormatUint(cfg.ID(), 16),
+	}
+	lstats := stats.NewLeaderStats(strconv.FormatUint(cfg.ID(), 16))
+
 	s := &EtcdServer{
 	s := &EtcdServer{
 		store:      st,
 		store:      st,
 		node:       n,
 		node:       n,
@@ -181,7 +207,9 @@ func NewServer(cfg *ServerConfig) *EtcdServer {
 			*wal.WAL
 			*wal.WAL
 			*snap.Snapshotter
 			*snap.Snapshotter
 		}{w, ss},
 		}{w, ss},
-		send:         Sender(cfg.Transport, cls),
+		stats:        sstats,
+		lstats:       lstats,
+		send:         Sender(cfg.Transport, cls, sstats, lstats),
 		ticker:       time.Tick(100 * time.Millisecond),
 		ticker:       time.Tick(100 * time.Millisecond),
 		syncTicker:   time.Tick(500 * time.Millisecond),
 		syncTicker:   time.Tick(500 * time.Millisecond),
 		snapCount:    cfg.SnapCount,
 		snapCount:    cfg.SnapCount,
@@ -208,6 +236,7 @@ func (s *EtcdServer) start() {
 	}
 	}
 	s.w = wait.New()
 	s.w = wait.New()
 	s.done = make(chan struct{})
 	s.done = make(chan struct{})
+	s.stats.Initialize()
 	// TODO: if this is an empty log, writes all peer infos
 	// TODO: if this is an empty log, writes all peer infos
 	// into the first entry
 	// into the first entry
 	go s.run()
 	go s.run()
@@ -334,6 +363,22 @@ func (s *EtcdServer) Do(ctx context.Context, r pb.Request) (Response, error) {
 	}
 	}
 }
 }
 
 
+func (s *EtcdServer) SelfStats() *stats.ServerStats {
+	s.stats.LeaderInfo.Uptime = time.Now().Sub(s.stats.LeaderInfo.StartTime).String()
+	s.stats.SendingPkgRate, s.stats.SendingBandwidthRate = s.stats.SendRates()
+	s.stats.RecvingPkgRate, s.stats.RecvingBandwidthRate = s.stats.RecvRates()
+	return s.stats
+}
+
+func (s *EtcdServer) LeaderStats() *stats.LeaderStats {
+	// TODO(jonboulle): need to lock access to lstats, set it to nil when not leader, ...
+	return s.lstats
+}
+
+func (s *EtcdServer) StoreStats() []byte {
+	return s.store.JsonStats()
+}
+
 func (s *EtcdServer) AddMember(ctx context.Context, memb Member) error {
 func (s *EtcdServer) AddMember(ctx context.Context, memb Member) error {
 	// TODO: move Member to protobuf type
 	// TODO: move Member to protobuf type
 	b, err := json.Marshal(memb)
 	b, err := json.Marshal(memb)

+ 68 - 0
etcdserver/stats/leader.go

@@ -0,0 +1,68 @@
+package stats
+
+import (
+	"math"
+	"time"
+)
+
+// LeaderStats is used by the leader in an etcd cluster, and encapsulates
+// statistics about communication with its followers
+type LeaderStats struct {
+	// TODO(jonboulle): clarify that these are IDs, not names
+	Leader    string                    `json:"leader"`
+	Followers map[string]*FollowerStats `json:"followers"`
+}
+
+// NewLeaderStats generates a new LeaderStats with the given id as leader
+func NewLeaderStats(id string) *LeaderStats {
+	return &LeaderStats{
+		Leader:    id,
+		Followers: make(map[string]*FollowerStats),
+	}
+}
+
+// FollowerStats encapsulates various statistics about a follower in an etcd cluster
+type FollowerStats struct {
+	Latency struct {
+		Current           float64 `json:"current"`
+		Average           float64 `json:"average"`
+		averageSquare     float64
+		StandardDeviation float64 `json:"standardDeviation"`
+		Minimum           float64 `json:"minimum"`
+		Maximum           float64 `json:"maximum"`
+	} `json:"latency"`
+
+	Counts struct {
+		Fail    uint64 `json:"fail"`
+		Success uint64 `json:"success"`
+	} `json:"counts"`
+}
+
+// Succ updates the FollowerStats with a successful send
+func (fs *FollowerStats) Succ(d time.Duration) {
+	total := float64(fs.Counts.Success) * fs.Latency.Average
+	totalSquare := float64(fs.Counts.Success) * fs.Latency.averageSquare
+
+	fs.Counts.Success++
+
+	fs.Latency.Current = float64(d) / (1000000.0)
+
+	if fs.Latency.Current > fs.Latency.Maximum {
+		fs.Latency.Maximum = fs.Latency.Current
+	}
+
+	if fs.Latency.Current < fs.Latency.Minimum {
+		fs.Latency.Minimum = fs.Latency.Current
+	}
+
+	fs.Latency.Average = (total + fs.Latency.Current) / float64(fs.Counts.Success)
+	fs.Latency.averageSquare = (totalSquare + fs.Latency.Current*fs.Latency.Current) / float64(fs.Counts.Success)
+
+	// sdv = sqrt(avg(x^2) - avg(x)^2)
+	fs.Latency.StandardDeviation = math.Sqrt(fs.Latency.averageSquare - fs.Latency.Average*fs.Latency.Average)
+}
+
+// Fail updates the FollowerStats with an unsuccessful send
+func (fs *FollowerStats) Fail() {
+	fs.Counts.Fail++
+}

+ 96 - 0
etcdserver/stats/queue.go

@@ -0,0 +1,96 @@
+package stats
+
+import (
+	"sync"
+	"time"
+)
+
+const (
+	queueCapacity = 200
+)
+
+// RequestStats represent the stats for a request.
+// It encapsulates the sending time and the size of the request.
+type RequestStats struct {
+	SendingTime time.Time
+	Size        int
+}
+
+type statsQueue struct {
+	items        [queueCapacity]*RequestStats
+	size         int
+	front        int
+	back         int
+	totalReqSize int
+	rwl          sync.RWMutex
+}
+
+func (q *statsQueue) Len() int {
+	return q.size
+}
+
+func (q *statsQueue) ReqSize() int {
+	return q.totalReqSize
+}
+
+// FrontAndBack gets the front and back elements in the queue
+// We must grab front and back together with the protection of the lock
+func (q *statsQueue) frontAndBack() (*RequestStats, *RequestStats) {
+	q.rwl.RLock()
+	defer q.rwl.RUnlock()
+	if q.size != 0 {
+		return q.items[q.front], q.items[q.back]
+	}
+	return nil, nil
+}
+
+// Insert function insert a RequestStats into the queue and update the records
+func (q *statsQueue) Insert(p *RequestStats) {
+	q.rwl.Lock()
+	defer q.rwl.Unlock()
+
+	q.back = (q.back + 1) % queueCapacity
+
+	if q.size == queueCapacity { //dequeue
+		q.totalReqSize -= q.items[q.front].Size
+		q.front = (q.back + 1) % queueCapacity
+	} else {
+		q.size++
+	}
+
+	q.items[q.back] = p
+	q.totalReqSize += q.items[q.back].Size
+
+}
+
+// Rate function returns the package rate and byte rate
+func (q *statsQueue) Rate() (float64, float64) {
+	front, back := q.frontAndBack()
+
+	if front == nil || back == nil {
+		return 0, 0
+	}
+
+	if time.Now().Sub(back.SendingTime) > time.Second {
+		q.Clear()
+		return 0, 0
+	}
+
+	sampleDuration := back.SendingTime.Sub(front.SendingTime)
+
+	pr := float64(q.Len()) / float64(sampleDuration) * float64(time.Second)
+
+	br := float64(q.ReqSize()) / float64(sampleDuration) * float64(time.Second)
+
+	return pr, br
+}
+
+// Clear function clear up the statsQueue
+func (q *statsQueue) Clear() {
+	q.rwl.Lock()
+	defer q.rwl.Unlock()
+	q.back = -1
+	q.front = 0
+	q.size = 0
+	q.totalReqSize = 0
+}

+ 110 - 0
etcdserver/stats/server.go

@@ -0,0 +1,110 @@
+package stats
+
+import (
+	"sync"
+	"time"
+
+	"github.com/coreos/etcd/raft"
+)
+
+// ServerStats encapsulates various statistics about an EtcdServer and its
+// communication with other members of the cluster
+type ServerStats struct {
+	Name string `json:"name"`
+	// TODO(jonboulle): use ID instead of name?
+	ID        string         `json:"id"`
+	State     raft.StateType `json:"state"`
+	StartTime time.Time      `json:"startTime"`
+
+	LeaderInfo struct {
+		Name      string    `json:"leader"`
+		Uptime    string    `json:"uptime"`
+		StartTime time.Time `json:"startTime"`
+	} `json:"leaderInfo"`
+
+	RecvAppendRequestCnt uint64  `json:"recvAppendRequestCnt,"`
+	RecvingPkgRate       float64 `json:"recvPkgRate,omitempty"`
+	RecvingBandwidthRate float64 `json:"recvBandwidthRate,omitempty"`
+
+	SendAppendRequestCnt uint64  `json:"sendAppendRequestCnt"`
+	SendingPkgRate       float64 `json:"sendPkgRate,omitempty"`
+	SendingBandwidthRate float64 `json:"sendBandwidthRate,omitempty"`
+
+	sendRateQueue *statsQueue
+	recvRateQueue *statsQueue
+
+	sync.Mutex
+}
+
+// Initialize clears the statistics of ServerStats and resets its start time
+func (ss *ServerStats) Initialize() {
+	if ss == nil {
+		return
+	}
+	now := time.Now()
+	ss.StartTime = now
+	ss.LeaderInfo.StartTime = now
+	ss.sendRateQueue = &statsQueue{
+		back: -1,
+	}
+	ss.recvRateQueue = &statsQueue{
+		back: -1,
+	}
+}
+
+// RecvRates calculates and returns the rate of received append requests
+func (ss *ServerStats) RecvRates() (float64, float64) {
+	return ss.recvRateQueue.Rate()
+}
+
+// SendRates calculates and returns the rate of sent append requests
+func (ss *ServerStats) SendRates() (float64, float64) {
+	return ss.sendRateQueue.Rate()
+}
+
+// RecvAppendReq updates the ServerStats in response to an AppendRequest
+// from the given leader being received
+func (ss *ServerStats) RecvAppendReq(leader string, reqSize int) {
+	ss.Lock()
+	defer ss.Unlock()
+
+	now := time.Now()
+
+	ss.State = raft.StateFollower
+	if leader != ss.LeaderInfo.Name {
+		ss.LeaderInfo.Name = leader
+		ss.LeaderInfo.StartTime = now
+	}
+
+	ss.recvRateQueue.Insert(
+		&RequestStats{
+			SendingTime: now,
+			Size:        reqSize,
+		},
+	)
+	ss.RecvAppendRequestCnt++
+}
+
+// SendAppendReq updates the ServerStats in response to an AppendRequest
+// being sent by this server
+func (ss *ServerStats) SendAppendReq(reqSize int) {
+	ss.Lock()
+	defer ss.Unlock()
+
+	now := time.Now()
+
+	if ss.State != raft.StateLeader {
+		ss.State = raft.StateLeader
+		ss.LeaderInfo.Name = ss.ID
+		ss.LeaderInfo.StartTime = now
+	}
+
+	ss.sendRateQueue.Insert(
+		&RequestStats{
+			SendingTime: now,
+			Size:        reqSize,
+		},
+	)
+
+	ss.SendAppendRequestCnt++
+}

+ 4 - 0
raft/raft.go

@@ -34,6 +34,10 @@ func (st StateType) String() string {
 	return stmap[uint64(st)]
 	return stmap[uint64(st)]
 }
 }
 
 
+func (st StateType) MarshalJSON() ([]byte, error) {
+	return []byte(fmt.Sprintf("%q", st.String())), nil
+}
+
 type progress struct {
 type progress struct {
 	match, next uint64
 	match, next uint64
 }
 }