Browse Source

Merge pull request #55 from xiangli-cmu/master

Update
Xiang Li 12 years ago
parent
commit
a790fbc975
12 changed files with 486 additions and 71 deletions
  1. 14 0
      client_handlers.go
  2. 1 1
      command.go
  3. 16 5
      etcd.go
  4. 20 0
      etcd_long_test.go
  5. 4 4
      store/keyword_test.go
  6. 91 8
      store/store.go
  7. 90 4
      store/store_test.go
  8. 21 0
      store/test.go
  9. 46 16
      store/tree.go
  10. 159 20
      store/tree_store_test.go
  11. 2 13
      store/watcher_test.go
  12. 22 0
      test.go

+ 14 - 0
client_handlers.go

@@ -324,6 +324,20 @@ func WatchHttpHandler(w http.ResponseWriter, req *http.Request) {
 
 
 }
 }
 
 
+// TestHandler
+func TestHttpHandler(w http.ResponseWriter, req *http.Request) {
+	testType := req.URL.Path[len("/test/"):]
+
+	if testType == "speed" {
+		directSet()
+		w.WriteHeader(http.StatusOK)
+		w.Write([]byte("speed test success"))
+		return
+	}
+
+	w.WriteHeader(http.StatusBadRequest)
+}
+
 // Convert string duration to time format
 // Convert string duration to time format
 func durationToExpireTime(strDuration string) (time.Time, error) {
 func durationToExpireTime(strDuration string) (time.Time, error) {
 	if strDuration != "" {
 	if strDuration != "" {

+ 1 - 1
command.go

@@ -102,7 +102,7 @@ func (c *WatchCommand) Apply(server *raft.Server) (interface{}, error) {
 	res := <-watcher.C
 	res := <-watcher.C
 
 
 	if res == nil {
 	if res == nil {
-		return nil, fmt.Errorf("watcher is cleared")
+		return nil, fmt.Errorf("Clearing watch")
 	}
 	}
 
 
 	return json.Marshal(res)
 	return json.Marshal(res)

+ 16 - 5
etcd.go

@@ -14,8 +14,10 @@ import (
 	"io/ioutil"
 	"io/ioutil"
 	"net"
 	"net"
 	"net/http"
 	"net/http"
+	"net/url"
 	"os"
 	"os"
 	"os/signal"
 	"os/signal"
+	"path"
 	"runtime/pprof"
 	"runtime/pprof"
 	"strings"
 	"strings"
 	"time"
 	"time"
@@ -267,9 +269,6 @@ func startRaft(securityType int) {
 
 
 	raftServer.Start()
 	raftServer.Start()
 
 
-	// start to response to raft requests
-	go startRaftTransport(info.RaftPort, securityType)
-
 	if raftServer.IsLogEmpty() {
 	if raftServer.IsLogEmpty() {
 
 
 		// start as a leader in a new cluster
 		// start as a leader in a new cluster
@@ -340,6 +339,9 @@ func startRaft(securityType int) {
 		go raftServer.Snapshot()
 		go raftServer.Snapshot()
 	}
 	}
 
 
+	// start to response to raft requests
+	go startRaftTransport(info.RaftPort, securityType)
+
 }
 }
 
 
 // Create transporter using by raft server
 // Create transporter using by raft server
@@ -437,6 +439,7 @@ func startClientTransport(port int, st int) {
 	http.HandleFunc("/machines", MachinesHttpHandler)
 	http.HandleFunc("/machines", MachinesHttpHandler)
 	http.HandleFunc("/", VersionHttpHandler)
 	http.HandleFunc("/", VersionHttpHandler)
 	http.HandleFunc("/stats", StatsHttpHandler)
 	http.HandleFunc("/stats", StatsHttpHandler)
+	http.HandleFunc("/test/", TestHttpHandler)
 
 
 	switch st {
 	switch st {
 
 
@@ -628,11 +631,19 @@ func joinCluster(s *raft.Server, serverName string) error {
 				return nil
 				return nil
 			}
 			}
 			if resp.StatusCode == http.StatusTemporaryRedirect {
 			if resp.StatusCode == http.StatusTemporaryRedirect {
+
 				address := resp.Header.Get("Location")
 				address := resp.Header.Get("Location")
-				debugf("Leader is %s", address)
 				debugf("Send Join Request to %s", address)
 				debugf("Send Join Request to %s", address)
+				u, err := url.Parse(address)
+
+				if err != nil {
+					return fmt.Errorf("Unable to join: %s", err.Error())
+				}
+
 				json.NewEncoder(&b).Encode(command)
 				json.NewEncoder(&b).Encode(command)
-				resp, err = t.Post(fmt.Sprintf("%s/join", address), &b)
+
+				resp, err = t.Post(path.Join(u.Host, u.Path), &b)
+
 			} else if resp.StatusCode == http.StatusBadRequest {
 			} else if resp.StatusCode == http.StatusBadRequest {
 				debug("Reach max number machines in the cluster")
 				debug("Reach max number machines in the cluster")
 				return fmt.Errorf(errors[103])
 				return fmt.Errorf(errors[103])

+ 20 - 0
etcd_long_test.go

@@ -3,6 +3,7 @@ package main
 import (
 import (
 	"fmt"
 	"fmt"
 	"math/rand"
 	"math/rand"
+	"net/http"
 	"os"
 	"os"
 	"strconv"
 	"strconv"
 	"strings"
 	"strings"
@@ -119,3 +120,22 @@ func TestKillRandom(t *testing.T) {
 	<-leaderChan
 	<-leaderChan
 
 
 }
 }
+
+func BenchmarkEtcdDirectCall(b *testing.B) {
+	procAttr := new(os.ProcAttr)
+	procAttr.Files = []*os.File{nil, os.Stdout, os.Stderr}
+
+	clusterSize := 3
+	_, etcds, _ := createCluster(clusterSize, procAttr)
+
+	defer destroyCluster(etcds)
+
+	time.Sleep(time.Second)
+
+	b.ResetTimer()
+	for i := 0; i < b.N; i++ {
+		resp, _ := http.Get("http://0.0.0.0:4001/test/speed")
+		resp.Body.Close()
+	}
+
+}

+ 4 - 4
store/keyword_test.go

@@ -7,25 +7,25 @@ import (
 func TestKeywords(t *testing.T) {
 func TestKeywords(t *testing.T) {
 	keyword := CheckKeyword("_etcd")
 	keyword := CheckKeyword("_etcd")
 	if !keyword {
 	if !keyword {
-		t.Fatal("machines should be keyword")
+		t.Fatal("_etcd should be keyword")
 	}
 	}
 
 
 	keyword = CheckKeyword("/_etcd")
 	keyword = CheckKeyword("/_etcd")
 
 
 	if !keyword {
 	if !keyword {
-		t.Fatal("/machines should be keyword")
+		t.Fatal("/_etcd should be keyword")
 	}
 	}
 
 
 	keyword = CheckKeyword("/_etcd/")
 	keyword = CheckKeyword("/_etcd/")
 
 
 	if !keyword {
 	if !keyword {
-		t.Fatal("/machines/ contains keyword prefix")
+		t.Fatal("/_etcd/ contains keyword prefix")
 	}
 	}
 
 
 	keyword = CheckKeyword("/_etcd/node1")
 	keyword = CheckKeyword("/_etcd/node1")
 
 
 	if !keyword {
 	if !keyword {
-		t.Fatal("/machines/* contains keyword prefix")
+		t.Fatal("/_etcd/* contains keyword prefix")
 	}
 	}
 
 
 	keyword = CheckKeyword("/nokeyword/_etcd/node1")
 	keyword = CheckKeyword("/nokeyword/_etcd/node1")

+ 91 - 8
store/store.go

@@ -5,6 +5,7 @@ import (
 	"fmt"
 	"fmt"
 	"path"
 	"path"
 	"strconv"
 	"strconv"
+	"sync"
 	"time"
 	"time"
 )
 )
 
 
@@ -20,6 +21,14 @@ type Store struct {
 	// key-value store structure
 	// key-value store structure
 	Tree *tree
 	Tree *tree
 
 
+	// This mutex protects everything except add watcher member.
+	// Add watch member does not depend on the current state of the store.
+	// And watch will return when other protected function is called and reach
+	// the watching condition.
+	// It is needed so that clone() can atomically replicate the Store
+	// and do the log snapshot in a go routine.
+	mutex sync.Mutex
+
 	// WatcherHub is where we register all the clients
 	// WatcherHub is where we register all the clients
 	// who issue a watch request
 	// who issue a watch request
 	watcher *WatcherHub
 	watcher *WatcherHub
@@ -136,9 +145,16 @@ func (s *Store) SetMessager(messager *chan string) {
 	s.messager = messager
 	s.messager = messager
 }
 }
 
 
-// Set the key to value with expiration time
 func (s *Store) Set(key string, value string, expireTime time.Time, index uint64) ([]byte, error) {
 func (s *Store) Set(key string, value string, expireTime time.Time, index uint64) ([]byte, error) {
+	s.mutex.Lock()
+	defer s.mutex.Unlock()
+
+	return s.internalSet(key, value, expireTime, index)
+
+}
 
 
+// Set the key to value with expiration time
+func (s *Store) internalSet(key string, value string, expireTime time.Time, index uint64) ([]byte, error) {
 	//Update index
 	//Update index
 	s.Index = index
 	s.Index = index
 
 
@@ -161,7 +177,7 @@ func (s *Store) Set(key string, value string, expireTime time.Time, index uint64
 	// the key may be expired, we should not add the node
 	// the key may be expired, we should not add the node
 	// also if the node exist, we need to delete the node
 	// also if the node exist, we need to delete the node
 	if isExpire && expireTime.Sub(time.Now()) < 0 {
 	if isExpire && expireTime.Sub(time.Now()) < 0 {
-		return s.Delete(key, index)
+		return s.internalDelete(key, index)
 	}
 	}
 
 
 	var TTL int64
 	var TTL int64
@@ -290,6 +306,9 @@ func (s *Store) internalGet(key string) *Response {
 // If key is a file return the file
 // If key is a file return the file
 // If key is a directory reuturn an array of files
 // If key is a directory reuturn an array of files
 func (s *Store) Get(key string) ([]byte, error) {
 func (s *Store) Get(key string) ([]byte, error) {
+	s.mutex.Lock()
+	defer s.mutex.Unlock()
+
 	resps, err := s.RawGet(key)
 	resps, err := s.RawGet(key)
 
 
 	if err != nil {
 	if err != nil {
@@ -309,9 +328,36 @@ func (s *Store) RawGet(key string) ([]*Response, error) {
 
 
 	key = path.Clean("/" + key)
 	key = path.Clean("/" + key)
 
 
-	nodes, keys, dirs, ok := s.Tree.list(key)
+	nodes, keys, ok := s.Tree.list(key)
 
 
 	if ok {
 	if ok {
+
+		node, ok := nodes.(*Node)
+
+		if ok {
+			resps := make([]*Response, 1)
+
+			isExpire := !node.ExpireTime.Equal(PERMANENT)
+
+			resps[0] = &Response{
+				Action: "GET",
+				Index:  s.Index,
+				Key:    key,
+				Value:  node.Value,
+			}
+
+			// Update ttl
+			if isExpire {
+				TTL := int64(node.ExpireTime.Sub(time.Now()) / time.Second)
+				resps[0].Expiration = &node.ExpireTime
+				resps[0].TTL = TTL
+			}
+
+			return resps, nil
+		}
+
+		nodes, _ := nodes.([]*Node)
+
 		resps := make([]*Response, len(nodes))
 		resps := make([]*Response, len(nodes))
 		for i := 0; i < len(nodes); i++ {
 		for i := 0; i < len(nodes); i++ {
 
 
@@ -326,7 +372,7 @@ func (s *Store) RawGet(key string) ([]*Response, error) {
 				Key:    path.Join(key, keys[i]),
 				Key:    path.Join(key, keys[i]),
 			}
 			}
 
 
-			if !dirs[i] {
+			if len(nodes[i].Value) != 0 {
 				resps[i].Value = nodes[i].Value
 				resps[i].Value = nodes[i].Value
 			} else {
 			} else {
 				resps[i].Dir = true
 				resps[i].Dir = true
@@ -348,8 +394,14 @@ func (s *Store) RawGet(key string) ([]*Response, error) {
 	return nil, err
 	return nil, err
 }
 }
 
 
-// Delete the key
 func (s *Store) Delete(key string, index uint64) ([]byte, error) {
 func (s *Store) Delete(key string, index uint64) ([]byte, error) {
+	s.mutex.Lock()
+	defer s.mutex.Unlock()
+	return s.internalDelete(key, index)
+}
+
+// Delete the key
+func (s *Store) internalDelete(key string, index uint64) ([]byte, error) {
 
 
 	// Update stats
 	// Update stats
 	s.BasicStats.Deletes++
 	s.BasicStats.Deletes++
@@ -404,6 +456,9 @@ func (s *Store) Delete(key string, index uint64) ([]byte, error) {
 
 
 // Set the value of the key to the value if the given prevValue is equal to the value of the key
 // Set the value of the key to the value if the given prevValue is equal to the value of the key
 func (s *Store) TestAndSet(key string, prevValue string, value string, expireTime time.Time, index uint64) ([]byte, error) {
 func (s *Store) TestAndSet(key string, prevValue string, value string, expireTime time.Time, index uint64) ([]byte, error) {
+	s.mutex.Lock()
+	defer s.mutex.Unlock()
+
 	// Update stats
 	// Update stats
 	s.BasicStats.TestAndSets++
 	s.BasicStats.TestAndSets++
 
 
@@ -417,7 +472,7 @@ func (s *Store) TestAndSet(key string, prevValue string, value string, expireTim
 	if resp.Value == prevValue {
 	if resp.Value == prevValue {
 
 
 		// If test success, do set
 		// If test success, do set
-		return s.Set(key, value, expireTime, index)
+		return s.internalSet(key, value, expireTime, index)
 	} else {
 	} else {
 
 
 		// If fails, return err
 		// If fails, return err
@@ -452,6 +507,7 @@ func (s *Store) monitorExpiration(key string, update chan time.Time, expireTime
 				return
 				return
 
 
 			} else {
 			} else {
+				s.mutex.Lock()
 
 
 				s.Tree.delete(key)
 				s.Tree.delete(key)
 
 
@@ -462,6 +518,7 @@ func (s *Store) monitorExpiration(key string, update chan time.Time, expireTime
 					Expiration: &node.ExpireTime,
 					Expiration: &node.ExpireTime,
 					Index:      s.Index,
 					Index:      s.Index,
 				}
 				}
+				s.mutex.Unlock()
 
 
 				msg, err := json.Marshal(resp)
 				msg, err := json.Marshal(resp)
 
 
@@ -520,9 +577,34 @@ func (s *Store) addToResponseMap(index uint64, resp *Response) {
 	}
 	}
 }
 }
 
 
+func (s *Store) clone() *Store {
+	newStore := &Store{
+		ResponseMaxSize:    s.ResponseMaxSize,
+		ResponseCurrSize:   s.ResponseCurrSize,
+		ResponseStartIndex: s.ResponseStartIndex,
+		Index:              s.Index,
+		BasicStats:         s.BasicStats,
+	}
+
+	newStore.Tree = s.Tree.clone()
+	newStore.ResponseMap = make(map[string]*Response)
+
+	for index, response := range s.ResponseMap {
+		newStore.ResponseMap[index] = response
+	}
+
+	return newStore
+}
+
 // Save the current state of the storage system
 // Save the current state of the storage system
 func (s *Store) Save() ([]byte, error) {
 func (s *Store) Save() ([]byte, error) {
-	b, err := json.Marshal(s)
+	// first we clone the store
+	// json is very slow, we cannot hold the lock for such a long time
+	s.mutex.Lock()
+	cloneStore := s.clone()
+	s.mutex.Unlock()
+
+	b, err := json.Marshal(cloneStore)
 	if err != nil {
 	if err != nil {
 		fmt.Println(err)
 		fmt.Println(err)
 		return nil, err
 		return nil, err
@@ -532,7 +614,8 @@ func (s *Store) Save() ([]byte, error) {
 
 
 // Recovery the state of the stroage system from a previous state
 // Recovery the state of the stroage system from a previous state
 func (s *Store) Recovery(state []byte) error {
 func (s *Store) Recovery(state []byte) error {
-
+	s.mutex.Lock()
+	defer s.mutex.Unlock()
 	// we need to stop all the current watchers
 	// we need to stop all the current watchers
 	// recovery will clear watcherHub
 	// recovery will clear watcherHub
 	s.watcher.stopWatchers()
 	s.watcher.stopWatchers()

+ 90 - 4
store/store_test.go

@@ -2,7 +2,6 @@ package store
 
 
 import (
 import (
 	"encoding/json"
 	"encoding/json"
-	"fmt"
 	"testing"
 	"testing"
 	"time"
 	"time"
 )
 )
@@ -70,9 +69,6 @@ func TestSaveAndRecovery(t *testing.T) {
 }
 }
 
 
 func TestExpire(t *testing.T) {
 func TestExpire(t *testing.T) {
-	fmt.Println(time.Now())
-	fmt.Println("TEST EXPIRE")
-
 	// test expire
 	// test expire
 	s := CreateStore(100)
 	s := CreateStore(100)
 	s.Set("foo", "bar", time.Now().Add(time.Second*1), 0)
 	s.Set("foo", "bar", time.Now().Add(time.Second*1), 0)
@@ -134,3 +130,93 @@ func TestExpire(t *testing.T) {
 	}
 	}
 
 
 }
 }
+
+func BenchmarkStoreSet(b *testing.B) {
+	s := CreateStore(100)
+
+	keys := GenKeys(10000, 5)
+
+	b.ResetTimer()
+	for i := 0; i < b.N; i++ {
+
+		for i, key := range keys {
+			s.Set(key, "barbarbarbarbar", time.Unix(0, 0), uint64(i))
+		}
+
+		s = CreateStore(100)
+	}
+}
+
+func BenchmarkStoreGet(b *testing.B) {
+	s := CreateStore(100)
+
+	keys := GenKeys(10000, 5)
+
+	for i, key := range keys {
+		s.Set(key, "barbarbarbarbar", time.Unix(0, 0), uint64(i))
+	}
+
+	b.ResetTimer()
+	for i := 0; i < b.N; i++ {
+
+		for _, key := range keys {
+			s.Get(key)
+		}
+
+	}
+}
+
+func BenchmarkStoreSnapshotCopy(b *testing.B) {
+	s := CreateStore(100)
+
+	keys := GenKeys(10000, 5)
+
+	for i, key := range keys {
+		s.Set(key, "barbarbarbarbar", time.Unix(0, 0), uint64(i))
+	}
+
+	var state []byte
+
+	b.ResetTimer()
+	for i := 0; i < b.N; i++ {
+		s.clone()
+	}
+	b.SetBytes(int64(len(state)))
+}
+
+func BenchmarkSnapshotSaveJson(b *testing.B) {
+	s := CreateStore(100)
+
+	keys := GenKeys(10000, 5)
+
+	for i, key := range keys {
+		s.Set(key, "barbarbarbarbar", time.Unix(0, 0), uint64(i))
+	}
+
+	var state []byte
+
+	b.ResetTimer()
+	for i := 0; i < b.N; i++ {
+		state, _ = s.Save()
+	}
+	b.SetBytes(int64(len(state)))
+}
+
+func BenchmarkSnapshotRecovery(b *testing.B) {
+	s := CreateStore(100)
+
+	keys := GenKeys(10000, 5)
+
+	for i, key := range keys {
+		s.Set(key, "barbarbarbarbar", time.Unix(0, 0), uint64(i))
+	}
+
+	state, _ := s.Save()
+
+	b.ResetTimer()
+	for i := 0; i < b.N; i++ {
+		newStore := CreateStore(100)
+		newStore.Recovery(state)
+	}
+	b.SetBytes(int64(len(state)))
+}

+ 21 - 0
store/test.go

@@ -0,0 +1,21 @@
+package store
+
+import (
+	"math/rand"
+	"strconv"
+)
+
+// GenKeys randomly generate num of keys with max depth
+func GenKeys(num int, depth int) []string {
+	keys := make([]string, num)
+	for i := 0; i < num; i++ {
+
+		keys[i] = "/foo/"
+		depth := rand.Intn(depth) + 1
+
+		for j := 0; j < depth; j++ {
+			keys[i] += "/" + strconv.Itoa(rand.Int())
+		}
+	}
+	return keys
+}

+ 46 - 16
store/tree.go

@@ -4,6 +4,7 @@ import (
 	"path"
 	"path"
 	"sort"
 	"sort"
 	"strings"
 	"strings"
+	"time"
 )
 )
 
 
 //------------------------------------------------------------------------------
 //------------------------------------------------------------------------------
@@ -41,7 +42,7 @@ func (s tnWithKeySlice) Swap(i, j int)      { s[i], s[j] = s[j], s[i] }
 // CONSTANT VARIABLE
 // CONSTANT VARIABLE
 
 
 // Represent an empty node
 // Represent an empty node
-var emptyNode = Node{".", PERMANENT, nil}
+var emptyNode = Node{"", PERMANENT, nil}
 
 
 //------------------------------------------------------------------------------
 //------------------------------------------------------------------------------
 //
 //
@@ -158,35 +159,28 @@ func (t *tree) get(key string) (Node, bool) {
 }
 }
 
 
 // get the internalNode of the key
 // get the internalNode of the key
-func (t *tree) list(directory string) ([]Node, []string, []bool, bool) {
+func (t *tree) list(directory string) (interface{}, []string, bool) {
 	treeNode, ok := t.internalGet(directory)
 	treeNode, ok := t.internalGet(directory)
 
 
 	if !ok {
 	if !ok {
-		return nil, nil, nil, ok
+		return nil, nil, ok
+
 	} else {
 	} else {
 		if !treeNode.Dir {
 		if !treeNode.Dir {
-			nodes := make([]Node, 1)
-			nodes[0] = treeNode.InternalNode
-			return nodes, make([]string, 1), make([]bool, 1), true
+			return &treeNode.InternalNode, nil, ok
 		}
 		}
 		length := len(treeNode.NodeMap)
 		length := len(treeNode.NodeMap)
-		nodes := make([]Node, length)
+		nodes := make([]*Node, length)
 		keys := make([]string, length)
 		keys := make([]string, length)
-		dirs := make([]bool, length)
-		i := 0
 
 
+		i := 0
 		for key, node := range treeNode.NodeMap {
 		for key, node := range treeNode.NodeMap {
-			nodes[i] = node.InternalNode
+			nodes[i] = &node.InternalNode
 			keys[i] = key
 			keys[i] = key
-			if node.Dir {
-				dirs[i] = true
-			} else {
-				dirs[i] = false
-			}
 			i++
 			i++
 		}
 		}
 
 
-		return nodes, keys, dirs, ok
+		return nodes, keys, ok
 	}
 	}
 }
 }
 
 
@@ -223,6 +217,42 @@ func (t *tree) traverse(f func(string, *Node), sort bool) {
 	}
 	}
 }
 }
 
 
+// clone() will return a deep cloned tree
+func (t *tree) clone() *tree {
+	newTree := new(tree)
+	newTree.Root = &treeNode{
+		Node{
+			"/",
+			time.Unix(0, 0),
+			nil,
+		},
+		true,
+		make(map[string]*treeNode),
+	}
+	recursiveClone(t.Root, newTree.Root)
+	return newTree
+}
+
+// recursiveClone is a helper function for clone()
+func recursiveClone(tnSrc *treeNode, tnDes *treeNode) {
+	if !tnSrc.Dir {
+		tnDes.InternalNode = tnSrc.InternalNode
+		return
+
+	} else {
+		tnDes.InternalNode = tnSrc.InternalNode
+		tnDes.Dir = true
+		tnDes.NodeMap = make(map[string]*treeNode)
+
+		for key, tn := range tnSrc.NodeMap {
+			newTn := new(treeNode)
+			recursiveClone(tn, newTn)
+			tnDes.NodeMap[key] = newTn
+		}
+
+	}
+}
+
 // deep first search to traverse the tree
 // deep first search to traverse the tree
 // apply the func f to each internal node
 // apply the func f to each internal node
 func dfs(key string, t *treeNode, f func(string, *Node)) {
 func dfs(key string, t *treeNode, f func(string, *Node)) {

+ 159 - 20
store/tree_store_test.go

@@ -12,18 +12,18 @@ func TestStoreGet(t *testing.T) {
 
 
 	ts := &tree{
 	ts := &tree{
 		&treeNode{
 		&treeNode{
-			CreateTestNode("/"),
+			NewTestNode("/"),
 			true,
 			true,
 			make(map[string]*treeNode),
 			make(map[string]*treeNode),
 		},
 		},
 	}
 	}
 
 
 	// create key
 	// create key
-	ts.set("/foo", CreateTestNode("bar"))
+	ts.set("/foo", NewTestNode("bar"))
 	// change value
 	// change value
-	ts.set("/foo", CreateTestNode("barbar"))
+	ts.set("/foo", NewTestNode("barbar"))
 	// create key
 	// create key
-	ts.set("/hello/foo", CreateTestNode("barbarbar"))
+	ts.set("/hello/foo", NewTestNode("barbarbar"))
 	treeNode, ok := ts.get("/foo")
 	treeNode, ok := ts.get("/foo")
 
 
 	if !ok {
 	if !ok {
@@ -43,7 +43,7 @@ func TestStoreGet(t *testing.T) {
 	}
 	}
 
 
 	// create a key under other key
 	// create a key under other key
-	ok = ts.set("/foo/foo", CreateTestNode("bar"))
+	ok = ts.set("/foo/foo", NewTestNode("bar"))
 	if ok {
 	if ok {
 		t.Fatalf("shoud not add key under a exisiting key")
 		t.Fatalf("shoud not add key under a exisiting key")
 	}
 	}
@@ -61,35 +61,31 @@ func TestStoreGet(t *testing.T) {
 	}
 	}
 
 
 	// test list
 	// test list
-	ts.set("/hello/fooo", CreateTestNode("barbarbar"))
-	ts.set("/hello/foooo/foo", CreateTestNode("barbarbar"))
+	ts.set("/hello/fooo", NewTestNode("barbarbar"))
+	ts.set("/hello/foooo/foo", NewTestNode("barbarbar"))
 
 
-	nodes, keys, dirs, ok := ts.list("/hello")
+	nodes, keys, ok := ts.list("/hello")
 
 
 	if !ok {
 	if !ok {
 		t.Fatalf("cannot list!")
 		t.Fatalf("cannot list!")
 	} else {
 	} else {
+		nodes, _ := nodes.([]*Node)
 		length := len(nodes)
 		length := len(nodes)
 
 
 		for i := 0; i < length; i++ {
 		for i := 0; i < length; i++ {
-			fmt.Println(keys[i], "=", nodes[i].Value, "[", dirs[i], "]")
+			fmt.Println(keys[i], "=", nodes[i].Value)
 		}
 		}
 	}
 	}
 
 
-	// speed test
+	keys = GenKeys(100, 10)
+
 	for i := 0; i < 100; i++ {
 	for i := 0; i < 100; i++ {
-		key := "/"
-		depth := rand.Intn(10)
-		for j := 0; j < depth; j++ {
-			key += "/" + strconv.Itoa(rand.Int()%10)
-		}
 		value := strconv.Itoa(rand.Int())
 		value := strconv.Itoa(rand.Int())
-		ts.set(key, CreateTestNode(value))
-		treeNode, ok := ts.get(key)
+		ts.set(keys[i], NewTestNode(value))
+		treeNode, ok := ts.get(keys[i])
 
 
 		if !ok {
 		if !ok {
 			continue
 			continue
-			//t.Fatalf("Expect to get node, but not")
 		}
 		}
 		if treeNode.Value != value {
 		if treeNode.Value != value {
 			t.Fatalf("Expect value %s, but got %s", value, treeNode.Value)
 			t.Fatalf("Expect value %s, but got %s", value, treeNode.Value)
@@ -99,10 +95,153 @@ func TestStoreGet(t *testing.T) {
 	ts.traverse(f, true)
 	ts.traverse(f, true)
 }
 }
 
 
+func TestTreeClone(t *testing.T) {
+	keys := GenKeys(10000, 10)
+
+	ts := &tree{
+		&treeNode{
+			NewTestNode("/"),
+			true,
+			make(map[string]*treeNode),
+		},
+	}
+
+	backTs := &tree{
+		&treeNode{
+			NewTestNode("/"),
+			true,
+			make(map[string]*treeNode),
+		},
+	}
+
+	// generate the first tree
+	for _, key := range keys {
+		value := strconv.Itoa(rand.Int())
+		ts.set(key, NewTestNode(value))
+		backTs.set(key, NewTestNode(value))
+	}
+
+	copyTs := ts.clone()
+
+	// test if they are identical
+	copyTs.traverse(ts.contain, false)
+
+	// remove all the keys from first tree
+	for _, key := range keys {
+		ts.delete(key)
+	}
+
+	// test if they are identical
+	// make sure changes in the first tree will affect the copy one
+	copyTs.traverse(backTs.contain, false)
+
+}
+
+func BenchmarkTreeStoreSet(b *testing.B) {
+
+	keys := GenKeys(10000, 10)
+
+	b.ResetTimer()
+	for i := 0; i < b.N; i++ {
+
+		ts := &tree{
+			&treeNode{
+				NewTestNode("/"),
+				true,
+				make(map[string]*treeNode),
+			},
+		}
+
+		for _, key := range keys {
+			value := strconv.Itoa(rand.Int())
+			ts.set(key, NewTestNode(value))
+		}
+	}
+}
+
+func BenchmarkTreeStoreGet(b *testing.B) {
+
+	keys := GenKeys(10000, 10)
+
+	ts := &tree{
+		&treeNode{
+			NewTestNode("/"),
+			true,
+			make(map[string]*treeNode),
+		},
+	}
+
+	for _, key := range keys {
+		value := strconv.Itoa(rand.Int())
+		ts.set(key, NewTestNode(value))
+	}
+
+	b.ResetTimer()
+	for i := 0; i < b.N; i++ {
+		for _, key := range keys {
+			ts.get(key)
+		}
+	}
+}
+
+func BenchmarkTreeStoreCopy(b *testing.B) {
+	keys := GenKeys(10000, 10)
+
+	ts := &tree{
+		&treeNode{
+			NewTestNode("/"),
+			true,
+			make(map[string]*treeNode),
+		},
+	}
+
+	for _, key := range keys {
+		value := strconv.Itoa(rand.Int())
+		ts.set(key, NewTestNode(value))
+	}
+
+	b.ResetTimer()
+	for i := 0; i < b.N; i++ {
+		ts.clone()
+	}
+}
+
+func BenchmarkTreeStoreList(b *testing.B) {
+
+	keys := GenKeys(10000, 10)
+
+	ts := &tree{
+		&treeNode{
+			NewTestNode("/"),
+			true,
+			make(map[string]*treeNode),
+		},
+	}
+
+	for _, key := range keys {
+		value := strconv.Itoa(rand.Int())
+		ts.set(key, NewTestNode(value))
+	}
+
+	b.ResetTimer()
+	for i := 0; i < b.N; i++ {
+		for _, key := range keys {
+			ts.list(key)
+		}
+	}
+}
+
+func (t *tree) contain(key string, node *Node) {
+	_, ok := t.get(key)
+	if !ok {
+		panic("tree do not contain the given key")
+	}
+}
+
 func f(key string, n *Node) {
 func f(key string, n *Node) {
-	fmt.Println(key, "=", n.Value)
+	return
 }
 }
 
 
-func CreateTestNode(value string) Node {
+func NewTestNode(value string) Node {
 	return Node{value, time.Unix(0, 0), nil}
 	return Node{value, time.Unix(0, 0), nil}
 }
 }

+ 2 - 13
store/watcher_test.go

@@ -1,8 +1,6 @@
 package store
 package store
 
 
 import (
 import (
-	"math/rand"
-	"strconv"
 	"testing"
 	"testing"
 	"time"
 	"time"
 )
 )
@@ -62,16 +60,7 @@ func TestWatch(t *testing.T) {
 func BenchmarkWatch(b *testing.B) {
 func BenchmarkWatch(b *testing.B) {
 	s := CreateStore(100)
 	s := CreateStore(100)
 
 
-	key := make([]string, 10000)
-	for i := 0; i < 10000; i++ {
-
-		key[i] = "/foo/"
-		depth := rand.Intn(10)
-
-		for j := 0; j < depth; j++ {
-			key[i] += "/" + strconv.Itoa(rand.Int()%10)
-		}
-	}
+	keys := GenKeys(10000, 10)
 
 
 	b.ResetTimer()
 	b.ResetTimer()
 	for i := 0; i < b.N; i++ {
 	for i := 0; i < b.N; i++ {
@@ -80,7 +69,7 @@ func BenchmarkWatch(b *testing.B) {
 			// create a new watcher
 			// create a new watcher
 			watchers[i] = NewWatcher()
 			watchers[i] = NewWatcher()
 			// add to the watchers list
 			// add to the watchers list
-			s.AddWatcher(key[i], watchers[i], 0)
+			s.AddWatcher(keys[i], watchers[i], 0)
 		}
 		}
 
 
 		s.watcher.stopWatchers()
 		s.watcher.stopWatchers()

+ 22 - 0
test.go

@@ -166,6 +166,28 @@ func getLeader(addr string) (string, error) {
 
 
 }
 }
 
 
+func directSet() {
+	c := make(chan bool, 1000)
+	for i := 0; i < 1000; i++ {
+		go send(c)
+	}
+
+	for i := 0; i < 1000; i++ {
+		<-c
+	}
+}
+
+func send(c chan bool) {
+	for i := 0; i < 10; i++ {
+		command := &SetCommand{}
+		command.Key = "foo"
+		command.Value = "bar"
+		command.ExpireTime = time.Unix(0, 0)
+		raftServer.Do(command)
+	}
+	c <- true
+}
+
 // Dial with timeout
 // Dial with timeout
 func dialTimeoutFast(network, addr string) (net.Conn, error) {
 func dialTimeoutFast(network, addr string) (net.Conn, error) {
 	return net.DialTimeout(network, addr, time.Millisecond*10)
 	return net.DialTimeout(network, addr, time.Millisecond*10)