tests/robustness: Document analysing watch issue

Signed-off-by: Marek Siarkowicz <siarkowicz@google.com>
storage-doc
Marek Siarkowicz 2023-04-05 22:40:39 +02:00
parent f6bb874298
commit 1e41d95ab2
4 changed files with 72 additions and 23 deletions

View File

@ -53,9 +53,10 @@ Report includes multiple types of files:
### Example analysis of linearization issue
Let's analyse issue [#14370].
Let's reproduce and analyse robustness test report for issue [#14370].
To reproduce the issue by yourself run `make test-robustness-issue14370`.
After a couple of tries robustness tests should report `Model is not linearizable` and save report locally.
After a couple of tries robustness tests should fail with a log `Model is not linearizable` and save report locally.
Lineralization issues are easiest to analyse via history visualization.
Open `/tmp/TestRobustness_Issue14370/history.html` file in your browser.
Jump to the error in linearization by clicking `[ jump to first error ]` on the top of the page.
@ -68,4 +69,31 @@ All following requests are invalid (connected with red line) as they have revisi
Etcd guarantee that revision is non-decreasing, so this shows a bug in etcd as there is no way revision should decrease.
This is consistent with the root cause of [#14370] as it was issue with process crash causing last write to be lost.
[#14370]: https://github.com/etcd-io/etcd/issues/14370
[#14370]: https://github.com/etcd-io/etcd/issues/14370
### Example analysis of watch issue
Let's reproduce and analyse robustness test report for issue [#15271].
To reproduce the issue by yourself run `make test-robustness-issue15271`.
After a couple of tries robustness tests should fail with a logs `Broke watch guarantee` and save report locally.
Watch issues are easiest to analyse by reading the recorded watch history.
Watch history is recorded for each member separated in different subdirectory under `/tmp/TestRobustness_Issue15271/`
Open `responses.json` for member mentioned in log `Broke watch guarantee`.
For example for member `TestRobustnessIssue15271-test-1` open `/tmp/TestRobustness_Issue15271/TestRobustnessIssue15271-test-1/responses.json`.
Each line consists of json blob corresponding to single watch response observed by client.
Look for lines with `mod_revision` equal to revision mentioned in the first log with `Broke watch guarantee`
You should see two lines where the `mod_revision` decreases like ones below:
```
{"Header":{"cluster_id":12951239930360520062,"member_id":16914881897345358027,"revision":2574,"raft_term":2},"Events":[{"kv":{"key":"Ng==","create_revision":2303,"mod_revision":2574,"version":46,"value":"Mjg5OA=="}}],"CompactRevision":0,"Canceled":false,"Created":false}
{"Header":{"cluster_id":12951239930360520062,"member_id":16914881897345358027,"revision":7708,"raft_term":2},"Events":[{"kv":{"key":"NQ==","create_revision":5,"mod_revision":91,"version":10,"value":"MTAy"}}, ... }
```
Up to the first line the `mod_revision` of events within responses only increased up to a value of `2574`.
However, the following line includes an event with `mod_revision` equal `91`.
If you follow the `mod_revision` throughout the file you should notice that watch replayed revisions second time.
This is incorrect and breaks `Ordered` and `Unique` [watch guarantees].
This is consistent with the root cause of [#14370] where member reconnecting to cluster will incorrectly resend revisions.
[#15271]: https://github.com/etcd-io/etcd/issues/15271

View File

@ -167,6 +167,18 @@ func TestRobustness(t *testing.T) {
e2e.WithClusterSize(1),
),
})
if v.Compare(version.V3_5) >= 0 {
scenarios = append(scenarios, scenario{
name: "Issue15271",
failpoint: BlackholeUntilSnapshot,
traffic: &HighTraffic,
config: *e2e.NewConfig(
e2e.WithSnapshotCount(100),
e2e.WithPeerProxy(true),
e2e.WithIsPeerTLS(true),
),
})
}
snapshotOptions := []e2e.EPClusterOption{
e2e.WithGoFailEnabled(true),
e2e.WithSnapshotCount(100),
@ -217,7 +229,7 @@ func testRobustness(ctx context.Context, t *testing.T, lg *zap.Logger, config e2
forcestopCluster(r.clus)
watchProgressNotifyEnabled := r.clus.Cfg.WatchProcessNotifyInterval != 0
validateWatchResponses(t, r.responses, traffic.requestProgress || watchProgressNotifyEnabled)
validateWatchResponses(t, r.clus, r.responses, traffic.requestProgress || watchProgressNotifyEnabled)
r.events = watchEvents(r.responses)
validateEventsMatch(t, r.events)

View File

@ -15,6 +15,11 @@ test-robustness-issue14685: /tmp/etcd-v3.5.5-failpoints/bin
GO_TEST_FLAGS='-v --run=TestRobustness/Issue14685 --count 100 --failfast --bin-dir=/tmp/etcd-v3.5.5-failpoints/bin' make test-robustness && \
echo "Failed to reproduce" || echo "Successful reproduction"
.PHONY: test-robustness-issue15271
test-robustness-issue15271: /tmp/etcd-v3.5.7-failpoints/bin
GO_TEST_FLAGS='-v --run=TestRobustness/Issue15271 --count 100 --failfast --bin-dir=/tmp/etcd-v3.5.7-failpoints/bin' make test-robustness && \
echo "Failed to reproduce" || echo "Successful reproduction"
# Failpoints
GOFAIL_VERSION = $(shell cd tools/mod && go list -m -f {{.Version}} go.etcd.io/gofail)

View File

@ -132,25 +132,25 @@ func watchResponsesMaxRevision(responses []watchResponse) int64 {
return maxRevision
}
func validateWatchResponses(t *testing.T, responses [][]watchResponse, expectProgressNotify bool) {
for _, memberResponses := range responses {
validateMemberWatchResponses(t, memberResponses, expectProgressNotify)
func validateWatchResponses(t *testing.T, clus *e2e.EtcdProcessCluster, responses [][]watchResponse, expectProgressNotify bool) {
for i, member := range clus.Procs {
validateMemberWatchResponses(t, member.Config().Name, responses[i], expectProgressNotify)
}
}
func validateMemberWatchResponses(t *testing.T, responses []watchResponse, expectProgressNotify bool) {
func validateMemberWatchResponses(t *testing.T, memberId string, responses []watchResponse, expectProgressNotify bool) {
// Validate watch is correctly configured to ensure proper testing
validateGotAtLeastOneProgressNotify(t, responses, expectProgressNotify)
validateGotAtLeastOneProgressNotify(t, memberId, responses, expectProgressNotify)
// Validate etcd watch properties defined in https://etcd.io/docs/v3.6/learning/api/#watch-streams
validateOrderedAndReliable(t, responses)
validateUnique(t, responses)
validateAtomic(t, responses)
validateOrderedAndReliable(t, memberId, responses)
validateUnique(t, memberId, responses)
validateAtomic(t, memberId, responses)
// Validate kubernetes usage of watch
validateRenewable(t, responses)
validateRenewable(t, memberId, responses)
}
func validateGotAtLeastOneProgressNotify(t *testing.T, responses []watchResponse, expectProgressNotify bool) {
func validateGotAtLeastOneProgressNotify(t *testing.T, memberId string, responses []watchResponse, expectProgressNotify bool) {
var gotProgressNotify = false
var lastHeadRevision int64 = 1
for _, resp := range responses {
@ -161,16 +161,16 @@ func validateGotAtLeastOneProgressNotify(t *testing.T, responses []watchResponse
lastHeadRevision = resp.Header.Revision
}
if gotProgressNotify != expectProgressNotify {
t.Errorf("Expected at least one progress notify: %v, got: %v", expectProgressNotify, gotProgressNotify)
t.Errorf("Progress notify does not match, expect: %v, got: %v, member: %q", expectProgressNotify, gotProgressNotify, memberId)
}
}
func validateRenewable(t *testing.T, responses []watchResponse) {
func validateRenewable(t *testing.T, memberId string, responses []watchResponse) {
var lastProgressNotifyRevision int64 = 0
for _, resp := range responses {
for _, event := range resp.Events {
if event.Kv.ModRevision <= lastProgressNotifyRevision {
t.Errorf("BROKE: Renewable - watch can renewed using revision in last progress notification; Progress notification guarantees that previous events have been already delivered, eventRevision: %d, progressNotifyRevision: %d", event.Kv.ModRevision, lastProgressNotifyRevision)
t.Errorf("Broke watch guarantee: Renewable - watch can renewed using revision in last progress notification; Progress notification guarantees that previous events have been already delivered, eventRevision: %d, progressNotifyRevision: %d, member: %q", event.Kv.ModRevision, lastProgressNotifyRevision, memberId)
}
}
if resp.IsProgressNotify() {
@ -179,19 +179,23 @@ func validateRenewable(t *testing.T, responses []watchResponse) {
}
}
func validateOrderedAndReliable(t *testing.T, responses []watchResponse) {
func validateOrderedAndReliable(t *testing.T, memberId string, responses []watchResponse) {
var lastEventRevision int64 = 1
for _, resp := range responses {
for _, event := range resp.Events {
if event.Kv.ModRevision != lastEventRevision && event.Kv.ModRevision != lastEventRevision+1 {
t.Errorf("BROKE: 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, lastRevision: %d, currentRevision: %d", lastEventRevision, event.Kv.ModRevision)
if event.Kv.ModRevision < lastEventRevision {
t.Errorf("Broke watch guarantee: Ordered - events are ordered by revision; an event will never appear on a watch if it precedes an event in time that has already been posted, lastRevision: %d, currentRevision: %d, member: %q", lastEventRevision, event.Kv.ModRevision, memberId)
} else {
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, lastRevision: %d, currentRevision: %d, member: %q", lastEventRevision, event.Kv.ModRevision, memberId)
}
}
lastEventRevision = event.Kv.ModRevision
}
}
}
func validateUnique(t *testing.T, responses []watchResponse) {
func validateUnique(t *testing.T, memberId string, responses []watchResponse) {
type revisionKey struct {
revision int64
key string
@ -201,19 +205,19 @@ func validateUnique(t *testing.T, responses []watchResponse) {
for _, event := range resp.Events {
rk := revisionKey{key: string(event.Kv.Key), revision: event.Kv.ModRevision}
if _, found := uniqueOperations[rk]; found {
t.Errorf("BROKE: Unique - an event will never appear on a watch twice, key: %q, revision: %d", rk.key, rk.revision)
t.Errorf("Broke watch guarantee: Unique - an event will never appear on a watch twice, key: %q, revision: %d, member: %q", rk.key, rk.revision, memberId)
}
uniqueOperations[rk] = struct{}{}
}
}
}
func validateAtomic(t *testing.T, responses []watchResponse) {
func validateAtomic(t *testing.T, memberId string, responses []watchResponse) {
var lastEventRevision int64 = 1
for _, resp := range responses {
if len(resp.Events) > 0 {
if resp.Events[0].Kv.ModRevision == lastEventRevision {
t.Errorf("BROKE: Atomic - a list of events is guaranteed to encompass complete revisions; updates in the same revision over multiple keys will not be split over several lists of events, previousListEventRevision: %d, currentListEventRevision: %d", lastEventRevision, resp.Events[0].Kv.ModRevision)
t.Errorf("Broke watch guarantee: Atomic - a list of events is guaranteed to encompass complete revisions; updates in the same revision over multiple keys will not be split over several lists of events, previousListEventRevision: %d, currentListEventRevision: %d, member: %q", lastEventRevision, resp.Events[0].Kv.ModRevision, memberId)
}
lastEventRevision = resp.Events[len(resp.Events)-1].Kv.ModRevision
}