Skip to content

Commit 8f5cd02

Browse files
committed
storage: introduce metrics for intent resolutions
In particular, this tracks poisoning aborts, which create abort span entries (see #25233). Some refactoring to make it easier to add such counters in the future was carried out. Release note: None
1 parent d556fa9 commit 8f5cd02

File tree

11 files changed

+267
-98
lines changed

11 files changed

+267
-98
lines changed

pkg/storage/batcheval/cmd_lease.go

Lines changed: 6 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -37,11 +37,11 @@ func declareKeysRequestLease(
3737

3838
func newFailedLeaseTrigger(isTransfer bool) result.Result {
3939
var trigger result.Result
40-
trigger.Local.LeaseMetricsResult = new(result.LeaseMetricsType)
40+
trigger.Local.Metrics = new(result.Metrics)
4141
if isTransfer {
42-
*trigger.Local.LeaseMetricsResult = result.LeaseTransferError
42+
trigger.Local.Metrics.LeaseTransferError = 1
4343
} else {
44-
*trigger.Local.LeaseMetricsResult = result.LeaseRequestError
44+
trigger.Local.Metrics.LeaseRequestError = 1
4545
}
4646
return trigger
4747
}
@@ -157,11 +157,11 @@ func evalNewLease(
157157
// the merge aborts.)
158158
pd.Local.MaybeWatchForMerge = true
159159

160-
pd.Local.LeaseMetricsResult = new(result.LeaseMetricsType)
160+
pd.Local.Metrics = new(result.Metrics)
161161
if isTransfer {
162-
*pd.Local.LeaseMetricsResult = result.LeaseTransferSuccess
162+
pd.Local.Metrics.LeaseTransferSuccess = 1
163163
} else {
164-
*pd.Local.LeaseMetricsResult = result.LeaseRequestSuccess
164+
pd.Local.Metrics.LeaseRequestSuccess = 1
165165
}
166166
return pd, nil
167167
}

pkg/storage/batcheval/cmd_resolve_intent.go

Lines changed: 22 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -54,6 +54,20 @@ func declareKeysResolveIntent(
5454
declareKeysResolveIntentCombined(desc, header, req, spans)
5555
}
5656

57+
func resolveToMetricType(status roachpb.TransactionStatus, poison bool) *result.Metrics {
58+
var typ result.Metrics
59+
if WriteAbortSpanOnResolve(status) {
60+
if poison {
61+
typ.ResolvePoison = 1
62+
} else {
63+
typ.ResolveAbort = 1
64+
}
65+
} else {
66+
typ.ResolveCommit = 1
67+
}
68+
return &typ
69+
}
70+
5771
// ResolveIntent resolves a write intent from the specified key
5872
// according to the status of the transaction which created it.
5973
func ResolveIntent(
@@ -75,8 +89,14 @@ func ResolveIntent(
7589
if err := engine.MVCCResolveWriteIntent(ctx, batch, ms, intent); err != nil {
7690
return result.Result{}, err
7791
}
92+
93+
var res result.Result
94+
res.Local.Metrics = resolveToMetricType(args.Status, args.Poison)
95+
7896
if WriteAbortSpanOnResolve(args.Status) {
79-
return result.Result{}, SetAbortSpan(ctx, cArgs.EvalCtx, batch, ms, args.IntentTxn, args.Poison)
97+
if err := SetAbortSpan(ctx, cArgs.EvalCtx, batch, ms, args.IntentTxn, args.Poison); err != nil {
98+
return result.Result{}, err
99+
}
80100
}
81-
return result.Result{}, nil
101+
return res, nil
82102
}

pkg/storage/batcheval/cmd_resolve_intent_range.go

Lines changed: 8 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -79,8 +79,14 @@ func ResolveIntentRange(
7979
reply.ResumeSpan = resumeSpan
8080
reply.ResumeReason = roachpb.RESUME_KEY_LIMIT
8181
}
82+
83+
var res result.Result
84+
res.Local.Metrics = resolveToMetricType(args.Status, args.Poison)
85+
8286
if WriteAbortSpanOnResolve(args.Status) {
83-
return result.Result{}, SetAbortSpan(ctx, cArgs.EvalCtx, batch, ms, args.IntentTxn, args.Poison)
87+
if err := SetAbortSpan(ctx, cArgs.EvalCtx, batch, ms, args.IntentTxn, args.Poison); err != nil {
88+
return result.Result{}, err
89+
}
8490
}
85-
return result.Result{}, nil
91+
return res, nil
8692
}

pkg/storage/batcheval/result/lease_metrics.go

Lines changed: 0 additions & 27 deletions
This file was deleted.
Lines changed: 38 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,38 @@
1+
// Copyright 2014 The Cockroach Authors.
2+
//
3+
// Licensed under the Apache License, Version 2.0 (the "License");
4+
// you may not use this file except in compliance with the License.
5+
// You may obtain a copy of the License at
6+
//
7+
// http://www.apache.org/licenses/LICENSE-2.0
8+
//
9+
// Unless required by applicable law or agreed to in writing, software
10+
// distributed under the License is distributed on an "AS IS" BASIS,
11+
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
12+
// implied. See the License for the specific language governing
13+
// permissions and limitations under the License.
14+
15+
package result
16+
17+
// Metrics tracks various counters related to command applications and
18+
// their outcomes.
19+
type Metrics struct {
20+
LeaseRequestSuccess int // lease request evaluated successfully
21+
LeaseRequestError int // lease request error at evaluation time
22+
LeaseTransferSuccess int // lease transfer evaluated successfully
23+
LeaseTransferError int // lease transfer error at evaluation time
24+
ResolveCommit int // intent commit evaluated successfully
25+
ResolveAbort int // non-poisoning intent abort evaluated successfully
26+
ResolvePoison int // poisoning intent abort evaluated successfully
27+
}
28+
29+
// Add absorbs the supplied Metrics into the receiver.
30+
func (mt *Metrics) Add(o Metrics) {
31+
mt.LeaseRequestSuccess += o.LeaseRequestSuccess
32+
mt.LeaseRequestError += o.LeaseRequestError
33+
mt.LeaseTransferSuccess += o.LeaseTransferSuccess
34+
mt.LeaseTransferError += o.LeaseTransferError
35+
mt.ResolveCommit += o.ResolveCommit
36+
mt.ResolveAbort += o.ResolveAbort
37+
mt.ResolvePoison += o.ResolvePoison
38+
}

pkg/storage/batcheval/result/result.go

Lines changed: 8 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -48,14 +48,9 @@ type LocalResult struct {
4848
// commit fails, or we may accidentally make uncommitted values
4949
// live.
5050
EndTxns *[]EndTxnIntents
51-
// Whether we successfully or non-successfully requested a lease.
52-
//
53-
// TODO(tschottdorf): Update this counter correctly with prop-eval'ed KV
54-
// in the following case:
55-
// - proposal does not fail fast and goes through Raft
56-
// - downstream-of-Raft logic identifies a conflict and returns an error
57-
// The downstream-of-Raft logic does not exist at time of writing.
58-
LeaseMetricsResult *LeaseMetricsType
51+
// Metrics contains counters which are to be passed to the
52+
// metrics subsystem.
53+
Metrics *Metrics
5954

6055
// When set (in which case we better be the first range), call
6156
// GossipFirstRange if the Replica holds the lease.
@@ -309,12 +304,12 @@ func (p *Result) MergeAndDestroy(q Result) error {
309304
}
310305
q.Local.EndTxns = nil
311306

312-
if p.Local.LeaseMetricsResult == nil {
313-
p.Local.LeaseMetricsResult = q.Local.LeaseMetricsResult
314-
} else if q.Local.LeaseMetricsResult != nil {
315-
return errors.New("conflicting LeaseMetricsResult")
307+
if p.Local.Metrics == nil {
308+
p.Local.Metrics = q.Local.Metrics
309+
} else if q.Local.Metrics != nil {
310+
p.Local.Metrics.Add(*q.Local.Metrics)
316311
}
317-
q.Local.LeaseMetricsResult = nil
312+
q.Local.Metrics = nil
318313

319314
if p.Local.MaybeGossipNodeLiveness == nil {
320315
p.Local.MaybeGossipNodeLiveness = q.Local.MaybeGossipNodeLiveness

pkg/storage/batcheval/result/result_test.go

Lines changed: 23 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -65,3 +65,26 @@ func TestEvalResultIsZero(t *testing.T) {
6565
}
6666
}
6767
}
68+
69+
func TestMergeAndDestroy(t *testing.T) {
70+
var r0, r1, r2 Result
71+
r1.Local.Metrics = new(Metrics)
72+
r2.Local.Metrics = new(Metrics)
73+
74+
r1.Local.Metrics.LeaseRequestSuccess = 7
75+
76+
r2.Local.Metrics.ResolveAbort = 13
77+
r2.Local.Metrics.LeaseRequestSuccess = 2
78+
79+
if err := r0.MergeAndDestroy(r1); err != nil {
80+
t.Fatal(err)
81+
}
82+
83+
if err := r0.MergeAndDestroy(r2); err != nil {
84+
t.Fatal(err)
85+
}
86+
87+
if f, exp := *r1.Local.Metrics, (Metrics{LeaseRequestSuccess: 9, ResolveAbort: 13}); f != exp {
88+
t.Fatalf("expected %d, got %d", exp, f)
89+
}
90+
}

pkg/storage/client_metrics_test.go

Lines changed: 82 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -23,9 +23,13 @@ import (
2323
"github.com/pkg/errors"
2424

2525
"github.com/cockroachdb/cockroach/pkg/internal/client"
26+
"github.com/cockroachdb/cockroach/pkg/keys"
2627
"github.com/cockroachdb/cockroach/pkg/roachpb"
2728
"github.com/cockroachdb/cockroach/pkg/storage"
29+
"github.com/cockroachdb/cockroach/pkg/storage/batcheval/result"
30+
"github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb"
2831
"github.com/cockroachdb/cockroach/pkg/testutils"
32+
"github.com/cockroachdb/cockroach/pkg/util/hlc"
2933
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
3034
"github.com/cockroachdb/cockroach/pkg/util/metric"
3135
)
@@ -152,6 +156,84 @@ func verifyRocksDBStats(t *testing.T, s *storage.Store) {
152156
}
153157
}
154158

159+
// TestStoreResolveMetrics verifies that metrics related to intent resolution
160+
// are tracked properly.
161+
func TestStoreResolveMetrics(t *testing.T) {
162+
defer leaktest.AfterTest(t)()
163+
164+
// First prevent rot that would result from adding fields without handling
165+
// them everywhere.
166+
{
167+
act := fmt.Sprintf("%+v", result.Metrics{})
168+
exp := "{LeaseRequestSuccess:0 LeaseRequestError:0 LeaseTransferSuccess:0 LeaseTransferError:0 ResolveCommit:0 ResolveAbort:0 ResolvePoison:0}"
169+
if act != exp {
170+
t.Errorf("need to update this test due to added fields: %v", act)
171+
}
172+
}
173+
174+
mtc := &multiTestContext{}
175+
defer mtc.Stop()
176+
mtc.Start(t, 1)
177+
178+
ctx := context.Background()
179+
180+
span := roachpb.Span{Key: roachpb.Key("a"), EndKey: roachpb.Key("b")}
181+
182+
txn := roachpb.MakeTransaction("foo", span.Key, roachpb.MinUserPriority, enginepb.SERIALIZABLE, hlc.Timestamp{WallTime: 123}, 999)
183+
184+
const resolveCommitCount = int64(200)
185+
const resolveAbortCount = int64(800)
186+
const resolvePoisonCount = int64(2400)
187+
188+
var ba roachpb.BatchRequest
189+
{
190+
repl := mtc.stores[0].LookupReplica(keys.MustAddr(span.Key))
191+
var err error
192+
if ba.Replica, err = repl.GetReplicaDescriptor(); err != nil {
193+
t.Fatal(err)
194+
}
195+
ba.RangeID = repl.RangeID
196+
}
197+
198+
add := func(status roachpb.TransactionStatus, poison bool, n int64) {
199+
for i := int64(0); i < n; i++ {
200+
key := span.Key
201+
endKey := span.EndKey
202+
if i > n/2 {
203+
req := &roachpb.ResolveIntentRangeRequest{
204+
IntentTxn: txn.TxnMeta, Status: status, Poison: poison,
205+
}
206+
req.Key, req.EndKey = key, endKey
207+
ba.Add(req)
208+
continue
209+
}
210+
req := &roachpb.ResolveIntentRequest{
211+
IntentTxn: txn.TxnMeta, Status: status, Poison: poison,
212+
}
213+
req.Key = key
214+
ba.Add(req)
215+
}
216+
}
217+
218+
add(roachpb.COMMITTED, false, resolveCommitCount)
219+
add(roachpb.ABORTED, false, resolveAbortCount)
220+
add(roachpb.ABORTED, true, resolvePoisonCount)
221+
222+
if _, pErr := mtc.senders[0].Send(ctx, ba); pErr != nil {
223+
t.Fatal(pErr)
224+
}
225+
226+
if exp, act := resolveCommitCount, mtc.stores[0].Metrics().ResolveCommitCount.Count(); act < exp || act > exp+50 {
227+
t.Errorf("expected around %d intent commits, saw %d", exp, act)
228+
}
229+
if exp, act := resolveAbortCount, mtc.stores[0].Metrics().ResolveAbortCount.Count(); act < exp || act > exp+50 {
230+
t.Errorf("expected around %d intent aborts, saw %d", exp, act)
231+
}
232+
if exp, act := resolvePoisonCount, mtc.stores[0].Metrics().ResolvePoisonCount.Count(); act < exp || act > exp+50 {
233+
t.Errorf("expected arounc %d abort span poisonings, saw %d", exp, act)
234+
}
235+
}
236+
155237
func TestStoreMetrics(t *testing.T) {
156238
defer leaktest.AfterTest(t)()
157239

0 commit comments

Comments
 (0)