Skip to content

Commit a2da8a8

Browse files
craig[bot]stevendannaRaduBerinderafissmiraradeva
committed
154939: stress: rename Stress to DevStress r=herkolategan,jeffswenson a=stevendanna This renames the current implementation of Stress to DevStress and then adds a new Stress function that covers both local and nightly stress runs. Epic: none Release note: None 155739: storage: add storage.tombstone_dense_compaction_threshold setting r=RaduBerinde a=RaduBerinde Add a cluster setting to control the tombstone-dense compaction threshold. Fixes: #155715 Release note: None 155795: workload/schemachange: remove FK violation prediction for INSERT operations r=rafiss a=rafiss Previously, the random schema workload attempted to predict whether INSERT operations would violate foreign key constraints by analyzing the database state and constraint definitions. This prediction logic involved: - Querying the database to check if FK violations would occur - Checking for ongoing FK constraint mutations - Distinguishing between expected and potential FK violations This commit removes all FK violation prediction logic and makes ForeignKeyViolation unconditionally a potential execution and commit error for INSERT operations. This aligns with the existing pattern used for adding FK constraints, where FK violations are always treated as potential errors due to the asynchronous validation job. This change simplifies the code by removing ~200 lines of complex prediction logic and deflakes tests that depend on accurate error handling during concurrent schema changes. Resolves: #155686 Resolves: #154715 Resolves: #152402 Epic: None Release note: None 🤖 Generated with [Claude Code](https://claude.com/claude-code) 155798: kvnemesis: log a metrics report r=stevendanna a=miraradeva This commit adds logging of some key metrics at the end of each kvnemesis run. Fixes: #153793 Release note: None ---- This is what the report looks like for a run of `TestKVNemesisMultiNode`: ``` Metric | Node 1 | Node 2 | Node 3 | Node 4 ------------------------------------+----------------------+----------------------+----------------------+--------------------- raft.commands.proposed | 2906 | 117 | 273 | 118 raft.commands.reproposed.new-lai | 0 | 1 | 12 | 0 raft.commands.reproposed.unchanged | 78 | 0 | 0 | 0 txn.server_side.1PC.success | 84 | 0 | 1 | 0 txnrecovery.successes.committed | 0 | 0 | 0 | 0 txnwaitqueue.deadlocks_total | 0 | 0 | 4 | 0 txn.aborts | 13 | 10 | 8 | 12 txn.durations | μ=25ms p99=260ms | μ=82ms p99=671ms | μ=81ms p99=1.476s | μ=224ms p99=4.832s txn.restarts.writetooold | 8 | 1 | 2 | 1 txn.restarts.serializable | 1 | 0 | 0 | 0 txn.restarts.readwithinuncertainty | 0 | 0 | 0 | 0 ``` Co-authored-by: Steven Danna <[email protected]> Co-authored-by: Radu Berinde <[email protected]> Co-authored-by: Rafi Shamim <[email protected]> Co-authored-by: Mira Radeva <[email protected]>
5 parents bf25721 + 25f114c + 5e4f9c2 + 0688885 + 6ddfc2c commit a2da8a8

File tree

12 files changed

+131
-282
lines changed

12 files changed

+131
-282
lines changed

