Browse Source

refactor separate etcd index from raft index

Xiang Li 12 years ago
parent
commit
6156d5c790

+ 1 - 4
error/error.go

@@ -64,16 +64,14 @@ type Error struct {
 	Message   string `json:"message"`
 	Message   string `json:"message"`
 	Cause     string `json:"cause,omitempty"`
 	Cause     string `json:"cause,omitempty"`
 	Index     uint64 `json:"index"`
 	Index     uint64 `json:"index"`
-	Term      uint64 `json:"term"`
 }
 }
 
 
-func NewError(errorCode int, cause string, index uint64, term uint64) *Error {
+func NewError(errorCode int, cause string, index uint64) *Error {
 	return &Error{
 	return &Error{
 		ErrorCode: errorCode,
 		ErrorCode: errorCode,
 		Message:   errors[errorCode],
 		Message:   errors[errorCode],
 		Cause:     cause,
 		Cause:     cause,
 		Index:     index,
 		Index:     index,
-		Term:      term,
 	}
 	}
 }
 }
 
 
@@ -93,7 +91,6 @@ func (e Error) toJsonString() string {
 
 
 func (e Error) Write(w http.ResponseWriter) {
 func (e Error) Write(w http.ResponseWriter) {
 	w.Header().Add("X-Etcd-Index", fmt.Sprint(e.Index))
 	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
 	// 3xx is reft internal error
 	if e.ErrorCode/100 == 3 {
 	if e.ErrorCode/100 == 3 {
 		http.Error(w, e.toJsonString(), http.StatusInternalServerError)
 		http.Error(w, e.toJsonString(), http.StatusInternalServerError)

+ 10 - 10
server/join_command.go

@@ -14,20 +14,20 @@ func init() {
 
 
 // The JoinCommand adds a node to the cluster.
 // The JoinCommand adds a node to the cluster.
 type JoinCommand struct {
 type JoinCommand struct {
-	MinVersion int `json:"minVersion"`
-	MaxVersion int `json:"maxVersion"`
-	Name        string `json:"name"`
-	RaftURL     string `json:"raftURL"`
-	EtcdURL     string `json:"etcdURL"`
+	MinVersion int    `json:"minVersion"`
+	MaxVersion int    `json:"maxVersion"`
+	Name       string `json:"name"`
+	RaftURL    string `json:"raftURL"`
+	EtcdURL    string `json:"etcdURL"`
 }
 }
 
 
 func NewJoinCommand(minVersion int, maxVersion int, name, raftUrl, etcdUrl string) *JoinCommand {
 func NewJoinCommand(minVersion int, maxVersion int, name, raftUrl, etcdUrl string) *JoinCommand {
 	return &JoinCommand{
 	return &JoinCommand{
 		MinVersion: minVersion,
 		MinVersion: minVersion,
 		MaxVersion: maxVersion,
 		MaxVersion: maxVersion,
-		Name:        name,
-		RaftURL:     raftUrl,
-		EtcdURL:     etcdUrl,
+		Name:       name,
+		RaftURL:    raftUrl,
+		EtcdURL:    etcdUrl,
 	}
 	}
 }
 }
 
 
@@ -54,11 +54,11 @@ func (c *JoinCommand) Apply(server raft.Server) (interface{}, error) {
 	// Check machine number in the cluster
 	// Check machine number in the cluster
 	if ps.registry.Count() == ps.MaxClusterSize {
 	if ps.registry.Count() == ps.MaxClusterSize {
 		log.Debug("Reject join request from ", c.Name)
 		log.Debug("Reject join request from ", c.Name)
-		return []byte{0}, etcdErr.NewError(etcdErr.EcodeNoMoreMachine, "", server.CommitIndex(), server.Term())
+		return []byte{0}, etcdErr.NewError(etcdErr.EcodeNoMoreMachine, "", server.CommitIndex())
 	}
 	}
 
 
 	// Add to shared machine registry.
 	// Add to shared machine registry.
-	ps.registry.Register(c.Name, c.RaftURL, c.EtcdURL, server.CommitIndex(), server.Term())
+	ps.registry.Register(c.Name, c.RaftURL, c.EtcdURL)
 
 
 	// Add peer in raft
 	// Add peer in raft
 	err := server.AddPeer(c.Name, "")
 	err := server.AddPeer(c.Name, "")

+ 9 - 9
server/registry.go

@@ -38,20 +38,20 @@ func NewRegistry(s store.Store) *Registry {
 }
 }
 
 
 // Adds a node to the registry.
 // Adds a node to the registry.
-func (r *Registry) Register(name string, peerURL string, url string, commitIndex uint64, term uint64) error {
+func (r *Registry) Register(name string, peerURL string, url string) error {
 	r.Lock()
 	r.Lock()
 	defer r.Unlock()
 	defer r.Unlock()
 
 
 	// Write data to store.
 	// Write data to store.
 	key := path.Join(RegistryKey, name)
 	key := path.Join(RegistryKey, name)
 	value := fmt.Sprintf("raft=%s&etcd=%s", peerURL, url)
 	value := fmt.Sprintf("raft=%s&etcd=%s", peerURL, url)
-	_, err := r.store.Create(key, value, false, store.Permanent, commitIndex, term)
+	_, err := r.store.Create(key, value, false, store.Permanent)
 	log.Debugf("Register: %s", name)
 	log.Debugf("Register: %s", name)
 	return err
 	return err
 }
 }
 
 
 // Removes a node from the registry.
 // Removes a node from the registry.
-func (r *Registry) Unregister(name string, commitIndex uint64, term uint64) error {
+func (r *Registry) Unregister(name string) error {
 	r.Lock()
 	r.Lock()
 	defer r.Unlock()
 	defer r.Unlock()
 
 
@@ -59,14 +59,14 @@ func (r *Registry) Unregister(name string, commitIndex uint64, term uint64) erro
 	// delete(r.nodes, name)
 	// delete(r.nodes, name)
 
 
 	// Remove the key from the store.
 	// Remove the key from the store.
-	_, err := r.store.Delete(path.Join(RegistryKey, name), false, commitIndex, term)
+	_, err := r.store.Delete(path.Join(RegistryKey, name), false)
 	log.Debugf("Unregister: %s", name)
 	log.Debugf("Unregister: %s", name)
 	return err
 	return err
 }
 }
 
 
 // Returns the number of nodes in the cluster.
 // Returns the number of nodes in the cluster.
 func (r *Registry) Count() int {
 func (r *Registry) Count() int {
-	e, err := r.store.Get(RegistryKey, false, false, 0, 0)
+	e, err := r.store.Get(RegistryKey, false, false)
 	if err != nil {
 	if err != nil {
 		return 0
 		return 0
 	}
 	}
@@ -133,7 +133,7 @@ func (r *Registry) urls(leaderName, selfName string, url func(name string) (stri
 	}
 	}
 
 
 	// Retrieve a list of all nodes.
 	// Retrieve a list of all nodes.
-	if e, _ := r.store.Get(RegistryKey, false, false, 0, 0); e != nil {
+	if e, _ := r.store.Get(RegistryKey, false, false); e != nil {
 		// Lookup the URL for each one.
 		// Lookup the URL for each one.
 		for _, pair := range e.KVPairs {
 		for _, pair := range e.KVPairs {
 			_, name := filepath.Split(pair.Key)
 			_, name := filepath.Split(pair.Key)
@@ -160,7 +160,7 @@ func (r *Registry) load(name string) {
 	}
 	}
 
 
 	// Retrieve from store.
 	// Retrieve from store.
-	e, err := r.store.Get(path.Join(RegistryKey, name), false, false, 0, 0)
+	e, err := r.store.Get(path.Join(RegistryKey, name), false, false)
 	if err != nil {
 	if err != nil {
 		return
 		return
 	}
 	}
@@ -173,7 +173,7 @@ func (r *Registry) load(name string) {
 
 
 	// Create node.
 	// Create node.
 	r.nodes[name] = &node{
 	r.nodes[name] = &node{
-		url:         m["etcd"][0],
-		peerURL:     m["raft"][0],
+		url:     m["etcd"][0],
+		peerURL: m["raft"][0],
 	}
 	}
 }
 }

+ 1 - 1
server/remove_command.go

@@ -27,7 +27,7 @@ func (c *RemoveCommand) Apply(server raft.Server) (interface{}, error) {
 	ps, _ := server.Context().(*PeerServer)
 	ps, _ := server.Context().(*PeerServer)
 
 
 	// Remove node from the shared registry.
 	// Remove node from the shared registry.
-	err := ps.registry.Unregister(c.Name, server.CommitIndex(), server.Term())
+	err := ps.registry.Unregister(c.Name)
 
 
 	// Delete from stats
 	// Delete from stats
 	delete(ps.followersStats.Followers, c.Name)
 	delete(ps.followersStats.Followers, c.Name)

+ 4 - 4
server/server.go

@@ -234,7 +234,7 @@ func (s *Server) Dispatch(c raft.Command, w http.ResponseWriter, req *http.Reque
 		}
 		}
 
 
 		if result == nil {
 		if result == nil {
-			return etcdErr.NewError(300, "Empty result from raft", store.UndefIndex, store.UndefTerm)
+			return etcdErr.NewError(300, "Empty result from raft", s.Store().Index())
 		}
 		}
 
 
 		// response for raft related commands[join/remove]
 		// response for raft related commands[join/remove]
@@ -268,7 +268,7 @@ func (s *Server) Dispatch(c raft.Command, w http.ResponseWriter, req *http.Reque
 
 
 		// No leader available.
 		// No leader available.
 		if leader == "" {
 		if leader == "" {
-			return etcdErr.NewError(300, "", store.UndefIndex, store.UndefTerm)
+			return etcdErr.NewError(300, "", s.Store().Index())
 		}
 		}
 
 
 		var url string
 		var url string
@@ -317,7 +317,7 @@ func (s *Server) GetVersionHandler(w http.ResponseWriter, req *http.Request) err
 func (s *Server) GetLeaderHandler(w http.ResponseWriter, req *http.Request) error {
 func (s *Server) GetLeaderHandler(w http.ResponseWriter, req *http.Request) error {
 	leader := s.peerServer.RaftServer().Leader()
 	leader := s.peerServer.RaftServer().Leader()
 	if leader == "" {
 	if leader == "" {
-		return etcdErr.NewError(etcdErr.EcodeLeaderElect, "", store.UndefIndex, store.UndefTerm)
+		return etcdErr.NewError(etcdErr.EcodeLeaderElect, "", s.Store().Index())
 	}
 	}
 	w.WriteHeader(http.StatusOK)
 	w.WriteHeader(http.StatusOK)
 	url, _ := s.registry.PeerURL(leader)
 	url, _ := s.registry.PeerURL(leader)
@@ -348,7 +348,7 @@ func (s *Server) GetLeaderStatsHandler(w http.ResponseWriter, req *http.Request)
 
 
 	leader := s.peerServer.RaftServer().Leader()
 	leader := s.peerServer.RaftServer().Leader()
 	if leader == "" {
 	if leader == "" {
-		return etcdErr.NewError(300, "", store.UndefIndex, store.UndefTerm)
+		return etcdErr.NewError(300, "", s.Store().Index())
 	}
 	}
 	hostname, _ := s.registry.ClientURL(leader)
 	hostname, _ := s.registry.ClientURL(leader)
 	redirect(hostname, w, req)
 	redirect(hostname, w, req)

+ 1 - 1
server/v1/get_key_handler.go

@@ -13,7 +13,7 @@ func GetKeyHandler(w http.ResponseWriter, req *http.Request, s Server) error {
 	key := "/" + vars["key"]
 	key := "/" + vars["key"]
 
 
 	// Retrieve the key from the store.
 	// Retrieve the key from the store.
-	event, err := s.Store().Get(key, false, false, s.CommitIndex(), s.Term())
+	event, err := s.Store().Get(key, false, false)
 	if err != nil {
 	if err != nil {
 		return err
 		return err
 	}
 	}

+ 2 - 2
server/v1/set_key_handler.go

@@ -19,13 +19,13 @@ func SetKeyHandler(w http.ResponseWriter, req *http.Request, s Server) error {
 	// Parse non-blank value.
 	// Parse non-blank value.
 	value := req.Form.Get("value")
 	value := req.Form.Get("value")
 	if len(value) == 0 {
 	if len(value) == 0 {
-		return etcdErr.NewError(200, "Set", store.UndefIndex, store.UndefTerm)
+		return etcdErr.NewError(200, "Set", s.Store().Index())
 	}
 	}
 
 
 	// Convert time-to-live to an expiration time.
 	// Convert time-to-live to an expiration time.
 	expireTime, err := store.TTL(req.Form.Get("ttl"))
 	expireTime, err := store.TTL(req.Form.Get("ttl"))
 	if err != nil {
 	if err != nil {
-		return etcdErr.NewError(202, "Set", store.UndefIndex, store.UndefTerm)
+		return etcdErr.NewError(202, "Set", s.Store().Index())
 	}
 	}
 
 
 	// If the "prevValue" is specified then test-and-set. Otherwise create a new key.
 	// If the "prevValue" is specified then test-and-set. Otherwise create a new key.

+ 3 - 4
server/v1/watch_key_handler.go

@@ -6,7 +6,6 @@ import (
 	"strconv"
 	"strconv"
 
 
 	etcdErr "github.com/coreos/etcd/error"
 	etcdErr "github.com/coreos/etcd/error"
-	"github.com/coreos/etcd/store"
 	"github.com/gorilla/mux"
 	"github.com/gorilla/mux"
 )
 )
 
 
@@ -21,14 +20,14 @@ func WatchKeyHandler(w http.ResponseWriter, req *http.Request, s Server) error {
 	if req.Method == "POST" {
 	if req.Method == "POST" {
 		sinceIndex, err = strconv.ParseUint(string(req.FormValue("index")), 10, 64)
 		sinceIndex, err = strconv.ParseUint(string(req.FormValue("index")), 10, 64)
 		if err != nil {
 		if err != nil {
-			return etcdErr.NewError(203, "Watch From Index", store.UndefIndex, store.UndefTerm)
+			return etcdErr.NewError(203, "Watch From Index", s.Store().Index())
 		}
 		}
 	}
 	}
 
 
 	// Start the watcher on the store.
 	// Start the watcher on the store.
-	c, err := s.Store().Watch(key, false, sinceIndex, s.CommitIndex(), s.Term())
+	c, err := s.Store().Watch(key, false, sinceIndex)
 	if err != nil {
 	if err != nil {
-		return etcdErr.NewError(500, key, store.UndefIndex, store.UndefTerm)
+		return etcdErr.NewError(500, key, s.Store().Index())
 	}
 	}
 	event := <-c
 	event := <-c
 
 

+ 10 - 26
server/v2/get_handler.go

@@ -16,7 +16,6 @@ import (
 func GetHandler(w http.ResponseWriter, req *http.Request, s Server) error {
 func GetHandler(w http.ResponseWriter, req *http.Request, s Server) error {
 	var err error
 	var err error
 	var event *store.Event
 	var event *store.Event
-	events := make([]*store.Event, 0)
 
 
 	vars := mux.Vars(req)
 	vars := mux.Vars(req)
 	key := "/" + vars["key"]
 	key := "/" + vars["key"]
@@ -42,51 +41,36 @@ func GetHandler(w http.ResponseWriter, req *http.Request, s Server) error {
 		if waitIndex != "" {
 		if waitIndex != "" {
 			sinceIndex, err = strconv.ParseUint(string(req.FormValue("waitIndex")), 10, 64)
 			sinceIndex, err = strconv.ParseUint(string(req.FormValue("waitIndex")), 10, 64)
 			if err != nil {
 			if err != nil {
-				return etcdErr.NewError(etcdErr.EcodeIndexNaN, "Watch From Index", store.UndefIndex, store.UndefTerm)
+				return etcdErr.NewError(etcdErr.EcodeIndexNaN, "Watch From Index", s.Store().Index())
 			}
 			}
 		}
 		}
 
 
 		// Start the watcher on the store.
 		// Start the watcher on the store.
-		eventChan, err := s.Store().Watch(key, recursive, sinceIndex, s.CommitIndex(), s.Term())
+		eventChan, err := s.Store().Watch(key, recursive, sinceIndex)
 		if err != nil {
 		if err != nil {
-			return etcdErr.NewError(500, key, store.UndefIndex, store.UndefTerm)
+			return etcdErr.NewError(500, key, s.Store().Index())
 		}
 		}
 
 
 		cn, _ := w.(http.CloseNotifier)
 		cn, _ := w.(http.CloseNotifier)
 		closeChan := cn.CloseNotify()
 		closeChan := cn.CloseNotify()
 
 
-	eventLoop:
-		for {
-			select {
-			case <-closeChan:
-				return nil
-			case event = <-eventChan:
-				// for events other than expire, just one event for one watch
-				// for expire event, we might have a stream of events
-				// we use a nil item to terminate the expire event stream
-				if event != nil && event.Action == store.Expire {
-					events = append(events, event)
-				} else {
-					events = append(events, event)
-					break eventLoop
-				}
-			}
+		select {
+		case <-closeChan:
+			return nil
+		case event = <-eventChan:
 		}
 		}
 
 
 	} else { //get
 	} else { //get
 		// Retrieve the key from the store.
 		// Retrieve the key from the store.
-		event, err = s.Store().Get(key, recursive, sorted, s.CommitIndex(), s.Term())
+		event, err = s.Store().Get(key, recursive, sorted)
 		if err != nil {
 		if err != nil {
 			return err
 			return err
 		}
 		}
 	}
 	}
 
 
-	var b []byte
-
-	w.Header().Add("X-Etcd-Index", fmt.Sprint(events[0].Index))
-	w.Header().Add("X-Etcd-Term", fmt.Sprint(events[0].Term))
+	w.Header().Add("X-Etcd-Index", fmt.Sprint(event.Index))
 	w.WriteHeader(http.StatusOK)
 	w.WriteHeader(http.StatusOK)
-	b, _ = json.Marshal(events)
+	b, _ := json.Marshal(event)
 
 
 	w.Write(b)
 	w.Write(b)
 
 

+ 1 - 1
server/v2/post_handler.go

@@ -15,7 +15,7 @@ func PostHandler(w http.ResponseWriter, req *http.Request, s Server) error {
 	value := req.FormValue("value")
 	value := req.FormValue("value")
 	expireTime, err := store.TTL(req.FormValue("ttl"))
 	expireTime, err := store.TTL(req.FormValue("ttl"))
 	if err != nil {
 	if err != nil {
-		return etcdErr.NewError(etcdErr.EcodeTTLNaN, "Create", store.UndefIndex, store.UndefTerm)
+		return etcdErr.NewError(etcdErr.EcodeTTLNaN, "Create", s.Store().Index())
 	}
 	}
 
 
 	c := s.Store().CommandFactory().CreateCreateCommand(key, value, expireTime, true)
 	c := s.Store().CommandFactory().CreateCreateCommand(key, value, expireTime, true)

+ 4 - 4
server/v2/put_handler.go

@@ -22,7 +22,7 @@ func PutHandler(w http.ResponseWriter, req *http.Request, s Server) error {
 	value := req.Form.Get("value")
 	value := req.Form.Get("value")
 	expireTime, err := store.TTL(req.Form.Get("ttl"))
 	expireTime, err := store.TTL(req.Form.Get("ttl"))
 	if err != nil {
 	if err != nil {
-		return etcdErr.NewError(etcdErr.EcodeTTLNaN, "Update", store.UndefIndex, store.UndefTerm)
+		return etcdErr.NewError(etcdErr.EcodeTTLNaN, "Update", s.Store().Index())
 	}
 	}
 
 
 	_, valueOk := req.Form["prevValue"]
 	_, valueOk := req.Form["prevValue"]
@@ -59,7 +59,7 @@ func PutHandler(w http.ResponseWriter, req *http.Request, s Server) error {
 
 
 		// bad previous index
 		// bad previous index
 		if err != nil {
 		if err != nil {
-			return etcdErr.NewError(etcdErr.EcodeIndexNaN, "CompareAndSwap", store.UndefIndex, store.UndefTerm)
+			return etcdErr.NewError(etcdErr.EcodeIndexNaN, "CompareAndSwap", s.Store().Index())
 		}
 		}
 	} else {
 	} else {
 		prevIndex = 0
 		prevIndex = 0
@@ -67,7 +67,7 @@ func PutHandler(w http.ResponseWriter, req *http.Request, s Server) error {
 
 
 	if valueOk {
 	if valueOk {
 		if prevValue == "" {
 		if prevValue == "" {
-			return etcdErr.NewError(etcdErr.EcodePrevValueRequired, "CompareAndSwap", store.UndefIndex, store.UndefTerm)
+			return etcdErr.NewError(etcdErr.EcodePrevValueRequired, "CompareAndSwap", s.Store().Index())
 		}
 		}
 	}
 	}
 
 
@@ -88,7 +88,7 @@ func CreateHandler(w http.ResponseWriter, req *http.Request, s Server, key, valu
 func UpdateHandler(w http.ResponseWriter, req *http.Request, s Server, key, value string, expireTime time.Time) error {
 func UpdateHandler(w http.ResponseWriter, req *http.Request, s Server, key, value string, expireTime time.Time) error {
 	// Update should give at least one option
 	// Update should give at least one option
 	if value == "" && expireTime.Sub(store.Permanent) == 0 {
 	if value == "" && expireTime.Sub(store.Permanent) == 0 {
-		return etcdErr.NewError(etcdErr.EcodeValueOrTTLRequired, "Update", store.UndefIndex, store.UndefTerm)
+		return etcdErr.NewError(etcdErr.EcodeValueOrTTLRequired, "Update", s.Store().Index())
 	}
 	}
 
 
 	c := s.Store().CommandFactory().CreateUpdateCommand(key, value, expireTime)
 	c := s.Store().CommandFactory().CreateUpdateCommand(key, value, expireTime)

+ 1 - 1
server/v2/tests/delete_handler_test.go

@@ -24,6 +24,6 @@ func TestV2DeleteKey(t *testing.T) {
 		resp, err = tests.DeleteForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), url.Values{})
 		resp, err = tests.DeleteForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), url.Values{})
 		body := tests.ReadBody(resp)
 		body := tests.ReadBody(resp)
 		assert.Nil(t, err, "")
 		assert.Nil(t, err, "")
-		assert.Equal(t, string(body), `{"action":"delete","key":"/foo/bar","prevValue":"XXX","index":4,"term":0}`, "")
+		assert.Equal(t, string(body), `{"action":"delete","key":"/foo/bar","prevValue":"XXX","index":2}`, "")
 	})
 	})
 }
 }

+ 15 - 8
server/v2/tests/get_handler_test.go

@@ -27,8 +27,7 @@ func TestV2GetKey(t *testing.T) {
 		assert.Equal(t, body["action"], "get", "")
 		assert.Equal(t, body["action"], "get", "")
 		assert.Equal(t, body["key"], "/foo/bar", "")
 		assert.Equal(t, body["key"], "/foo/bar", "")
 		assert.Equal(t, body["value"], "XXX", "")
 		assert.Equal(t, body["value"], "XXX", "")
-		assert.Equal(t, body["index"], 3, "")
-		assert.Equal(t, body["term"], 0, "")
+		assert.Equal(t, body["index"], 1, "")
 	})
 	})
 }
 }
 
 
@@ -55,7 +54,7 @@ func TestV2GetKeyRecursively(t *testing.T) {
 		assert.Equal(t, body["action"], "get", "")
 		assert.Equal(t, body["action"], "get", "")
 		assert.Equal(t, body["key"], "/foo", "")
 		assert.Equal(t, body["key"], "/foo", "")
 		assert.Equal(t, body["dir"], true, "")
 		assert.Equal(t, body["dir"], true, "")
-		assert.Equal(t, body["index"], 4, "")
+		assert.Equal(t, body["index"], 2, "")
 		assert.Equal(t, len(body["kvs"].([]interface{})), 2, "")
 		assert.Equal(t, len(body["kvs"].([]interface{})), 2, "")
 
 
 		kv0 := body["kvs"].([]interface{})[0].(map[string]interface{})
 		kv0 := body["kvs"].([]interface{})[0].(map[string]interface{})
@@ -81,9 +80,11 @@ func TestV2GetKeyRecursively(t *testing.T) {
 func TestV2WatchKey(t *testing.T) {
 func TestV2WatchKey(t *testing.T) {
 	tests.RunServer(func(s *server.Server) {
 	tests.RunServer(func(s *server.Server) {
 		var body map[string]interface{}
 		var body map[string]interface{}
+		c := make(chan bool)
 		go func() {
 		go func() {
 			resp, _ := tests.Get(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar?wait=true"))
 			resp, _ := tests.Get(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar?wait=true"))
 			body = tests.ReadBodyJSON(resp)
 			body = tests.ReadBodyJSON(resp)
+			c <- true
 		}()
 		}()
 
 
 		// Make sure response didn't fire early.
 		// Make sure response didn't fire early.
@@ -98,12 +99,19 @@ func TestV2WatchKey(t *testing.T) {
 
 
 		// A response should follow from the GET above.
 		// A response should follow from the GET above.
 		time.Sleep(1 * time.Millisecond)
 		time.Sleep(1 * time.Millisecond)
+
+		select {
+		case <-c:
+
+		default:
+			t.Fatal("cannot get watch result")
+		}
+
 		assert.NotNil(t, body, "")
 		assert.NotNil(t, body, "")
 		assert.Equal(t, body["action"], "set", "")
 		assert.Equal(t, body["action"], "set", "")
 		assert.Equal(t, body["key"], "/foo/bar", "")
 		assert.Equal(t, body["key"], "/foo/bar", "")
 		assert.Equal(t, body["value"], "XXX", "")
 		assert.Equal(t, body["value"], "XXX", "")
-		assert.Equal(t, body["index"], 3, "")
-		assert.Equal(t, body["term"], 0, "")
+		assert.Equal(t, body["index"], 1, "")
 	})
 	})
 }
 }
 
 
@@ -118,7 +126,7 @@ func TestV2WatchKeyWithIndex(t *testing.T) {
 		var body map[string]interface{}
 		var body map[string]interface{}
 		c := make(chan bool)
 		c := make(chan bool)
 		go func() {
 		go func() {
-			resp, _ := tests.Get(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar?wait=true&waitIndex=4"))
+			resp, _ := tests.Get(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar?wait=true&waitIndex=2"))
 			body = tests.ReadBodyJSON(resp)
 			body = tests.ReadBodyJSON(resp)
 			c <- true
 			c <- true
 		}()
 		}()
@@ -156,7 +164,6 @@ func TestV2WatchKeyWithIndex(t *testing.T) {
 		assert.Equal(t, body["action"], "set", "")
 		assert.Equal(t, body["action"], "set", "")
 		assert.Equal(t, body["key"], "/foo/bar", "")
 		assert.Equal(t, body["key"], "/foo/bar", "")
 		assert.Equal(t, body["value"], "YYY", "")
 		assert.Equal(t, body["value"], "YYY", "")
-		assert.Equal(t, body["index"], 4, "")
-		assert.Equal(t, body["term"], 0, "")
+		assert.Equal(t, body["index"], 2, "")
 	})
 	})
 }
 }

+ 4 - 4
server/v2/tests/post_handler_test.go

@@ -21,18 +21,18 @@ func TestV2CreateUnique(t *testing.T) {
 		resp, _ := tests.PostForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), nil)
 		resp, _ := tests.PostForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), nil)
 		body := tests.ReadBodyJSON(resp)
 		body := tests.ReadBodyJSON(resp)
 		assert.Equal(t, body["action"], "create", "")
 		assert.Equal(t, body["action"], "create", "")
-		assert.Equal(t, body["key"], "/foo/bar/3", "")
+		assert.Equal(t, body["key"], "/foo/bar/1", "")
 		assert.Equal(t, body["dir"], true, "")
 		assert.Equal(t, body["dir"], true, "")
-		assert.Equal(t, body["index"], 3, "")
+		assert.Equal(t, body["index"], 1, "")
 
 
 		// Second POST should add next index to list.
 		// Second POST should add next index to list.
 		resp, _ = tests.PostForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), nil)
 		resp, _ = tests.PostForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), nil)
 		body = tests.ReadBodyJSON(resp)
 		body = tests.ReadBodyJSON(resp)
-		assert.Equal(t, body["key"], "/foo/bar/4", "")
+		assert.Equal(t, body["key"], "/foo/bar/2", "")
 
 
 		// POST to a different key should add index to that list.
 		// POST to a different key should add index to that list.
 		resp, _ = tests.PostForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/baz"), nil)
 		resp, _ = tests.PostForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/baz"), nil)
 		body = tests.ReadBodyJSON(resp)
 		body = tests.ReadBodyJSON(resp)
-		assert.Equal(t, body["key"], "/foo/baz/5", "")
+		assert.Equal(t, body["key"], "/foo/baz/3", "")
 	})
 	})
 }
 }

