Selaa lähdekoodia

*: kv range to return current revision

This changes the behavior of KV's range and tx range to return
current revision rather than range revision. This makes populating
range response header easier.
Gyu-Ho Lee 10 vuotta sitten
vanhempi
commit
f6215574f2
5 muutettua tiedostoa jossa 123 lisäystä ja 10 poistoa
  1. 111 0
      clientv3/integration/client_test.go
  2. 1 4
      integration/v3_grpc_test.go
  3. 4 0
      storage/kv.go
  4. 2 2
      storage/kv_test.go
  5. 5 4
      storage/kvstore.go

+ 111 - 0
clientv3/integration/client_test.go

@@ -16,12 +16,14 @@ package integration
 
 
 import (
 import (
 	"bytes"
 	"bytes"
+	"reflect"
 	"testing"
 	"testing"
 
 
 	"github.com/coreos/etcd/clientv3"
 	"github.com/coreos/etcd/clientv3"
 	"github.com/coreos/etcd/integration"
 	"github.com/coreos/etcd/integration"
 	"github.com/coreos/etcd/lease"
 	"github.com/coreos/etcd/lease"
 	"github.com/coreos/etcd/pkg/testutil"
 	"github.com/coreos/etcd/pkg/testutil"
+	"github.com/coreos/etcd/storage/storagepb"
 )
 )
 
 
 func TestKVPut(t *testing.T) {
 func TestKVPut(t *testing.T) {
@@ -61,3 +63,112 @@ func TestKVPut(t *testing.T) {
 		}
 		}
 	}
 	}
 }
 }
+
+func TestKVRange(t *testing.T) {
+	defer testutil.AfterTest(t)
+
+	clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
+	defer clus.Terminate(t)
+
+	kv := clientv3.NewKV(clus.RandClient())
+
+	keySet := []string{"a", "b", "c", "c", "c", "foo", "foo/abc", "fop"}
+	for i, key := range keySet {
+		if _, err := kv.Put(key, "", lease.NoLease); err != nil {
+			t.Fatalf("#%d: couldn't put %q (%v)", i, key, err)
+		}
+	}
+	resp, err := kv.Get(keySet[0], 0)
+	if err != nil {
+		t.Fatalf("couldn't get key (%v)", err)
+	}
+	wheader := resp.Header
+
+	tests := []struct {
+		begin, end string
+		rev        int64
+		sortOption *clientv3.SortOption
+
+		wantSet []*storagepb.KeyValue
+	}{
+		// range first two
+		{
+			"a", "c",
+			0,
+			nil,
+
+			[]*storagepb.KeyValue{
+				{Key: []byte("a"), Value: nil, CreateRevision: 2, ModRevision: 2, Version: 1},
+				{Key: []byte("b"), Value: nil, CreateRevision: 3, ModRevision: 3, Version: 1},
+			},
+		},
+		// range all with rev
+		{
+			"a", "x",
+			2,
+			nil,
+
+			[]*storagepb.KeyValue{
+				{Key: []byte("a"), Value: nil, CreateRevision: 2, ModRevision: 2, Version: 1},
+			},
+		},
+		// range all with SortByKey, SortAscend
+		{
+			"a", "x",
+			0,
+			&clientv3.SortOption{Target: clientv3.SortByKey, Order: clientv3.SortAscend},
+
+			[]*storagepb.KeyValue{
+				{Key: []byte("a"), Value: nil, CreateRevision: 2, ModRevision: 2, Version: 1},
+				{Key: []byte("b"), Value: nil, CreateRevision: 3, ModRevision: 3, Version: 1},
+				{Key: []byte("c"), Value: nil, CreateRevision: 4, ModRevision: 6, Version: 3},
+				{Key: []byte("foo"), Value: nil, CreateRevision: 7, ModRevision: 7, Version: 1},
+				{Key: []byte("foo/abc"), Value: nil, CreateRevision: 8, ModRevision: 8, Version: 1},
+				{Key: []byte("fop"), Value: nil, CreateRevision: 9, ModRevision: 9, Version: 1},
+			},
+		},
+		// range all with SortByCreatedRev, SortDescend
+		{
+			"a", "x",
+			0,
+			&clientv3.SortOption{Target: clientv3.SortByCreatedRev, Order: clientv3.SortDescend},
+
+			[]*storagepb.KeyValue{
+				{Key: []byte("fop"), Value: nil, CreateRevision: 9, ModRevision: 9, Version: 1},
+				{Key: []byte("foo/abc"), Value: nil, CreateRevision: 8, ModRevision: 8, Version: 1},
+				{Key: []byte("foo"), Value: nil, CreateRevision: 7, ModRevision: 7, Version: 1},
+				{Key: []byte("c"), Value: nil, CreateRevision: 4, ModRevision: 6, Version: 3},
+				{Key: []byte("b"), Value: nil, CreateRevision: 3, ModRevision: 3, Version: 1},
+				{Key: []byte("a"), Value: nil, CreateRevision: 2, ModRevision: 2, Version: 1},
+			},
+		},
+		// range all with SortByModifiedRev, SortDescend
+		{
+			"a", "x",
+			0,
+			&clientv3.SortOption{Target: clientv3.SortByModifiedRev, Order: clientv3.SortDescend},
+
+			[]*storagepb.KeyValue{
+				{Key: []byte("fop"), Value: nil, CreateRevision: 9, ModRevision: 9, Version: 1},
+				{Key: []byte("foo/abc"), Value: nil, CreateRevision: 8, ModRevision: 8, Version: 1},
+				{Key: []byte("foo"), Value: nil, CreateRevision: 7, ModRevision: 7, Version: 1},
+				{Key: []byte("c"), Value: nil, CreateRevision: 4, ModRevision: 6, Version: 3},
+				{Key: []byte("b"), Value: nil, CreateRevision: 3, ModRevision: 3, Version: 1},
+				{Key: []byte("a"), Value: nil, CreateRevision: 2, ModRevision: 2, Version: 1},
+			},
+		},
+	}
+
+	for i, tt := range tests {
+		resp, err := kv.Range(tt.begin, tt.end, 0, tt.rev, tt.sortOption)
+		if err != nil {
+			t.Fatalf("#%d: couldn't range (%v)", i, err)
+		}
+		if !reflect.DeepEqual(wheader, resp.Header) {
+			t.Fatalf("#%d: wheader expected %+v, got %+v", i, wheader, resp.Header)
+		}
+		if !reflect.DeepEqual(tt.wantSet, resp.Kvs) {
+			t.Fatalf("#%d: resp.Kvs expected %+v, got %+v", i, tt.wantSet, resp.Kvs)
+		}
+	}
+}

