From bbfd0276ed50a3e69bc5a72a9e7aef54bef1488d Mon Sep 17 00:00:00 2001 From: Mira Radeva Date: Tue, 17 Dec 2024 10:45:47 -0500 Subject: [PATCH 1/2] storeliveness: rename liveness interval to support duration The liveness interval indicates the length of each period of requested (or extended) support in store liveness. It is not really an interval as much as a duration. This commit renames "liveness interval" to "support duration", which also aligns with the notion of a lease duration. Part of: #133613 Release note: None --- pkg/kv/kvserver/storeliveness/config.go | 9 ++++----- .../kvserver/storeliveness/requester_state.go | 9 +++++---- .../storeliveness/store_liveness_test.go | 2 +- .../kvserver/storeliveness/support_manager.go | 2 +- .../storeliveness/support_manager_test.go | 12 +++++------ pkg/kv/kvserver/storeliveness/testdata/basic | 2 +- .../storeliveness/testdata/liveness_interval | 14 ++++++------- .../storeliveness/testdata/multi-store | 4 ++-- .../storeliveness/testdata/requester_state | 20 +++++++++---------- .../kvserver/storeliveness/testdata/restart | 6 +++--- .../storeliveness/testdata/supporter_state | 2 +- 11 files changed, 41 insertions(+), 41 deletions(-) diff --git a/pkg/kv/kvserver/storeliveness/config.go b/pkg/kv/kvserver/storeliveness/config.go index 749375b1e81f..41f26afcbc37 100644 --- a/pkg/kv/kvserver/storeliveness/config.go +++ b/pkg/kv/kvserver/storeliveness/config.go @@ -11,10 +11,9 @@ import ( ) // Options includes all Store Liveness durations needed by the SupportManager. -// TODO(mira): make sure these are initialized correctly as part of #125066. type Options struct { - // LivenessInterval determines the Store Liveness support expiration time. - LivenessInterval time.Duration + // SupportDuration determines the Store Liveness support expiration time. + SupportDuration time.Duration // HeartbeatInterval determines how often Store Liveness sends heartbeats. HeartbeatInterval time.Duration // SupportExpiryInterval determines how often Store Liveness checks if support @@ -31,12 +30,12 @@ type Options struct { // NewOptions instantiates the Store Liveness Options. func NewOptions( - livenessInterval time.Duration, + supportDuration time.Duration, heartbeatInterval time.Duration, supportWithdrawalGracePeriod time.Duration, ) Options { return Options{ - LivenessInterval: livenessInterval, + SupportDuration: supportDuration, HeartbeatInterval: heartbeatInterval, SupportExpiryInterval: 100 * time.Millisecond, IdleSupportFromInterval: 1 * time.Minute, diff --git a/pkg/kv/kvserver/storeliveness/requester_state.go b/pkg/kv/kvserver/storeliveness/requester_state.go index 9b3c7ee3cf4c..305ccc779872 100644 --- a/pkg/kv/kvserver/storeliveness/requester_state.go +++ b/pkg/kv/kvserver/storeliveness/requester_state.go @@ -333,10 +333,11 @@ func (rsfu *requesterStateForUpdate) getHeartbeatsToSend( } // updateMaxRequested forwards the current MaxRequested timestamp to now + -// interval, where now is the node's clock timestamp and interval is the -// liveness interval. -func (rsfu *requesterStateForUpdate) updateMaxRequested(now hlc.Timestamp, interval time.Duration) { - newMaxRequested := now.Add(interval.Nanoseconds(), 0) +// support duration, where now is the node's clock timestamp. +func (rsfu *requesterStateForUpdate) updateMaxRequested( + now hlc.Timestamp, supportDuration time.Duration, +) { + newMaxRequested := now.Add(supportDuration.Nanoseconds(), 0) meta := rsfu.getMeta() if meta.MaxRequested.Forward(newMaxRequested) { // Update the entire meta struct to ensure MaxEpoch is not overwritten. diff --git a/pkg/kv/kvserver/storeliveness/store_liveness_test.go b/pkg/kv/kvserver/storeliveness/store_liveness_test.go index 05d2ce7947bc..33d91593eab4 100644 --- a/pkg/kv/kvserver/storeliveness/store_liveness_test.go +++ b/pkg/kv/kvserver/storeliveness/store_liveness_test.go @@ -64,7 +64,7 @@ func TestStoreLiveness(t *testing.T) { case "send-heartbeats": now := parseTimestamp(t, d, "now") manual.AdvanceTo(now.GoTime()) - sm.options.LivenessInterval = parseDuration(t, d, "liveness-interval") + sm.options.SupportDuration = parseDuration(t, d, "support-duration") sm.maybeAddStores(ctx) sm.sendHeartbeats(ctx) heartbeats := sender.drainSentMessages() diff --git a/pkg/kv/kvserver/storeliveness/support_manager.go b/pkg/kv/kvserver/storeliveness/support_manager.go index 584538319e06..941598a58af9 100644 --- a/pkg/kv/kvserver/storeliveness/support_manager.go +++ b/pkg/kv/kvserver/storeliveness/support_manager.go @@ -286,7 +286,7 @@ func (sm *SupportManager) sendHeartbeats(ctx context.Context) { } rsfu := sm.requesterStateHandler.checkOutUpdate() defer sm.requesterStateHandler.finishUpdate(rsfu) - livenessInterval := sm.options.LivenessInterval + livenessInterval := sm.options.SupportDuration heartbeats := rsfu.getHeartbeatsToSend(sm.storeID, sm.clock.Now(), livenessInterval) if err := rsfu.write(ctx, sm.engine); err != nil { log.Warningf(ctx, "failed to write requester meta: %v", err) diff --git a/pkg/kv/kvserver/storeliveness/support_manager_test.go b/pkg/kv/kvserver/storeliveness/support_manager_test.go index 36870e7f94ab..1db3005f5848 100644 --- a/pkg/kv/kvserver/storeliveness/support_manager_test.go +++ b/pkg/kv/kvserver/storeliveness/support_manager_test.go @@ -27,7 +27,7 @@ var ( store = slpb.StoreIdent{NodeID: roachpb.NodeID(1), StoreID: roachpb.StoreID(1)} remoteStore = slpb.StoreIdent{NodeID: roachpb.NodeID(2), StoreID: roachpb.StoreID(2)} options = Options{ - LivenessInterval: 6 * time.Millisecond, + SupportDuration: 6 * time.Millisecond, HeartbeatInterval: 3 * time.Millisecond, SupportExpiryInterval: 1 * time.Millisecond, IdleSupportFromInterval: 1 * time.Minute, @@ -135,7 +135,7 @@ func TestSupportManagerProvidesSupport(t *testing.T) { From: remoteStore, To: sm.storeID, Epoch: slpb.Epoch(1), - Expiration: sm.clock.Now().AddDuration(options.LivenessInterval), + Expiration: sm.clock.Now().AddDuration(options.SupportDuration), } require.NoError(t, sm.HandleMessage(heartbeat)) @@ -261,11 +261,11 @@ func TestSupportManagerRestart(t *testing.T) { From: remoteStore, To: sm.storeID, Epoch: slpb.Epoch(1), - Expiration: clock.Now().AddDuration(sm.options.LivenessInterval), + Expiration: clock.Now().AddDuration(sm.options.SupportDuration), } sm.handleMessages(ctx, []*slpb.Message{heartbeatResp, heartbeat}) manual.Resume() - manual.Increment(sm.options.LivenessInterval.Nanoseconds()) + manual.Increment(sm.options.SupportDuration.Nanoseconds()) sm.withdrawSupport(ctx) withdrawalTime := sm.supporterStateHandler.supporterState.meta.MaxWithdrawn.ToTimestamp() @@ -322,7 +322,7 @@ func TestSupportManagerDiskStall(t *testing.T) { From: remoteStore, To: sm.storeID, Epoch: slpb.Epoch(1), - Expiration: clock.Now().AddDuration(sm.options.LivenessInterval), + Expiration: clock.Now().AddDuration(sm.options.SupportDuration), } sm.handleMessages(ctx, []*slpb.Message{heartbeatResp, heartbeat}) @@ -379,7 +379,7 @@ func TestSupportManagerReceiveQueueLimit(t *testing.T) { From: remoteStore, To: sm.storeID, Epoch: slpb.Epoch(1), - Expiration: clock.Now().AddDuration(sm.options.LivenessInterval), + Expiration: clock.Now().AddDuration(sm.options.SupportDuration), } for i := 0; i < maxReceiveQueueSize; i++ { diff --git a/pkg/kv/kvserver/storeliveness/testdata/basic b/pkg/kv/kvserver/storeliveness/testdata/basic index cdc25174c55f..4a1e27d9ca54 100644 --- a/pkg/kv/kvserver/storeliveness/testdata/basic +++ b/pkg/kv/kvserver/storeliveness/testdata/basic @@ -10,7 +10,7 @@ support-from node-id=2 store-id=2 ---- epoch: 0, expiration: 0,0 -send-heartbeats now=100 liveness-interval=10s +send-heartbeats now=100 support-duration=10s ---- heartbeats: {Type:MsgHeartbeat From:{NodeID:1 StoreID:1} To:{NodeID:2 StoreID:2} Epoch:1 Expiration:110.000000000,0} diff --git a/pkg/kv/kvserver/storeliveness/testdata/liveness_interval b/pkg/kv/kvserver/storeliveness/testdata/liveness_interval index af3459f7b8a1..03c10cd0a6f0 100644 --- a/pkg/kv/kvserver/storeliveness/testdata/liveness_interval +++ b/pkg/kv/kvserver/storeliveness/testdata/liveness_interval @@ -1,6 +1,6 @@ # ------------------------------------------------------------- # In this test a store (n1, s1) requests support with different -# values for the liveness interval. +# values for the support duration. # ------------------------------------------------------------- support-from node-id=2 store-id=2 @@ -9,10 +9,10 @@ epoch: 0, expiration: 0,0 # ------------------------------------------------------------- # Store (n1, s1) requests and receives support with -# liveness-interval=10s. +# support-duration=10s. # ------------------------------------------------------------- -send-heartbeats now=100 liveness-interval=10s +send-heartbeats now=100 support-duration=10s ---- heartbeats: {Type:MsgHeartbeat From:{NodeID:1 StoreID:1} To:{NodeID:2 StoreID:2} Epoch:1 Expiration:110.000000000,0} @@ -27,11 +27,11 @@ epoch: 1, expiration: 110.000000000,0 # ------------------------------------------------------------- -# Store (n1, s1) requests support with liveness-interval=20s +# Store (n1, s1) requests support with support-duration=20s # and successfully extends support from (n2, s2). # ------------------------------------------------------------- -send-heartbeats now=101 liveness-interval=20s +send-heartbeats now=101 support-duration=20s ---- heartbeats: {Type:MsgHeartbeat From:{NodeID:1 StoreID:1} To:{NodeID:2 StoreID:2} Epoch:1 Expiration:121.000000000,0} @@ -46,11 +46,11 @@ epoch: 1, expiration: 121.000000000,0 # ------------------------------------------------------------- -# Store (n1, s1) requests support with liveness-interval=5s. +# Store (n1, s1) requests support with support-duration=5s. # The support expiration does not regress. # ------------------------------------------------------------- -send-heartbeats now=102 liveness-interval=5s +send-heartbeats now=102 support-duration=5s ---- heartbeats: {Type:MsgHeartbeat From:{NodeID:1 StoreID:1} To:{NodeID:2 StoreID:2} Epoch:1 Expiration:121.000000000,0} diff --git a/pkg/kv/kvserver/storeliveness/testdata/multi-store b/pkg/kv/kvserver/storeliveness/testdata/multi-store index 561bf167c742..ff651cc93556 100644 --- a/pkg/kv/kvserver/storeliveness/testdata/multi-store +++ b/pkg/kv/kvserver/storeliveness/testdata/multi-store @@ -17,7 +17,7 @@ support-from node-id=2 store-id=4 ---- epoch: 0, expiration: 0,0 -send-heartbeats now=100 liveness-interval=10s +send-heartbeats now=100 support-duration=10s ---- heartbeats: {Type:MsgHeartbeat From:{NodeID:1 StoreID:1} To:{NodeID:1 StoreID:2} Epoch:1 Expiration:110.000000000,0} @@ -113,7 +113,7 @@ support for: {Target:{NodeID:2 StoreID:3} Epoch:4 Expiration:0,0} {Target:{NodeID:2 StoreID:4} Epoch:4 Expiration:104.000000000,0} -send-heartbeats now=200 liveness-interval=10s +send-heartbeats now=200 support-duration=10s ---- heartbeats: {Type:MsgHeartbeat From:{NodeID:1 StoreID:1} To:{NodeID:1 StoreID:2} Epoch:1 Expiration:210.000000000,0} diff --git a/pkg/kv/kvserver/storeliveness/testdata/requester_state b/pkg/kv/kvserver/storeliveness/testdata/requester_state index de6c16360756..f1613d6908b7 100644 --- a/pkg/kv/kvserver/storeliveness/testdata/requester_state +++ b/pkg/kv/kvserver/storeliveness/testdata/requester_state @@ -11,7 +11,7 @@ epoch: 0, expiration: 0,0 # Store (n1, s1) successfully establishes support. # ------------------------------------------------------------- -send-heartbeats now=100 liveness-interval=10s +send-heartbeats now=100 support-duration=10s ---- heartbeats: {Type:MsgHeartbeat From:{NodeID:1 StoreID:1} To:{NodeID:2 StoreID:2} Epoch:1 Expiration:110.000000000,0} @@ -36,7 +36,7 @@ support from: # Store (n1, s1) successfully extends support. # ------------------------------------------------------------- -send-heartbeats now=200 liveness-interval=10s +send-heartbeats now=200 support-duration=10s ---- heartbeats: {Type:MsgHeartbeat From:{NodeID:1 StoreID:1} To:{NodeID:2 StoreID:2} Epoch:1 Expiration:210.000000000,0} @@ -54,7 +54,7 @@ epoch: 1, expiration: 210.000000000,0 # Store (n1, s1) loses support. # ------------------------------------------------------------- -send-heartbeats now=300 liveness-interval=10s +send-heartbeats now=300 support-duration=10s ---- heartbeats: {Type:MsgHeartbeat From:{NodeID:1 StoreID:1} To:{NodeID:2 StoreID:2} Epoch:1 Expiration:310.000000000,0} @@ -79,7 +79,7 @@ support from: # Store (n1, s1) re-establishes support at a higher epoch. # ------------------------------------------------------------- -send-heartbeats now=400 liveness-interval=10s +send-heartbeats now=400 support-duration=10s ---- heartbeats: {Type:MsgHeartbeat From:{NodeID:1 StoreID:1} To:{NodeID:2 StoreID:2} Epoch:2 Expiration:410.000000000,0} @@ -134,7 +134,7 @@ epoch: 2, expiration: 410.000000000,0 # Store (n1, s1) requests support but receives no response. # ------------------------------------------------------------- -send-heartbeats now=500 liveness-interval=10s +send-heartbeats now=500 support-duration=10s ---- heartbeats: {Type:MsgHeartbeat From:{NodeID:1 StoreID:1} To:{NodeID:2 StoreID:2} Epoch:2 Expiration:510.000000000,0} @@ -170,7 +170,7 @@ meta: support from: {Target:{NodeID:2 StoreID:2} Epoch:2 Expiration:410.000000000,0} -send-heartbeats now=550 liveness-interval=10s +send-heartbeats now=550 support-duration=10s ---- heartbeats: {Type:MsgHeartbeat From:{NodeID:1 StoreID:1} To:{NodeID:2 StoreID:2} Epoch:2 Expiration:560.000000000,0} @@ -185,7 +185,7 @@ meta: support from: {Target:{NodeID:2 StoreID:2} Epoch:2 Expiration:410.000000000,0} -send-heartbeats now=600 liveness-interval=10s +send-heartbeats now=600 support-duration=10s ---- heartbeats: @@ -193,7 +193,7 @@ support-from node-id=2 store-id=2 ---- epoch: 2, expiration: 410.000000000,0 -send-heartbeats now=700 liveness-interval=10s +send-heartbeats now=700 support-duration=10s ---- heartbeats: {Type:MsgHeartbeat From:{NodeID:1 StoreID:1} To:{NodeID:2 StoreID:2} Epoch:2 Expiration:710.000000000,0} @@ -206,7 +206,7 @@ heartbeats: error-on-write on=true ---- -send-heartbeats now=800 liveness-interval=10s +send-heartbeats now=800 support-duration=10s ---- heartbeats: @@ -220,7 +220,7 @@ support from: error-on-write on=false ---- -send-heartbeats now=900 liveness-interval=10s +send-heartbeats now=900 support-duration=10s ---- heartbeats: {Type:MsgHeartbeat From:{NodeID:1 StoreID:1} To:{NodeID:2 StoreID:2} Epoch:2 Expiration:910.000000000,0} diff --git a/pkg/kv/kvserver/storeliveness/testdata/restart b/pkg/kv/kvserver/storeliveness/testdata/restart index 145afc622dcb..27377a40e816 100644 --- a/pkg/kv/kvserver/storeliveness/testdata/restart +++ b/pkg/kv/kvserver/storeliveness/testdata/restart @@ -12,7 +12,7 @@ epoch: 0, expiration: 0,0 # Each store also withdraws support from the other. # ------------------------------------------------------------- -send-heartbeats now=100 liveness-interval=10s +send-heartbeats now=100 support-duration=10s ---- heartbeats: {Type:MsgHeartbeat From:{NodeID:1 StoreID:1} To:{NodeID:2 StoreID:2} Epoch:1 Expiration:110.000000000,0} @@ -130,7 +130,7 @@ support for: # from (n2, s2), so it doesn't send any heartbeats. # ------------------------------------------------------------- -send-heartbeats now=400 liveness-interval=10s +send-heartbeats now=400 support-duration=10s ---- heartbeats: @@ -142,7 +142,7 @@ epoch: 0, expiration: 0,0 # Store (n1, s1) sends heartbeats with an incremented epoch. # ------------------------------------------------------------- -send-heartbeats now=500 liveness-interval=10s +send-heartbeats now=500 support-duration=10s ---- heartbeats: {Type:MsgHeartbeat From:{NodeID:1 StoreID:1} To:{NodeID:2 StoreID:2} Epoch:3 Expiration:510.000000000,0} diff --git a/pkg/kv/kvserver/storeliveness/testdata/supporter_state b/pkg/kv/kvserver/storeliveness/testdata/supporter_state index 0a83b7c9bb35..b7ffc268758a 100644 --- a/pkg/kv/kvserver/storeliveness/testdata/supporter_state +++ b/pkg/kv/kvserver/storeliveness/testdata/supporter_state @@ -116,7 +116,7 @@ epoch: 2, support provided: true # the past. # ------------------------------------------------------------- -send-heartbeats now=301 liveness-interval=10s # just to forward the clock +send-heartbeats now=301 support-duration=10s # just to forward the clock ---- heartbeats: From ffa4eb13ef8999e9755e803e32692f8d200d433f Mon Sep 17 00:00:00 2001 From: Mira Radeva Date: Tue, 17 Dec 2024 10:39:12 -0500 Subject: [PATCH 2/2] base: decrease store liveness durations Previously, store liveness used a heartbeat interval and support duration of 3s and 6s, respectively. This matched the lease extension and lease duration, respectively. However, these values were not well aligned with Raft's election timeout (4s) and jitter (up to 2s), so when a follower had to campaign after withdrawing support from the leader, the store liveness durations added up to the Raft timeout and jitter, instead of being subsumed by them. This commit reduces the store liveness heartbeat interval and support duration to 1s and 3s, respectively. Fixes: #133613 Release note: None --- pkg/base/config.go | 42 ++++++++++++++++++++------ pkg/base/testdata/raft_config | 4 ++- pkg/base/testdata/raft_config_recovery | 9 +++--- pkg/kv/kvserver/client_raft_test.go | 6 ++-- 4 files changed, 43 insertions(+), 18 deletions(-) diff --git a/pkg/base/config.go b/pkg/base/config.go index 57ce7cb0dba8..02c0b8579915 100644 --- a/pkg/base/config.go +++ b/pkg/base/config.go @@ -28,8 +28,8 @@ import ( // Base config defaults. // // When changing these, TestDefaultRaftConfig must also be updated via -rewrite, -// and the result copied to the defaultRangeLeaseRaftElectionTimeoutMultiplier -// comment with any adjustments to the surrounding reasoning. +// and the result copied to the defaultRangeLeaseDuration comment with any +// adjustments to the surrounding reasoning. const ( defaultInsecure = false defaultUser = username.RootUser @@ -186,13 +186,13 @@ var ( // Total latency [ 3.03s - 7.20s] // // Leader lease acquisition (including raft election): - // - Store Liveness heartbeat offset (0-1 heartbeat interval) [-3.00s - 0.00s] - // - Store Liveness expiration (constant) [ 6.00s - 6.00s] + // - Store Liveness heartbeat offset (0-1 heartbeat interval) [-1.00s - 0.00s] + // - Store Liveness expiration (constant) [ 3.00s - 3.00s] // - Store Liveness withdrawal (0-1 withdrawal interval) [ 0.00s - 0.10s] // - Raft election timeout jitter (random 0x-1x timeout) [ 0.00s - 2.00s] // - Election (3x RTT: prevote, vote, append) [ 0.03s - 1.20s] // - Lease acquisition (1x RTT: append) [ 0.01s - 0.40s] - // Total latency [ 3.04s - 9.70s] + // Total latency [ 2.04s - 6.70s] // // (generated by TestDefaultRaftConfig) // @@ -228,6 +228,16 @@ var ( DefaultRPCHeartbeatTimeout = envutil.EnvOrDefaultDuration( "COCKROACH_RPC_HEARTBEAT_TIMEOUT", 3*NetworkTimeout) + // defaultStoreLivenessHeartbeatInterval is the default value for + // StoreLivenessHeartbeatInterval. + defaultStoreLivenessHeartbeatInterval = envutil.EnvOrDefaultDuration( + "COCKROACH_STORE_LIVENESS_HEARTBEAT_INTERVAL", time.Second) + + // defaultStoreLivenessSupportDuration is the default value for + // StoreLivenessSupportDuration. + defaultStoreLivenessSupportDuration = envutil.EnvOrDefaultDuration( + "COCKROACH_STORE_LIVENESS_SUPPORT_DURATION", 3*time.Second) + // defaultRaftTickInterval is the default resolution of the Raft timer. defaultRaftTickInterval = envutil.EnvOrDefaultDuration( "COCKROACH_RAFT_TICK_INTERVAL", 500*time.Millisecond) @@ -549,6 +559,14 @@ type RaftConfig struct { // RaftHeartbeatIntervalTicks is the number of ticks that pass between heartbeats. RaftHeartbeatIntervalTicks int64 + // StoreLivenessHeartbeatInterval determines how ofter stores request and + // extend store liveness support. + StoreLivenessHeartbeatInterval time.Duration + + // StoreLivenessSupportDuration is the duration of store liveness support that + // stores request and extend. + StoreLivenessSupportDuration time.Duration + // RangeLeaseRaftElectionTimeoutMultiplier specifies the range lease duration. RangeLeaseDuration time.Duration // RangeLeaseRenewalFraction specifies what fraction the range lease renewal @@ -658,6 +676,12 @@ func (cfg *RaftConfig) SetDefaults() { if cfg.RaftHeartbeatIntervalTicks == 0 { cfg.RaftHeartbeatIntervalTicks = defaultRaftHeartbeatIntervalTicks } + if cfg.StoreLivenessHeartbeatInterval == 0 { + cfg.StoreLivenessHeartbeatInterval = defaultStoreLivenessHeartbeatInterval + } + if cfg.StoreLivenessSupportDuration == 0 { + cfg.StoreLivenessSupportDuration = defaultStoreLivenessSupportDuration + } if cfg.RangeLeaseDuration == 0 { cfg.RangeLeaseDuration = defaultRangeLeaseDuration } @@ -769,11 +793,9 @@ func (cfg RaftConfig) NodeLivenessDurations() (livenessActive, livenessRenewal t } // StoreLivenessDurations computes durations for store liveness heartbeat -// interval and liveness interval. -func (cfg RaftConfig) StoreLivenessDurations() (livenessInterval, heartbeatInterval time.Duration) { - livenessInterval = cfg.RangeLeaseDuration - heartbeatInterval = time.Duration(float64(livenessInterval) * livenessRenewalFraction) - return +// interval and support duration. +func (cfg RaftConfig) StoreLivenessDurations() (supportDuration, heartbeatInterval time.Duration) { + return cfg.StoreLivenessSupportDuration, cfg.StoreLivenessHeartbeatInterval } // SentinelGossipTTL is time-to-live for the gossip sentinel, which is gossiped diff --git a/pkg/base/testdata/raft_config b/pkg/base/testdata/raft_config index d90ebe7e97f2..7d92a43eb78a 100644 --- a/pkg/base/testdata/raft_config +++ b/pkg/base/testdata/raft_config @@ -5,6 +5,8 @@ echo RaftElectionTimeoutTicks: (int64) 4, RaftReproposalTimeoutTicks: (int64) 6, RaftHeartbeatIntervalTicks: (int64) 2, + StoreLivenessHeartbeatInterval: (time.Duration) 1s, + StoreLivenessSupportDuration: (time.Duration) 3s, RangeLeaseDuration: (time.Duration) 6s, RangeLeaseRenewalFraction: (float64) 0.5, RaftEnableCheckQuorum: (bool) true, @@ -24,5 +26,5 @@ RaftReproposalTimeout: 3s RangeLeaseDurations: active=6s renewal=3s RangeLeaseAcquireTimeout: 4s NodeLivenessDurations: active=6s renewal=3s -StoreLivenessDurations: active=6s renewal=3s +StoreLivenessDurations: active=3s renewal=1s SentinelGossipTTL: 3s diff --git a/pkg/base/testdata/raft_config_recovery b/pkg/base/testdata/raft_config_recovery index e7f8d5968b1a..13d2f620e422 100644 --- a/pkg/base/testdata/raft_config_recovery +++ b/pkg/base/testdata/raft_config_recovery @@ -1,6 +1,5 @@ # Any changes in this result should be copied to the comment on -# defaultRangeLeaseRaftElectionTimeoutMultiplier, and the corresponding -# reasoning should be adjusted. +# defaultRangeLeaseDuration, and the corresponding reasoning should be adjusted. echo ---- // Raft election (fortification disabled): @@ -23,10 +22,10 @@ echo // Total latency [ 3.03s - 7.20s] // // Leader lease acquisition (including raft election): -// - Store Liveness heartbeat offset (0-1 heartbeat interval) [-3.00s - 0.00s] -// - Store Liveness expiration (constant) [ 6.00s - 6.00s] +// - Store Liveness heartbeat offset (0-1 heartbeat interval) [-1.00s - 0.00s] +// - Store Liveness expiration (constant) [ 3.00s - 3.00s] // - Store Liveness withdrawal (0-1 withdrawal interval) [ 0.00s - 0.10s] // - Raft election timeout jitter (random 0x-1x timeout) [ 0.00s - 2.00s] // - Election (3x RTT: prevote, vote, append) [ 0.03s - 1.20s] // - Lease acquisition (1x RTT: append) [ 0.01s - 0.40s] -// Total latency [ 3.04s - 9.70s] +// Total latency [ 2.04s - 6.70s] diff --git a/pkg/kv/kvserver/client_raft_test.go b/pkg/kv/kvserver/client_raft_test.go index 0dcf69e2e9ad..40f65d706b64 100644 --- a/pkg/kv/kvserver/client_raft_test.go +++ b/pkg/kv/kvserver/client_raft_test.go @@ -6603,7 +6603,9 @@ func TestRaftCheckQuorum(t *testing.T) { }, 10*time.Second, 500*time.Millisecond) t.Logf("n1 stepped down as a leader") - // n2 or n3 should elect a new leader. + // n2 or n3 should elect a new leader. At this point, the store liveness + // SupportWithdrawalGracePeriod may not have expired yet, so this step waits + // a little longer. var leaderStatus *raft.Status require.Eventually(t, func() bool { for _, status := range []*raft.Status{repl2.RaftStatus(), repl3.RaftStatus()} { @@ -6614,7 +6616,7 @@ func TestRaftCheckQuorum(t *testing.T) { } } return false - }, 10*time.Second, 500*time.Millisecond) + }, 20*time.Second, 500*time.Millisecond) t.Logf("n%d became leader", leaderStatus.ID) // n1 shouldn't become a leader.