Skip to content

Commit 267a621

Browse files
authored
Merge pull request #9940 from wenjiaswe/automated-cherry-pick-of-#9761-upstream-release-3.3
Automated cherry pick of #9761
2 parents ffe52f7 + 143fc4c commit 267a621

File tree

8 files changed

+85
-9
lines changed

8 files changed

+85
-9
lines changed

etcdserver/metrics.go

+14
Original file line numberDiff line numberDiff line change
@@ -41,6 +41,18 @@ var (
4141
Name: "leader_changes_seen_total",
4242
Help: "The number of leader changes seen.",
4343
})
44+
heartbeatSendFailures = prometheus.NewCounter(prometheus.CounterOpts{
45+
Namespace: "etcd",
46+
Subsystem: "server",
47+
Name: "heartbeat_send_failures_total",
48+
Help: "The total number of leader heartbeat send failures (likely overloaded from slow disk).",
49+
})
50+
slowApplies = prometheus.NewCounter(prometheus.CounterOpts{
51+
Namespace: "etcd",
52+
Subsystem: "server",
53+
Name: "slow_apply_total",
54+
Help: "The total number of slow apply requests (likely overloaded from slow disk).",
55+
})
4456
proposalsCommitted = prometheus.NewGauge(prometheus.GaugeOpts{
4557
Namespace: "etcd",
4658
Subsystem: "server",
@@ -96,6 +108,8 @@ func init() {
96108
prometheus.MustRegister(hasLeader)
97109
prometheus.MustRegister(isLeader)
98110
prometheus.MustRegister(leaderChanges)
111+
prometheus.MustRegister(heartbeatSendFailures)
112+
prometheus.MustRegister(slowApplies)
99113
prometheus.MustRegister(proposalsCommitted)
100114
prometheus.MustRegister(proposalsApplied)
101115
prometheus.MustRegister(proposalsPending)

etcdserver/raft.go

+1
Original file line numberDiff line numberDiff line change
@@ -346,6 +346,7 @@ func (r *raftNode) processMessages(ms []raftpb.Message) []raftpb.Message {
346346
// TODO: limit request rate.
347347
plog.Warningf("failed to send out heartbeat on time (exceeded the %v timeout for %v)", r.heartbeat, exceed)
348348
plog.Warningf("server is likely overloaded")
349+
heartbeatSendFailures.Inc()
349350
}
350351
}
351352
}

etcdserver/util.go

+1
Original file line numberDiff line numberDiff line change
@@ -146,6 +146,7 @@ func warnOfExpensiveGenericRequest(now time.Time, reqStringer fmt.Stringer, pref
146146
result = resp
147147
}
148148
plog.Warningf("%srequest %q with result %q took too long (%v) to execute", prefix, reqStringer.String(), result, d)
149+
slowApplies.Inc()
149150
}
150151
}
151152

mvcc/backend/backend.go

+5
Original file line numberDiff line numberDiff line change
@@ -291,6 +291,8 @@ func (b *backend) Defrag() error {
291291
}
292292

