Browse Source

Merge pull request #408 from xiangli-cmu/compareAndDelete

Compare and delete
Xiang Li 12 years ago
parent
commit
715b4d7bfc

+ 32 - 1
server/v2/delete_handler.go

@@ -2,7 +2,9 @@ package v2
 
 
 import (
 import (
 	"net/http"
 	"net/http"
+	"strconv"
 
 
+	etcdErr "github.com/coreos/etcd/error"
 	"github.com/gorilla/mux"
 	"github.com/gorilla/mux"
 )
 )
 
 
@@ -13,6 +15,35 @@ func DeleteHandler(w http.ResponseWriter, req *http.Request, s Server) error {
 	recursive := (req.FormValue("recursive") == "true")
 	recursive := (req.FormValue("recursive") == "true")
 	dir := (req.FormValue("dir") == "true")
 	dir := (req.FormValue("dir") == "true")
 
 
-	c := s.Store().CommandFactory().CreateDeleteCommand(key, dir, recursive)
+	req.ParseForm()
+	_, valueOk := req.Form["prevValue"]
+	_, indexOk := req.Form["prevIndex"]
+
+	if !valueOk && !indexOk {
+		c := s.Store().CommandFactory().CreateDeleteCommand(key, dir, recursive)
+		return s.Dispatch(c, w, req)
+	}
+
+	var err error
+	prevIndex := uint64(0)
+	prevValue := req.Form.Get("prevValue")
+
+	if indexOk {
+		prevIndexStr := req.Form.Get("prevIndex")
+		prevIndex, err = strconv.ParseUint(prevIndexStr, 10, 64)
+
+		// bad previous index
+		if err != nil {
+			return etcdErr.NewError(etcdErr.EcodeIndexNaN, "CompareAndDelete", s.Store().Index())
+		}
+	}
+
+	if valueOk {
+		if prevValue == "" {
+			return etcdErr.NewError(etcdErr.EcodePrevValueRequired, "CompareAndDelete", s.Store().Index())
+		}
+	}
+
+	c := s.Store().CommandFactory().CreateCompareAndDeleteCommand(key, prevValue, prevIndex)
 	return s.Dispatch(c, w, req)
 	return s.Dispatch(c, w, req)
 }
 }

+ 111 - 0
server/v2/tests/delete_handler_test.go

@@ -90,3 +90,114 @@ func TestV2DeleteDirectoryRecursiveImpliesDir(t *testing.T) {
 		assert.Equal(t, string(body), `{"action":"delete","node":{"key":"/foo","dir":true,"modifiedIndex":3,"createdIndex":2}}`, "")
 		assert.Equal(t, string(body), `{"action":"delete","node":{"key":"/foo","dir":true,"modifiedIndex":3,"createdIndex":2}}`, "")
 	})
 	})
 }
 }
