Browse Source

Merge remote-tracking branch 'xiangli-cmu/newStore' into 0.2

Brandon Philips 12 years ago
parent
commit
0facc24016

+ 58 - 0
Documentation/errorcode.md

@@ -0,0 +1,58 @@
+Error Code
+======
+
+This document describes the error code in **Etcd** project.
+
+It's categorized into four groups:
+
+- Command Related Error
+- Post Form Related Error
+- Raft Related Error
+- Etcd Related Error
+
+Error code corresponding strerror
+------
+
+    const (
+        EcodeKeyNotFound    = 100
+        EcodeTestFailed     = 101
+        EcodeNotFile        = 102
+        EcodeNoMoreMachine  = 103
+        EcodeNotDir         = 104
+        EcodeNodeExist      = 105
+        EcodeKeyIsPreserved = 106
+
+        EcodeValueRequired     = 200
+        EcodePrevValueRequired = 201
+        EcodeTTLNaN            = 202
+        EcodeIndexNaN          = 203
+
+        EcodeRaftInternal = 300
+        EcodeLeaderElect  = 301
+
+        EcodeWatcherCleared = 400
+        EcodeEventIndexCleared = 401
+    )
+
+    // command related errors
+    errors[100] = "Key Not Found"
+    errors[101] = "Test Failed" //test and set
+    errors[102] = "Not A File"
+    errors[103] = "Reached the max number of machines in the cluster"
+    errors[104] = "Not A Directory"
+    errors[105] = "Already exists" // create
+    errors[106] = "The prefix of given key is a keyword in etcd"
+
+    // Post form related errors
+    errors[200] = "Value is Required in POST form"
+    errors[201] = "PrevValue is Required in POST form"
+    errors[202] = "The given TTL in POST form is not a number"
+    errors[203] = "The given index in POST form is not a number"
+
+    // raft related errors
+    errors[300] = "Raft Internal Error"
+    errors[301] = "During Leader Election"
+
+    // etcd related errors
+    errors[400] = "watcher is cleared due to etcd recovery"
+    errors[401] = "The event in requested index is outdated and cleared"

+ 101 - 0
Documentation/etcd-file-system.md

@@ -0,0 +1,101 @@
+#Etcd File System
+
+## Structure
+[TODO]
+![alt text](./img/etcd_fs_structure.jpg "etcd file system structure")
+
+## Node
+In **Etcd**, the **Node** is the rudimentary element constructing the whole.
+Currently **Etcd** file system is comprised in a Unix-like way of files and directories, and they are two kinds of nodes different in:
+
+- **File Node** has data associated with it.
+- **Directory Node** has children nodes associated with it.
+
+Besides the file and directory difference, all nodes have common attributes and operations as follows:
+
+### Attributes:
+- **Expiration Time** [optional]
+
+  The node will be deleted when it expires.
+
+- **ACL**
+
+  The path of access control list of the node.
+
+### Operation:
+- **Get** (path, recursive, sorted)
+
+  Get the content of the node
+    - If the node is a file, the data of the file will be returned.
+    - If the node is a directory, the child nodes of the directory will be returned.
+    - If recursive is true, it will recursively get the nodes of the directory.
+    - If sorted is true, the result will be sorted based on the path.
+
+- **Create** (path, value[optional], ttl [optional])
+
+  Create a file. Create operation will help to create intermediate directories with no expiration time.
+    - If the file already exists, create will fail.
+    - If the value is given, set will create a file.
+    - If the value is not given, set will crate a directory.
+    - If ttl is given, the node will be deleted when it expires.
+
+- **Update** (path, value[optional], ttl [optional])
+
+  Update the content of the node.
+    - If the value is given, the value of the key will be updated.
+    - If ttl is given, the expiration time of the node will be updated.
+
+- **Delete** (path, recursive)
+
+  Delete the node of given path.
+    - If the node is a directory:
+    - If recursive is true, the operation will delete all nodes under the directory.
+    - If recursive is false, error will be returned.
+
+- **TestAndSet** (path, prevValue [prevIndex], value, ttl)
+
+  Atomic *test and set* value to a file. If test succeeds, this operation will change the previous value of the file to the given value.
+    - If the prevValue is given, it will test against previous value of 
+    the node.
+    - If the prevValue is empty, it will test if the node is not existing.
+    - If the prevValue is not empty, it will test if the prevValue is equal to the current value of the file.
+    - If the prevIndex is given, it will test if the create/last modified index of the node is equal to prevIndex.
+
+- **Renew** (path, ttl)
+
+  Set the node's expiration time to (current time + ttl)
+
+## ACL
+
+### Theory
+Etcd exports a Unix-like file system interface consisting of files and directories, collectively called nodes.
+Each node has various meta-data, including three names of access control lists used to control reading, writing and changing (change ACL names for the node).
+
+We are storing the ACL names for nodes under a special *ACL* directory.
+Each node has ACL name corresponding to one file within *ACL* dir.
+Unless overridden, a node naturally inherits the ACL names of its parent directory on creation.
+
+For each ACL name, it has three children: *R (Reading)*, *W (Writing)*, *C (Changing)*
+
+Each permission is also a node. Under the node it contains the users who have this permission for the file refering to this ACL name.
+
+### Example
+[TODO]
+### Diagram
+[TODO]
+
+### Interface
+
+Testing permissions:
+
+- (node *Node) get_perm()
+- (node *Node) has_perm(perm string, user string)
+
+Setting/Changing permissions:
+
+- (node *Node) set_perm(perm string)
+- (node *Node) change_ACLname(aclname string)
+
+
+## User Group
+[TODO]

BIN
Documentation/img/etcd_fs_structure.jpg


+ 85 - 32
command.go

