Skip to content

Commit b136ab6

Browse files
craig[bot]tbgherkolategan
committed
154640: roachtest: add mma-and-count versions of allocbench r=wenyihu6 a=tbg Fixes #154638. Epic: CRDB-55052 155155: build: increase timeouts for microbenchmark CI run groups r=tbg,rickystewart a=herkolategan We've been hitting these limits due to some minor changes in the underlying benchmarks in these groups. This ups the limit so the job doesn't time out. Epic: None Release note: None 155168: tests: disable lease queue in Benchmark{,Parallel}Sysbench r=tbg a=tbg **tests: disable lease queue in Benchmark{,Parallel}Sysbench** This keeps leases on s1, which reduces variance in the benchmark. --- **tests: also wait for full replication on sysbenchKV** Only the SQL flavor was previously checking for full replication. Epic: None 155172: kv: migrate off log.Dev r=tbg a=tbg These are the last calls to `log.Dev`. Closes #152584. Epic: CRDB-52421 Co-authored-by: Tobias Grieger <[email protected]> Co-authored-by: Herko Lategan <[email protected]>
5 parents 165e40b + f20f5bf + 87f5dda + 96e44cc + 89b6743 commit b136ab6

File tree

9 files changed

+61
-24
lines changed

9 files changed

+61
-24
lines changed

.github/workflows/microbenchmarks-ci.yaml

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -47,7 +47,7 @@ jobs:
4747
pkg: ${{ env.PACKAGE }}
4848
run-group-1:
4949
runs-on: [self-hosted, ubuntu_2004_microbench]
50-
timeout-minutes: 60
50+
timeout-minutes: 90
5151
needs: [base, head]
5252
steps:
5353
- name: Checkout
@@ -60,7 +60,7 @@ jobs:
6060
group: 1
6161
run-group-2:
6262
runs-on: [self-hosted, ubuntu_2004_microbench]
63-
timeout-minutes: 60
63+
timeout-minutes: 90
6464
needs: [base, head]
6565
steps:
6666
- name: Checkout

pkg/cmd/roachtest/tests/allocation_bench.go

