Xiang Li 12 years ago
parent
commit
2d7c1be164

+ 30 - 7
README.md

@@ -187,17 +187,17 @@ The watch command returns immediately with the same response as previous.
 
 
 Etcd can be used as a centralized coordination service in a cluster and `TestAndSet` is the most basic operation to build distributed lock service. This command will set the value only if the client provided `prevValue` is equal the current key value.
 Etcd can be used as a centralized coordination service in a cluster and `TestAndSet` is the most basic operation to build distributed lock service. This command will set the value only if the client provided `prevValue` is equal the current key value.
 
 
-Here is a simple example. Let's create a key-value pair first: `testAndSet=one`.
+Here is a simple example. Let's create a key-value pair first: `foo=one`.
 
 
 ```sh
 ```sh
-curl -L http://127.0.0.1:4001/v1/keys/testAndSet -d value=one
+curl -L http://127.0.0.1:4001/v1/keys/foo -d value=one
 ```
 ```
 
 
-Let's try an invaild `TestAndSet` command.
+Let's try an invalid `TestAndSet` command.
 We can give another parameter prevValue to set command to make it a TestAndSet command.
 We can give another parameter prevValue to set command to make it a TestAndSet command.
 
 
 ```sh
 ```sh
-curl -L http://127.0.0.1:4001/v1/keys/testAndSet -d prevValue=two -d value=three
+curl -L http://127.0.0.1:4001/v1/keys/foo -d prevValue=two -d value=three
 ```
 ```
 
 
 This will try to test if the previous of the key is two, it is change it to three.
 This will try to test if the previous of the key is two, it is change it to three.
@@ -208,16 +208,16 @@ This will try to test if the previous of the key is two, it is change it to thre
 
 
 which means `testAndSet` failed.
 which means `testAndSet` failed.
 
 
-Let us try a vaild one.
+Let us try a valid one.
 
 
 ```sh
 ```sh
-curl -L http://127.0.0.1:4001/v1/keys/testAndSet -d prevValue=one -d value=two
+curl -L http://127.0.0.1:4001/v1/keys/foo -d prevValue=one -d value=two
 ```
 ```
 
 
 The response should be
 The response should be
 
 
 ```json
 ```json
-{"action":"SET","key":"/testAndSet","prevValue":"one","value":"two","index":10}
+{"action":"SET","key":"/foo","prevValue":"one","value":"two","index":10}
 ```
 ```
 
 
 We successfully changed the value from “one” to “two”, since we give the correct previous value.
 We successfully changed the value from “one” to “two”, since we give the correct previous value.
