Merge pull request #135 from xiangli-cmu/moreStats

WIP More stats
release-0.4
Xiang Li 2013-09-10 09:29:12 -07:00
commit 23e99b57a6
14 changed files with 327 additions and 58 deletions

View File

@ -4,12 +4,13 @@ import (
"encoding/binary"
"encoding/json"
"fmt"
etcdErr "github.com/coreos/etcd/error"
"github.com/coreos/etcd/store"
"github.com/coreos/go-raft"
"os"
"path"
"time"
etcdErr "github.com/coreos/etcd/error"
"github.com/coreos/etcd/store"
"github.com/coreos/go-raft"
)
const commandPrefix = "etcd:"
@ -169,6 +170,10 @@ func (c *JoinCommand) Apply(raftServer *raft.Server) (interface{}, error) {
value := fmt.Sprintf("raft=%s&etcd=%s&raftVersion=%s", c.RaftURL, c.EtcdURL, c.RaftVersion)
etcdStore.Set(key, value, time.Unix(0, 0), raftServer.CommitIndex())
if c.Name != r.Name() {
r.peersStats[c.Name] = &raftPeerStats{MinLatency: 1 << 63}
}
return b, err
}
@ -193,6 +198,7 @@ func (c *RemoveCommand) Apply(raftServer *raft.Server) (interface{}, error) {
key := path.Join("_etcd/machines", c.Name)
_, err := etcdStore.Delete(key, raftServer.CommitIndex())
delete(r.peersStats, c.Name)
if err != nil {
return []byte{0}, err

View File

@ -4,13 +4,14 @@ import (
"crypto/tls"
"flag"
"fmt"
"github.com/coreos/etcd/store"
"github.com/coreos/go-raft"
"io/ioutil"
"net/url"
"os"
"strings"
"time"
"github.com/coreos/etcd/store"
"github.com/coreos/go-raft"
)
//------------------------------------------------------------------------------

View File

@ -2,12 +2,13 @@ package main
import (
"fmt"
etcdErr "github.com/coreos/etcd/error"
"github.com/coreos/etcd/store"
"github.com/coreos/go-raft"
"net/http"
"strconv"
"strings"
etcdErr "github.com/coreos/etcd/error"
"github.com/coreos/etcd/store"
"github.com/coreos/go-raft"
)
//-------------------------------------------------------------------
@ -228,6 +229,7 @@ func VersionHttpHandler(w http.ResponseWriter, req *http.Request) error {
func StatsHttpHandler(w http.ResponseWriter, req *http.Request) error {
w.WriteHeader(http.StatusOK)
w.Write(etcdStore.Stats())
w.Write(r.Stats())
return nil
}

View File

@ -31,7 +31,7 @@ func newEtcdServer(name string, urlStr string, listenHost string, tlsConf *TLSCo
// Start to listen and response etcd client command
func (e *etcdServer) ListenAndServe() {
infof("etcd server [%s:%s]", e.name, e.url)
infof("etcd server [name %s, listen on %s, advertised url %s]", e.name, e.Server.Addr, e.url)
if e.tlsConf.Scheme == "http" {
fatal(e.Server.ListenAndServe())

View File

@ -2,8 +2,6 @@ package main
import (
"fmt"
"github.com/coreos/etcd/test"
"github.com/coreos/go-etcd/etcd"
"math/rand"
"net/http"
"net/http/httptest"
@ -13,6 +11,9 @@ import (
"strings"
"testing"
"time"
"github.com/coreos/etcd/test"
"github.com/coreos/go-etcd/etcd"
)
// Create a single node and try to set value

View File

@ -2,8 +2,9 @@ package main
import (
"encoding/json"
"github.com/coreos/go-raft"
"net/http"
"github.com/coreos/go-raft"
)
//-------------------------------------------------------------
@ -41,6 +42,9 @@ func AppendEntriesHttpHandler(w http.ResponseWriter, req *http.Request) {
if err == nil {
debugf("[recv] POST %s/log/append [%d]", r.url, len(aereq.Entries))
r.serverStats.RecvAppendReq(aereq.LeaderName, int(req.ContentLength))
if resp := r.AppendEntries(aereq); resp != nil {
w.WriteHeader(http.StatusOK)
json.NewEncoder(w).Encode(resp)

View File

@ -6,23 +6,26 @@ import (
"encoding/binary"
"encoding/json"
"fmt"
etcdErr "github.com/coreos/etcd/error"
"github.com/coreos/go-raft"
"io/ioutil"
"net/http"
"net/url"
"time"
etcdErr "github.com/coreos/etcd/error"
"github.com/coreos/go-raft"
)
type raftServer struct {
*raft.Server
version string
joinIndex uint64
name string
url string
listenHost string
tlsConf *TLSConfig
tlsInfo *TLSInfo
version string
joinIndex uint64
name string
url string
listenHost string
tlsConf *TLSConfig
tlsInfo *TLSInfo
peersStats map[string]*raftPeerStats
serverStats *raftServerStats
}
var r *raftServer
@ -45,6 +48,16 @@ func newRaftServer(name string, url string, listenHost string, tlsConf *TLSConfi
listenHost: listenHost,
tlsConf: tlsConf,
tlsInfo: tlsInfo,
peersStats: make(map[string]*raftPeerStats),
serverStats: &raftServerStats{
StartTime: time.Now(),
sendRateQueue: &statsQueue{
back: -1,
},
recvRateQueue: &statsQueue{
back: -1,
},
},
}
}
@ -93,7 +106,7 @@ func (r *raftServer) ListenAndServe() {
}
ok := joinCluster(cluster)
if !ok {
warn("the whole cluster dies! restart the cluster")
warn("the entire cluster is down! this machine will restart the cluster.")
}
debugf("%s restart as a follower", r.name)
@ -136,7 +149,7 @@ func startAsFollower() {
// Start to listen and response raft command
func (r *raftServer) startTransport(scheme string, tlsConf tls.Config) {
infof("raft server [%s:%s]", r.name, r.listenHost)
infof("raft server [name %s, listen on %s, advertised url %s]", r.name, r.listenHost, r.url)
raftMux := http.NewServeMux()
@ -268,6 +281,33 @@ func joinByMachine(s *raft.Server, machine string, scheme string) error {
return fmt.Errorf("Unable to join: %v", err)
}
func (r *raftServer) Stats() []byte {
r.serverStats.LeaderUptime = time.Now().Sub(r.serverStats.leaderStartTime).String()
queue := r.serverStats.sendRateQueue
r.serverStats.SendingPkgRate, r.serverStats.SendingBandwidthRate = queue.Rate()
queue = r.serverStats.recvRateQueue
r.serverStats.RecvingPkgRate, r.serverStats.RecvingBandwidthRate = queue.Rate()
sBytes, err := json.Marshal(r.serverStats)
if err != nil {
warn(err)
}
if r.State() == raft.Leader {
pBytes, _ := json.Marshal(r.peersStats)
b := append(sBytes, pBytes...)
return b
}
return sBytes
}
// Register commands to raft server
func registerCommands() {
raft.RegisterCommand(&JoinCommand{})

195
raft_stats.go Normal file
View File

@ -0,0 +1,195 @@
package main
import (
"math"
"sync"
"time"
"github.com/coreos/go-raft"
)
const (
queueCapacity = 200
)
// packageStats represent the stats we need for a package.
// It has sending time and the size of the package.
type packageStats struct {
sendingTime time.Time
size int
}
// NewPackageStats creates a pacakgeStats and return the pointer to it.
func NewPackageStats(now time.Time, size int) *packageStats {
return &packageStats{
sendingTime: now,
size: size,
}
}
// Time return the sending time of the package.
func (ps *packageStats) Time() time.Time {
return ps.sendingTime
}
type raftServerStats struct {
State string `json:"state"`
StartTime time.Time `json:"startTime"`
Leader string `json:"leader"`
LeaderUptime string `json:"leaderUptime"`
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"`
leaderStartTime time.Time
sendRateQueue *statsQueue
recvRateQueue *statsQueue
}
func (ss *raftServerStats) RecvAppendReq(leaderName string, pkgSize int) {
ss.State = raft.Follower
if leaderName != ss.Leader {
ss.Leader = leaderName
ss.leaderStartTime = time.Now()
}
ss.recvRateQueue.Insert(NewPackageStats(time.Now(), pkgSize))
ss.RecvAppendRequestCnt++
}
func (ss *raftServerStats) SendAppendReq(pkgSize int) {
now := time.Now()
if ss.State != raft.Leader {
ss.State = raft.Leader
ss.Leader = r.Name()
ss.leaderStartTime = now
}
ss.sendRateQueue.Insert(NewPackageStats(time.Now(), pkgSize))
ss.SendAppendRequestCnt++
}
type raftPeerStats struct {
Latency float64 `json:"latency"`
AvgLatency float64 `json:"averageLatency"`
avgLatencySquare float64
SdvLatency float64 `json:"sdvLatency"`
MinLatency float64 `json:"minLatency"`
MaxLatency float64 `json:"maxLatency"`
FailCnt uint64 `json:"failsCount"`
SuccCnt uint64 `json:"successCount"`
}
// Succ function update the raftPeerStats with a successful send
func (ps *raftPeerStats) Succ(d time.Duration) {
total := float64(ps.SuccCnt) * ps.AvgLatency
totalSquare := float64(ps.SuccCnt) * ps.avgLatencySquare
ps.SuccCnt++
ps.Latency = float64(d) / (1000000.0)
if ps.Latency > ps.MaxLatency {
ps.MaxLatency = ps.Latency
}
if ps.Latency < ps.MinLatency {
ps.MinLatency = ps.Latency
}
ps.AvgLatency = (total + ps.Latency) / float64(ps.SuccCnt)
ps.avgLatencySquare = (totalSquare + ps.Latency*ps.Latency) / float64(ps.SuccCnt)
// sdv = sqrt(avg(x^2) - avg(x)^2)
ps.SdvLatency = math.Sqrt(ps.avgLatencySquare - ps.AvgLatency*ps.AvgLatency)
}
// Fail function update the raftPeerStats with a unsuccessful send
func (ps *raftPeerStats) Fail() {
ps.FailCnt++
}
type statsQueue struct {
items [queueCapacity]*packageStats
size int
front int
back int
totalPkgSize int
rwl sync.RWMutex
}
func (q *statsQueue) Len() int {
return q.size
}
func (q *statsQueue) PkgSize() int {
return q.totalPkgSize
}
// 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() (*packageStats, *packageStats) {
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 packageStats into the queue and update the records
func (q *statsQueue) Insert(p *packageStats) {
q.rwl.Lock()
defer q.rwl.Unlock()
q.back = (q.back + 1) % queueCapacity
if q.size == queueCapacity { //dequeue
q.totalPkgSize -= q.items[q.front].size
q.front = (q.back + 1) % queueCapacity
} else {
q.size++
}
q.items[q.back] = p
q.totalPkgSize += 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.Time()) > time.Second {
q.Clear()
return 0, 0
}
sampleDuration := back.Time().Sub(front.Time())
pr := float64(q.Len()) / float64(sampleDuration) * float64(time.Second)
br := float64(q.PkgSize()) / 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.totalPkgSize = 0
}

View File

@ -3,11 +3,12 @@ package store
import (
"encoding/json"
"fmt"
etcdErr "github.com/coreos/etcd/error"
"path"
"strconv"
"sync"
"time"
etcdErr "github.com/coreos/etcd/error"
)
//------------------------------------------------------------------------------

View File

@ -1781,7 +1781,7 @@ func isASCIIDigit(c byte) bool {
// but it's so remote we're prepared to pretend it's nonexistent - since the
// C++ generator lowercases names, it's extremely unlikely to have two fields
// with different capitalizations.
// In short, _my_field_name_2 becomes XMyFieldName2.
// In short, _my_field_name_2 becomes XMyFieldName_2.
func CamelCase(s string) string {
if s == "" {
return ""

View File

@ -927,26 +927,25 @@ func (s *Server) processRequestVoteRequest(req *RequestVoteRequest) (*RequestVot
// Adds a peer to the server.
func (s *Server) AddPeer(name string, connectiongString string) error {
s.debugln("server.peer.add: ", name, len(s.peers))
defer s.writeConf()
// Do not allow peers to be added twice.
if s.peers[name] != nil {
return nil
}
// Skip the Peer if it has the same name as the Server
if s.name == name {
return nil
if s.name != name {
peer := newPeer(s, name, connectiongString, s.heartbeatTimeout)
if s.State() == Leader {
peer.startHeartbeat()
}
s.peers[peer.Name] = peer
}
peer := newPeer(s, name, connectiongString, s.heartbeatTimeout)
if s.State() == Leader {
peer.startHeartbeat()
}
s.peers[peer.Name] = peer
s.debugln("server.peer.conf.write: ", name)
// Write the configuration to file.
s.writeConf()
return nil
}
@ -955,26 +954,24 @@ func (s *Server) AddPeer(name string, connectiongString string) error {
func (s *Server) RemovePeer(name string) error {
s.debugln("server.peer.remove: ", name, len(s.peers))
defer s.writeConf()
// Skip the Peer if it has the same name as the Server
if name != s.Name() {
// Return error if peer doesn't exist.
peer := s.peers[name]
if peer == nil {
return fmt.Errorf("raft: Peer not found: %s", name)
}
if name == s.Name() {
// when the removed node restart, it should be able
// to know it has been removed before. So we need
// to update knownCommitIndex
return nil
}
// Return error if peer doesn't exist.
peer := s.peers[name]
if peer == nil {
return fmt.Errorf("raft: Peer not found: %s", name)
// Stop peer and remove it.
if s.State() == Leader {
peer.stopHeartbeat(true)
}
delete(s.peers, name)
}
// Stop peer and remove it.
if s.State() == Leader {
peer.stopHeartbeat(true)
}
delete(s.peers, name)
// Write the configuration to file.
s.writeConf()
return nil
}

View File

@ -6,7 +6,6 @@ import (
"fmt"
"hash/crc32"
"os"
"syscall"
)
//------------------------------------------------------------------------------
@ -54,7 +53,7 @@ func (ss *Snapshot) save() error {
}
// force the change writting to disk
syscall.Fsync(int(file.Fd()))
file.Sync()
return err
}

View File

@ -5,11 +5,12 @@ import (
"crypto/tls"
"encoding/json"
"fmt"
"github.com/coreos/go-raft"
"io"
"net"
"net/http"
"time"
"github.com/coreos/go-raft"
)
// Transporter layer for communication between raft nodes
@ -54,17 +55,38 @@ func dialTimeout(network, addr string) (net.Conn, error) {
func (t *transporter) SendAppendEntriesRequest(server *raft.Server, peer *raft.Peer, req *raft.AppendEntriesRequest) *raft.AppendEntriesResponse {
var aersp *raft.AppendEntriesResponse
var b bytes.Buffer
json.NewEncoder(&b).Encode(req)
size := b.Len()
r.serverStats.SendAppendReq(size)
u, _ := nameToRaftURL(peer.Name)
debugf("Send LogEntries to %s ", u)
thisPeerStats, ok := r.peersStats[peer.Name]
start := time.Now()
resp, err := t.Post(fmt.Sprintf("%s/log/append", u), &b)
end := time.Now()
if err != nil {
debugf("Cannot send AppendEntriesRequest to %s: %s", u, err)
if ok {
thisPeerStats.Fail()
}
} else {
if ok {
thisPeerStats.Succ(end.Sub(start))
}
}
r.peersStats[peer.Name] = thisPeerStats
if resp != nil {
defer resp.Body.Close()
aersp = &raft.AppendEntriesResponse{}

View File

@ -3,7 +3,6 @@ package main
import (
"encoding/json"
"fmt"
"github.com/coreos/etcd/web"
"io"
"log"
"net"
@ -14,6 +13,8 @@ import (
"runtime/pprof"
"strconv"
"time"
"github.com/coreos/etcd/web"
)
//--------------------------------------