tests: Move linearizability model and identity to dedicated packages

Signed-off-by: Marek Siarkowicz <siarkowicz@google.com>
This commit is contained in:
Marek Siarkowicz 2023-01-16 19:16:21 +01:00
parent 677e5281e0
commit 96e2a7fbd6
9 changed files with 104 additions and 96 deletions

View File

@ -22,14 +22,16 @@ import (
"go.etcd.io/etcd/api/v3/mvccpb" "go.etcd.io/etcd/api/v3/mvccpb"
clientv3 "go.etcd.io/etcd/client/v3" clientv3 "go.etcd.io/etcd/client/v3"
"go.etcd.io/etcd/tests/v3/linearizability/identity"
"go.etcd.io/etcd/tests/v3/linearizability/model"
) )
type recordingClient struct { type recordingClient struct {
client clientv3.Client client clientv3.Client
history *appendableHistory history *model.AppendableHistory
} }
func NewClient(endpoints []string, ids idProvider) (*recordingClient, error) { func NewClient(endpoints []string, ids identity.Provider) (*recordingClient, error) {
cc, err := clientv3.New(clientv3.Config{ cc, err := clientv3.New(clientv3.Config{
Endpoints: endpoints, Endpoints: endpoints,
Logger: zap.NewNop(), Logger: zap.NewNop(),
@ -41,7 +43,7 @@ func NewClient(endpoints []string, ids idProvider) (*recordingClient, error) {
} }
return &recordingClient{ return &recordingClient{
client: *cc, client: *cc,
history: newAppendableHistory(ids), history: model.NewAppendableHistory(ids),
}, nil }, nil
} }

View File

@ -12,16 +12,16 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
package linearizability package identity
import "sync/atomic" import "sync/atomic"
type idProvider interface { type Provider interface {
ClientId() int ClientId() int
RequestId() int RequestId() int
} }
func newIdProvider() idProvider { func NewIdProvider() Provider {
return &atomicProvider{} return &atomicProvider{}
} }

View File

@ -12,19 +12,19 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
package linearizability package identity
import ( import (
"sync" "sync"
) )
type clientId2LeaseIdMapper interface { type LeaseIdStorage interface {
LeaseId(int) int64 LeaseId(int) int64
AddLeaseId(int, int64) AddLeaseId(int, int64)
RemoveLeaseId(int) RemoveLeaseId(int)
} }
func newClientId2LeaseIdMapper() clientId2LeaseIdMapper { func NewLeaseIdStorage() LeaseIdStorage {
return &atomicClientId2LeaseIdMapper{m: map[int]int64{}} return &atomicClientId2LeaseIdMapper{m: map[int]int64{}}
} }

View File

@ -31,6 +31,8 @@ import (
"golang.org/x/time/rate" "golang.org/x/time/rate"
"go.etcd.io/etcd/tests/v3/framework/e2e" "go.etcd.io/etcd/tests/v3/framework/e2e"
"go.etcd.io/etcd/tests/v3/linearizability/identity"
"go.etcd.io/etcd/tests/v3/linearizability/model"
) )
const ( const (
@ -139,14 +141,14 @@ func testLinearizability(ctx context.Context, t *testing.T, clus *e2e.EtcdProces
func patchOperationBasedOnWatchEvents(operations []porcupine.Operation, watchEvents []watchEvent) []porcupine.Operation { func patchOperationBasedOnWatchEvents(operations []porcupine.Operation, watchEvents []watchEvent) []porcupine.Operation {
newOperations := make([]porcupine.Operation, 0, len(operations)) newOperations := make([]porcupine.Operation, 0, len(operations))
persisted := map[EtcdOperation]watchEvent{} persisted := map[model.EtcdOperation]watchEvent{}
for _, op := range watchEvents { for _, op := range watchEvents {
persisted[op.Op] = op persisted[op.Op] = op
} }
lastObservedOperation := lastOperationObservedInWatch(operations, persisted) lastObservedOperation := lastOperationObservedInWatch(operations, persisted)
for _, op := range operations { for _, op := range operations {
resp := op.Output.(EtcdResponse) resp := op.Output.(model.EtcdResponse)
if resp.Err == nil || op.Call > lastObservedOperation.Call { if resp.Err == nil || op.Call > lastObservedOperation.Call {
// No need to patch successfully requests and cannot patch requests outside observed window. // No need to patch successfully requests and cannot patch requests outside observed window.
newOperations = append(newOperations, op) newOperations = append(newOperations, op)
@ -156,7 +158,7 @@ func patchOperationBasedOnWatchEvents(operations []porcupine.Operation, watchEve
if event != nil { if event != nil {
// Set revision and time based on watchEvent. // Set revision and time based on watchEvent.
op.Return = event.Time.UnixNano() op.Return = event.Time.UnixNano()
op.Output = EtcdResponse{ op.Output = model.EtcdResponse{
Revision: event.Revision, Revision: event.Revision,
ResultUnknown: true, ResultUnknown: true,
} }
@ -173,7 +175,7 @@ func patchOperationBasedOnWatchEvents(operations []porcupine.Operation, watchEve
return newOperations return newOperations
} }
func lastOperationObservedInWatch(operations []porcupine.Operation, watchEvents map[EtcdOperation]watchEvent) porcupine.Operation { func lastOperationObservedInWatch(operations []porcupine.Operation, watchEvents map[model.EtcdOperation]watchEvent) porcupine.Operation {
var maxCallTime int64 var maxCallTime int64
var lastOperation porcupine.Operation var lastOperation porcupine.Operation
for _, op := range operations { for _, op := range operations {
@ -186,17 +188,17 @@ func lastOperationObservedInWatch(operations []porcupine.Operation, watchEvents
return lastOperation return lastOperation
} }
func matchWatchEvent(op porcupine.Operation, watchEvents map[EtcdOperation]watchEvent) (event *watchEvent, hasUniqueWriteOperation bool) { func matchWatchEvent(op porcupine.Operation, watchEvents map[model.EtcdOperation]watchEvent) (event *watchEvent, hasUniqueWriteOperation bool) {
request := op.Input.(EtcdRequest) request := op.Input.(model.EtcdRequest)
for _, etcdOp := range request.Ops { for _, etcdOp := range request.Ops {
if isWrite(etcdOp.Type) && inUnique(etcdOp.Type) { if model.IsWrite(etcdOp.Type) && model.IsUnique(etcdOp.Type) {
// We expect all put to be unique as they write unique value. // We expect all put to be unique as they write unique value.
hasUniqueWriteOperation = true hasUniqueWriteOperation = true
opType := etcdOp.Type opType := etcdOp.Type
if opType == PutWithLease { if opType == model.PutWithLease {
opType = Put opType = model.Put
} }
event, ok := watchEvents[EtcdOperation{ event, ok := watchEvents[model.EtcdOperation{
Type: opType, Type: opType,
Key: etcdOp.Key, Key: etcdOp.Key,
Value: etcdOp.Value, Value: etcdOp.Value,
@ -210,9 +212,9 @@ func matchWatchEvent(op porcupine.Operation, watchEvents map[EtcdOperation]watch
} }
func hasWriteOperation(op porcupine.Operation) bool { func hasWriteOperation(op porcupine.Operation) bool {
request := op.Input.(EtcdRequest) request := op.Input.(model.EtcdRequest)
for _, etcdOp := range request.Ops { for _, etcdOp := range request.Ops {
if isWrite(etcdOp.Type) { if model.IsWrite(etcdOp.Type) {
return true return true
} }
} }
@ -255,9 +257,9 @@ func simulateTraffic(ctx context.Context, t *testing.T, clus *e2e.EtcdProcessClu
mux := sync.Mutex{} mux := sync.Mutex{}
endpoints := clus.EndpointsV3() endpoints := clus.EndpointsV3()
ids := newIdProvider() ids := identity.NewIdProvider()
lm := newClientId2LeaseIdMapper() lm := identity.NewLeaseIdStorage()
h := history{} h := model.History{}
limiter := rate.NewLimiter(rate.Limit(config.maximalQPS), 200) limiter := rate.NewLimiter(rate.Limit(config.maximalQPS), 200)
startTime := time.Now() startTime := time.Now()
@ -269,15 +271,15 @@ func simulateTraffic(ctx context.Context, t *testing.T, clus *e2e.EtcdProcessClu
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }
go func(c *recordingClient) { go func(c *recordingClient, clientId int) {
defer wg.Done() defer wg.Done()
defer c.Close() defer c.Close()
config.traffic.Run(ctx, c, limiter, ids, lm) config.traffic.Run(ctx, clientId, c, limiter, ids, lm)
mux.Lock() mux.Lock()
h = h.Merge(c.history.history) h = h.Merge(c.history.History)
mux.Unlock() mux.Unlock()
}(c) }(c, i)
} }
wg.Wait() wg.Wait()
endTime := time.Now() endTime := time.Now()
@ -322,7 +324,7 @@ func checkOperationsAndPersistResults(t *testing.T, operations []porcupine.Opera
t.Error(err) t.Error(err)
} }
linearizable, info := porcupine.CheckOperationsVerbose(etcdModel, operations, 0) linearizable, info := porcupine.CheckOperationsVerbose(model.Etcd, operations, 0)
if linearizable != porcupine.Ok { if linearizable != porcupine.Ok {
t.Error("Model is not linearizable") t.Error("Model is not linearizable")
persistMemberDataDir(t, clus, path) persistMemberDataDir(t, clus, path)
@ -330,7 +332,7 @@ func checkOperationsAndPersistResults(t *testing.T, operations []porcupine.Opera
visualizationPath := filepath.Join(path, "history.html") visualizationPath := filepath.Join(path, "history.html")
t.Logf("saving visualization to %q", visualizationPath) t.Logf("saving visualization to %q", visualizationPath)
err = porcupine.VisualizePath(etcdModel, info, visualizationPath) err = porcupine.VisualizePath(model.Etcd, info, visualizationPath)
if err != nil { if err != nil {
t.Errorf("Failed to visualize, err: %v", err) t.Errorf("Failed to visualize, err: %v", err)
} }

View File

@ -12,7 +12,7 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
package linearizability package model
import ( import (
"time" "time"
@ -20,28 +20,29 @@ import (
"github.com/anishathalye/porcupine" "github.com/anishathalye/porcupine"
clientv3 "go.etcd.io/etcd/client/v3" clientv3 "go.etcd.io/etcd/client/v3"
"go.etcd.io/etcd/tests/v3/linearizability/identity"
) )
type appendableHistory struct { type AppendableHistory struct {
// id of the next write operation. If needed a new id might be requested from idProvider. // id of the next write operation. If needed a new id might be requested from idProvider.
id int id int
idProvider idProvider idProvider identity.Provider
history History
} }
func newAppendableHistory(ids idProvider) *appendableHistory { func NewAppendableHistory(ids identity.Provider) *AppendableHistory {
return &appendableHistory{ return &AppendableHistory{
id: ids.ClientId(), id: ids.ClientId(),
idProvider: ids, idProvider: ids,
history: history{ History: History{
successful: []porcupine.Operation{}, successful: []porcupine.Operation{},
failed: []porcupine.Operation{}, failed: []porcupine.Operation{},
}, },
} }
} }
func (h *appendableHistory) AppendGet(key string, start, end time.Time, resp *clientv3.GetResponse) { func (h *AppendableHistory) AppendGet(key string, start, end time.Time, resp *clientv3.GetResponse) {
var readData string var readData string
if len(resp.Kvs) == 1 { if len(resp.Kvs) == 1 {
readData = string(resp.Kvs[0].Value) readData = string(resp.Kvs[0].Value)
@ -59,7 +60,7 @@ func (h *appendableHistory) AppendGet(key string, start, end time.Time, resp *cl
}) })
} }
func (h *appendableHistory) AppendPut(key, value string, start, end time.Time, resp *clientv3.PutResponse, err error) { func (h *AppendableHistory) AppendPut(key, value string, start, end time.Time, resp *clientv3.PutResponse, err error) {
request := putRequest(key, value) request := putRequest(key, value)
if err != nil { if err != nil {
h.appendFailed(request, start, err) h.appendFailed(request, start, err)
@ -78,7 +79,7 @@ func (h *appendableHistory) AppendPut(key, value string, start, end time.Time, r
}) })
} }
func (h *appendableHistory) AppendPutWithLease(key, value string, leaseID int64, start, end time.Time, resp *clientv3.PutResponse, err error) { func (h *AppendableHistory) AppendPutWithLease(key, value string, leaseID int64, start, end time.Time, resp *clientv3.PutResponse, err error) {
request := putWithLeaseRequest(key, value, leaseID) request := putWithLeaseRequest(key, value, leaseID)
if err != nil { if err != nil {
h.appendFailed(request, start, err) h.appendFailed(request, start, err)
@ -97,7 +98,7 @@ func (h *appendableHistory) AppendPutWithLease(key, value string, leaseID int64,
}) })
} }
func (h *appendableHistory) AppendLeaseGrant(start, end time.Time, resp *clientv3.LeaseGrantResponse, err error) { func (h *AppendableHistory) AppendLeaseGrant(start, end time.Time, resp *clientv3.LeaseGrantResponse, err error) {
var leaseID int64 var leaseID int64
if resp != nil { if resp != nil {
leaseID = int64(resp.ID) leaseID = int64(resp.ID)
@ -120,7 +121,7 @@ func (h *appendableHistory) AppendLeaseGrant(start, end time.Time, resp *clientv
}) })
} }
func (h *appendableHistory) AppendLeaseRevoke(id int64, start time.Time, end time.Time, resp *clientv3.LeaseRevokeResponse, err error) { func (h *AppendableHistory) AppendLeaseRevoke(id int64, start time.Time, end time.Time, resp *clientv3.LeaseRevokeResponse, err error) {
request := leaseRevokeRequest(id) request := leaseRevokeRequest(id)
if err != nil { if err != nil {
h.appendFailed(request, start, err) h.appendFailed(request, start, err)
@ -139,7 +140,7 @@ func (h *appendableHistory) AppendLeaseRevoke(id int64, start time.Time, end tim
}) })
} }
func (h *appendableHistory) AppendDelete(key string, start, end time.Time, resp *clientv3.DeleteResponse, err error) { func (h *AppendableHistory) AppendDelete(key string, start, end time.Time, resp *clientv3.DeleteResponse, err error) {
request := deleteRequest(key) request := deleteRequest(key)
if err != nil { if err != nil {
h.appendFailed(request, start, err) h.appendFailed(request, start, err)
@ -160,7 +161,7 @@ func (h *appendableHistory) AppendDelete(key string, start, end time.Time, resp
}) })
} }
func (h *appendableHistory) AppendTxn(key, expectValue, newValue string, start, end time.Time, resp *clientv3.TxnResponse, err error) { func (h *AppendableHistory) AppendTxn(key, expectValue, newValue string, start, end time.Time, resp *clientv3.TxnResponse, err error) {
request := txnRequest(key, expectValue, newValue) request := txnRequest(key, expectValue, newValue)
if err != nil { if err != nil {
h.appendFailed(request, start, err) h.appendFailed(request, start, err)
@ -179,7 +180,7 @@ func (h *appendableHistory) AppendTxn(key, expectValue, newValue string, start,
}) })
} }
func (h *appendableHistory) appendFailed(request EtcdRequest, start time.Time, err error) { func (h *AppendableHistory) appendFailed(request EtcdRequest, start time.Time, err error) {
h.failed = append(h.failed, porcupine.Operation{ h.failed = append(h.failed, porcupine.Operation{
ClientId: h.id, ClientId: h.id,
Input: request, Input: request,
@ -256,15 +257,15 @@ func leaseRevokeResponse(revision int64) EtcdResponse {
return EtcdResponse{OpsResult: []EtcdOperationResult{{}}, Revision: revision} return EtcdResponse{OpsResult: []EtcdOperationResult{{}}, Revision: revision}
} }
type history struct { type History struct {
successful []porcupine.Operation successful []porcupine.Operation
// failed requests are kept separate as we don't know return time of failed operations. // failed requests are kept separate as we don't know return time of failed operations.
// Based on https://github.com/anishathalye/porcupine/issues/10 // Based on https://github.com/anishathalye/porcupine/issues/10
failed []porcupine.Operation failed []porcupine.Operation
} }
func (h history) Merge(h2 history) history { func (h History) Merge(h2 History) History {
result := history{ result := History{
successful: make([]porcupine.Operation, 0, len(h.successful)+len(h2.successful)), successful: make([]porcupine.Operation, 0, len(h.successful)+len(h2.successful)),
failed: make([]porcupine.Operation, 0, len(h.failed)+len(h2.failed)), failed: make([]porcupine.Operation, 0, len(h.failed)+len(h2.failed)),
} }
@ -275,7 +276,7 @@ func (h history) Merge(h2 history) history {
return result return result
} }
func (h history) Operations() []porcupine.Operation { func (h History) Operations() []porcupine.Operation {
operations := make([]porcupine.Operation, 0, len(h.successful)+len(h.failed)) operations := make([]porcupine.Operation, 0, len(h.successful)+len(h.failed))
var maxTime int64 var maxTime int64
for _, op := range h.successful { for _, op := range h.successful {

View File

@ -12,7 +12,7 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
package linearizability package model
import ( import (
"encoding/json" "encoding/json"
@ -35,11 +35,33 @@ const (
LeaseRevoke OperationType = "leaseRevoke" LeaseRevoke OperationType = "leaseRevoke"
) )
func isWrite(t OperationType) bool { var Etcd = porcupine.Model{
Init: func() interface{} {
return "[]" // empty PossibleStates
},
Step: func(st interface{}, in interface{}, out interface{}) (bool, interface{}) {
var states PossibleStates
err := json.Unmarshal([]byte(st.(string)), &states)
if err != nil {
panic(err)
}
ok, states := step(states, in.(EtcdRequest), out.(EtcdResponse))
data, err := json.Marshal(states)
if err != nil {
panic(err)
}
return ok, string(data)
},
DescribeOperation: func(in, out interface{}) string {
return describeEtcdRequestResponse(in.(EtcdRequest), out.(EtcdResponse))
},
}
func IsWrite(t OperationType) bool {
return t == Put || t == Delete || t == PutWithLease || t == LeaseRevoke || t == LeaseGrant return t == Put || t == Delete || t == PutWithLease || t == LeaseRevoke || t == LeaseGrant
} }
func inUnique(t OperationType) bool { func IsUnique(t OperationType) bool {
return t == Put || t == PutWithLease return t == Put || t == PutWithLease
} }
@ -92,28 +114,6 @@ type EtcdState struct {
Leases map[int64]EtcdLease Leases map[int64]EtcdLease
} }
var etcdModel = porcupine.Model{
Init: func() interface{} {
return "[]" // empty PossibleStates
},
Step: func(st interface{}, in interface{}, out interface{}) (bool, interface{}) {
var states PossibleStates
err := json.Unmarshal([]byte(st.(string)), &states)
if err != nil {
panic(err)
}
ok, states := step(states, in.(EtcdRequest), out.(EtcdResponse))
data, err := json.Marshal(states)
if err != nil {
panic(err)
}
return ok, string(data)
},
DescribeOperation: func(in, out interface{}) string {
return describeEtcdRequestResponse(in.(EtcdRequest), out.(EtcdResponse))
},
}
func describeEtcdRequestResponse(request EtcdRequest, response EtcdResponse) string { func describeEtcdRequestResponse(request EtcdRequest, response EtcdResponse) string {
prefix := describeEtcdOperations(request.Ops) prefix := describeEtcdOperations(request.Ops)
if len(request.Conds) != 0 { if len(request.Conds) != 0 {

View File

@ -12,7 +12,7 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
package linearizability package model
import ( import (
"errors" "errors"
@ -521,12 +521,12 @@ func TestModelStep(t *testing.T) {
} }
for _, tc := range tcs { for _, tc := range tcs {
t.Run(tc.name, func(t *testing.T) { t.Run(tc.name, func(t *testing.T) {
state := etcdModel.Init() state := Etcd.Init()
for _, op := range tc.operations { for _, op := range tc.operations {
ok, newState := etcdModel.Step(state, op.req, op.resp) ok, newState := Etcd.Step(state, op.req, op.resp)
if ok != !op.failure { if ok != !op.failure {
t.Logf("state: %v", state) t.Logf("state: %v", state)
t.Errorf("Unexpected operation result, expect: %v, got: %v, operation: %s", !op.failure, ok, etcdModel.DescribeOperation(op.req, op.resp)) t.Errorf("Unexpected operation result, expect: %v, got: %v, operation: %s", !op.failure, ok, Etcd.DescribeOperation(op.req, op.resp))
} }
if ok { if ok {
state = newState state = newState
@ -601,7 +601,7 @@ func TestModelDescribe(t *testing.T) {
}, },
} }
for _, tc := range tcs { for _, tc := range tcs {
assert.Equal(t, tc.expectDescribe, etcdModel.DescribeOperation(tc.req, tc.resp)) assert.Equal(t, tc.expectDescribe, Etcd.DescribeOperation(tc.req, tc.resp))
} }
} }

View File

@ -23,16 +23,18 @@ import (
"golang.org/x/time/rate" "golang.org/x/time/rate"
"go.etcd.io/etcd/api/v3/mvccpb" "go.etcd.io/etcd/api/v3/mvccpb"
"go.etcd.io/etcd/tests/v3/linearizability/identity"
"go.etcd.io/etcd/tests/v3/linearizability/model"
) )
var ( var (
DefaultLeaseTTL int64 = 7200 DefaultLeaseTTL int64 = 7200
RequestTimeout = 40 * time.Millisecond RequestTimeout = 40 * time.Millisecond
DefaultTraffic Traffic = readWriteSingleKey{keyCount: 4, leaseTTL: DefaultLeaseTTL, writes: []opChance{{operation: Put, chance: 50}, {operation: Delete, chance: 10}, {operation: PutWithLease, chance: 10}, {operation: LeaseRevoke, chance: 10}, {operation: Txn, chance: 20}}} DefaultTraffic Traffic = readWriteSingleKey{keyCount: 4, leaseTTL: DefaultLeaseTTL, writes: []opChance{{operation: model.Put, chance: 50}, {operation: model.Delete, chance: 10}, {operation: model.PutWithLease, chance: 10}, {operation: model.LeaseRevoke, chance: 10}, {operation: model.Txn, chance: 20}}}
) )
type Traffic interface { type Traffic interface {
Run(ctx context.Context, c *recordingClient, limiter *rate.Limiter, ids idProvider, lm clientId2LeaseIdMapper) Run(ctx context.Context, clientId int, c *recordingClient, limiter *rate.Limiter, ids identity.Provider, lm identity.LeaseIdStorage)
} }
type readWriteSingleKey struct { type readWriteSingleKey struct {
@ -42,11 +44,11 @@ type readWriteSingleKey struct {
} }
type opChance struct { type opChance struct {
operation OperationType operation model.OperationType
chance int chance int
} }
func (t readWriteSingleKey) Run(ctx context.Context, c *recordingClient, limiter *rate.Limiter, ids idProvider, lm clientId2LeaseIdMapper) { func (t readWriteSingleKey) Run(ctx context.Context, clientId int, c *recordingClient, limiter *rate.Limiter, ids identity.Provider, lm identity.LeaseIdStorage) {
for { for {
select { select {
@ -61,7 +63,7 @@ func (t readWriteSingleKey) Run(ctx context.Context, c *recordingClient, limiter
continue continue
} }
// Provide each write with unique id to make it easier to validate operation history. // Provide each write with unique id to make it easier to validate operation history.
t.Write(ctx, c, limiter, key, fmt.Sprintf("%d", ids.RequestId()), lm, c.history.id, resp) t.Write(ctx, c, limiter, key, fmt.Sprintf("%d", ids.RequestId()), lm, clientId, resp)
} }
} }
@ -75,22 +77,22 @@ func (t readWriteSingleKey) Read(ctx context.Context, c *recordingClient, limite
return resp, err return resp, err
} }
func (t readWriteSingleKey) Write(ctx context.Context, c *recordingClient, limiter *rate.Limiter, key string, newValue string, lm clientId2LeaseIdMapper, cid int, lastValues []*mvccpb.KeyValue) error { func (t readWriteSingleKey) Write(ctx context.Context, c *recordingClient, limiter *rate.Limiter, key string, newValue string, lm identity.LeaseIdStorage, cid int, lastValues []*mvccpb.KeyValue) error {
writeCtx, cancel := context.WithTimeout(ctx, RequestTimeout) writeCtx, cancel := context.WithTimeout(ctx, RequestTimeout)
var err error var err error
switch t.pickWriteOperation() { switch t.pickWriteOperation() {
case Put: case model.Put:
err = c.Put(writeCtx, key, newValue) err = c.Put(writeCtx, key, newValue)
case Delete: case model.Delete:
err = c.Delete(writeCtx, key) err = c.Delete(writeCtx, key)
case Txn: case model.Txn:
var expectValue string var expectValue string
if len(lastValues) != 0 { if len(lastValues) != 0 {
expectValue = string(lastValues[0].Value) expectValue = string(lastValues[0].Value)
} }
err = c.Txn(writeCtx, key, expectValue, newValue) err = c.Txn(writeCtx, key, expectValue, newValue)
case PutWithLease: case model.PutWithLease:
leaseId := lm.LeaseId(cid) leaseId := lm.LeaseId(cid)
if leaseId == 0 { if leaseId == 0 {
leaseId, err = c.LeaseGrant(writeCtx, t.leaseTTL) leaseId, err = c.LeaseGrant(writeCtx, t.leaseTTL)
@ -104,7 +106,7 @@ func (t readWriteSingleKey) Write(ctx context.Context, c *recordingClient, limit
err = c.PutWithLease(putCtx, key, newValue, leaseId) err = c.PutWithLease(putCtx, key, newValue, leaseId)
putCancel() putCancel()
} }
case LeaseRevoke: case model.LeaseRevoke:
leaseId := lm.LeaseId(cid) leaseId := lm.LeaseId(cid)
if leaseId != 0 { if leaseId != 0 {
err = c.LeaseRevoke(writeCtx, leaseId) err = c.LeaseRevoke(writeCtx, leaseId)
@ -123,7 +125,7 @@ func (t readWriteSingleKey) Write(ctx context.Context, c *recordingClient, limit
return err return err
} }
func (t readWriteSingleKey) pickWriteOperation() OperationType { func (t readWriteSingleKey) pickWriteOperation() model.OperationType {
sum := 0 sum := 0
for _, op := range t.writes { for _, op := range t.writes {
sum += op.chance sum += op.chance

View File

@ -25,6 +25,7 @@ import (
"go.etcd.io/etcd/api/v3/mvccpb" "go.etcd.io/etcd/api/v3/mvccpb"
clientv3 "go.etcd.io/etcd/client/v3" clientv3 "go.etcd.io/etcd/client/v3"
"go.etcd.io/etcd/tests/v3/framework/e2e" "go.etcd.io/etcd/tests/v3/framework/e2e"
"go.etcd.io/etcd/tests/v3/linearizability/model"
) )
func collectClusterWatchEvents(ctx context.Context, t *testing.T, clus *e2e.EtcdProcessCluster) [][]watchEvent { func collectClusterWatchEvents(ctx context.Context, t *testing.T, clus *e2e.EtcdProcessCluster) [][]watchEvent {
@ -69,17 +70,17 @@ func collectMemberWatchEvents(ctx context.Context, t *testing.T, c *clientv3.Cli
lastRevision = resp.Header.Revision lastRevision = resp.Header.Revision
time := time.Now() time := time.Now()
for _, event := range resp.Events { for _, event := range resp.Events {
var op OperationType var op model.OperationType
switch event.Type { switch event.Type {
case mvccpb.PUT: case mvccpb.PUT:
op = Put op = model.Put
case mvccpb.DELETE: case mvccpb.DELETE:
op = Delete op = model.Delete
} }
events = append(events, watchEvent{ events = append(events, watchEvent{
Time: time, Time: time,
Revision: event.Kv.ModRevision, Revision: event.Kv.ModRevision,
Op: EtcdOperation{ Op: model.EtcdOperation{
Type: op, Type: op,
Key: string(event.Kv.Key), Key: string(event.Kv.Key),
Value: string(event.Kv.Value), Value: string(event.Kv.Value),
@ -94,7 +95,7 @@ func collectMemberWatchEvents(ctx context.Context, t *testing.T, c *clientv3.Cli
} }
type watchEvent struct { type watchEvent struct {
Op EtcdOperation Op model.EtcdOperation
Revision int64 Revision int64
Time time.Time Time time.Time
} }