Skip to content

Commit

Permalink
base: add data-driven test for default Raft config
Browse files Browse the repository at this point in the history
Release note: None
  • Loading branch information
erikgrinaker committed Dec 9, 2022
1 parent 183e425 commit dcdeb8b
Show file tree
Hide file tree
Showing 6 changed files with 167 additions and 1 deletion.
2 changes: 1 addition & 1 deletion go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -122,6 +122,7 @@ require (
github.com/containerd/containerd v1.5.4
github.com/coreos/go-oidc v2.2.1+incompatible
github.com/dave/dst v0.24.0
github.com/davecgh/go-spew v1.1.1
github.com/docker/distribution v2.7.1+incompatible
github.com/docker/docker v20.10.17+incompatible
github.com/docker/go-connections v0.4.0
Expand Down Expand Up @@ -251,7 +252,6 @@ require (
github.com/cespare/xxhash/v2 v2.1.2 // indirect
github.com/charmbracelet/bubbletea v0.22.2-0.20221007125427-0e76ba142aa1 // indirect
github.com/charmbracelet/lipgloss v0.6.0 // indirect
github.com/davecgh/go-spew v1.1.1 // indirect
github.com/decred/dcrd/dcrec/secp256k1/v4 v4.1.0 // indirect
github.com/dimchansky/utfbom v1.1.1 // indirect
github.com/djherbis/atime v1.1.0 // indirect
Expand Down
4 changes: 4 additions & 0 deletions pkg/base/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -52,21 +52,25 @@ go_test(
srcs = [
"addr_validation_test.go",
"cluster_id_test.go",
"config_test.go",
"main_test.go",
"node_id_test.go",
"store_spec_test.go",
],
args = ["-test.timeout=55s"],
data = glob(["testdata/**"]),
deps = [
":base",
"//pkg/roachpb",
"//pkg/security/securityassets",
"//pkg/security/securitytest",
"//pkg/testutils",
"//pkg/testutils/echotest",
"//pkg/util/leaktest",
"//pkg/util/log",
"//pkg/util/uuid",
"@com_github_cockroachdb_errors//:errors",
"@com_github_davecgh_go_spew//spew",
"@com_github_stretchr_testify//require",
],
)
Expand Down
2 changes: 2 additions & 0 deletions pkg/base/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,8 @@ import (
)

// Base config defaults.
//
// When changing these, TestDefaultRaftConfig must also be updated via -rewrite.
const (
defaultInsecure = false
defaultUser = username.RootUser
Expand Down
124 changes: 124 additions & 0 deletions pkg/base/config_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,124 @@
// Copyright 2022 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 base_test

import (
"fmt"
"testing"
"time"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/echotest"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/davecgh/go-spew/spew"
)

func TestDefaultRaftConfig(t *testing.T) {
defer leaktest.AfterTest(t)()

var cfg base.RaftConfig
cfg.SetDefaults()

// Assert the config and various derived values.
leaseActive, leaseRenewal := cfg.RangeLeaseDurations()
nodeActive, nodeRenewal := cfg.NodeLivenessDurations()
raftElectionTimeout := cfg.RaftElectionTimeout()
raftHeartbeatInterval := cfg.RaftTickInterval * time.Duration(cfg.RaftHeartbeatIntervalTicks)

{
var s string
s += spew.Sdump(cfg)
s += fmt.Sprintf("RaftHeartbeatInterval: %s\n", raftHeartbeatInterval)
s += fmt.Sprintf("RaftElectionTimeout: %s\n", raftElectionTimeout)
s += fmt.Sprintf("RangeLeaseDurations: active=%s renewal=%s\n", leaseActive, leaseRenewal)
s += fmt.Sprintf("RangeLeaseAcquireTimeout: %s\n", cfg.RangeLeaseAcquireTimeout())
s += fmt.Sprintf("NodeLivenessDurations: active=%s renewal=%s\n", nodeActive, nodeRenewal)
s += fmt.Sprintf("SentinelGossipTTL: %s\n", cfg.SentinelGossipTTL())
echotest.Require(t, s, testutils.TestDataPath(t, "raft_config"))
}

// Generate and assert the derived recovery intervals.
const (
minRTT = 10 * time.Millisecond
maxRTT = 400 * time.Millisecond // max GCP inter-region RTT is ~350ms
maxElectionMultiplier = 2
)

type interval struct {
name string
min, max time.Duration
}

formatIntervals := func(name string, intervals []interval) string {
// Format intervals and append min/max sum.
var minSum, maxSum time.Duration
var formatted []interval
for _, ival := range intervals {
ival.name = "- " + ival.name
formatted = append(formatted, ival)
minSum += ival.min
maxSum += ival.max
}
formatted = append(formatted, interval{name: "Total latency", min: minSum, max: maxSum})

s := "// " + name + ":\n"
for _, ival := range formatted {
s += fmt.Sprintf("// %-46s [%5.2fs -%5.2fs]\n",
ival.name, ival.min.Seconds(), ival.max.Seconds())
}
return s
}

var s string
s += formatIntervals("Raft election", []interval{
{
"Heartbeat offset (0-1 heartbeat interval)",
-raftHeartbeatInterval,
0,
},
{
fmt.Sprintf("Election timeout (random 1x-%dx timeout)", maxElectionMultiplier),
raftElectionTimeout,
maxElectionMultiplier * raftElectionTimeout,
},
{
"Election (3x RTT: prevote, vote, append)",
3 * minRTT,
3 * maxRTT,
},
})
s += "//\n"
s += formatIntervals("Lease acquisition", []interval{
{
"Heartbeat offset (0-1 heartbeat interval)",
-leaseRenewal,
0,
},
{
"Lease expiration (constant)",
leaseActive,
leaseActive,
},
{
"Liveness epoch bump (2x RTT: CPut + append)",
2 * minRTT,
2 * maxRTT,
},
{
"Lease acquisition (1x RTT: append)",
minRTT,
maxRTT,
},
})

echotest.Require(t, s, testutils.TestDataPath(t, "raft_config_recovery"))
}
22 changes: 22 additions & 0 deletions pkg/base/testdata/raft_config
Original file line number Diff line number Diff line change
@@ -0,0 +1,22 @@
echo
----
(base.RaftConfig) {
RaftTickInterval: (time.Duration) 200ms,
RaftElectionTimeoutTicks: (int) 15,
RaftHeartbeatIntervalTicks: (int) 5,
RangeLeaseRaftElectionTimeoutMultiplier: (float64) 3,
RangeLeaseRenewalFraction: (float64) 0.5,
RaftLogTruncationThreshold: (int64) 16777216,
RaftProposalQuota: (int64) 8388608,
RaftMaxUncommittedEntriesSize: (uint64) 16777216,
RaftMaxSizePerMsg: (uint64) 32768,
RaftMaxCommittedSizePerReady: (uint64) 67108864,
RaftMaxInflightMsgs: (int) 128,
RaftDelaySplitToSuppressSnapshotTicks: (int) 245
}
RaftHeartbeatInterval: 1s
RaftElectionTimeout: 3s
RangeLeaseDurations: active=9s renewal=4.5s
RangeLeaseAcquireTimeout: 6s
NodeLivenessDurations: active=9s renewal=4.5s
SentinelGossipTTL: 4.5s
14 changes: 14 additions & 0 deletions pkg/base/testdata/raft_config_recovery
Original file line number Diff line number Diff line change
@@ -0,0 +1,14 @@
echo
----
// Raft election:
// - Heartbeat offset (0-1 heartbeat interval) [-1.00s - 0.00s]
// - Election timeout (random 1x-2x timeout) [ 3.00s - 6.00s]
// - Election (3x RTT: prevote, vote, append) [ 0.03s - 1.20s]
// Total latency [ 2.03s - 7.20s]
//
// Lease acquisition:
// - Heartbeat offset (0-1 heartbeat interval) [-4.50s - 0.00s]
// - Lease expiration (constant) [ 9.00s - 9.00s]
// - Liveness epoch bump (2x RTT: CPut + append) [ 0.02s - 0.80s]
// - Lease acquisition (1x RTT: append) [ 0.01s - 0.40s]
// Total latency [ 4.53s -10.20s]

0 comments on commit dcdeb8b

Please sign in to comment.