*: 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.
This commit is contained in:
Gyu-Ho Lee 2016-01-30 16:51:12 -08:00
parent 6577df17d6
commit f6215574f2
5 changed files with 123 additions and 10 deletions

View File

@ -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)
}
}
}

View File

@ -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 wrev := int64(len(tt.putKeys) + 1)
if wrev == 0 {
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)
} }

View File

@ -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)

View File

@ -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]}, {2, 4, kvs[:1]},
{3, 3, kvs[:2]}, {3, 4, kvs[:2]},
{4, 4, kvs}, {4, 4, kvs},
} }

View File

@ -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) { func (s *store) rangeKeys(key, end []byte, limit, rangeRev int64) (kvs []storagepb.KeyValue, curRev int64, err error) {
curRev := int64(s.currentRev.main) 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) {