pkg/backup/restore_test.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -107,7 +107,7 @@ func TestRestoreRetryFastFails(t *testing.T) {
107107
// runtime does not exceed the max duration of the retry policy, or
108108
// else very few attempts will be made.
109109
maxDuration := 500 * time.Millisecond
110-
if skip.Stress() {
110+
if skip.DevStress() {
111111
// Under stress, the restore will take longer to complete, so we need to
112112
// increase max duration accordingly.
113113
maxDuration = 1500 * time.Millisecond

pkg/ccl/kvccl/kvtenantccl/upgradeinterlockccl/local_test_util_test.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -107,7 +107,7 @@ func runTest(t *testing.T, variant sharedtestutil.TestVariant, test sharedtestut
107107
// extend the TTL by 10x to handle cases where the system is too
108108
// overloaded to heartbeat at sub-second intervals.
109109
ttlOverride := 250 * time.Millisecond
110-
if skip.Stress() {
110+
if skip.DevStress() {
111111
ttlOverride *= 10
112112
}
113113
heartbeatOverride := ttlOverride / 10

pkg/crosscluster/replicationtestutils/testutils.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -582,7 +582,7 @@ func (c *TenantStreamingClusters) SrcExec(exec srcInitExecFunc) {
582582

583583
func WaitUntilStartTimeReached(t *testing.T, db *sqlutils.SQLRunner, ingestionJobID jobspb.JobID) {
584584
timeout := 45 * time.Second
585-
if skip.Stress() || util.RaceEnabled {
585+
if skip.DevStress() || util.RaceEnabled {
586586
timeout *= 5
587587
}
588588
testutils.SucceedsWithin(t, func() error {

pkg/kv/kvnemesis/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -113,6 +113,7 @@ go_test(
113113
"//pkg/util/hlc",
114114
"//pkg/util/leaktest",
115115
"//pkg/util/log",
116+
"//pkg/util/metric",
116117
"//pkg/util/randutil",
117118
"//pkg/util/stop",
118119
"//pkg/util/syncutil",

pkg/kv/kvnemesis/kvnemesis_test.go

Lines changed: 82 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -13,6 +13,7 @@ import (
1313
"os"
1414
"path"
1515
"strconv"
16+
"strings"
1617
"testing"
1718
"time"
1819

@@ -46,6 +47,7 @@ import (
4647
"github.com/cockroachdb/cockroach/pkg/util/hlc"
4748
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
4849
"github.com/cockroachdb/cockroach/pkg/util/log"
50+
"github.com/cockroachdb/cockroach/pkg/util/metric"
4951
"github.com/cockroachdb/cockroach/pkg/util/randutil"
5052
"github.com/cockroachdb/cockroach/pkg/util/stop"
5153
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
@@ -698,19 +700,91 @@ func testKVNemesisImpl(t testing.TB, cfg kvnemesisTestCfg) {
698700
env := &Env{SQLDBs: sqlDBs, Tracker: tr, L: logger, Partitioner: &partitioner, Restarter: tc}
699701
failures, err := RunNemesis(ctx, rng, env, config, cfg.concurrency, cfg.numSteps, cfg.mode, dbs...)
700702

701-
for i := 0; i < cfg.numNodes; i++ {
702-
t.Logf("[%d] proposed: %d", i,
703-
tc.GetFirstStoreFromServer(t, i).Metrics().RaftCommandsProposed.Count())
704-
t.Logf("[%d] reproposed unchanged: %d", i,
705-
tc.GetFirstStoreFromServer(t, i).Metrics().RaftCommandsReproposed.Count())
706-
t.Logf("[%d] reproposed with new LAI: %d", i,
707-
tc.GetFirstStoreFromServer(t, i).Metrics().RaftCommandsReproposedLAI.Count())
708-
}
703+
logMetricsReport(t, tc)
709704

710705
require.NoError(t, err, `%+v`, err)
711706
require.Zero(t, len(failures), "kvnemesis detected failures") // they've been logged already
712707
}
713708

709+
func logMetricsReport(t testing.TB, tc *testcluster.TestCluster) {
710+
metricsOfInterest := []string{
711+
// Raft command metrics
712+
"raft.commands.proposed",
713+
"raft.commands.reproposed.new-lai",
714+
"raft.commands.reproposed.unchanged",
715+
// Transaction metrics
716+
"txn.aborts",
717+
"txn.durations",
718+
"txn.restarts.writetooold",
719+
"txn.restarts.serializable",
720+
"txn.restarts.readwithinuncertainty",
721+
"txn.server_side.1PC.success",
722+
"txnrecovery.failures",
723+
"txnrecovery.successes.aborted",
724+
"txnrecovery.successes.committed",
725+
"txnwaitqueue.deadlocks_total",
726+
}
727+
728+
numNodes := tc.NumServers()
729+
nodeMetrics := make([]map[string]string, numNodes)
730+
metricsMap := make(map[string]struct{}, len(metricsOfInterest))
731+
for _, m := range metricsOfInterest {
732+
metricsMap[m] = struct{}{}
733+
}
734+
for i := 0; i < numNodes; i++ {
735+
nodeMetrics[i] = make(map[string]string)
736+
processMetric := func(name string, v interface{}) {
737+
switch val := v.(type) {
738+
case *metric.Counter:
739+
nodeMetrics[i][name] = fmt.Sprintf("%d", val.Count())
740+
case *metric.Gauge:
741+
nodeMetrics[i][name] = fmt.Sprintf("%d", val.Value())
742+
case metric.IHistogram:
743+
// TODO(mira): Currently, histograms are assumed to store values of type
744+
// time duration.
745+
snapshot := val.CumulativeSnapshot()
746+
makePretty := func(nanos float64) time.Duration {
747+
return time.Duration(nanos).Round(time.Millisecond)
748+
}
749+
meanDur := makePretty(snapshot.Mean())
750+
p50Dur := makePretty(snapshot.ValueAtQuantile(50))
751+
p99Dur := makePretty(snapshot.ValueAtQuantile(99))
752+
nodeMetrics[i][name] = fmt.Sprintf("μ=%v p50=%v p99=%v", meanDur, p50Dur, p99Dur)
753+
default:
754+
nodeMetrics[i][name] = fmt.Sprintf("unknown:%T", v)
755+
}
756+
}
757+
758+
clientRegistry := tc.Server(i).StorageLayer().MetricsRecorder().AppRegistry()
759+
nodeRegistry := tc.Server(i).StorageLayer().MetricsRecorder().NodeRegistry()
760+
storeID := tc.GetFirstStoreFromServer(t, i).StoreID()
761+
storeRegistry := tc.Server(i).StorageLayer().MetricsRecorder().StoreRegistry(storeID)
762+
clientRegistry.Select(metricsMap, func(name string, v interface{}) { processMetric(name, v) })
763+
nodeRegistry.Select(metricsMap, func(name string, v interface{}) { processMetric(name, v) })
764+
storeRegistry.Select(metricsMap, func(name string, v interface{}) { processMetric(name, v) })
765+
}
766+
767+
header := fmt.Sprintf("%-35s", "Metric")
768+
for i := 0; i < numNodes; i++ {
769+
header += fmt.Sprintf(" | %-30s", fmt.Sprintf("Node %d", i+1))
770+
}
771+
t.Log(header)
772+
773+
separator := strings.Repeat("-", 35)
774+
for i := 0; i < numNodes; i++ {
775+
separator += "-+-" + strings.Repeat("-", 30)
776+
}
777+
t.Log(separator)
778+
779+
for _, metricName := range metricsOfInterest {
780+
row := fmt.Sprintf("%-35s", metricName)
781+
for i := 0; i < numNodes; i++ {
782+
row += fmt.Sprintf(" | %-30s", nodeMetrics[i][metricName])
783+
}
784+
t.Log(row)
785+
}
786+
}
787+
714788
// TestRunReproductionSteps is a helper that allows quickly running a kvnemesis
715789
// history.
716790
func TestRunReproductionSteps(t *testing.T) {

pkg/server/status/recorder.go

Lines changed: 15 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -244,6 +244,21 @@ func (mr *MetricsRecorder) AppRegistry() *metric.Registry {
244244
return mr.mu.appRegistry
245245
}
246246

247+
// NodeRegistry returns the metric registry for node-level metrics.
248+
func (mr *MetricsRecorder) NodeRegistry() *metric.Registry {
249+
mr.mu.Lock()
250+
defer mr.mu.Unlock()
251+
return mr.mu.logRegistry
252+
}
253+
254+
// StoreRegistry returns the metric registry for store-level metrics
255+
// corresponding to the provided store ID.
256+
func (mr *MetricsRecorder) StoreRegistry(id roachpb.StoreID) *metric.Registry {
257+
mr.mu.Lock()
258+
defer mr.mu.Unlock()
259+
return mr.mu.storeRegistries[id]
260+
}
261+
247262
// AddNode adds various metric registries an initialized server, along
248263
// with its descriptor and start time.
249264
// The registries are:

pkg/sql/mvcc_statistics_update_job_test.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -203,7 +203,7 @@ func TestTenantGlobalAggregatedLivebytes(t *testing.T) {
203203
// Exact match for non stress tests, and allow values to differ by up to
204204
// 5% in stress situations.
205205
confidenceLevel := 0.0
206-
if skip.Stress() {
206+
if skip.DevStress() {
207207
confidenceLevel = 0.05
208208
}
209209
testutils.SucceedsSoon(t, func() error {

pkg/storage/pebble.go

Lines changed: 11 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -524,6 +524,14 @@ var baselineDeletionRate = settings.RegisterIntSetting(
524524
settings.NonNegativeInt,
525525
)
526526

527+
var tombstoneDenseCompactionThreshold = settings.RegisterIntSetting(
528+
settings.ApplicationLevel,
529+
"storage.tombstone_dense_compaction_threshold",
530+
"percentage of tombstone-dense data blocks that trigger a compaction (0 = disabled)",
531+
10, // 10%
532+
settings.IntInRange(0, 100),
533+
)
534+
527535
// EngineComparer is a pebble.Comparer object that implements MVCC-specific
528536
// comparator settings for use with Pebble.
529537
var EngineComparer = func() pebble.Comparer {
@@ -962,6 +970,9 @@ func newPebble(ctx context.Context, cfg engineConfig) (p *Pebble, err error) {
962970
cfg.opts.TargetByteDeletionRate = func() int {
963971
return int(baselineDeletionRate.Get(&cfg.settings.SV))
964972
}
973+
cfg.opts.Experimental.TombstoneDenseCompactionThreshold = func() float64 {
974+
return 0.01 * float64(tombstoneDenseCompactionThreshold.Get(&cfg.settings.SV))
975+
}
965976
if cfg.opts.Experimental.UseDeprecatedCompensatedScore == nil {
966977
cfg.opts.Experimental.UseDeprecatedCompensatedScore = func() bool {
967978
return useDeprecatedCompensatedScore.Get(&cfg.settings.SV)

pkg/testutils/skip/skip.go

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -188,8 +188,8 @@ func UnderDuressWithIssue(t SkippableTest, githubIssueID int, args ...interface{
188188
}
189189
}
190190

191-
// Duress catures the conditions that currently lead us to
192-
// believe that tests may be slower than normal.
191+
// Duress captures the conditions that currently lead us to believe that tests
192+
// may be slower than normal.
193193
func Duress() bool {
194194
return util.RaceEnabled || Stress() || syncutil.DeadlockEnabled
195195
}

pkg/testutils/skip/stress.go

Lines changed: 14 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -7,17 +7,24 @@ package skip
77

88
import "github.com/cockroachdb/cockroach/pkg/util/envutil"
99

10-
var nightlyStress = envutil.EnvOrDefaultBool("COCKROACH_NIGHTLY_STRESS", false)
10+
var (
11+
nightlyStress = envutil.EnvOrDefaultBool("COCKROACH_NIGHTLY_STRESS", false)
12+
stress = envutil.EnvOrDefaultBool("COCKROACH_STRESS", false)
13+
)
1114

12-
var stress = envutil.EnvOrDefaultBool("COCKROACH_STRESS", false)
13-
14-
// NightlyStress returns true iff the process is running as part of CockroachDB's
15-
// nightly stress tests.
15+
// NightlyStress returns true iff the process is running as part of
16+
// CockroachDB's nightly stress tests.
1617
func NightlyStress() bool {
1718
return nightlyStress
1819
}
1920

20-
// Stress returns true iff the process is running under a local _dev_ instance of the stress, i.e., ./dev test ... --stress
21-
func Stress() bool {
21+
// DevStress returns true iff the process is running under a local _dev_ instance
22+
// of the stress, i.e., ./dev test ... --stress
23+
func DevStress() bool {
2224
return stress
2325
}
26+
27+
// Stress returns true iff the process is running under local or nightly stress.
28+
func Stress() bool {
29+
return DevStress() || NightlyStress()
30+
}

0 commit comments

Comments
 (0)