Skip to content

Commit f967a6a

Browse files
craig[bot]kev-caopetermattiskyle-a-wongdt
committed
159006: backup: remove TestBackupSharedProcessTenantNodeDown r=msbutler a=kev-cao This test is flaky and superfluous as our mixed version roachtests already test backup with offline nodes. Fixes: #158654 Release note: None 159152: roachprod/vm: fix JSON serialization of VM.Errors field r=petermattis a=petermattis The VM.Errors field was typed as []error, but Go's encoding/json cannot unmarshal into an error interface type because it doesn't know what concrete type to create. This caused roachprod list to fail when loading cached cluster info containing VMs with errors: json: cannot unmarshal object into Go struct field VM.vms.errors of type error This change introduces a VMError wrapper type that: - Implements the error interface - Implements json.Marshaler to serialize as a JSON string (error message) - Implements json.Unmarshaler to handle both the new format (strings) and legacy format (empty objects from broken serialization) - Implements Unwrap() so errors.Is/As still work All cloud providers (AWS, Azure, GCE, IBM) are updated to use the new VMError type via NewVMError(). The BadInstanceErrors() function now returns map[string]vm.List (grouped by error message string) instead of map[error]vm.List since serialization loses pointer identity. Release note: None Epic: None Co-authored-by: Claude <[email protected]> 159179: ui: bump cluster-ui version to 26.2.0-prerelease.0 r=kyle-a-wong a=kyle-a-wong Epic: REL-3909 Resolves: CRDB-57502 Release note: None 159200: sql: add query id to recorded statement stats in UDFs / SPs r=kyle-a-wong a=kyle-a-wong Generates a query id for sub statements executed in UDFs and SPs. This is used by the insights system to provide execution specific insights. Epic: None Release note: None 159253: cli/debug: narrow scope of --include-goroutine-stacks to just debug=2 r=dt a=dt Release note: none. Epic: none. 159280: kvserver: rm unused ID in snapWriteBuilder r=arulajmani a=pav-kv After #155953, this ID is not used. Epic: CRDB-55220 159281: backup: disable AC in datadriven tests r=dt a=dt These tests, via 'new-cluster', run a large number of nodes in a single process, overloading that process and causing bulk work to starve if it is restricted to using strictly spare capacity (of which there is none). Release note: none. Epic: none. Co-authored-by: Kevin Cao <[email protected]> Co-authored-by: Peter Mattis <[email protected]> Co-authored-by: Kyle Wong <[email protected]> Co-authored-by: David Taylor <[email protected]> Co-authored-by: Pavel Kalinnikov <[email protected]>
8 parents 85270c9 + c538bad + 18280f7 + 3f81247 + a532703 + ddc72f3 + 8ffc411 + f4f2f75 commit f967a6a

File tree

22 files changed

+172
-164
lines changed

22 files changed

+172
-164
lines changed