+ 11 - 11
server/v2/tests/put_handler_test.go

@@ -22,7 +22,7 @@ func TestV2SetKey(t *testing.T) {
 		resp, err := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v)
 		resp, err := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v)
 		body := tests.ReadBody(resp)
 		body := tests.ReadBody(resp)
 		assert.Nil(t, err, "")
 		assert.Nil(t, err, "")
-		assert.Equal(t, string(body), `{"action":"set","key":"/foo/bar","value":"XXX","index":3,"term":0}`, "")
+		assert.Equal(t, string(body), `{"action":"set","key":"/foo/bar","value":"XXX","index":1}`, "")
 	})
 	})
 }
 }
 
 
@@ -42,7 +42,7 @@ func TestV2SetKeyWithTTL(t *testing.T) {
 
 
 		// Make sure the expiration date is correct.
 		// Make sure the expiration date is correct.
 		expiration, _ := time.Parse(time.RFC3339Nano, body["expiration"].(string))
 		expiration, _ := time.Parse(time.RFC3339Nano, body["expiration"].(string))
-		assert.Equal(t, expiration.Sub(t0) / time.Second, 20, "")
+		assert.Equal(t, expiration.Sub(t0)/time.Second, 20, "")
 	})
 	})
 }
 }
 
 
@@ -110,7 +110,7 @@ func TestV2UpdateKeySuccess(t *testing.T) {
 		v.Set("value", "XXX")
 		v.Set("value", "XXX")
 		resp, _ := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v)
 		resp, _ := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v)
 		tests.ReadBody(resp)
 		tests.ReadBody(resp)
-		
+
 		v.Set("value", "YYY")
 		v.Set("value", "YYY")
 		v.Set("prevExist", "true")
 		v.Set("prevExist", "true")
 		resp, _ = tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v)
 		resp, _ = tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v)
@@ -160,7 +160,7 @@ func TestV2UpdateKeyFailOnMissingDirectory(t *testing.T) {
 // Ensures that a key is set only if the previous index matches.
 // Ensures that a key is set only if the previous index matches.
 //
 //
 //   $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=XXX
 //   $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=XXX
-//   $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=YYY -d prevIndex=3
+//   $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=YYY -d prevIndex=1
 //
 //
 func TestV2SetKeyCASOnIndexSuccess(t *testing.T) {
 func TestV2SetKeyCASOnIndexSuccess(t *testing.T) {
 	tests.RunServer(func(s *server.Server) {
 	tests.RunServer(func(s *server.Server) {
@@ -169,13 +169,13 @@ func TestV2SetKeyCASOnIndexSuccess(t *testing.T) {
 		resp, _ := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v)
 		resp, _ := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v)
 		tests.ReadBody(resp)
 		tests.ReadBody(resp)
 		v.Set("value", "YYY")
 		v.Set("value", "YYY")
-		v.Set("prevIndex", "3")
+		v.Set("prevIndex", "1")
 		resp, _ = tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v)
 		resp, _ = tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v)
 		body := tests.ReadBodyJSON(resp)
 		body := tests.ReadBodyJSON(resp)
 		assert.Equal(t, body["action"], "compareAndSwap", "")
 		assert.Equal(t, body["action"], "compareAndSwap", "")
 		assert.Equal(t, body["prevValue"], "XXX", "")
 		assert.Equal(t, body["prevValue"], "XXX", "")
 		assert.Equal(t, body["value"], "YYY", "")
 		assert.Equal(t, body["value"], "YYY", "")
-		assert.Equal(t, body["index"], 4, "")
+		assert.Equal(t, body["index"], 2, "")
 	})
 	})
 }
 }
 
 
@@ -196,8 +196,8 @@ func TestV2SetKeyCASOnIndexFail(t *testing.T) {
 		body := tests.ReadBodyJSON(resp)
 		body := tests.ReadBodyJSON(resp)
 		assert.Equal(t, body["errorCode"], 101, "")
 		assert.Equal(t, body["errorCode"], 101, "")
 		assert.Equal(t, body["message"], "Test Failed", "")
 		assert.Equal(t, body["message"], "Test Failed", "")
-		assert.Equal(t, body["cause"], "[ != XXX] [10 != 3]", "")
-		assert.Equal(t, body["index"], 4, "")
+		assert.Equal(t, body["cause"], "[ != XXX] [10 != 1]", "")
+		assert.Equal(t, body["index"], 1, "")
 	})
 	})
 }
 }
 
 