+
+// Ensures that a key is deleted if the previous index matches
+//
+//   $ curl -X PUT localhost:4001/v2/keys/foo -d value=XXX
+//   $ curl -X DELETE localhost:4001/v2/keys/foo?prevIndex=2
+//
+func TestV2DeleteKeyCADOnIndexSuccess(t *testing.T) {
+	tests.RunServer(func(s *server.Server) {
+		v := url.Values{}
+		v.Set("value", "XXX")
+		resp, err := tests.PutForm(fmt.Sprintf("%s%s", s.URL(), "/v2/keys/foo"), v)
+		tests.ReadBody(resp)
+		resp, err = tests.DeleteForm(fmt.Sprintf("%s%s", s.URL(), "/v2/keys/foo?prevIndex=2"), url.Values{})
+		assert.Nil(t, err, "")
+		body := tests.ReadBodyJSON(resp)
+		assert.Equal(t, body["action"], "compareAndDelete", "")
+
+		node := body["node"].(map[string]interface{})
+		assert.Equal(t, node["key"], "/foo", "")
+		assert.Equal(t, node["modifiedIndex"], 3, "")
+	})
+}
+
+// Ensures that a key is not deleted if the previous index does not match
+//
+//   $ curl -X PUT localhost:4001/v2/keys/foo -d value=XXX
+//   $ curl -X DELETE localhost:4001/v2/keys/foo?prevIndex=100
+//
+func TestV2DeleteKeyCADOnIndexFail(t *testing.T) {
+	tests.RunServer(func(s *server.Server) {
+		v := url.Values{}
+		v.Set("value", "XXX")
+		resp, err := tests.PutForm(fmt.Sprintf("%s%s", s.URL(), "/v2/keys/foo"), v)
+		tests.ReadBody(resp)
+		resp, err = tests.DeleteForm(fmt.Sprintf("%s%s", s.URL(), "/v2/keys/foo?prevIndex=100"), url.Values{})
+		assert.Nil(t, err, "")
+		body := tests.ReadBodyJSON(resp)
+		assert.Equal(t, body["errorCode"], 101)
+	})
+}
+
+// Ensures that an error is thrown if an invalid previous index is provided.
+//
+//   $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=XXX
+//   $ curl -X DELETE localhost:4001/v2/keys/foo/bar?prevIndex=bad_index
+//
+func TestV2DeleteKeyCADWithInvalidIndex(t *testing.T) {
+	tests.RunServer(func(s *server.Server) {
+		v := url.Values{}
+		v.Set("value", "XXX")
+		resp, _ := tests.PutForm(fmt.Sprintf("%s%s", s.URL(), "/v2/keys/foo/bar"), v)
+		tests.ReadBody(resp)
+		resp, _ = tests.DeleteForm(fmt.Sprintf("%s%s", s.URL(), "/v2/keys/foo/bar?prevIndex=bad_index"), v)
+		body := tests.ReadBodyJSON(resp)
+		assert.Equal(t, body["errorCode"], 203)
+	})
+}
+
+// Ensures that a key is deleted only if the previous value matches.
+//
+//   $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=XXX
+//   $ curl -X DELETE localhost:4001/v2/keys/foo/bar?prevValue=XXX
+//
+func TestV2DeleteKeyCADOnValueSuccess(t *testing.T) {
+	tests.RunServer(func(s *server.Server) {
+		v := url.Values{}
+		v.Set("value", "XXX")
+		resp, _ := tests.PutForm(fmt.Sprintf("%s%s", s.URL(), "/v2/keys/foo/bar"), v)
+		tests.ReadBody(resp)
+		resp, _ = tests.DeleteForm(fmt.Sprintf("%s%s", s.URL(), "/v2/keys/foo/bar?prevValue=XXX"), v)
+		body := tests.ReadBodyJSON(resp)
+		assert.Equal(t, body["action"], "compareAndDelete", "")
+
+		node := body["node"].(map[string]interface{})
+		assert.Equal(t, node["modifiedIndex"], 3, "")
+	})
+}
+
+// Ensures that a key is not deleted if the previous value does not match.
+//
+//   $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=XXX
+//   $ curl -X DELETE localhost:4001/v2/keys/foo/bar?prevValue=YYY
+//
+func TestV2DeleteKeyCADOnValueFail(t *testing.T) {
+	tests.RunServer(func(s *server.Server) {
+		v := url.Values{}
+		v.Set("value", "XXX")
+		resp, _ := tests.PutForm(fmt.Sprintf("%s%s", s.URL(), "/v2/keys/foo/bar"), v)
+		tests.ReadBody(resp)
+		resp, _ = tests.DeleteForm(fmt.Sprintf("%s%s", s.URL(), "/v2/keys/foo/bar?prevValue=YYY"), v)
+		body := tests.ReadBodyJSON(resp)
+		assert.Equal(t, body["errorCode"], 101)
+	})
+}
+
+// Ensures that an error is thrown if an invalid previous value is provided.
+//
+//   $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=XXX
+//   $ curl -X DELETE localhost:4001/v2/keys/foo/bar?prevIndex=
+//
+func TestV2DeleteKeyCADWithInvalidValue(t *testing.T) {
+	tests.RunServer(func(s *server.Server) {
+		v := url.Values{}
+		v.Set("value", "XXX")
+		resp, _ := tests.PutForm(fmt.Sprintf("%s%s", s.URL(), "/v2/keys/foo/bar"), v)
+		tests.ReadBody(resp)
+		resp, _ = tests.DeleteForm(fmt.Sprintf("%s%s", s.URL(), "/v2/keys/foo/bar?prevValue="), v)
+		body := tests.ReadBodyJSON(resp)
+		assert.Equal(t, body["errorCode"], 201)
+	})
+}

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

