Skip to content

Commit

Permalink
tests: Reproducing 14890
Browse files Browse the repository at this point in the history
  • Loading branch information
serathius committed Dec 6, 2022
1 parent 403f113 commit a13b1a1
Show file tree
Hide file tree
Showing 7 changed files with 278 additions and 39 deletions.
2 changes: 1 addition & 1 deletion .github/workflows/linearizability.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,7 @@ jobs:
make build
mkdir -p /tmp/linearizability
cat server/etcdserver/raft.fail.go
EXPECT_DEBUG=true GO_TEST_FLAGS='-v --count 60 --failfast --run TestLinearizability' RESULTS_DIR=/tmp/linearizability make test-linearizability
EXPECT_DEBUG=true GO_TEST_FLAGS='-v --count 1 --failfast --run TestLinearizability' RESULTS_DIR=/tmp/linearizability make test-linearizability
- uses: actions/upload-artifact@v2
if: always()
with:
Expand Down
4 changes: 2 additions & 2 deletions client/v2/keys.go
Original file line number Diff line number Diff line change
Expand Up @@ -147,7 +147,7 @@ type WatcherOptions struct {

type CreateInOrderOptions struct {
// TTL defines a period of time after-which the Node should
// expire and no longer exist. Values <= 0 are ignored. Given
// expire and no longer exist. Elements <= 0 are ignored. Given
// that the zero-value is ignored, TTL cannot be used to set
// a TTL of 0.
TTL time.Duration
Expand Down Expand Up @@ -177,7 +177,7 @@ type SetOptions struct {
PrevExist PrevExistType

// TTL defines a period of time after-which the Node should
// expire and no longer exist. Values <= 0 are ignored. Given
// expire and no longer exist. Elements <= 0 are ignored. Given
// that the zero-value is ignored, TTL cannot be used to set
// a TTL of 0.
TTL time.Duration
Expand Down
138 changes: 138 additions & 0 deletions tests/linearizability/append_model.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,138 @@
// Copyright 2022 The etcd Authors
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

package linearizability

import (
"encoding/json"
"fmt"
"strings"

"github.com/anishathalye/porcupine"
)

const (
Append Operation = "append"
)

type AppendRequest struct {
Op Operation
Key string
AppendData string
}

type AppendResponse struct {
GetData string
}

type AppendState struct {
Key string
Elements []string
}

var appendModel = porcupine.Model{
Init: func() interface{} { return "{}" },
Step: func(st interface{}, in interface{}, out interface{}) (bool, interface{}) {
var state AppendState
err := json.Unmarshal([]byte(st.(string)), &state)
if err != nil {
panic(err)
}
ok, state := appendModelStep(state, in.(AppendRequest), out.(AppendResponse))
data, err := json.Marshal(state)
if err != nil {
panic(err)
}
return ok, string(data)
},
DescribeOperation: func(in, out interface{}) string {
request := in.(AppendRequest)
response := out.(AppendResponse)
switch request.Op {
case Get:
elements := strings.Split(response.GetData, ",")
return fmt.Sprintf("get(%q) -> %q", request.Key, elements[len(elements)-1])
case Append:
return fmt.Sprintf("append(%q, %q)", request.Key, request.AppendData)
default:
return "<invalid>"
}
},
}

func appendModelStep(state AppendState, request AppendRequest, response AppendResponse) (bool, AppendState) {
if request.Key == "" {
panic("invalid request")
}
if state.Key == "" {
return true, initAppendState(request, response)
}
if state.Key != request.Key {
panic("Multiple keys not supported")
}
switch request.Op {
case Get:
return stepAppendGet(state, request, response)
case Append:
return stepAppend(state, request, response)
default:
panic("Unknown operation")
}
}

func initAppendState(request AppendRequest, response AppendResponse) AppendState {
state := AppendState{
Key: request.Key,
}
switch request.Op {
case Get:
state.Elements = elements(response)
case Append:
state.Elements = []string{request.AppendData}
default:
panic("Unknown operation")
}
return state
}

func stepAppendGet(state AppendState, request AppendRequest, response AppendResponse) (bool, AppendState) {
newElements := elements(response)
if len(newElements) < len(state.Elements) {
return false, state
}

for i := 0; i < len(state.Elements); i++ {
if state.Elements[i] != newElements[i] {
return false, state
}
}
state.Elements = newElements
return true, state
}

func stepAppend(state AppendState, request AppendRequest, response AppendResponse) (bool, AppendState) {
if request.AppendData == "" {
panic("unsupported empty appendData")
}
state.Elements = append(state.Elements, request.AppendData)
return true, state
}

func elements(response AppendResponse) []string {
elements := strings.Split(response.GetData, ",")
if len(elements) == 1 && elements[0] == "" {
elements = []string{}
}
return elements
}
68 changes: 68 additions & 0 deletions tests/linearizability/append_model_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,68 @@
// Copyright 2022 The etcd Authors
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

package linearizability

import (
"testing"
)

func TestAppendModel(t *testing.T) {
tcs := []struct {
name string
operations []testAppendOperation
}{
{
name: "Append appends",
operations: []testAppendOperation{
{req: AppendRequest{Key: "key", Op: Append, AppendData: "1"}, resp: AppendResponse{}},
{req: AppendRequest{Key: "key", Op: Get}, resp: AppendResponse{GetData: "1"}},
{req: AppendRequest{Key: "key", Op: Append, AppendData: "2"}, resp: AppendResponse{}},
{req: AppendRequest{Key: "key", Op: Get}, resp: AppendResponse{GetData: "1,3"}, failure: true},
{req: AppendRequest{Key: "key", Op: Get}, resp: AppendResponse{GetData: "1,2"}},
},
},
{
name: "Get validates prefix matches",
operations: []testAppendOperation{
{req: AppendRequest{Key: "key", Op: Get}, resp: AppendResponse{GetData: ""}},
{req: AppendRequest{Key: "key", Op: Get}, resp: AppendResponse{GetData: "1"}},
{req: AppendRequest{Key: "key", Op: Get}, resp: AppendResponse{GetData: "2"}, failure: true},
{req: AppendRequest{Key: "key", Op: Append, AppendData: "2"}, resp: AppendResponse{}},
{req: AppendRequest{Key: "key", Op: Get}, resp: AppendResponse{GetData: "1,3"}, failure: true},
{req: AppendRequest{Key: "key", Op: Get}, resp: AppendResponse{GetData: "1,2,3"}},
{req: AppendRequest{Key: "key", Op: Get}, resp: AppendResponse{GetData: "2,3"}, failure: true},
},
},
}
for _, tc := range tcs {
var ok bool
t.Run(tc.name, func(t *testing.T) {
state := appendModel.Init()
for _, op := range tc.operations {
t.Logf("state: %v", state)
ok, state = appendModel.Step(state, op.req, op.resp)
if ok != !op.failure {
t.Errorf("Unexpected operation result, expect: %v, got: %v, operation: %s", !op.failure, ok, appendModel.DescribeOperation(op.req, op.resp))
}
}
})
}
}

type testAppendOperation struct {
req AppendRequest
resp AppendResponse
failure bool
}
2 changes: 1 addition & 1 deletion tests/linearizability/history.go
Original file line number Diff line number Diff line change
Expand Up @@ -123,7 +123,7 @@ func (h *appendableHistory) appendFailed(request EtcdRequest, start time.Time, e
})
// Operations of single client needs to be sequential.
// As we don't know return time of failed operations, all new writes need to be done with new client id.
h.id = h.idProvider.ClientId()
//h.id = h.idProvider.ClientId()
}

type history struct {
Expand Down
102 changes: 67 additions & 35 deletions tests/linearizability/linearizability_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -43,47 +43,34 @@ func TestLinearizability(t *testing.T) {
tcs := []struct {
name string
failpoint Failpoint
traffic Traffic
config e2e.EtcdProcessClusterConfig
}{
{
name: "ClusterOfSize1",
failpoint: RandomFailpoint,
name: "Issue14890",
traffic: AppendOnly,
failpoint: KillFailpoint,
config: *e2e.NewConfig(
e2e.WithClusterSize(1),
e2e.WithGoFailEnabled(true),
e2e.WithCompactionBatchLimit(100), // required for compactBeforeCommitBatch and compactAfterCommitBatch failpoints
),
},
{
name: "ClusterOfSize3",
failpoint: RandomFailpoint,
config: *e2e.NewConfig(
e2e.WithGoFailEnabled(true),
e2e.WithCompactionBatchLimit(100), // required for compactBeforeCommitBatch and compactAfterCommitBatch failpoints
),
},
{
name: "Issue14370",
failpoint: RaftBeforeSavePanic,
config: *e2e.NewConfig(
e2e.WithClusterSize(1),
e2e.WithGoFailEnabled(true),
e2e.WithClusterSize(5),
),
},
}
for _, tc := range tcs {
t.Run(tc.name, func(t *testing.T) {
failpoint := FailpointConfig{
failpoint: tc.failpoint,
count: 1,
count: 60,
retries: 3,
waitBetweenTriggers: waitBetweenFailpointTriggers,
}
traffic := trafficConfig{
minimalQPS: minimalQPS,
maximalQPS: maximalQPS,
clientCount: 8,
traffic: DefaultTraffic,
minimalQPS: 100,
maximalQPS: 1000,
traffic: tc.traffic,
clientCount: 10,
}
if tc.traffic == nil {
tc.traffic = DefaultTraffic
}
testLinearizability(context.Background(), t, tc.config, failpoint, traffic)
})
Expand Down Expand Up @@ -193,18 +180,63 @@ func checkOperationsAndPersistResults(t *testing.T, operations []porcupine.Opera
t.Error(err)
}

linearizable, info := porcupine.CheckOperationsVerbose(etcdModel, operations, 0)
if linearizable != porcupine.Ok {
t.Error("Model is not linearizable")
persistMemberDataDir(t, clus, path)
appendOperations := []porcupine.Operation{}
isAppendOnly := true
appendCheck:
for _, op := range operations {
req := op.Input.(EtcdRequest)
resp := op.Output.(EtcdResponse)
switch req.Op {
case Get:
appendOperations = append(appendOperations, porcupine.Operation{
ClientId: op.ClientId,
Input: AppendRequest{Op: Get, Key: req.Key},
Call: op.Call,
Output: AppendResponse{GetData: resp.GetData},
Return: op.Return,
})
case Txn:
if resp.Err != nil || !resp.TxnSucceeded {
continue
}
elements := strings.Split(req.TxnNewData, ",")
appendOperations = append(appendOperations, porcupine.Operation{
ClientId: op.ClientId,
Input: AppendRequest{Op: Append, Key: req.Key, AppendData: elements[len(elements)-1]},
Call: op.Call,
Output: AppendResponse{GetData: resp.GetData},
Return: op.Return,
})
default:
isAppendOnly = false
break appendCheck
}
}

visualizationPath := filepath.Join(path, "history.html")
t.Logf("saving visualization to %q", visualizationPath)
err = porcupine.VisualizePath(etcdModel, info, visualizationPath)
if err != nil {
t.Errorf("Failed to visualize, err: %v", err)
if isAppendOnly {
t.Log("Using append model")
linearizable, info := porcupine.CheckOperationsVerbose(appendModel, appendOperations, 0)
if linearizable != porcupine.Ok {
t.Error("Model is not linearizable")
persistMemberDataDir(t, clus, path)
}
err = porcupine.VisualizePath(appendModel, info, visualizationPath)
if err != nil {
t.Errorf("Failed to visualize, err: %v", err)
}
} else {
t.Error("Using etcd model")
linearizable, info := porcupine.CheckOperationsVerbose(etcdModel, operations, 0)
if linearizable != porcupine.Ok {
t.Error("Model is not linearizable")
persistMemberDataDir(t, clus, path)
}
err = porcupine.VisualizePath(etcdModel, info, visualizationPath)
if err != nil {
t.Errorf("Failed to visualize, err: %v", err)
}
}
t.Logf("saving visualization to %q", visualizationPath)
}

func persistMemberDataDir(t *testing.T, clus *e2e.EtcdProcessCluster, path string) {
Expand Down
1 change: 1 addition & 0 deletions tests/linearizability/traffic.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import (

var (
DefaultTraffic Traffic = readWriteSingleKey{key: "key", writes: []opChance{{operation: Put, chance: 50}, {operation: Delete, chance: 5}, {operation: Txn, chance: 5}}}
AppendOnly Traffic = readWriteSingleKey{key: "key", writes: []opChance{{operation: Txn, chance: 100}}}
)

type Traffic interface {
Expand Down

0 comments on commit a13b1a1

Please sign in to comment.