+ 1 - 4
integration/v3_grpc_test.go

@@ -1188,10 +1188,7 @@ func TestV3RangeRequest(t *testing.T) {
 			if resp.More != tt.wmores[j] {
 			if resp.More != tt.wmores[j] {
 				t.Errorf("#%d.%d: bad more. got = %v, want = %v, ", i, j, resp.More, tt.wmores[j])
 				t.Errorf("#%d.%d: bad more. got = %v, want = %v, ", i, j, resp.More, tt.wmores[j])
 			}
 			}
-			wrev := req.Revision
-			if wrev == 0 {
-				wrev = int64(len(tt.putKeys) + 1)
-			}
+			wrev := int64(len(tt.putKeys) + 1)
 			if resp.Header.Revision != wrev {
 			if resp.Header.Revision != wrev {
 				t.Errorf("#%d.%d: bad header revision. got = %d. want = %d", i, j, resp.Header.Revision, wrev)
 				t.Errorf("#%d.%d: bad header revision. got = %d. want = %d", i, j, resp.Header.Revision, wrev)
 			}
 			}

+ 4 - 0
storage/kv.go

@@ -30,6 +30,7 @@ type KV interface {
 	FirstRev() int64
 	FirstRev() int64
 
 
 	// Range gets the keys in the range at rangeRev.
 	// Range gets the keys in the range at rangeRev.
+	// The returned rev is the current revision of the KV when the operation is executed.
 	// If rangeRev <=0, range gets the keys at currentRev.
 	// If rangeRev <=0, range gets the keys at currentRev.
 	// If `end` is nil, the request returns the key.
 	// If `end` is nil, the request returns the key.
 	// If `end` is not nil, it gets the keys in range [key, range_end).
 	// If `end` is not nil, it gets the keys in range [key, range_end).
@@ -41,11 +42,13 @@ type KV interface {
 	// attach a lease to a key-value pair as meta-data. KV implementation does not validate the lease
 	// attach a lease to a key-value pair as meta-data. KV implementation does not validate the lease
 	// id.
 	// id.
 	// A put also increases the rev of the store, and generates one event in the event history.
 	// A put also increases the rev of the store, and generates one event in the event history.
+	// The returned rev is the current revision of the KV when the operation is executed.
 	Put(key, value []byte, lease lease.LeaseID) (rev int64)
 	Put(key, value []byte, lease lease.LeaseID) (rev int64)
 
 
 	// DeleteRange deletes the given range from the store.
 	// DeleteRange deletes the given range from the store.
 	// A deleteRange increases the rev of the store if any key in the range exists.
 	// A deleteRange increases the rev of the store if any key in the range exists.
 	// The number of key deleted will be returned.
 	// The number of key deleted will be returned.
+	// The returned rev is the current revision of the KV when the operation is executed.
 	// It also generates one event for each key delete in the event history.
 	// It also generates one event for each key delete in the event history.
 	// if the `end` is nil, deleteRange deletes the key.
 	// if the `end` is nil, deleteRange deletes the key.
 	// if the `end` is not nil, deleteRange deletes the keys in range [key, range_end).
 	// if the `end` is not nil, deleteRange deletes the keys in range [key, range_end).
@@ -58,6 +61,7 @@ type KV interface {
 	TxnBegin() int64
 	TxnBegin() int64
 	// TxnEnd ends the on-going txn with txn ID. If the on-going txn ID is not matched, error is returned.
 	// TxnEnd ends the on-going txn with txn ID. If the on-going txn ID is not matched, error is returned.
 	TxnEnd(txnID int64) error
 	TxnEnd(txnID int64) error
+	// TxnRange returns the current revision of the KV when the operation is executed.
 	TxnRange(txnID int64, key, end []byte, limit, rangeRev int64) (kvs []storagepb.KeyValue, rev int64, err error)
 	TxnRange(txnID int64, key, end []byte, limit, rangeRev int64) (kvs []storagepb.KeyValue, rev int64, err error)
 	TxnPut(txnID int64, key, value []byte, lease lease.LeaseID) (rev int64, err error)
 	TxnPut(txnID int64, key, value []byte, lease lease.LeaseID) (rev int64, err error)
 	TxnDeleteRange(txnID int64, key, end []byte) (n, rev int64, err error)
 	TxnDeleteRange(txnID int64, key, end []byte) (n, rev int64, err error)

+ 2 - 2
storage/kv_test.go

@@ -153,8 +153,8 @@ func testKVRangeRev(t *testing.T, f rangeFunc) {
 	}{
 	}{
 		{-1, 4, kvs},
 		{-1, 4, kvs},
 		{0, 4, kvs},
 		{0, 4, kvs},
-		{2, 2, kvs[:1]},
-		{3, 3, kvs[:2]},
+		{2, 4, kvs[:1]},
+		{3, 4, kvs[:2]},
 		{4, 4, kvs},
 		{4, 4, kvs},
 	}
 	}
 
 

+ 5 - 4
storage/kvstore.go

@@ -353,8 +353,8 @@ func (a *store) Equal(b *store) bool {
 }
 }
 
 
 // range is a keyword in Go, add Keys suffix.
 // range is a keyword in Go, add Keys suffix.
-func (s *store) rangeKeys(key, end []byte, limit, rangeRev int64) (kvs []storagepb.KeyValue, rev int64, err error) {
-	curRev := int64(s.currentRev.main)
+func (s *store) rangeKeys(key, end []byte, limit, rangeRev int64) (kvs []storagepb.KeyValue, curRev int64, err error) {
+	curRev = int64(s.currentRev.main)
 	if s.currentRev.sub > 0 {
 	if s.currentRev.sub > 0 {
 		curRev += 1
 		curRev += 1
 	}
 	}
@@ -362,6 +362,7 @@ func (s *store) rangeKeys(key, end []byte, limit, rangeRev int64) (kvs []storage
 	if rangeRev > curRev {
 	if rangeRev > curRev {
 		return nil, s.currentRev.main, ErrFutureRev
 		return nil, s.currentRev.main, ErrFutureRev
 	}
 	}
+	var rev int64
 	if rangeRev <= 0 {
 	if rangeRev <= 0 {
 		rev = curRev
 		rev = curRev
 	} else {
 	} else {
@@ -373,7 +374,7 @@ func (s *store) rangeKeys(key, end []byte, limit, rangeRev int64) (kvs []storage
 
 
 	_, revpairs := s.kvindex.Range(key, end, int64(rev))
 	_, revpairs := s.kvindex.Range(key, end, int64(rev))
 	if len(revpairs) == 0 {
 	if len(revpairs) == 0 {
-		return nil, rev, nil
+		return nil, curRev, nil
 	}
 	}
 
 
 	for _, revpair := range revpairs {
 	for _, revpair := range revpairs {
@@ -393,7 +394,7 @@ func (s *store) rangeKeys(key, end []byte, limit, rangeRev int64) (kvs []storage
 			break
 			break
 		}
 		}
 	}
 	}
-	return kvs, rev, nil
+	return kvs, curRev, nil
 }
 }
 
 
 func (s *store) put(key, value []byte, leaseID lease.LeaseID) {
 func (s *store) put(key, value []byte, leaseID lease.LeaseID) {