@@ -214,7 +214,7 @@ func TestV2WatchKeyInDir(t *testing.T) {
 		}()
 		}()
 
 
 		// wait for expiration, we do have a up to 500 millisecond delay
 		// wait for expiration, we do have a up to 500 millisecond delay
-		time.Sleep(1500 * time.Millisecond)
+		time.Sleep(2000 * time.Millisecond)
 
 
 		select {
 		select {
 		case <-c:
 		case <-c:

+ 1 - 0
store/command_factory.go

@@ -22,6 +22,7 @@ type CommandFactory interface {
 	CreateDeleteCommand(key string, dir, recursive bool) raft.Command
 	CreateDeleteCommand(key string, dir, recursive bool) raft.Command
 	CreateCompareAndSwapCommand(key string, value string, prevValue string,
 	CreateCompareAndSwapCommand(key string, value string, prevValue string,
 		prevIndex uint64, expireTime time.Time) raft.Command
 		prevIndex uint64, expireTime time.Time) raft.Command
+	CreateCompareAndDeleteCommand(key string, prevValue string, prevIndex uint64) raft.Command
 	CreateSyncCommand(now time.Time) raft.Command
 	CreateSyncCommand(now time.Time) raft.Command
 }
 }
 
 

+ 8 - 7
store/event.go

@@ -1,13 +1,14 @@
 package store
 package store
 
 
 const (
 const (
-	Get            = "get"
-	Create         = "create"
-	Set            = "set"
-	Update         = "update"
-	Delete         = "delete"
-	CompareAndSwap = "compareAndSwap"
-	Expire         = "expire"
+	Get              = "get"
+	Create           = "create"
+	Set              = "set"
+	Update           = "update"
+	Delete           = "delete"
+	CompareAndSwap   = "compareAndSwap"
+	CompareAndDelete = "compareAndDelete"
+	Expire           = "expire"
 )
 )
 
 
 type Event struct {
 type Event struct {

+ 7 - 0
store/node.go

@@ -306,6 +306,13 @@ func (n *node) UpdateTTL(expireTime time.Time) {
 	}
 	}
 }
 }
 
 
+func (n *node) Compare(prevValue string, prevIndex uint64) bool {
+	compareValue := (prevValue == "" || n.Value == prevValue)
+	compareIndex := (prevIndex == 0 || n.ModifiedIndex == prevIndex)
+
+	return compareValue && compareIndex
+}
+
 // Clone function clone the node recursively and return the new node.
 // Clone function clone the node recursively and return the new node.
 // If the node is a directory, it will clone all the content under this directory.
 // If the node is a directory, it will clone all the content under this directory.
 // 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.

+ 13 - 1
store/stats.go

