Merge pull request #15325 from serathius/linearizability-refactor-report

tests: Refactor file structure
dependabot/go_modules/go.uber.org/atomic-1.10.0
Marek Siarkowicz 2023-02-16 11:23:56 +01:00 committed by GitHub
commit e603d92727
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
7 changed files with 366 additions and 345 deletions

View File

@ -81,6 +81,39 @@ var (
}}
)
func triggerFailpoints(ctx context.Context, t *testing.T, lg *zap.Logger, clus *e2e.EtcdProcessCluster, config FailpointConfig) {
var err error
successes := 0
failures := 0
for _, proc := range clus.Procs {
if !config.failpoint.Available(proc) {
t.Errorf("Failpoint %q not available on %s", config.failpoint.Name(), proc.Config().Name)
return
}
}
for successes < config.count && failures < config.retries {
time.Sleep(config.waitBetweenTriggers)
lg.Info("Triggering failpoint\n", zap.String("failpoint", config.failpoint.Name()))
err = config.failpoint.Trigger(ctx, t, lg, clus)
if err != nil {
lg.Info("Failed to trigger failpoint", zap.String("failpoint", config.failpoint.Name()), zap.Error(err))
failures++
continue
}
successes++
}
if successes < config.count || failures >= config.retries {
t.Errorf("failed to trigger failpoints enough times, err: %v", err)
}
}
type FailpointConfig struct {
failpoint Failpoint
count int
retries int
waitBetweenTriggers time.Duration
}
type Failpoint interface {
Trigger(ctx context.Context, t *testing.T, lg *zap.Logger, clus *e2e.EtcdProcessCluster) error
Name() string

View File

@ -16,24 +16,15 @@ package linearizability
import (
"context"
"encoding/json"
"os"
"path/filepath"
"strings"
"sync"
"testing"
"time"
"github.com/anishathalye/porcupine"
"github.com/google/go-cmp/cmp"
"github.com/google/go-cmp/cmp/cmpopts"
"go.uber.org/zap"
"go.uber.org/zap/zaptest"
"golang.org/x/sync/errgroup"
"golang.org/x/time/rate"
"go.etcd.io/etcd/tests/v3/framework/e2e"
"go.etcd.io/etcd/tests/v3/linearizability/identity"
"go.etcd.io/etcd/tests/v3/linearizability/model"
)
@ -198,42 +189,7 @@ func testLinearizability(ctx context.Context, t *testing.T, lg *zap.Logger, conf
validateEventsMatch(t, r.events)
r.patchedOperations = patchOperationBasedOnWatchEvents(r.operations, longestHistory(r.events))
r.visualizeHistory = validateOperationHistoryAndReturnVisualize(t, lg, r.patchedOperations)
}
type report struct {
lg *zap.Logger
clus *e2e.EtcdProcessCluster
responses [][]watchResponse
events [][]watchEvent
operations []porcupine.Operation
patchedOperations []porcupine.Operation
visualizeHistory func(path string)
}
func (r *report) Report(t *testing.T) {
path := testResultsDirectory(t)
if t.Failed() {
for i, member := range r.clus.Procs {
memberDataDir := filepath.Join(path, member.Config().Name)
persistMemberDataDir(t, r.lg, member, memberDataDir)
if r.responses != nil {
persistWatchResponses(t, r.lg, filepath.Join(memberDataDir, "responses.json"), r.responses[i])
}
if r.events != nil {
persistWatchEvents(t, r.lg, filepath.Join(memberDataDir, "events.json"), r.events[i])
}
}
if r.operations != nil {
persistOperationHistory(t, r.lg, filepath.Join(path, "full-history.json"), r.operations)
}
if r.patchedOperations != nil {
persistOperationHistory(t, r.lg, filepath.Join(path, "patched-history.json"), r.patchedOperations)
}
}
if r.visualizeHistory != nil {
r.visualizeHistory(filepath.Join(path, "history.html"))
}
r.visualizeHistory = model.ValidateOperationHistoryAndReturnVisualize(t, lg, r.patchedOperations)
}
func runScenario(ctx context.Context, t *testing.T, lg *zap.Logger, clus *e2e.EtcdProcessCluster, traffic trafficConfig, failpoint FailpointConfig) (operations []porcupine.Operation, responses [][]watchResponse) {
@ -261,265 +217,6 @@ func runScenario(ctx context.Context, t *testing.T, lg *zap.Logger, clus *e2e.Et
return operations, responses
}
func patchOperationBasedOnWatchEvents(operations []porcupine.Operation, watchEvents []watchEvent) []porcupine.Operation {
newOperations := make([]porcupine.Operation, 0, len(operations))
persisted := map[model.EtcdOperation]watchEvent{}
for _, op := range watchEvents {
persisted[op.Op] = op
}
lastObservedOperation := lastOperationObservedInWatch(operations, persisted)
for _, op := range operations {
request := op.Input.(model.EtcdRequest)
resp := op.Output.(model.EtcdResponse)
if resp.Err == nil || op.Call > lastObservedOperation.Call || request.Type != model.Txn {
// Cannot patch those requests.
newOperations = append(newOperations, op)
continue
}
event := matchWatchEvent(request.Txn, persisted)
if event != nil {
// Set revision and time based on watchEvent.
op.Return = event.Time.UnixNano()
op.Output = model.EtcdResponse{
Revision: event.Revision,
ResultUnknown: true,
}
newOperations = append(newOperations, op)
continue
}
if hasNonUniqueWriteOperation(request.Txn) && !hasUniqueWriteOperation(request.Txn) {
// Leave operation as it is as we cannot match non-unique operations to watch events.
newOperations = append(newOperations, op)
continue
}
// Remove non persisted operations
}
return newOperations
}
func lastOperationObservedInWatch(operations []porcupine.Operation, watchEvents map[model.EtcdOperation]watchEvent) porcupine.Operation {
var maxCallTime int64
var lastOperation porcupine.Operation
for _, op := range operations {
request := op.Input.(model.EtcdRequest)
if request.Type != model.Txn {
continue
}
event := matchWatchEvent(request.Txn, watchEvents)
if event != nil && op.Call > maxCallTime {
maxCallTime = op.Call
lastOperation = op
}
}
return lastOperation
}
func matchWatchEvent(request *model.TxnRequest, watchEvents map[model.EtcdOperation]watchEvent) *watchEvent {
for _, etcdOp := range request.Ops {
if etcdOp.Type == model.Put {
// Remove LeaseID which is not exposed in watch.
event, ok := watchEvents[model.EtcdOperation{
Type: etcdOp.Type,
Key: etcdOp.Key,
Value: etcdOp.Value,
}]
if ok {
return &event
}
}
}
return nil
}
func hasNonUniqueWriteOperation(request *model.TxnRequest) bool {
for _, etcdOp := range request.Ops {
if etcdOp.Type == model.Put || etcdOp.Type == model.Delete {
return true
}
}
return false
}
func hasUniqueWriteOperation(request *model.TxnRequest) bool {
for _, etcdOp := range request.Ops {
if etcdOp.Type == model.Put {
return true
}
}
return false
}
func triggerFailpoints(ctx context.Context, t *testing.T, lg *zap.Logger, clus *e2e.EtcdProcessCluster, config FailpointConfig) {
var err error
successes := 0
failures := 0
for _, proc := range clus.Procs {
if !config.failpoint.Available(proc) {
t.Errorf("Failpoint %q not available on %s", config.failpoint.Name(), proc.Config().Name)
return
}
}
for successes < config.count && failures < config.retries {
time.Sleep(config.waitBetweenTriggers)
lg.Info("Triggering failpoint\n", zap.String("failpoint", config.failpoint.Name()))
err = config.failpoint.Trigger(ctx, t, lg, clus)
if err != nil {
lg.Info("Failed to trigger failpoint", zap.String("failpoint", config.failpoint.Name()), zap.Error(err))
failures++
continue
}
successes++
}
if successes < config.count || failures >= config.retries {
t.Errorf("failed to trigger failpoints enough times, err: %v", err)
}
}
type FailpointConfig struct {
failpoint Failpoint
count int
retries int
waitBetweenTriggers time.Duration
}
func simulateTraffic(ctx context.Context, t *testing.T, lg *zap.Logger, clus *e2e.EtcdProcessCluster, config trafficConfig) []porcupine.Operation {
mux := sync.Mutex{}
endpoints := clus.EndpointsV3()
ids := identity.NewIdProvider()
lm := identity.NewLeaseIdStorage()
h := model.History{}
limiter := rate.NewLimiter(rate.Limit(config.maximalQPS), 200)
startTime := time.Now()
wg := sync.WaitGroup{}
for i := 0; i < config.clientCount; i++ {
wg.Add(1)
endpoints := []string{endpoints[i%len(endpoints)]}
c, err := NewClient(endpoints, ids, startTime)
if err != nil {
t.Fatal(err)
}
go func(c *recordingClient, clientId int) {
defer wg.Done()
defer c.Close()
config.traffic.Run(ctx, clientId, c, limiter, ids, lm)
mux.Lock()
h = h.Merge(c.history.History)
mux.Unlock()
}(c, i)
}
wg.Wait()
endTime := time.Now()
operations := h.Operations()
lg.Info("Recorded operations", zap.Int("count", len(operations)))
qps := float64(len(operations)) / float64(endTime.Sub(startTime)) * float64(time.Second)
lg.Info("Average traffic", zap.Float64("qps", qps))
if qps < config.minimalQPS {
t.Errorf("Requiring minimal %f qps for test results to be reliable, got %f qps", config.minimalQPS, qps)
}
return operations
}
type trafficConfig struct {
name string
minimalQPS float64
maximalQPS float64
clientCount int
traffic Traffic
}
func watchEvents(responses [][]watchResponse) [][]watchEvent {
ops := make([][]watchEvent, len(responses))
for i, resps := range responses {
ops[i] = toWatchEvents(resps)
}
return ops
}
func validateEventsMatch(t *testing.T, histories [][]watchEvent) {
longestHistory := longestHistory(histories)
for i := 0; i < len(histories); i++ {
length := len(histories[i])
// We compare prefix of watch events, as we are not guaranteed to collect all events from each node.
if diff := cmp.Diff(longestHistory[:length], histories[i][:length], cmpopts.IgnoreFields(watchEvent{}, "Time")); diff != "" {
t.Error("Events in watches do not match")
}
}
}
func longestHistory(histories [][]watchEvent) []watchEvent {
longestIndex := 0
for i, history := range histories {
if len(history) > len(histories[longestIndex]) {
longestIndex = i
}
}
return histories[longestIndex]
}
// return visualize as porcupine.linearizationInfo used to generate visualization is private
func validateOperationHistoryAndReturnVisualize(t *testing.T, lg *zap.Logger, operations []porcupine.Operation) (visualize func(basepath string)) {
linearizable, info := porcupine.CheckOperationsVerbose(model.Etcd, operations, 5*time.Minute)
if linearizable == porcupine.Illegal {
t.Error("Model is not linearizable")
}
if linearizable == porcupine.Unknown {
t.Error("Linearization timed out")
}
return func(path string) {
lg.Info("Saving visualization", zap.String("path", path))
err := porcupine.VisualizePath(model.Etcd, info, path)
if err != nil {
t.Errorf("Failed to visualize, err: %v", err)
}
}
}
func persistOperationHistory(t *testing.T, lg *zap.Logger, path string, operations []porcupine.Operation) {
lg.Info("Saving operation history", zap.String("path", path))
file, err := os.OpenFile(path, os.O_RDWR|os.O_CREATE|os.O_TRUNC, 0755)
if err != nil {
t.Errorf("Failed to save operation history: %v", err)
return
}
defer file.Close()
encoder := json.NewEncoder(file)
for _, op := range operations {
err := encoder.Encode(op)
if err != nil {
t.Errorf("Failed to encode operation: %v", err)
}
}
}
func persistMemberDataDir(t *testing.T, lg *zap.Logger, member e2e.EtcdProcess, path string) {
lg.Info("Saving member data dir", zap.String("member", member.Config().Name), zap.String("path", path))
err := os.Rename(member.Config().DataDirPath, path)
if err != nil {
t.Fatal(err)
}
}
func testResultsDirectory(t *testing.T) string {
path, err := filepath.Abs(filepath.Join(resultsDirectory, strings.ReplaceAll(t.Name(), "/", "_")))
if err != nil {
t.Fatal(err)
}
err = os.RemoveAll(path)
if err != nil {
t.Fatal(err)
}
err = os.MkdirAll(path, 0700)
if err != nil {
t.Fatal(err)
}
return path
}
// forcestopCluster stops the etcd member with signal kill.
func forcestopCluster(clus *e2e.EtcdProcessCluster) error {
for _, member := range clus.Procs {

View File

@ -15,27 +15,13 @@
package linearizability
import (
"os"
"path/filepath"
"testing"
"go.etcd.io/etcd/tests/v3/framework"
)
var testRunner = framework.E2eTestRunner
var resultsDirectory string
func TestMain(m *testing.M) {
var ok bool
var err error
resultsDirectory, ok = os.LookupEnv("RESULTS_DIR")
if !ok {
resultsDirectory = "/tmp/"
}
resultsDirectory, err = filepath.Abs(resultsDirectory)
if err != nil {
panic(err)
}
testRunner.TestMain(m)
}

View File

@ -16,15 +16,35 @@ package model
import (
"fmt"
"testing"
"time"
"github.com/anishathalye/porcupine"
"go.uber.org/zap"
"go.etcd.io/etcd/api/v3/etcdserverpb"
clientv3 "go.etcd.io/etcd/client/v3"
"go.etcd.io/etcd/tests/v3/linearizability/identity"
)
// ValidateOperationHistoryAndReturnVisualize return visualize as porcupine.linearizationInfo used to generate visualization is private.
func ValidateOperationHistoryAndReturnVisualize(t *testing.T, lg *zap.Logger, operations []porcupine.Operation) (visualize func(basepath string)) {
linearizable, info := porcupine.CheckOperationsVerbose(Etcd, operations, 5*time.Minute)
if linearizable == porcupine.Illegal {
t.Error("Model is not linearizable")
}
if linearizable == porcupine.Unknown {
t.Error("Linearization timed out")
}
return func(path string) {
lg.Info("Saving visualization", zap.String("path", path))
err := porcupine.VisualizePath(Etcd, info, path)
if err != nil {
t.Errorf("Failed to visualize, err: %v", err)
}
}
}
type AppendableHistory struct {
// id of the next write operation. If needed a new id might be requested from idProvider.
id int

View File

@ -0,0 +1,146 @@
// 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"
"os"
"path/filepath"
"strings"
"testing"
"github.com/anishathalye/porcupine"
"go.uber.org/zap"
"go.etcd.io/etcd/tests/v3/framework/e2e"
)
type report struct {
lg *zap.Logger
clus *e2e.EtcdProcessCluster
responses [][]watchResponse
events [][]watchEvent
operations []porcupine.Operation
patchedOperations []porcupine.Operation
visualizeHistory func(path string)
}
func testResultsDirectory(t *testing.T) string {
resultsDirectory, ok := os.LookupEnv("RESULTS_DIR")
if !ok {
resultsDirectory = "/tmp/"
}
resultsDirectory, err := filepath.Abs(resultsDirectory)
if err != nil {
panic(err)
}
path, err := filepath.Abs(filepath.Join(resultsDirectory, strings.ReplaceAll(t.Name(), "/", "_")))
if err != nil {
t.Fatal(err)
}
err = os.RemoveAll(path)
if err != nil {
t.Fatal(err)
}
err = os.MkdirAll(path, 0700)
if err != nil {
t.Fatal(err)
}
return path
}
func (r *report) Report(t *testing.T) {
path := testResultsDirectory(t)
if t.Failed() {
for i, member := range r.clus.Procs {
memberDataDir := filepath.Join(path, member.Config().Name)
persistMemberDataDir(t, r.lg, member, memberDataDir)
if r.responses != nil {
persistWatchResponses(t, r.lg, filepath.Join(memberDataDir, "responses.json"), r.responses[i])
}
if r.events != nil {
persistWatchEvents(t, r.lg, filepath.Join(memberDataDir, "events.json"), r.events[i])
}
}
if r.operations != nil {
persistOperationHistory(t, r.lg, filepath.Join(path, "full-history.json"), r.operations)
}
if r.patchedOperations != nil {
persistOperationHistory(t, r.lg, filepath.Join(path, "patched-history.json"), r.patchedOperations)
}
}
if r.visualizeHistory != nil {
r.visualizeHistory(filepath.Join(path, "history.html"))
}
}
func persistMemberDataDir(t *testing.T, lg *zap.Logger, member e2e.EtcdProcess, path string) {
lg.Info("Saving member data dir", zap.String("member", member.Config().Name), zap.String("path", path))
err := os.Rename(member.Config().DataDirPath, path)
if err != nil {
t.Fatal(err)
}
}
func persistWatchResponses(t *testing.T, lg *zap.Logger, path string, responses []watchResponse) {
lg.Info("Saving watch responses", zap.String("path", path))
file, err := os.OpenFile(path, os.O_RDWR|os.O_CREATE|os.O_TRUNC, 0755)
if err != nil {
t.Errorf("Failed to save watch history: %v", err)
return
}
defer file.Close()
encoder := json.NewEncoder(file)
for _, resp := range responses {
err := encoder.Encode(resp)
if err != nil {
t.Errorf("Failed to encode response: %v", err)
}
}
}
func persistWatchEvents(t *testing.T, lg *zap.Logger, path string, events []watchEvent) {
lg.Info("Saving watch events", zap.String("path", path))
file, err := os.OpenFile(path, os.O_RDWR|os.O_CREATE|os.O_TRUNC, 0755)
if err != nil {
t.Errorf("Failed to save watch history: %v", err)
return
}
defer file.Close()
encoder := json.NewEncoder(file)
for _, event := range events {
err := encoder.Encode(event)
if err != nil {
t.Errorf("Failed to encode response: %v", err)
}
}
}
func persistOperationHistory(t *testing.T, lg *zap.Logger, path string, operations []porcupine.Operation) {
lg.Info("Saving operation history", zap.String("path", path))
file, err := os.OpenFile(path, os.O_RDWR|os.O_CREATE|os.O_TRUNC, 0755)
if err != nil {
t.Errorf("Failed to save operation history: %v", err)
return
}
defer file.Close()
encoder := json.NewEncoder(file)
for _, op := range operations {
err := encoder.Encode(op)
if err != nil {
t.Errorf("Failed to encode operation: %v", err)
}
}
}

View File

@ -19,12 +19,17 @@ import (
"fmt"
"math/rand"
"strings"
"sync"
"testing"
"time"
"github.com/anishathalye/porcupine"
"go.uber.org/zap"
"golang.org/x/time/rate"
"go.etcd.io/etcd/api/v3/mvccpb"
clientv3 "go.etcd.io/etcd/client/v3"
"go.etcd.io/etcd/tests/v3/framework/e2e"
"go.etcd.io/etcd/tests/v3/linearizability/identity"
"go.etcd.io/etcd/tests/v3/linearizability/model"
)
@ -49,6 +54,55 @@ const (
Defragment TrafficRequestType = "defragment"
)
func simulateTraffic(ctx context.Context, t *testing.T, lg *zap.Logger, clus *e2e.EtcdProcessCluster, config trafficConfig) []porcupine.Operation {
mux := sync.Mutex{}
endpoints := clus.EndpointsV3()
ids := identity.NewIdProvider()
lm := identity.NewLeaseIdStorage()
h := model.History{}
limiter := rate.NewLimiter(rate.Limit(config.maximalQPS), 200)
startTime := time.Now()
wg := sync.WaitGroup{}
for i := 0; i < config.clientCount; i++ {
wg.Add(1)
endpoints := []string{endpoints[i%len(endpoints)]}
c, err := NewClient(endpoints, ids, startTime)
if err != nil {
t.Fatal(err)
}
go func(c *recordingClient, clientId int) {
defer wg.Done()
defer c.Close()
config.traffic.Run(ctx, clientId, c, limiter, ids, lm)
mux.Lock()
h = h.Merge(c.history.History)
mux.Unlock()
}(c, i)
}
wg.Wait()
endTime := time.Now()
operations := h.Operations()
lg.Info("Recorded operations", zap.Int("count", len(operations)))
qps := float64(len(operations)) / float64(endTime.Sub(startTime)) * float64(time.Second)
lg.Info("Average traffic", zap.Float64("qps", qps))
if qps < config.minimalQPS {
t.Errorf("Requiring minimal %f qps for test results to be reliable, got %f qps", config.minimalQPS, qps)
}
return operations
}
type trafficConfig struct {
name string
minimalQPS float64
maximalQPS float64
clientCount int
traffic Traffic
}
type Traffic interface {
Run(ctx context.Context, clientId int, c *recordingClient, limiter *rate.Limiter, ids identity.Provider, lm identity.LeaseIdStorage)
}

View File

@ -16,12 +16,13 @@ package linearizability
import (
"context"
"encoding/json"
"os"
"sync"
"testing"
"time"
"github.com/anishathalye/porcupine"
"github.com/google/go-cmp/cmp"
"github.com/google/go-cmp/cmp/cmpopts"
"go.uber.org/zap"
"go.etcd.io/etcd/api/v3/mvccpb"
@ -177,36 +178,120 @@ type watchEvent struct {
Time time.Time
}
func persistWatchResponses(t *testing.T, lg *zap.Logger, path string, responses []watchResponse) {
lg.Info("Saving watch responses", zap.String("path", path))
file, err := os.OpenFile(path, os.O_RDWR|os.O_CREATE|os.O_TRUNC, 0755)
if err != nil {
t.Errorf("Failed to save watch history: %v", err)
return
func patchOperationBasedOnWatchEvents(operations []porcupine.Operation, watchEvents []watchEvent) []porcupine.Operation {
newOperations := make([]porcupine.Operation, 0, len(operations))
persisted := map[model.EtcdOperation]watchEvent{}
for _, op := range watchEvents {
persisted[op.Op] = op
}
defer file.Close()
encoder := json.NewEncoder(file)
for _, resp := range responses {
err := encoder.Encode(resp)
if err != nil {
t.Errorf("Failed to encode response: %v", err)
lastObservedOperation := lastOperationObservedInWatch(operations, persisted)
for _, op := range operations {
request := op.Input.(model.EtcdRequest)
resp := op.Output.(model.EtcdResponse)
if resp.Err == nil || op.Call > lastObservedOperation.Call || request.Type != model.Txn {
// Cannot patch those requests.
newOperations = append(newOperations, op)
continue
}
event := matchWatchEvent(request.Txn, persisted)
if event != nil {
// Set revision and time based on watchEvent.
op.Return = event.Time.UnixNano()
op.Output = model.EtcdResponse{
Revision: event.Revision,
ResultUnknown: true,
}
newOperations = append(newOperations, op)
continue
}
if hasNonUniqueWriteOperation(request.Txn) && !hasUniqueWriteOperation(request.Txn) {
// Leave operation as it is as we cannot match non-unique operations to watch events.
newOperations = append(newOperations, op)
continue
}
// Remove non persisted operations
}
return newOperations
}
func lastOperationObservedInWatch(operations []porcupine.Operation, watchEvents map[model.EtcdOperation]watchEvent) porcupine.Operation {
var maxCallTime int64
var lastOperation porcupine.Operation
for _, op := range operations {
request := op.Input.(model.EtcdRequest)
if request.Type != model.Txn {
continue
}
event := matchWatchEvent(request.Txn, watchEvents)
if event != nil && op.Call > maxCallTime {
maxCallTime = op.Call
lastOperation = op
}
}
return lastOperation
}
func matchWatchEvent(request *model.TxnRequest, watchEvents map[model.EtcdOperation]watchEvent) *watchEvent {
for _, etcdOp := range request.Ops {
if etcdOp.Type == model.Put {
// Remove LeaseID which is not exposed in watch.
event, ok := watchEvents[model.EtcdOperation{
Type: etcdOp.Type,
Key: etcdOp.Key,
Value: etcdOp.Value,
}]
if ok {
return &event
}
}
}
return nil
}
func hasNonUniqueWriteOperation(request *model.TxnRequest) bool {
for _, etcdOp := range request.Ops {
if etcdOp.Type == model.Put || etcdOp.Type == model.Delete {
return true
}
}
return false
}
func hasUniqueWriteOperation(request *model.TxnRequest) bool {
for _, etcdOp := range request.Ops {
if etcdOp.Type == model.Put {
return true
}
}
return false
}
func watchEvents(responses [][]watchResponse) [][]watchEvent {
ops := make([][]watchEvent, len(responses))
for i, resps := range responses {
ops[i] = toWatchEvents(resps)
}
return ops
}
func validateEventsMatch(t *testing.T, histories [][]watchEvent) {
longestHistory := longestHistory(histories)
for i := 0; i < len(histories); i++ {
length := len(histories[i])
// We compare prefix of watch events, as we are not guaranteed to collect all events from each node.
if diff := cmp.Diff(longestHistory[:length], histories[i][:length], cmpopts.IgnoreFields(watchEvent{}, "Time")); diff != "" {
t.Error("Events in watches do not match")
}
}
}
func persistWatchEvents(t *testing.T, lg *zap.Logger, path string, events []watchEvent) {
lg.Info("Saving watch events", zap.String("path", path))
file, err := os.OpenFile(path, os.O_RDWR|os.O_CREATE|os.O_TRUNC, 0755)
if err != nil {
t.Errorf("Failed to save watch history: %v", err)
return
}
defer file.Close()
encoder := json.NewEncoder(file)
for _, event := range events {
err := encoder.Encode(event)
if err != nil {
t.Errorf("Failed to encode response: %v", err)
func longestHistory(histories [][]watchEvent) []watchEvent {
longestIndex := 0
for i, history := range histories {
if len(history) > len(histories[longestIndex]) {
longestIndex = i
}
}
return histories[longestIndex]
}