Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
137608: base: decrease store liveness durations r=miraradeva a=miraradeva

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](#133613)

Release note: None

137845: diagnostics: honor stopper quiescence when reporting diagnostics r=xinhaoz a=dhartunian

Previously, we would block on the HTTP diagnostics request until the preset timeout regardless of stopper quiescence. This change ensures that the `ctx` passed to the diagnostics reporter and the HTTP client is cancelled if the stopper is quiescing.

Resolves: #136739

Release note: None

Co-authored-by: Mira Radeva <[email protected]>
Co-authored-by: David Hartunian <[email protected]>
  • Loading branch information
3 people committed Dec 20, 2024
3 parents 12812af + ffa4eb1 + f573cae commit 002ea87
Show file tree
Hide file tree
Showing 17 changed files with 121 additions and 61 deletions.
42 changes: 32 additions & 10 deletions pkg/base/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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)
//
Expand Down Expand Up @@ -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)
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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
}
Expand Down Expand Up @@ -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
Expand Down
4 changes: 3 additions & 1 deletion pkg/base/testdata/raft_config
Original file line number Diff line number Diff line change
Expand Up @@ -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,
Expand All @@ -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
9 changes: 4 additions & 5 deletions pkg/base/testdata/raft_config_recovery
Original file line number Diff line number Diff line change
@@ -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):
Expand All @@ -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]
33 changes: 32 additions & 1 deletion pkg/ccl/serverccl/diagnosticsccl/reporter_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,14 @@ import (

const elemName = "somestring"

var setTelemetryHttpTimeout = func(newVal time.Duration) func() {
prior := diagnostics.TelemetryHttpTimeout
diagnostics.TelemetryHttpTimeout = newVal
return func() {
diagnostics.TelemetryHttpTimeout = prior
}
}

func TestTenantReport(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
Expand Down Expand Up @@ -297,7 +305,7 @@ func TestTelemetry_SuccessfulTelemetryPing(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

diagnostics.TelemetryHttpTimeout = 3 * time.Second
defer setTelemetryHttpTimeout(3 * time.Second)()
rt := startReporterTest(t, base.TestIsSpecificToStorageLayerAndNeedsASystemTenant)
defer rt.Close()

Expand Down Expand Up @@ -366,6 +374,29 @@ func TestTelemetry_SuccessfulTelemetryPing(t *testing.T) {

}

// This test will block on `stopper.Stop` if the diagnostics reporter
// doesn't honor stopper quiescence when making its HTTP request.
func TestTelemetryQuiesce(t *testing.T) {
defer leaktest.AfterTest(t)()
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

defer setTelemetryHttpTimeout(100 * time.Second)()
rt := startReporterTest(t, base.TestIsSpecificToStorageLayerAndNeedsASystemTenant)
defer rt.Close()

ctx := context.Background()
setupCluster(t, rt.serverDB)

defer rt.diagServer.SetWaitSeconds(200)()
dr := rt.server.DiagnosticsReporter().(*diagnostics.Reporter)
stopper := rt.server.Stopper()

dr.PeriodicallyReportDiagnostics(ctx, stopper)
stopper.Stop(ctx)
<-stopper.IsStopped()
}

func TestUsageQuantization(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
Expand Down
6 changes: 4 additions & 2 deletions pkg/kv/kvserver/client_raft_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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()} {
Expand All @@ -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.
Expand Down
9 changes: 4 additions & 5 deletions pkg/kv/kvserver/storeliveness/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -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,
Expand Down
9 changes: 5 additions & 4 deletions pkg/kv/kvserver/storeliveness/requester_state.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/storeliveness/store_liveness_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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()
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/storeliveness/support_manager.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
12 changes: 6 additions & 6 deletions pkg/kv/kvserver/storeliveness/support_manager_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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,
Expand Down Expand Up @@ -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))

Expand Down Expand Up @@ -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()

Expand Down Expand Up @@ -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})

Expand Down Expand Up @@ -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++ {
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/storeliveness/testdata/basic
Original file line number Diff line number Diff line change
Expand Up @@ -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}
Expand Down
14 changes: 7 additions & 7 deletions pkg/kv/kvserver/storeliveness/testdata/liveness_interval
Original file line number Diff line number Diff line change
@@ -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
Expand All @@ -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}
Expand All @@ -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}
Expand All @@ -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}
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/storeliveness/testdata/multi-store
Original file line number Diff line number Diff line change
Expand Up @@ -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}
Expand Down Expand Up @@ -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}
Expand Down
Loading

0 comments on commit 002ea87

Please sign in to comment.