@@ -35,6 +35,8 @@ const (
 	GetSuccess
 	GetSuccess
 	GetFail
 	GetFail
 	ExpireCount
 	ExpireCount
+	CompareAndDeleteSuccess
+	CompareAndDeleteFail
 )
 )
 
 
 type Stats struct {
 type Stats struct {
@@ -63,6 +65,10 @@ type Stats struct {
 	CompareAndSwapSuccess uint64 `json:"compareAndSwapSuccess"`
 	CompareAndSwapSuccess uint64 `json:"compareAndSwapSuccess"`
 	CompareAndSwapFail    uint64 `json:"compareAndSwapFail"`
 	CompareAndSwapFail    uint64 `json:"compareAndSwapFail"`
 
 
+	// Number of compareAndDelete requests
+	CompareAndDeleteSuccess uint64 `json:"compareAndDeleteSuccess"`
+	CompareAndDeleteFail    uint64 `json:"compareAndDeleteFail"`
+
 	ExpireCount uint64 `json:"expireCount"`
 	ExpireCount uint64 `json:"expireCount"`
 
 
 	Watchers uint64 `json:"watchers"`
 	Watchers uint64 `json:"watchers"`
@@ -76,7 +82,8 @@ func newStats() *Stats {
 func (s *Stats) clone() *Stats {
 func (s *Stats) clone() *Stats {
 	return &Stats{s.GetSuccess, s.GetFail, s.SetSuccess, s.SetFail,
 	return &Stats{s.GetSuccess, s.GetFail, s.SetSuccess, s.SetFail,
 		s.DeleteSuccess, s.DeleteFail, s.UpdateSuccess, s.UpdateFail, s.CreateSuccess,
 		s.DeleteSuccess, s.DeleteFail, s.UpdateSuccess, s.UpdateFail, s.CreateSuccess,
-		s.CreateFail, s.CompareAndSwapSuccess, s.CompareAndSwapFail, s.Watchers, s.ExpireCount}
+		s.CreateFail, s.CompareAndSwapSuccess, s.CompareAndSwapFail,
+		s.CompareAndDeleteSuccess, s.CompareAndDeleteFail, s.Watchers, s.ExpireCount}
 }
 }
 
 
 // Status() return the statistics info of etcd storage its recent start
 // Status() return the statistics info of etcd storage its recent start
@@ -93,6 +100,7 @@ func (s *Stats) TotalTranscations() uint64 {
 	return s.SetSuccess + s.SetFail +
 	return s.SetSuccess + s.SetFail +
 		s.DeleteSuccess + s.DeleteFail +
 		s.DeleteSuccess + s.DeleteFail +
 		s.CompareAndSwapSuccess + s.CompareAndSwapFail +
 		s.CompareAndSwapSuccess + s.CompareAndSwapFail +
+		s.CompareAndDeleteSuccess + s.CompareAndDeleteFail +
 		s.UpdateSuccess + s.UpdateFail
 		s.UpdateSuccess + s.UpdateFail
 }
 }
 
 
@@ -122,6 +130,10 @@ func (s *Stats) Inc(field int) {
 		atomic.AddUint64(&s.CompareAndSwapSuccess, 1)
 		atomic.AddUint64(&s.CompareAndSwapSuccess, 1)
 	case CompareAndSwapFail:
 	case CompareAndSwapFail:
 		atomic.AddUint64(&s.CompareAndSwapFail, 1)
 		atomic.AddUint64(&s.CompareAndSwapFail, 1)
+	case CompareAndDeleteSuccess:
+		atomic.AddUint64(&s.CompareAndDeleteSuccess, 1)
+	case CompareAndDeleteFail:
+		atomic.AddUint64(&s.CompareAndDeleteFail, 1)
 	case ExpireCount:
 	case ExpireCount:
 		atomic.AddUint64(&s.ExpireCount, 1)
 		atomic.AddUint64(&s.ExpireCount, 1)
 	}
 	}

+ 66 - 23
store/store.go

@@ -51,6 +51,7 @@ type Store interface {
 	CompareAndSwap(nodePath string, prevValue string, prevIndex uint64,
 	CompareAndSwap(nodePath string, prevValue string, prevIndex uint64,
 		value string, expireTime time.Time) (*Event, error)
 		value string, expireTime time.Time) (*Event, error)
 	Delete(nodePath string, recursive, dir bool) (*Event, error)
 	Delete(nodePath string, recursive, dir bool) (*Event, error)
+	CompareAndDelete(nodePath string, prevValue string, prevIndex uint64) (*Event, error)
 	Watch(prefix string, recursive bool, sinceIndex uint64) (<-chan *Event, error)
 	Watch(prefix string, recursive bool, sinceIndex uint64) (<-chan *Event, error)
 
 
 	Save() ([]byte, error)
 	Save() ([]byte, error)
@@ -207,37 +208,37 @@ func (s *store) CompareAndSwap(nodePath string, prevValue string, prevIndex uint
 		return nil, err
 		return nil, err
 	}
 	}
 
 
-	if n.IsDir() { // can only test and set file
+	if n.IsDir() { // can only compare and swap file
 		s.Stats.Inc(CompareAndSwapFail)
 		s.Stats.Inc(CompareAndSwapFail)
 		return nil, etcdErr.NewError(etcdErr.EcodeNotFile, nodePath, s.CurrentIndex)
 		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) {
-		// update etcd index
-		s.CurrentIndex++
+	if !n.Compare(prevValue, prevIndex) {
+		cause := fmt.Sprintf("[%v != %v] [%v != %v]", prevValue, n.Value, prevIndex, n.ModifiedIndex)
+		s.Stats.Inc(CompareAndSwapFail)
+		return nil, etcdErr.NewError(etcdErr.EcodeTestFailed, cause, s.CurrentIndex)
+	}
 
 
-		e := newEvent(CompareAndSwap, nodePath, s.CurrentIndex, n.CreatedIndex)
-		eNode := e.Node
+	// update etcd index
+	s.CurrentIndex++
 
 
-		eNode.PrevValue = n.Value
+	e := newEvent(CompareAndSwap, nodePath, s.CurrentIndex, n.CreatedIndex)
+	eNode := e.Node
 
 
-		// if test succeed, write the value
-		n.Write(value, s.CurrentIndex)
-		n.UpdateTTL(expireTime)
+	eNode.PrevValue = n.Value
 
 
-		eNode.Value = value
-		eNode.Expiration, eNode.TTL = n.ExpirationAndTTL()
+	// if test succeed, write the value
+	n.Write(value, s.CurrentIndex)
+	n.UpdateTTL(expireTime)
 
 
-		s.WatcherHub.notify(e)
-		s.Stats.Inc(CompareAndSwapSuccess)
-		return e, nil
-	}
+	eNode.Value = value
+	eNode.Expiration, eNode.TTL = n.ExpirationAndTTL()
 
 
-	cause := fmt.Sprintf("[%v != %v] [%v != %v]", prevValue, n.Value, prevIndex, n.ModifiedIndex)
-	s.Stats.Inc(CompareAndSwapFail)
-	return nil, etcdErr.NewError(etcdErr.EcodeTestFailed, cause, s.CurrentIndex)
+	s.WatcherHub.notify(e)
+	s.Stats.Inc(CompareAndSwapSuccess)
+	return e, nil
 }
 }
 
 
 // Delete function deletes the node at the given path.
 // Delete function deletes the node at the given path.
@@ -257,8 +258,6 @@ func (s *store) Delete(nodePath string, dir, recursive bool) (*Event, error) {
 		dir = true
 		dir = true
 	}
 	}
 
 