@@ -2,7 +2,6 @@ package main
 
 import (
 	"encoding/binary"
-	"encoding/json"
 	"fmt"
 	"os"
 	"path"
@@ -25,29 +24,63 @@ type Command interface {
 	Apply(server *raft.Server) (interface{}, error)
 }
 
-// Set command
-type SetCommand struct {
+// Create command
+type CreateCommand struct {
+	Key               string    `json:"key"`
+	Value             string    `json:"value"`
+	ExpireTime        time.Time `json:"expireTime"`
+	IncrementalSuffix bool      `json:"incrementalSuffix"`
+	Force             bool      `json:"force"`
+}
+
+// The name of the create command in the log
+func (c *CreateCommand) CommandName() string {
+	return commandName("create")
+}
+
+// Create node
+func (c *CreateCommand) Apply(server *raft.Server) (interface{}, error) {
+	e, err := etcdStore.Create(c.Key, c.Value, c.IncrementalSuffix, c.Force, c.ExpireTime, server.CommitIndex(), server.Term())
+
+	if err != nil {
+		debug(err)
+		return nil, err
+	}
+
+	return e, nil
+}
+
+// Update command
+type UpdateCommand struct {
 	Key        string    `json:"key"`
 	Value      string    `json:"value"`
 	ExpireTime time.Time `json:"expireTime"`
 }
 
-// The name of the set command in the log
-func (c *SetCommand) CommandName() string {
-	return commandName("set")
+// The name of the update command in the log
+func (c *UpdateCommand) CommandName() string {
+	return commandName("update")
 }
 
-// Set the key-value pair
-func (c *SetCommand) Apply(server *raft.Server) (interface{}, error) {
-	return etcdStore.Set(c.Key, c.Value, c.ExpireTime, server.CommitIndex())
+// Update node
+func (c *UpdateCommand) Apply(server *raft.Server) (interface{}, error) {
+	e, err := etcdStore.Update(c.Key, c.Value, c.ExpireTime, server.CommitIndex(), server.Term())
+
+	if err != nil {
+		debug(err)
+		return nil, err
+	}
+
+	return e, nil
 }
 
 // TestAndSet command
 type TestAndSetCommand struct {
 	Key        string    `json:"key"`
 	Value      string    `json:"value"`
-	PrevValue  string    `json: prevValue`
 	ExpireTime time.Time `json:"expireTime"`
+	PrevValue  string    `json: prevValue`
+	PrevIndex  uint64    `json: prevValue`
 }
 
 // The name of the testAndSet command in the log
@@ -57,12 +90,22 @@ 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) {
-	return etcdStore.TestAndSet(c.Key, c.PrevValue, c.Value, c.ExpireTime, server.CommitIndex())
+	e, err := etcdStore.TestAndSet(c.Key, c.PrevValue, c.PrevIndex,
+		c.Value, c.ExpireTime, server.CommitIndex(), server.Term())
+
+	if err != nil {
+		debug(err)
+		return nil, err
+	}
+
+	return e, nil
 }
 
 // Get command
 type GetCommand struct {
-	Key string `json:"key"`
+	Key       string `json:"key"`
+	Recursive bool   `json:"recursive"`
+	Sorted    bool   `json:"sorted"`
 }
 
 // The name of the get command in the log
@@ -72,12 +115,20 @@ func (c *GetCommand) CommandName() string {
 
 // Get the value of key
 func (c *GetCommand) Apply(server *raft.Server) (interface{}, error) {
-	return etcdStore.Get(c.Key)
+	e, err := etcdStore.Get(c.Key, c.Recursive, c.Sorted, server.CommitIndex(), server.Term())
+
+	if err != nil {
+		debug(err)
+		return nil, err
+	}
+
+	return e, nil
 }
 
 // Delete command
 type DeleteCommand struct {
-	Key string `json:"key"`
+	Key       string `json:"key"`
+	Recursive bool   `json:"recursive"`
 }
 
 // The name of the delete command in the log
@@ -87,13 +138,21 @@ func (c *DeleteCommand) CommandName() string {
 
 // Delete the key
 func (c *DeleteCommand) Apply(server *raft.Server) (interface{}, error) {
-	return etcdStore.Delete(c.Key, server.CommitIndex())
+	e, err := etcdStore.Delete(c.Key, c.Recursive, server.CommitIndex(), server.Term())
+
+	if err != nil {
+		debug(err)
+		return nil, err
+	}
+
+	return e, nil
 }
 
 // Watch command
 type WatchCommand struct {
 	Key        string `json:"key"`
 	SinceIndex uint64 `json:"sinceIndex"`
+	Recursive  bool   `json:"recursive"`
 }
 
 // The name of the watch command in the log
@@ -102,20 +161,15 @@ func (c *WatchCommand) CommandName() string {
 }
 
 func (c *WatchCommand) Apply(server *raft.Server) (interface{}, error) {
-	// create a new watcher
-	watcher := store.NewWatcher()
-
-	// add to the watchers list
-	etcdStore.AddWatcher(c.Key, watcher, c.SinceIndex)
+	eventChan, err := etcdStore.Watch(c.Key, c.Recursive, c.SinceIndex, server.CommitIndex(), server.Term())
 
-	// wait for the notification for any changing
-	res := <-watcher.C
-
-	if res == nil {
-		return nil, fmt.Errorf("Clearing watch")
+	if err != nil {
+		return nil, err
 	}
 
-	return json.Marshal(res)
+	e := <-eventChan
+
+	return e, nil
 }
 
 // JoinCommand
@@ -144,12 +198,12 @@ func (c *JoinCommand) CommandName() string {
 func (c *JoinCommand) Apply(raftServer *raft.Server) (interface{}, error) {
 
 	// check if the join command is from a previous machine, who lost all its previous log.
-	response, _ := etcdStore.RawGet(path.Join("_etcd/machines", c.Name))
+	e, _ := etcdStore.Get(path.Join("/_etcd/machines", c.Name), false, false, raftServer.CommitIndex(), raftServer.Term())
 
 	b := make([]byte, 8)
 	binary.PutUvarint(b, raftServer.CommitIndex())
 
-	if response != nil {
+	if e != nil {
 		return b, nil
 	}
 
@@ -157,7 +211,7 @@ func (c *JoinCommand) Apply(raftServer *raft.Server) (interface{}, error) {
 	num := machineNum()
 	if num == maxClusterSize {
 		debug("Reject join request from ", c.Name)
-		return []byte{0}, etcdErr.NewError(103, "")
+		return []byte{0}, etcdErr.NewError(etcdErr.EcodeNoMoreMachine, "", raftServer.CommitIndex(), raftServer.Term())
 	}
 
 	addNameToURL(c.Name, c.RaftVersion, c.RaftURL, c.EtcdURL)
@@ -168,7 +222,7 @@ func (c *JoinCommand) Apply(raftServer *raft.Server) (interface{}, error) {
 	// add machine in etcd storage
 	key := path.Join("_etcd/machines", c.Name)
 	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.Create(key, value, false, false, store.Permanent, raftServer.CommitIndex(), raftServer.Term())
 
 	// add peer stats
 	if c.Name != r.Name() {
@@ -199,8 +253,7 @@ func (c *RemoveCommand) Apply(raftServer *raft.Server) (interface{}, error) {
 	// remove machine in etcd storage
 	key := path.Join("_etcd/machines", c.Name)
 
-	_, err := etcdStore.Delete(key, raftServer.CommitIndex())
-
+	_, err := etcdStore.Delete(key, false, raftServer.CommitIndex(), raftServer.Term())
 	// delete from stats
 	delete(r.followersStats.Followers, c.Name)
 

+ 46 - 17
error/error.go

@@ -2,37 +2,60 @@ package error
 
 import (
 	"encoding/json"
+	"fmt"
 	"net/http"
 )
 
 var errors map[int]string
 
-const ()
+const (
+	EcodeKeyNotFound    = 100
+	EcodeTestFailed     = 101
+	EcodeNotFile        = 102
+	EcodeNoMoreMachine  = 103
+	EcodeNotDir         = 104
+	EcodeNodeExist      = 105
+	EcodeKeyIsPreserved = 106
+
+	EcodeValueRequired      = 200
+	EcodePrevValueRequired  = 201
+	EcodeTTLNaN             = 202
+	EcodeIndexNaN           = 203
+	EcodeValueOrTTLRequired = 204
+
+	EcodeRaftInternal = 300
+	EcodeLeaderElect  = 301
+
+	EcodeWatcherCleared    = 400
+	EcodeEventIndexCleared = 401
+)
 
 func init() {
 	errors = make(map[int]string)
 
 	// command related errors
-	errors[100] = "Key Not Found"
-	errors[101] = "The given PrevValue is not equal to the value of the key"
-	errors[102] = "Not A File"
-	errors[103] = "Reached the max number of machines in the cluster"
+	errors[EcodeKeyNotFound] = "Key Not Found"
+	errors[EcodeTestFailed] = "Test Failed" //test and set
+	errors[EcodeNotFile] = "Not A File"
+	errors[EcodeNoMoreMachine] = "Reached the max number of machines in the cluster"
+	errors[EcodeNotDir] = "Not A Directory"
+	errors[EcodeNodeExist] = "Already exists" // create
+	errors[EcodeKeyIsPreserved] = "The prefix of given key is a keyword in etcd"
 
 	// Post form related errors
-	errors[200] = "Value is Required in POST form"
-	errors[201] = "PrevValue is Required in POST form"
-	errors[202] = "The given TTL in POST form is not a number"
-	errors[203] = "The given index in POST form is not a number"
+	errors[EcodeValueRequired] = "Value is Required in POST form"
+	errors[EcodePrevValueRequired] = "PrevValue is Required in POST form"
+	errors[EcodeTTLNaN] = "The given TTL in POST form is not a number"
+	errors[EcodeIndexNaN] = "The given index in POST form is not a number"
+	errors[EcodeValueOrTTLRequired] = "Value or TTL is required in POST form"
 
 	// raft related errors
-	errors[300] = "Raft Internal Error"
-	errors[301] = "During Leader Election"
-
-	// keyword
-	errors[400] = "The prefix of the given key is a keyword in etcd"
+	errors[EcodeRaftInternal] = "Raft Internal Error"
+	errors[EcodeLeaderElect] = "During Leader Election"
 
 	// etcd related errors
-	errors[500] = "watcher is cleared due to etcd recovery"
+	errors[EcodeWatcherCleared] = "watcher is cleared due to etcd recovery"
+	errors[EcodeEventIndexCleared] = "The event in requested index is outdated and cleared"
 
 }
 
@@ -40,13 +63,17 @@ type Error struct {
 	ErrorCode int    `json:"errorCode"`
 	Message   string `json:"message"`
 	Cause     string `json:"cause,omitempty"`
+	Index     uint64 `json:"index"`
+	Term      uint64 `json:"term"`
 }
 
-func NewError(errorCode int, cause string) Error {
-	return Error{
+func NewError(errorCode int, cause string, index uint64, term uint64) *Error {
+	return &Error{
 		ErrorCode: errorCode,
 		Message:   errors[errorCode],
 		Cause:     cause,
+		Index:     index,
+		Term:      term,
 	}
 }
 
@@ -65,6 +92,8 @@ func (e Error) toJsonString() string {
 }
 
 func (e Error) Write(w http.ResponseWriter) {
+	w.Header().Add("X-Etcd-Index", fmt.Sprint(e.Index))
+	w.Header().Add("X-Etcd-Term", fmt.Sprint(e.Term))
 	// 3xx is reft internal error
 	if e.ErrorCode/100 == 3 {
 		http.Error(w, e.toJsonString(), http.StatusInternalServerError)

+ 2 - 1
etcd.go

@@ -199,7 +199,8 @@ func main() {
 	info := getInfo(dirPath)
 
 	// Create etcd key-value store
-	etcdStore = store.CreateStore(maxSize)
+	etcdStore = store.New()
+
 	snapConf = newSnapshotConf()
 
 	// Create etcd and raft server

+ 248 - 0
etcd_handler_v1.go

@@ -0,0 +1,248 @@
+package main
+
+import (
+	"encoding/json"
+	"net/http"
+	"strconv"
+
+	etcdErr "github.com/coreos/etcd/error"
+	"github.com/coreos/etcd/store"
+	"github.com/coreos/go-raft"
+)
+
+//-------------------------------------------------------------------
+// Handlers to handle etcd-store related request via etcd url
+//-------------------------------------------------------------------
+// Multiplex GET/POST/DELETE request to corresponding handlers
+func MultiplexerV1(w http.ResponseWriter, req *http.Request) error {
+
+	switch req.Method {
+	case "GET":
+		return GetHttpHandlerV1(w, req)
+	case "POST":
+		return SetHttpHandlerV1(w, req)
+	case "PUT":
+		return SetHttpHandlerV1(w, req)
+	case "DELETE":
+		return DeleteHttpHandlerV1(w, req)
+	default:
+		w.WriteHeader(http.StatusMethodNotAllowed)
+		return nil
+	}
+}
+
+//--------------------------------------
+// State sensitive handlers
+// Set/Delete will dispatch to leader
+//--------------------------------------
+
+// Set Command Handler
+func SetHttpHandlerV1(w http.ResponseWriter, req *http.Request) error {
+	key := req.URL.Path[len("/v1/keys/"):]
+
+	debugf("[recv] POST %v/v1/keys/%s [%s]", e.url, key, req.RemoteAddr)
+
+	req.ParseForm()
+
+	value := req.Form.Get("value")
+
+	if len(value) == 0 {
+		return etcdErr.NewError(200, "Set", store.UndefIndex, store.UndefTerm)
+	}
+
+	strDuration := req.Form.Get("ttl")
+
+	expireTime, err := durationToExpireTime(strDuration)
+
+	if err != nil {
+		return etcdErr.NewError(202, "Set", store.UndefIndex, store.UndefTerm)
+	}
+
+	if prevValueArr, ok := req.Form["prevValue"]; ok && len(prevValueArr) > 0 {
+		command := &TestAndSetCommand{
+			Key:        key,
+			Value:      value,
+			PrevValue:  prevValueArr[0],
+			ExpireTime: expireTime,
+		}
+
+		return dispatchEtcdCommandV1(command, w, req)
+
+	} else {
+		command := &CreateCommand{
+			Key:        key,
+			Value:      value,
+			ExpireTime: expireTime,
+			Force:      true,
+		}
+
+		return dispatchEtcdCommandV1(command, w, req)
+	}
+}
+
+// Delete Handler
+func DeleteHttpHandlerV1(w http.ResponseWriter, req *http.Request) error {
+	key := req.URL.Path[len("/v1/keys/"):]
+
+	debugf("[recv] DELETE %v/v1/keys/%s [%s]", e.url, key, req.RemoteAddr)
+
+	command := &DeleteCommand{
+		Key: key,
+	}
+
+	return dispatchEtcdCommandV1(command, w, req)
+}
+
+//--------------------------------------
+// State non-sensitive handlers
+// will not dispatch to leader
+// TODO: add sensitive version for these
+// command?
+//--------------------------------------
+
+// Get Handler
+func GetHttpHandlerV1(w http.ResponseWriter, req *http.Request) error {
+	key := req.URL.Path[len("/v1/keys/"):]
+
+	debugf("[recv] GET %s/v1/keys/%s [%s]", e.url, key, req.RemoteAddr)
+
+	command := &GetCommand{
+		Key: key,
+	}
+
+	if event, err := command.Apply(r.Server); err != nil {
+		return err
+	} else {
+		event, _ := event.(*store.Event)
+
+		response := eventToResponse(event)
+		bytes, _ := json.Marshal(response)
+
+		w.WriteHeader(http.StatusOK)
+
+		w.Write(bytes)
+
+		return nil
+	}
+
+}
+
+// Watch handler
+func WatchHttpHandlerV1(w http.ResponseWriter, req *http.Request) error {
+	key := req.URL.Path[len("/v1/watch/"):]
+
+	command := &WatchCommand{
+		Key: key,
+	}
+
+	if req.Method == "GET" {
+		debugf("[recv] GET %s/watch/%s [%s]", e.url, key, req.RemoteAddr)
+		command.SinceIndex = 0
+
+	} else if req.Method == "POST" {
+		// watch from a specific index
+
+		debugf("[recv] POST %s/watch/%s [%s]", e.url, key, req.RemoteAddr)
+		content := req.FormValue("index")
+
+		sinceIndex, err := strconv.ParseUint(string(content), 10, 64)
+		if err != nil {
+			return etcdErr.NewError(203, "Watch From Index", store.UndefIndex, store.UndefTerm)
+		}
+		command.SinceIndex = sinceIndex
+
+	} else {
+		w.WriteHeader(http.StatusMethodNotAllowed)
+		return nil
+	}
+
+	if event, err := command.Apply(r.Server); err != nil {
+		return etcdErr.NewError(500, key, store.UndefIndex, store.UndefTerm)
+	} else {
+		event, _ := event.(*store.Event)
+
+		response := eventToResponse(event)
+		bytes, _ := json.Marshal(response)
+
+		w.WriteHeader(http.StatusOK)
+
+		w.Write(bytes)
+		return nil
+	}
+
+}
+
+// Dispatch the command to leader
+func dispatchEtcdCommandV1(c Command, w http.ResponseWriter, req *http.Request) error {
+	return dispatchV1(c, w, req, nameToEtcdURL)
+}
+
+func dispatchV1(c Command, w http.ResponseWriter, req *http.Request, toURL func(name string) (string, bool)) error {
+	if r.State() == raft.Leader {
+		if event, err := r.Do(c); err != nil {
+			return err
+		} else {
+			if event == nil {
+				return etcdErr.NewError(300, "Empty result from raft", store.UndefIndex, store.UndefTerm)
+			}
+
+			event, _ := event.(*store.Event)
+
+			response := eventToResponse(event)
+			bytes, _ := json.Marshal(response)
+
+			w.WriteHeader(http.StatusOK)
+			w.Write(bytes)
+			return nil
+
+		}
+
+	} else {
+		leader := r.Leader()
+		// current no leader
+		if leader == "" {
+			return etcdErr.NewError(300, "", store.UndefIndex, store.UndefTerm)
+		}
+		url, _ := toURL(leader)
+
+		redirect(url, w, req)
+
+		return nil
+	}
+}
+
+func eventToResponse(event *store.Event) interface{} {
+	if !event.Dir {
+		response := &store.Response{
+			Action:     event.Action,
+			Key:        event.Key,
+			Value:      event.Value,
+			PrevValue:  event.PrevValue,
+			Index:      event.Index,
+			TTL:        event.TTL,
+			Expiration: event.Expiration,
+		}
+
+		if response.Action == store.Create || response.Action == store.Update {
+			response.Action = "set"
+			if response.PrevValue == "" {
+				response.NewKey = true
+			}
+		}
+
+		return response
+	} else {
+		responses := make([]*store.Response, len(event.KVPairs))
+
+		for i, kv := range event.KVPairs {
+			responses[i] = &store.Response{
+				Action: event.Action,
+				Key:    kv.Key,
+				Value:  kv.Value,
+				Dir:    kv.Dir,
+				Index:  event.Index,
+			}
+		}
+		return responses
+	}
+}

+ 146 - 107
etcd_handlers.go

@@ -1,6 +1,7 @@
 package main
 
 import (
+	"encoding/json"
 	"fmt"
 	"net/http"
 	"strconv"
@@ -19,11 +20,15 @@ func NewEtcdMuxer() *http.ServeMux {
 	// external commands
 	etcdMux := http.NewServeMux()
 	etcdMux.Handle("/"+version+"/keys/", errorHandler(Multiplexer))
-	etcdMux.Handle("/"+version+"/watch/", errorHandler(WatchHttpHandler))
 	etcdMux.Handle("/"+version+"/leader", errorHandler(LeaderHttpHandler))
 	etcdMux.Handle("/"+version+"/machines", errorHandler(MachinesHttpHandler))
 	etcdMux.Handle("/"+version+"/stats/", errorHandler(StatsHttpHandler))
 	etcdMux.Handle("/version", errorHandler(VersionHttpHandler))
+
+	etcdMux.Handle("/v1/keys/", errorHandler(MultiplexerV1))
+	etcdMux.Handle("/v1/leader", errorHandler(LeaderHttpHandler))
+	etcdMux.Handle("/v1/machines", errorHandler(MachinesHttpHandler))
+	etcdMux.Handle("/v1/stats/", errorHandler(StatsHttpHandler))
 	etcdMux.HandleFunc("/test/", TestHttpHandler)
 	return etcdMux
 }
@@ -51,8 +56,8 @@ func addCorsHeader(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 etcdErr, ok := e.(etcdErr.Error); ok {
-			debug("Return error: ", etcdErr.Error())
+		if etcdErr, ok := e.(*etcdErr.Error); ok {
+			debug("Return error: ", (*etcdErr).Error())
 			etcdErr.Write(w)
 		} else {
 			http.Error(w, e.Error(), http.StatusInternalServerError)
@@ -67,15 +72,17 @@ func Multiplexer(w http.ResponseWriter, req *http.Request) error {
 	case "GET":
 		return GetHttpHandler(w, req)
 	case "POST":
-		return SetHttpHandler(w, req)
+		return CreateHttpHandler(w, req)
 	case "PUT":
-		return SetHttpHandler(w, req)
+		return UpdateHttpHandler(w, req)
 	case "DELETE":
 		return DeleteHttpHandler(w, req)
 	default:
 		w.WriteHeader(http.StatusMethodNotAllowed)
 		return nil
 	}
+
+	return nil
 }
 
 //--------------------------------------
@@ -83,102 +90,118 @@ func Multiplexer(w http.ResponseWriter, req *http.Request) error {
 // Set/Delete will dispatch to leader
 //--------------------------------------
 
-// Set Command Handler
-func SetHttpHandler(w http.ResponseWriter, req *http.Request) error {
-	key := req.URL.Path[len("/v1/keys/"):]
+func CreateHttpHandler(w http.ResponseWriter, req *http.Request) error {
+	key := getNodePath(req.URL.Path)
+
+	debugf("recv.post[%v] [%v%v]\n", req.RemoteAddr, req.Host, req.URL)
+
+	value := req.FormValue("value")
+
+	expireTime, err := durationToExpireTime(req.FormValue("ttl"))
+
+	if err != nil {
+		return etcdErr.NewError(etcdErr.EcodeTTLNaN, "Create", store.UndefIndex, store.UndefTerm)
+	}
+
+	command := &CreateCommand{
+		Key:        key,
+		Value:      value,
+		ExpireTime: expireTime,
+	}
 
-	if store.CheckKeyword(key) {
-		return etcdErr.NewError(400, "Set")
+	if req.FormValue("incremental") == "true" {
+		command.IncrementalSuffix = true
 	}
 
-	debugf("[recv] POST %v/v1/keys/%s [%s]", e.url, key, req.RemoteAddr)
+	return dispatchEtcdCommand(command, w, req)
+
+}
+
+func UpdateHttpHandler(w http.ResponseWriter, req *http.Request) error {
+	key := getNodePath(req.URL.Path)
+
+	debugf("recv.put[%v] [%v%v]\n", req.RemoteAddr, req.Host, req.URL)
 
 	req.ParseForm()
 
 	value := req.Form.Get("value")
 
-	if len(value) == 0 {
-		return etcdErr.NewError(200, "Set")
+	expireTime, err := durationToExpireTime(req.Form.Get("ttl"))
+
+	if err != nil {
+		return etcdErr.NewError(etcdErr.EcodeTTLNaN, "Update", store.UndefIndex, store.UndefTerm)
 	}
 
-	strDuration := req.Form.Get("ttl")
+	// update should give at least one option
+	if value == "" && expireTime.Sub(store.Permanent) == 0 {
+		return etcdErr.NewError(etcdErr.EcodeValueOrTTLRequired, "Update", store.UndefIndex, store.UndefTerm)
+	}
 
-	expireTime, err := durationToExpireTime(strDuration)
+	prevValue, valueOk := req.Form["prevValue"]
 
-	if err != nil {
-		return etcdErr.NewError(202, "Set")
-	}
+	prevIndexStr, indexOk := req.Form["prevIndex"]
 
-	if prevValueArr, ok := req.Form["prevValue"]; ok && len(prevValueArr) > 0 {
-		command := &TestAndSetCommand{
+	if !valueOk && !indexOk { // update without test
+		command := &UpdateCommand{
 			Key:        key,
 			Value:      value,
-			PrevValue:  prevValueArr[0],
 			ExpireTime: expireTime,
 		}
 
-		return dispatch(command, w, req, true)
+		return dispatchEtcdCommand(command, w, req)
 
-	} else {
-		command := &SetCommand{
-			Key:        key,
-			Value:      value,
-			ExpireTime: expireTime,
+	} else { // update with test
+		var prevIndex uint64
+
+		if indexOk {
+			prevIndex, err = strconv.ParseUint(prevIndexStr[0], 10, 64)
+
+			// bad previous index
+			if err != nil {
+				return etcdErr.NewError(etcdErr.EcodeIndexNaN, "Update", store.UndefIndex, store.UndefTerm)
+			}
+		} else {
+			prevIndex = 0
+		}
+
+		command := &TestAndSetCommand{
+			Key:       key,
+			Value:     value,
+			PrevValue: prevValue[0],
+			PrevIndex: prevIndex,
 		}
 
-		return dispatch(command, w, req, true)
+		return dispatchEtcdCommand(command, w, req)
 	}
+
 }
 
 // Delete Handler
 func DeleteHttpHandler(w http.ResponseWriter, req *http.Request) error {
-	key := req.URL.Path[len("/v1/keys/"):]
+	key := getNodePath(req.URL.Path)
 
-	debugf("[recv] DELETE %v/v1/keys/%s [%s]", e.url, key, req.RemoteAddr)
+	debugf("recv.delete[%v] [%v%v]\n", req.RemoteAddr, req.Host, req.URL)
 
 	command := &DeleteCommand{
 		Key: key,
 	}
 
-	return dispatch(command, w, req, true)
+	if req.FormValue("recursive") == "true" {
+		command.Recursive = true
+	}
+
+	return dispatchEtcdCommand(command, w, req)
 }
 
 // Dispatch the command to leader
-func dispatch(c Command, w http.ResponseWriter, req *http.Request, etcd bool) error {
-
-	if r.State() == raft.Leader {
-		if body, err := r.Do(c); err != nil {
-			return err
-		} else {
-			if body == nil {
-				return etcdErr.NewError(300, "Empty result from raft")
-			} else {
-				body, _ := body.([]byte)
-				w.WriteHeader(http.StatusOK)
-				w.Write(body)
-				return nil
-			}
-		}
-
-	} else {
-		leader := r.Leader()
-		// current no leader
-		if leader == "" {
-			return etcdErr.NewError(300, "")
-		}
-
-		redirect(leader, etcd, w, req)
-
-		return nil
-	}
-	return etcdErr.NewError(300, "")
+func dispatchEtcdCommand(c Command, w http.ResponseWriter, req *http.Request) error {
+	return dispatch(c, w, req, nameToEtcdURL)
 }
 
 //--------------------------------------
 // State non-sensitive handlers
-// will not dispatch to leader
-// TODO: add sensitive version for these
-// command?
+// command with consistent option will
+// still dispatch to the leader
 //--------------------------------------
 
 // Handler to return the current leader's raft address
@@ -189,9 +212,10 @@ func LeaderHttpHandler(w http.ResponseWriter, req *http.Request) error {
 		w.WriteHeader(http.StatusOK)
 		raftURL, _ := nameToRaftURL(leader)
 		w.Write([]byte(raftURL))
+
 		return nil
 	} else {
-		return etcdErr.NewError(301, "")
+		return etcdErr.NewError(etcdErr.EcodeLeaderElect, "", store.UndefIndex, store.UndefTerm)
 	}
 }
 
@@ -201,6 +225,7 @@ func MachinesHttpHandler(w http.ResponseWriter, req *http.Request) error {
 
 	w.WriteHeader(http.StatusOK)
 	w.Write([]byte(strings.Join(machines, ", ")))
+
 	return nil
 }
 
@@ -208,16 +233,17 @@ func MachinesHttpHandler(w http.ResponseWriter, req *http.Request) error {
 func VersionHttpHandler(w http.ResponseWriter, req *http.Request) error {
 	w.WriteHeader(http.StatusOK)
 	fmt.Fprintf(w, "etcd %s", releaseVersion)
+
 	return nil
 }
 
 // Handler to return the basic stats of etcd
 func StatsHttpHandler(w http.ResponseWriter, req *http.Request) error {
 	option := req.URL.Path[len("/v1/stats/"):]
+	w.WriteHeader(http.StatusOK)
 
 	switch option {
 	case "self":
-		w.WriteHeader(http.StatusOK)
 		w.Write(r.Stats())
 	case "leader":
 		if r.State() == raft.Leader {
@@ -226,76 +252,88 @@ func StatsHttpHandler(w http.ResponseWriter, req *http.Request) error {
 			leader := r.Leader()
 			// current no leader
 			if leader == "" {
-				return etcdErr.NewError(300, "")
+				return etcdErr.NewError(300, "", store.UndefIndex, store.UndefTerm)
 			}
-			redirect(leader, true, w, req)
+			hostname, _ := nameToEtcdURL(leader)
+			redirect(hostname, w, req)
 		}
 	case "store":
-		w.WriteHeader(http.StatusOK)
-		w.Write(etcdStore.Stats())
+		w.Write(etcdStore.JsonStats())
 	}
 
 	return nil
 }
 
-// Get Handler
 func GetHttpHandler(w http.ResponseWriter, req *http.Request) error {
-	key := req.URL.Path[len("/v1/keys/"):]
+	var err error
+	var event interface{}
+	debugf("recv.get[%v] [%v%v]\n", req.RemoteAddr, req.Host, req.URL)
 
-	debugf("[recv] GET %s/v1/keys/%s [%s]", e.url, key, req.RemoteAddr)
-
-	command := &GetCommand{
-		Key: key,
+	if req.FormValue("consistent") == "true" && r.State() != raft.Leader {
+		// help client to redirect the request to the current leader
+		leader := r.Leader()
+		hostname, _ := nameToEtcdURL(leader)
+		redirect(hostname, w, req)
+		return nil
 	}
 
-	if body, err := command.Apply(r.Server); err != nil {
-		return err
-	} else {
-		body, _ := body.([]byte)
-		w.WriteHeader(http.StatusOK)
-		w.Write(body)
+	key := getNodePath(req.URL.Path)
 
-		return nil
-	}
+	recursive := req.FormValue("recursive")
 
-}
+	if req.FormValue("wait") == "true" { // watch
+		command := &WatchCommand{
+			Key: key,
+		}
 
-// Watch handler
-func WatchHttpHandler(w http.ResponseWriter, req *http.Request) error {
-	key := req.URL.Path[len("/v1/watch/"):]
+		if recursive == "true" {
+			command.Recursive = true
+		}
 
-	command := &WatchCommand{
-		Key: key,
-	}
+		indexStr := req.FormValue("wait_index")
+		if indexStr != "" {
+			sinceIndex, err := strconv.ParseUint(indexStr, 10, 64)
+
+			if err != nil {
+				return etcdErr.NewError(etcdErr.EcodeIndexNaN, "Watch From Index", store.UndefIndex, store.UndefTerm)
+			}
 
-	if req.Method == "GET" {
-		debugf("[recv] GET %s/watch/%s [%s]", e.url, key, req.RemoteAddr)
-		command.SinceIndex = 0
+			command.SinceIndex = sinceIndex
+		}
 
-	} else if req.Method == "POST" {
-		// watch from a specific index
+		event, err = command.Apply(r.Server)
 
-		debugf("[recv] POST %s/watch/%s [%s]", e.url, key, req.RemoteAddr)
-		content := req.FormValue("index")
+	} else { //get
 
-		sinceIndex, err := strconv.ParseUint(string(content), 10, 64)
-		if err != nil {
-			return etcdErr.NewError(203, "Watch From Index")
+		command := &GetCommand{
+			Key: key,
 		}
-		command.SinceIndex = sinceIndex
 
-	} else {
-		w.WriteHeader(http.StatusMethodNotAllowed)
-		return nil
+		sorted := req.FormValue("sorted")
+		if sorted == "true" {
+			command.Sorted = true
+		}
+
+		if recursive == "true" {
+			command.Recursive = true
+		}
+
+		event, err = command.Apply(r.Server)
 	}
 
-	if body, err := command.Apply(r.Server); err != nil {
-		return etcdErr.NewError(500, key)
+	if err != nil {
+		return err
+
 	} else {
+		event, _ := event.(*store.Event)
+		bytes, _ := json.Marshal(event)
+
+		w.Header().Add("X-Etcd-Index", fmt.Sprint(event.Index))
+		w.Header().Add("X-Etcd-Term", fmt.Sprint(event.Term))
 		w.WriteHeader(http.StatusOK)
 
-		body, _ := body.([]byte)
-		w.Write(body)
+		w.Write(bytes)
+
 		return nil
 	}
 
@@ -309,6 +347,7 @@ func TestHttpHandler(w http.ResponseWriter, req *http.Request) {
 		directSet()
 		w.WriteHeader(http.StatusOK)
 		w.Write([]byte("speed test success"))
+
 		return
 	}
 

+ 3 - 3
etcd_test.go

@@ -49,7 +49,7 @@ func TestSingleNode(t *testing.T) {
 
 	result, err = c.Set("foo", "bar", 100)
 
-	if err != nil || result.Key != "/foo" || result.Value != "bar" || result.PrevValue != "bar" || result.TTL != 99 {
+	if err != nil || result.Key != "/foo" || result.Value != "bar" || result.PrevValue != "bar" || result.TTL != 100 {
 		if err != nil {
 			t.Fatal(err)
 		}
@@ -61,7 +61,7 @@ func TestSingleNode(t *testing.T) {
 	// 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 || result.Key != "/foo" || result.Value != "foobar" || result.PrevValue != "bar" || result.TTL != 100 || !match {
 		if err != nil {
 			t.Fatal(err)
 		}
@@ -223,7 +223,7 @@ func templateTestSimpleMultiNode(t *testing.T, tls bool) {
 
 	result, err = c.Set("foo", "bar", 100)
 
-	if err != nil || result.Key != "/foo" || result.Value != "bar" || result.PrevValue != "bar" || result.TTL != 99 {
+	if err != nil || result.Key != "/foo" || result.Value != "bar" || result.PrevValue != "bar" || result.TTL != 100 {
 		if err != nil {
 			t.Fatal(err)
 		}

+ 6 - 2
machines.go

@@ -2,9 +2,13 @@ package main
 
 // machineNum returns the number of machines in the cluster
 func machineNum() int {
-	response, _ := etcdStore.RawGet("_etcd/machines")
+	e, err := etcdStore.Get("/_etcd/machines", false, false, r.CommitIndex(), r.Term())
 
-	return len(response)
+	if err != nil {
+		return 0
+	}
+
+	return len(e.KVPairs)
 }
 
 // getMachines gets the current machines in the cluster

+ 7 - 3
name_url_map.go

@@ -49,16 +49,20 @@ func addNameToURL(name string, version string, raftURL string, etcdURL string) {
 }
 
 func readURL(nodeName string, urlName string) (string, bool) {
-	// if fails, try to recover from etcd storage
+	if nodeName == "" {
+		return "", false
+	}
+
+	// convert nodeName to url from etcd storage
 	key := path.Join("/_etcd/machines", nodeName)
 
-	resps, err := etcdStore.RawGet(key)
+	e, err := etcdStore.Get(key, false, false, r.CommitIndex(), r.Term())
 
 	if err != nil {
 		return "", false
 	}
 
-	m, err := url.ParseQuery(resps[0].Value)
+	m, err := url.ParseQuery(e.Value)
 
 	if err != nil {
 		panic("Failed to parse machines entry")

+ 6 - 3
raft_handlers.go

@@ -104,7 +104,7 @@ func JoinHttpHandler(w http.ResponseWriter, req *http.Request) error {
 
 	if err := decodeJsonRequest(req, command); err == nil {
 		debugf("Receive Join Request from %s", command.Name)
-		return dispatch(command, w, req, false)
+		return dispatchRaftCommand(command, w, req)
 	} else {
 		w.WriteHeader(http.StatusInternalServerError)
 		return nil
@@ -125,8 +125,7 @@ func RemoveHttpHandler(w http.ResponseWriter, req *http.Request) {
 
 	debugf("[recv] Remove Request [%s]", command.Name)
 
-	dispatch(command, w, req, false)
-
+	dispatchRaftCommand(command, w, req)
 }
 
 // Response to the name request
@@ -142,3 +141,7 @@ func RaftVersionHttpHandler(w http.ResponseWriter, req *http.Request) {
 	w.WriteHeader(http.StatusOK)
 	w.Write([]byte(r.version))
 }
+
+func dispatchRaftCommand(c Command, w http.ResponseWriter, req *http.Request) error {
+	return dispatch(c, w, req, nameToRaftURL)
+}

+ 3 - 1
raft_server.go

@@ -317,9 +317,11 @@ func (r *raftServer) PeerStats() []byte {
 func registerCommands() {
 	raft.RegisterCommand(&JoinCommand{})
 	raft.RegisterCommand(&RemoveCommand{})
-	raft.RegisterCommand(&SetCommand{})
 	raft.RegisterCommand(&GetCommand{})
 	raft.RegisterCommand(&DeleteCommand{})
 	raft.RegisterCommand(&WatchCommand{})
 	raft.RegisterCommand(&TestAndSetCommand{})
+
+	raft.RegisterCommand(&CreateCommand{})
+	raft.RegisterCommand(&UpdateCommand{})
 }

+ 5 - 5
snapshot.go

@@ -20,17 +20,17 @@ var snapConf *snapshotConf
 
 func newSnapshotConf() *snapshotConf {
 	// check snapshot every 3 seconds and the threshold is 20K
-	return &snapshotConf{time.Second * 3, etcdStore.TotalWrites(), 20 * 1000}
+	return &snapshotConf{time.Second * 3, 0, 20 * 1000}
 }
 
 func monitorSnapshot() {
 	for {
 		time.Sleep(snapConf.checkingInterval)
-		currentWrites := etcdStore.TotalWrites() - snapConf.lastWrites
-
-		if currentWrites > snapConf.writesThr {
+		//currentWrites := etcdStore.TotalWrites() - snapConf.lastWrites
+		currentWrites := 0
+		if uint64(currentWrites) > snapConf.writesThr {
 			r.TakeSnapshot()
-			snapConf.lastWrites = etcdStore.TotalWrites()
+			snapConf.lastWrites = 0
 		}
 	}
 }

+ 195 - 0
store/event.go

@@ -0,0 +1,195 @@
+package store
+
+import (
+	"fmt"
+	"strings"
+	"sync"
+	"time"
+
+	etcdErr "github.com/coreos/etcd/error"
+)
+
+const (
+	Get        = "get"
+	Create     = "create"
+	Update     = "update"
+	Delete     = "delete"
+	TestAndSet = "testAndSet"
+	Expire     = "expire"
+)
+
+const (
+	UndefIndex = 0
+	UndefTerm  = 0
+)
+
+type Event struct {
+	Action     string         `json:"action"`
+	Key        string         `json:"key, omitempty"`
+	Dir        bool           `json:"dir,omitempty"`
+	PrevValue  string         `json:"prevValue,omitempty"`
+	Value      string         `json:"value,omitempty"`
+	KVPairs    []KeyValuePair `json:"kvs,omitempty"`
+	Expiration *time.Time     `json:"expiration,omitempty"`
+	TTL        int64          `json:"ttl,omitempty"` // Time to live in second
+	// The command index of the raft machine when the command is executed
+	Index uint64 `json:"index"`
+	Term  uint64 `json:"term"`
+}
+
+// When user list a directory, we add all the node into key-value pair slice
+type KeyValuePair struct {
+	Key     string         `json:"key, omitempty"`
+	Value   string         `json:"value,omitempty"`
+	Dir     bool           `json:"dir,omitempty"`
+	KVPairs []KeyValuePair `json:"kvs,omitempty"`
+}
+
+// interfaces for sorting
+func (k KeyValuePair) Len() int {
+	return len(k.KVPairs)
+}
+
+func (k KeyValuePair) Less(i, j int) bool {
+	return k.KVPairs[i].Key < k.KVPairs[j].Key
+}
+
+func (k KeyValuePair) Swap(i, j int) {
+	k.KVPairs[i], k.KVPairs[j] = k.KVPairs[j], k.KVPairs[i]
+}
+
+func newEvent(action string, key string, index uint64, term uint64) *Event {
+	return &Event{
+		Action: action,
+		Key:    key,
+		Index:  index,
+		Term:   term,
+	}
+}
+
+type eventQueue struct {
+	Events   []*Event
+	Size     int
+	Front    int
+	Capacity int
+}
+
+func (eq *eventQueue) back() int {
+	return (eq.Front + eq.Size - 1 + eq.Capacity) % eq.Capacity
+}
+
+func (eq *eventQueue) insert(e *Event) {
+	index := (eq.back() + 1) % eq.Capacity
+
+	eq.Events[index] = e
+
+	if eq.Size == eq.Capacity { //dequeue
+		eq.Front = (index + 1) % eq.Capacity
+	} else {
+		eq.Size++
+	}
+
+}
+
+type EventHistory struct {
+	Queue      eventQueue
+	StartIndex uint64
+	LastIndex  uint64
+	LastTerm   uint64
+	DupCnt     uint64 // help to compute the watching point with duplicated indexes in the queue
+	rwl        sync.RWMutex
+}
+
+func newEventHistory(capacity int) *EventHistory {
+	return &EventHistory{
+		Queue: eventQueue{
+			Capacity: capacity,
+			Events:   make([]*Event, capacity),
+		},
+	}
+}
+
+// addEvent function adds event into the eventHistory
+func (eh *EventHistory) addEvent(e *Event) *Event {
+	eh.rwl.Lock()
+	defer eh.rwl.Unlock()
+
+	var duped uint64
+
+	if e.Index == UndefIndex {
+		e.Index = eh.LastIndex
+		e.Term = eh.LastTerm
+		duped = 1
+	}
+
+	eh.Queue.insert(e)
+
+	eh.LastIndex = e.Index
+	eh.LastTerm = e.Term
+	eh.DupCnt += duped
+
+	eh.StartIndex = eh.Queue.Events[eh.Queue.Front].Index
+
+	return e
+}
+
+// scan function is enumerating events from the index in history and
+// stops till the first point where the key has identified prefix
+func (eh *EventHistory) scan(prefix string, index uint64) (*Event, *etcdErr.Error) {
+	eh.rwl.RLock()
+	defer eh.rwl.RUnlock()
+
+	start := index - eh.StartIndex
+
+	// the index should locate after the event history's StartIndex
+	if start < 0 {
+		return nil,
+			etcdErr.NewError(etcdErr.EcodeEventIndexCleared,
+				fmt.Sprintf("the requested history has been cleared [%v/%v]",
+					eh.StartIndex, index), UndefIndex, UndefTerm)
+	}
+
+	// the index should locate before the size of the queue minus the duplicate count
+	if start >= (uint64(eh.Queue.Size) - eh.DupCnt) { // future index
+		return nil, nil
+	}
+
+	i := int((start + uint64(eh.Queue.Front)) % uint64(eh.Queue.Capacity))
+
+	for {
+		e := eh.Queue.Events[i]
+		if strings.HasPrefix(e.Key, prefix) && index <= e.Index { // make sure we bypass the smaller one
+			return e, nil
+		}
+
+		i = (i + 1) % eh.Queue.Capacity
+
+		if i == eh.Queue.back() { // find nothing, return and watch from current index
+			return nil, nil
+		}
+	}
+}
+
+// clone will be protected by a stop-world lock
+// do not need to obtain internal lock
+func (eh *EventHistory) clone() *EventHistory {
+	clonedQueue := eventQueue{
+		Capacity: eh.Queue.Capacity,
+		Events:   make([]*Event, eh.Queue.Capacity),
+		Size:     eh.Queue.Size,
+		Front:    eh.Queue.Front,
+	}
+
+	for i, e := range eh.Queue.Events {
+		clonedQueue.Events[i] = e
+	}
+
+	return &EventHistory{
+		StartIndex: eh.StartIndex,
+		Queue:      clonedQueue,
+		LastIndex:  eh.LastIndex,
+		LastTerm:   eh.LastTerm,
+		DupCnt:     eh.DupCnt,
+	}
+
+}

+ 66 - 0
store/event_test.go

@@ -0,0 +1,66 @@
+package store
+
+import (
+	"testing"
+)
+
+// TestEventQueue tests a queue with capacity = 100
+// Add 200 events into that queue, and test if the
+// previous 100 events have been swapped out.
+func TestEventQueue(t *testing.T) {
+
+	eh := newEventHistory(100)
+
+	// Add
+	for i := 0; i < 200; i++ {
+		e := newEvent(Create, "/foo", uint64(i), 1)
+		eh.addEvent(e)
+	}
+
+	// Test
+	j := 100
+	i := eh.Queue.Front
+	n := eh.Queue.Size
+	for ; n > 0; n-- {
+		e := eh.Queue.Events[i]
+		if e.Index != uint64(j) {
+			t.Fatalf("queue error!")
+		}
+		j++
+		i = (i + 1) % eh.Queue.Capacity
+	}
+}
+
+func TestScanHistory(t *testing.T) {
+	eh := newEventHistory(100)
+
+	// Add
+	eh.addEvent(newEvent(Create, "/foo", 1, 1))
+	eh.addEvent(newEvent(Create, "/foo/bar", 2, 1))
+	eh.addEvent(newEvent(Create, "/foo/foo", 3, 1))
+	eh.addEvent(newEvent(Create, "/foo/bar/bar", 4, 1))
+	eh.addEvent(newEvent(Create, "/foo/foo/foo", 5, 1))
+
+	e, err := eh.scan("/foo", 1)
+	if err != nil || e.Index != 1 {
+		t.Fatalf("scan error [/foo] [1] %v", e.Index)
+	}
+
+	e, err = eh.scan("/foo/bar", 1)
+
+	if err != nil || e.Index != 2 {
+		t.Fatalf("scan error [/foo/bar] [2] %v", e.Index)
+	}
+
+	e, err = eh.scan("/foo/bar", 3)
+
+	if err != nil || e.Index != 4 {
+		t.Fatalf("scan error [/foo/bar/bar] [4] %v", e.Index)
+	}
+
+	e, err = eh.scan("/foo/bar", 6)
+
+	if e != nil {
+		t.Fatalf("bad index shoud reuturn nil")
+	}
+}

+ 0 - 37
store/keyword_test.go

@@ -1,37 +0,0 @@
-package store
-
-import (
-	"testing"
-)
-
-func TestKeywords(t *testing.T) {
-	keyword := CheckKeyword("_etcd")
-	if !keyword {
-		t.Fatal("_etcd should be keyword")
-	}
-
-	keyword = CheckKeyword("/_etcd")
-
-	if !keyword {
-		t.Fatal("/_etcd should be keyword")
-	}
-
-	keyword = CheckKeyword("/_etcd/")
-
-	if !keyword {
-		t.Fatal("/_etcd/ contains keyword prefix")
-	}
-
-	keyword = CheckKeyword("/_etcd/node1")
-
-	if !keyword {
-		t.Fatal("/_etcd/* contains keyword prefix")
-	}
-
-	keyword = CheckKeyword("/nokeyword/_etcd/node1")
-
-	if keyword {
-		t.Fatal("this does not contain keyword prefix")
-	}
-
-}

+ 0 - 33
store/keywords.go

@@ -1,33 +0,0 @@
-package store
-
-import (
-	"path"
-	"strings"
-)
-
-// keywords for internal useage
-// Key for string keyword; Value for only checking prefix
-var keywords = map[string]bool{
-	"/_etcd":          true,
-	"/ephemeralNodes": true,
-}
-
-// CheckKeyword will check if the key contains the keyword.
-// For now, we only check for prefix.
-func CheckKeyword(key string) bool {
-	key = path.Clean("/" + key)
-
-	// find the second "/"
-	i := strings.Index(key[1:], "/")
-
-	var prefix string
-
-	if i == -1 {
-		prefix = key
-	} else {
-		prefix = key[:i+1]
-	}
-	_, ok := keywords[prefix]
-
-	return ok
-}

+ 363 - 0
store/node.go

@@ -0,0 +1,363 @@
+package store
+
+import (
+	"path"
+	"sort"
+	"time"
+
+	etcdErr "github.com/coreos/etcd/error"
+)
+
+var (
+	Permanent time.Time
+)
+
+const (
+	normal = iota
+	removed
+)
+
+type Node struct {
+	Path          string
+	CreateIndex   uint64
+	CreateTerm    uint64
+	ModifiedIndex uint64
+	ModifiedTerm  uint64
+	Parent        *Node `json:"-"`
+	ExpireTime    time.Time
+	ACL           string
+	Value         string           // for key-value pair
+	Children      map[string]*Node // for directory
+	status        int
+	stopExpire    chan bool // stop expire routine channel
+}
+
+func newFile(nodePath string, value string, createIndex uint64, createTerm uint64, parent *Node, ACL string, expireTime time.Time) *Node {
+	return &Node{
+		Path:          nodePath,
+		CreateIndex:   createIndex,
+		CreateTerm:    createTerm,
+		ModifiedIndex: createIndex,
+		ModifiedTerm:  createTerm,
+		Parent:        parent,
+		ACL:           ACL,
+		stopExpire:    make(chan bool, 1),
+		ExpireTime:    expireTime,
+		Value:         value,
+	}
+}
+
+func newDir(nodePath string, createIndex uint64, createTerm uint64, parent *Node, ACL string, expireTime time.Time) *Node {
+	return &Node{
+		Path:        nodePath,
+		CreateIndex: createIndex,
+		CreateTerm:  createTerm,
+		Parent:      parent,
+		ACL:         ACL,
+		stopExpire:  make(chan bool, 1),
+		ExpireTime:  expireTime,
+		Children:    make(map[string]*Node),
+	}
+}
+
+// Remove function remove the node.
+// If the node is a directory and recursive is true, the function will recursively remove
+// add nodes under the receiver node.
+func (n *Node) Remove(recursive bool, callback func(path string)) *etcdErr.Error {
+	if n.status == removed { // check race between remove and expire
+		return nil
+	}
+
+	if !n.IsDir() { // file node: key-value pair
+		_, name := path.Split(n.Path)
+
+		if n.Parent != nil && n.Parent.Children[name] == n {
+			// This is the only pointer to Node object
+			// Handled by garbage collector
+			delete(n.Parent.Children, name)
+
+			if callback != nil {
+				callback(n.Path)
+			}
+
+			n.stopExpire <- true
+			n.status = removed
+
+		}
+
+		return nil
+	}
+
+	if !recursive {
+		return etcdErr.NewError(etcdErr.EcodeNotFile, "", UndefIndex, UndefTerm)
+	}
+
+	for _, child := range n.Children { // delete all children
+		child.Remove(true, callback)
+	}
+
+	// delete self
+	_, name := path.Split(n.Path)
+	if n.Parent != nil && n.Parent.Children[name] == n {
+		delete(n.Parent.Children, name)
+
+		if callback != nil {
+			callback(n.Path)
+		}
+
+		n.stopExpire <- true
+		n.status = removed
+	}
+
+	return nil
+}
+
+// Read function gets the value of the node.
+// If the receiver node is not a key-value pair, a "Not A File" error will be returned.
+func (n *Node) Read() (string, *etcdErr.Error) {
+	if n.IsDir() {
+		return "", etcdErr.NewError(etcdErr.EcodeNotFile, "", UndefIndex, UndefTerm)
+	}
+
+	return n.Value, nil
+}
+
+// Write function set the value of the node to the given value.
+// If the receiver node is a directory, a "Not A File" error will be returned.
+func (n *Node) Write(value string, index uint64, term uint64) *etcdErr.Error {
+	if n.IsDir() {
+		return etcdErr.NewError(etcdErr.EcodeNotFile, "", UndefIndex, UndefTerm)
+	}
+
+	n.Value = value
+	n.ModifiedIndex = index
+	n.ModifiedTerm = term
+
+	return nil
+}
+
+// List function return a slice of nodes under the receiver node.
+// If the receiver node is not a directory, a "Not A Directory" error will be returned.
+func (n *Node) List() ([]*Node, *etcdErr.Error) {
+	if !n.IsDir() {
+		return nil, etcdErr.NewError(etcdErr.EcodeNotDir, "", UndefIndex, UndefTerm)
+	}
+
+	nodes := make([]*Node, len(n.Children))
+
+	i := 0
+	for _, node := range n.Children {
+		nodes[i] = node
+		i++
+	}
+
+	return nodes, nil
+}
+
+// GetChild function returns the child node under the directory node.
+// On success, it returns the file node
+func (n *Node) GetChild(name string) (*Node, *etcdErr.Error) {
+	if !n.IsDir() {
+		return nil, etcdErr.NewError(etcdErr.EcodeNotDir, n.Path, UndefIndex, UndefTerm)
+	}
+
+	child, ok := n.Children[name]
+
+	if ok {
+		return child, nil
+	}
+
+	return nil, nil
+}
+
+// Add function adds a node to the receiver node.
+// If the receiver is not a directory, a "Not A Directory" error will be returned.
+// If there is a existing node with the same name under the directory, a "Already Exist"
+// error will be returned
+func (n *Node) Add(child *Node) *etcdErr.Error {
+	if !n.IsDir() {
+		return etcdErr.NewError(etcdErr.EcodeNotDir, "", UndefIndex, UndefTerm)
+	}
+
+	_, name := path.Split(child.Path)
+
+	_, ok := n.Children[name]
+
+	if ok {
+		return etcdErr.NewError(etcdErr.EcodeNodeExist, "", UndefIndex, UndefTerm)
+	}
+
+	n.Children[name] = child
+
+	return nil
+}
+
+// Clone function clone the node recursively and return the new node.
+// If the node is a directory, it will clone all the content under this directory.
+// If the node is a key-value pair, it will clone the pair.
+func (n *Node) Clone() *Node {
+	if !n.IsDir() {
+		return newFile(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)
+
+	for key, child := range n.Children {
+		clone.Children[key] = child.Clone()
+	}
+
+	return clone
+}
+
+func (n *Node) recoverAndclean(s *Store) {
+	if n.IsDir() {
+		for _, child := range n.Children {
+			child.Parent = n
+			child.recoverAndclean(s)
+		}
+	}
+
+	n.stopExpire = make(chan bool, 1)
+
+	n.Expire(s)
+}
+
+// Expire function will test if the node is expired.
+// if the node is already expired, delete the node and return.
+// if the node is permemant (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) {
+	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.Remove(true, nil)
+		s.Stats.Inc(ExpireCount)
+
+		return
+	}
+
+	if duration == 0 { // Permanent Node
+		return
+	}
+
+	go func() { // do monitoring
+		select {
+		// if timeout, delete the node
+		case <-time.After(duration):
+
+			s.worldLock.Lock()
+			defer s.worldLock.Unlock()
+
+			e := newEvent(Expire, n.Path, UndefIndex, UndefTerm)
+			s.WatcherHub.notify(e)
+
+			n.Remove(true, nil)
+			s.Stats.Inc(ExpireCount)
+
+			return
+
+		// if stopped, return
+		case <-n.stopExpire:
+			return
+		}
+	}()
+}
+
+// IsHidden function checks if the node is a hidden node. A hidden node
+// will begin with '_'
+// A hidden node will not be shown via get command under a directory
+// For example if we have /foo/_hidden and /foo/notHidden, get "/foo"
+// will only return /foo/notHidden
+func (n *Node) IsHidden() bool {
+	_, name := path.Split(n.Path)
+
+	return name[0] == '_'
+}
+
+func (n *Node) IsPermanent() bool {
+	return n.ExpireTime.Sub(Permanent) == 0
+}
+
+func (n *Node) IsExpired() (bool, time.Duration) {
+	if n.IsPermanent() {
+		return false, 0
+	}
+
+	duration := n.ExpireTime.Sub(time.Now())
+	if duration <= 0 {
+		return true, 0
+	}
+
+	return false, duration
+}
+
+// IsDir function checks whether the node is a directory.
+// If the node is a directory, the function will return true.
+// Otherwise the function will return false.
+func (n *Node) IsDir() bool {
+	return !(n.Children == nil)
+}
+
+func (n *Node) Pair(recurisive, sorted bool) KeyValuePair {
+	if n.IsDir() {
+		pair := KeyValuePair{
+			Key: n.Path,
+			Dir: true,
+		}
+
+		if !recurisive {
+			return pair
+		}
+
+		children, _ := n.List()
+		pair.KVPairs = make([]KeyValuePair, len(children))
+
+		// we do not use the index in the children slice directly
+		// we need to skip the hidden one
+		i := 0
+
+		for _, child := range children {
+
+			if child.IsHidden() { // get will not list hidden node
+				continue
+			}
+
+			pair.KVPairs[i] = child.Pair(recurisive, sorted)
+
+			i++
+		}
+
+		// eliminate hidden nodes
+		pair.KVPairs = pair.KVPairs[:i]
+		if sorted {
+			sort.Sort(pair)
+		}
+
+		return pair
+	}
+
+	return KeyValuePair{
+		Key:   n.Path,
+		Value: n.Value,
+	}
+}
+
+func (n *Node) UpdateTTL(expireTime time.Time, s *Store) {
+	if !n.IsPermanent() {
+		expired, _ := n.IsExpired()
+
+		if !expired {
+			n.stopExpire <- true // suspend it to modify the expiration
+		}
+	}
+
+	if expireTime.Sub(Permanent) != 0 {
+		n.ExpireTime = expireTime
+		n.Expire(s)
+	}
+}

+ 26 - 0
store/response_v1.go

@@ -0,0 +1,26 @@
+package store
+
+import (
+	"time"
+)
+
+// The response from the store to the user who issue a command
+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"`
+}

+ 81 - 13
store/stats.go

@@ -2,32 +2,100 @@ package store
 
 import (
 	"encoding/json"
+	"sync/atomic"
 )
 
-type EtcdStats struct {
+const (
+	SetSuccess        = 100
+	SetFail           = 101
+	DeleteSuccess     = 102
+	DeleteFail        = 103
+	UpdateSuccess     = 104
+	UpdateFail        = 105
+	TestAndSetSuccess = 106
+	TestAndSetFail    = 107
+	GetSuccess        = 110
+	GetFail           = 111
+	ExpireCount       = 112
+)
+
+type Stats struct {
+
 	// Number of get requests
-	Gets uint64 `json:"gets"`
+	GetSuccess uint64 `json:"getsSuccess"`
+	GetFail    uint64 `json:"getsFail"`
 
 	// Number of sets requests
-	Sets uint64 `json:"sets"`
+	SetSuccess uint64 `json:"setsSuccess"`
+	SetFail    uint64 `json:"setsFail"`
 
 	// Number of delete requests
-	Deletes uint64 `json:"deletes"`
+	DeleteSuccess uint64 `json:"deleteSuccess"`
+	DeleteFail    uint64 `json:"deleteFail"`
+
+	// Number of update requests
+	UpdateSuccess uint64 `json:"updateSuccess"`
+	UpdateFail    uint64 `json:"updateFail"`
 
 	// Number of testAndSet requests
-	TestAndSets uint64 `json:"testAndSets"`
+	TestAndSetSuccess uint64 `json:"testAndSetSuccess"`
+	TestAndSetFail    uint64 `json:"testAndSetFail"`
+	ExpireCount       uint64 `json:"expireCount"`
+
+	Watchers uint64 `json:"watchers"`
+}
+
+func newStats() *Stats {
+	s := new(Stats)
+	return s
+}
+
+func (s *Stats) clone() *Stats {
+	return &Stats{s.GetSuccess, s.GetFail, s.SetSuccess, s.SetFail,
+		s.DeleteSuccess, s.DeleteFail, s.UpdateSuccess, s.UpdateFail,
+		s.TestAndSetSuccess, s.TestAndSetFail, s.Watchers, s.ExpireCount}
 }
 
-// Stats returns the basic statistics information of etcd storage since its recent start
-func (s *Store) Stats() []byte {
-	b, _ := json.Marshal(s.BasicStats)
+// Status() return the statistics info of etcd storage its recent start
+func (s *Stats) toJson() []byte {
+	b, _ := json.Marshal(s)
 	return b
 }
 
-// TotalWrites returns the total write operations
-// It helps with snapshot
-func (s *Store) TotalWrites() uint64 {
-	bs := s.BasicStats
+func (s *Stats) TotalReads() uint64 {
+	return s.GetSuccess + s.GetFail
+}
+
+func (s *Stats) TotalWrites() uint64 {
+	return s.SetSuccess + s.SetFail +
+		s.DeleteSuccess + s.DeleteFail +
+		s.TestAndSetSuccess + s.TestAndSetFail +
+		s.UpdateSuccess + s.UpdateFail
+}
 
-	return bs.Deletes + bs.Sets + bs.TestAndSets
+func (s *Stats) Inc(field int) {
+	switch field {
+	case SetSuccess:
+		atomic.AddUint64(&s.SetSuccess, 1)
+	case SetFail:
+		atomic.AddUint64(&s.SetFail, 1)
+	case DeleteSuccess:
+		atomic.AddUint64(&s.DeleteSuccess, 1)
+	case DeleteFail:
+		atomic.AddUint64(&s.DeleteFail, 1)
+	case GetSuccess:
+		atomic.AddUint64(&s.GetSuccess, 1)
+	case GetFail:
+		atomic.AddUint64(&s.GetFail, 1)
+	case UpdateSuccess:
+		atomic.AddUint64(&s.UpdateSuccess, 1)
+	case UpdateFail:
+		atomic.AddUint64(&s.UpdateFail, 1)
+	case TestAndSetSuccess:
+		atomic.AddUint64(&s.TestAndSetSuccess, 1)
+	case TestAndSetFail:
+		atomic.AddUint64(&s.TestAndSetFail, 1)
+	case ExpireCount:
+		atomic.AddUint64(&s.ExpireCount, 1)
+	}
 }

+ 165 - 0
store/stats_test.go

@@ -0,0 +1,165 @@
+package store
+
+import (
+	"math/rand"
+	"testing"
+	"time"
+)
+
+func TestBasicStats(t *testing.T) {
+	s := New()
+	keys := GenKeys(rand.Intn(100), 5)
+
+	var i uint64
+	var GetSuccess, GetFail, SetSuccess, SetFail, DeleteSuccess, DeleteFail uint64
+	var UpdateSuccess, UpdateFail, TestAndSetSuccess, TestAndSetFail, watcher_number uint64
+
+	for _, k := range keys {
+		i++
+		_, err := s.Create(k, "bar", false, false, time.Now().Add(time.Second*time.Duration(rand.Intn(6))), i, 1)
+		if err != nil {
+			SetFail++
+		} else {
+			SetSuccess++
+		}
+	}
+
+	time.Sleep(time.Second * 3)
+
+	for _, k := range keys {
+		_, err := s.Get(k, false, false, i, 1)
+		if err != nil {
+			GetFail++
+		} else {
+			GetSuccess++
+		}
+	}
+
+	for _, k := range keys {
+		i++
+		_, err := s.Update(k, "foo", time.Now().Add(time.Second*time.Duration(rand.Intn(6))), i, 1)
+		if err != nil {
+			UpdateFail++
+		} else {
+			UpdateSuccess++
+		}
+	}
+
+	time.Sleep(time.Second * 3)
+
+	for _, k := range keys {
+		_, err := s.Get(k, false, false, i, 1)
+		if err != nil {
+			GetFail++
+		} else {
+			GetSuccess++
+		}
+	}
+
+	for _, k := range keys {
+		i++
+		_, err := s.TestAndSet(k, "foo", 0, "bar", Permanent, i, 1)
+		if err != nil {
+			TestAndSetFail++
+		} else {
+			TestAndSetSuccess++
+		}
+	}
+
+	for _, k := range keys {
+		s.Watch(k, false, 0, i, 1)
+		watcher_number++
+	}
+
+	for _, k := range keys {
+		_, err := s.Get(k, false, false, i, 1)
+		if err != nil {
+			GetFail++
+		} else {
+			GetSuccess++
+		}
+	}
+
+	for _, k := range keys {
+		i++
+		_, err := s.Delete(k, false, i, 1)
+		if err != nil {
+			DeleteFail++
+		} else {
+			watcher_number--
+			DeleteSuccess++
+		}
+	}
+
+	for _, k := range keys {
+		_, err := s.Get(k, false, false, i, 1)
+		if err != nil {
+			GetFail++
+		} else {
+			GetSuccess++
+		}
+	}
+
+	if GetSuccess != s.Stats.GetSuccess {
+		t.Fatalf("GetSuccess [%d] != Stats.GetSuccess [%d]", GetSuccess, s.Stats.GetSuccess)
+	}
+
+	if GetFail != s.Stats.GetFail {
+		t.Fatalf("GetFail [%d] != Stats.GetFail [%d]", GetFail, s.Stats.GetFail)
+	}
+
+	if SetSuccess != s.Stats.SetSuccess {
+		t.Fatalf("SetSuccess [%d] != Stats.SetSuccess [%d]", SetSuccess, s.Stats.SetSuccess)
+	}
+
+	if SetFail != s.Stats.SetFail {
+		t.Fatalf("SetFail [%d] != Stats.SetFail [%d]", SetFail, s.Stats.SetFail)
+	}
+
+	if DeleteSuccess != s.Stats.DeleteSuccess {
+		t.Fatalf("DeleteSuccess [%d] != Stats.DeleteSuccess [%d]", DeleteSuccess, s.Stats.DeleteSuccess)
+	}
+
+	if DeleteFail != s.Stats.DeleteFail {
+		t.Fatalf("DeleteFail [%d] != Stats.DeleteFail [%d]", DeleteFail, s.Stats.DeleteFail)
+	}
+
+	if UpdateSuccess != s.Stats.UpdateSuccess {
+		t.Fatalf("UpdateSuccess [%d] != Stats.UpdateSuccess [%d]", UpdateSuccess, s.Stats.UpdateSuccess)
+	}
+
+	if UpdateFail != s.Stats.UpdateFail {
+		t.Fatalf("UpdateFail [%d] != Stats.UpdateFail [%d]", UpdateFail, s.Stats.UpdateFail)
+	}
+
+	if TestAndSetSuccess != s.Stats.TestAndSetSuccess {
+		t.Fatalf("TestAndSetSuccess [%d] != Stats.TestAndSetSuccess [%d]", TestAndSetSuccess, s.Stats.TestAndSetSuccess)
+	}
+
+	if TestAndSetFail != s.Stats.TestAndSetFail {
+		t.Fatalf("TestAndSetFail [%d] != Stats.TestAndSetFail [%d]", TestAndSetFail, s.Stats.TestAndSetFail)
+	}
+
+	s = New()
+	SetSuccess = 0
+	SetFail = 0
+
+	for _, k := range keys {
+		i++
+		_, err := s.Create(k, "bar", false, false, time.Now().Add(time.Second*3), i, 1)
+		if err != nil {
+			SetFail++
+		} else {
+			SetSuccess++
+		}
+	}
+
+	time.Sleep(6 * time.Second)
+
+	ExpireCount := SetSuccess
+
+	if ExpireCount != s.Stats.ExpireCount {
+		t.Fatalf("ExpireCount [%d] != Stats.ExpireCount [%d]", ExpireCount, s.Stats.ExpireCount)
+	}
+
+}

+ 296 - 492
store/store.go

@@ -4,660 +4,464 @@ import (
 	"encoding/json"
 	"fmt"
 	"path"
+	"sort"
 	"strconv"
+	"strings"
 	"sync"
 	"time"
 
 	etcdErr "github.com/coreos/etcd/error"
 )
 
-//------------------------------------------------------------------------------
-//
-// Typedefs
-//
-//------------------------------------------------------------------------------
-
-// The main struct of the Key-Value store
 type Store struct {
+	Root       *Node
+	WatcherHub *watcherHub
+	Index      uint64
+	Term       uint64
+	Stats      *Stats
+	worldLock  sync.RWMutex // stop the world lock
+}
 
-	// key-value store structure
-	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.RWMutex
-
-	// WatcherHub is where we register all the clients
-	// who issue a watch request
-	watcher *WatcherHub
-
-	// The string channel to send messages to the outside world
-	// Now we use it to send changes to the hub of the web service
-	messager chan<- string
-
-	// A map to keep the recent response to the clients
-	ResponseMap map[string]*Response
-
-	// The max number of the recent responses we can record
-	ResponseMaxSize int
+func New() *Store {
+	s := new(Store)
+	s.Root = newDir("/", 0, 0, nil, "", Permanent)
+	s.Stats = newStats()
+	s.WatcherHub = newWatchHub(1000)
 
-	// The current number of the recent responses we have recorded
-	ResponseCurrSize uint
+	return s
+}
 
-	// The index of the first recent responses we have
-	ResponseStartIndex uint64
+func (s *Store) Get(nodePath string, recursive, sorted bool, index uint64, term uint64) (*Event, error) {
+	s.worldLock.RLock()
+	defer s.worldLock.RUnlock()
 
-	// Current index of the raft machine
-	Index uint64
+	nodePath = path.Clean(path.Join("/", nodePath))
 
-	// Basic statistics information of etcd storage
-	BasicStats EtcdStats
-}
+	n, err := s.internalGet(nodePath, index, term)
 
-// A Node represents a Value in the Key-Value pair in the store
-// It has its value, expire time and a channel used to update the
-// expire time (since we do countdown in a go routine, we need to
-// communicate with it via channel)
-type Node struct {
-	// The string value of the node
-	Value string `json:"value"`
+	if err != nil {
+		s.Stats.Inc(GetFail)
+		return nil, err
+	}
 
-	// If the node is a permanent one the ExprieTime will be Unix(0,0)
-	// Otherwise after the expireTime, the node will be deleted
-	ExpireTime time.Time `json:"expireTime"`
+	e := newEvent(Get, nodePath, index, term)
 
-	// A channel to update the expireTime of the node
-	update chan time.Time `json:"-"`
-}
+	if n.IsDir() { // node is dir
+		e.Dir = true
 
-// The response from the store to the user who issue a command
-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"`
+		children, _ := n.List()
+		e.KVPairs = make([]KeyValuePair, len(children))
 
-	// If the key did not exist before the action,
-	// this field should be set to true
-	NewKey bool `json:"newKey,omitempty"`
+		// we do not use the index in the children slice directly
+		// we need to skip the hidden one
+		i := 0
 
-	Expiration *time.Time `json:"expiration,omitempty"`
+		for _, child := range children {
 
-	// Time to live in second
-	TTL int64 `json:"ttl,omitempty"`
+			if child.IsHidden() { // get will not list hidden node
+				continue
+			}
 
-	// The command index of the raft machine when the command is executed
-	Index uint64 `json:"index"`
-}
+			e.KVPairs[i] = child.Pair(recursive, sorted)
 
-// A listNode represent the simplest Key-Value pair with its type
-// It is only used when do list opeartion
-// We want to have a file system like store, thus we distingush "file"
-// and "directory"
-type ListNode struct {
-	Key   string
-	Value string
-	Type  string
-}
+			i++
+		}
 
-var PERMANENT = time.Unix(0, 0)
+		// eliminate hidden nodes
+		e.KVPairs = e.KVPairs[:i]
 
-//------------------------------------------------------------------------------
-//
-// Methods
-//
-//------------------------------------------------------------------------------
+		rootPairs := KeyValuePair{
+			KVPairs: e.KVPairs,
+		}
 
-// Create a new stroe
-// Arguement max is the max number of response we want to record
-func CreateStore(max int) *Store {
-	s := new(Store)
+		if sorted {
+			sort.Sort(rootPairs)
+		}
 
-	s.messager = nil
-
-	s.ResponseMap = make(map[string]*Response)
-	s.ResponseStartIndex = 0
-	s.ResponseMaxSize = max
-	s.ResponseCurrSize = 0
-
-	s.Tree = &tree{
-		&treeNode{
-			Node{
-				"/",
-				time.Unix(0, 0),
-				nil,
-			},
-			true,
-			make(map[string]*treeNode),
-		},
+	} else { // node is file
+		e.Value = n.Value
 	}
 
-	s.watcher = newWatcherHub()
+	if n.ExpireTime.Sub(Permanent) != 0 {
+		e.Expiration = &n.ExpireTime
+		e.TTL = int64(n.ExpireTime.Sub(time.Now())/time.Second) + 1
+	}
 
-	return s
-}
+	s.Stats.Inc(GetSuccess)
 
-// Set the messager of the store
-func (s *Store) SetMessager(messager chan<- string) {
-	s.messager = messager
+	return e, nil
 }
 
-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)
+// 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,
+	expireTime time.Time, index uint64, term uint64) (*Event, error) {
+	nodePath = path.Clean(path.Join("/", nodePath))
 
+	s.worldLock.Lock()
+	defer s.worldLock.Unlock()
+	return s.internalCreate(nodePath, value, incrementalSuffix, force, expireTime, index, term, Create)
 }
 
-// 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
-	s.Index = index
+// 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, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) {
+	s.worldLock.Lock()
+	defer s.worldLock.Unlock()
+	nodePath = path.Clean(path.Join("/", nodePath))
 
-	//Update stats
-	s.BasicStats.Sets++
+	n, err := s.internalGet(nodePath, index, term)
 
-	key = path.Clean("/" + key)
+	if err != nil { // if the node does not exist, return error
+		s.Stats.Inc(UpdateFail)
 
-	isExpire := !expireTime.Equal(PERMANENT)
-
-	// base response
-	resp := Response{
-		Action: "SET",
-		Key:    key,
-		Value:  value,
-		Index:  index,
-	}
-
-	// When the slow follower receive the set command
-	// the key may be expired, we should not add the node
-	// also if the node exist, we need to delete the node
-	if isExpire && expireTime.Sub(time.Now()) < 0 {
-		return s.internalDelete(key, index)
-	}
-
-	var TTL int64
-
-	// Update ttl
-	if isExpire {
-		TTL = int64(expireTime.Sub(time.Now()) / time.Second)
-		resp.Expiration = &expireTime
-		resp.TTL = TTL
+		return nil, err
 	}
 
-	// Get the node
-	node, ok := s.Tree.get(key)
+	e := newEvent(Update, nodePath, s.Index, s.Term)
 
-	if ok {
-		// Update when node exists
+	if n.IsDir() { // if the node is a directory, we can only update ttl
+		if len(value) != 0 {
+			s.Stats.Inc(UpdateFail)
 
-		// Node is not permanent
-		if !node.ExpireTime.Equal(PERMANENT) {
-
-			// If node is not permanent
-			// Update its expireTime
-			node.update <- expireTime
+			err := etcdErr.NewError(etcdErr.EcodeNotFile, nodePath, index, term)
+			return nil, err
+		}
 
-		} else {
-
-			// If we want the permanent node to have expire time
-			// We need to create a go routine with a channel
-			if isExpire {
-				node.update = make(chan time.Time)
-				go s.monitorExpiration(key, node.update, expireTime)
-			}
+	} else { // if the node is a file, we can update value and ttl
+		e.PrevValue = n.Value
 
+		if len(value) != 0 {
+			e.Value = value
 		}
 
-		// Update the information of the node
-		s.Tree.set(key, Node{value, expireTime, node.update})
+		n.Write(value, index, term)
+	}
 
-		resp.PrevValue = node.Value
+	// update ttl
+	n.UpdateTTL(expireTime, s)
 
-		s.watcher.notify(resp)
+	if n.ExpireTime.Sub(Permanent) != 0 {
+		e.Expiration = &n.ExpireTime
+		e.TTL = int64(expireTime.Sub(time.Now())/time.Second) + 1
+	}
+	s.WatcherHub.notify(e)
 
-		msg, err := json.Marshal(resp)
+	s.Stats.Inc(UpdateSuccess)
 
-		// Send to the messager
-		if s.messager != nil && err == nil {
-			s.messager <- string(msg)
-		}
+	return e, nil
+}
 
-		s.addToResponseMap(index, &resp)
+func (s *Store) TestAndSet(nodePath string, prevValue string, prevIndex uint64,
+	value string, expireTime time.Time, index uint64, term uint64) (*Event, error) {
 
-		return msg, err
+	nodePath = path.Clean(path.Join("/", nodePath))
 
-		// Add new node
-	} else {
+	s.worldLock.Lock()
+	defer s.worldLock.Unlock()
 
-		update := make(chan time.Time)
-
-		ok := s.Tree.set(key, Node{value, expireTime, update})
+	if prevValue == "" && prevIndex == 0 { // try create
+		return s.internalCreate(nodePath, value, false, false, expireTime, index, term, TestAndSet)
+	}
 
-		if !ok {
-			return nil, etcdErr.NewError(102, "set: "+key)
-		}
+	n, err := s.internalGet(nodePath, index, term)
 
-		if isExpire {
-			go s.monitorExpiration(key, update, expireTime)
-		}
+	if err != nil {
+		s.Stats.Inc(TestAndSetFail)
+		return nil, err
+	}
 
-		resp.NewKey = true
+	if n.IsDir() { // can only test and set file
+		s.Stats.Inc(TestAndSetFail)
+		return nil, etcdErr.NewError(etcdErr.EcodeNotFile, nodePath, index, term)
+	}
 
-		msg, err := json.Marshal(resp)
+	if n.Value == prevValue || n.ModifiedIndex == prevIndex {
+		// if test succeed, write the value
+		e := newEvent(TestAndSet, nodePath, index, term)
+		e.PrevValue = n.Value
+		e.Value = value
+		n.Write(value, index, term)
 
-		// Nofity the watcher
-		s.watcher.notify(resp)
+		n.UpdateTTL(expireTime, s)
 
-		// Send to the messager
-		if s.messager != nil && err == nil {
-			s.messager <- string(msg)
+		if n.ExpireTime.Sub(Permanent) != 0 {
+			e.Expiration = &n.ExpireTime
+			e.TTL = int64(expireTime.Sub(time.Now())/time.Second) + 1
 		}
 
-		s.addToResponseMap(index, &resp)
-		return msg, err
+		s.WatcherHub.notify(e)
+		s.Stats.Inc(TestAndSetSuccess)
+		return e, nil
 	}
 
+	cause := fmt.Sprintf("[%v != %v] [%v != %v]", prevValue, n.Value, prevIndex, n.ModifiedIndex)
+	s.Stats.Inc(TestAndSetFail)
+	return nil, etcdErr.NewError(etcdErr.EcodeTestFailed, cause, index, term)
 }
 
-// Get the value of the key and return the raw response
-func (s *Store) internalGet(key string) *Response {
+// 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) {
+	nodePath = path.Clean(path.Join("/", nodePath))
 
-	key = path.Clean("/" + key)
+	s.worldLock.Lock()
+	defer s.worldLock.Unlock()
 
-	node, ok := s.Tree.get(key)
+	n, err := s.internalGet(nodePath, index, term)
 
-	if ok {
-		var TTL int64
-		var isExpire bool = false
-
-		isExpire = !node.ExpireTime.Equal(PERMANENT)
-
-		resp := &Response{
-			Action: "GET",
-			Key:    key,
-			Value:  node.Value,
-			Index:  s.Index,
-		}
-
-		// Update ttl
-		if isExpire {
-			TTL = int64(node.ExpireTime.Sub(time.Now()) / time.Second)
-			resp.Expiration = &node.ExpireTime
-			resp.TTL = TTL
-		}
+	if err != nil { // if the node does not exist, return error
+		s.Stats.Inc(DeleteFail)
+		return nil, err
+	}
 
-		return resp
+	e := newEvent(Delete, nodePath, index, term)
 
+	if n.IsDir() {
+		e.Dir = true
 	} else {
-		// we do not found the key
-		return nil
+		e.PrevValue = n.Value
 	}
-}
 
-// Get all the items under key
-// If key is a file return the file
-// If key is a directory reuturn an array of files
-func (s *Store) Get(key string) ([]byte, error) {
-	s.mutex.RLock()
-	defer s.mutex.RUnlock()
+	callback := func(path string) { // notify function
+		s.WatcherHub.notifyWithPath(e, path, true)
+	}
 
-	resps, err := s.RawGet(key)
+	err = n.Remove(recursive, callback)
 
 	if err != nil {
+		s.Stats.Inc(DeleteFail)
 		return nil, err
 	}
 
-	key = path.Clean("/" + key)
-
-	// If the number of resps == 1 and the response key
-	// is the key we query, a signal key-value should
-	// be returned
-	if len(resps) == 1 && resps[0].Key == key {
-		return json.Marshal(resps[0])
-	}
+	s.WatcherHub.notify(e)
+	s.Stats.Inc(DeleteSuccess)
 
-	return json.Marshal(resps)
+	return e, nil
 }
 
-func (s *Store) rawGetNode(key string, node *Node) ([]*Response, error) {
-	resps := make([]*Response, 1)
+func (s *Store) Watch(prefix string, recursive bool, sinceIndex uint64, index uint64, term uint64) (<-chan *Event, error) {
+	prefix = path.Clean(path.Join("/", prefix))
 
-	isExpire := !node.ExpireTime.Equal(PERMANENT)
+	s.worldLock.RLock()
+	defer s.worldLock.RUnlock()
 
-	resps[0] = &Response{
-		Action: "GET",
-		Index:  s.Index,
-		Key:    key,
-		Value:  node.Value,
+	s.Index, s.Term = index, term
+
+	var c <-chan *Event
+	var err *etcdErr.Error
+
+	if sinceIndex == 0 {
+		c, err = s.WatcherHub.watch(prefix, recursive, index+1)
+
+	} else {
+		c, err = s.WatcherHub.watch(prefix, recursive, sinceIndex)
 	}
 
-	// Update ttl
-	if isExpire {
-		TTL := int64(node.ExpireTime.Sub(time.Now()) / time.Second)
-		resps[0].Expiration = &node.ExpireTime
-		resps[0].TTL = TTL
+	if err != nil {
+		err.Index = index
+		err.Term = term
 	}
 
-	return resps, nil
+	return c, err
 }
 
-func (s *Store) rawGetNodeList(key string, keys []string, nodes []*Node) ([]*Response, error) {
-	resps := make([]*Response, len(nodes))
-
-	// TODO: check if nodes and keys are the same length
-	for i := 0; i < len(nodes); i++ {
-		var TTL int64
-		var isExpire bool = false
+// 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) {
+	components := strings.Split(nodePath, "/")
 
-		isExpire = !nodes[i].ExpireTime.Equal(PERMANENT)
+	curr := s.Root
+	var err *etcdErr.Error
 
-		resps[i] = &Response{
-			Action: "GET",
-			Index:  s.Index,
-			Key:    path.Join(key, keys[i]),
+	for i := 1; i < len(components); i++ {
+		if len(components[i]) == 0 { // ignore empty string
+			return curr, nil
 		}
 
-		if len(nodes[i].Value) != 0 {
-			resps[i].Value = nodes[i].Value
-		} else {
-			resps[i].Dir = true
-		}
-
-		// Update ttl
-		if isExpire {
-			TTL = int64(nodes[i].ExpireTime.Sub(time.Now()) / time.Second)
-			resps[i].Expiration = &nodes[i].ExpireTime
-			resps[i].TTL = TTL
+		curr, err = walkFunc(curr, components[i])
+		if err != nil {
+			return nil, err
 		}
 
 	}
 
-	return resps, nil
+	return curr, nil
 }
 
-func (s *Store) RawGet(key string) ([]*Response, error) {
-	// Update stats
-	s.BasicStats.Gets++
-
-	key = path.Clean("/" + key)
+func (s *Store) internalCreate(nodePath string, value string, incrementalSuffix bool, force bool,
+	expireTime time.Time, index uint64, term uint64, action string) (*Event, error) {
 
-	nodes, keys, ok := s.Tree.list(key)
-	if !ok {
-		return nil, etcdErr.NewError(100, "get: "+key)
+	if incrementalSuffix { // append unique incremental suffix to the node path
+		nodePath += "_" + strconv.FormatUint(index, 10)
 	}
 
-	switch node := nodes.(type) {
-	case *Node:
-		return s.rawGetNode(key, node)
-	case []*Node:
-		return s.rawGetNodeList(key, keys, node)
-	default:
-		panic("invalid cast ")
-	}
-}
-
-func (s *Store) Delete(key string, index uint64) ([]byte, error) {
-	s.mutex.Lock()
-	defer s.mutex.Unlock()
-	return s.internalDelete(key, index)
-}
+	nodePath = path.Clean(path.Join("/", nodePath))
 
-// Delete the key
-func (s *Store) internalDelete(key string, index uint64) ([]byte, error) {
+	// make sure we can create the node
+	_, err := s.internalGet(nodePath, index, term)
 
-	// Update stats
-	s.BasicStats.Deletes++
-
-	key = path.Clean("/" + key)
-
-	// Update index
-	s.Index = index
-
-	node, ok := s.Tree.get(key)
-
-	if !ok {
-		return nil, etcdErr.NewError(100, "delete: "+key)
+	if err == nil && !force { // key already exists
+		s.Stats.Inc(SetFail)
+		return nil, etcdErr.NewError(etcdErr.EcodeNodeExist, nodePath, index, term)
 	}
 
-	resp := Response{
-		Action:    "DELETE",
-		Key:       key,
-		PrevValue: node.Value,
-		Index:     index,
+	if err != nil && err.ErrorCode == 104 { // we cannot create the key due to meet a file while walking
+		s.Stats.Inc(SetFail)
+		return nil, err
 	}
 
-	if node.ExpireTime.Equal(PERMANENT) {
-
-		s.Tree.delete(key)
+	dir, newNodeName := path.Split(nodePath)
 
-	} else {
-		resp.Expiration = &node.ExpireTime
-		// Kill the expire go routine
-		node.update <- PERMANENT
-		s.Tree.delete(key)
+	// walk through the nodePath, create dirs and get the last directory node
+	d, err := s.walk(dir, s.checkDir)
 
+	if err != nil {
+		s.Stats.Inc(SetFail)
+		fmt.Println("1: bad create")
+		return nil, err
 	}
 
-	msg, err := json.Marshal(resp)
+	e := newEvent(action, nodePath, s.Index, s.Term)
+
+	if force { // force will try to replace a existing file
+		n, _ := d.GetChild(newNodeName)
+		if n != nil {
+			if n.IsDir() {
+				return nil, etcdErr.NewError(etcdErr.EcodeNotFile, nodePath, index, term)
+			}
+			e.PrevValue, _ = n.Read()
 
-	s.watcher.notify(resp)
+			n.Remove(false, nil)
 
-	// notify the messager
-	if s.messager != nil && err == nil {
-		s.messager <- string(msg)
+		}
 	}
 
-	s.addToResponseMap(index, &resp)
+	var n *Node
 
-	return msg, err
-}
+	if len(value) != 0 { // create file
+		e.Value = value
 
-// 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) {
-	s.mutex.Lock()
-	defer s.mutex.Unlock()
+		n = newFile(nodePath, value, s.Index, s.Term, d, "", expireTime)
 
-	// Update stats
-	s.BasicStats.TestAndSets++
+	} else { // create directory
+		e.Dir = true
 
-	resp := s.internalGet(key)
+		n = newDir(nodePath, s.Index, s.Term, d, "", expireTime)
 
-	if resp == nil {
-		if prevValue != "" {
-			errmsg := fmt.Sprintf("TestAndSet: key not found and previousValue is not empty %s:%s ", key, prevValue)
-			return nil, etcdErr.NewError(100, errmsg)
-		}
-		return s.internalSet(key, value, expireTime, index)
 	}
 
-	if resp.Value == prevValue {
-
-		// If test succeed, do set
-		return s.internalSet(key, value, expireTime, index)
-	} else {
+	err = d.Add(n)
 
-		// If fails, return err
-		return nil, etcdErr.NewError(101, fmt.Sprintf("TestAndSet: %s!=%s",
-			resp.Value, prevValue))
+	if err != nil {
+		s.Stats.Inc(SetFail)
+		return nil, err
 	}
 
-}
+	// Node with TTL
+	if expireTime.Sub(Permanent) != 0 {
+		n.Expire(s)
+		e.Expiration = &n.ExpireTime
+		e.TTL = int64(expireTime.Sub(time.Now())/time.Second) + 1
+	}
 
-// Add a channel to the watchHub.
-// The watchHub will send response to the channel when any key under the prefix
-// changes [since the sinceIndex if given]
-func (s *Store) AddWatcher(prefix string, watcher *Watcher, sinceIndex uint64) error {
-	return s.watcher.addWatcher(prefix, watcher, sinceIndex, s.ResponseStartIndex, s.Index, s.ResponseMap)
+	s.WatcherHub.notify(e)
+	s.Stats.Inc(SetSuccess)
+	return e, nil
 }
 
-// This function should be created as a go routine to delete the key-value pair
-// when it reaches expiration time
-
-func (s *Store) monitorExpiration(key string, update chan time.Time, expireTime time.Time) {
-
-	duration := expireTime.Sub(time.Now())
-
-	for {
-		select {
-
-		// Timeout delete the node
-		case <-time.After(duration):
-			node, ok := s.Tree.get(key)
-
-			if !ok {
-				return
+// InternalGet function get the node of the given nodePath.
+func (s *Store) internalGet(nodePath string, index uint64, term uint64) (*Node, *etcdErr.Error) {
+	nodePath = path.Clean(path.Join("/", nodePath))
 
-			} else {
-				s.mutex.Lock()
+	// update file system known index and term
+	s.Index, s.Term = index, term
 
-				s.Tree.delete(key)
+	walkFunc := func(parent *Node, name string) (*Node, *etcdErr.Error) {
 
-				resp := Response{
-					Action:     "DELETE",
-					Key:        key,
-					PrevValue:  node.Value,
-					Expiration: &node.ExpireTime,
-					Index:      s.Index,
-				}
-				s.mutex.Unlock()
-
-				msg, err := json.Marshal(resp)
-
-				s.watcher.notify(resp)
-
-				// notify the messager
-				if s.messager != nil && err == nil {
-					s.messager <- string(msg)
-				}
+		if !parent.IsDir() {
+			err := etcdErr.NewError(etcdErr.EcodeNotDir, parent.Path, index, term)
+			return nil, err
+		}
 
-				return
+		child, ok := parent.Children[name]
+		if ok {
+			return child, nil
+		}
 
-			}
+		return nil, etcdErr.NewError(etcdErr.EcodeKeyNotFound, path.Join(parent.Path, name), index, term)
+	}
 
-		case updateTime := <-update:
-			// Update duration
-			// If the node become a permanent one, the go routine is
-			// not needed
-			if updateTime.Equal(PERMANENT) {
-				return
-			}
+	f, err := s.walk(nodePath, walkFunc)
 
-			// Update duration
-			duration = updateTime.Sub(time.Now())
-		}
+	if err != nil {
+		return nil, err
 	}
+	return f, nil
 }
 
-// When we receive a command that will change the state of the key-value store
-// We will add the result of it to the ResponseMap for the use of watch command
-// Also we may remove the oldest response when we add new one
-func (s *Store) addToResponseMap(index uint64, resp *Response) {
+// checkDir function will check whether the component is a directory under parent 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) {
+	subDir, ok := parent.Children[dirName]
 
-	// zero case
-	if s.ResponseMaxSize == 0 {
-		return
+	if ok {
+		return subDir, nil
 	}
 
-	strIndex := strconv.FormatUint(index, 10)
-	s.ResponseMap[strIndex] = resp
+	n := newDir(path.Join(parent.Path, dirName), s.Index, s.Term, parent, parent.ACL, Permanent)
 
-	// unlimited
-	if s.ResponseMaxSize < 0 {
-		s.ResponseCurrSize++
-		return
-	}
+	parent.Children[dirName] = n
 
-	// if we reach the max point, we need to delete the most latest
-	// response and update the startIndex
-	if s.ResponseCurrSize == uint(s.ResponseMaxSize) {
-		s.ResponseStartIndex++
-		delete(s.ResponseMap, strconv.FormatUint(s.ResponseStartIndex, 10))
-	} else {
-		s.ResponseCurrSize++
-	}
+	return n, nil
 }
 
-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)
+// Save function saves the static state of the store system.
+// 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) {
+	s.worldLock.Lock()
 
-	for index, response := range s.ResponseMap {
-		newStore.ResponseMap[index] = response
-	}
+	clonedStore := New()
+	clonedStore.Index = s.Index
+	clonedStore.Term = s.Term
+	clonedStore.Root = s.Root.Clone()
+	clonedStore.WatcherHub = s.WatcherHub.clone()
+	clonedStore.Stats = s.Stats.clone()
 
-	return newStore
-}
+	s.worldLock.Unlock()
 
-// Save the current state of the storage system
-func (s *Store) Save() ([]byte, error) {
-	// 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(clonedStore)
 
-	b, err := json.Marshal(cloneStore)
 	if err != nil {
-		fmt.Println(err)
 		return nil, err
 	}
+
 	return b, nil
 }
 
-// Recovery the state of the stroage system from a previous state
+// recovery function recovery the store system from a static state.
+// 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 {
-	s.mutex.Lock()
-	defer s.mutex.Unlock()
-	// we need to stop all the current watchers
-	// recovery will clear watcherHub
-	s.watcher.stopWatchers()
-
+	s.worldLock.Lock()
+	defer s.worldLock.Unlock()
 	err := json.Unmarshal(state, s)
 
-	// The only thing need to change after the recovery is the
-	// node with expiration time, we need to delete all the node
-	// that have been expired and setup go routines to monitor the
-	// other ones
-	s.checkExpiration()
-
-	return err
-}
+	if err != nil {
+		return err
+	}
 
-// Clean the expired nodes
-// Set up go routines to mon
-func (s *Store) checkExpiration() {
-	s.Tree.traverse(s.checkNode, false)
+	s.Root.recoverAndclean(s)
+	return nil
 }
 
-// Check each node
-func (s *Store) checkNode(key string, node *Node) {
-
-	if node.ExpireTime.Equal(PERMANENT) {
-		return
-	} else {
-		if node.ExpireTime.Sub(time.Now()) >= time.Second {
-
-			node.update = make(chan time.Time)
-			go s.monitorExpiration(key, node.update, node.ExpireTime)
-
-		} else {
-			// we should delete this node
-			s.Tree.delete(key)
-		}
-	}
+func (s *Store) JsonStats() []byte {
+	s.Stats.Watchers = uint64(s.WatcherHub.count)
+	return s.Stats.toJson()
 }

+ 430 - 130
store/store_test.go

@@ -1,258 +1,558 @@
 package store
 
 import (
-	"encoding/json"
+	"math/rand"
+	"strconv"
 	"testing"
 	"time"
 )
 
-func TestStoreGetDelete(t *testing.T) {
+func TestCreateAndGet(t *testing.T) {
+	s := New()
 
-	s := CreateStore(100)
-	s.Set("foo", "bar", time.Unix(0, 0), 1)
-	res, err := s.Get("foo")
+	s.Create("/foobar", "bar", false, false, Permanent, 1, 1)
+
+	// already exist, create should fail
+	_, err := s.Create("/foobar", "bar", false, false, Permanent, 1, 1)
+
+	if err == nil {
+		t.Fatal("Create should fail")
+	}
+
+	s.Delete("/foobar", true, 1, 1)
+
+	// this should create successfully
+	createAndGet(s, "/foobar", t)
+	createAndGet(s, "/foo/bar", t)
+	createAndGet(s, "/foo/foo/bar", t)
+
+	// meet file, create should fail
+	_, err = s.Create("/foo/bar/bar", "bar", false, false, Permanent, 2, 1)
+
+	if err == nil {
+		t.Fatal("Create should fail")
+	}
+
+	// create a directory
+	_, err = s.Create("/fooDir", "", false, false, Permanent, 3, 1)
 
 	if err != nil {
-		t.Fatalf("Unknown error")
+		t.Fatal("Cannot create /fooDir")
 	}
 
-	var result Response
-	json.Unmarshal(res, &result)
+	e, err := s.Get("/fooDir", false, false, 3, 1)
 
-	if result.Value != "bar" {
-		t.Fatalf("Cannot get stored value")
+	if err != nil || e.Dir != true {
+		t.Fatal("Cannot create /fooDir ")
 	}
 
-	s.Delete("foo", 2)
-	_, err = s.Get("foo")
+	// create a file under directory
+	_, err = s.Create("/fooDir/bar", "bar", false, false, Permanent, 4, 1)
 
-	if err == nil {
-		t.Fatalf("Got deleted value")
+	if err != nil {
+		t.Fatal("Cannot create /fooDir/bar = bar")
 	}
 }
 
-func TestTestAndSet(t *testing.T) {
-	s := CreateStore(100)
-	s.Set("foo", "bar", time.Unix(0, 0), 1)
+func TestUpdateFile(t *testing.T) {
+	s := New()
 
-	_, err := s.TestAndSet("foo", "barbar", "barbar", time.Unix(0, 0), 2)
+	_, err := s.Create("/foo/bar", "bar", false, false, Permanent, 1, 1)
 
-	if err == nil {
-		t.Fatalf("test bar == barbar should fail")
+	if err != nil {
+		t.Fatalf("cannot create %s=bar [%s]", "/foo/bar", err.Error())
 	}
 
-	_, err = s.TestAndSet("foo", "bar", "barbar", time.Unix(0, 0), 3)
+	_, err = s.Update("/foo/bar", "barbar", Permanent, 2, 1)
 
 	if err != nil {
-		t.Fatalf("test bar == bar should succeed")
+		t.Fatalf("cannot update %s=barbar [%s]", "/foo/bar", err.Error())
 	}
 
-	_, err = s.TestAndSet("foo", "", "barbar", time.Unix(0, 0), 4)
+	e, err := s.Get("/foo/bar", false, false, 2, 1)
 
-	if err == nil {
-		t.Fatalf("test empty == bar should fail")
+	if err != nil {
+		t.Fatalf("cannot get %s [%s]", "/foo/bar", err.Error())
 	}
 
-	_, err = s.TestAndSet("fooo", "bar", "barbar", time.Unix(0, 0), 5)
+	if e.Value != "barbar" {
+		t.Fatalf("expect value of %s is barbar [%s]", "/foo/bar", e.Value)
+	}
 
-	if err == nil {
-		t.Fatalf("test bar == non-existing key should fail")
+	// create a directory, update its ttl, to see if it will be deleted
+	_, err = s.Create("/foo/foo", "", false, false, Permanent, 3, 1)
+
+	if err != nil {
+		t.Fatalf("cannot create dir [%s] [%s]", "/foo/foo", err.Error())
+	}
+
+	_, err = s.Create("/foo/foo/foo1", "bar1", false, false, Permanent, 4, 1)
+
+	if err != nil {
+		t.Fatal("cannot create [%s]", err.Error())
+	}
+
+	_, err = s.Create("/foo/foo/foo2", "", false, false, Permanent, 5, 1)
+	if err != nil {
+		t.Fatal("cannot create [%s]", err.Error())
 	}
 
-	_, err = s.TestAndSet("fooo", "", "bar", time.Unix(0, 0), 6)
+	_, err = s.Create("/foo/foo/foo2/boo", "boo1", false, false, Permanent, 6, 1)
+	if err != nil {
+		t.Fatal("cannot create [%s]", err.Error())
+	}
 
+	expire := time.Now().Add(time.Second * 2)
+	_, err = s.Update("/foo/foo", "", expire, 7, 1)
 	if err != nil {
-		t.Fatalf("test empty == non-existing key should succeed")
+		t.Fatalf("cannot update dir [%s] [%s]", "/foo/foo", err.Error())
+	}
+
+	// sleep 50ms, it should still reach the node
+	time.Sleep(time.Microsecond * 50)
+	e, err = s.Get("/foo/foo", true, false, 7, 1)
+
+	if err != nil || e.Key != "/foo/foo" {
+		t.Fatalf("cannot get dir before expiration [%s]", err.Error())
+	}
+
+	if e.KVPairs[0].Key != "/foo/foo/foo1" || e.KVPairs[0].Value != "bar1" {
+		t.Fatalf("cannot get sub node before expiration [%s]", err.Error())
+	}
+
+	if e.KVPairs[1].Key != "/foo/foo/foo2" || e.KVPairs[1].Dir != true {
+		t.Fatalf("cannot get sub dir before expiration [%s]", err.Error())
+	}
+
+	/*if e.KVPairs[2].Key != "/foo/foo/foo2/boo" || e.KVPairs[2].Value != "boo1" {
+		t.Fatalf("cannot get sub node of sub dir before expiration [%s]", err.Error())
+	}*/
+
+	// wait for expiration
+	time.Sleep(time.Second * 3)
+	e, err = s.Get("/foo/foo", true, false, 7, 1)
+
+	if err == nil {
+		t.Fatal("still can get dir after expiration [%s]")
+	}
+
+	_, err = s.Get("/foo/foo/foo1", true, false, 7, 1)
+	if err == nil {
+		t.Fatal("still can get sub node after expiration [%s]")
+	}
+
+	_, err = s.Get("/foo/foo/foo2", true, false, 7, 1)
+	if err == nil {
+		t.Fatal("still can get sub dir after expiration [%s]")
+	}
+
+	_, err = s.Get("/foo/foo/foo2/boo", true, false, 7, 1)
+	if err == nil {
+		t.Fatalf("still can get sub node of sub dir after expiration [%s]", err.Error())
 	}
 
 }
 
-func TestSaveAndRecovery(t *testing.T) {
+func TestListDirectory(t *testing.T) {
+	s := New()
 
-	s := CreateStore(100)
-	s.Set("foo", "bar", time.Unix(0, 0), 1)
-	s.Set("foo2", "bar2", time.Now().Add(time.Second*5), 2)
-	state, err := s.Save()
+	// create dir /foo
+	// set key-value /foo/foo=bar
+	s.Create("/foo/foo", "bar", false, false, Permanent, 1, 1)
+
+	// create dir /foo/fooDir
+	// set key-value /foo/fooDir/foo=bar
+	s.Create("/foo/fooDir/foo", "bar", false, false, Permanent, 2, 1)
+
+	e, err := s.Get("/foo", true, false, 2, 1)
 
 	if err != nil {
-		t.Fatalf("Cannot Save %s", err)
+		t.Fatalf("%v", err)
+	}
+
+	if len(e.KVPairs) != 2 {
+		t.Fatalf("wrong number of kv pairs [%d/2]", len(e.KVPairs))
 	}
 
-	newStore := CreateStore(100)
+	if e.KVPairs[0].Key != "/foo/foo" || e.KVPairs[0].Value != "bar" {
+		t.Fatalf("wrong kv [/foo/foo/ / %s] -> [bar / %s]", e.KVPairs[0].Key, e.KVPairs[0].Value)
+	}
+
+	if e.KVPairs[1].Key != "/foo/fooDir" || e.KVPairs[1].Dir != true {
+		t.Fatalf("wrong kv [/foo/fooDir/ / %s] -> [true / %v]", e.KVPairs[1].Key, e.KVPairs[1].Dir)
+	}
+
+	if e.KVPairs[1].KVPairs[0].Key != "/foo/fooDir/foo" || e.KVPairs[1].KVPairs[0].Value != "bar" {
+		t.Fatalf("wrong kv [/foo/fooDir/foo / %s] -> [bar / %v]", e.KVPairs[1].KVPairs[0].Key, e.KVPairs[1].KVPairs[0].Value)
+	}
+	// test hidden node
+
+	// create dir /foo/_hidden
+	// set key-value /foo/_hidden/foo -> bar
+	s.Create("/foo/_hidden/foo", "bar", false, false, Permanent, 3, 1)
+
+	e, _ = s.Get("/foo", false, false, 2, 1)
+
+	if len(e.KVPairs) != 2 {
+		t.Fatalf("hidden node is not hidden! %s", e.KVPairs[2].Key)
+	}
+}
 
-	// wait for foo2 expires
-	time.Sleep(time.Second * 6)
+func TestRemove(t *testing.T) {
+	s := New()
 
-	newStore.Recovery(state)
+	s.Create("/foo", "bar", false, false, Permanent, 1, 1)
+	_, err := s.Delete("/foo", false, 1, 1)
 
-	res, err := newStore.Get("foo")
+	if err != nil {
+		t.Fatalf("cannot delete %s [%s]", "/foo", err.Error())
+	}
 
-	var result Response
-	json.Unmarshal(res, &result)
+	_, err = s.Get("/foo", false, false, 1, 1)
 
-	if result.Value != "bar" {
-		t.Fatalf("Recovery Fail")
+	if err == nil || err.Error() != "Key Not Found" {
+		t.Fatalf("can get the node after deletion")
 	}
 
-	res, err = newStore.Get("foo2")
+	s.Create("/foo/bar", "bar", false, false, Permanent, 1, 1)
+	s.Create("/foo/car", "car", false, false, Permanent, 1, 1)
+	s.Create("/foo/dar/dar", "dar", false, false, Permanent, 1, 1)
+
+	_, err = s.Delete("/foo", false, 1, 1)
 
 	if err == nil {
-		t.Fatalf("Get expired value")
+		t.Fatalf("should not be able to delete a directory without recursive")
 	}
 
-	s.Delete("foo", 3)
+	_, err = s.Delete("/foo", true, 1, 1)
 
+	if err != nil {
+		t.Fatalf("cannot delete %s [%s]", "/foo", err.Error())
+	}
+
+	_, err = s.Get("/foo", false, false, 1, 1)
+
+	if err == nil || err.Error() != "Key Not Found" {
+		t.Fatalf("can get the node after deletion ")
+	}
 }
 
 func TestExpire(t *testing.T) {
-	// test expire
-	s := CreateStore(100)
-	s.Set("foo", "bar", time.Now().Add(time.Second*1), 0)
-	time.Sleep(2 * time.Second)
+	s := New()
+
+	expire := time.Now().Add(time.Second)
+
+	s.Create("/foo", "bar", false, false, expire, 1, 1)
+
+	_, err := s.Get("/foo", false, false, 1, 1)
+
+	if err != nil {
+		t.Fatalf("can not get the node")
+	}
+
+	time.Sleep(time.Second * 2)
 
-	_, err := s.Get("foo")
+	_, err = s.Get("/foo", false, false, 1, 1)
 
 	if err == nil {
-		t.Fatalf("Got expired value")
+		t.Fatalf("can get the node after expiration time")
 	}
 
-	//test change expire time
-	s.Set("foo", "bar", time.Now().Add(time.Second*10), 1)
+	// test if we can reach the node before expiration
+	expire = time.Now().Add(time.Second)
+	s.Create("/foo", "bar", false, false, expire, 1, 1)
 
-	_, err = s.Get("foo")
+	time.Sleep(time.Millisecond * 50)
+	_, err = s.Get("/foo", false, false, 1, 1)
 
 	if err != nil {
-		t.Fatalf("Cannot get Value")
+		t.Fatalf("cannot get the node before expiration", err.Error())
 	}
 
-	s.Set("foo", "barbar", time.Now().Add(time.Second*1), 2)
+	expire = time.Now().Add(time.Second)
 
-	time.Sleep(2 * time.Second)
+	s.Create("/foo", "bar", false, false, expire, 1, 1)
+	_, err = s.Delete("/foo", false, 1, 1)
 
-	_, err = s.Get("foo")
+	if err != nil {
+		t.Fatalf("cannot delete the node before expiration", err.Error())
+	}
+}
+
+func TestTestAndSet(t *testing.T) { // TODO prevValue == nil ?
+	s := New()
+	s.Create("/foo", "bar", false, false, Permanent, 1, 1)
 
+	// test on wrong previous value
+	_, err := s.TestAndSet("/foo", "barbar", 0, "car", Permanent, 2, 1)
 	if err == nil {
-		t.Fatalf("Got expired value")
+		t.Fatal("test and set should fail barbar != bar")
 	}
 
-	// test change expire to stable
-	s.Set("foo", "bar", time.Now().Add(time.Second*1), 3)
+	// test on value
+	e, err := s.TestAndSet("/foo", "bar", 0, "car", Permanent, 3, 1)
 
-	s.Set("foo", "bar", time.Unix(0, 0), 4)
+	if err != nil {
+		t.Fatal("test and set should succeed bar == bar")
+	}
 
-	time.Sleep(2 * time.Second)
+	if e.PrevValue != "bar" || e.Value != "car" {
+		t.Fatalf("[%v/%v] [%v/%v]", e.PrevValue, "bar", e.Value, "car")
+	}
 
-	_, err = s.Get("foo")
+	// test on index
+	e, err = s.TestAndSet("/foo", "", 3, "bar", Permanent, 4, 1)
 
 	if err != nil {
-		t.Fatalf("Cannot get Value")
+		t.Fatal("test and set should succeed index 3 == 3")
 	}
 
-	// test stable to expire
-	s.Set("foo", "bar", time.Now().Add(time.Second*1), 5)
-	time.Sleep(2 * time.Second)
-	_, err = s.Get("foo")
+	if e.PrevValue != "car" || e.Value != "bar" {
+		t.Fatalf("[%v/%v] [%v/%v]", e.PrevValue, "car", e.Value, "bar")
+	}
+}
 
-	if err == nil {
-		t.Fatalf("Got expired value")
+func TestWatch(t *testing.T) {
+	s := New()
+	// 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)
+
+	e := nonblockingRetrive(c)
+	if e.Key != "/foo/foo/foo" || e.Action != Create {
+		t.Fatal("watch for Create node fails ", e)
 	}
 
-	// test set older node
-	s.Set("foo", "bar", time.Now().Add(-time.Second*1), 6)
-	_, err = s.Get("foo")
+	c, _ = s.Watch("/foo/foo/foo", false, 0, 1, 1)
+	s.Update("/foo/foo/foo", "car", Permanent, 2, 1)
+	e = nonblockingRetrive(c)
+	if e.Key != "/foo/foo/foo" || e.Action != Update {
+		t.Fatal("watch for Update node fails ", e)
+	}
 
-	if err == nil {
-		t.Fatalf("Got expired value")
+	c, _ = s.Watch("/foo/foo/foo", false, 0, 2, 1)
+	s.TestAndSet("/foo/foo/foo", "car", 0, "bar", Permanent, 3, 1)
+	e = nonblockingRetrive(c)
+	if e.Key != "/foo/foo/foo" || e.Action != TestAndSet {
+		t.Fatal("watch for TestAndSet node fails")
 	}
 
-}
+	c, _ = s.Watch("/foo/foo/foo", false, 0, 3, 1)
+	s.Delete("/foo", true, 4, 1) //recursively delete
+	e = nonblockingRetrive(c)
+	if e.Key != "/foo" || e.Action != Delete {
+		t.Fatal("watch for Delete node fails ", e)
+	}
+
+	// watch at a prefix
+	c, _ = s.Watch("/foo", true, 0, 4, 1)
+	s.Create("/foo/foo/boo", "bar", false, false, Permanent, 5, 1)
+	e = nonblockingRetrive(c)
+	if e.Key != "/foo/foo/boo" || e.Action != Create {
+		t.Fatal("watch for Create subdirectory fails")
+	}
 
-func BenchmarkStoreSet(b *testing.B) {
-	s := CreateStore(100)
+	c, _ = s.Watch("/foo", true, 0, 5, 1)
+	s.Update("/foo/foo/boo", "foo", Permanent, 6, 1)
+	e = nonblockingRetrive(c)
+	if e.Key != "/foo/foo/boo" || e.Action != Update {
+		t.Fatal("watch for Update subdirectory fails")
+	}
 
-	keys := GenKeys(10000, 5)
+	c, _ = s.Watch("/foo", true, 0, 6, 1)
+	s.TestAndSet("/foo/foo/boo", "foo", 0, "bar", Permanent, 7, 1)
+	e = nonblockingRetrive(c)
+	if e.Key != "/foo/foo/boo" || e.Action != TestAndSet {
+		t.Fatal("watch for TestAndSet subdirectory fails")
+	}
 
-	b.ResetTimer()
-	for i := 0; i < b.N; i++ {
+	c, _ = s.Watch("/foo", true, 0, 7, 1)
+	s.Delete("/foo/foo/boo", false, 8, 1)
+	e = nonblockingRetrive(c)
+	if e.Key != "/foo/foo/boo" || e.Action != Delete {
+		t.Fatal("watch for Delete subdirectory fails")
+	}
 
-		for i, key := range keys {
-			s.Set(key, "barbarbarbarbar", time.Unix(0, 0), uint64(i))
-		}
+	// watch expire
+	s.Create("/foo/foo/boo", "foo", false, false, time.Now().Add(time.Second*1), 9, 1)
+	c, _ = s.Watch("/foo", true, 0, 9, 1)
+	time.Sleep(time.Second * 2)
+	e = nonblockingRetrive(c)
+	if e.Key != "/foo/foo/boo" || e.Action != Expire || e.Index != 9 {
+		t.Fatal("watch for Expiration of Create() subdirectory fails ", e)
+	}
 
-		s = CreateStore(100)
+	s.Create("/foo/foo/boo", "foo", false, false, Permanent, 10, 1)
+	s.Update("/foo/foo/boo", "bar", time.Now().Add(time.Second*1), 11, 1)
+	c, _ = s.Watch("/foo", true, 0, 11, 1)
+	time.Sleep(time.Second * 2)
+	e = nonblockingRetrive(c)
+	if e.Key != "/foo/foo/boo" || e.Action != Expire || e.Index != 11 {
+		t.Fatal("watch for Expiration of Update() subdirectory fails ", e)
+	}
+
+	s.Create("/foo/foo/boo", "foo", false, false, Permanent, 12, 1)
+	s.TestAndSet("/foo/foo/boo", "foo", 0, "bar", time.Now().Add(time.Second*1), 13, 1)
+	c, _ = s.Watch("/foo", true, 0, 13, 1)
+	time.Sleep(time.Second * 2)
+	e = nonblockingRetrive(c)
+	if e.Key != "/foo/foo/boo" || e.Action != Expire || e.Index != 13 {
+		t.Fatal("watch for Expiration of TestAndSet() subdirectory fails ", e)
 	}
 }
 
-func BenchmarkStoreGet(b *testing.B) {
-	s := CreateStore(100)
+func TestSort(t *testing.T) {
+	s := New()
 
-	keys := GenKeys(10000, 5)
+	// simulating random creation
+	keys := GenKeys(80, 4)
 
-	for i, key := range keys {
-		s.Set(key, "barbarbarbarbar", time.Unix(0, 0), uint64(i))
+	i := uint64(1)
+	for _, k := range keys {
+		_, err := s.Create(k, "bar", false, false, Permanent, i, 1)
+		if err != nil {
+			panic(err)
+		} else {
+			i++
+		}
 	}
 
-	b.ResetTimer()
-	for i := 0; i < b.N; i++ {
+	e, err := s.Get("/foo", true, true, i, 1)
+	if err != nil {
+		t.Fatalf("get dir nodes failed [%s]", err.Error())
+	}
+
+	for i, k := range e.KVPairs[:len(e.KVPairs)-1] {
 
-		for _, key := range keys {
-			s.Get(key)
+		if k.Key >= e.KVPairs[i+1].Key {
+			t.Fatalf("sort failed, [%s] should be placed after [%s]", k.Key, e.KVPairs[i+1].Key)
+		}
+
+		if k.Dir {
+			recursiveTestSort(k, t)
 		}
 
 	}
+
+	if k := e.KVPairs[len(e.KVPairs)-1]; k.Dir {
+		recursiveTestSort(k, t)
+	}
 }
 
-func BenchmarkStoreSnapshotCopy(b *testing.B) {
-	s := CreateStore(100)
+func TestSaveAndRecover(t *testing.T) {
+	s := New()
+
+	// simulating random creation
+	keys := GenKeys(8, 4)
+
+	i := uint64(1)
+	for _, k := range keys {
+		_, err := s.Create(k, "bar", false, false, Permanent, i, 1)
+		if err != nil {
+			panic(err)
+		} else {
+			i++
+		}
+	}
+
+	// create a node with expiration
+	// test if we can reach the node before expiration
+
+	expire := time.Now().Add(time.Second)
+	s.Create("/foo/foo", "bar", false, false, expire, 1, 1)
+	b, err := s.Save()
+
+	cloneFs := New()
+	time.Sleep(2 * time.Second)
 
-	keys := GenKeys(10000, 5)
+	cloneFs.Recovery(b)
 
-	for i, key := range keys {
-		s.Set(key, "barbarbarbarbar", time.Unix(0, 0), uint64(i))
+	for i, k := range keys {
+		_, err := cloneFs.Get(k, false, false, uint64(i), 1)
+		if err != nil {
+			panic(err)
+		}
 	}
 
-	var state []byte
+	// lock to avoid racing with Expire()
+	s.worldLock.RLock()
+	defer s.worldLock.RUnlock()
 
-	b.ResetTimer()
-	for i := 0; i < b.N; i++ {
-		s.clone()
+	if s.WatcherHub.EventHistory.StartIndex != cloneFs.WatcherHub.EventHistory.StartIndex {
+		t.Fatalf("Error recovered event history start index[%v/%v]",
+			s.WatcherHub.EventHistory.StartIndex, cloneFs.WatcherHub.EventHistory.StartIndex)
 	}
-	b.SetBytes(int64(len(state)))
-}
 
-func BenchmarkSnapshotSaveJson(b *testing.B) {
-	s := CreateStore(100)
+	for i = 0; int(i) < cloneFs.WatcherHub.EventHistory.Queue.Size; i++ {
+		if s.WatcherHub.EventHistory.Queue.Events[i].Key !=
+			cloneFs.WatcherHub.EventHistory.Queue.Events[i].Key {
+			t.Fatal("Error recovered event history")
+		}
+	}
 
-	keys := GenKeys(10000, 5)
+	_, err = s.Get("/foo/foo", false, false, 1, 1)
 
-	for i, key := range keys {
-		s.Set(key, "barbarbarbarbar", time.Unix(0, 0), uint64(i))
+	if err == nil || err.Error() != "Key Not Found" {
+		t.Fatalf("can get the node after deletion ")
 	}
+}
+
+// GenKeys randomly generate num of keys with max depth
+func GenKeys(num int, depth int) []string {
+	rand.Seed(time.Now().UnixNano())
+	keys := make([]string, num)
+	for i := 0; i < num; i++ {
 
-	var state []byte
+		keys[i] = "/foo"
+		depth := rand.Intn(depth) + 1
 
-	b.ResetTimer()
-	for i := 0; i < b.N; i++ {
-		state, _ = s.Save()
+		for j := 0; j < depth; j++ {
+			keys[i] += "/" + strconv.Itoa(rand.Int())
+		}
 	}
-	b.SetBytes(int64(len(state)))
+
+	return keys
 }
 
-func BenchmarkSnapshotRecovery(b *testing.B) {
-	s := CreateStore(100)
+func createAndGet(s *Store, path string, t *testing.T) {
+	_, err := s.Create(path, "bar", false, false, Permanent, 1, 1)
+
+	if err != nil {
+		t.Fatalf("cannot create %s=bar [%s]", path, err.Error())
+	}
+
+	e, err := s.Get(path, false, false, 1, 1)
+
+	if err != nil {
+		t.Fatalf("cannot get %s [%s]", path, err.Error())
+	}
 
-	keys := GenKeys(10000, 5)
+	if e.Value != "bar" {
+		t.Fatalf("expect value of %s is bar [%s]", path, e.Value)
+	}
+}
 
-	for i, key := range keys {
-		s.Set(key, "barbarbarbarbar", time.Unix(0, 0), uint64(i))
+func recursiveTestSort(k KeyValuePair, t *testing.T) {
+	for i, v := range k.KVPairs[:len(k.KVPairs)-1] {
+		if v.Key >= k.KVPairs[i+1].Key {
+			t.Fatalf("sort failed, [%s] should be placed after [%s]", v.Key, k.KVPairs[i+1].Key)
+		}
+
+		if v.Dir {
+			recursiveTestSort(v, t)
+		}
 	}
 
-	state, _ := s.Save()
+	if v := k.KVPairs[len(k.KVPairs)-1]; v.Dir {
+		recursiveTestSort(v, t)
+	}
+}
 
-	b.ResetTimer()
-	for i := 0; i < b.N; i++ {
-		newStore := CreateStore(100)
-		newStore.Recovery(state)
+func nonblockingRetrive(c <-chan *Event) *Event {
+	select {
+	case e := <-c:
+		return e
+	default:
+		return nil
 	}
-	b.SetBytes(int64(len(state)))
 }

+ 0 - 21
store/test.go

@@ -1,21 +0,0 @@
-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
-}

+ 0 - 318
store/tree.go

@@ -1,318 +0,0 @@
-package store
-
-import (
-	"path"
-	"sort"
-	"strings"
-	"time"
-)
-
-//------------------------------------------------------------------------------
-//
-// Typedefs
-//
-//------------------------------------------------------------------------------
-
-// A file system like tree structure. Each non-leaf node of the tree has a hashmap to
-// store its children nodes. Leaf nodes has no hashmap (a nil pointer)
-type tree struct {
-	Root *treeNode
-}
-
-// A treeNode wraps a Node. It has a hashmap to keep records of its children treeNodes.
-type treeNode struct {
-	InternalNode Node
-	Dir          bool
-	NodeMap      map[string]*treeNode
-}
-
-// TreeNode with its key. We use it when we need to sort the treeNodes.
-type tnWithKey struct {
-	key string
-	tn  *treeNode
-}
-
-// Define type and functions to match sort interface
-type tnWithKeySlice []tnWithKey
-
-func (s tnWithKeySlice) Len() int           { return len(s) }
-func (s tnWithKeySlice) Less(i, j int) bool { return s[i].key < s[j].key }
-func (s tnWithKeySlice) Swap(i, j int)      { s[i], s[j] = s[j], s[i] }
-
-// CONSTANT VARIABLE
-
-// Represent an empty node
-var emptyNode = Node{"", PERMANENT, nil}
-
-//------------------------------------------------------------------------------
-//
-// Methods
-//
-//------------------------------------------------------------------------------
-
-// Set the key to the given value, return true if success
-// If any intermidate path of the key is not a directory type, it will fail
-// For example if the /foo = Node(bar) exists, set /foo/foo = Node(barbar)
-// will fail.
-func (t *tree) set(key string, value Node) bool {
-
-	nodesName := split(key)
-
-	// avoid set value to "/"
-	if len(nodesName) == 1 && len(nodesName[0]) == 0 {
-		return false
-	}
-
-	nodeMap := t.Root.NodeMap
-
-	i := 0
-	newDir := false
-
-	// go through all the path
-	for i = 0; i < len(nodesName)-1; i++ {
-
-		// if we meet a new directory, all the directory after it must be new
-		if newDir {
-			tn := &treeNode{emptyNode, true, make(map[string]*treeNode)}
-			nodeMap[nodesName[i]] = tn
-			nodeMap = tn.NodeMap
-			continue
-		}
-
-		// get the node from the nodeMap of the current level
-		tn, ok := nodeMap[nodesName[i]]
-
-		if !ok {
-			// add a new directory and set newDir to true
-			newDir = true
-			tn := &treeNode{emptyNode, true, make(map[string]*treeNode)}
-			nodeMap[nodesName[i]] = tn
-			nodeMap = tn.NodeMap
-
-		} else if ok && !tn.Dir {
-
-			// if we meet a non-directory node, we cannot set the key
-			return false
-		} else {
-
-			// update the nodeMap to next level
-			nodeMap = tn.NodeMap
-		}
-
-	}
-
-	// Add the last node
-	tn, ok := nodeMap[nodesName[i]]
-
-	if !ok {
-		// we add a new treeNode
-		tn := &treeNode{value, false, nil}
-		nodeMap[nodesName[i]] = tn
-
-	} else {
-		if tn.Dir {
-			return false
-		}
-		// we change the value of a old Treenode
-		tn.InternalNode = value
-	}
-	return true
-
-}
-
-// Get the tree node of the key
-func (t *tree) internalGet(key string) (*treeNode, bool) {
-	nodesName := split(key)
-
-	// should be able to get root
-	if len(nodesName) == 1 && nodesName[0] == "" {
-		return t.Root, true
-	}
-
-	nodeMap := t.Root.NodeMap
-
-	var i int
-
-	for i = 0; i < len(nodesName)-1; i++ {
-		node, ok := nodeMap[nodesName[i]]
-		if !ok || !node.Dir {
-			return nil, false
-		}
-		nodeMap = node.NodeMap
-	}
-
-	tn, ok := nodeMap[nodesName[i]]
-	if ok {
-		return tn, ok
-	} else {
-		return nil, ok
-	}
-}
-
-// get the internalNode of the key
-func (t *tree) get(key string) (Node, bool) {
-	tn, ok := t.internalGet(key)
-
-	if ok {
-		if tn.Dir {
-			return emptyNode, false
-		}
-		return tn.InternalNode, ok
-	} else {
-		return emptyNode, ok
-	}
-}
-
-// get the internalNode of the key
-func (t *tree) list(directory string) (interface{}, []string, bool) {
-	treeNode, ok := t.internalGet(directory)
-
-	if !ok {
-		return nil, nil, ok
-
-	} else {
-		if !treeNode.Dir {
-			return &treeNode.InternalNode, nil, ok
-		}
-		length := len(treeNode.NodeMap)
-		nodes := make([]*Node, length)
-		keys := make([]string, length)
-
-		i := 0
-		for key, node := range treeNode.NodeMap {
-			nodes[i] = &node.InternalNode
-			keys[i] = key
-			i++
-		}
-
-		return nodes, keys, ok
-	}
-}
-
-// delete the key, return true if success
-func (t *tree) delete(key string) bool {
-	nodesName := split(key)
-
-	nodeMap := t.Root.NodeMap
-
-	var i int
-
-	for i = 0; i < len(nodesName)-1; i++ {
-		node, ok := nodeMap[nodesName[i]]
-		if !ok || !node.Dir {
-			return false
-		}
-		nodeMap = node.NodeMap
-	}
-
-	node, ok := nodeMap[nodesName[i]]
-	if ok && !node.Dir {
-		delete(nodeMap, nodesName[i])
-		return true
-	}
-	return false
-}
-
-// traverse wrapper
-func (t *tree) traverse(f func(string, *Node), sort bool) {
-	if sort {
-		sortDfs("", t.Root, f)
-	} else {
-		dfs("", t.Root, f)
-	}
-}
-
-// 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
-// apply the func f to each internal node
-func dfs(key string, t *treeNode, f func(string, *Node)) {
-
-	// base case
-	if len(t.NodeMap) == 0 {
-		f(key, &t.InternalNode)
-
-		// recursion
-	} else {
-		for tnKey, tn := range t.NodeMap {
-			tnKey := key + "/" + tnKey
-			dfs(tnKey, tn, f)
-		}
-	}
-}
-
-// sort deep first search to traverse the tree
-// apply the func f to each internal node
-func sortDfs(key string, t *treeNode, f func(string, *Node)) {
-	// base case
-	if len(t.NodeMap) == 0 {
-		f(key, &t.InternalNode)
-
-		// recursion
-	} else {
-
-		s := make(tnWithKeySlice, len(t.NodeMap))
-		i := 0
-
-		// copy
-		for tnKey, tn := range t.NodeMap {
-			tnKey := key + "/" + tnKey
-			s[i] = tnWithKey{tnKey, tn}
-			i++
-		}
-
-		// sort
-		sort.Sort(s)
-
-		// traverse
-		for i = 0; i < len(t.NodeMap); i++ {
-			sortDfs(s[i].key, s[i].tn, f)
-		}
-	}
-}
-
-// split the key by '/', get the intermediate node name
-func split(key string) []string {
-	key = "/" + key
-	key = path.Clean(key)
-
-	// get the intermidate nodes name
-	nodesName := strings.Split(key, "/")
-	// we do not need the root node, since we start with it
-	nodesName = nodesName[1:]
-	return nodesName
-}

+ 0 - 247
store/tree_store_test.go

@@ -1,247 +0,0 @@
-package store
-
-import (
-	"fmt"
-	"math/rand"
-	"strconv"
-	"testing"
-	"time"
-)
-
-func TestStoreGet(t *testing.T) {
-
-	ts := &tree{
-		&treeNode{
-			NewTestNode("/"),
-			true,
-			make(map[string]*treeNode),
-		},
-	}
-
-	// create key
-	ts.set("/foo", NewTestNode("bar"))
-	// change value
-	ts.set("/foo", NewTestNode("barbar"))
-	// create key
-	ts.set("/hello/foo", NewTestNode("barbarbar"))
-	treeNode, ok := ts.get("/foo")
-
-	if !ok {
-		t.Fatalf("Expect to get node, but not")
-	}
-	if treeNode.Value != "barbar" {
-		t.Fatalf("Expect value barbar, but got %s", treeNode.Value)
-	}
-
-	// create key
-	treeNode, ok = ts.get("/hello/foo")
-	if !ok {
-		t.Fatalf("Expect to get node, but not")
-	}
-	if treeNode.Value != "barbarbar" {
-		t.Fatalf("Expect value barbarbar, but got %s", treeNode.Value)
-	}
-
-	// create a key under other key
-	ok = ts.set("/foo/foo", NewTestNode("bar"))
-	if ok {
-		t.Fatalf("shoud not add key under a exisiting key")
-	}
-
-	// delete a key
-	ok = ts.delete("/foo")
-	if !ok {
-		t.Fatalf("cannot delete key")
-	}
-
-	// delete a directory
-	ok = ts.delete("/hello")
-	if ok {
-		t.Fatalf("Expect cannot delet /hello, but deleted! ")
-	}
-
-	// test list
-	ts.set("/hello/fooo", NewTestNode("barbarbar"))
-	ts.set("/hello/foooo/foo", NewTestNode("barbarbar"))
-
-	nodes, keys, ok := ts.list("/hello")
-
-	if !ok {
-		t.Fatalf("cannot list!")
-	} else {
-		nodes, _ := nodes.([]*Node)
-		length := len(nodes)
-
-		for i := 0; i < length; i++ {
-			fmt.Println(keys[i], "=", nodes[i].Value)
-		}
-	}
-
-	keys = GenKeys(100, 10)
-
-	for i := 0; i < 100; i++ {
-		value := strconv.Itoa(rand.Int())
-		ts.set(keys[i], NewTestNode(value))
-		treeNode, ok := ts.get(keys[i])
-
-		if !ok {
-			continue
-		}
-		if treeNode.Value != value {
-			t.Fatalf("Expect value %s, but got %s", value, treeNode.Value)
-		}
-
-	}
-	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) {
-	return
-}
-
-func NewTestNode(value string) Node {
-	return Node{value, time.Unix(0, 0), nil}
-}

+ 83 - 87
store/watcher.go

@@ -1,129 +1,125 @@
 package store
 
 import (
+	"container/list"
 	"path"
-	"strconv"
 	"strings"
-)
+	"sync/atomic"
 
-//------------------------------------------------------------------------------
-//
-// Typedefs
-//
-//------------------------------------------------------------------------------
+	etcdErr "github.com/coreos/etcd/error"
+)
 
-// WatcherHub is where the client register its watcher
-type WatcherHub struct {
-	watchers map[string][]*Watcher
+type watcherHub struct {
+	watchers     map[string]*list.List
+	count        int64 // current number of watchers.
+	EventHistory *EventHistory
 }
 
-// Currently watcher only contains a response channel
-type Watcher struct {
-	C chan *Response
+type watcher struct {
+	eventChan  chan *Event
+	recursive  bool
+	sinceIndex uint64
 }
 
-// Create a new watcherHub
-func newWatcherHub() *WatcherHub {
-	w := new(WatcherHub)
-	w.watchers = make(map[string][]*Watcher)
-	return w
+func newWatchHub(capacity int) *watcherHub {
+	return &watcherHub{
+		watchers:     make(map[string]*list.List),
+		EventHistory: newEventHistory(capacity),
+	}
 }
 
-// Create a new watcher
-func NewWatcher() *Watcher {
-	return &Watcher{C: make(chan *Response, 1)}
-}
+// watch function returns an Event channel.
+// If recursive is true, the first change after index under prefix will be sent to the event channel.
+// If recursive is false, the first change after index at prefix will be sent to the event channel.
+// If index is zero, watch will start from the current index + 1.
+func (wh *watcherHub) watch(prefix string, recursive bool, index uint64) (<-chan *Event, *etcdErr.Error) {
+	eventChan := make(chan *Event, 1)
 
-// Add a watcher to the watcherHub
-func (w *WatcherHub) addWatcher(prefix string, watcher *Watcher, sinceIndex uint64,
-	responseStartIndex uint64, currentIndex uint64, resMap map[string]*Response) error {
+	e, err := wh.EventHistory.scan(prefix, index)
 
-	prefix = path.Clean("/" + prefix)
+	if err != nil {
+		return nil, err
+	}
 
-	if sinceIndex != 0 && sinceIndex >= responseStartIndex {
-		for i := sinceIndex; i <= currentIndex; i++ {
-			if checkResponse(prefix, i, resMap) {
-				watcher.C <- resMap[strconv.FormatUint(i, 10)]
-				return nil
-			}
-		}
+	if e != nil {
+		eventChan <- e
+		return eventChan, nil
 	}
 
-	_, ok := w.watchers[prefix]
+	w := &watcher{
+		eventChan:  eventChan,
+		recursive:  recursive,
+		sinceIndex: index - 1, // to catch Expire()
+	}
 
-	if !ok {
-		w.watchers[prefix] = make([]*Watcher, 0)
+	l, ok := wh.watchers[prefix]
+
+	if ok { // add the new watcher to the back of the list
+		l.PushBack(w)
+
+	} else { // create a new list and add the new watcher
+		l := list.New()
+		l.PushBack(w)
+		wh.watchers[prefix] = l
 	}
 
-	w.watchers[prefix] = append(w.watchers[prefix], watcher)
+	atomic.AddInt64(&wh.count, 1)
 
-	return nil
+	return eventChan, nil
 }
 
-// Check if the response has what we are watching
-func checkResponse(prefix string, index uint64, resMap map[string]*Response) bool {
+func (wh *watcherHub) notifyWithPath(e *Event, path string, force bool) {
+	l, ok := wh.watchers[path]
 
-	resp, ok := resMap[strconv.FormatUint(index, 10)]
+	if ok {
+		curr := l.Front()
+		notifiedAll := true
 
-	if !ok {
-		// not storage system command
-		return false
-	} else {
-		path := resp.Key
-		if strings.HasPrefix(path, prefix) {
-			prefixLen := len(prefix)
-			if len(path) == prefixLen || path[prefixLen] == '/' {
-				return true
+		for {
+			if curr == nil { // we have reached the end of the list
+				if notifiedAll {
+					// if we have notified all watcher in the list
+					// we can delete the list
+					delete(wh.watchers, path)
+				}
+
+				break
+			}
+
+			next := curr.Next() // save the next
+
+			w, _ := curr.Value.(*watcher)
+			if (w.recursive || force || e.Key == path) && e.Index >= w.sinceIndex {
+				w.eventChan <- e
+				l.Remove(curr)
+				atomic.AddInt64(&wh.count, -1)
+			} else {
+				notifiedAll = false
 			}
 
+			curr = next // go to the next one
 		}
 	}
-
-	return false
 }
 
-// Notify the watcher a action happened
-func (w *WatcherHub) notify(resp Response) error {
-	resp.Key = path.Clean(resp.Key)
+func (wh *watcherHub) notify(e *Event) {
+	e = wh.EventHistory.addEvent(e)
+
+	segments := strings.Split(e.Key, "/")
 
-	segments := strings.Split(resp.Key, "/")
 	currPath := "/"
 
-	// walk through all the pathes
+	// walk through all the paths
 	for _, segment := range segments {
 		currPath = path.Join(currPath, segment)
-
-		watchers, ok := w.watchers[currPath]
-
-		if ok {
-
-			newWatchers := make([]*Watcher, 0)
-			// notify all the watchers
-			for _, watcher := range watchers {
-				watcher.C <- &resp
-			}
-
-			if len(newWatchers) == 0 {
-				// we have notified all the watchers at this path
-				// delete the map
-				delete(w.watchers, currPath)
-			} else {
-				w.watchers[currPath] = newWatchers
-			}
-		}
-
+		wh.notifyWithPath(e, currPath, false)
 	}
-
-	return nil
 }
 
-// stopWatchers stops all the watchers
-// This function is used when the etcd recovery from a snapshot at runtime
-func (w *WatcherHub) stopWatchers() {
-	for _, subWatchers := range w.watchers {
-		for _, watcher := range subWatchers {
-			watcher.C <- nil
-		}
+func (wh *watcherHub) clone() *watcherHub {
+	clonedHistory := wh.EventHistory.clone()
+
+	return &watcherHub{
+		EventHistory: clonedHistory,
 	}
-	w.watchers = nil
 }

+ 31 - 60
store/watcher_test.go

@@ -2,83 +2,54 @@ package store
 
 import (
 	"testing"
-	"time"
 )
 
-func TestWatch(t *testing.T) {
-
-	s := CreateStore(100)
-
-	watchers := make([]*Watcher, 10)
-
-	for i, _ := range watchers {
-
-		// create a new watcher
-		watchers[i] = NewWatcher()
-		// add to the watchers list
-		s.AddWatcher("foo", watchers[i], 0)
-
+func TestWatcher(t *testing.T) {
+	s := New()
+	wh := s.WatcherHub
+	c, err := wh.watch("/foo", true, 1)
+	if err != nil {
+		t.Fatal("%v", err)
 	}
 
-	s.Set("/foo/foo", "bar", time.Unix(0, 0), 1)
-
-	for _, watcher := range watchers {
-
-		// wait for the notification for any changing
-		res := <-watcher.C
-
-		if res == nil {
-			t.Fatal("watcher is cleared")
-		}
+	select {
+	case <-c:
+		t.Fatal("should not receive from channel before send the event")
+	default:
+		// do nothing
 	}
 
-	for i, _ := range watchers {
+	e := newEvent(Create, "/foo/bar", 1, 1)
+
+	wh.notify(e)
 
-		// create a new watcher
-		watchers[i] = NewWatcher()
-		// add to the watchers list
-		s.AddWatcher("foo/foo/foo", watchers[i], 0)
+	re := <-c
 
+	if e != re {
+		t.Fatal("recv != send")
 	}
 
-	s.watcher.stopWatchers()
+	c, _ = wh.watch("/foo", false, 2)
 
-	for _, watcher := range watchers {
+	e = newEvent(Create, "/foo/bar", 2, 1)
 
-		// wait for the notification for any changing
-		res := <-watcher.C
+	wh.notify(e)
 
-		if res != nil {
-			t.Fatal("watcher is cleared")
-		}
+	select {
+	case re = <-c:
+		t.Fatal("should not receive from channel if not recursive ", re)
+	default:
+		// do nothing
 	}
-}
 
-// BenchmarkWatch creates 10K watchers watch at /foo/[path] each time.
-// Path is randomly chosen with max depth 10.
-// It should take less than 15ms to wake up 10K watchers.
-func BenchmarkWatch(b *testing.B) {
-	s := CreateStore(100)
+	e = newEvent(Create, "/foo", 3, 1)
 
-	keys := GenKeys(10000, 10)
+	wh.notify(e)
 
-	b.ResetTimer()
-	for i := 0; i < b.N; i++ {
-		watchers := make([]*Watcher, 10000)
-		for i := 0; i < 10000; i++ {
-			// create a new watcher
-			watchers[i] = NewWatcher()
-			// add to the watchers list
-			s.AddWatcher(keys[i], watchers[i], 0)
-		}
+	re = <-c
 
-		s.watcher.stopWatchers()
-
-		for _, watcher := range watchers {
-			// wait for the notification for any changing
-			<-watcher.C
-		}
-
-		s.watcher = newWatcherHub()
+	if e != re {
+		t.Fatal("recv != send")
 	}
+
 }

+ 13 - 15
third_party/github.com/coreos/go-log/log/commands.go

@@ -1,4 +1,5 @@
 package log
+
 // Copyright 2013, David Fisher. All rights reserved.
 //
 // Licensed under the Apache License, Version 2.0 (the "License");
@@ -42,7 +43,6 @@ 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,7 +99,6 @@ func (logger *Logger) Debugf(format string, v ...interface{}) {
 	logger.Log(PriDebug, fmt.Sprintf(format, v...))
 }
 
-
 func Emergency(v ...interface{}) {
 	defaultLogger.Log(PriEmerg, v...)
 }
@@ -158,57 +157,56 @@ 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 - 0
third_party/github.com/coreos/go-log/log/fields.go

@@ -1,4 +1,5 @@
 package log
+
 // Copyright 2013, David Fisher. All rights reserved.
 //
 // Licensed under the Apache License, Version 2.0 (the "License");

+ 1 - 0
third_party/github.com/coreos/go-log/log/priority.go

@@ -1,4 +1,5 @@
 package log
+
 // Copyright 2013, David Fisher. All rights reserved.
 //
 // Licensed under the Apache License, Version 2.0 (the "License");

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

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

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

@@ -3,6 +3,7 @@ package journal
 
 import (
 	"bytes"
+	"encoding/binary"
 	"errors"
 	"fmt"
 	"io"
@@ -12,7 +13,6 @@ import (
 	"strconv"
 	"strings"
 	"syscall"
-	"encoding/binary"
 )
 
 // Priority of a journal message

+ 67 - 21
util.go

@@ -13,8 +13,11 @@ import (
 	"strconv"
 	"time"
 
+	etcdErr "github.com/coreos/etcd/error"
+	"github.com/coreos/etcd/store"
 	"github.com/coreos/etcd/web"
 	"github.com/coreos/go-log/log"
+	"github.com/coreos/go-raft"
 )
 
 //--------------------------------------
@@ -27,12 +30,12 @@ func durationToExpireTime(strDuration string) (time.Time, error) {
 		duration, err := strconv.Atoi(strDuration)
 
 		if err != nil {
-			return time.Unix(0, 0), err
+			return store.Permanent, err
 		}
 		return time.Now().Add(time.Second * (time.Duration)(duration)), nil
 
 	} else {
-		return time.Unix(0, 0), nil
+		return store.Permanent, nil
 	}
 }
 
@@ -44,7 +47,7 @@ var storeMsg chan string
 // Help to send msg from store to webHub
 func webHelper() {
 	storeMsg = make(chan string)
-	etcdStore.SetMessager(storeMsg)
+	// etcdStore.SetMessager(storeMsg)
 	for {
 		// transfer the new msg to webHub
 		web.Hub().Send(<-storeMsg)
@@ -64,6 +67,61 @@ func startWebInterface() {
 // HTTP Utilities
 //--------------------------------------
 
+func dispatch(c Command, w http.ResponseWriter, req *http.Request, toURL func(name string) (string, bool)) error {
+	if r.State() == raft.Leader {
+		if response, err := r.Do(c); err != nil {
+			return err
+		} else {
+			if response == nil {
+				return etcdErr.NewError(300, "Empty response from raft", store.UndefIndex, store.UndefTerm)
+			}
+
+			event, ok := response.(*store.Event)
+			if ok {
+				bytes, err := json.Marshal(event)
+				if err != nil {
+					fmt.Println(err)
+				}
+
+				w.Header().Add("X-Etcd-Index", fmt.Sprint(event.Index))
+				w.Header().Add("X-Etcd-Term", fmt.Sprint(event.Term))
+				w.WriteHeader(http.StatusOK)
+				w.Write(bytes)
+
+				return nil
+			}
+
+			bytes, _ := response.([]byte)
+			w.WriteHeader(http.StatusOK)
+			w.Write(bytes)
+
+			return nil
+		}
+
+	} else {
+		leader := r.Leader()
+		// current no leader
+		if leader == "" {
+			return etcdErr.NewError(300, "", store.UndefIndex, store.UndefTerm)
+		}
+		url, _ := toURL(leader)
+
+		redirect(url, w, req)
+
+		return nil
+	}
+}
+
+func redirect(hostname string, w http.ResponseWriter, req *http.Request) {
+	path := req.URL.Path
+
+	url := hostname + path
+
+	debugf("Redirect to %s", url)
+
+	http.Redirect(w, req, url, http.StatusTemporaryRedirect)
+}
+
 func decodeJsonRequest(req *http.Request, data interface{}) error {
 	decoder := json.NewDecoder(req.Body)
 	if err := decoder.Decode(&data); err != nil && err != io.EOF {
@@ -128,29 +186,17 @@ func sanitizeListenHost(listen string, advertised string) string {
 	return net.JoinHostPort(listen, aport)
 }
 
-func redirect(node string, etcd bool, w http.ResponseWriter, req *http.Request) {
-	var url string
-	path := req.URL.Path
-
-	if etcd {
-		etcdAddr, _ := nameToEtcdURL(node)
-		url = etcdAddr + path
-	} else {
-		raftAddr, _ := nameToRaftURL(node)
-		url = raftAddr + path
-	}
-
-	debugf("Redirect to %s", url)
-
-	http.Redirect(w, req, url, http.StatusTemporaryRedirect)
-}
-
 func check(err error) {
 	if err != nil {
 		fatal(err)
 	}
 }
 
+func getNodePath(urlPath string) string {
+	pathPrefixLen := len("/" + version + "/keys")
+	return urlPath[pathPrefixLen:]
+}
+
 //--------------------------------------
 // Log
 //--------------------------------------
@@ -228,7 +274,7 @@ func directSet() {
 
 func send(c chan bool) {
 	for i := 0; i < 10; i++ {
-		command := &SetCommand{}
+		command := &UpdateCommand{}
 		command.Key = "foo"
 		command.Value = "bar"
 		command.ExpireTime = time.Unix(0, 0)

+ 1 - 1
version.go

@@ -1,6 +1,6 @@
 package main
 
-const version = "v1"
+const version = "v2"
 
 // TODO: The release version (generated from the git tag) will be the raft
 // protocol version for now. When things settle down we will fix it like the