@@ -465,6 +465,16 @@ If you are using SSL for server to server communication, you must use it on all
 
 
 - [go-etcd](https://github.com/coreos/go-etcd)
 - [go-etcd](https://github.com/coreos/go-etcd)
 
 
+**Java libraries**
+
+- [justinsb/jetcd](https://github.com/justinsb/jetcd)
+- [diwakergupta/jetcd](https://github.com/diwakergupta/jetcd)
+
+
+**Python libraries**
+
+- [transitorykris/etcd-py](https://github.com/transitorykris/etcd-py)
+
 **Node libraries**
 **Node libraries**
 
 
 - [stianeikeland/node-etcd](https://github.com/stianeikeland/node-etcd)
 - [stianeikeland/node-etcd](https://github.com/stianeikeland/node-etcd)
@@ -487,6 +497,19 @@ If you are using SSL for server to server communication, you must use it on all
 - [mattn/etcd-vim](https://github.com/mattn/etcd-vim) - SET and GET keys from inside vim
 - [mattn/etcd-vim](https://github.com/mattn/etcd-vim) - SET and GET keys from inside vim
 - [mattn/etcdenv](https://github.com/mattn/etcdenv) - "env" shebang with etcd integration
 - [mattn/etcdenv](https://github.com/mattn/etcdenv) - "env" shebang with etcd integration
 
 
+## FAQ
+
+### What size cluster should I use?
+
+Every command the client sends to the master is broadcast it to all of the followers.
+But, the command is not be committed until the majority of the cluster machines receive that command.
+
+Because of this majority voting property the ideal cluster should be kept small to keep speed up and be made up of an odd number of machines.
+
+Odd numbers are good because if you have 8 machines the majority will be 5 and if you have 9 machines the majority with be 5.
+The result is that an 8 machine cluster can tolerate 3 machine failures and a 9 machine cluster can tolerate 4 nodes failures.
+And in the best case when all 9 machines are responding the cluster will perform at the speed of the fastest 5 nodes.
+
 ## Project Details
 ## Project Details
 
 
 ### Versioning
 ### Versioning

+ 5 - 0
command.go

@@ -239,6 +239,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)
 	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())
 	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
 	return b, err
 }
 }
 
 
@@ -263,6 +267,7 @@ func (c *RemoveCommand) Apply(raftServer *raft.Server) (interface{}, error) {
 	key := path.Join("_etcd/machines", c.Name)
 	key := path.Join("_etcd/machines", c.Name)
 
 
 	_, err := etcdStore.Delete(key, raftServer.CommitIndex())
 	_, err := etcdStore.Delete(key, raftServer.CommitIndex())
+	delete(r.peersStats, c.Name)
 
 
 	if err != nil {
 	if err != nil {
 		return []byte{0}, err
 		return []byte{0}, err

+ 27 - 0
etcd.go

@@ -3,7 +3,9 @@ package main
 import (
 import (
 	"crypto/tls"
 	"crypto/tls"
 	"flag"
 	"flag"
+	"fmt"
 	"io/ioutil"
 	"io/ioutil"
+	"net/url"
 	"os"
 	"os"
 	"strings"
 	"strings"
 	"time"
 	"time"
@@ -42,6 +44,9 @@ var (
 	maxClusterSize int
 	maxClusterSize int
 
 
 	cpuprofile string
 	cpuprofile string
+
+	cors     string
+	corsList map[string]bool
 )
 )
 
 
 func init() {
 func init() {
@@ -79,6 +84,8 @@ func init() {
 	flag.IntVar(&maxClusterSize, "maxsize", 9, "the max size of the cluster")
 	flag.IntVar(&maxClusterSize, "maxsize", 9, "the max size of the cluster")
 
 
 	flag.StringVar(&cpuprofile, "cpuprofile", "", "write cpu profile to file")
 	flag.StringVar(&cpuprofile, "cpuprofile", "", "write cpu profile to file")
+
+	flag.StringVar(&cors, "cors", "", "whitelist origins for cross-origin resource sharing (e.g. '*' or 'http://localhost:8001,etc')")
 }
 }
 
 
 const (
 const (
@@ -155,6 +162,8 @@ func main() {
 		raft.SetLogLevel(raft.Debug)
 		raft.SetLogLevel(raft.Debug)
 	}
 	}
 
 
+	parseCorsFlag()
+
 	if machines != "" {
 	if machines != "" {
 		cluster = strings.Split(machines, ",")
 		cluster = strings.Split(machines, ",")
 	} else if machinesFile != "" {
 	} else if machinesFile != "" {
@@ -211,3 +220,21 @@ func main() {
 	e.ListenAndServe()
 	e.ListenAndServe()
 
 
 }
 }
+
+// parseCorsFlag gathers up the cors whitelist and puts it into the corsList.
+func parseCorsFlag() {
+	if cors != "" {
+		corsList = make(map[string]bool)
+		list := strings.Split(cors, ",")
+		for _, v := range list {
+			fmt.Println(v)
+			if v != "*" {
+				_, err := url.Parse(v)
+				if err != nil {
+					panic(fmt.Sprintf("bad cors url: %s", err))
+				}
+			}
+			corsList[v] = true
+		}
+	}
+}

+ 23 - 1
etcd_handlers.go

@@ -28,7 +28,26 @@ func NewEtcdMuxer() *http.ServeMux {
 
 
 type errorHandler func(http.ResponseWriter, *http.Request) error
 type errorHandler func(http.ResponseWriter, *http.Request) error
 
 
+// addCorsHeader parses the request Origin header and loops through the user
+// provided allowed origins and sets the Access-Control-Allow-Origin header if
+// there is a match.
+func addCorsHeader(w http.ResponseWriter, r *http.Request) {
+	val, ok := corsList["*"]
+	if val && ok {
+		w.Header().Add("Access-Control-Allow-Origin", "*")
+		return
+	}
+
+	requestOrigin := r.Header.Get("Origin")
+	val, ok = corsList[requestOrigin]
+	if val && ok {
+		w.Header().Add("Access-Control-Allow-Origin", requestOrigin)
+		return
+	}
+}
+
 func (fn errorHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
 func (fn errorHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
+	addCorsHeader(w, r)
 	if e := fn(w, r); e != nil {
 	if e := fn(w, r); e != nil {
 		if etcdErr, ok := e.(etcdErr.Error); ok {
 		if etcdErr, ok := e.(etcdErr.Error); ok {
 			debug("Return error: ", etcdErr.Error())
 			debug("Return error: ", etcdErr.Error())
@@ -68,7 +87,9 @@ func CreateHttpHandler(w http.ResponseWriter, req *http.Request) error {
 
 
 	debugf("recv.post[%v] [%v%v]\n", req.RemoteAddr, req.Host, req.URL)
 	debugf("recv.post[%v] [%v%v]\n", req.RemoteAddr, req.Host, req.URL)
 
 
-	value := req.FormValue("value")
+	req.ParseForm()
+
+	value := req.Form.Get("value")
 
 
 	ttl := req.FormValue("ttl")
 	ttl := req.FormValue("ttl")
 
 
@@ -248,6 +269,7 @@ func VersionHttpHandler(w http.ResponseWriter, req *http.Request) error {
 func StatsHttpHandler(w http.ResponseWriter, req *http.Request) error {
 func StatsHttpHandler(w http.ResponseWriter, req *http.Request) error {
 	w.WriteHeader(http.StatusOK)
 	w.WriteHeader(http.StatusOK)
 	w.Write(etcdStore.Stats())
 	w.Write(etcdStore.Stats())
+	w.Write(r.Stats())
 	return nil
 	return nil
 }
 }
 
 

+ 1 - 1
etcd_server.go

@@ -31,7 +31,7 @@ func newEtcdServer(name string, urlStr string, listenHost string, tlsConf *TLSCo
 // Start to listen and response etcd client command
 // Start to listen and response etcd client command
 func (e *etcdServer) ListenAndServe() {
 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" {
 	if e.tlsConf.Scheme == "http" {
 		fatal(e.Server.ListenAndServe())
 		fatal(e.Server.ListenAndServe())

+ 26 - 0
etcd_test.go

@@ -55,6 +55,32 @@ func TestSingleNode(t *testing.T) {
 		}
 		}
 		t.Fatalf("Set 2 failed with %s %s %v", result.Key, result.Value, result.TTL)
 		t.Fatalf("Set 2 failed with %s %s %v", result.Key, result.Value, result.TTL)
 	}
 	}
+
+	// Add a test-and-set test
+
+	// First, we'll test we can change the value if we get it write
+	result, match, err := c.TestAndSet("foo", "bar", "foobar", 100)
+
+	if err != nil || result.Key != "/foo" || result.Value != "foobar" || result.PrevValue != "bar" || result.TTL != 99 || !match {
+		if err != nil {
+			t.Fatal(err)
+		}
+		t.Fatalf("Set 3 failed with %s %s %v", result.Key, result.Value, result.TTL)
+	}
+
+	// Next, we'll make sure we can't set it without the correct prior value
+	_, _, err = c.TestAndSet("foo", "bar", "foofoo", 100)
+
+	if err == nil {
+		t.Fatalf("Set 4 expecting error when setting key with incorrect previous value")
+	}
+
+	// Finally, we'll make sure a blank previous value still counts as a test-and-set and still has to match
+	_, _, err = c.TestAndSet("foo", "", "barbar", 100)
+
+	if err == nil {
+		t.Fatalf("Set 5 expecting error when setting key with blank (incorrect) previous value")
+	}
 }
 }
 
 
 // TestInternalVersionFail will ensure that etcd does not come up if the internal raft
 // TestInternalVersionFail will ensure that etcd does not come up if the internal raft

+ 3 - 0
raft_handlers.go

@@ -42,6 +42,9 @@ func AppendEntriesHttpHandler(w http.ResponseWriter, req *http.Request) {
 
 
 	if err == nil {
 	if err == nil {
 		debugf("[recv] POST %s/log/append [%d]", r.url, len(aereq.Entries))
 		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 {
 		if resp := r.AppendEntries(aereq); resp != nil {
 			w.WriteHeader(http.StatusOK)
 			w.WriteHeader(http.StatusOK)
 			json.NewEncoder(w).Encode(resp)
 			json.NewEncoder(w).Encode(resp)

+ 51 - 11
raft_server.go

@@ -6,23 +6,26 @@ import (
 	"encoding/binary"
 	"encoding/binary"
 	"encoding/json"
 	"encoding/json"
 	"fmt"
 	"fmt"
-	etcdErr "github.com/coreos/etcd/error"
-	"github.com/coreos/go-raft"
 	"io/ioutil"
 	"io/ioutil"
 	"net/http"
 	"net/http"
 	"net/url"
 	"net/url"
 	"time"
 	"time"
+
+	etcdErr "github.com/coreos/etcd/error"
+	"github.com/coreos/go-raft"
 )
 )
 
 
 type raftServer struct {
 type raftServer struct {
 	*raft.Server
 	*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
 var r *raftServer
@@ -45,6 +48,16 @@ func newRaftServer(name string, url string, listenHost string, tlsConf *TLSConfi
 		listenHost: listenHost,
 		listenHost: listenHost,
 		tlsConf:    tlsConf,
 		tlsConf:    tlsConf,
 		tlsInfo:    tlsInfo,
 		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)
 		ok := joinCluster(cluster)
 		if !ok {
 		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)
 		debugf("%s restart as a follower", r.name)
@@ -136,7 +149,7 @@ func startAsFollower() {
 
 
 // Start to listen and response raft command
 // Start to listen and response raft command
 func (r *raftServer) startTransport(scheme string, tlsConf tls.Config) {
 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()
 	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)
 	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
 // Register commands to raft server
 func registerCommands() {
 func registerCommands() {
 	raft.RegisterCommand(&JoinCommand{})
 	raft.RegisterCommand(&JoinCommand{})

+ 195 - 0
raft_stats.go

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

+ 2 - 1
store/store.go

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

+ 1 - 1
third_party/code.google.com/p/goprotobuf/protoc-gen-go/generator/generator.go

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

+ 23 - 26
third_party/github.com/coreos/go-raft/server.go

@@ -927,26 +927,25 @@ func (s *Server) processRequestVoteRequest(req *RequestVoteRequest) (*RequestVot
 // Adds a peer to the server.
 // 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))
 	s.debugln("server.peer.add: ", name, len(s.peers))
-	defer s.writeConf()
+
 	// Do not allow peers to be added twice.
 	// Do not allow peers to be added twice.
 	if s.peers[name] != nil {
 	if s.peers[name] != nil {
 		return nil
 		return nil
 	}
 	}
 
 
 	// Skip the Peer if it has the same name as the Server
 	// 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)
 
 
-	peer := newPeer(s, name, connectiongString, s.heartbeatTimeout)
+		if s.State() == Leader {
+			peer.startHeartbeat()
+		}
 
 
-	if s.State() == Leader {
-		peer.startHeartbeat()
+		s.peers[peer.Name] = peer
 	}
 	}
 
 
-	s.peers[peer.Name] = peer
-
-	s.debugln("server.peer.conf.write: ", name)
+	// Write the configuration to file.
+	s.writeConf()
 
 
 	return nil
 	return nil
 }
 }
@@ -955,26 +954,24 @@ func (s *Server) AddPeer(name string, connectiongString string) error {
 func (s *Server) RemovePeer(name string) error {
 func (s *Server) RemovePeer(name string) error {
 	s.debugln("server.peer.remove: ", name, len(s.peers))
 	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)
+		}
 
 
-	// Stop peer and remove it.
-	if s.State() == Leader {
-		peer.stopHeartbeat(true)
+		delete(s.peers, name)
 	}
 	}
 
 
-	delete(s.peers, name)
+	// Write the configuration to file.
+	s.writeConf()
 
 
 	return nil
 	return nil
 }
 }

+ 1 - 2
third_party/github.com/coreos/go-raft/snapshot.go

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

+ 23 - 1
transporter.go

@@ -5,11 +5,12 @@ import (
 	"crypto/tls"
 	"crypto/tls"
 	"encoding/json"
 	"encoding/json"
 	"fmt"
 	"fmt"
-	"github.com/coreos/go-raft"
 	"io"
 	"io"
 	"net"
 	"net"
 	"net/http"
 	"net/http"
 	"time"
 	"time"
+
+	"github.com/coreos/go-raft"
 )
 )
 
 
 // Transporter layer for communication between raft nodes
 // 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 {
 func (t *transporter) SendAppendEntriesRequest(server *raft.Server, peer *raft.Peer, req *raft.AppendEntriesRequest) *raft.AppendEntriesResponse {
 	var aersp *raft.AppendEntriesResponse
 	var aersp *raft.AppendEntriesResponse
 	var b bytes.Buffer
 	var b bytes.Buffer
+
 	json.NewEncoder(&b).Encode(req)
 	json.NewEncoder(&b).Encode(req)
 
 
+	size := b.Len()
+
+	r.serverStats.SendAppendReq(size)
+
 	u, _ := nameToRaftURL(peer.Name)
 	u, _ := nameToRaftURL(peer.Name)
+
 	debugf("Send LogEntries to %s ", u)
 	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)
 	resp, err := t.Post(fmt.Sprintf("%s/log/append", u), &b)
 
 
+	end := time.Now()
+
 	if err != nil {
 	if err != nil {
 		debugf("Cannot send AppendEntriesRequest to %s: %s", u, err)
 		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 {
 	if resp != nil {
 		defer resp.Body.Close()
 		defer resp.Body.Close()
 		aersp = &raft.AppendEntriesResponse{}
 		aersp = &raft.AppendEntriesResponse{}