Lines changed: 24 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -44,6 +44,7 @@ const (
4444
type allocationBenchSpec struct {
4545
nodes, cpus int
4646
load allocBenchLoad
47+
lbrMode string // see kvserverbase.LoadBasedRebalancingMode
4748

4849
startRecord time.Duration
4950
samples int
@@ -172,7 +173,7 @@ func (r kvAllocBenchEventRunner) run(ctx context.Context, c cluster.Cluster, t t
172173
return c.RunE(ctx, option.WithNodes(c.WorkloadNode()), runCmd)
173174
}
174175
func registerAllocationBench(r registry.Registry) {
175-
for _, spec := range []allocationBenchSpec{
176+
specTemplates := []allocationBenchSpec{
176177
// TODO(kvoli): Add a background event runner and implement events for
177178
// import and index backfills.
178179
{
@@ -242,8 +243,19 @@ func registerAllocationBench(r registry.Registry) {
242243
},
243244
},
244245
},
245-
} {
246-
registerAllocationBenchSpec(r, spec)
246+
}
247+
for _, spec := range specTemplates {
248+
{
249+
spec := spec
250+
spec.lbrMode = "leases and replicas"
251+
registerAllocationBenchSpec(r, spec)
252+
}
253+
{
254+
spec := spec
255+
spec.lbrMode = "multi-metric and count"
256+
spec.load.desc += "/lbr=mmc"
257+
registerAllocationBenchSpec(r, spec)
258+
}
247259
}
248260
}
249261

@@ -260,7 +272,7 @@ func registerAllocationBenchSpec(r registry.Registry, allocSpec allocationBenchS
260272
Timeout: time.Duration(allocSpec.samples) * time.Hour,
261273
NonReleaseBlocker: true,
262274
CompatibleClouds: registry.AllExceptAWS,
263-
Suites: registry.Suites(registry.Nightly),
275+
Suites: registry.Suites(registry.Weekly),
264276
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
265277
runAllocationBench(ctx, t, c, allocSpec)
266278
},
@@ -273,11 +285,18 @@ func setupAllocationBench(
273285
t.Status("starting cluster")
274286
for i := 1; i <= spec.nodes; i++ {
275287
// Don't start a backup schedule as this test reports to roachperf.
288+
settings := install.MakeClusterSettings()
289+
settings.Env = append(settings.Env, "COCKROACH_ALLOW_MMA=true")
276290
startOpts := option.NewStartOpts(option.NoBackupSchedule)
277291
startOpts.RoachprodOpts.ExtraArgs = append(startOpts.RoachprodOpts.ExtraArgs,
278292
"--vmodule=store_rebalancer=2,allocator=2,replicate_queue=2")
279-
c.Start(ctx, t.L(), startOpts, install.MakeClusterSettings(), c.Node(i))
293+
c.Start(ctx, t.L(), startOpts, settings, c.Node(i))
280294
}
295+
require.NotEmpty(t, spec.lbrMode, "lbrMode must be set")
296+
_, err := c.Conn(ctx, t.L(), 1).ExecContext(ctx, fmt.Sprintf(
297+
`SET CLUSTER SETTING kv.allocator.load_based_rebalancing = '%s';`,
298+
spec.lbrMode))
299+
require.NoError(t, err)
281300

282301
return setupStatCollector(ctx, t, c, spec)
283302
}

pkg/kv/client_test.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -176,7 +176,7 @@ func TestClientRetryNonTxn(t *testing.T) {
176176
}
177177
notify <- struct{}{}
178178
if err != nil {
179-
log.Dev.Errorf(context.Background(), "error on non-txn request: %s", err)
179+
log.KvExec.Errorf(context.Background(), "error on non-txn request: %s", err)
180180
}
181181
doneCall <- errors.Wrapf(
182182
err, "%d: expected success on non-txn call to %s",

pkg/kv/db.go

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -218,7 +218,7 @@ func (s *CrossRangeTxnWrapperSender) Send(
218218
ctx context.Context, ba *kvpb.BatchRequest,
219219
) (*kvpb.BatchResponse, *kvpb.Error) {
220220
if ba.Txn != nil {
221-
log.Dev.Fatalf(ctx, "CrossRangeTxnWrapperSender can't handle transactional requests")
221+
log.KvExec.Fatalf(ctx, "CrossRangeTxnWrapperSender can't handle transactional requests")
222222
}
223223

224224
br, pErr := s.wrapped.Send(ctx, ba)
@@ -1167,7 +1167,7 @@ func (db *DB) sendUsingSender(
11671167
br, pErr := sender.Send(ctx, ba)
11681168
if pErr != nil {
11691169
if log.V(1) {
1170-
log.Dev.Infof(ctx, "failed batch: %s", pErr)
1170+
log.KvExec.Infof(ctx, "failed batch: %s", pErr)
11711171
}
11721172
return nil, pErr
11731173
}

pkg/kv/range_lookup.go

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -249,13 +249,13 @@ func RangeLookup(
249249
if rc == kvpb.INCONSISTENT {
250250
return nil, nil, nil
251251
}
252-
log.Dev.Warningf(ctx, "range lookup of key %s found only non-matching ranges %v; retrying",
252+
log.KvExec.Warningf(ctx, "range lookup of key %s found only non-matching ranges %v; retrying",
253253
key, prefetchedRanges)
254254
}
255255

256256
ctxErr := ctx.Err()
257257
if ctxErr == nil {
258-
log.Dev.Fatalf(ctx, "retry loop broke before context expired")
258+
log.KvExec.Fatalf(ctx, "retry loop broke before context expired")
259259
}
260260
return nil, nil, ctxErr
261261
}
@@ -320,7 +320,7 @@ func lookupRangeFwdScan(
320320
RequestHeader: kvpb.RequestHeaderFromSpan(bounds.AsRawSpanWithNoLocals()),
321321
})
322322
if !TestingIsRangeLookup(ba) {
323-
log.Dev.Fatalf(ctx, "BatchRequest %v not detectable as RangeLookup", ba)
323+
log.KvExec.Fatalf(ctx, "BatchRequest %v not detectable as RangeLookup", ba)
324324
}
325325

326326
br, pErr := sender.Send(ctx, ba)
@@ -394,7 +394,7 @@ func lookupRangeRevScan(
394394
RequestHeader: kvpb.RequestHeaderFromSpan(revBounds.AsRawSpanWithNoLocals()),
395395
})
396396
if !TestingIsRangeLookup(ba) {
397-
log.Dev.Fatalf(ctx, "BatchRequest %v not detectable as RangeLookup", ba)
397+
log.KvExec.Fatalf(ctx, "BatchRequest %v not detectable as RangeLookup", ba)
398398
}
399399

400400
br, pErr := sender.Send(ctx, ba)

pkg/kv/txn.go

Lines changed: 7 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -406,7 +406,7 @@ func (txn *Txn) TestingSetPriority(priority enginepb.TxnPriority) {
406406
// non-randomized, priority for the transaction.
407407
txn.mu.userPriority = roachpb.UserPriority(-priority)
408408
if err := txn.mu.sender.SetUserPriority(txn.mu.userPriority); err != nil {
409-
log.Dev.Fatalf(context.TODO(), "%+v", err)
409+
log.KvExec.Fatalf(context.TODO(), "%+v", err)
410410
}
411411
txn.mu.Unlock()
412412
}
@@ -1056,7 +1056,7 @@ func (txn *Txn) rollback(ctx context.Context) *kvpb.Error {
10561056
// already committed. We don't spam the logs with those.
10571057
log.VEventf(ctx, 2, "async rollback failed: %s", pErr)
10581058
} else {
1059-
log.Dev.Infof(ctx, "async rollback failed: %s", pErr)
1059+
log.KvExec.Infof(ctx, "async rollback failed: %s", pErr)
10601060
}
10611061
}
10621062
return nil
@@ -1172,7 +1172,7 @@ func (txn *Txn) exec(ctx context.Context, fn func(context.Context, *Txn) error)
11721172
// We sent transactional requests, so the TxnCoordSender was supposed to
11731173
// turn retryable errors into TransactionRetryWithProtoRefreshError. Note that this
11741174
// applies only in the case where this is the root transaction.
1175-
log.Dev.Fatalf(ctx, "unexpected UnhandledRetryableError at the txn.exec() level: %s", err)
1175+
log.KvExec.Fatalf(ctx, "unexpected UnhandledRetryableError at the txn.exec() level: %s", err)
11761176
}
11771177
} else if t := (*kvpb.TransactionRetryWithProtoRefreshError)(nil); errors.As(err, &t) {
11781178
if txn.ID() != t.PrevTxnID {
@@ -1219,13 +1219,13 @@ func (txn *Txn) exec(ctx context.Context, fn func(context.Context, *Txn) error)
12191219
txn.DebugName(), attempt, err, maxRetries, rollbackErr,
12201220
),
12211221
ErrAutoRetryLimitExhausted)
1222-
log.Dev.Warningf(ctx, "%v", err)
1222+
log.KvExec.Warningf(ctx, "%v", err)
12231223
break
12241224
}
12251225