@@ -236,7 +236,7 @@ func TestV2SetKeyCASOnValueSuccess(t *testing.T) {
 		assert.Equal(t, body["action"], "compareAndSwap", "")
 		assert.Equal(t, body["action"], "compareAndSwap", "")
 		assert.Equal(t, body["prevValue"], "XXX", "")
 		assert.Equal(t, body["prevValue"], "XXX", "")
 		assert.Equal(t, body["value"], "YYY", "")
 		assert.Equal(t, body["value"], "YYY", "")
-		assert.Equal(t, body["index"], 4, "")
+		assert.Equal(t, body["index"], 2, "")
 	})
 	})
 }
 }
 
 
@@ -257,8 +257,8 @@ func TestV2SetKeyCASOnValueFail(t *testing.T) {
 		body := tests.ReadBodyJSON(resp)
 		body := tests.ReadBodyJSON(resp)
 		assert.Equal(t, body["errorCode"], 101, "")
 		assert.Equal(t, body["errorCode"], 101, "")
 		assert.Equal(t, body["message"], "Test Failed", "")
 		assert.Equal(t, body["message"], "Test Failed", "")
-		assert.Equal(t, body["cause"], "[AAA != XXX] [0 != 3]", "")
-		assert.Equal(t, body["index"], 4, "")
+		assert.Equal(t, body["cause"], "[AAA != XXX] [0 != 1]", "")
+		assert.Equal(t, body["index"], 1, "")
 	})
 	})
 }
 }
 
 

+ 2 - 9
store/event.go

@@ -14,11 +14,6 @@ const (
 	Expire         = "expire"
 	Expire         = "expire"
 )
 )
 
 
-const (
-	UndefIndex = 0
-	UndefTerm  = 0
-)
-
 type Event struct {
 type Event struct {
 	Action     string     `json:"action"`
 	Action     string     `json:"action"`
 	Key        string     `json:"key, omitempty"`
 	Key        string     `json:"key, omitempty"`
@@ -28,17 +23,15 @@ type Event struct {
 	KVPairs    kvPairs    `json:"kvs,omitempty"`
 	KVPairs    kvPairs    `json:"kvs,omitempty"`
 	Expiration *time.Time `json:"expiration,omitempty"`
 	Expiration *time.Time `json:"expiration,omitempty"`
 	TTL        int64      `json:"ttl,omitempty"` // Time to live in second
 	TTL        int64      `json:"ttl,omitempty"` // Time to live in second
-	// The command index of the raft machine when the command is executed
+	// The index of the etcd state machine when the comment is executed
 	Index uint64 `json:"index"`
 	Index uint64 `json:"index"`
-	Term  uint64 `json:"term"`
 }
 }
 
 
-func newEvent(action string, key string, index uint64, term uint64) *Event {
+func newEvent(action string, key string, index uint64) *Event {
 	return &Event{
 	return &Event{
 		Action: action,
 		Action: action,
 		Key:    key,
 		Key:    key,
 		Index:  index,
 		Index:  index,
-		Term:   term,
 	}
 	}
 }
 }
 
 

+ 4 - 24
store/event_history.go

@@ -12,8 +12,6 @@ type EventHistory struct {
 	Queue      eventQueue
 	Queue      eventQueue
 	StartIndex uint64
 	StartIndex uint64
 	LastIndex  uint64
 	LastIndex  uint64
-	LastTerm   uint64
-	DupCnt     uint64 // help to compute the watching point with duplicated indexes in the queue
 	rwl        sync.RWMutex
 	rwl        sync.RWMutex
 }
 }
 
 
@@ -31,14 +29,9 @@ func (eh *EventHistory) addEvent(e *Event) *Event {
 	eh.rwl.Lock()
 	eh.rwl.Lock()
 	defer eh.rwl.Unlock()
 	defer eh.rwl.Unlock()
 
 
-	if e.Index == eh.LastIndex {
-		eh.DupCnt += 1
-	}
-
 	eh.Queue.insert(e)
 	eh.Queue.insert(e)
 
 
 	eh.LastIndex = e.Index
 	eh.LastIndex = e.Index
-	eh.LastTerm = e.Term
 
 
 	eh.StartIndex = eh.Queue.Events[eh.Queue.Front].Index
 	eh.StartIndex = eh.Queue.Events[eh.Queue.Front].Index
 
 
@@ -47,7 +40,7 @@ func (eh *EventHistory) addEvent(e *Event) *Event {
 
 
 // scan function is enumerating events from the index in history and
 // scan function is enumerating events from the index in history and
 // stops till the first point where the key has identified prefix
 // stops till the first point where the key has identified prefix
-func (eh *EventHistory) scan(prefix string, index uint64) ([]*Event, *etcdErr.Error) {
+func (eh *EventHistory) scan(prefix string, index uint64) (*Event, *etcdErr.Error) {
 	eh.rwl.RLock()
 	eh.rwl.RLock()
 	defer eh.rwl.RUnlock()
 	defer eh.rwl.RUnlock()
 
 
@@ -56,7 +49,7 @@ func (eh *EventHistory) scan(prefix string, index uint64) ([]*Event, *etcdErr.Er
 		return nil,
 		return nil,
 			etcdErr.NewError(etcdErr.EcodeEventIndexCleared,
 			etcdErr.NewError(etcdErr.EcodeEventIndexCleared,
 				fmt.Sprintf("the requested history has been cleared [%v/%v]",
 				fmt.Sprintf("the requested history has been cleared [%v/%v]",
-					eh.StartIndex, index), UndefIndex, UndefTerm)
+					eh.StartIndex, index), 0)
 	}
 	}
 
 
 	// the index should locate before the size of the queue minus the duplicate count
 	// the index should locate before the size of the queue minus the duplicate count
@@ -66,28 +59,17 @@ func (eh *EventHistory) scan(prefix string, index uint64) ([]*Event, *etcdErr.Er
 
 
 	i := eh.Queue.Front
 	i := eh.Queue.Front
 
 
-	events := make([]*Event, 0)
-	var eventIndex uint64
-
 	for {
 	for {
 		e := eh.Queue.Events[i]
 		e := eh.Queue.Events[i]
 
 
-		if eventIndex != 0 && eventIndex != e.Index {
-			return events, nil
-		}
-
 		if strings.HasPrefix(e.Key, prefix) && index <= e.Index { // make sure we bypass the smaller one
 		if strings.HasPrefix(e.Key, prefix) && index <= e.Index { // make sure we bypass the smaller one
-			eventIndex = e.Index
-			events = append(events, e)
+			return e, nil
 		}
 		}
 
 
 		i = (i + 1) % eh.Queue.Capacity
 		i = (i + 1) % eh.Queue.Capacity
 
 
 		if i > eh.Queue.back() {
 		if i > eh.Queue.back() {
-			if eventIndex == 0 { // find nothing, return and watch from current index
-				return nil, nil
-			}
-			return events, nil
+			return nil, nil
 		}
 		}
 	}
 	}
 }
 }
@@ -110,8 +92,6 @@ func (eh *EventHistory) clone() *EventHistory {
 		StartIndex: eh.StartIndex,
 		StartIndex: eh.StartIndex,
 		Queue:      clonedQueue,
 		Queue:      clonedQueue,
 		LastIndex:  eh.LastIndex,
 		LastIndex:  eh.LastIndex,
-		LastTerm:   eh.LastTerm,
-		DupCnt:     eh.DupCnt,
 	}
 	}
 
 
 }
 }

+ 12 - 12
store/event_test.go