-	nextIndex := s.CurrentIndex + 1
-
 	n, err := s.internalGet(nodePath)
 	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
@@ -266,6 +265,7 @@ func (s *store) Delete(nodePath string, dir, recursive bool) (*Event, error) {
 		return nil, err
 		return nil, err
 	}
 	}
 
 
+	nextIndex := s.CurrentIndex + 1
 	e := newEvent(Delete, nodePath, nextIndex, n.CreatedIndex)
 	e := newEvent(Delete, nodePath, nextIndex, n.CreatedIndex)
 	eNode := e.Node
 	eNode := e.Node
 
 
@@ -276,7 +276,7 @@ func (s *store) Delete(nodePath string, dir, recursive bool) (*Event, error) {
 	}
 	}
 
 
 	callback := func(path string) { // notify function
 	callback := func(path string) { // notify function
-		// notify the watchers with delted set true
+		// notify the watchers with deleted set true
 		s.WatcherHub.notifyWatchers(e, path, true)
 		s.WatcherHub.notifyWatchers(e, path, true)
 	}
 	}
 
 
@@ -296,9 +296,52 @@ func (s *store) Delete(nodePath string, dir, recursive bool) (*Event, error) {
 	return e, nil
 	return e, nil
 }
 }
 
 
+func (s *store) CompareAndDelete(nodePath string, prevValue string, prevIndex uint64) (*Event, error) {
+	nodePath = path.Clean(path.Join("/", nodePath))
+
+	s.worldLock.Lock()
+	defer s.worldLock.Unlock()
+
+	n, err := s.internalGet(nodePath)
+
+	if err != nil { // if the node does not exist, return error
+		s.Stats.Inc(CompareAndDeleteFail)
+		return nil, err
+	}
+
+	if n.IsDir() { // can only compare and delete file
+		s.Stats.Inc(CompareAndSwapFail)
+		return nil, etcdErr.NewError(etcdErr.EcodeNotFile, nodePath, s.CurrentIndex)
+	}
+
+	// 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.
+	if !n.Compare(prevValue, prevIndex) {
+		cause := fmt.Sprintf("[%v != %v] [%v != %v]", prevValue, n.Value, prevIndex, n.ModifiedIndex)
+		s.Stats.Inc(CompareAndDeleteFail)
+		return nil, etcdErr.NewError(etcdErr.EcodeTestFailed, cause, s.CurrentIndex)
+	}
+
+	// update etcd index
+	s.CurrentIndex++
+
+	e := newEvent(CompareAndDelete, nodePath, s.CurrentIndex, n.CreatedIndex)
+
+	callback := func(path string) { // notify function
+		// notify the watchers with deleted set true
+		s.WatcherHub.notifyWatchers(e, path, true)
+	}
+
+	// delete a key-value pair, no error should happen
+	n.Remove(false, false, callback)
+
+	s.WatcherHub.notify(e)
+	s.Stats.Inc(CompareAndDeleteSuccess)
+	return e, nil
+}
+
 func (s *store) Watch(key string, recursive bool, sinceIndex uint64) (<-chan *Event, error) {
 func (s *store) Watch(key string, recursive bool, sinceIndex uint64) (<-chan *Event, error) {
 	key = path.Clean(path.Join("/", key))
 	key = path.Clean(path.Join("/", key))
-
 	nextIndex := s.CurrentIndex + 1
 	nextIndex := s.CurrentIndex + 1
 
 
 	s.worldLock.RLock()
 	s.worldLock.RLock()

+ 51 - 0
store/store_test.go

@@ -337,7 +337,58 @@ func TestRootRdOnly(t *testing.T) {
 
 
 	_, err = s.CompareAndSwap("/", "", 0, "", Permanent)
 	_, err = s.CompareAndSwap("/", "", 0, "", Permanent)
 	assert.NotNil(t, err, "")
 	assert.NotNil(t, err, "")
+}
+
+func TestStoreCompareAndDeletePrevValue(t *testing.T) {
+	s := newStore()
+	s.Create("/foo", false, "bar", false, Permanent)
+	e, err := s.CompareAndDelete("/foo", "bar", 0)
+	assert.Nil(t, err, "")
+	assert.Equal(t, e.Action, "compareAndDelete", "")
+	assert.Equal(t, e.Node.Key, "/foo", "")
+}
+
+func TestStoreCompareAndDeletePrevValueFailsIfNotMatch(t *testing.T) {
+	s := newStore()
+	s.Create("/foo", false, "bar", false, Permanent)
+	e, _err := s.CompareAndDelete("/foo", "baz", 0)
+	err := _err.(*etcdErr.Error)
+	assert.Equal(t, err.ErrorCode, etcdErr.EcodeTestFailed, "")
+	assert.Equal(t, err.Message, "Compare failed", "")
+	assert.Nil(t, e, "")
+	e, _ = s.Get("/foo", false, false)
+	assert.Equal(t, e.Node.Value, "bar", "")
+}
+
+func TestStoreCompareAndDeletePrevIndex(t *testing.T) {
+	s := newStore()
+	s.Create("/foo", false, "bar", false, Permanent)
+	e, err := s.CompareAndDelete("/foo", "", 1)
+	assert.Nil(t, err, "")
+	assert.Equal(t, e.Action, "compareAndDelete", "")
+}
+
+func TestStoreCompareAndDeletePrevIndexFailsIfNotMatch(t *testing.T) {
+	s := newStore()
+	s.Create("/foo", false, "bar", false, Permanent)
+	e, _err := s.CompareAndDelete("/foo", "", 100)
+	assert.NotNil(t, _err, "")
+	err := _err.(*etcdErr.Error)
+	assert.Equal(t, err.ErrorCode, etcdErr.EcodeTestFailed, "")
+	assert.Equal(t, err.Message, "Compare failed", "")
+	assert.Nil(t, e, "")
+	e, _ = s.Get("/foo", false, false)
+	assert.Equal(t, e.Node.Value, "bar", "")
+}
 
 
+// Ensure that the store cannot delete a directory.
+func TestStoreCompareAndDeleteDiretoryFail(t *testing.T) {
+	s := newStore()
+	s.Create("/foo", true, "", false, Permanent)
+	_, _err := s.CompareAndDelete("/foo", "", 0)
+	assert.NotNil(t, _err, "")
+	err := _err.(*etcdErr.Error)
+	assert.Equal(t, err.ErrorCode, etcdErr.EcodeNotFile, "")
 }
 }
 
 
 // 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.

+ 9 - 0
store/v2/command_factory.go

@@ -75,6 +75,15 @@ func (f *CommandFactory) CreateCompareAndSwapCommand(key string, value string, p
 	}
 	}
 }
 }
 
 
