-
Notifications
You must be signed in to change notification settings - Fork 3.8k
/
liveness.go
1569 lines (1438 loc) · 57.4 KB
/
liveness.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
// Copyright 2020 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
package liveness
import (
"bytes"
"context"
"fmt"
"strconv"
"sync/atomic"
"time"
"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/gossip"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/util/contextutil"
"github.com/cockroachdb/cockroach/pkg/util/grpcutil"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/metric"
"github.com/cockroachdb/cockroach/pkg/util/retry"
"github.com/cockroachdb/cockroach/pkg/util/stop"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/cockroachdb/cockroach/pkg/util/syncutil/singleflight"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/redact"
)
var (
// ErrMissingRecord is returned when asking for liveness information
// about a node for which nothing is known. This happens when attempting to
// {d,r}ecommission a non-existent node.
ErrMissingRecord = errors.New("missing liveness record")
// ErrRecordCacheMiss is returned when asking for the liveness
// record of a given node and it is not found in the in-memory cache.
ErrRecordCacheMiss = errors.New("liveness record not found in cache")
// errChangeMembershipStatusFailed is returned when we're not able to
// conditionally write the target membership status. It's safe to retry
// when encountering this error.
errChangeMembershipStatusFailed = errors.New("failed to change the membership status")
// ErrEpochIncremented is returned when a heartbeat request fails because
// the underlying liveness record has had its epoch incremented.
ErrEpochIncremented = errors.New("heartbeat failed on epoch increment")
// ErrEpochAlreadyIncremented is returned by IncrementEpoch when
// someone else has already incremented the epoch to the desired
// value.
ErrEpochAlreadyIncremented = errors.New("epoch already incremented")
)
type errRetryLiveness struct {
error
}
func (e *errRetryLiveness) Cause() error {
return e.error
}
func (e *errRetryLiveness) Error() string {
return fmt.Sprintf("%T: %s", *e, e.error)
}
func isErrRetryLiveness(ctx context.Context, err error) bool {
if errors.HasType(err, (*roachpb.AmbiguousResultError)(nil)) {
// We generally want to retry ambiguous errors immediately, except if the
// ctx is canceled - in which case the ambiguous error is probably caused
// by the cancellation (and in any case it's pointless to retry with a
// canceled ctx).
return ctx.Err() == nil
} else if errors.HasType(err, (*roachpb.TransactionStatusError)(nil)) {
// 21.2 nodes can return a TransactionStatusError when they should have
// returned an AmbiguousResultError.
// TODO(andrei): Remove this in 22.2.
return true
} else if errors.Is(err, kv.OnePCNotAllowedError{}) {
return true
}
return false
}
// Node liveness metrics counter names.
var (
metaLiveNodes = metric.Metadata{
Name: "liveness.livenodes",
Help: "Number of live nodes in the cluster (will be 0 if this node is not itself live)",
Measurement: "Nodes",
Unit: metric.Unit_COUNT,
}
metaHeartbeatsInFlight = metric.Metadata{
Name: "liveness.heartbeatsinflight",
Help: "Number of in-flight liveness heartbeats from this node",
Measurement: "Requests",
Unit: metric.Unit_COUNT,
}
metaHeartbeatSuccesses = metric.Metadata{
Name: "liveness.heartbeatsuccesses",
Help: "Number of successful node liveness heartbeats from this node",
Measurement: "Messages",
Unit: metric.Unit_COUNT,
}
metaHeartbeatFailures = metric.Metadata{
Name: "liveness.heartbeatfailures",
Help: "Number of failed node liveness heartbeats from this node",
Measurement: "Messages",
Unit: metric.Unit_COUNT,
}
metaEpochIncrements = metric.Metadata{
Name: "liveness.epochincrements",
Help: "Number of times this node has incremented its liveness epoch",
Measurement: "Epochs",
Unit: metric.Unit_COUNT,
}
metaHeartbeatLatency = metric.Metadata{
Name: "liveness.heartbeatlatency",
Help: "Node liveness heartbeat latency",
Measurement: "Latency",
Unit: metric.Unit_NANOSECONDS,
}
)
// Metrics holds metrics for use with node liveness activity.
type Metrics struct {
LiveNodes *metric.Gauge
HeartbeatsInFlight *metric.Gauge
HeartbeatSuccesses *metric.Counter
HeartbeatFailures telemetry.CounterWithMetric
EpochIncrements telemetry.CounterWithMetric
HeartbeatLatency metric.IHistogram
}
// IsLiveCallback is invoked when a node's IsLive state changes to true.
// Callbacks can be registered via NodeLiveness.RegisterCallback().
type IsLiveCallback func(livenesspb.Liveness)
// HeartbeatCallback is invoked whenever this node updates its own liveness status,
// indicating that it is alive.
type HeartbeatCallback func(context.Context)
// NodeLiveness is a centralized failure detector that coordinates
// with the epoch-based range system to provide for leases of
// indefinite length (replacing frequent per-range lease renewals with
// heartbeats to the liveness system).
//
// It is also used as a general-purpose failure detector, but it is
// not ideal for this purpose. It is inefficient due to the use of
// replicated durable writes, and is not very sensitive (it primarily
// tests connectivity from the node to the liveness range; a node with
// a failing disk could still be considered live by this system).
//
// The persistent state of node liveness is stored in the KV layer,
// near the beginning of the keyspace. These are normal MVCC keys,
// written by CPut operations in 1PC transactions (the use of
// transactions and MVCC is regretted because it means that the
// liveness span depends on MVCC GC and can get overwhelmed if GC is
// not working. Transactions were used only to piggyback on the
// transaction commit trigger). The leaseholder of the liveness range
// gossips its contents whenever they change (only the changed
// portion); other nodes rarely read from this range directly.
//
// The use of conditional puts is crucial to maintain the guarantees
// needed by epoch-based leases. Both the Heartbeat and IncrementEpoch
// on this type require an expected value to be passed in; see
// comments on those methods for more.
//
// TODO(bdarnell): Also document interaction with draining and decommissioning.
type NodeLiveness struct {
ambientCtx log.AmbientContext
stopper *stop.Stopper
clock *hlc.Clock
db *kv.DB
gossip *gossip.Gossip
livenessThreshold time.Duration
renewalDuration time.Duration
selfSem chan struct{}
st *cluster.Settings
otherSem chan struct{}
// heartbeatPaused contains an atomically-swapped number representing a bool
// (1 or 0). heartbeatToken is a channel containing a token which is taken
// when heartbeating or when pausing the heartbeat. Used for testing.
heartbeatPaused uint32
heartbeatToken chan struct{}
metrics Metrics
onNodeDecommissioned func(livenesspb.Liveness) // noop if nil
onNodeDecommissioning OnNodeDecommissionCallback // noop if nil
engineSyncs *singleflight.Group
mu struct {
syncutil.RWMutex
onIsLive []IsLiveCallback // see NodeLivenessOptions.OnSelfLive
// nodes is an in-memory cache of liveness records that NodeLiveness
// knows about (having learnt of them through gossip or through KV).
// It's a look-aside cache, and is accessed primarily through
// `getLivenessLocked` and callers.
//
// TODO(irfansharif): The caching story for NodeLiveness is a bit
// complicated. This can be attributed to the fact that pre-20.2, we
// weren't always guaranteed for us liveness records for every given
// node. Because of this it wasn't possible to have a
// look-through cache (it wouldn't know where to fetch from if a record
// was found to be missing).
//
// Now that we're always guaranteed to have a liveness records present,
// we should change this out to be a look-through cache instead (it can
// fall back to KV when a given record is missing). This would help
// simplify our current structure where do the following:
//
// - Consult this cache to find an existing liveness record
// - If missing, fetch the record from KV
// - Update the liveness record in KV
// - Add the updated record into this cache (see `maybeUpdate`)
//
// (See `Start` for an example of this pattern.)
//
// What we want instead is a bit simpler:
//
// - Consult this cache to find an existing liveness record
// - If missing, fetch the record from KV, update and return from cache
// - Update the liveness record in KV
// - Add the updated record into this cache
//
// More concretely, we want `getLivenessRecordFromKV` to be tucked away
// within `getLivenessLocked`.
nodes map[roachpb.NodeID]Record
onSelfLive HeartbeatCallback // set in Start()
// Before heartbeating, we write to each of these engines to avoid
// maintaining liveness when a local disks is stalled.
engines []storage.Engine // set in Start()
}
}
// Record is a liveness record that has been read from the database, together
// with its database encoding. The encoding is useful for CPut-ing an update to
// the liveness record: the raw value will act as the expected value. This way
// the proto's encoding can change without the CPut failing.
type Record struct {
livenesspb.Liveness
// raw represents the raw bytes read from the database - suitable to pass to a
// CPut. Nil if the value doesn't exist in the DB.
raw []byte
}
// NodeLivenessOptions is the input to NewNodeLiveness.
//
// Note that there is yet another struct, NodeLivenessStartOptions, which
// is supplied when the instance is started. This is necessary as during
// server startup, some inputs can only be constructed at Start time. The
// separation has grown organically and various options could in principle
// be moved back and forth.
type NodeLivenessOptions struct {
AmbientCtx log.AmbientContext
Stopper *stop.Stopper
Settings *cluster.Settings
Gossip *gossip.Gossip
Clock *hlc.Clock
DB *kv.DB
LivenessThreshold time.Duration
RenewalDuration time.Duration
HistogramWindowInterval time.Duration
// OnNodeDecommissioned is invoked whenever the instance learns that a
// node was permanently removed from the cluster. This method must be
// idempotent as it may be invoked multiple times and defaults to a
// noop.
OnNodeDecommissioned func(livenesspb.Liveness)
// OnNodeDecommissioning is invoked when a node is detected to be
// decommissioning.
OnNodeDecommissioning OnNodeDecommissionCallback
}
// NewNodeLiveness returns a new instance of NodeLiveness configured
// with the specified gossip instance.
func NewNodeLiveness(opts NodeLivenessOptions) *NodeLiveness {
nl := &NodeLiveness{
ambientCtx: opts.AmbientCtx,
stopper: opts.Stopper,
clock: opts.Clock,
db: opts.DB,
gossip: opts.Gossip,
livenessThreshold: opts.LivenessThreshold,
renewalDuration: opts.RenewalDuration,
selfSem: make(chan struct{}, 1),
st: opts.Settings,
otherSem: make(chan struct{}, 1),
heartbeatToken: make(chan struct{}, 1),
onNodeDecommissioned: opts.OnNodeDecommissioned,
onNodeDecommissioning: opts.OnNodeDecommissioning,
engineSyncs: singleflight.NewGroup("engine sync", "engine"),
}
nl.metrics = Metrics{
LiveNodes: metric.NewFunctionalGauge(metaLiveNodes, nl.numLiveNodes),
HeartbeatsInFlight: metric.NewGauge(metaHeartbeatsInFlight),
HeartbeatSuccesses: metric.NewCounter(metaHeartbeatSuccesses),
HeartbeatFailures: telemetry.NewCounterWithMetric(metaHeartbeatFailures),
EpochIncrements: telemetry.NewCounterWithMetric(metaEpochIncrements),
HeartbeatLatency: metric.NewHistogram(metric.HistogramOptions{
Mode: metric.HistogramModePreferHdrLatency,
Metadata: metaHeartbeatLatency,
Duration: opts.HistogramWindowInterval,
Buckets: metric.NetworkLatencyBuckets,
}),
}
nl.mu.nodes = make(map[roachpb.NodeID]Record)
nl.heartbeatToken <- struct{}{}
// NB: we should consider moving this registration to .Start() once we
// have ensured that nobody uses the server's KV client (kv.DB) before
// nl.Start() is invoked. At the time of writing this invariant does
// not hold (which is a problem, since the node itself won't be live
// at this point, and requests routed to it will hang).
livenessRegex := gossip.MakePrefixPattern(gossip.KeyNodeLivenessPrefix)
nl.gossip.RegisterCallback(livenessRegex, nl.livenessGossipUpdate)
return nl
}
var errNodeDrainingSet = errors.New("node is already draining")
func (nl *NodeLiveness) sem(nodeID roachpb.NodeID) chan struct{} {
if nodeID == nl.gossip.NodeID.Get() {
return nl.selfSem
}
return nl.otherSem
}
// SetDraining attempts to update this node's liveness record to put itself
// into the draining state.
//
// The reporter callback, if non-nil, is called on a best effort basis
// to report work that needed to be done and which may or may not have
// been done by the time this call returns. See the explanation in
// pkg/server/drain.go for details.
func (nl *NodeLiveness) SetDraining(
ctx context.Context, drain bool, reporter func(int, redact.SafeString),
) error {
ctx = nl.ambientCtx.AnnotateCtx(ctx)
for r := retry.StartWithCtx(ctx, base.DefaultRetryOptions()); r.Next(); {
oldLivenessRec, ok := nl.SelfEx()
if !ok {
// There was a cache miss, let's now fetch the record from KV
// directly.
nodeID := nl.gossip.NodeID.Get()
livenessRec, err := nl.getLivenessRecordFromKV(ctx, nodeID)
if err != nil {
return err
}
oldLivenessRec = livenessRec
}
if err := nl.setDrainingInternal(ctx, oldLivenessRec, drain, reporter); err != nil {
if log.V(1) {
log.Infof(ctx, "attempting to set liveness draining status to %v: %v", drain, err)
}
if grpcutil.IsConnectionRejected(err) {
return err
}
continue
}
return nil
}
if err := ctx.Err(); err != nil {
return err
}
return errors.New("failed to drain self")
}
// SetMembershipStatus changes the liveness record to reflect the target
// membership status. It does so idempotently, and may retry internally until it
// observes its target state durably persisted. It returns whether it was able
// to change the membership status (as opposed to it returning early when
// finding the target status possibly set by another node).
func (nl *NodeLiveness) SetMembershipStatus(
ctx context.Context, nodeID roachpb.NodeID, targetStatus livenesspb.MembershipStatus,
) (statusChanged bool, err error) {
ctx = nl.ambientCtx.AnnotateCtx(ctx)
attempt := func() (bool, error) {
// Allow only one decommissioning attempt in flight per node at a time.
// This is required for correct results since we may otherwise race with
// concurrent `IncrementEpoch` calls and get stuck in a situation in
// which the cached liveness is has decommissioning=false while it's
// really true, and that means that SetDecommissioning becomes a no-op
// (which is correct) but that our cached liveness never updates to
// reflect that.
//
// See https://github.com/cockroachdb/cockroach/issues/17995.
sem := nl.sem(nodeID)
select {
case sem <- struct{}{}:
case <-ctx.Done():
return false, ctx.Err()
}
defer func() {
<-sem
}()
// We need the current liveness in each iteration.
//
// We ignore any liveness record in Gossip because we may have to fall back
// to the KV store anyway. The scenario in which this is needed is:
// - kill node 2 and stop node 1
// - wait for node 2's liveness record's Gossip entry to expire on all surviving nodes
// - restart node 1; it'll never see node 2 in `GetLiveness` unless the whole
// node liveness span gets regossiped (unlikely if it wasn't the lease holder
// for that span)
// - can't decommission node 2 from node 1 without KV fallback.
//
// See #20863.
//
// NB: this also de-flakes TestNodeLivenessDecommissionAbsent; running
// decommissioning commands in a tight loop on different nodes sometimes
// results in unintentional no-ops (due to the Gossip lag); this could be
// observed by users in principle, too.
//
// TODO(bdarnell): This is the one place where a range other than
// the leaseholder reads from this range. Should this read from
// gossip instead? (I have vague concerns about concurrent reads
// and timestamp cache pushes causing problems here)
var oldLiveness livenesspb.Liveness
kv, err := nl.db.Get(ctx, keys.NodeLivenessKey(nodeID))
if err != nil {
return false, errors.Wrap(err, "unable to get liveness")
}
if kv.Value == nil {
// We must be trying to decommission a node that does not exist.
return false, ErrMissingRecord
}
if err := kv.Value.GetProto(&oldLiveness); err != nil {
return false, errors.Wrap(err, "invalid liveness record")
}
oldLivenessRec := Record{
Liveness: oldLiveness,
raw: kv.Value.TagAndDataBytes(),
}
// We may have discovered a Liveness not yet received via Gossip.
// Offer it to make sure that when we actually try to update the
// liveness, the previous view is correct. This, too, is required to
// de-flake TestNodeLivenessDecommissionAbsent.
nl.maybeUpdate(ctx, oldLivenessRec)
return nl.setMembershipStatusInternal(ctx, oldLivenessRec, targetStatus)
}
for {
statusChanged, err := attempt()
if errors.Is(err, errChangeMembershipStatusFailed) {
// Expected when epoch incremented, it's safe to retry.
continue
}
return statusChanged, err
}
}
func (nl *NodeLiveness) setDrainingInternal(
ctx context.Context, oldLivenessRec Record, drain bool, reporter func(int, redact.SafeString),
) error {
nodeID := nl.gossip.NodeID.Get()
sem := nl.sem(nodeID)
// Allow only one attempt to set the draining field at a time.
select {
case sem <- struct{}{}:
case <-ctx.Done():
return ctx.Err()
}
defer func() {
<-sem
}()
if oldLivenessRec.Liveness == (livenesspb.Liveness{}) {
return errors.AssertionFailedf("invalid old liveness record; found to be empty")
}
// Let's compute what our new liveness record should be. We start off with a
// copy of our existing liveness record.
newLiveness := oldLivenessRec.Liveness
if reporter != nil && drain && !newLiveness.Draining {
// Report progress to the Drain RPC.
reporter(1, "liveness record")
}
newLiveness.Draining = drain
update := livenessUpdate{
oldLiveness: oldLivenessRec.Liveness,
newLiveness: newLiveness,
oldRaw: oldLivenessRec.raw,
ignoreCache: true,
}
written, err := nl.updateLiveness(ctx, update, func(actual Record) error {
nl.maybeUpdate(ctx, actual)
if actual.Draining == update.newLiveness.Draining {
return errNodeDrainingSet
}
return errors.New("failed to update liveness record because record has changed")
})
if err != nil {
if log.V(1) {
log.Infof(ctx, "updating liveness record: %v", err)
}
if errors.Is(err, errNodeDrainingSet) {
return nil
}
return err
}
nl.maybeUpdate(ctx, written)
return nil
}
// livenessUpdate contains the information for CPutting a new version of a
// liveness record. It has both the new and the old version of the proto.
type livenessUpdate struct {
newLiveness livenesspb.Liveness
oldLiveness livenesspb.Liveness
// When ignoreCache is set, we won't assume that our in-memory cached version
// of the liveness record is accurate and will use a CPut on the liveness
// table with the old value supplied by the client (oldRaw). This is used for
// operations that don't want to deal with the inconsistencies of using the
// cache.
//
// When ignoreCache is not set, the state of the cache is checked against old and,
// if they don't correspond, the CPut is considered to have failed.
//
// When ignoreCache is set, oldRaw needs to be set as well.
ignoreCache bool
// oldRaw is the raw value from which `old` was decoded. Used for CPuts as the
// existing value. Note that we don't simply marshal `old` as that would break
// if unmarshalling/marshaling doesn't round-trip. Nil means that a liveness
// record for the respected node is not expected to exist in the database.
//
// oldRaw must not be set when ignoreCache is not set.
oldRaw []byte
}
// CreateLivenessRecord creates a liveness record for the node specified by the
// given node ID. This is typically used when adding a new node to a running
// cluster, or when bootstrapping a cluster through a given node.
//
// This is a pared down version of Start; it exists only to durably
// persist a liveness to record the node's existence. Nodes will heartbeat their
// records after starting up, and incrementing to epoch=1 when doing so, at
// which point we'll set an appropriate expiration timestamp, gossip the
// liveness record, and update our in-memory representation of it.
//
// NB: An existing liveness record is not overwritten by this method, we return
// an error instead.
func (nl *NodeLiveness) CreateLivenessRecord(ctx context.Context, nodeID roachpb.NodeID) error {
for r := retry.StartWithCtx(ctx, base.DefaultRetryOptions()); r.Next(); {
// We start off at epoch=0, entrusting the initial heartbeat to increment it
// to epoch=1 to signal the very first time the node is up and running.
liveness := livenesspb.Liveness{NodeID: nodeID, Epoch: 0}
// We skip adding an expiration, we only really care about the liveness
// record existing within KV.
v := new(roachpb.Value)
err := nl.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error {
b := txn.NewBatch()
key := keys.NodeLivenessKey(nodeID)
if err := v.SetProto(&liveness); err != nil {
log.Fatalf(ctx, "failed to marshall proto: %s", err)
}
// Given we're looking to create a new liveness record here, we don't
// expect to find anything.
b.CPut(key, v, nil)
// We don't bother adding a gossip trigger, that'll happen with the
// first heartbeat. We still keep it as a 1PC commit to avoid leaving
// write intents.
b.AddRawRequest(&roachpb.EndTxnRequest{
Commit: true,
Require1PC: true,
})
return txn.Run(ctx, b)
})
if err == nil {
// We'll learn about this liveness record through gossip eventually, so we
// don't bother updating our in-memory view of node liveness.
log.Infof(ctx, "created liveness record for n%d", nodeID)
return nil
}
if !isErrRetryLiveness(ctx, err) {
return err
}
log.VEventf(ctx, 2, "failed to create liveness record for node %d, because of %s. retrying...", nodeID, err)
}
if err := ctx.Err(); err != nil {
return err
}
return errors.AssertionFailedf("unexpected problem while creating liveness record for node %d", nodeID)
}
func (nl *NodeLiveness) setMembershipStatusInternal(
ctx context.Context, oldLivenessRec Record, targetStatus livenesspb.MembershipStatus,
) (statusChanged bool, err error) {
if oldLivenessRec.Liveness == (livenesspb.Liveness{}) {
return false, errors.AssertionFailedf("invalid old liveness record; found to be empty")
}
// Let's compute what our new liveness record should be. We start off with a
// copy of our existing liveness record.
newLiveness := oldLivenessRec.Liveness
newLiveness.Membership = targetStatus
if oldLivenessRec.Membership == newLiveness.Membership {
// No-op. Return early.
return false, nil
} else if oldLivenessRec.Membership.Decommissioned() &&
newLiveness.Membership.Decommissioning() {
// Marking a decommissioned node for decommissioning is a no-op. We
// return early.
return false, nil
}
if err := livenesspb.ValidateTransition(oldLivenessRec.Liveness, newLiveness); err != nil {
return false, err
}
update := livenessUpdate{
newLiveness: newLiveness,
oldLiveness: oldLivenessRec.Liveness,
oldRaw: oldLivenessRec.raw,
ignoreCache: true,
}
statusChanged = true
if _, err := nl.updateLiveness(ctx, update, func(actual Record) error {
if actual.Membership != update.newLiveness.Membership {
// We're racing with another attempt at updating the liveness
// record, we error out in order to retry.
return errChangeMembershipStatusFailed
}
// The found liveness membership status is the same as the target one,
// so we consider our work done. We inform the caller that this attempt
// was a no-op.
statusChanged = false
return nil
}); err != nil {
return false, err
}
return statusChanged, nil
}
// GetLivenessThreshold returns the maximum duration between heartbeats
// before a node is considered not-live.
func (nl *NodeLiveness) GetLivenessThreshold() time.Duration {
return nl.livenessThreshold
}
// IsLive returns whether or not the specified node is considered live based on
// whether or not its liveness has expired regardless of the liveness status. It
// is an error if the specified node is not in the local liveness table.
func (nl *NodeLiveness) IsLive(nodeID roachpb.NodeID) (bool, error) {
liveness, ok := nl.GetLiveness(nodeID)
if !ok {
// TODO(irfansharif): We only expect callers to supply us with node IDs
// they learnt through existing liveness records, which implies we
// should never find ourselves here. We should clean up this conditional
// once we re-visit the caching structure used within NodeLiveness;
// we should be able to return ErrMissingRecord instead.
return false, ErrRecordCacheMiss
}
// NB: We use clock.Now().GoTime() instead of clock.PhysicalTime() in order to
// consider clock signals from other nodes.
return liveness.IsLive(nl.clock.Now().GoTime()), nil
}
// IsAvailable returns whether or not the specified node is available to serve
// requests. It checks both the liveness and decommissioned states, but not
// draining or decommissioning (since it may still be a leaseholder for ranges).
// Returns false if the node is not in the local liveness table.
func (nl *NodeLiveness) IsAvailable(nodeID roachpb.NodeID) bool {
liveness, ok := nl.GetLiveness(nodeID)
return ok && liveness.IsLive(nl.clock.Now().GoTime()) && !liveness.Membership.Decommissioned()
}
// IsAvailableNotDraining returns whether or not the specified node is available
// to serve requests (i.e. it is live and not decommissioned) and is not in the
// process of draining/decommissioning. Note that draining/decommissioning nodes
// could still be leaseholders for ranges until drained, so this should not be
// used when the caller needs to be able to contact leaseholders directly.
// Returns false if the node is not in the local liveness table.
func (nl *NodeLiveness) IsAvailableNotDraining(nodeID roachpb.NodeID) bool {
liveness, ok := nl.GetLiveness(nodeID)
return ok &&
liveness.IsLive(nl.clock.Now().GoTime()) &&
!liveness.Membership.Decommissioning() &&
!liveness.Membership.Decommissioned() &&
!liveness.Draining
}
// OnNodeDecommissionCallback is a callback that is invoked when a node is
// detected to be decommissioning.
type OnNodeDecommissionCallback func(nodeID roachpb.NodeID)
// NodeLivenessStartOptions are the arguments to `NodeLiveness.Start`.
type NodeLivenessStartOptions struct {
Engines []storage.Engine
// OnSelfLive is invoked after every successful heartbeat
// of the local liveness instance's heartbeat loop.
OnSelfLive HeartbeatCallback
}
// Start starts a periodic heartbeat to refresh this node's last
// heartbeat in the node liveness table. The optionally provided
// HeartbeatCallback will be invoked whenever this node updates its
// own liveness. The slice of engines will be written to before each
// heartbeat to avoid maintaining liveness in the presence of disk stalls.
func (nl *NodeLiveness) Start(ctx context.Context, opts NodeLivenessStartOptions) {
log.VEventf(ctx, 1, "starting node liveness instance")
retryOpts := base.DefaultRetryOptions()
retryOpts.Closer = nl.stopper.ShouldQuiesce()
if len(opts.Engines) == 0 {
// Avoid silently forgetting to pass the engines. It happened before.
log.Fatalf(ctx, "must supply at least one engine")
}
nl.mu.Lock()
nl.mu.onSelfLive = opts.OnSelfLive
nl.mu.engines = opts.Engines
nl.mu.Unlock()
_ = nl.stopper.RunAsyncTaskEx(ctx, stop.TaskOpts{TaskName: "liveness-hb", SpanOpt: stop.SterileRootSpan}, func(context.Context) {
ambient := nl.ambientCtx
ambient.AddLogTag("liveness-hb", nil)
ctx, cancel := nl.stopper.WithCancelOnQuiesce(context.Background())
defer cancel()
ctx, sp := ambient.AnnotateCtxWithSpan(ctx, "liveness heartbeat loop")
defer sp.Finish()
incrementEpoch := true
heartbeatInterval := nl.livenessThreshold - nl.renewalDuration
ticker := time.NewTicker(heartbeatInterval)
defer ticker.Stop()
for {
select {
case <-nl.heartbeatToken:
case <-nl.stopper.ShouldQuiesce():
return
}
// Give the context a timeout approximately as long as the time we
// have left before our liveness entry expires.
if err := contextutil.RunWithTimeout(ctx, "node liveness heartbeat", nl.renewalDuration,
func(ctx context.Context) error {
// Retry heartbeat in the event the conditional put fails.
for r := retry.StartWithCtx(ctx, retryOpts); r.Next(); {
oldLiveness, ok := nl.Self()
if !ok {
nodeID := nl.gossip.NodeID.Get()
liveness, err := nl.getLivenessFromKV(ctx, nodeID)
if err != nil {
log.Infof(ctx, "unable to get liveness record from KV: %s", err)
if grpcutil.IsConnectionRejected(err) {
return err
}
continue
}
oldLiveness = liveness
}
if err := nl.heartbeatInternal(ctx, oldLiveness, incrementEpoch); err != nil {
if errors.Is(err, ErrEpochIncremented) {
log.Infof(ctx, "%s; retrying", err)
continue
}
return err
}
incrementEpoch = false // don't increment epoch after first heartbeat
break
}
return nil
}); err != nil {
log.Warningf(ctx, heartbeatFailureLogFormat, err)
}
nl.heartbeatToken <- struct{}{}
select {
case <-ticker.C:
case <-nl.stopper.ShouldQuiesce():
return
}
}
})
}
const heartbeatFailureLogFormat = `failed node liveness heartbeat: %+v
An inability to maintain liveness will prevent a node from participating in a
cluster. If this problem persists, it may be a sign of resource starvation or
of network connectivity problems. For help troubleshooting, visit:
https://www.cockroachlabs.com/docs/stable/cluster-setup-troubleshooting.html#node-liveness-issues
`
// PauseHeartbeatLoopForTest stops the periodic heartbeat. The function
// waits until it acquires the heartbeatToken (unless heartbeat was
// already paused); this ensures that no heartbeats happen after this is
// called. Returns a closure to call to re-enable the heartbeat loop.
// This function is only safe for use in tests.
func (nl *NodeLiveness) PauseHeartbeatLoopForTest() func() {
if swapped := atomic.CompareAndSwapUint32(&nl.heartbeatPaused, 0, 1); swapped {
<-nl.heartbeatToken
}
return func() {
if swapped := atomic.CompareAndSwapUint32(&nl.heartbeatPaused, 1, 0); swapped {
nl.heartbeatToken <- struct{}{}
}
}
}
// PauseSynchronousHeartbeatsForTest disables all node liveness
// heartbeats triggered from outside the normal Start loop.
// Returns a closure to call to re-enable synchronous heartbeats. Only
// safe for use in tests.
func (nl *NodeLiveness) PauseSynchronousHeartbeatsForTest() func() {
nl.selfSem <- struct{}{}
nl.otherSem <- struct{}{}
return func() {
<-nl.selfSem
<-nl.otherSem
}
}
// PauseAllHeartbeatsForTest disables all node liveness heartbeats,
// including those triggered from outside the normal Start
// loop. Returns a closure to call to re-enable heartbeats. Only safe
// for use in tests.
func (nl *NodeLiveness) PauseAllHeartbeatsForTest() func() {
enableLoop := nl.PauseHeartbeatLoopForTest()
enableSync := nl.PauseSynchronousHeartbeatsForTest()
return func() {
enableLoop()
enableSync()
}
}
var errNodeAlreadyLive = errors.New("node already live")
// Heartbeat is called to update a node's expiration timestamp. This
// method does a conditional put on the node liveness record, and if
// successful, stores the updated liveness record in the nodes map.
//
// The liveness argument is the expected previous value of this node's
// liveness.
//
// If this method returns nil, the node's liveness has been extended,
// relative to the previous value. It may or may not still be alive
// when this method returns.
//
// On failure, this method returns ErrEpochIncremented, although this
// may not necessarily mean that the epoch was actually incremented.
// TODO(bdarnell): Fix error semantics here.
//
// This method is rarely called directly; heartbeats are normally sent
// by the Start loop.
// TODO(bdarnell): Should we just remove this synchronous heartbeat completely?
func (nl *NodeLiveness) Heartbeat(ctx context.Context, liveness livenesspb.Liveness) error {
return nl.heartbeatInternal(ctx, liveness, false /* increment epoch */)
}
func (nl *NodeLiveness) heartbeatInternal(
ctx context.Context, oldLiveness livenesspb.Liveness, incrementEpoch bool,
) (err error) {
ctx, sp := tracing.EnsureChildSpan(ctx, nl.ambientCtx.Tracer, "liveness heartbeat")
defer sp.Finish()
defer func(start time.Time) {
dur := timeutil.Since(start)
nl.metrics.HeartbeatLatency.RecordValue(dur.Nanoseconds())
if dur > time.Second {
log.Warningf(ctx, "slow heartbeat took %s; err=%v", dur, err)
}
}(timeutil.Now())
// Collect a clock reading from before we begin queuing on the heartbeat
// semaphore. This method (attempts to, see [*]) guarantees that, if
// successful, the liveness record's expiration will be at least the
// liveness threshold above the time that the method was called.
// Collecting this clock reading before queuing allows us to enforce
// this while avoiding redundant liveness heartbeats during thundering
// herds without needing to explicitly coalesce heartbeats.
//
// [*]: see TODO below about how errNodeAlreadyLive handling does not
// enforce this guarantee.
beforeQueueTS := nl.clock.Now()
minExpiration := beforeQueueTS.Add(nl.livenessThreshold.Nanoseconds(), 0).ToLegacyTimestamp()
// Before queueing, record the heartbeat as in-flight.
nl.metrics.HeartbeatsInFlight.Inc(1)
defer nl.metrics.HeartbeatsInFlight.Dec(1)
// Allow only one heartbeat at a time.
nodeID := nl.gossip.NodeID.Get()
sem := nl.sem(nodeID)
select {
case sem <- struct{}{}:
case <-ctx.Done():
return ctx.Err()
}
defer func() {
<-sem
}()
// If we are not intending to increment the node's liveness epoch, detect
// whether this heartbeat is needed anymore. It is possible that we queued
// for long enough on the semaphore such that other heartbeat attempts ahead
// of us already incremented the expiration past what we wanted. Note that
// if we allowed the heartbeat to proceed in this case, we know that it
// would hit a ConditionFailedError and return a errNodeAlreadyLive down
// below.
if !incrementEpoch {
curLiveness, ok := nl.Self()
if ok && minExpiration.Less(curLiveness.Expiration) {
return nil
}
}
if oldLiveness == (livenesspb.Liveness{}) {
return errors.AssertionFailedf("invalid old liveness record; found to be empty")
}
// Let's compute what our new liveness record should be. Start off with our
// existing view of things.
newLiveness := oldLiveness
if incrementEpoch {
newLiveness.Epoch++
newLiveness.Draining = false // clear draining field
}
// Grab a new clock reading to compute the new expiration time,
// since we may have queued on the semaphore for a while.
afterQueueTS := nl.clock.Now()
newLiveness.Expiration = afterQueueTS.Add(nl.livenessThreshold.Nanoseconds(), 0).ToLegacyTimestamp()
// This guards against the system clock moving backwards. As long
// as the cockroach process is running, checks inside hlc.Clock
// will ensure that the clock never moves backwards, but these
// checks don't work across process restarts.
if newLiveness.Expiration.Less(oldLiveness.Expiration) {
return errors.Errorf("proposed liveness update expires earlier than previous record")
}
update := livenessUpdate{
oldLiveness: oldLiveness,
newLiveness: newLiveness,
}
written, err := nl.updateLiveness(ctx, update, func(actual Record) error {
// Update liveness to actual value on mismatch.
nl.maybeUpdate(ctx, actual)
// If the actual liveness is different than expected, but is
// considered live, treat the heartbeat as a success. This can
// happen when the periodic heartbeater races with a concurrent
// lease acquisition.
//
// TODO(bdarnell): If things are very slow, the new liveness may
// have already expired and we'd incorrectly return
// ErrEpochIncremented. Is this check even necessary? The common
// path through this method doesn't check whether the liveness
// expired while in flight, so maybe we don't have to care about
// that and only need to distinguish between same and different
// epochs in our return value.
//
// TODO(nvanbenschoten): Unlike the early return above, this doesn't
// guarantee that the resulting expiration is past minExpiration,
// only that it's different than our oldLiveness. Is that ok? It
// hasn't caused issues so far, but we might want to detect this
// case and retry, at least in the case of the liveness heartbeat
// loop. The downside of this is that a heartbeat that's intending
// to bump the expiration of a record out 9s into the future may
// return a success even if the expiration is only 5 seconds in the
// future. The next heartbeat will then start with only 0.5 seconds
// before expiration.
if actual.IsLive(nl.clock.Now().GoTime()) && !incrementEpoch {
return errNodeAlreadyLive
}
// Otherwise, return error.
return ErrEpochIncremented
})
if err != nil {
if errors.Is(err, errNodeAlreadyLive) {
nl.metrics.HeartbeatSuccesses.Inc(1)
return nil