12261226
const warnEvery = 10
12271227
if attempt%warnEvery == 0 {
1228-
log.Dev.Warningf(ctx, "have retried transaction: %s %d times, most recently because of the "+
1228+
log.KvExec.Warningf(ctx, "have retried transaction: %s %d times, most recently because of the "+
12291229
"retryable error: %s. Is the transaction stuck in a retry loop?", txn.DebugName(), attempt, err)
12301230
}
12311231

@@ -1395,7 +1395,7 @@ func (txn *Txn) Send(
13951395
if requestTxnID != retryErr.PrevTxnID {
13961396
// KV should not return errors for transactions other than the one that sent
13971397
// the request.
1398-
log.Dev.Fatalf(ctx, "retryable error for the wrong txn. "+
1398+
log.KvExec.Fatalf(ctx, "retryable error for the wrong txn. "+
13991399
"requestTxnID: %s, retryErr.PrevTxnID: %s. retryErr: %s",
14001400
requestTxnID, retryErr.PrevTxnID, retryErr)
14011401
}
@@ -1629,7 +1629,7 @@ func (txn *Txn) UpdateStateOnRemoteRetryableErr(ctx context.Context, pErr *kvpb.
16291629
defer txn.mu.Unlock()
16301630

16311631
if pErr.TransactionRestart() == kvpb.TransactionRestart_NONE {
1632-
log.Dev.Fatalf(ctx, "unexpected non-retryable error: %s", pErr)
1632+
log.KvExec.Fatalf(ctx, "unexpected non-retryable error: %s", pErr)
16331633
}
16341634

16351635
// If the transaction has been reset since this request was sent,

pkg/kv/txn_test.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -432,7 +432,7 @@ func TestWrongTxnRetry(t *testing.T) {
432432

433433
var retries int
434434
txnClosure := func(ctx context.Context, outerTxn *Txn) error {
435-
log.Dev.Infof(ctx, "outer retry")
435+
log.KvExec.Infof(ctx, "outer retry")
436436
retries++
437437
// Ensure the KV transaction is created.
438438
if err := outerTxn.Put(ctx, "a", "b"); err != nil {

pkg/sql/tests/sysbench_test.go

Lines changed: 17 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,7 @@ import (
2626
"github.com/cockroachdb/cockroach/pkg/keys"
2727
"github.com/cockroachdb/cockroach/pkg/kv"
2828
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
29+
"github.com/cockroachdb/cockroach/pkg/kv/kvserver"
2930
"github.com/cockroachdb/cockroach/pkg/roachpb"
3031
"github.com/cockroachdb/cockroach/pkg/rpc"
3132
"github.com/cockroachdb/cockroach/pkg/rpc/nodedialer"
@@ -175,14 +176,29 @@ func newTestCluster(
175176
NoLoopbackDialer: !localRPCFastPath,
176177
},
177178
},
179+
Store: &kvserver.StoreTestingKnobs{
180+
// Disable the lease queue to keep leases on s1 (otherwise, lease
181+
// count rebalancing might move one lease, and splits might copy
182+
// that lease to a number of additional ranges). Communication
183+
// between the gateway node (always n1) and the KV servers always
184+
// goes through TCP regardless of whether the gateway node equals
185+
// the KV node, but there are still subtle (and not well understood)
186+
// performance differences between then n1->n1 and n1->n[23] cases,
187+
// which add variance to the results.
188+
DisableLeaseQueue: true,
189+
},
178190
}
179-
return serverutils.StartCluster(b, nodes, base.TestClusterArgs{
191+
tc := serverutils.StartCluster(b, nodes, base.TestClusterArgs{
180192
ServerArgs: base.TestServerArgs{
181193
Settings: st,
182194
CacheSize: cacheSize,
183195
Knobs: knobs,
184196
}},
185197
)
198+
if nodes > 1 {
199+
try0(tc.WaitForFullReplication())
200+
}
201+
return tc
186202
}
187203

188204
// sysbenchSQL is SQL-based implementation of sysbenchDriver. It runs SQL
@@ -199,7 +215,6 @@ func newSysbenchSQL(nodes int, localRPCFastPath bool) sysbenchDriverConstructor
199215
for i := 0; i < nodes; i++ {
200216
tc.Server(i).SQLServer().(*sql.Server).GetExecutorConfig().LicenseEnforcer.Disable(ctx)
201217
}
202-
try0(tc.WaitForFullReplication())
203218
pgURL, cleanupURL := tc.ApplicationLayer(0).PGUrl(b, serverutils.DBName(sysbenchDB))
204219
cleanup := func() {
205220
cleanupURL()

scripts/trigger-pr-roachtest.sh

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -42,6 +42,9 @@ dump_json() {
4242
fi
4343
}
4444

45+
# NB: if you're trying to trigger a weekly roachtest, need to use
46+
# 'Cockroach_Nightlies_RoachtestWeeklyBazel' as the buildType below.
47+
4548
json_payload=$(jq -n \
4649
--arg branch_name "$pr" \
4750
--arg tests "$tests" \

0 commit comments

Comments
 (0)