+// CreateCompareAndDeleteCommand creates a version 2 command to conditionally delete a key from the store.
+func (f *CommandFactory) CreateCompareAndDeleteCommand(key string, prevValue string, prevIndex uint64) raft.Command {
+	return &CompareAndDeleteCommand{
+		Key:       key,
+		PrevValue: prevValue,
+		PrevIndex: prevIndex,
+	}
+}
+
 func (f *CommandFactory) CreateSyncCommand(now time.Time) raft.Command {
 func (f *CommandFactory) CreateSyncCommand(now time.Time) raft.Command {
 	return &SyncCommand{
 	return &SyncCommand{
 		Time: time.Now(),
 		Time: time.Now(),

+ 37 - 0
store/v2/compare_and_delete_command.go

@@ -0,0 +1,37 @@
+package v2
+
+import (
+	"github.com/coreos/etcd/log"
+	"github.com/coreos/etcd/store"
+	"github.com/coreos/raft"
+)
+
+func init() {
+	raft.RegisterCommand(&CompareAndDeleteCommand{})
+}
+
+// The CompareAndDelete performs a conditional delete on a key in the store.
+type CompareAndDeleteCommand struct {
+	Key       string `json:"key"`
+	PrevValue string `json:"prevValue"`
+	PrevIndex uint64 `json:"prevIndex"`
+}
+
+// The name of the compareAndDelete command in the log
+func (c *CompareAndDeleteCommand) CommandName() string {
+	return "etcd:compareAndDelete"
+}
+
+// Set the key-value pair if the current value of the key equals to the given prevValue
+func (c *CompareAndDeleteCommand) Apply(server raft.Server) (interface{}, error) {
+	s, _ := server.StateMachine().(store.Store)
+
+	e, err := s.CompareAndDelete(c.Key, c.PrevValue, c.PrevIndex)
+
+	if err != nil {
+		log.Debug(err)
+		return nil, err
+	}
+
+	return e, nil
+}

+ 2 - 2
tests/functional/simple_snapshot_test.go

@@ -56,7 +56,7 @@ func TestSimpleSnapshot(t *testing.T) {
 
 
 	index, _ := strconv.Atoi(snapshots[0].Name()[2:5])
 	index, _ := strconv.Atoi(snapshots[0].Name()[2:5])
 
 
-	if index <= 507 || index >= 510 {
+	if index < 507 || index > 510 {
 		t.Fatal("wrong name of snapshot :", snapshots[0].Name())
 		t.Fatal("wrong name of snapshot :", snapshots[0].Name())
 	}
 	}
 
 
@@ -89,7 +89,7 @@ func TestSimpleSnapshot(t *testing.T) {
 
 
 	index, _ = strconv.Atoi(snapshots[0].Name()[2:6])
 	index, _ = strconv.Atoi(snapshots[0].Name()[2:6])
 
 
-	if index <= 1014 || index >= 1017 {
+	if index < 1014 || index > 1017 {
 		t.Fatal("wrong name of snapshot :", snapshots[0].Name())
 		t.Fatal("wrong name of snapshot :", snapshots[0].Name())
 	}
 	}
 }
 }