293293
func (b *backend) defrag() error {
294+
now := time.Now()
295+
294296
// TODO: make this non-blocking?
295297
// lock batchTx to ensure nobody is using previous tx, and then
296298
// close previous ongoing tx.
@@ -354,6 +356,9 @@ func (b *backend) defrag() error {
354356
atomic.StoreInt64(&b.size, size)
355357
atomic.StoreInt64(&b.sizeInUse, size-(int64(db.Stats().FreePageN)*int64(db.Info().PageSize)))
356358

359+
took := time.Since(now)
360+
defragDurations.Observe(took.Seconds())
361+
357362
return nil
358363
}
359364

mvcc/backend/batch_tx.go

+7-5
Original file line numberDiff line numberDiff line change
@@ -136,15 +136,15 @@ func unsafeForEach(tx *bolt.Tx, bucket []byte, visitor func(k, v []byte) error)
136136
// Commit commits a previous tx and begins a new writable one.
137137
func (t *batchTx) Commit() {
138138
t.Lock()
139-
defer t.Unlock()
140139
t.commit(false)
140+
t.Unlock()
141141
}
142142

143143
// CommitAndStop commits the previous tx and does not create a new one.
144144
func (t *batchTx) CommitAndStop() {
145145
t.Lock()
146-
defer t.Unlock()
147146
t.commit(true)
147+
t.Unlock()
148148
}
149149

150150
func (t *batchTx) Unlock() {
@@ -162,9 +162,11 @@ func (t *batchTx) commit(stop bool) {
162162
}
163163

164164
start := time.Now()
165+
165166
// gofail: var beforeCommit struct{}
166167
err := t.tx.Commit()
167168
// gofail: var afterCommit struct{}
169+
168170
commitDurations.Observe(time.Since(start).Seconds())
169171
atomic.AddInt64(&t.backend.commits, 1)
170172

@@ -209,21 +211,21 @@ func (t *batchTxBuffered) Unlock() {
209211

210212
func (t *batchTxBuffered) Commit() {
211213
t.Lock()
212-
defer t.Unlock()
213214
t.commit(false)
215+
t.Unlock()
214216
}
215217

216218
func (t *batchTxBuffered) CommitAndStop() {
217219
t.Lock()
218-
defer t.Unlock()
219220
t.commit(true)
221+
t.Unlock()
220222
}
221223

222224
func (t *batchTxBuffered) commit(stop bool) {
223225
// all read txs must be closed to acquire boltdb commit rwlock
224226
t.backend.readTx.mu.Lock()
225-
defer t.backend.readTx.mu.Unlock()
226227
t.unsafeCommit(stop)
228+
t.backend.readTx.mu.Unlock()
227229
}
228230

229231
func (t *batchTxBuffered) unsafeCommit(stop bool) {

mvcc/backend/metrics.go

+20-2
Original file line numberDiff line numberDiff line change
@@ -22,20 +22,38 @@ var (
2222
Subsystem: "disk",
2323
Name: "backend_commit_duration_seconds",
2424
Help: "The latency distributions of commit called by backend.",
25-
Buckets: prometheus.ExponentialBuckets(0.001, 2, 14),
25+
26+
// lowest bucket start of upper bound 0.001 sec (1 ms) with factor 2
27+
// highest bucket start of 0.001 sec * 2^13 == 8.192 sec
28+
Buckets: prometheus.ExponentialBuckets(0.001, 2, 14),
29+
})
30+
31+
defragDurations = prometheus.NewHistogram(prometheus.HistogramOpts{
32+
Namespace: "etcd",
33+
Subsystem: "disk",
34+
Name: "backend_defrag_duration_seconds",
35+
Help: "The latency distribution of backend defragmentation.",
36+
37+
// 100 MB usually takes 1 sec, so start with 10 MB of 100 ms
38+
// lowest bucket start of upper bound 0.1 sec (100 ms) with factor 2
39+
// highest bucket start of 0.1 sec * 2^12 == 409.6 sec
40+
Buckets: prometheus.ExponentialBuckets(.1, 2, 13),
2641
})
2742

2843
snapshotDurations = prometheus.NewHistogram(prometheus.HistogramOpts{
2944
Namespace: "etcd",
3045
Subsystem: "disk",
3146
Name: "backend_snapshot_duration_seconds",
3247
Help: "The latency distribution of backend snapshots.",
33-
// 10 ms -> 655 seconds
48+
49+
// lowest bucket start of upper bound 0.01 sec (10 ms) with factor 2
50+
// highest bucket start of 0.01 sec * 2^16 == 655.36 sec
3451
Buckets: prometheus.ExponentialBuckets(.01, 2, 17),
3552
})
3653
)
3754

3855
func init() {
3956
prometheus.MustRegister(commitDurations)
57+
prometheus.MustRegister(defragDurations)
4058
prometheus.MustRegister(snapshotDurations)
4159
}

mvcc/kvstore.go

+10-1
Original file line numberDiff line numberDiff line change
@@ -156,12 +156,18 @@ func (s *store) compactBarrier(ctx context.Context, ch chan struct{}) {
156156
}
157157

158158
func (s *store) Hash() (hash uint32, revision int64, err error) {
159+
start := time.Now()
160+
159161
s.b.ForceCommit()
160162
h, err := s.b.Hash(DefaultIgnores)
163+
164+
hashDurations.Observe(time.Since(start).Seconds())
161165
return h, s.currentRev, err
162166
}
163167

164168
func (s *store) HashByRev(rev int64) (hash uint32, currentRev int64, compactRev int64, err error) {
169+
start := time.Now()
170+
165171
s.mu.RLock()
166172
s.revMu.RLock()
167173
compactRev, currentRev = s.compactMainRev, s.currentRev
@@ -206,7 +212,10 @@ func (s *store) HashByRev(rev int64) (hash uint32, currentRev int64, compactRev
206212
h.Write(v)
207213
return nil
208214
})
209-
return h.Sum32(), currentRev, compactRev, err
215+
hash = h.Sum32()
216+
217+
hashRevDurations.Observe(time.Since(start).Seconds())
218+
return hash, currentRev, compactRev, err
210219
}
211220

212221
func (s *store) Compact(rev int64) (<-chan struct{}, error) {

mvcc/metrics.go

+27-1
Original file line numberDiff line numberDiff line change
@@ -181,7 +181,31 @@ var (
181181
)
182182
// overridden by mvcc initialization
183183
reportDbTotalSizeInUseInBytesMu sync.RWMutex
184-
reportDbTotalSizeInUseInBytes = func() float64 { return 0 }
184+
reportDbTotalSizeInUseInBytes func() float64 = func() float64 { return 0 }
185+
186+
hashDurations = prometheus.NewHistogram(prometheus.HistogramOpts{
187+
Namespace: "etcd",
188+
Subsystem: "mvcc",
189+
Name: "hash_duration_seconds",
190+
Help: "The latency distribution of storage hash operation.",
191+
192+
// 100 MB usually takes 100 ms, so start with 10 MB of 10 ms
193+
// lowest bucket start of upper bound 0.01 sec (10 ms) with factor 2
194+
// highest bucket start of 0.01 sec * 2^14 == 163.84 sec
195+
Buckets: prometheus.ExponentialBuckets(.01, 2, 15),
196+
})
197+
198+
hashRevDurations = prometheus.NewHistogram(prometheus.HistogramOpts{
199+
Namespace: "etcd",
200+
Subsystem: "mvcc",
201+
Name: "hash_rev_duration_seconds",
202+
Help: "The latency distribution of storage hash by revision operation.",
203+
204+
// 100 MB usually takes 100 ms, so start with 10 MB of 10 ms
205+
// lowest bucket start of upper bound 0.01 sec (10 ms) with factor 2
206+
// highest bucket start of 0.01 sec * 2^14 == 163.84 sec
207+
Buckets: prometheus.ExponentialBuckets(.01, 2, 15),
208+
})
185209
)
186210

187211
func init() {
@@ -202,6 +226,8 @@ func init() {
202226
prometheus.MustRegister(dbTotalSizeDebugging)
203227
prometheus.MustRegister(dbTotalSize)
204228
prometheus.MustRegister(dbTotalSizeInUse)
229+
prometheus.MustRegister(hashDurations)
230+
prometheus.MustRegister(hashRevDurations)
205231
}
206232

207233
// ReportEventReceived reports that an event is received.

0 commit comments

Comments
 (0)