mirror of
https://github.com/etcd-io/etcd.git
synced 2024-09-27 06:25:44 +00:00
tests: Support multiple keys in linearizability tests
Signed-off-by: Marek Siarkowicz <siarkowicz@google.com>
This commit is contained in:
parent
a4c1b3a9e2
commit
ff71f34368
@ -47,9 +47,8 @@ type EtcdResponse struct {
|
||||
}
|
||||
|
||||
type EtcdState struct {
|
||||
Revision int64
|
||||
Key string
|
||||
Value string
|
||||
Revision int64
|
||||
KeyValues map[string]string
|
||||
}
|
||||
|
||||
var etcdModel = porcupine.Model{
|
||||
@ -140,20 +139,20 @@ func initStates(request EtcdRequest, response EtcdResponse) []EtcdState {
|
||||
return []EtcdState{}
|
||||
}
|
||||
state := EtcdState{
|
||||
Key: request.Key,
|
||||
Revision: response.Revision,
|
||||
Revision: response.Revision,
|
||||
KeyValues: map[string]string{},
|
||||
}
|
||||
switch request.Op {
|
||||
case Get:
|
||||
if response.GetData != "" {
|
||||
state.Value = response.GetData
|
||||
state.KeyValues[request.Key] = response.GetData
|
||||
}
|
||||
case Put:
|
||||
state.Value = request.PutData
|
||||
state.KeyValues[request.Key] = request.PutData
|
||||
case Delete:
|
||||
case Txn:
|
||||
if response.TxnSucceeded {
|
||||
state.Value = request.TxnNewData
|
||||
state.KeyValues[request.Key] = request.TxnNewData
|
||||
}
|
||||
return []EtcdState{}
|
||||
default:
|
||||
@ -163,25 +162,27 @@ func initStates(request EtcdRequest, response EtcdResponse) []EtcdState {
|
||||
}
|
||||
|
||||
func stepState(s EtcdState, request EtcdRequest) (EtcdState, EtcdResponse) {
|
||||
if s.Key != request.Key {
|
||||
panic("multiple keys not supported")
|
||||
newKVs := map[string]string{}
|
||||
for k, v := range s.KeyValues {
|
||||
newKVs[k] = v
|
||||
}
|
||||
s.KeyValues = newKVs
|
||||
resp := EtcdResponse{}
|
||||
switch request.Op {
|
||||
case Get:
|
||||
resp.GetData = s.Value
|
||||
resp.GetData = s.KeyValues[request.Key]
|
||||
case Put:
|
||||
s.Value = request.PutData
|
||||
s.KeyValues[request.Key] = request.PutData
|
||||
s.Revision += 1
|
||||
case Delete:
|
||||
if s.Value != "" {
|
||||
s.Value = ""
|
||||
if _, ok := s.KeyValues[request.Key]; ok {
|
||||
delete(s.KeyValues, request.Key)
|
||||
s.Revision += 1
|
||||
resp.Deleted = 1
|
||||
}
|
||||
case Txn:
|
||||
if s.Value == request.TxnExpectData {
|
||||
s.Value = request.TxnNewData
|
||||
if val := s.KeyValues[request.Key]; val == request.TxnExpectData {
|
||||
s.KeyValues[request.Key] = request.TxnNewData
|
||||
s.Revision += 1
|
||||
resp.TxnSucceeded = true
|
||||
}
|
||||
|
@ -51,19 +51,16 @@ func TestModel(t *testing.T) {
|
||||
{
|
||||
name: "Get response data should match put",
|
||||
operations: []testOperation{
|
||||
{req: EtcdRequest{Op: Put, Key: "key", PutData: "1"}, resp: EtcdResponse{Revision: 1}},
|
||||
{req: EtcdRequest{Op: Get, Key: "key"}, resp: EtcdResponse{GetData: "2", Revision: 1}, failure: true},
|
||||
{req: EtcdRequest{Op: Get, Key: "key"}, resp: EtcdResponse{GetData: "2", Revision: 2}, failure: true},
|
||||
{req: EtcdRequest{Op: Get, Key: "key"}, resp: EtcdResponse{GetData: "1", Revision: 1}},
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "Get revision should be equal to put",
|
||||
operations: []testOperation{
|
||||
{req: EtcdRequest{Op: Put, Key: "key"}, resp: EtcdResponse{Revision: 2}},
|
||||
{req: EtcdRequest{Op: Get, Key: "key"}, resp: EtcdResponse{Revision: 1}, failure: true},
|
||||
{req: EtcdRequest{Op: Get, Key: "key"}, resp: EtcdResponse{Revision: 3}, failure: true},
|
||||
{req: EtcdRequest{Op: Get, Key: "key"}, resp: EtcdResponse{Revision: 2}},
|
||||
{req: EtcdRequest{Op: Put, Key: "key1", PutData: "11"}, resp: EtcdResponse{Revision: 1}},
|
||||
{req: EtcdRequest{Op: Put, Key: "key2", PutData: "12"}, resp: EtcdResponse{Revision: 2}},
|
||||
{req: EtcdRequest{Op: Get, Key: "key1"}, resp: EtcdResponse{GetData: "11", Revision: 1}, failure: true},
|
||||
{req: EtcdRequest{Op: Get, Key: "key1"}, resp: EtcdResponse{GetData: "12", Revision: 1}, failure: true},
|
||||
{req: EtcdRequest{Op: Get, Key: "key1"}, resp: EtcdResponse{GetData: "12", Revision: 2}, failure: true},
|
||||
{req: EtcdRequest{Op: Get, Key: "key1"}, resp: EtcdResponse{GetData: "11", Revision: 2}},
|
||||
{req: EtcdRequest{Op: Get, Key: "key2"}, resp: EtcdResponse{GetData: "11", Revision: 2}, failure: true},
|
||||
{req: EtcdRequest{Op: Get, Key: "key2"}, resp: EtcdResponse{GetData: "12", Revision: 1}, failure: true},
|
||||
{req: EtcdRequest{Op: Get, Key: "key2"}, resp: EtcdResponse{GetData: "11", Revision: 1}, failure: true},
|
||||
{req: EtcdRequest{Op: Get, Key: "key2"}, resp: EtcdResponse{GetData: "12", Revision: 2}},
|
||||
},
|
||||
},
|
||||
{
|
||||
@ -97,7 +94,7 @@ func TestModel(t *testing.T) {
|
||||
{
|
||||
name: "Put can fail and be lost before delete",
|
||||
operations: []testOperation{
|
||||
{req: EtcdRequest{Op: Get, Key: "key"}, resp: EtcdResponse{Revision: 1}},
|
||||
{req: EtcdRequest{Op: Delete, Key: "key"}, resp: EtcdResponse{Revision: 1}},
|
||||
{req: EtcdRequest{Op: Put, Key: "key", PutData: "2"}, resp: EtcdResponse{Err: errors.New("failed")}},
|
||||
{req: EtcdRequest{Op: Delete, Key: "key"}, resp: EtcdResponse{Revision: 1}},
|
||||
},
|
||||
@ -138,7 +135,7 @@ func TestModel(t *testing.T) {
|
||||
name: "Put can fail but be persisted and increase revision before delete",
|
||||
operations: []testOperation{
|
||||
// One failed request, one persisted.
|
||||
{req: EtcdRequest{Op: Get, Key: "key"}, resp: EtcdResponse{Revision: 1}},
|
||||
{req: EtcdRequest{Op: Delete, Key: "key"}, resp: EtcdResponse{Revision: 1}},
|
||||
{req: EtcdRequest{Op: Put, Key: "key", PutData: "2"}, resp: EtcdResponse{Err: errors.New("failed")}},
|
||||
{req: EtcdRequest{Op: Delete, Key: "key"}, resp: EtcdResponse{Deleted: 1, Revision: 1}, failure: true},
|
||||
{req: EtcdRequest{Op: Delete, Key: "key"}, resp: EtcdResponse{Deleted: 1, Revision: 2}, failure: true},
|
||||
@ -173,11 +170,20 @@ func TestModel(t *testing.T) {
|
||||
{
|
||||
name: "Delete only increases revision on success",
|
||||
operations: []testOperation{
|
||||
{req: EtcdRequest{Op: Put, Key: "key", PutData: "1"}, resp: EtcdResponse{Revision: 1}},
|
||||
{req: EtcdRequest{Op: Delete, Key: "key"}, resp: EtcdResponse{Deleted: 1, Revision: 1}, failure: true},
|
||||
{req: EtcdRequest{Op: Delete, Key: "key"}, resp: EtcdResponse{Deleted: 1, Revision: 2}},
|
||||
{req: EtcdRequest{Op: Delete, Key: "key"}, resp: EtcdResponse{Deleted: 0, Revision: 3}, failure: true},
|
||||
{req: EtcdRequest{Op: Delete, Key: "key"}, resp: EtcdResponse{Deleted: 0, Revision: 2}},
|
||||
{req: EtcdRequest{Op: Put, Key: "key1", PutData: "11"}, resp: EtcdResponse{Revision: 1}},
|
||||
{req: EtcdRequest{Op: Put, Key: "key2", PutData: "12"}, resp: EtcdResponse{Revision: 2}},
|
||||
{req: EtcdRequest{Op: Delete, Key: "key1"}, resp: EtcdResponse{Deleted: 1, Revision: 2}, failure: true},
|
||||
{req: EtcdRequest{Op: Delete, Key: "key1"}, resp: EtcdResponse{Deleted: 1, Revision: 3}},
|
||||
{req: EtcdRequest{Op: Delete, Key: "key1"}, resp: EtcdResponse{Deleted: 0, Revision: 4}, failure: true},
|
||||
{req: EtcdRequest{Op: Delete, Key: "key1"}, resp: EtcdResponse{Deleted: 0, Revision: 3}},
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "Delete not existing key",
|
||||
operations: []testOperation{
|
||||
{req: EtcdRequest{Op: Get, Key: "key"}, resp: EtcdResponse{Revision: 1}},
|
||||
{req: EtcdRequest{Op: Delete, Key: "key"}, resp: EtcdResponse{Deleted: 1, Revision: 2}, failure: true},
|
||||
{req: EtcdRequest{Op: Delete, Key: "key"}, resp: EtcdResponse{Deleted: 0, Revision: 1}},
|
||||
},
|
||||
},
|
||||
{
|
||||
@ -287,8 +293,10 @@ func TestModel(t *testing.T) {
|
||||
{
|
||||
name: "Txn can expect on empty key",
|
||||
operations: []testOperation{
|
||||
{req: EtcdRequest{Op: Get, Key: "key"}, resp: EtcdResponse{Revision: 1}},
|
||||
{req: EtcdRequest{Op: Txn, Key: "key", TxnExpectData: "", TxnNewData: "2"}, resp: EtcdResponse{Revision: 2, TxnSucceeded: true}},
|
||||
{req: EtcdRequest{Op: Get, Key: "key1"}, resp: EtcdResponse{Revision: 1}},
|
||||
{req: EtcdRequest{Op: Txn, Key: "key1", TxnExpectData: "", TxnNewData: "2"}, resp: EtcdResponse{Revision: 2, TxnSucceeded: true}},
|
||||
{req: EtcdRequest{Op: Txn, Key: "key2", TxnExpectData: "", TxnNewData: "3"}, resp: EtcdResponse{Revision: 3, TxnSucceeded: true}},
|
||||
{req: EtcdRequest{Op: Txn, Key: "key3", TxnExpectData: "4", TxnNewData: "4"}, resp: EtcdResponse{Revision: 4}, failure: true},
|
||||
},
|
||||
},
|
||||
{
|
||||
|
@ -25,7 +25,7 @@ import (
|
||||
)
|
||||
|
||||
var (
|
||||
DefaultTraffic Traffic = readWriteSingleKey{key: "key", writes: []opChance{{operation: Put, chance: 90}, {operation: Delete, chance: 5}, {operation: Txn, chance: 5}}}
|
||||
DefaultTraffic Traffic = readWriteSingleKey{keyCount: 4, writes: []opChance{{operation: Put, chance: 60}, {operation: Delete, chance: 20}, {operation: Txn, chance: 20}}}
|
||||
)
|
||||
|
||||
type Traffic interface {
|
||||
@ -33,8 +33,8 @@ type Traffic interface {
|
||||
}
|
||||
|
||||
type readWriteSingleKey struct {
|
||||
key string
|
||||
writes []opChance
|
||||
keyCount int
|
||||
writes []opChance
|
||||
}
|
||||
|
||||
type opChance struct {
|
||||
@ -50,19 +50,20 @@ func (t readWriteSingleKey) Run(ctx context.Context, c *recordingClient, limiter
|
||||
return
|
||||
default:
|
||||
}
|
||||
key := fmt.Sprintf("%d", rand.Int()%t.keyCount)
|
||||
// Execute one read per one write to avoid operation history include too many failed writes when etcd is down.
|
||||
resp, err := t.Read(ctx, c, limiter)
|
||||
resp, err := t.Read(ctx, c, limiter, key)
|
||||
if err != nil {
|
||||
continue
|
||||
}
|
||||
// Provide each write with unique id to make it easier to validate operation history.
|
||||
t.Write(ctx, c, limiter, ids.RequestId(), resp)
|
||||
t.Write(ctx, c, limiter, key, fmt.Sprintf("%d", ids.RequestId()), resp)
|
||||
}
|
||||
}
|
||||
|
||||
func (t readWriteSingleKey) Read(ctx context.Context, c *recordingClient, limiter *rate.Limiter) ([]*mvccpb.KeyValue, error) {
|
||||
func (t readWriteSingleKey) Read(ctx context.Context, c *recordingClient, limiter *rate.Limiter, key string) ([]*mvccpb.KeyValue, error) {
|
||||
getCtx, cancel := context.WithTimeout(ctx, 20*time.Millisecond)
|
||||
resp, err := c.Get(getCtx, t.key)
|
||||
resp, err := c.Get(getCtx, key)
|
||||
cancel()
|
||||
if err == nil {
|
||||
limiter.Wait(ctx)
|
||||
@ -70,21 +71,21 @@ func (t readWriteSingleKey) Read(ctx context.Context, c *recordingClient, limite
|
||||
return resp, err
|
||||
}
|
||||
|
||||
func (t readWriteSingleKey) Write(ctx context.Context, c *recordingClient, limiter *rate.Limiter, id int, kvs []*mvccpb.KeyValue) error {
|
||||
func (t readWriteSingleKey) Write(ctx context.Context, c *recordingClient, limiter *rate.Limiter, key string, newValue string, lastValues []*mvccpb.KeyValue) error {
|
||||
putCtx, cancel := context.WithTimeout(ctx, 20*time.Millisecond)
|
||||
|
||||
var err error
|
||||
switch t.pickWriteOperation() {
|
||||
case Put:
|
||||
err = c.Put(putCtx, t.key, fmt.Sprintf("%d", id))
|
||||
err = c.Put(putCtx, key, newValue)
|
||||
case Delete:
|
||||
err = c.Delete(putCtx, t.key)
|
||||
err = c.Delete(putCtx, key)
|
||||
case Txn:
|
||||
var value string
|
||||
if len(kvs) != 0 {
|
||||
value = string(kvs[0].Value)
|
||||
var expectValue string
|
||||
if len(lastValues) != 0 {
|
||||
expectValue = string(lastValues[0].Value)
|
||||
}
|
||||
err = c.Txn(putCtx, t.key, value, fmt.Sprintf("%d", id))
|
||||
err = c.Txn(putCtx, key, expectValue, newValue)
|
||||
default:
|
||||
panic("invalid operation")
|
||||
}
|
||||
|
Loading…
x
Reference in New Issue
Block a user