pkg/backup/BUILD.bazel

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -293,7 +293,6 @@ go_test(
293293
"//pkg/sql/sem/tree",
294294
"//pkg/sql/sessiondata",
295295
"//pkg/sql/sqlclustersettings",
296-
"//pkg/sql/sqlliveness/slbase",
297296
"//pkg/sql/sqltestutils",
298297
"//pkg/sql/stats",
299298
"//pkg/storage",

pkg/backup/backup_tenant_test.go

Lines changed: 0 additions & 71 deletions
Original file line numberDiff line numberDiff line change
@@ -9,19 +9,16 @@ import (
99
"context"
1010
"fmt"
1111
"testing"
12-
"time"
1312

1413
"github.com/cockroachdb/cockroach/pkg/base"
1514
"github.com/cockroachdb/cockroach/pkg/ccl/multiregionccl/multiregionccltestutils"
1615
_ "github.com/cockroachdb/cockroach/pkg/cloud/impl" // register cloud storage providers
1716
"github.com/cockroachdb/cockroach/pkg/jobs"
1817
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
19-
"github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilitiespb"
2018
"github.com/cockroachdb/cockroach/pkg/roachpb"
2119
"github.com/cockroachdb/cockroach/pkg/sql"
2220
_ "github.com/cockroachdb/cockroach/pkg/sql/importer"
2321
"github.com/cockroachdb/cockroach/pkg/sql/sqlclustersettings"
24-
"github.com/cockroachdb/cockroach/pkg/sql/sqlliveness/slbase"
2522
"github.com/cockroachdb/cockroach/pkg/testutils"
2623
"github.com/cockroachdb/cockroach/pkg/testutils/jobutils"
2724
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
@@ -30,78 +27,10 @@ import (
3027
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
3128
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
3229
"github.com/cockroachdb/cockroach/pkg/util/log"
33-
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
3430
"github.com/cockroachdb/errors"
3531
"github.com/stretchr/testify/require"
3632
)
3733

38-
func TestBackupSharedProcessTenantNodeDown(t *testing.T) {
39-
defer leaktest.AfterTest(t)()
40-
defer log.Scope(t).Close(t)
41-
42-
ctx := context.Background()
43-
44-
skip.UnderRace(t, "multi-node, multi-tenant test too slow under race")
45-
skip.UnderDeadlock(t, "too slow under deadlock detector")
46-
params := base.TestClusterArgs{
47-
ServerArgs: base.TestServerArgs{
48-
DefaultTestTenant: base.TestControlsTenantsExplicitly,
49-
},
50-
}
51-
params.ServerArgs.Knobs.JobsTestingKnobs = jobs.NewTestingKnobsWithShortIntervals()
52-
tc, hostDB, _, cleanup := backupRestoreTestSetupWithParams(t, multiNode, 0, /* numAccounts */
53-
InitManualReplication, params)
54-
defer cleanup()
55-
56-
hostDB.Exec(t, "ALTER TENANT ALL SET CLUSTER SETTING server.sqlliveness.ttl='2s'")
57-
hostDB.Exec(t, "ALTER TENANT ALL SET CLUSTER SETTING server.sqlliveness.heartbeat='250ms'")
58-
59-
testTenantID := roachpb.MustMakeTenantID(11)
60-
tenantApp, tenantDB, err := tc.Server(0).TenantController().StartSharedProcessTenant(ctx,
61-
base.TestSharedProcessTenantArgs{
62-
TenantID: testTenantID,
63-
TenantName: "test",
64-
})
65-
require.NoError(t, err)
66-
67-
tc.GrantTenantCapabilities(
68-
ctx, t, testTenantID,
69-
map[tenantcapabilitiespb.ID]string{tenantcapabilitiespb.CanUseNodelocalStorage: "true"})
70-
71-
tenantSQL := sqlutils.MakeSQLRunner(tenantDB)
72-
tenantSQL.Exec(t, "CREATE TABLE foo AS SELECT generate_series(1, 4000)")
73-
tenantSQL.Exec(t, "ALTER TABLE foo SPLIT AT VALUES (500), (1000), (1500), (2000), (2500), (3000)")
74-
tenantSQL.Exec(t, "ALTER TABLE foo SCATTER")
75-
76-
t.Log("waiting for SQL instances")
77-
waitStart := timeutil.Now()
78-
for i := 1; i < multiNode; i++ {
79-
testutils.SucceedsSoon(t, func() error {
80-
t.Logf("waiting for server %d", i)
81-
db, err := tc.Server(i).SystemLayer().SQLConnE(serverutils.DBName("cluster:test/defaultdb"))
82-
if err != nil {
83-
return err
84-
}
85-
return db.Ping()
86-
})
87-
}
88-
t.Logf("all SQL instances (took %s)", timeutil.Since(waitStart))
89-
90-
// Shut down a node.
91-
t.Log("shutting down server 2 (n3)")
92-
tc.StopServer(2)
93-
94-
// We use succeeds soon here since it still takes some time
95-
// for instance-based planning to recognize the downed node.
96-
sv := &tenantApp.ClusterSettings().SV
97-
padding := 10 * time.Second
98-
timeout := slbase.DefaultTTL.Get(sv) + slbase.DefaultHeartBeat.Get(sv) + padding
99-
testutils.SucceedsWithin(t, func() error {
100-
_, err := tenantDB.Exec("BACKUP INTO 'nodelocal://1/worker-failure'")
101-
return err
102-
}, timeout)
103-
}
104-
10534
func TestBackupTenantImportingTable(t *testing.T) {
10635
defer leaktest.AfterTest(t)()
10736
defer log.Scope(t).Close(t)

pkg/backup/datadriven_test.go

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -27,6 +27,7 @@ import (
2727
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
2828
"github.com/cockroachdb/cockroach/pkg/keys"
2929
"github.com/cockroachdb/cockroach/pkg/kv"
30+
"github.com/cockroachdb/cockroach/pkg/kv/bulk"
3031
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord"
3132
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
3233
"github.com/cockroachdb/cockroach/pkg/kv/kvserver"
@@ -45,6 +46,7 @@ import (
4546
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
4647
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
4748
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
49+
"github.com/cockroachdb/cockroach/pkg/util/admission"
4850
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
4951
"github.com/cockroachdb/datadriven"
5052
"github.com/cockroachdb/errors"
@@ -194,6 +196,12 @@ func (d *datadrivenTestState) addCluster(t *testing.T, cfg clusterCfg) error {
194196
closedts.SideTransportCloseInterval.Override(context.Background(), &settings.SV, 10*time.Millisecond)
195197
kvserver.RangeFeedRefreshInterval.Override(context.Background(), &settings.SV, 10*time.Millisecond)
196198
sql.TempObjectWaitInterval.Override(context.Background(), &settings.SV, time.Millisecond)
199+
// Disable AC yielding as these tests can run many in-process clusters at once
200+
// and overload the host. Generally overload would mean bulk work, which only
201+
// uses strictly spare capacitym gets starved, but these tests expect it to
202+
// still run (just slowly, along with everything else).
203+
bulk.YieldIfNoPacer.Override(context.Background(), &settings.SV, false)
204+
admission.YieldInPacer.Override(context.Background(), &settings.SV, false)
197205
params.ServerArgs.Settings = settings
198206

199207
clusterSize := cfg.nodes

pkg/cli/cliflags/flags.go

Lines changed: 7 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -1825,12 +1825,13 @@ Labs support.
18251825
ZipIncludeGoroutineStacks = FlagInfo{
18261826
Name: "include-goroutine-stacks",
18271827
Description: `
1828-
Fetch stack traces for all goroutines running on each targeted node in nodes/*/stacks.txt
1829-
and nodes/*/stacks_with_labels.txt files. Note that fetching stack traces for all goroutines is
1830-
a "stop-the-world" operation, which can momentarily have negative impacts on SQL service
1831-
latency. Note that any periodic goroutine dumps previously taken on the node will still be
1832-
included in nodes/*/goroutines/*.txt.gz, as these would have already been generated and don't
1833-
require any additional stop-the-world operations to be collected.
1828+
Fetch full stack traces for all goroutines running on each targeted node in nodes/*/stacks.txt files.
1829+
Note that fetching text stack traces for all goroutines incurs a brief "stop-the-world" pause of each
1830+
node which can momentarily have negative impacts on SQL service latency. This flag only controls
1831+
collection of new full dump of all current goroutine stacks -- any previously recorded, periodic
1832+
goroutine dumps retained in the logs directories are still included (in nodes/*/goroutines/*.txt.gz)
1833+
and collection of aggregate counts of current goroutine stacks -- which does not incur a stop-the-world
1834+
pause -- remains enabled regardless of this flag's value.
18341835
`,
18351836
}
18361837

pkg/cli/context.go

Lines changed: 3 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -359,10 +359,9 @@ type zipContext struct {
359359
includeRangeInfo bool
360360

361361
// includeStacks fetches all goroutines running on each targeted node in
362-
// nodes/*/stacks.txt and nodes/*/stacks_with_labels.txt files. Note that
363-
// fetching stack traces for all goroutines is a temporary "stop the world"
364-
// operation, which can momentarily have negative impacts on SQL service
365-
// latency.
362+
// nodes/*/stacks.txt. Note that fetching debug=2 stack traces for all
363+
// goroutines incurs a temporary "stop the world" operation, which can
364+
// momentarily have negative impacts on SQL service latency.
366365
includeStacks bool
367366

368367
// includeRunningJobTraces includes the active traces of each running

pkg/cli/testdata/zip/testzip_exclude_goroutine_stacks

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -25,6 +25,7 @@ debug zip --concurrency=1 --cpu-profile-duration=1s --validate-zip-file=false --
2525
[node 1] requesting data for debug/nodes/1/details... received response... writing JSON output: debug/nodes/1/details.json... done
2626
[node 1] requesting data for debug/nodes/1/gossip... received response... writing JSON output: debug/nodes/1/gossip.json... done
2727
[node 1] Skipping fetching goroutine stacks. Enable via the --include-goroutine-stacks flag.
28+
[node 1] requesting stacks with labels... received response... writing binary output: debug/nodes/1/stacks_with_labels.txt... done
2829
[node 1] requesting heap profile... received response... writing binary output: debug/nodes/1/heap.pprof... done
2930
[node 1] requesting engine stats... received response... writing binary output: debug/nodes/1/lsm.txt... done
3031
[node 1] requesting heap profile list... received response... done

pkg/cli/zip_per_node.go

Lines changed: 27 additions & 21 deletions
Original file line numberDiff line numberDiff line change
@@ -579,6 +579,12 @@ func (zc *debugZipContext) getCurrentHeapProfile(
579579
func (zc *debugZipContext) getStackInformation(
580580
ctx context.Context, nodePrinter *zipReporter, id string, prefix string,
581581
) error {
582+
// zipCtx.includeStacks controls specifically the inclusion of stacks.txt, as
583+
// the debug=2 collection scheme has a "stop the world" implementation that
584+
// makes it potentially disruptive and thus has an option to skip it. This
585+
// option does _not_ need to skip collection of stack groups (debug=1) or the
586+
// binary goroutine profile (debug=3 or debug=0), as these do not have that
587+
// same stop-the-world disruption.
582588
if zipCtx.includeStacks {
583589
if zipCtx.files.shouldIncludeFile(stacksFileName) {
584590
var stacksData []byte
@@ -600,32 +606,32 @@ func (zc *debugZipContext) getStackInformation(
600606
} else {
601607
nodePrinter.info("skipping %s due to file filters", stacksFileName)
602608
}
609+
} else {
610+
nodePrinter.info("Skipping fetching goroutine stacks. Enable via the --%s flag.", cliflags.ZipIncludeGoroutineStacks.Name)
611+
}
603612

604-
var stacksDataWithLabels []byte
605-
if zipCtx.files.shouldIncludeFile(stacksWithLabelFileName) {
606-
s := nodePrinter.start("requesting stacks with labels")
607-
requestErr := zc.runZipFn(ctx, s,
608-
func(ctx context.Context) error {
609-
stacks, err := zc.status.Stacks(ctx, &serverpb.StacksRequest{
610-
NodeId: id,
611-
Type: serverpb.StacksType_GOROUTINE_STACKS_DEBUG_1,
612-
})
613-
if err == nil {
614-
stacksDataWithLabels = stacks.Data
615-
}
616-
return err
613+
var stacksDataWithLabels []byte
614+
if zipCtx.files.shouldIncludeFile(stacksWithLabelFileName) {
615+
s := nodePrinter.start("requesting stacks with labels")
616+
requestErr := zc.runZipFn(ctx, s,
617+
func(ctx context.Context) error {
618+
stacks, err := zc.status.Stacks(ctx, &serverpb.StacksRequest{
619+
NodeId: id,
620+
Type: serverpb.StacksType_GOROUTINE_STACKS_DEBUG_1,
617621
})
618-
if zipCtx.redact {
619-
stacksDataWithLabels = redactStackTrace(stacksDataWithLabels)
620-
}
621-
if err := zc.z.createRawOrError(s, prefix+"/"+stacksWithLabelFileName, stacksDataWithLabels, requestErr); err != nil {
622+
if err == nil {
623+
stacksDataWithLabels = stacks.Data
624+
}
622625
return err
623-
}
624-
} else {
625-
nodePrinter.info("skipping %s due to file filters", stacksWithLabelFileName)
626+
})
627+
if zipCtx.redact {
628+
stacksDataWithLabels = redactStackTrace(stacksDataWithLabels)
629+
}
630+
if err := zc.z.createRawOrError(s, prefix+"/"+stacksWithLabelFileName, stacksDataWithLabels, requestErr); err != nil {
631+
return err
626632
}
627633
} else {
628-
nodePrinter.info("Skipping fetching goroutine stacks. Enable via the --%s flag.", cliflags.ZipIncludeGoroutineStacks.Name)
634+
nodePrinter.info("skipping %s due to file filters", stacksWithLabelFileName)
629635
}
630636
return nil
631637
}

pkg/cmd/roachprod/cli/BUILD.bazel

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -30,7 +30,6 @@ go_library(
3030
"//pkg/roachprod/logger",
3131
"//pkg/roachprod/roachprodutil",
3232
"//pkg/roachprod/ssh",
33-
"//pkg/roachprod/ui",
3433
"//pkg/roachprod/vm",
3534
"//pkg/roachprod/vm/gce",
3635
"//pkg/util/envutil",

pkg/cmd/roachprod/cli/commands.go

Lines changed: 7 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -26,7 +26,6 @@ import (
2626
"github.com/cockroachdb/cockroach/pkg/roachprod/config"
2727
"github.com/cockroachdb/cockroach/pkg/roachprod/install"
2828
"github.com/cockroachdb/cockroach/pkg/roachprod/roachprodutil"
29-
"github.com/cockroachdb/cockroach/pkg/roachprod/ui"
3029
"github.com/cockroachdb/cockroach/pkg/roachprod/vm"
3130
"github.com/cockroachdb/cockroach/pkg/roachprod/vm/gce"
3231
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
@@ -547,15 +546,15 @@ hosts file.
547546
if listDetails {
548547
collated := filteredCloud.BadInstanceErrors()
549548

550-
// Sort by Error() value for stable output
551-
var errors ui.ErrorsByError
552-
for err := range collated {
553-
errors = append(errors, err)
549+
// Sort by error message for stable output
550+
var errMsgs []string
551+
for msg := range collated {
552+
errMsgs = append(errMsgs, msg)
554553
}
555-
sort.Sort(errors)
554+
sort.Strings(errMsgs)
556555

557-
for _, e := range errors {
558-
fmt.Printf("%s: %s\n", e, collated[e].Names())
556+
for _, msg := range errMsgs {
557+
fmt.Printf("%s: %s\n", msg, collated[msg].Names())
559558
}
560559
}
561560
}

pkg/kv/bulk/cpu_pacer.go

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -25,7 +25,8 @@ var cpuPacerRequestDuration = settings.RegisterDurationSetting(
2525
50*time.Millisecond,
2626
)
2727

28-
var yieldIfNoPacer = settings.RegisterBoolSetting(
28+
// YieldIfNoPacer is exported so it can be overridden in tests.
29+
var YieldIfNoPacer = settings.RegisterBoolSetting(
2930
settings.ApplicationLevel,
3031
"bulkio.elastic_cpu_control.always_yield.enabled",
3132
"if true, yield the CPU as needed even when time-based elastic pacing is not enabled",
@@ -38,7 +39,7 @@ func NewCPUPacer(ctx context.Context, db *kv.DB, setting *settings.BoolSetting)
3839
if db == nil || db.AdmissionPacerFactory == nil || !setting.Get(db.SettingsValues()) {
3940
log.Dev.Infof(ctx, "admission control is not configured to pace bulk ingestion")
4041

41-
if db != nil && yieldIfNoPacer.Get(db.SettingsValues()) {
42+
if db != nil && YieldIfNoPacer.Get(db.SettingsValues()) {
4243
// Return a Pacer that just yields.
4344
return &admission.Pacer{Yield: true}
4445
}

0 commit comments

Comments
 (0)