@@ -13,7 +13,7 @@ func TestEventQueue(t *testing.T) {
 
 
 	// Add
 	// Add
 	for i := 0; i < 200; i++ {
 	for i := 0; i < 200; i++ {
-		e := newEvent(Create, "/foo", uint64(i), 1)
+		e := newEvent(Create, "/foo", uint64(i))
 		eh.addEvent(e)
 		eh.addEvent(e)
 	}
 	}
 
 
@@ -35,27 +35,27 @@ func TestScanHistory(t *testing.T) {
 	eh := newEventHistory(100)
 	eh := newEventHistory(100)
 
 
 	// Add
 	// 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))
+	eh.addEvent(newEvent(Create, "/foo", 1))
+	eh.addEvent(newEvent(Create, "/foo/bar", 2))
+	eh.addEvent(newEvent(Create, "/foo/foo", 3))
+	eh.addEvent(newEvent(Create, "/foo/bar/bar", 4))
+	eh.addEvent(newEvent(Create, "/foo/foo/foo", 5))
 
 
 	e, err := eh.scan("/foo", 1)
 	e, err := eh.scan("/foo", 1)
-	if err != nil || e[0].Index != 1 {
-		t.Fatalf("scan error [/foo] [1] %v", e[0].Index)
+	if err != nil || e.Index != 1 {
+		t.Fatalf("scan error [/foo] [1] %v", e.Index)
 	}
 	}
 
 
 	e, err = eh.scan("/foo/bar", 1)
 	e, err = eh.scan("/foo/bar", 1)
 
 
-	if err != nil || e[0].Index != 2 {
-		t.Fatalf("scan error [/foo/bar] [2] %v", e[0].Index)
+	if err != nil || e.Index != 2 {
+		t.Fatalf("scan error [/foo/bar] [2] %v", e.Index)
 	}
 	}
 
 
 	e, err = eh.scan("/foo/bar", 3)
 	e, err = eh.scan("/foo/bar", 3)
 
 
-	if err != nil || e[0].Index != 4 {
-		t.Fatalf("scan error [/foo/bar/bar] [4] %v", e[0].Index)
+	if err != nil || e.Index != 4 {
+		t.Fatalf("scan error [/foo/bar/bar] [4] %v", e.Index)
 	}
 	}
 
 
 	e, err = eh.scan("/foo/bar", 6)
 	e, err = eh.scan("/foo/bar", 6)

+ 2 - 2
store/heap_test.go

@@ -14,7 +14,7 @@ func TestHeapPushPop(t *testing.T) {
 	for i := 0; i < 10; i++ {
 	for i := 0; i < 10; i++ {
 		path := fmt.Sprintf("%v", 10-i)
 		path := fmt.Sprintf("%v", 10-i)
 		m := time.Duration(10 - i)
 		m := time.Duration(10 - i)
-		n := newKV(nil, path, path, 0, 0, nil, "", time.Now().Add(time.Second*m))
+		n := newKV(nil, path, path, 0, nil, "", time.Now().Add(time.Second*m))
 		h.push(n)
 		h.push(n)
 	}
 	}
 
 
@@ -40,7 +40,7 @@ func TestHeapUpdate(t *testing.T) {
 	for i, n := range kvs {
 	for i, n := range kvs {
 		path := fmt.Sprintf("%v", 10-i)
 		path := fmt.Sprintf("%v", 10-i)
 		m := time.Duration(10 - i)
 		m := time.Duration(10 - i)
-		n = newKV(nil, path, path, 0, 0, nil, "", time.Now().Add(time.Second*m))
+		n = newKV(nil, path, path, 0, nil, "", time.Now().Add(time.Second*m))
 		kvs[i] = n
 		kvs[i] = n
 		h.push(n)
 		h.push(n)
 	}
 	}

+ 14 - 23
store/node.go

@@ -8,11 +8,6 @@ import (
 	etcdErr "github.com/coreos/etcd/error"
 	etcdErr "github.com/coreos/etcd/error"
 )
 )
 
 
-const (
-	normal = iota
-	removed
-)
-
 var Permanent time.Time
 var Permanent time.Time
 
 
 // Node is the basic element in the store system.
 // Node is the basic element in the store system.
@@ -26,7 +21,7 @@ type Node struct {
 	ModifiedIndex uint64
 	ModifiedIndex uint64
 	ModifiedTerm  uint64
 	ModifiedTerm  uint64
 
 
-	Parent *Node `json:"-"` // should not encode this field! avoid cyclical dependency.
+	Parent *Node `json:"-"` // should not encode this field! avoid circular dependency.
 
 
 	ExpireTime time.Time
 	ExpireTime time.Time
 	ACL        string
 	ACL        string
@@ -39,14 +34,12 @@ type Node struct {
 
 
 // newKV creates a Key-Value pair
 // newKV creates a Key-Value pair
 func newKV(store *store, nodePath string, value string, createIndex uint64,
 func newKV(store *store, nodePath string, value string, createIndex uint64,
-	createTerm uint64, parent *Node, ACL string, expireTime time.Time) *Node {
+	parent *Node, ACL string, expireTime time.Time) *Node {
 
 
 	return &Node{
 	return &Node{
 		Path:          nodePath,
 		Path:          nodePath,
 		CreateIndex:   createIndex,
 		CreateIndex:   createIndex,
-		CreateTerm:    createTerm,
 		ModifiedIndex: createIndex,
 		ModifiedIndex: createIndex,
-		ModifiedTerm:  createTerm,
 		Parent:        parent,
 		Parent:        parent,
 		ACL:           ACL,
 		ACL:           ACL,
 		store:         store,
 		store:         store,
@@ -56,13 +49,12 @@ func newKV(store *store, nodePath string, value string, createIndex uint64,
 }
 }
 
 
 // newDir creates a directory
 // newDir creates a directory
-func newDir(store *store, nodePath string, createIndex uint64, createTerm uint64,
-	parent *Node, ACL string, expireTime time.Time) *Node {
+func newDir(store *store, nodePath string, createIndex uint64, parent *Node,
+	ACL string, expireTime time.Time) *Node {
 
 
 	return &Node{
 	return &Node{
 		Path:        nodePath,
 		Path:        nodePath,
 		CreateIndex: createIndex,
 		CreateIndex: createIndex,
-		CreateTerm:  createTerm,
 		Parent:      parent,
 		Parent:      parent,
 		ACL:         ACL,
 		ACL:         ACL,
 		ExpireTime:  expireTime,
 		ExpireTime:  expireTime,
@@ -101,7 +93,7 @@ func (n *Node) IsDir() bool {
 // If the receiver node is not a key-value pair, a "Not A File" error will be returned.
 // 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) {
 func (n *Node) Read() (string, *etcdErr.Error) {
 	if n.IsDir() {
 	if n.IsDir() {
-		return "", etcdErr.NewError(etcdErr.EcodeNotFile, "", UndefIndex, UndefTerm)
+		return "", etcdErr.NewError(etcdErr.EcodeNotFile, "", n.store.Index())
 	}
 	}
 
 
 	return n.Value, nil
 	return n.Value, nil
@@ -109,14 +101,13 @@ func (n *Node) Read() (string, *etcdErr.Error) {
 
 
 // Write function set the value of the node to the given value.
 // 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.
 // 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 {
+func (n *Node) Write(value string, index uint64) *etcdErr.Error {
 	if n.IsDir() {
 	if n.IsDir() {
-		return etcdErr.NewError(etcdErr.EcodeNotFile, "", UndefIndex, UndefTerm)
+		return etcdErr.NewError(etcdErr.EcodeNotFile, "", n.store.Index())
 	}
 	}
 
 
 	n.Value = value
 	n.Value = value
 	n.ModifiedIndex = index
 	n.ModifiedIndex = index
-	n.ModifiedTerm = term
 
 
 	return nil
 	return nil
 }
 }
@@ -132,7 +123,7 @@ func (n *Node) ExpirationAndTTL() (*time.Time, int64) {
 // If the receiver node is not a directory, a "Not A Directory" error will be returned.
 // If the receiver node is not a directory, a "Not A Directory" error will be returned.
 func (n *Node) List() ([]*Node, *etcdErr.Error) {
 func (n *Node) List() ([]*Node, *etcdErr.Error) {
 	if !n.IsDir() {
 	if !n.IsDir() {
-		return nil, etcdErr.NewError(etcdErr.EcodeNotDir, "", UndefIndex, UndefTerm)
+		return nil, etcdErr.NewError(etcdErr.EcodeNotDir, "", n.store.Index())
 	}
 	}
 
 
 	nodes := make([]*Node, len(n.Children))
 	nodes := make([]*Node, len(n.Children))
@@ -150,7 +141,7 @@ func (n *Node) List() ([]*Node, *etcdErr.Error) {
 // On success, it returns the file node
 // On success, it returns the file node
 func (n *Node) GetChild(name string) (*Node, *etcdErr.Error) {
 func (n *Node) GetChild(name string) (*Node, *etcdErr.Error) {
 	if !n.IsDir() {
 	if !n.IsDir() {
-		return nil, etcdErr.NewError(etcdErr.EcodeNotDir, n.Path, UndefIndex, UndefTerm)
+		return nil, etcdErr.NewError(etcdErr.EcodeNotDir, n.Path, n.store.Index())
 	}
 	}
 
 
 	child, ok := n.Children[name]
 	child, ok := n.Children[name]
@@ -168,7 +159,7 @@ func (n *Node) GetChild(name string) (*Node, *etcdErr.Error) {
 // error will be returned
 // error will be returned
 func (n *Node) Add(child *Node) *etcdErr.Error {
 func (n *Node) Add(child *Node) *etcdErr.Error {
 	if !n.IsDir() {
 	if !n.IsDir() {
-		return etcdErr.NewError(etcdErr.EcodeNotDir, "", UndefIndex, UndefTerm)
+		return etcdErr.NewError(etcdErr.EcodeNotDir, "", n.store.Index())
 	}
 	}
 
 
 	_, name := path.Split(child.Path)
 	_, name := path.Split(child.Path)
@@ -176,7 +167,7 @@ func (n *Node) Add(child *Node) *etcdErr.Error {
 	_, ok := n.Children[name]
 	_, ok := n.Children[name]
 
 
 	if ok {
 	if ok {
-		return etcdErr.NewError(etcdErr.EcodeNodeExist, "", UndefIndex, UndefTerm)
+		return etcdErr.NewError(etcdErr.EcodeNodeExist, "", n.store.Index())
 	}
 	}
 
 
 	n.Children[name] = child
 	n.Children[name] = child
@@ -189,7 +180,7 @@ func (n *Node) Remove(recursive bool, callback func(path string)) *etcdErr.Error
 
 
 	if n.IsDir() && !recursive {
 	if n.IsDir() && !recursive {
 		// cannot delete a directory without set recursive to true
 		// cannot delete a directory without set recursive to true
-		return etcdErr.NewError(etcdErr.EcodeNotFile, "", UndefIndex, UndefTerm)
+		return etcdErr.NewError(etcdErr.EcodeNotFile, "", n.store.Index())
 	}
 	}
 
 
 	if !n.IsDir() { // key-value pair
 	if !n.IsDir() { // key-value pair
@@ -309,10 +300,10 @@ func (n *Node) UpdateTTL(expireTime time.Time) {
 // If the node is a key-value pair, it will clone the pair.
 // If the node is a key-value pair, it will clone the pair.
 func (n *Node) Clone() *Node {
 func (n *Node) Clone() *Node {
 	if !n.IsDir() {
 	if !n.IsDir() {
-		return newKV(n.store, n.Path, n.Value, n.CreateIndex, n.CreateTerm, n.Parent, n.ACL, n.ExpireTime)
+		return newKV(n.store, n.Path, n.Value, n.CreateIndex, n.Parent, n.ACL, n.ExpireTime)
 	}
 	}
 
 
-	clone := newDir(n.store, n.Path, n.CreateIndex, n.CreateTerm, n.Parent, n.ACL, n.ExpireTime)
+	clone := newDir(n.store, n.Path, n.CreateIndex, n.Parent, n.ACL, n.ExpireTime)
 
 
 	for key, child := range n.Children {
 	for key, child := range n.Children {
 		clone.Children[key] = child.Clone()
 		clone.Children[key] = child.Clone()

+ 19 - 19
store/stats_test.go

@@ -10,85 +10,85 @@ import (
 // Ensure that a successful Get is recorded in the stats.
 // Ensure that a successful Get is recorded in the stats.
 func TestStoreStatsGetSuccess(t *testing.T) {
 func TestStoreStatsGetSuccess(t *testing.T) {
 	s := newStore()
 	s := newStore()
-	s.Create("/foo", "bar", false, Permanent, 3, 1)
-	s.Get("/foo", false, false, 3, 1)
+	s.Create("/foo", "bar", false, Permanent)
+	s.Get("/foo", false, false)
 	assert.Equal(t, uint64(1), s.Stats.GetSuccess, "")
 	assert.Equal(t, uint64(1), s.Stats.GetSuccess, "")
 }
 }
 
 
 // Ensure that a failed Get is recorded in the stats.
 // Ensure that a failed Get is recorded in the stats.
 func TestStoreStatsGetFail(t *testing.T) {
 func TestStoreStatsGetFail(t *testing.T) {
 	s := newStore()
 	s := newStore()
-	s.Create("/foo", "bar", false, Permanent, 3, 1)
-	s.Get("/no_such_key", false, false, 3, 1)
+	s.Create("/foo", "bar", false, Permanent)
+	s.Get("/no_such_key", false, false)
 	assert.Equal(t, uint64(1), s.Stats.GetFail, "")
 	assert.Equal(t, uint64(1), s.Stats.GetFail, "")
 }
 }
 
 
 // Ensure that a successful Create is recorded in the stats.
 // Ensure that a successful Create is recorded in the stats.
 func TestStoreStatsCreateSuccess(t *testing.T) {
 func TestStoreStatsCreateSuccess(t *testing.T) {
 	s := newStore()
 	s := newStore()
-	s.Create("/foo", "bar", false, Permanent, 3, 1)
+	s.Create("/foo", "bar", false, Permanent)
 	assert.Equal(t, uint64(1), s.Stats.CreateSuccess, "")
 	assert.Equal(t, uint64(1), s.Stats.CreateSuccess, "")
 }
 }
 
 
 // Ensure that a failed Create is recorded in the stats.
 // Ensure that a failed Create is recorded in the stats.
 func TestStoreStatsCreateFail(t *testing.T) {
 func TestStoreStatsCreateFail(t *testing.T) {
 	s := newStore()
 	s := newStore()
-	s.Create("/foo", "", false, Permanent, 3, 1)
-	s.Create("/foo", "bar", false, Permanent, 4, 1)
+	s.Create("/foo", "", false, Permanent)
+	s.Create("/foo", "bar", false, Permanent)
 	assert.Equal(t, uint64(1), s.Stats.CreateFail, "")
 	assert.Equal(t, uint64(1), s.Stats.CreateFail, "")
 }
 }
 
 
 // Ensure that a successful Update is recorded in the stats.
 // Ensure that a successful Update is recorded in the stats.
 func TestStoreStatsUpdateSuccess(t *testing.T) {
 func TestStoreStatsUpdateSuccess(t *testing.T) {
 	s := newStore()
 	s := newStore()
-	s.Create("/foo", "bar", false, Permanent, 3, 1)
-	s.Update("/foo", "baz", Permanent, 4, 1)
+	s.Create("/foo", "bar", false, Permanent)
+	s.Update("/foo", "baz", Permanent)
 	assert.Equal(t, uint64(1), s.Stats.UpdateSuccess, "")
 	assert.Equal(t, uint64(1), s.Stats.UpdateSuccess, "")
 }
 }
 
 
 // Ensure that a failed Update is recorded in the stats.
 // Ensure that a failed Update is recorded in the stats.
 func TestStoreStatsUpdateFail(t *testing.T) {
 func TestStoreStatsUpdateFail(t *testing.T) {
 	s := newStore()
 	s := newStore()
-	s.Update("/foo", "bar", Permanent, 4, 1)
+	s.Update("/foo", "bar", Permanent)
 	assert.Equal(t, uint64(1), s.Stats.UpdateFail, "")
 	assert.Equal(t, uint64(1), s.Stats.UpdateFail, "")
 }
 }
 
 
 // Ensure that a successful CAS is recorded in the stats.
 // Ensure that a successful CAS is recorded in the stats.
 func TestStoreStatsCompareAndSwapSuccess(t *testing.T) {
 func TestStoreStatsCompareAndSwapSuccess(t *testing.T) {
 	s := newStore()
 	s := newStore()
-	s.Create("/foo", "bar", false, Permanent, 3, 1)
-	s.CompareAndSwap("/foo", "bar", 0, "baz", Permanent, 4, 1)
+	s.Create("/foo", "bar", false, Permanent)
+	s.CompareAndSwap("/foo", "bar", 0, "baz", Permanent)
 	assert.Equal(t, uint64(1), s.Stats.CompareAndSwapSuccess, "")
 	assert.Equal(t, uint64(1), s.Stats.CompareAndSwapSuccess, "")
 }
 }
 
 
 // Ensure that a failed CAS is recorded in the stats.
 // Ensure that a failed CAS is recorded in the stats.
 func TestStoreStatsCompareAndSwapFail(t *testing.T) {
 func TestStoreStatsCompareAndSwapFail(t *testing.T) {
 	s := newStore()
 	s := newStore()
-	s.Create("/foo", "bar", false, Permanent, 3, 1)
-	s.CompareAndSwap("/foo", "wrong_value", 0, "baz", Permanent, 4, 1)
+	s.Create("/foo", "bar", false, Permanent)
+	s.CompareAndSwap("/foo", "wrong_value", 0, "baz", Permanent)
 	assert.Equal(t, uint64(1), s.Stats.CompareAndSwapFail, "")
 	assert.Equal(t, uint64(1), s.Stats.CompareAndSwapFail, "")
 }
 }
 
 
 // Ensure that a successful Delete is recorded in the stats.
 // Ensure that a successful Delete is recorded in the stats.
 func TestStoreStatsDeleteSuccess(t *testing.T) {
 func TestStoreStatsDeleteSuccess(t *testing.T) {
 	s := newStore()
 	s := newStore()
-	s.Create("/foo", "bar", false, Permanent, 3, 1)
-	s.Delete("/foo", false, 4, 1)
+	s.Create("/foo", "bar", false, Permanent)
+	s.Delete("/foo", false)
 	assert.Equal(t, uint64(1), s.Stats.DeleteSuccess, "")
 	assert.Equal(t, uint64(1), s.Stats.DeleteSuccess, "")
 }
 }
 
 
 // Ensure that a failed Delete is recorded in the stats.
 // Ensure that a failed Delete is recorded in the stats.
 func TestStoreStatsDeleteFail(t *testing.T) {
 func TestStoreStatsDeleteFail(t *testing.T) {
 	s := newStore()
 	s := newStore()
-	s.Delete("/foo", false, 4, 1)
+	s.Delete("/foo", false)
 	assert.Equal(t, uint64(1), s.Stats.DeleteFail, "")
 	assert.Equal(t, uint64(1), s.Stats.DeleteFail, "")
 }
 }
 
 
 //Ensure that the number of expirations is recorded in the stats.
 //Ensure that the number of expirations is recorded in the stats.
 func TestStoreStatsExpireCount(t *testing.T) {
 func TestStoreStatsExpireCount(t *testing.T) {
 	s := newStore()
 	s := newStore()
-	go mockSyncService(s.deleteExpiredKeys)
-	s.Create("/foo", "bar", false, time.Now().Add(500*time.Millisecond), 3, 1)
+	go mockSyncService(s.DeleteExpiredKeys)
+	s.Create("/foo", "bar", false, time.Now().Add(500*time.Millisecond))
 	assert.Equal(t, uint64(0), s.Stats.ExpireCount, "")
 	assert.Equal(t, uint64(0), s.Stats.ExpireCount, "")
 	time.Sleep(600 * time.Millisecond)
 	time.Sleep(600 * time.Millisecond)
 	assert.Equal(t, uint64(1), s.Stats.ExpireCount, "")
 	assert.Equal(t, uint64(1), s.Stats.ExpireCount, "")

+ 83 - 68
store/store.go

@@ -19,27 +19,27 @@ const defaultVersion = 2
 type Store interface {
 type Store interface {
 	Version() int
 	Version() int
 	CommandFactory() CommandFactory
 	CommandFactory() CommandFactory
-	Get(nodePath string, recursive, sorted bool, index uint64, term uint64) (*Event, error)
-	Set(nodePath string, value string, expireTime time.Time, index uint64, term uint64) (*Event, error)
-	Update(nodePath string, newValue string, expireTime time.Time, index uint64, term uint64) (*Event, error)
-	Create(nodePath string, value string, incrementalSuffix bool, expireTime time.Time,
-		index uint64, term uint64) (*Event, error)
+	Index() uint64
+	Get(nodePath string, recursive, sorted bool) (*Event, error)
+	Set(nodePath string, value string, expireTime time.Time) (*Event, error)
+	Update(nodePath string, newValue string, expireTime time.Time) (*Event, error)
+	Create(nodePath string, value string, incrementalSuffix bool,
+		expireTime time.Time) (*Event, error)
 	CompareAndSwap(nodePath string, prevValue string, prevIndex uint64,
 	CompareAndSwap(nodePath string, prevValue string, prevIndex uint64,
-		value string, expireTime time.Time, index uint64, term uint64) (*Event, error)
-	Delete(nodePath string, recursive bool, index uint64, term uint64) (*Event, error)
-	Watch(prefix string, recursive bool, sinceIndex uint64, index uint64, term uint64) (<-chan *Event, error)
+		value string, expireTime time.Time) (*Event, error)
+	Delete(nodePath string, recursive bool) (*Event, error)
+	Watch(prefix string, recursive bool, sinceIndex uint64) (<-chan *Event, error)
 	Save() ([]byte, error)
 	Save() ([]byte, error)
 	Recovery(state []byte) error
 	Recovery(state []byte) error
 	TotalTransactions() uint64
 	TotalTransactions() uint64
 	JsonStats() []byte
 	JsonStats() []byte
-	DeleteExpiredKeys(cutoff time.Time, index uint64, term uint64)
+	DeleteExpiredKeys(cutoff time.Time)
 }
 }
 
 
 type store struct {
 type store struct {
 	Root           *Node
 	Root           *Node
 	WatcherHub     *watcherHub
 	WatcherHub     *watcherHub
-	Index          uint64
-	Term           uint64
+	CurrentIndex   uint64
 	Stats          *Stats
 	Stats          *Stats
 	CurrentVersion int
 	CurrentVersion int
 	ttlKeyHeap     *ttlKeyHeap  // need to recovery manually
 	ttlKeyHeap     *ttlKeyHeap  // need to recovery manually
@@ -53,7 +53,7 @@ func New() Store {
 func newStore() *store {
 func newStore() *store {
 	s := new(store)
 	s := new(store)
 	s.CurrentVersion = defaultVersion
 	s.CurrentVersion = defaultVersion
-	s.Root = newDir(s, "/", UndefIndex, UndefTerm, nil, "", Permanent)
+	s.Root = newDir(s, "/", s.CurrentIndex, nil, "", Permanent)
 	s.Stats = newStats()
 	s.Stats = newStats()
 	s.WatcherHub = newWatchHub(1000)
 	s.WatcherHub = newWatchHub(1000)
 	s.ttlKeyHeap = newTtlKeyHeap()
 	s.ttlKeyHeap = newTtlKeyHeap()
@@ -65,6 +65,11 @@ func (s *store) Version() int {
 	return s.CurrentVersion
 	return s.CurrentVersion
 }
 }
 
 
+// Retrieves current of the store
+func (s *store) Index() uint64 {
+	return s.CurrentIndex
+}
+
 // CommandFactory retrieves the command factory for the current version of the store.
 // CommandFactory retrieves the command factory for the current version of the store.
 func (s *store) CommandFactory() CommandFactory {
 func (s *store) CommandFactory() CommandFactory {
 	return GetCommandFactory(s.Version())
 	return GetCommandFactory(s.Version())
@@ -73,20 +78,20 @@ func (s *store) CommandFactory() CommandFactory {
 // Get function returns a get event.
 // Get function returns a get event.
 // If recursive is true, it will return all the content under the node path.
 // If recursive is true, it will return all the content under the node path.
 // If sorted is true, it will sort the content by keys.
 // If sorted is true, it will sort the content by keys.
-func (s *store) Get(nodePath string, recursive, sorted bool, index uint64, term uint64) (*Event, error) {
+func (s *store) Get(nodePath string, recursive, sorted bool) (*Event, error) {
 	s.worldLock.RLock()
 	s.worldLock.RLock()
 	defer s.worldLock.RUnlock()
 	defer s.worldLock.RUnlock()
 
 
 	nodePath = path.Clean(path.Join("/", nodePath))
 	nodePath = path.Clean(path.Join("/", nodePath))
 
 
-	n, err := s.internalGet(nodePath, index, term)
+	n, err := s.internalGet(nodePath)
 
 
 	if err != nil {
 	if err != nil {
 		s.Stats.Inc(GetFail)
 		s.Stats.Inc(GetFail)
 		return nil, err
 		return nil, err
 	}
 	}
 
 
-	e := newEvent(Get, nodePath, index, term)
+	e := newEvent(Get, nodePath, s.CurrentIndex)
 
 
 	if n.IsDir() { // node is a directory
 	if n.IsDir() { // node is a directory
 		e.Dir = true
 		e.Dir = true
@@ -128,13 +133,12 @@ func (s *store) Get(nodePath string, recursive, sorted bool, index uint64, term
 // Create function creates the Node at nodePath. Create will help to create intermediate directories with no ttl.
 // 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 the node has already existed, create will fail.
 // If any node on the path is a file, create will fail.
 // If any node on the path is a file, create will fail.
-func (s *store) Create(nodePath string, value string, unique bool,
-	expireTime time.Time, index uint64, term uint64) (*Event, error) {
+func (s *store) Create(nodePath string, value string, unique bool, expireTime time.Time) (*Event, error) {
 	nodePath = path.Clean(path.Join("/", nodePath))
 	nodePath = path.Clean(path.Join("/", nodePath))
 
 
 	s.worldLock.Lock()
 	s.worldLock.Lock()
 	defer s.worldLock.Unlock()
 	defer s.worldLock.Unlock()
-	e, err := s.internalCreate(nodePath, value, unique, false, expireTime, index, term, Create)
+	e, err := s.internalCreate(nodePath, value, unique, false, expireTime, Create)
 
 
 	if err == nil {
 	if err == nil {
 		s.Stats.Inc(CreateSuccess)
 		s.Stats.Inc(CreateSuccess)
@@ -146,12 +150,12 @@ func (s *store) Create(nodePath string, value string, unique bool,
 }
 }
 
 
 // Set function creates or replace the Node at nodePath.
 // Set function creates or replace the Node at nodePath.
-func (s *store) Set(nodePath string, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) {
+func (s *store) Set(nodePath string, value string, expireTime time.Time) (*Event, error) {
 	nodePath = path.Clean(path.Join("/", nodePath))
 	nodePath = path.Clean(path.Join("/", nodePath))
 
 
 	s.worldLock.Lock()
 	s.worldLock.Lock()
 	defer s.worldLock.Unlock()
 	defer s.worldLock.Unlock()
-	e, err := s.internalCreate(nodePath, value, false, true, expireTime, index, term, Set)
+	e, err := s.internalCreate(nodePath, value, false, true, expireTime, Set)
 
 
 	if err == nil {
 	if err == nil {
 		s.Stats.Inc(SetSuccess)
 		s.Stats.Inc(SetSuccess)
@@ -163,14 +167,14 @@ func (s *store) Set(nodePath string, value string, expireTime time.Time, index u
 }
 }
 
 
 func (s *store) CompareAndSwap(nodePath string, prevValue string, prevIndex uint64,
 func (s *store) CompareAndSwap(nodePath string, prevValue string, prevIndex uint64,
-	value string, expireTime time.Time, index uint64, term uint64) (*Event, error) {
+	value string, expireTime time.Time) (*Event, error) {
 
 
 	nodePath = path.Clean(path.Join("/", nodePath))
 	nodePath = path.Clean(path.Join("/", nodePath))
 
 
 	s.worldLock.Lock()
 	s.worldLock.Lock()
 	defer s.worldLock.Unlock()
 	defer s.worldLock.Unlock()
 
 
-	n, err := s.internalGet(nodePath, index, term)
+	n, err := s.internalGet(nodePath)
 
 
 	if err != nil {
 	if err != nil {
 		s.Stats.Inc(CompareAndSwapFail)
 		s.Stats.Inc(CompareAndSwapFail)
@@ -179,17 +183,20 @@ func (s *store) CompareAndSwap(nodePath string, prevValue string, prevIndex uint
 
 
 	if n.IsDir() { // can only test and set file
 	if n.IsDir() { // can only test and set file
 		s.Stats.Inc(CompareAndSwapFail)
 		s.Stats.Inc(CompareAndSwapFail)
-		return nil, etcdErr.NewError(etcdErr.EcodeNotFile, nodePath, index, term)
+		return nil, etcdErr.NewError(etcdErr.EcodeNotFile, nodePath, s.CurrentIndex)
 	}
 	}
 
 
 	// If both of the prevValue and prevIndex are given, we will test both of them.
 	// If both of the prevValue and prevIndex are given, we will test both of them.
 	// Command will be executed, only if both of the tests are successful.
 	// Command will be executed, only if both of the tests are successful.
 	if (prevValue == "" || n.Value == prevValue) && (prevIndex == 0 || n.ModifiedIndex == prevIndex) {
 	if (prevValue == "" || n.Value == prevValue) && (prevIndex == 0 || n.ModifiedIndex == prevIndex) {
-		e := newEvent(CompareAndSwap, nodePath, index, term)
+		// update etcd index
+		s.CurrentIndex++
+
+		e := newEvent(CompareAndSwap, nodePath, s.CurrentIndex)
 		e.PrevValue = n.Value
 		e.PrevValue = n.Value
 
 
 		// if test succeed, write the value
 		// if test succeed, write the value
-		n.Write(value, index, term)
+		n.Write(value, s.CurrentIndex)
 		n.UpdateTTL(expireTime)
 		n.UpdateTTL(expireTime)
 
 
 		e.Value = value
 		e.Value = value
@@ -202,25 +209,25 @@ func (s *store) CompareAndSwap(nodePath string, prevValue string, prevIndex uint
 
 
 	cause := fmt.Sprintf("[%v != %v] [%v != %v]", prevValue, n.Value, prevIndex, n.ModifiedIndex)
 	cause := fmt.Sprintf("[%v != %v] [%v != %v]", prevValue, n.Value, prevIndex, n.ModifiedIndex)
 	s.Stats.Inc(CompareAndSwapFail)
 	s.Stats.Inc(CompareAndSwapFail)
-	return nil, etcdErr.NewError(etcdErr.EcodeTestFailed, cause, index, term)
+	return nil, etcdErr.NewError(etcdErr.EcodeTestFailed, cause, s.CurrentIndex)
 }
 }
 
 
 // Delete function deletes the node at the given path.
 // Delete function deletes the node at the given path.
 // If the node is a directory, recursive must be true to delete it.
 // 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) {
+func (s *store) Delete(nodePath string, recursive bool) (*Event, error) {
 	nodePath = path.Clean(path.Join("/", nodePath))
 	nodePath = path.Clean(path.Join("/", nodePath))
 
 
 	s.worldLock.Lock()
 	s.worldLock.Lock()
 	defer s.worldLock.Unlock()
 	defer s.worldLock.Unlock()
 
 
-	n, err := s.internalGet(nodePath, index, term)
+	n, err := s.internalGet(nodePath)
 
 
 	if err != nil { // if the node does not exist, return error
 	if err != nil { // if the node does not exist, return error
 		s.Stats.Inc(DeleteFail)
 		s.Stats.Inc(DeleteFail)
 		return nil, err
 		return nil, err
 	}
 	}
 
 
-	e := newEvent(Delete, nodePath, index, term)
+	e := newEvent(Delete, nodePath, s.CurrentIndex)
 
 
 	if n.IsDir() {
 	if n.IsDir() {
 		e.Dir = true
 		e.Dir = true
@@ -240,33 +247,38 @@ func (s *store) Delete(nodePath string, recursive bool, index uint64, term uint6
 		return nil, err
 		return nil, err
 	}
 	}
 
 
+	// update etcd index
+	s.CurrentIndex++
+	e.Index = s.CurrentIndex
+
 	s.WatcherHub.notify(e)
 	s.WatcherHub.notify(e)
 	s.Stats.Inc(DeleteSuccess)
 	s.Stats.Inc(DeleteSuccess)
 
 
 	return e, nil
 	return e, nil
 }
 }
 
 
-func (s *store) Watch(prefix string, recursive bool, sinceIndex uint64, index uint64, term uint64) (<-chan *Event, error) {
+func (s *store) Watch(prefix string, recursive bool, sinceIndex uint64) (<-chan *Event, error) {
 	prefix = path.Clean(path.Join("/", prefix))
 	prefix = path.Clean(path.Join("/", prefix))
 
 
+	nextIndex := s.CurrentIndex + 1
+
 	s.worldLock.RLock()
 	s.worldLock.RLock()
 	defer s.worldLock.RUnlock()
 	defer s.worldLock.RUnlock()
 
 
-	s.Index, s.Term = index, term
-
 	var c <-chan *Event
 	var c <-chan *Event
 	var err *etcdErr.Error
 	var err *etcdErr.Error
 
 
 	if sinceIndex == 0 {
 	if sinceIndex == 0 {
-		c, err = s.WatcherHub.watch(prefix, recursive, index+1)
+		c, err = s.WatcherHub.watch(prefix, recursive, nextIndex)
 
 
 	} else {
 	} else {
 		c, err = s.WatcherHub.watch(prefix, recursive, sinceIndex)
 		c, err = s.WatcherHub.watch(prefix, recursive, sinceIndex)
 	}
 	}
 
 
 	if err != nil {
 	if err != nil {
-		err.Index = index
-		err.Term = term
+		// watchhub do not know the current Index
+		// we need to attach the currentIndex here
+		err.Index = s.CurrentIndex
 		return nil, err
 		return nil, err
 	}
 	}
 
 
@@ -298,52 +310,59 @@ func (s *store) walk(nodePath string, walkFunc func(prev *Node, component string
 // Update function updates the value/ttl of the node.
 // 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 file, the value and the ttl can be updated.
 // If the node is a directory, only the ttl can be updated.
 // If the node is a directory, only the ttl can be updated.
-func (s *store) Update(nodePath string, newValue string, expireTime time.Time, index uint64, term uint64) (*Event, error) {
+func (s *store) Update(nodePath string, newValue string, expireTime time.Time) (*Event, error) {
 	s.worldLock.Lock()
 	s.worldLock.Lock()
 	defer s.worldLock.Unlock()
 	defer s.worldLock.Unlock()
+
+	currIndex, nextIndex := s.CurrentIndex, s.CurrentIndex+1
+
 	nodePath = path.Clean(path.Join("/", nodePath))
 	nodePath = path.Clean(path.Join("/", nodePath))
 
 
-	n, err := s.internalGet(nodePath, index, term)
+	n, err := s.internalGet(nodePath)
 
 
 	if err != nil { // if the node does not exist, return error
 	if err != nil { // if the node does not exist, return error
 		s.Stats.Inc(UpdateFail)
 		s.Stats.Inc(UpdateFail)
 		return nil, err
 		return nil, err
 	}
 	}
 
 
-	e := newEvent(Update, nodePath, s.Index, s.Term)
+	e := newEvent(Update, nodePath, nextIndex)
 
 
 	if len(newValue) != 0 {
 	if len(newValue) != 0 {
 		if n.IsDir() {
 		if n.IsDir() {
 			// if the node is a directory, we cannot update value
 			// if the node is a directory, we cannot update value
 			s.Stats.Inc(UpdateFail)
 			s.Stats.Inc(UpdateFail)
-			return nil, etcdErr.NewError(etcdErr.EcodeNotFile, nodePath, index, term)
+			return nil, etcdErr.NewError(etcdErr.EcodeNotFile, nodePath, currIndex)
 		}
 		}
 
 
 		e.PrevValue = n.Value
 		e.PrevValue = n.Value
-		n.Write(newValue, index, term)
+		n.Write(newValue, nextIndex)
+		e.Value = newValue
+	} else {
+		// do not update value
+		e.Value = n.Value
 	}
 	}
 
 
 	// update ttl
 	// update ttl
 	n.UpdateTTL(expireTime)
 	n.UpdateTTL(expireTime)
 
 
-	e.Value = newValue
-
 	e.Expiration, e.TTL = n.ExpirationAndTTL()
 	e.Expiration, e.TTL = n.ExpirationAndTTL()
 
 
 	s.WatcherHub.notify(e)
 	s.WatcherHub.notify(e)
 
 
 	s.Stats.Inc(UpdateSuccess)
 	s.Stats.Inc(UpdateSuccess)
 
 
+	s.CurrentIndex = nextIndex
+
 	return e, nil
 	return e, nil
 }
 }
 
 
 func (s *store) internalCreate(nodePath string, value string, unique bool, replace bool,
 func (s *store) internalCreate(nodePath string, value string, unique bool, replace bool,
-	expireTime time.Time, index uint64, term uint64, action string) (*Event, error) {
+	expireTime time.Time, action string) (*Event, error) {
 
 
-	s.Index, s.Term = index, term
+	currIndex, nextIndex := s.CurrentIndex, s.CurrentIndex+1
 
 
 	if unique { // append unique item under the node path
 	if unique { // append unique item under the node path
-		nodePath += "/" + strconv.FormatUint(index, 10)
+		nodePath += "/" + strconv.FormatUint(nextIndex, 10)
 	}
 	}
 
 
 	nodePath = path.Clean(path.Join("/", nodePath))
 	nodePath = path.Clean(path.Join("/", nodePath))
@@ -355,11 +374,11 @@ func (s *store) internalCreate(nodePath string, value string, unique bool, repla
 
 
 	if err != nil {
 	if err != nil {
 		s.Stats.Inc(SetFail)
 		s.Stats.Inc(SetFail)
-		err.Index, err.Term = s.Index, s.Term
+		err.Index = currIndex
 		return nil, err
 		return nil, err
 	}
 	}
 
 
-	e := newEvent(action, nodePath, s.Index, s.Term)
+	e := newEvent(action, nodePath, nextIndex)
 
 
 	n, _ := d.GetChild(newNodeName)
 	n, _ := d.GetChild(newNodeName)
 
 
@@ -367,25 +386,25 @@ func (s *store) internalCreate(nodePath string, value string, unique bool, repla
 	if n != nil {
 	if n != nil {
 		if replace {
 		if replace {
 			if n.IsDir() {
 			if n.IsDir() {
-				return nil, etcdErr.NewError(etcdErr.EcodeNotFile, nodePath, index, term)
+				return nil, etcdErr.NewError(etcdErr.EcodeNotFile, nodePath, currIndex)
 			}
 			}
 			e.PrevValue, _ = n.Read()
 			e.PrevValue, _ = n.Read()
 
 
 			n.Remove(false, nil)
 			n.Remove(false, nil)
 		} else {
 		} else {
-			return nil, etcdErr.NewError(etcdErr.EcodeNodeExist, nodePath, index, term)
+			return nil, etcdErr.NewError(etcdErr.EcodeNodeExist, nodePath, currIndex)
 		}
 		}
 	}
 	}
 
 
 	if len(value) != 0 { // create file
 	if len(value) != 0 { // create file
 		e.Value = value
 		e.Value = value
 
 
-		n = newKV(s, nodePath, value, index, term, d, "", expireTime)
+		n = newKV(s, nodePath, value, nextIndex, d, "", expireTime)
 
 
 	} else { // create directory
 	} else { // create directory
 		e.Dir = true
 		e.Dir = true
 
 
-		n = newDir(s, nodePath, index, term, d, "", expireTime)
+		n = newDir(s, nodePath, nextIndex, d, "", expireTime)
 
 
 	}
 	}
 
 
@@ -399,23 +418,20 @@ func (s *store) internalCreate(nodePath string, value string, unique bool, repla
 		e.Expiration, e.TTL = n.ExpirationAndTTL()
 		e.Expiration, e.TTL = n.ExpirationAndTTL()
 	}
 	}
 
 
+	s.CurrentIndex = nextIndex
+
 	s.WatcherHub.notify(e)
 	s.WatcherHub.notify(e)
 	return e, nil
 	return e, nil
 }
 }
 
 
 // InternalGet function get the node of the given nodePath.
 // InternalGet function get the node of the given nodePath.
-func (s *store) internalGet(nodePath string, index uint64, term uint64) (*Node, *etcdErr.Error) {
+func (s *store) internalGet(nodePath string) (*Node, *etcdErr.Error) {
 	nodePath = path.Clean(path.Join("/", nodePath))
 	nodePath = path.Clean(path.Join("/", nodePath))
 
 
-	// update file system known index and term
-	if index > s.Index {
-		s.Index, s.Term = index, term
-	}
-
 	walkFunc := func(parent *Node, name string) (*Node, *etcdErr.Error) {
 	walkFunc := func(parent *Node, name string) (*Node, *etcdErr.Error) {
 
 
 		if !parent.IsDir() {
 		if !parent.IsDir() {
-			err := etcdErr.NewError(etcdErr.EcodeNotDir, parent.Path, index, term)
+			err := etcdErr.NewError(etcdErr.EcodeNotDir, parent.Path, s.CurrentIndex)
 			return nil, err
 			return nil, err
 		}
 		}
 
 
@@ -424,7 +440,7 @@ func (s *store) internalGet(nodePath string, index uint64, term uint64) (*Node,
 			return child, nil
 			return child, nil
 		}
 		}
 
 
-		return nil, etcdErr.NewError(etcdErr.EcodeKeyNotFound, path.Join(parent.Path, name), index, term)
+		return nil, etcdErr.NewError(etcdErr.EcodeKeyNotFound, path.Join(parent.Path, name), s.CurrentIndex)
 	}
 	}
 
 
 	f, err := s.walk(nodePath, walkFunc)
 	f, err := s.walk(nodePath, walkFunc)
@@ -436,12 +452,10 @@ func (s *store) internalGet(nodePath string, index uint64, term uint64) (*Node,
 }
 }
 
 
 // deleteExpiredKyes will delete all
 // deleteExpiredKyes will delete all
-func (s *store) DeleteExpiredKeys(cutoff time.Time, index uint64, term uint64) {
+func (s *store) DeleteExpiredKeys(cutoff time.Time) {
 	s.worldLock.Lock()
 	s.worldLock.Lock()
 	defer s.worldLock.Unlock()
 	defer s.worldLock.Unlock()
 
 
-	s.Index, s.Term = index, term
-
 	for {
 	for {
 		node := s.ttlKeyHeap.top()
 		node := s.ttlKeyHeap.top()
 		if node == nil || node.ExpireTime.After(cutoff) {
 		if node == nil || node.ExpireTime.After(cutoff) {
@@ -451,10 +465,12 @@ func (s *store) DeleteExpiredKeys(cutoff time.Time, index uint64, term uint64) {
 		s.ttlKeyHeap.pop()
 		s.ttlKeyHeap.pop()
 		node.Remove(true, nil)
 		node.Remove(true, nil)
 
 
+		s.CurrentIndex++
+
 		s.Stats.Inc(ExpireCount)
 		s.Stats.Inc(ExpireCount)
-		s.WatcherHub.notify(newEvent(Expire, node.Path, s.Index, s.Term))
+		s.WatcherHub.notify(newEvent(Expire, node.Path, s.CurrentIndex))
 	}
 	}
-	s.WatcherHub.clearPendingWatchers()
+
 }
 }
 
 
 // checkDir function will check whether the component is a directory under parent node.
 // checkDir function will check whether the component is a directory under parent node.
@@ -469,10 +485,10 @@ func (s *store) checkDir(parent *Node, dirName string) (*Node, *etcdErr.Error) {
 			return node, nil
 			return node, nil
 		}
 		}
 
 
-		return nil, etcdErr.NewError(etcdErr.EcodeNotDir, parent.Path, UndefIndex, UndefTerm)
+		return nil, etcdErr.NewError(etcdErr.EcodeNotDir, parent.Path, s.CurrentIndex)
 	}
 	}
 
 
-	n := newDir(s, path.Join(parent.Path, dirName), s.Index, s.Term, parent, parent.ACL, Permanent)
+	n := newDir(s, path.Join(parent.Path, dirName), s.CurrentIndex, parent, parent.ACL, Permanent)
 
 
 	parent.Children[dirName] = n
 	parent.Children[dirName] = n
 
 
@@ -487,8 +503,7 @@ func (s *store) Save() ([]byte, error) {
 	s.worldLock.Lock()
 	s.worldLock.Lock()
 
 
 	clonedStore := newStore()
 	clonedStore := newStore()
-	clonedStore.Index = s.Index
-	clonedStore.Term = s.Term
+	clonedStore.CurrentIndex = s.CurrentIndex
 	clonedStore.Root = s.Root.Clone()
 	clonedStore.Root = s.Root.Clone()
 	clonedStore.WatcherHub = s.WatcherHub.clone()
 	clonedStore.WatcherHub = s.WatcherHub.clone()
 	clonedStore.Stats = s.Stats.clone()
 	clonedStore.Stats = s.Stats.clone()

+ 99 - 101
store/store_test.go

@@ -11,8 +11,8 @@ import (
 // Ensure that the store can retrieve an existing value.
 // Ensure that the store can retrieve an existing value.
 func TestStoreGetValue(t *testing.T) {
 func TestStoreGetValue(t *testing.T) {
 	s := newStore()
 	s := newStore()
-	s.Create("/foo", "bar", false, Permanent, 2, 1)
-	e, err := s.Get("/foo", false, false, 2, 1)
+	s.Create("/foo", "bar", false, Permanent)
+	e, err := s.Get("/foo", false, false)
 	assert.Nil(t, err, "")
 	assert.Nil(t, err, "")
 	assert.Equal(t, e.Action, "get", "")
 	assert.Equal(t, e.Action, "get", "")
 	assert.Equal(t, e.Key, "/foo", "")
 	assert.Equal(t, e.Key, "/foo", "")
@@ -23,14 +23,14 @@ func TestStoreGetValue(t *testing.T) {
 // Note that hidden files should not be returned.
 // Note that hidden files should not be returned.
 func TestStoreGetDirectory(t *testing.T) {
 func TestStoreGetDirectory(t *testing.T) {
 	s := newStore()
 	s := newStore()
-	s.Create("/foo", "", false, Permanent, 2, 1)
-	s.Create("/foo/bar", "X", false, Permanent, 3, 1)
-	s.Create("/foo/_hidden", "*", false, Permanent, 4, 1)
-	s.Create("/foo/baz", "", false, Permanent, 5, 1)
-	s.Create("/foo/baz/bat", "Y", false, Permanent, 6, 1)
-	s.Create("/foo/baz/_hidden", "*", false, Permanent, 7, 1)
-	s.Create("/foo/baz/ttl", "Y", false, time.Now().Add(time.Second*3), 8, 1)
-	e, err := s.Get("/foo", true, false, 8, 1)
+	s.Create("/foo", "", false, Permanent)
+	s.Create("/foo/bar", "X", false, Permanent)
+	s.Create("/foo/_hidden", "*", false, Permanent)
+	s.Create("/foo/baz", "", false, Permanent)
+	s.Create("/foo/baz/bat", "Y", false, Permanent)
+	s.Create("/foo/baz/_hidden", "*", false, Permanent)
+	s.Create("/foo/baz/ttl", "Y", false, time.Now().Add(time.Second*3))
+	e, err := s.Get("/foo", true, false)
 	assert.Nil(t, err, "")
 	assert.Nil(t, err, "")
 	assert.Equal(t, e.Action, "get", "")
 	assert.Equal(t, e.Action, "get", "")
 	assert.Equal(t, e.Key, "/foo", "")
 	assert.Equal(t, e.Key, "/foo", "")
@@ -53,13 +53,13 @@ func TestStoreGetDirectory(t *testing.T) {
 // Ensure that the store can retrieve a directory in sorted order.
 // Ensure that the store can retrieve a directory in sorted order.
 func TestStoreGetSorted(t *testing.T) {
 func TestStoreGetSorted(t *testing.T) {
 	s := newStore()
 	s := newStore()
-	s.Create("/foo", "", false, Permanent, 2, 1)
-	s.Create("/foo/x", "0", false, Permanent, 3, 1)
-	s.Create("/foo/z", "0", false, Permanent, 4, 1)
-	s.Create("/foo/y", "", false, Permanent, 5, 1)
-	s.Create("/foo/y/a", "0", false, Permanent, 6, 1)
-	s.Create("/foo/y/b", "0", false, Permanent, 7, 1)
-	e, err := s.Get("/foo", true, true, 8, 1)
+	s.Create("/foo", "", false, Permanent)
+	s.Create("/foo/x", "0", false, Permanent)
+	s.Create("/foo/z", "0", false, Permanent)
+	s.Create("/foo/y", "", false, Permanent)
+	s.Create("/foo/y/a", "0", false, Permanent)
+	s.Create("/foo/y/b", "0", false, Permanent)
+	e, err := s.Get("/foo", true, true)
 	assert.Nil(t, err, "")
 	assert.Nil(t, err, "")
 	assert.Equal(t, e.KVPairs[0].Key, "/foo/x", "")
 	assert.Equal(t, e.KVPairs[0].Key, "/foo/x", "")
 	assert.Equal(t, e.KVPairs[1].Key, "/foo/y", "")
 	assert.Equal(t, e.KVPairs[1].Key, "/foo/y", "")
@@ -71,7 +71,7 @@ func TestStoreGetSorted(t *testing.T) {
 // Ensure that the store can create a new key if it doesn't already exist.
 // Ensure that the store can create a new key if it doesn't already exist.
 func TestStoreCreateValue(t *testing.T) {
 func TestStoreCreateValue(t *testing.T) {
 	s := newStore()
 	s := newStore()
-	e, err := s.Create("/foo", "bar", false, Permanent, 2, 1)
+	e, err := s.Create("/foo", "bar", false, Permanent)
 	assert.Nil(t, err, "")
 	assert.Nil(t, err, "")
 	assert.Equal(t, e.Action, "create", "")
 	assert.Equal(t, e.Action, "create", "")
 	assert.Equal(t, e.Key, "/foo", "")
 	assert.Equal(t, e.Key, "/foo", "")
@@ -81,14 +81,13 @@ func TestStoreCreateValue(t *testing.T) {
 	assert.Nil(t, e.KVPairs, "")
 	assert.Nil(t, e.KVPairs, "")
 	assert.Nil(t, e.Expiration, "")
 	assert.Nil(t, e.Expiration, "")
 	assert.Equal(t, e.TTL, 0, "")
 	assert.Equal(t, e.TTL, 0, "")
-	assert.Equal(t, e.Index, uint64(2), "")
-	assert.Equal(t, e.Term, uint64(1), "")
+	assert.Equal(t, e.Index, uint64(1), "")
 }
 }
 
 
 // Ensure that the store can create a new directory if it doesn't already exist.
 // Ensure that the store can create a new directory if it doesn't already exist.
 func TestStoreCreateDirectory(t *testing.T) {
 func TestStoreCreateDirectory(t *testing.T) {
 	s := newStore()
 	s := newStore()
-	e, err := s.Create("/foo", "", false, Permanent, 2, 1)
+	e, err := s.Create("/foo", "", false, Permanent)
 	assert.Nil(t, err, "")
 	assert.Nil(t, err, "")
 	assert.Equal(t, e.Action, "create", "")
 	assert.Equal(t, e.Action, "create", "")
 	assert.Equal(t, e.Key, "/foo", "")
 	assert.Equal(t, e.Key, "/foo", "")
@@ -98,22 +97,21 @@ func TestStoreCreateDirectory(t *testing.T) {
 // Ensure that the store fails to create a key if it already exists.
 // Ensure that the store fails to create a key if it already exists.
 func TestStoreCreateFailsIfExists(t *testing.T) {
 func TestStoreCreateFailsIfExists(t *testing.T) {
 	s := newStore()
 	s := newStore()
-	s.Create("/foo", "", false, Permanent, 2, 1)
-	e, _err := s.Create("/foo", "", false, Permanent, 3, 1)
+	s.Create("/foo", "", false, Permanent)
+	e, _err := s.Create("/foo", "", false, Permanent)
 	err := _err.(*etcdErr.Error)
 	err := _err.(*etcdErr.Error)
 	assert.Equal(t, err.ErrorCode, etcdErr.EcodeNodeExist, "")
 	assert.Equal(t, err.ErrorCode, etcdErr.EcodeNodeExist, "")
 	assert.Equal(t, err.Message, "Already exists", "")
 	assert.Equal(t, err.Message, "Already exists", "")
 	assert.Equal(t, err.Cause, "/foo", "")
 	assert.Equal(t, err.Cause, "/foo", "")
-	assert.Equal(t, err.Index, uint64(3), "")
-	assert.Equal(t, err.Term, uint64(1), "")
+	assert.Equal(t, err.Index, uint64(1), "")
 	assert.Nil(t, e, 0, "")
 	assert.Nil(t, e, 0, "")
 }
 }
 
 
 // Ensure that the store can update a key if it already exists.
 // Ensure that the store can update a key if it already exists.
 func TestStoreUpdateValue(t *testing.T) {
 func TestStoreUpdateValue(t *testing.T) {
 	s := newStore()
 	s := newStore()
-	s.Create("/foo", "bar", false, Permanent, 2, 1)
-	e, err := s.Update("/foo", "baz", Permanent, 3, 1)
+	s.Create("/foo", "bar", false, Permanent)
+	e, err := s.Update("/foo", "baz", Permanent)
 	assert.Nil(t, err, "")
 	assert.Nil(t, err, "")
 	assert.Equal(t, e.Action, "update", "")
 	assert.Equal(t, e.Action, "update", "")
 	assert.Equal(t, e.Key, "/foo", "")
 	assert.Equal(t, e.Key, "/foo", "")
@@ -121,17 +119,16 @@ func TestStoreUpdateValue(t *testing.T) {
 	assert.Equal(t, e.PrevValue, "bar", "")
 	assert.Equal(t, e.PrevValue, "bar", "")
 	assert.Equal(t, e.Value, "baz", "")
 	assert.Equal(t, e.Value, "baz", "")
 	assert.Equal(t, e.TTL, 0, "")
 	assert.Equal(t, e.TTL, 0, "")
-	assert.Equal(t, e.Index, uint64(3), "")
-	assert.Equal(t, e.Term, uint64(1), "")
-	e, _ = s.Get("/foo", false, false, 3, 1)
+	assert.Equal(t, e.Index, uint64(2), "")
+	e, _ = s.Get("/foo", false, false)
 	assert.Equal(t, e.Value, "baz", "")
 	assert.Equal(t, e.Value, "baz", "")
 }
 }
 
 
 // Ensure that the store cannot update a directory.
 // Ensure that the store cannot update a directory.
 func TestStoreUpdateFailsIfDirectory(t *testing.T) {
 func TestStoreUpdateFailsIfDirectory(t *testing.T) {
 	s := newStore()
 	s := newStore()
-	s.Create("/foo", "", false, Permanent, 2, 1)
-	e, _err := s.Update("/foo", "baz", Permanent, 3, 1)
+	s.Create("/foo", "", false, Permanent)
+	e, _err := s.Update("/foo", "baz", Permanent)
 	err := _err.(*etcdErr.Error)
 	err := _err.(*etcdErr.Error)
 	assert.Equal(t, err.ErrorCode, etcdErr.EcodeNotFile, "")
 	assert.Equal(t, err.ErrorCode, etcdErr.EcodeNotFile, "")
 	assert.Equal(t, err.Message, "Not A File", "")
 	assert.Equal(t, err.Message, "Not A File", "")
@@ -142,14 +139,14 @@ func TestStoreUpdateFailsIfDirectory(t *testing.T) {
 // Ensure that the store can update the TTL on a value.
 // Ensure that the store can update the TTL on a value.
 func TestStoreUpdateValueTTL(t *testing.T) {
 func TestStoreUpdateValueTTL(t *testing.T) {
 	s := newStore()
 	s := newStore()
-	go mockSyncService(s.deleteExpiredKeys)
-	s.Create("/foo", "bar", false, Permanent, 2, 1)
-	_, err := s.Update("/foo", "baz", time.Now().Add(500*time.Millisecond), 3, 1)
-	e, _ := s.Get("/foo", false, false, 3, 1)
+	go mockSyncService(s.DeleteExpiredKeys)
+	s.Create("/foo", "bar", false, Permanent)
+	_, err := s.Update("/foo", "baz", time.Now().Add(500*time.Millisecond))
+	e, _ := s.Get("/foo", false, false)
 	assert.Equal(t, e.Value, "baz", "")
 	assert.Equal(t, e.Value, "baz", "")
 
 
 	time.Sleep(600 * time.Millisecond)
 	time.Sleep(600 * time.Millisecond)
-	e, err = s.Get("/foo", false, false, 3, 1)
+	e, err = s.Get("/foo", false, false)
 	assert.Nil(t, e, "")
 	assert.Nil(t, e, "")
 	assert.Equal(t, err.(*etcdErr.Error).ErrorCode, etcdErr.EcodeKeyNotFound, "")
 	assert.Equal(t, err.(*etcdErr.Error).ErrorCode, etcdErr.EcodeKeyNotFound, "")
 }
 }
@@ -157,15 +154,15 @@ func TestStoreUpdateValueTTL(t *testing.T) {
 // Ensure that the store can update the TTL on a directory.
 // Ensure that the store can update the TTL on a directory.
 func TestStoreUpdateDirTTL(t *testing.T) {
 func TestStoreUpdateDirTTL(t *testing.T) {
 	s := newStore()
 	s := newStore()
-	go mockSyncService(s.deleteExpiredKeys)
-	s.Create("/foo", "", false, Permanent, 2, 1)
-	s.Create("/foo/bar", "baz", false, Permanent, 3, 1)
-	_, err := s.Update("/foo", "", time.Now().Add(500*time.Millisecond), 3, 1)
-	e, _ := s.Get("/foo/bar", false, false, 3, 1)
+	go mockSyncService(s.DeleteExpiredKeys)
+	s.Create("/foo", "", false, Permanent)
+	s.Create("/foo/bar", "baz", false, Permanent)
+	_, err := s.Update("/foo", "", time.Now().Add(500*time.Millisecond))
+	e, _ := s.Get("/foo/bar", false, false)
 	assert.Equal(t, e.Value, "baz", "")
 	assert.Equal(t, e.Value, "baz", "")
 
 
 	time.Sleep(600 * time.Millisecond)
 	time.Sleep(600 * time.Millisecond)
-	e, err = s.Get("/foo/bar", false, false, 3, 1)
+	e, err = s.Get("/foo/bar", false, false)
 	assert.Nil(t, e, "")
 	assert.Nil(t, e, "")
 	assert.Equal(t, err.(*etcdErr.Error).ErrorCode, etcdErr.EcodeKeyNotFound, "")
 	assert.Equal(t, err.(*etcdErr.Error).ErrorCode, etcdErr.EcodeKeyNotFound, "")
 }
 }
@@ -173,8 +170,8 @@ func TestStoreUpdateDirTTL(t *testing.T) {
 // Ensure that the store can delete a value.
 // Ensure that the store can delete a value.
 func TestStoreDeleteValue(t *testing.T) {
 func TestStoreDeleteValue(t *testing.T) {
 	s := newStore()
 	s := newStore()
-	s.Create("/foo", "bar", false, Permanent, 2, 1)
-	e, err := s.Delete("/foo", false, 3, 1)
+	s.Create("/foo", "bar", false, Permanent)
+	e, err := s.Delete("/foo", false)
 	assert.Nil(t, err, "")
 	assert.Nil(t, err, "")
 	assert.Equal(t, e.Action, "delete", "")
 	assert.Equal(t, e.Action, "delete", "")
 }
 }
@@ -182,8 +179,8 @@ func TestStoreDeleteValue(t *testing.T) {
 // Ensure that the store can delete a directory if recursive is specified.
 // Ensure that the store can delete a directory if recursive is specified.
 func TestStoreDeleteDiretory(t *testing.T) {
 func TestStoreDeleteDiretory(t *testing.T) {
 	s := newStore()
 	s := newStore()
-	s.Create("/foo", "", false, Permanent, 2, 1)
-	e, err := s.Delete("/foo", true, 3, 1)
+	s.Create("/foo", "", false, Permanent)
+	e, err := s.Delete("/foo", true)
 	assert.Nil(t, err, "")
 	assert.Nil(t, err, "")
 	assert.Equal(t, e.Action, "delete", "")
 	assert.Equal(t, e.Action, "delete", "")
 }
 }
@@ -191,8 +188,8 @@ func TestStoreDeleteDiretory(t *testing.T) {
 // Ensure that the store cannot delete a directory if recursive is not specified.
 // Ensure that the store cannot delete a directory if recursive is not specified.
 func TestStoreDeleteDiretoryFailsIfNonRecursive(t *testing.T) {
 func TestStoreDeleteDiretoryFailsIfNonRecursive(t *testing.T) {
 	s := newStore()
 	s := newStore()
-	s.Create("/foo", "", false, Permanent, 2, 1)
-	e, _err := s.Delete("/foo", false, 3, 1)
+	s.Create("/foo", "", false, Permanent)
+	e, _err := s.Delete("/foo", false)
 	err := _err.(*etcdErr.Error)
 	err := _err.(*etcdErr.Error)
 	assert.Equal(t, err.ErrorCode, etcdErr.EcodeNotFile, "")
 	assert.Equal(t, err.ErrorCode, etcdErr.EcodeNotFile, "")
 	assert.Equal(t, err.Message, "Not A File", "")
 	assert.Equal(t, err.Message, "Not A File", "")
@@ -202,60 +199,60 @@ func TestStoreDeleteDiretoryFailsIfNonRecursive(t *testing.T) {
 // Ensure that the store can conditionally update a key if it has a previous value.
 // Ensure that the store can conditionally update a key if it has a previous value.
 func TestStoreCompareAndSwapPrevValue(t *testing.T) {
 func TestStoreCompareAndSwapPrevValue(t *testing.T) {
 	s := newStore()
 	s := newStore()
-	s.Create("/foo", "bar", false, Permanent, 2, 1)
-	e, err := s.CompareAndSwap("/foo", "bar", 0, "baz", Permanent, 3, 1)
+	s.Create("/foo", "bar", false, Permanent)
+	e, err := s.CompareAndSwap("/foo", "bar", 0, "baz", Permanent)
 	assert.Nil(t, err, "")
 	assert.Nil(t, err, "")
 	assert.Equal(t, e.Action, "compareAndSwap", "")
 	assert.Equal(t, e.Action, "compareAndSwap", "")
 	assert.Equal(t, e.PrevValue, "bar", "")
 	assert.Equal(t, e.PrevValue, "bar", "")
 	assert.Equal(t, e.Value, "baz", "")
 	assert.Equal(t, e.Value, "baz", "")
-	e, _ = s.Get("/foo", false, false, 3, 1)
+	e, _ = s.Get("/foo", false, false)
 	assert.Equal(t, e.Value, "baz", "")
 	assert.Equal(t, e.Value, "baz", "")
 }
 }
 
 
 // Ensure that the store cannot conditionally update a key if it has the wrong previous value.
 // Ensure that the store cannot conditionally update a key if it has the wrong previous value.
 func TestStoreCompareAndSwapPrevValueFailsIfNotMatch(t *testing.T) {
 func TestStoreCompareAndSwapPrevValueFailsIfNotMatch(t *testing.T) {
 	s := newStore()
 	s := newStore()
-	s.Create("/foo", "bar", false, Permanent, 2, 1)
-	e, _err := s.CompareAndSwap("/foo", "wrong_value", 0, "baz", Permanent, 3, 1)
+	s.Create("/foo", "bar", false, Permanent)
+	e, _err := s.CompareAndSwap("/foo", "wrong_value", 0, "baz", Permanent)
 	err := _err.(*etcdErr.Error)
 	err := _err.(*etcdErr.Error)
 	assert.Equal(t, err.ErrorCode, etcdErr.EcodeTestFailed, "")
 	assert.Equal(t, err.ErrorCode, etcdErr.EcodeTestFailed, "")
 	assert.Equal(t, err.Message, "Test Failed", "")
 	assert.Equal(t, err.Message, "Test Failed", "")
 	assert.Nil(t, e, "")
 	assert.Nil(t, e, "")
-	e, _ = s.Get("/foo", false, false, 3, 1)
+	e, _ = s.Get("/foo", false, false)
 	assert.Equal(t, e.Value, "bar", "")
 	assert.Equal(t, e.Value, "bar", "")
 }
 }
 
 
 // Ensure that the store can conditionally update a key if it has a previous index.
 // Ensure that the store can conditionally update a key if it has a previous index.
 func TestStoreCompareAndSwapPrevIndex(t *testing.T) {
 func TestStoreCompareAndSwapPrevIndex(t *testing.T) {
 	s := newStore()
 	s := newStore()
-	s.Create("/foo", "bar", false, Permanent, 2, 1)
-	e, err := s.CompareAndSwap("/foo", "", 2, "baz", Permanent, 3, 1)
+	s.Create("/foo", "bar", false, Permanent)
+	e, err := s.CompareAndSwap("/foo", "", 1, "baz", Permanent)
 	assert.Nil(t, err, "")
 	assert.Nil(t, err, "")
 	assert.Equal(t, e.Action, "compareAndSwap", "")
 	assert.Equal(t, e.Action, "compareAndSwap", "")
 	assert.Equal(t, e.PrevValue, "bar", "")
 	assert.Equal(t, e.PrevValue, "bar", "")
 	assert.Equal(t, e.Value, "baz", "")
 	assert.Equal(t, e.Value, "baz", "")
-	e, _ = s.Get("/foo", false, false, 3, 1)
+	e, _ = s.Get("/foo", false, false)
 	assert.Equal(t, e.Value, "baz", "")
 	assert.Equal(t, e.Value, "baz", "")
 }
 }
 
 
 // Ensure that the store cannot conditionally update a key if it has the wrong previous index.
 // Ensure that the store cannot conditionally update a key if it has the wrong previous index.
 func TestStoreCompareAndSwapPrevIndexFailsIfNotMatch(t *testing.T) {
 func TestStoreCompareAndSwapPrevIndexFailsIfNotMatch(t *testing.T) {
 	s := newStore()
 	s := newStore()
-	s.Create("/foo", "bar", false, Permanent, 2, 1)
-	e, _err := s.CompareAndSwap("/foo", "", 100, "baz", Permanent, 3, 1)
+	s.Create("/foo", "bar", false, Permanent)
+	e, _err := s.CompareAndSwap("/foo", "", 100, "baz", Permanent)
 	err := _err.(*etcdErr.Error)
 	err := _err.(*etcdErr.Error)
 	assert.Equal(t, err.ErrorCode, etcdErr.EcodeTestFailed, "")
 	assert.Equal(t, err.ErrorCode, etcdErr.EcodeTestFailed, "")
 	assert.Equal(t, err.Message, "Test Failed", "")
 	assert.Equal(t, err.Message, "Test Failed", "")
 	assert.Nil(t, e, "")
 	assert.Nil(t, e, "")
-	e, _ = s.Get("/foo", false, false, 3, 1)
+	e, _ = s.Get("/foo", false, false)
 	assert.Equal(t, e.Value, "bar", "")
 	assert.Equal(t, e.Value, "bar", "")
 }
 }
 
 
 // Ensure that the store can watch for key creation.
 // Ensure that the store can watch for key creation.
 func TestStoreWatchCreate(t *testing.T) {
 func TestStoreWatchCreate(t *testing.T) {
 	s := newStore()
 	s := newStore()
-	c, _ := s.Watch("/foo", false, 0, 0, 1)
-	s.Create("/foo", "bar", false, Permanent, 2, 1)
+	c, _ := s.Watch("/foo", false, 0)
+	s.Create("/foo", "bar", false, Permanent)
 	e := nbselect(c)
 	e := nbselect(c)
 	assert.Equal(t, e.Action, "create", "")
 	assert.Equal(t, e.Action, "create", "")
 	assert.Equal(t, e.Key, "/foo", "")
 	assert.Equal(t, e.Key, "/foo", "")
@@ -266,8 +263,8 @@ func TestStoreWatchCreate(t *testing.T) {
 // Ensure that the store can watch for recursive key creation.
 // Ensure that the store can watch for recursive key creation.
 func TestStoreWatchRecursiveCreate(t *testing.T) {
 func TestStoreWatchRecursiveCreate(t *testing.T) {
 	s := newStore()
 	s := newStore()
-	c, _ := s.Watch("/foo", true, 0, 0, 1)
-	s.Create("/foo/bar", "baz", false, Permanent, 2, 1)
+	c, _ := s.Watch("/foo", true, 0)
+	s.Create("/foo/bar", "baz", false, Permanent)
 	e := nbselect(c)
 	e := nbselect(c)
 	assert.Equal(t, e.Action, "create", "")
 	assert.Equal(t, e.Action, "create", "")
 	assert.Equal(t, e.Key, "/foo/bar", "")
 	assert.Equal(t, e.Key, "/foo/bar", "")
@@ -276,9 +273,9 @@ func TestStoreWatchRecursiveCreate(t *testing.T) {
 // Ensure that the store can watch for key updates.
 // Ensure that the store can watch for key updates.
 func TestStoreWatchUpdate(t *testing.T) {
 func TestStoreWatchUpdate(t *testing.T) {
 	s := newStore()
 	s := newStore()
-	s.Create("/foo", "bar", false, Permanent, 2, 1)
-	c, _ := s.Watch("/foo", false, 0, 0, 1)
-	s.Update("/foo", "baz", Permanent, 3, 1)
+	s.Create("/foo", "bar", false, Permanent)
+	c, _ := s.Watch("/foo", false, 0)
+	s.Update("/foo", "baz", Permanent)
 	e := nbselect(c)
 	e := nbselect(c)
 	assert.Equal(t, e.Action, "update", "")
 	assert.Equal(t, e.Action, "update", "")
 	assert.Equal(t, e.Key, "/foo", "")
 	assert.Equal(t, e.Key, "/foo", "")
@@ -287,9 +284,9 @@ func TestStoreWatchUpdate(t *testing.T) {
 // Ensure that the store can watch for recursive key updates.
 // Ensure that the store can watch for recursive key updates.
 func TestStoreWatchRecursiveUpdate(t *testing.T) {
 func TestStoreWatchRecursiveUpdate(t *testing.T) {
 	s := newStore()
 	s := newStore()
-	s.Create("/foo/bar", "baz", false, Permanent, 2, 1)
-	c, _ := s.Watch("/foo", true, 0, 0, 1)
-	s.Update("/foo/bar", "baz", Permanent, 3, 1)
+	s.Create("/foo/bar", "baz", false, Permanent)
+	c, _ := s.Watch("/foo", true, 0)
+	s.Update("/foo/bar", "baz", Permanent)
 	e := nbselect(c)
 	e := nbselect(c)
 	assert.Equal(t, e.Action, "update", "")
 	assert.Equal(t, e.Action, "update", "")
 	assert.Equal(t, e.Key, "/foo/bar", "")
 	assert.Equal(t, e.Key, "/foo/bar", "")
@@ -298,9 +295,9 @@ func TestStoreWatchRecursiveUpdate(t *testing.T) {
 // Ensure that the store can watch for key deletions.
 // Ensure that the store can watch for key deletions.
 func TestStoreWatchDelete(t *testing.T) {
 func TestStoreWatchDelete(t *testing.T) {
 	s := newStore()
 	s := newStore()
-	s.Create("/foo", "bar", false, Permanent, 2, 1)
-	c, _ := s.Watch("/foo", false, 0, 0, 1)
-	s.Delete("/foo", false, 3, 1)
+	s.Create("/foo", "bar", false, Permanent)
+	c, _ := s.Watch("/foo", false, 0)
+	s.Delete("/foo", false)
 	e := nbselect(c)
 	e := nbselect(c)
 	assert.Equal(t, e.Action, "delete", "")
 	assert.Equal(t, e.Action, "delete", "")
 	assert.Equal(t, e.Key, "/foo", "")
 	assert.Equal(t, e.Key, "/foo", "")
@@ -309,9 +306,9 @@ func TestStoreWatchDelete(t *testing.T) {
 // Ensure that the store can watch for recursive key deletions.
 // Ensure that the store can watch for recursive key deletions.
 func TestStoreWatchRecursiveDelete(t *testing.T) {
 func TestStoreWatchRecursiveDelete(t *testing.T) {
 	s := newStore()
 	s := newStore()
-	s.Create("/foo/bar", "baz", false, Permanent, 2, 1)
-	c, _ := s.Watch("/foo", true, 0, 0, 1)
-	s.Delete("/foo/bar", false, 3, 1)
+	s.Create("/foo/bar", "baz", false, Permanent)
+	c, _ := s.Watch("/foo", true, 0)
+	s.Delete("/foo/bar", false)
 	e := nbselect(c)
 	e := nbselect(c)
 	assert.Equal(t, e.Action, "delete", "")
 	assert.Equal(t, e.Action, "delete", "")
 	assert.Equal(t, e.Key, "/foo/bar", "")
 	assert.Equal(t, e.Key, "/foo/bar", "")
@@ -320,9 +317,9 @@ func TestStoreWatchRecursiveDelete(t *testing.T) {
 // Ensure that the store can watch for CAS updates.
 // Ensure that the store can watch for CAS updates.
 func TestStoreWatchCompareAndSwap(t *testing.T) {
 func TestStoreWatchCompareAndSwap(t *testing.T) {
 	s := newStore()
 	s := newStore()
-	s.Create("/foo", "bar", false, Permanent, 2, 1)
-	c, _ := s.Watch("/foo", false, 0, 0, 1)
-	s.CompareAndSwap("/foo", "bar", 0, "baz", Permanent, 3, 1)
+	s.Create("/foo", "bar", false, Permanent)
+	c, _ := s.Watch("/foo", false, 0)
+	s.CompareAndSwap("/foo", "bar", 0, "baz", Permanent)
 	e := nbselect(c)
 	e := nbselect(c)
 	assert.Equal(t, e.Action, "compareAndSwap", "")
 	assert.Equal(t, e.Action, "compareAndSwap", "")
 	assert.Equal(t, e.Key, "/foo", "")
 	assert.Equal(t, e.Key, "/foo", "")
@@ -331,9 +328,9 @@ func TestStoreWatchCompareAndSwap(t *testing.T) {
 // Ensure that the store can watch for recursive CAS updates.
 // Ensure that the store can watch for recursive CAS updates.
 func TestStoreWatchRecursiveCompareAndSwap(t *testing.T) {
 func TestStoreWatchRecursiveCompareAndSwap(t *testing.T) {
 	s := newStore()
 	s := newStore()
-	s.Create("/foo/bar", "baz", false, Permanent, 2, 1)
-	c, _ := s.Watch("/foo", true, 0, 0, 1)
-	s.CompareAndSwap("/foo/bar", "baz", 0, "bat", Permanent, 3, 1)
+	s.Create("/foo/bar", "baz", false, Permanent)
+	c, _ := s.Watch("/foo", true, 0)
+	s.CompareAndSwap("/foo/bar", "baz", 0, "bat", Permanent)
 	e := nbselect(c)
 	e := nbselect(c)
 	assert.Equal(t, e.Action, "compareAndSwap", "")
 	assert.Equal(t, e.Action, "compareAndSwap", "")
 	assert.Equal(t, e.Key, "/foo/bar", "")
 	assert.Equal(t, e.Key, "/foo/bar", "")
@@ -342,17 +339,18 @@ func TestStoreWatchRecursiveCompareAndSwap(t *testing.T) {
 // Ensure that the store can watch for key expiration.
 // Ensure that the store can watch for key expiration.
 func TestStoreWatchExpire(t *testing.T) {
 func TestStoreWatchExpire(t *testing.T) {
 	s := newStore()
 	s := newStore()
-	go mockSyncService(s.deleteExpiredKeys)
-	s.Create("/foo", "bar", false, time.Now().Add(500*time.Millisecond), 2, 1)
-	s.Create("/foofoo", "barbarbar", false, time.Now().Add(500*time.Millisecond), 2, 1)
+	go mockSyncService(s.DeleteExpiredKeys)
+	s.Create("/foo", "bar", false, time.Now().Add(500*time.Millisecond))
+	s.Create("/foofoo", "barbarbar", false, time.Now().Add(500*time.Millisecond))
 
 
-	c, _ := s.Watch("/", true, 0, 0, 1)
+	c, _ := s.Watch("/", true, 0)
 	e := nbselect(c)
 	e := nbselect(c)
 	assert.Nil(t, e, "")
 	assert.Nil(t, e, "")
 	time.Sleep(600 * time.Millisecond)
 	time.Sleep(600 * time.Millisecond)
 	e = nbselect(c)
 	e = nbselect(c)
 	assert.Equal(t, e.Action, "expire", "")
 	assert.Equal(t, e.Action, "expire", "")
 	assert.Equal(t, e.Key, "/foo", "")
 	assert.Equal(t, e.Key, "/foo", "")
+	c, _ = s.Watch("/", true, 4)
 	e = nbselect(c)
 	e = nbselect(c)
 	assert.Equal(t, e.Action, "expire", "")
 	assert.Equal(t, e.Action, "expire", "")
 	assert.Equal(t, e.Key, "/foofoo", "")
 	assert.Equal(t, e.Key, "/foofoo", "")
@@ -361,19 +359,19 @@ func TestStoreWatchExpire(t *testing.T) {
 // Ensure that the store can recover from a previously saved state.
 // Ensure that the store can recover from a previously saved state.
 func TestStoreRecover(t *testing.T) {
 func TestStoreRecover(t *testing.T) {
 	s := newStore()
 	s := newStore()
-	s.Create("/foo", "", false, Permanent, 2, 1)
-	s.Create("/foo/x", "bar", false, Permanent, 3, 1)
-	s.Create("/foo/y", "baz", false, Permanent, 4, 1)
+	s.Create("/foo", "", false, Permanent)
+	s.Create("/foo/x", "bar", false, Permanent)
+	s.Create("/foo/y", "baz", false, Permanent)
 	b, err := s.Save()
 	b, err := s.Save()
 
 
 	s2 := newStore()
 	s2 := newStore()
 	s2.Recovery(b)
 	s2.Recovery(b)
 
 
-	e, err := s.Get("/foo/x", false, false, 4, 1)
+	e, err := s.Get("/foo/x", false, false)
 	assert.Nil(t, err, "")
 	assert.Nil(t, err, "")
 	assert.Equal(t, e.Value, "bar", "")
 	assert.Equal(t, e.Value, "bar", "")
 
 
-	e, err = s.Get("/foo/y", false, false, 4, 1)
+	e, err = s.Get("/foo/y", false, false)
 	assert.Nil(t, err, "")
 	assert.Nil(t, err, "")
 	assert.Equal(t, e.Value, "baz", "")
 	assert.Equal(t, e.Value, "baz", "")
 }
 }
@@ -381,25 +379,25 @@ func TestStoreRecover(t *testing.T) {
 // Ensure that the store can recover from a previously saved state that includes an expiring key.
 // Ensure that the store can recover from a previously saved state that includes an expiring key.
 func TestStoreRecoverWithExpiration(t *testing.T) {
 func TestStoreRecoverWithExpiration(t *testing.T) {
 	s := newStore()
 	s := newStore()
-	go mockSyncService(s.deleteExpiredKeys)
-	s.Create("/foo", "", false, Permanent, 2, 1)
-	s.Create("/foo/x", "bar", false, Permanent, 3, 1)
-	s.Create("/foo/y", "baz", false, time.Now().Add(5*time.Millisecond), 4, 1)
+	go mockSyncService(s.DeleteExpiredKeys)
+	s.Create("/foo", "", false, Permanent)
+	s.Create("/foo/x", "bar", false, Permanent)
+	s.Create("/foo/y", "baz", false, time.Now().Add(5*time.Millisecond))
 	b, err := s.Save()
 	b, err := s.Save()
 
 
 	time.Sleep(10 * time.Millisecond)
 	time.Sleep(10 * time.Millisecond)
 
 
 	s2 := newStore()
 	s2 := newStore()
-	go mockSyncService(s2.deleteExpiredKeys)
+	go mockSyncService(s2.DeleteExpiredKeys)
 	s2.Recovery(b)
 	s2.Recovery(b)
 
 
 	time.Sleep(600 * time.Millisecond)
 	time.Sleep(600 * time.Millisecond)
 
 
-	e, err := s.Get("/foo/x", false, false, 4, 1)
+	e, err := s.Get("/foo/x", false, false)
 	assert.Nil(t, err, "")
 	assert.Nil(t, err, "")
 	assert.Equal(t, e.Value, "bar", "")
 	assert.Equal(t, e.Value, "bar", "")
 
 
-	e, err = s.Get("/foo/y", false, false, 4, 1)
+	e, err = s.Get("/foo/y", false, false)
 	assert.NotNil(t, err, "")
 	assert.NotNil(t, err, "")
 	assert.Nil(t, e, "")
 	assert.Nil(t, e, "")
 }
 }
@@ -414,9 +412,9 @@ func nbselect(c <-chan *Event) *Event {
 	}
 	}
 }
 }
 
 
-func mockSyncService(f func(now time.Time, index uint64, term uint64)) {
+func mockSyncService(f func(now time.Time)) {
 	ticker := time.Tick(time.Millisecond * 500)
 	ticker := time.Tick(time.Millisecond * 500)
 	for now := range ticker {
 	for now := range ticker {
-		f(now, 2, 1)
+		f(now)
 	}
 	}
 }
 }

+ 1 - 2
store/v2/compare_and_swap_command.go

@@ -30,8 +30,7 @@ func (c *CompareAndSwapCommand) CommandName() string {
 func (c *CompareAndSwapCommand) Apply(server raft.Server) (interface{}, error) {
 func (c *CompareAndSwapCommand) Apply(server raft.Server) (interface{}, error) {
 	s, _ := server.StateMachine().(store.Store)
 	s, _ := server.StateMachine().(store.Store)
 
 
-	e, err := s.CompareAndSwap(c.Key, c.PrevValue, c.PrevIndex,
-		c.Value, c.ExpireTime, server.CommitIndex(), server.Term())
+	e, err := s.CompareAndSwap(c.Key, c.PrevValue, c.PrevIndex, c.Value, c.ExpireTime)
 
 
 	if err != nil {
 	if err != nil {
 		log.Debug(err)
 		log.Debug(err)

+ 1 - 1
store/v2/create_command.go

@@ -29,7 +29,7 @@ func (c *CreateCommand) CommandName() string {
 func (c *CreateCommand) Apply(server raft.Server) (interface{}, error) {
 func (c *CreateCommand) Apply(server raft.Server) (interface{}, error) {
 	s, _ := server.StateMachine().(store.Store)
 	s, _ := server.StateMachine().(store.Store)
 
 
-	e, err := s.Create(c.Key, c.Value, c.Unique, c.ExpireTime, server.CommitIndex(), server.Term())
+	e, err := s.Create(c.Key, c.Value, c.Unique, c.ExpireTime)
 
 
 	if err != nil {
 	if err != nil {
 		log.Debug(err)
 		log.Debug(err)

+ 1 - 1
store/v2/delete_command.go

@@ -25,7 +25,7 @@ func (c *DeleteCommand) CommandName() string {
 func (c *DeleteCommand) Apply(server raft.Server) (interface{}, error) {
 func (c *DeleteCommand) Apply(server raft.Server) (interface{}, error) {
 	s, _ := server.StateMachine().(store.Store)
 	s, _ := server.StateMachine().(store.Store)
 
 
-	e, err := s.Delete(c.Key, c.Recursive, server.CommitIndex(), server.Term())
+	e, err := s.Delete(c.Key, c.Recursive)
 
 
 	if err != nil {
 	if err != nil {
 		log.Debug(err)
 		log.Debug(err)

+ 1 - 1
store/v2/set_command.go

@@ -29,7 +29,7 @@ func (c *SetCommand) Apply(server raft.Server) (interface{}, error) {
 	s, _ := server.StateMachine().(store.Store)
 	s, _ := server.StateMachine().(store.Store)
 
 
 	// create a new node or replace the old node.
 	// create a new node or replace the old node.
-	e, err := s.Set(c.Key, c.Value, c.ExpireTime, server.CommitIndex(), server.Term())
+	e, err := s.Set(c.Key, c.Value, c.ExpireTime)
 
 
 	if err != nil {
 	if err != nil {
 		log.Debug(err)
 		log.Debug(err)

+ 1 - 1
store/v2/update_command.go

@@ -27,7 +27,7 @@ func (c *UpdateCommand) CommandName() string {
 func (c *UpdateCommand) Apply(server raft.Server) (interface{}, error) {
 func (c *UpdateCommand) Apply(server raft.Server) (interface{}, error) {
 	s, _ := server.StateMachine().(store.Store)
 	s, _ := server.StateMachine().(store.Store)
 
 
-	e, err := s.Update(c.Key, c.Value, c.ExpireTime, server.CommitIndex(), server.Term())
+	e, err := s.Update(c.Key, c.Value, c.ExpireTime)
 
 
 	if err != nil {
 	if err != nil {
 		log.Debug(err)
 		log.Debug(err)

+ 14 - 50
store/watcher_hub.go

@@ -16,11 +16,9 @@ import (
 // event happens between the end of the first watch command and the start
 // event happens between the end of the first watch command and the start
 // of the second command.
 // of the second command.
 type watcherHub struct {
 type watcherHub struct {
-	watchers        map[string]*list.List
-	count           int64 // current number of watchers.
-	EventHistory    *EventHistory
-	pendingWatchers map[*list.Element]*list.List
-	pendingList     map[*list.List]string
+	watchers     map[string]*list.List
+	count        int64 // current number of watchers.
+	EventHistory *EventHistory
 }
 }
 
 
 // newWatchHub creates a watchHub. The capacity determines how many events we will
 // newWatchHub creates a watchHub. The capacity determines how many events we will
@@ -29,10 +27,8 @@ type watcherHub struct {
 // Ideally, it should smaller than 20K/s[max throughput] * 2 * 50ms[RTT] = 2000
 // Ideally, it should smaller than 20K/s[max throughput] * 2 * 50ms[RTT] = 2000
 func newWatchHub(capacity int) *watcherHub {
 func newWatchHub(capacity int) *watcherHub {
 	return &watcherHub{
 	return &watcherHub{
-		watchers:        make(map[string]*list.List),
-		EventHistory:    newEventHistory(capacity),
-		pendingWatchers: make(map[*list.Element]*list.List),
-		pendingList:     make(map[*list.List]string),
+		watchers:     make(map[string]*list.List),
+		EventHistory: newEventHistory(capacity),
 	}
 	}
 }
 }
 
 
@@ -41,22 +37,16 @@ func newWatchHub(capacity int) *watcherHub {
 // If recursive is false, the first change after index at 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.
 // 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) {
 func (wh *watcherHub) watch(prefix string, recursive bool, index uint64) (<-chan *Event, *etcdErr.Error) {
-	events, err := wh.EventHistory.scan(prefix, index)
+	event, err := wh.EventHistory.scan(prefix, index)
 
 
 	if err != nil {
 	if err != nil {
 		return nil, err
 		return nil, err
 	}
 	}
 
 
-	eventChan := make(chan *Event, len(events)+5) // use a buffered channel
+	eventChan := make(chan *Event, 1) // use a buffered channel
 
 
-	if events != nil {
-		for _, e := range events {
-			eventChan <- e
-		}
-
-		if events[0].Action == Expire {
-			eventChan <- nil
-		}
+	if event != nil {
+		eventChan <- event
 
 
 		return eventChan, nil
 		return eventChan, nil
 	}
 	}
@@ -123,16 +113,11 @@ func (wh *watcherHub) notifyWatchers(e *Event, path string, deleted bool) {
 
 
 			if w.notify(e, e.Key == path, deleted) {
 			if w.notify(e, e.Key == path, deleted) {
 
 
-				if e.Action == Expire {
-					wh.pendingWatchers[curr] = l
-					wh.pendingList[l] = path
-				} else {
-					// if we successfully notify a watcher
-					// we need to remove the watcher from the list
-					// and decrease the counter
-					l.Remove(curr)
-					atomic.AddInt64(&wh.count, -1)
-				}
+				// if we successfully notify a watcher
+				// we need to remove the watcher from the list
+				// and decrease the counter
+				l.Remove(curr)
+				atomic.AddInt64(&wh.count, -1)
 
 
 			}
 			}
 
 
@@ -141,27 +126,6 @@ func (wh *watcherHub) notifyWatchers(e *Event, path string, deleted bool) {
 	}
 	}
 }
 }
 
 
-func (wh *watcherHub) clearPendingWatchers() {
-	if len(wh.pendingWatchers) == 0 { // avoid making new maps
-		return
-	}
-
-	for e, l := range wh.pendingWatchers {
-		l.Remove(e)
-
-		if l.Len() == 0 {
-			path := wh.pendingList[l]
-			delete(wh.watchers, path)
-		}
-
-		w, _ := e.Value.(*watcher)
-		w.eventChan <- nil
-	}
-
-	wh.pendingWatchers = make(map[*list.Element]*list.List)
-	wh.pendingList = make(map[*list.List]string)
-}
-
 // clone function clones the watcherHub and return the cloned one.
 // clone function clones the watcherHub and return the cloned one.
 // only clone the static content. do not clone the current watchers.
 // only clone the static content. do not clone the current watchers.
 func (wh *watcherHub) clone() *watcherHub {
 func (wh *watcherHub) clone() *watcherHub {

+ 3 - 3
store/watcher_test.go

@@ -19,7 +19,7 @@ func TestWatcher(t *testing.T) {
 		// do nothing
 		// do nothing
 	}
 	}
 
 
-	e := newEvent(Create, "/foo/bar", 1, 1)
+	e := newEvent(Create, "/foo/bar", 1)
 
 
 	wh.notify(e)
 	wh.notify(e)
 
 
@@ -31,7 +31,7 @@ func TestWatcher(t *testing.T) {
 
 
 	c, _ = wh.watch("/foo", false, 2)
 	c, _ = wh.watch("/foo", false, 2)
 
 
-	e = newEvent(Create, "/foo/bar", 2, 1)
+	e = newEvent(Create, "/foo/bar", 2)
 
 
 	wh.notify(e)
 	wh.notify(e)
 
 
@@ -42,7 +42,7 @@ func TestWatcher(t *testing.T) {
 		// do nothing
 		// do nothing
 	}
 	}
 
 
-	e = newEvent(Create, "/foo", 3, 1)
+	e = newEvent(Create, "/foo", 3)
 
 
 	wh.notify(e)
 	wh.notify(e)