Merge pull request #17426 from MadhavJivrajani/robustness-tests-cleanup

tests/robustness: refactors, cleanups and followups
This commit is contained in:
Marek Siarkowicz 2024-02-15 09:23:05 +01:00 committed by GitHub
commit 840d486923
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
7 changed files with 45 additions and 37 deletions

View File

@ -31,6 +31,7 @@ Upon failure tests generate a report that can be used to attribute whether failu
It is recommended to run tests multiple times with failfast enabled. this can be done by setting `GO_TEST_FLAGS='--count=100 --failfast'`.
* `EXPECT_DEBUG=true` - to get logs from the cluster.
* `RESULTS_DIR` - to change location where results report will be saved.
* `PERSIST_RESULTS` - to persist the results report of the test. By default this will not be persisted in the case of a successful run.
## Analysing failure

View File

@ -19,7 +19,7 @@ import (
"strings"
)
func NewReplay(eventHistory []WatchEvent) *EtcdReplay {
func NewReplay(eventHistory []PersistedEvent) *EtcdReplay {
var lastEventRevision int64 = 1
for _, event := range eventHistory {
if event.Revision > lastEventRevision && event.Revision != lastEventRevision+1 {
@ -33,7 +33,7 @@ func NewReplay(eventHistory []WatchEvent) *EtcdReplay {
}
type EtcdReplay struct {
eventHistory []WatchEvent
eventHistory []PersistedEvent
// Cached state and event index used for it's calculation
cachedState *EtcdState

View File

@ -59,7 +59,8 @@ func testResultsDirectory(t *testing.T) string {
func (r *TestReport) Report(t *testing.T, force bool) {
path := testResultsDirectory(t)
if t.Failed() || force {
_, ok := os.LookupEnv("PERSIST_RESULTS")
if t.Failed() || force || ok {
for _, member := range r.Cluster.Procs {
memberDataDir := filepath.Join(path, fmt.Sprintf("server-%s", member.Config().Name))
persistMemberDataDir(t, r.Logger, member, memberDataDir)

View File

@ -52,7 +52,7 @@ func validateLinearizableOperationsAndVisualize(lg *zap.Logger, operations []por
}
}
func validateSerializableOperations(t *testing.T, lg *zap.Logger, operations []porcupine.Operation, totalEventHistory []model.WatchEvent) {
func validateSerializableOperations(t *testing.T, lg *zap.Logger, operations []porcupine.Operation, totalEventHistory []model.PersistedEvent) {
lg.Info("Validating serializable operations")
staleReads := filterSerializableReads(operations)
if len(staleReads) == 0 {

View File

@ -51,27 +51,27 @@ type Config struct {
ExpectRevisionUnique bool
}
func mergeWatchEventHistory(reports []report.ClientReport) ([]model.WatchEvent, error) {
func mergeWatchEventHistory(reports []report.ClientReport) ([]model.PersistedEvent, error) {
type revisionEvents struct {
events []model.WatchEvent
events []model.PersistedEvent
revision int64
clientId int
}
revisionToEvents := map[int64]revisionEvents{}
var lastClientId = 0
var lastRevision int64
events := []model.WatchEvent{}
events := []model.PersistedEvent{}
for _, r := range reports {
for _, op := range r.Watch {
for _, resp := range op.Responses {
for _, event := range resp.Events {
if event.Revision == lastRevision && lastClientId == r.ClientId {
events = append(events, event)
events = append(events, event.PersistedEvent)
} else {
if prev, found := revisionToEvents[lastRevision]; found {
// This assumes that there are txn that would be observed differently by two watches.
// TODO: Implement merging events from multiple watches about single revision based on operations.
if diff := cmp.Diff(prev.events, events, cmp.Comparer(compareWatchEvents)); diff != "" {
if diff := cmp.Diff(prev.events, events); diff != "" {
return nil, fmt.Errorf("events between clients %d and %d don't match, revision: %d, diff: %s", prev.clientId, lastClientId, lastRevision, diff)
}
} else {
@ -79,14 +79,14 @@ func mergeWatchEventHistory(reports []report.ClientReport) ([]model.WatchEvent,
}
lastClientId = r.ClientId
lastRevision = event.Revision
events = []model.WatchEvent{event}
events = []model.PersistedEvent{event.PersistedEvent}
}
}
}
}
}
if prev, found := revisionToEvents[lastRevision]; found {
if diff := cmp.Diff(prev.events, events, cmp.Comparer(compareWatchEvents)); diff != "" {
if diff := cmp.Diff(prev.events, events); diff != "" {
return nil, fmt.Errorf("events between clients %d and %d don't match, revision: %d, diff: %s", prev.clientId, lastClientId, lastRevision, diff)
}
} else {
@ -100,23 +100,9 @@ func mergeWatchEventHistory(reports []report.ClientReport) ([]model.WatchEvent,
sort.Slice(allRevisionEvents, func(i, j int) bool {
return allRevisionEvents[i].revision < allRevisionEvents[j].revision
})
var eventHistory []model.WatchEvent
var eventHistory []model.PersistedEvent
for _, revEvents := range allRevisionEvents {
eventHistory = append(eventHistory, revEvents.events...)
}
return eventHistory, nil
}
func compareWatchEvents(x, y []model.WatchEvent) bool {
if len(x) != len(y) {
return false
}
for i := 0; i < len(x); i++ {
if x[i].PersistedEvent != y[i].PersistedEvent {
return false
}
}
return true
}

View File

@ -89,6 +89,7 @@ func TestValidateWatch(t *testing.T) {
},
},
Revision: 2,
IsCreate: true,
},
},
},
@ -148,6 +149,7 @@ func TestValidateWatch(t *testing.T) {
},
},
Revision: 2,
IsCreate: true,
},
},
},
@ -182,6 +184,7 @@ func TestValidateWatch(t *testing.T) {
},
},
Revision: 2,
IsCreate: true,
},
},
},
@ -207,6 +210,7 @@ func TestValidateWatch(t *testing.T) {
},
},
Revision: 2,
IsCreate: true,
},
},
},

View File

@ -23,7 +23,7 @@ import (
"go.etcd.io/etcd/tests/v3/robustness/report"
)
func validateWatch(t *testing.T, lg *zap.Logger, cfg Config, reports []report.ClientReport, eventHistory []model.WatchEvent) {
func validateWatch(t *testing.T, lg *zap.Logger, cfg Config, reports []report.ClientReport, eventHistory []model.PersistedEvent) {
lg.Info("Validating watch")
// Validate etcd watch properties defined in https://etcd.io/docs/v3.6/learning/api_guarantees/#watch-apis
for _, r := range reports {
@ -35,6 +35,7 @@ func validateWatch(t *testing.T, lg *zap.Logger, cfg Config, reports []report.Cl
validateReliable(t, eventHistory, r)
validateResumable(t, eventHistory, r)
validatePrevKV(t, r, eventHistory)
validateCreateEvent(t, r, eventHistory)
}
}
}
@ -106,7 +107,7 @@ func validateAtomic(t *testing.T, report report.ClientReport) {
}
}
func validateReliable(t *testing.T, events []model.WatchEvent, report report.ClientReport) {
func validateReliable(t *testing.T, events []model.PersistedEvent, report report.ClientReport) {
for _, op := range report.Watch {
index := 0
revision := firstRevision(op)
@ -118,7 +119,7 @@ func validateReliable(t *testing.T, events []model.WatchEvent, report report.Cli
}
for _, resp := range op.Responses {
for _, event := range resp.Events {
if events[index].Match(op.Request) && events[index].PersistedEvent != event.PersistedEvent {
if events[index].Match(op.Request) && events[index] != event.PersistedEvent {
t.Errorf("Broke watch guarantee: Reliable - a sequence of events will never drop any subsequence of events; if there are events ordered in time as a < b < c, then if the watch receives events a and c, it is guaranteed to receive b, event missing: %+v, got: %+v", events[index], event)
}
index++
@ -127,7 +128,7 @@ func validateReliable(t *testing.T, events []model.WatchEvent, report report.Cli
}
}
func validateResumable(t *testing.T, events []model.WatchEvent, report report.ClientReport) {
func validateResumable(t *testing.T, events []model.PersistedEvent, report report.ClientReport) {
for _, op := range report.Watch {
index := 0
revision := op.Request.Revision
@ -139,7 +140,7 @@ func validateResumable(t *testing.T, events []model.WatchEvent, report report.Cl
}
firstEvent := firstWatchEvent(op)
// If watch is resumable, first event it gets should the first event that happened after the requested revision.
if firstEvent != nil && events[index].PersistedEvent != firstEvent.PersistedEvent {
if firstEvent != nil && events[index] != firstEvent.PersistedEvent {
t.Errorf("Resumable - A broken watch can be resumed by establishing a new watch starting after the last revision received in a watch event before the break, so long as the revision is in the history window, watch request: %+v, event missing: %+v, got: %+v", op.Request, events[index], *firstEvent)
}
}
@ -147,7 +148,7 @@ func validateResumable(t *testing.T, events []model.WatchEvent, report report.Cl
// validatePrevKV ensures that a watch response (if configured with WithPrevKV()) returns
// the appropriate response.
func validatePrevKV(t *testing.T, report report.ClientReport, history []model.WatchEvent) {
func validatePrevKV(t *testing.T, report report.ClientReport, history []model.PersistedEvent) {
replay := model.NewReplay(history)
for _, op := range report.Watch {
if !op.Request.WithPrevKV {
@ -171,11 +172,6 @@ func validatePrevKV(t *testing.T, report report.ClientReport, history []model.Wa
// i.e. prevKV is nil iff the event is a create event, we cannot reliably
// check that without knowing if compaction has run.
// A create event will not have an entry in our history and a non-create
// event *should* have an entry in our history.
if _, prevKeyExists := state.KeyValues[event.Key]; event.IsCreate == prevKeyExists {
t.Errorf("PrevKV - unexpected event ecountered, create event should not be in event history and update/delete event should be, event already exists: %t, is create event: %t, event: %+v", prevKeyExists, event.IsCreate, event)
}
// We allow PrevValue to be nil since in the face of compaction, etcd does not
// guarantee its presence.
if event.PrevValue != nil && *event.PrevValue != state.KeyValues[event.Key] {
@ -186,6 +182,26 @@ func validatePrevKV(t *testing.T, report report.ClientReport, history []model.Wa
}
}
func validateCreateEvent(t *testing.T, report report.ClientReport, history []model.PersistedEvent) {
replay := model.NewReplay(history)
for _, op := range report.Watch {
for _, resp := range op.Responses {
for _, event := range resp.Events {
// Get state state just before the current event.
state, err := replay.StateForRevision(event.Revision - 1)
if err != nil {
t.Error(err)
}
// A create event will not have an entry in our history and a non-create
// event *should* have an entry in our history.
if _, prevKeyExists := state.KeyValues[event.Key]; event.IsCreate == prevKeyExists {
t.Errorf("CreateEvent - unexpected event ecountered, create event should not be in event history and update/delete event should be, event already exists: %t, is create event: %t, event: %+v", prevKeyExists, event.IsCreate, event)
}
}
}
}
}
func firstRevision(op model.WatchOperation) int64 {
for _, resp := range op.Responses {
for _, event := range resp.Events {