From fd85617b70259ca24b232d6a21442c0e8fe4bbc2 Mon Sep 17 00:00:00 2001 From: Miral Gadani Date: Thu, 1 Sep 2022 15:11:44 -0400 Subject: [PATCH] roachtest: return vm createoptions to allow computed values to be reported Release justification: test-only change Release note: none --- pkg/cmd/roachtest/cluster.go | 62 +++++++-------- pkg/cmd/roachtest/cluster/BUILD.bazel | 1 - .../roachtest/cluster/cluster_interface.go | 3 +- pkg/cmd/roachtest/test_runner.go | 75 ++++++++++++------- pkg/cmd/roachtest/test_test.go | 5 +- .../roachtest/tests/multitenant_fairness.go | 2 +- 6 files changed, 84 insertions(+), 64 deletions(-) diff --git a/pkg/cmd/roachtest/cluster.go b/pkg/cmd/roachtest/cluster.go index 3494afd1b8b2..9b6037405abf 100644 --- a/pkg/cmd/roachtest/cluster.go +++ b/pkg/cmd/roachtest/cluster.go @@ -807,9 +807,9 @@ func (f *clusterFactory) clusterMock(cfg clusterConfig) *clusterImpl { // NOTE: setTest() needs to be called before a test can use this cluster. func (f *clusterFactory) newCluster( ctx context.Context, cfg clusterConfig, setStatus func(string), teeOpt logger.TeeOptType, -) (*clusterImpl, error) { +) (*clusterImpl, *vm.CreateOpts, error) { if ctx.Err() != nil { - return nil, errors.Wrap(ctx.Err(), "newCluster") + return nil, nil, errors.Wrap(ctx.Err(), "newCluster") } if overrideFlagset != nil && overrideFlagset.Changed("nodes") { @@ -820,9 +820,9 @@ func (f *clusterFactory) newCluster( // For tests, use a mock cluster. c := f.clusterMock(cfg) if err := f.r.registerCluster(c); err != nil { - return nil, err + return nil, nil, err } - return c, nil + return c, nil, nil } if cfg.localCluster { @@ -845,7 +845,7 @@ func (f *clusterFactory) newCluster( // We must release the allocation because cluster creation is not possible at this point. cfg.alloc.Release() - return nil, err + return nil, nil, err } if cfg.spec.Cloud != spec.Local { providerOptsContainer.SetProviderOpts(cfg.spec.Cloud, providerOpts) @@ -899,18 +899,18 @@ func (f *clusterFactory) newCluster( err = roachprod.Create(ctx, l, cfg.username, cfg.spec.NodeCount, createVMOpts, providerOptsContainer) if err == nil { if err := f.r.registerCluster(c); err != nil { - return nil, err + return nil, nil, err } c.status("idle") l.Close() - return c, nil + return c, &createVMOpts, nil } if errors.HasType(err, (*roachprod.ClusterAlreadyExistsError)(nil)) { // If the cluster couldn't be created because it existed already, bail. // In reality when this is hit is when running with the `local` flag // or a destroy from the previous iteration failed. - return nil, err + return nil, nil, err } l.PrintfCtx(ctx, "cluster creation failed, cleaning up in case it was partially created: %s", err) @@ -922,7 +922,7 @@ func (f *clusterFactory) newCluster( if i >= maxAttempts { // Here we have to release the alloc, as we are giving up. cfg.alloc.Release() - return nil, err + return nil, nil, err } // Try again to create the cluster. } @@ -1098,13 +1098,13 @@ func (c *clusterImpl) Node(i int) option.NodeListOption { // FetchLogs downloads the logs from the cluster using `roachprod get`. // The logs will be placed in the test's artifacts dir. -func (c *clusterImpl) FetchLogs(ctx context.Context, t test.Test) error { +func (c *clusterImpl) FetchLogs(ctx context.Context, l *logger.Logger) error { if c.spec.NodeCount == 0 { // No nodes can happen during unit tests and implies nothing to do. return nil } - t.L().Printf("fetching logs\n") + l.Printf("fetching logs\n") c.status("fetching logs") // Don't hang forever if we can't fetch the logs. @@ -1115,14 +1115,14 @@ func (c *clusterImpl) FetchLogs(ctx context.Context, t test.Test) error { } if err := c.Get(ctx, c.l, "logs" /* src */, path /* dest */); err != nil { - t.L().Printf("failed to fetch logs: %v", err) + l.Printf("failed to fetch logs: %v", err) if ctx.Err() != nil { return errors.Wrap(err, "cluster.FetchLogs") } } if err := c.RunE(ctx, c.All(), "mkdir -p logs/redacted && ./cockroach debug merge-logs --redact logs/*.log > logs/redacted/combined.log"); err != nil { - t.L().Printf("failed to redact logs: %v", err) + l.Printf("failed to redact logs: %v", err) if ctx.Err() != nil { return err } @@ -1174,17 +1174,17 @@ func (c *clusterImpl) CopyRoachprodState(ctx context.Context) error { // the first available node. They can be visualized via: // // `COCKROACH_DEBUG_TS_IMPORT_FILE=tsdump.gob ./cockroach start-single-node --insecure --store=$(mktemp -d)` -func (c *clusterImpl) FetchTimeseriesData(ctx context.Context, t test.Test) error { +func (c *clusterImpl) FetchTimeseriesData(ctx context.Context, l *logger.Logger) error { return contextutil.RunWithTimeout(ctx, "fetch tsdata", 5*time.Minute, func(ctx context.Context) error { node := 1 for ; node <= c.spec.NodeCount; node++ { - db, err := c.ConnE(ctx, t.L(), node) + db, err := c.ConnE(ctx, l, node) if err == nil { err = db.Ping() db.Close() } if err != nil { - t.L().Printf("node %d not responding to SQL, trying next one", node) + l.Printf("node %d not responding to SQL, trying next one", node) continue } break @@ -1211,7 +1211,7 @@ func (c *clusterImpl) FetchTimeseriesData(ctx context.Context, t test.Test) erro ); err != nil { return errors.Wrap(err, "cluster.FetchTimeseriesData") } - db, err := c.ConnE(ctx, t.L(), node) + db, err := c.ConnE(ctx, l, node) if err != nil { return err } @@ -1247,13 +1247,13 @@ COCKROACH_DEBUG_TS_IMPORT_FILE=tsdump.gob cockroach start-single-node --insecure // FetchDebugZip downloads the debug zip from the cluster using `roachprod ssh`. // The logs will be placed in the test's artifacts dir. -func (c *clusterImpl) FetchDebugZip(ctx context.Context, t test.Test) error { +func (c *clusterImpl) FetchDebugZip(ctx context.Context, l *logger.Logger) error { if c.spec.NodeCount == 0 { // No nodes can happen during unit tests and implies nothing to do. return nil } - t.L().Printf("fetching debug zip\n") + l.Printf("fetching debug zip\n") c.status("fetching debug zip") // Don't hang forever if we can't fetch the debug zip. @@ -1274,7 +1274,7 @@ func (c *clusterImpl) FetchDebugZip(ctx context.Context, t test.Test) error { si := strconv.Itoa(i) cmd := []string{"./cockroach", "debug", "zip", "--exclude-files='*.log,*.txt,*.pprof'", "--url", "{pgurl:" + si + "}", zipName} if err := c.RunE(ctx, c.All(), cmd...); err != nil { - t.L().Printf("./cockroach debug zip failed: %v", err) + l.Printf("./cockroach debug zip failed: %v", err) if i < c.spec.NodeCount { continue } @@ -1394,14 +1394,14 @@ func (c *clusterImpl) FailOnReplicaDivergence(ctx context.Context, t *testImpl) // FetchDmesg grabs the dmesg logs if possible. This requires being able to run // `sudo dmesg` on the remote nodes. -func (c *clusterImpl) FetchDmesg(ctx context.Context, t test.Test) error { +func (c *clusterImpl) FetchDmesg(ctx context.Context, l *logger.Logger) error { if c.spec.NodeCount == 0 || c.IsLocal() { // No nodes can happen during unit tests and implies nothing to do. // Also, don't grab dmesg on local runs. return nil } - t.L().Printf("fetching dmesg\n") + l.Printf("fetching dmesg\n") c.status("fetching dmesg") // Don't hang forever. @@ -1425,7 +1425,7 @@ func (c *clusterImpl) FetchDmesg(ctx context.Context, t test.Test) error { if result.Err != nil { // Store `Run` errors to return later (after copying files from successful nodes). combinedDmesgError = errors.CombineErrors(combinedDmesgError, result.Err) - t.L().Printf("running dmesg failed on node %d: %v", result.Node, result.Err) + l.Printf("running dmesg failed on node %d: %v", result.Node, result.Err) } else { // Only run `Get` on successful nodes to avoid pseudo-failure on `Get` caused by an earlier failure on `Run`. successfulNodes = append(successfulNodes, int(result.Node)) @@ -1434,7 +1434,7 @@ func (c *clusterImpl) FetchDmesg(ctx context.Context, t test.Test) error { // Get dmesg files from successful nodes only. if err := c.Get(ctx, c.l, name /* src */, path /* dest */, successfulNodes); err != nil { - t.L().Printf("getting dmesg files failed: %v", err) + l.Printf("getting dmesg files failed: %v", err) return errors.Wrap(err, "cluster.FetchDmesg") } @@ -1445,14 +1445,14 @@ func (c *clusterImpl) FetchDmesg(ctx context.Context, t test.Test) error { // FetchJournalctl grabs the journalctl logs if possible. This requires being // able to run `sudo journalctl` on the remote nodes. -func (c *clusterImpl) FetchJournalctl(ctx context.Context, t test.Test) error { +func (c *clusterImpl) FetchJournalctl(ctx context.Context, l *logger.Logger) error { if c.spec.NodeCount == 0 || c.IsLocal() { // No nodes can happen during unit tests and implies nothing to do. // Also, don't grab journalctl on local runs. return nil } - t.L().Printf("fetching journalctl\n") + l.Printf("fetching journalctl\n") c.status("fetching journalctl") // Don't hang forever. @@ -1476,7 +1476,7 @@ func (c *clusterImpl) FetchJournalctl(ctx context.Context, t test.Test) error { if result.Err != nil { // Store `Run` errors to return later (after copying files from successful nodes). combinedJournalctlError = errors.CombineErrors(combinedJournalctlError, result.Err) - t.L().Printf("running journalctl failed on node %d: %v", result.Node, result.Err) + l.Printf("running journalctl failed on node %d: %v", result.Node, result.Err) } else { // Only run `Get` on successful nodes to avoid pseudo-failure on `Get` caused by an earlier failure on `Run`. successfulNodes = append(successfulNodes, int(result.Node)) @@ -1485,7 +1485,7 @@ func (c *clusterImpl) FetchJournalctl(ctx context.Context, t test.Test) error { // Get files from successful nodes only. if err := c.Get(ctx, c.l, name /* src */, path /* dest */, successfulNodes); err != nil { - t.L().Printf("getting files failed: %v", err) + l.Printf("getting files failed: %v", err) return errors.Wrap(err, "cluster.FetchJournalctl") } @@ -1495,7 +1495,7 @@ func (c *clusterImpl) FetchJournalctl(ctx context.Context, t test.Test) error { } // FetchCores fetches any core files on the cluster. -func (c *clusterImpl) FetchCores(ctx context.Context, t test.Test) error { +func (c *clusterImpl) FetchCores(ctx context.Context, l *logger.Logger) error { if c.spec.NodeCount == 0 || c.IsLocal() { // No nodes can happen during unit tests and implies nothing to do. // Also, don't grab dmesg on local runs. @@ -1507,11 +1507,11 @@ func (c *clusterImpl) FetchCores(ctx context.Context, t test.Test) error { // from having the cores, but we should push them straight into a temp // bucket on S3 instead. OTOH, the ROI of this may be low; I don't know // of a recent example where we've wanted the Core dumps. - t.L().Printf("skipped fetching cores\n") + l.Printf("skipped fetching cores\n") return nil } - t.L().Printf("fetching cores\n") + l.Printf("fetching cores\n") c.status("fetching cores") // Don't hang forever. The core files can be large, so we give a generous diff --git a/pkg/cmd/roachtest/cluster/BUILD.bazel b/pkg/cmd/roachtest/cluster/BUILD.bazel index 736dab20a95c..afd151fae747 100644 --- a/pkg/cmd/roachtest/cluster/BUILD.bazel +++ b/pkg/cmd/roachtest/cluster/BUILD.bazel @@ -13,7 +13,6 @@ go_library( deps = [ "//pkg/cmd/roachtest/option", "//pkg/cmd/roachtest/spec", - "//pkg/cmd/roachtest/test", "//pkg/roachprod/install", "//pkg/roachprod/logger", "//pkg/roachprod/prometheus", diff --git a/pkg/cmd/roachtest/cluster/cluster_interface.go b/pkg/cmd/roachtest/cluster/cluster_interface.go index a1d42c28fe32..75d44b367e8f 100644 --- a/pkg/cmd/roachtest/cluster/cluster_interface.go +++ b/pkg/cmd/roachtest/cluster/cluster_interface.go @@ -17,7 +17,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/option" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/spec" - "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test" "github.com/cockroachdb/cockroach/pkg/roachprod/install" "github.com/cockroachdb/cockroach/pkg/roachprod/logger" "github.com/cockroachdb/cockroach/pkg/roachprod/prometheus" @@ -130,7 +129,7 @@ type Cluster interface { ctx context.Context, l *logger.Logger, src, dest, branch string, node option.NodeListOption, ) error - FetchTimeseriesData(ctx context.Context, t test.Test) error + FetchTimeseriesData(ctx context.Context, l *logger.Logger) error RefetchCertsFromNode(ctx context.Context, node int) error StartGrafana(ctx context.Context, l *logger.Logger, promCfg *prometheus.Config) error diff --git a/pkg/cmd/roachtest/test_runner.go b/pkg/cmd/roachtest/test_runner.go index 9895e1f4dd8a..ad7ca19674c1 100644 --- a/pkg/cmd/roachtest/test_runner.go +++ b/pkg/cmd/roachtest/test_runner.go @@ -37,6 +37,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/internal/team" "github.com/cockroachdb/cockroach/pkg/roachprod/config" "github.com/cockroachdb/cockroach/pkg/roachprod/logger" + "github.com/cockroachdb/cockroach/pkg/roachprod/vm" "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/quotapool" @@ -343,7 +344,7 @@ func defaultClusterAllocator( alloc *quotapool.IntAlloc, artifactsDir string, wStatus *workerStatus, - ) (*clusterImpl, error) { + ) (*clusterImpl, *vm.CreateOpts, error) { wStatus.SetStatus("creating cluster") defer wStatus.SetStatus("") @@ -353,7 +354,7 @@ func defaultClusterAllocator( logPath := filepath.Join(artifactsDir, runnerLogsDir, "cluster-create", existingClusterName+".log") clusterL, err := logger.RootLogger(logPath, lopt.tee) if err != nil { - return nil, err + return nil, nil, err } defer clusterL.Close() opt := attachOpt{ @@ -364,10 +365,10 @@ func defaultClusterAllocator( lopt.l.PrintfCtx(ctx, "Attaching to existing cluster %s for test %s", existingClusterName, t.Name) c, err := attachToExistingCluster(ctx, existingClusterName, clusterL, t.Cluster, opt, r.cr) if err == nil { - return c, nil + return c, nil, nil } if !errors.Is(err, errClusterNotFound) { - return nil, err + return nil, nil, err } // Fall through to create new cluster with name override. lopt.l.PrintfCtx( @@ -397,7 +398,7 @@ type clusterAllocatorFn func( alloc *quotapool.IntAlloc, artifactsDir string, wStatus *workerStatus, -) (*clusterImpl, error) +) (*clusterImpl, *vm.CreateOpts, error) // runWorker runs tests in a loop until work is exhausted. // @@ -544,13 +545,14 @@ func (r *testRunner) runWorker( } } var clusterCreateErr error + var vmCreateOpts *vm.CreateOpts if !testToRun.canReuseCluster { // Create a new cluster if can't reuse or reuse attempt failed. // N.B. non-reusable cluster would have been destroyed above. wStatus.SetTest(nil /* test */, testToRun) wStatus.SetStatus("creating cluster") - c, clusterCreateErr = allocateCluster(ctx, testToRun.spec, testToRun.alloc, artifactsRootDir, wStatus) + c, vmCreateOpts, clusterCreateErr = allocateCluster(ctx, testToRun.spec, testToRun.alloc, artifactsRootDir, wStatus) if clusterCreateErr != nil { atomic.AddInt32(&r.numClusterErrs, 1) shout(ctx, l, stdout, "Unable to create (or reuse) cluster for test %s due to: %s.", @@ -606,7 +608,7 @@ func (r *testRunner) runWorker( // N.B. issue title is of the form "roachtest: ${t.spec.Name} failed" (see UnitTestFormatter). t.spec.Name = "cluster_creation" t.spec.Owner = registry.OwnerDevInf - r.maybePostGithubIssue(ctx, l, t, stdout, issueOutput) + r.maybePostGithubIssue(ctx, l, t, c, vmCreateOpts, stdout, issueOutput) // Restore test name and owner. t.spec.Name = oldName t.spec.Owner = oldOwner @@ -632,7 +634,7 @@ func (r *testRunner) runWorker( wStatus.SetTest(t, testToRun) wStatus.SetStatus("running test") - err = r.runTest(ctx, t, testToRun.runNum, testToRun.runCount, c, stdout, testL) + err = r.runTest(ctx, t, testToRun.runNum, testToRun.runCount, c, vmCreateOpts, stdout, testL) } if err != nil { @@ -742,6 +744,7 @@ func (r *testRunner) runTest( runNum int, runCount int, c *clusterImpl, + vmCreateOpts *vm.CreateOpts, stdout io.Writer, l *logger.Logger, ) error { @@ -799,7 +802,7 @@ func (r *testRunner) runTest( shout(ctx, l, stdout, "--- FAIL: %s (%s)\n%s", runID, durationStr, output) - r.maybePostGithubIssue(ctx, l, t, stdout, output) + r.maybePostGithubIssue(ctx, l, t, c, vmCreateOpts, stdout, output) } else { shout(ctx, l, stdout, "--- PASS: %s (%s)", runID, durationStr) // If `##teamcity[testFailed ...]` is not present before `##teamCity[testFinished ...]`, @@ -1001,7 +1004,7 @@ func (r *testRunner) teardownTest( c.FailOnReplicaDivergence(ctx, t) if timedOut || t.Failed() { - r.collectClusterArtifacts(ctx, c, t) + r.collectClusterArtifacts(ctx, c, t.L()) } }) @@ -1041,7 +1044,13 @@ func (r *testRunner) shouldPostGithubIssue(t test.Test) bool { } func (r *testRunner) maybePostGithubIssue( - ctx context.Context, l *logger.Logger, t test.Test, stdout io.Writer, output string, + ctx context.Context, + l *logger.Logger, + t test.Test, + c *clusterImpl, + vmCreateOpts *vm.CreateOpts, + stdout io.Writer, + output string, ) { if !r.shouldPostGithubIssue(t) { return @@ -1084,6 +1093,15 @@ func (r *testRunner) maybePostGithubIssue( roachtestParam("ssd"): fmt.Sprintf("%d", spec.Cluster.SSDs), } + // these params can be probabilistically set if requested + if vmCreateOpts != nil { + clusterParams[roachtestParam("fs")] = vmCreateOpts.SSDOpts.FileSystem + } + + if c != nil { + clusterParams[roachtestParam("encrypted")] = fmt.Sprintf("%v", c.encAtRest) + } + req := issues.PostRequest{ MentionOnCreate: mention, ProjectColumnID: projColID, @@ -1114,7 +1132,10 @@ func (r *testRunner) maybePostGithubIssue( } // TODO(tbg): nothing in this method should have the `t`; they should have a `Logger` only. -func (r *testRunner) collectClusterArtifacts(ctx context.Context, c *clusterImpl, t test.Test) { +// Maybe done +func (r *testRunner) collectClusterArtifacts( + ctx context.Context, c *clusterImpl, l *logger.Logger, +) { // NB: fetch the logs even when we have a debug zip because // debug zip can't ever get the logs for down nodes. // We only save artifacts for failed tests in CI, so this @@ -1123,32 +1144,32 @@ func (r *testRunner) collectClusterArtifacts(ctx context.Context, c *clusterImpl // below has problems. For example, `debug zip` is known to // hang sometimes at the time of writing, see: // https://github.com/cockroachdb/cockroach/issues/39620 - t.L().PrintfCtx(ctx, "collecting cluster logs") + l.PrintfCtx(ctx, "collecting cluster logs") // Do this before collecting logs to make sure the file gets // downloaded below. if err := saveDiskUsageToLogsDir(ctx, c); err != nil { - t.L().Printf("failed to fetch disk uage summary: %s", err) + l.Printf("failed to fetch disk uage summary: %s", err) } - if err := c.FetchLogs(ctx, t); err != nil { - t.L().Printf("failed to download logs: %s", err) + if err := c.FetchLogs(ctx, l); err != nil { + l.Printf("failed to download logs: %s", err) } - if err := c.FetchDmesg(ctx, t); err != nil { - t.L().Printf("failed to fetch dmesg: %s", err) + if err := c.FetchDmesg(ctx, l); err != nil { + l.Printf("failed to fetch dmesg: %s", err) } - if err := c.FetchJournalctl(ctx, t); err != nil { - t.L().Printf("failed to fetch journalctl: %s", err) + if err := c.FetchJournalctl(ctx, l); err != nil { + l.Printf("failed to fetch journalctl: %s", err) } - if err := c.FetchCores(ctx, t); err != nil { - t.L().Printf("failed to fetch cores: %s", err) + if err := c.FetchCores(ctx, l); err != nil { + l.Printf("failed to fetch cores: %s", err) } if err := c.CopyRoachprodState(ctx); err != nil { - t.L().Printf("failed to copy roachprod state: %s", err) + l.Printf("failed to copy roachprod state: %s", err) } - if err := c.FetchTimeseriesData(ctx, t); err != nil { - t.L().Printf("failed to fetch timeseries data: %s", err) + if err := c.FetchTimeseriesData(ctx, l); err != nil { + l.Printf("failed to fetch timeseries data: %s", err) } - if err := c.FetchDebugZip(ctx, t); err != nil { - t.L().Printf("failed to collect zip: %s", err) + if err := c.FetchDebugZip(ctx, l); err != nil { + l.Printf("failed to collect zip: %s", err) } } diff --git a/pkg/cmd/roachtest/test_test.go b/pkg/cmd/roachtest/test_test.go index 91c9771dee96..fe824377bae4 100644 --- a/pkg/cmd/roachtest/test_test.go +++ b/pkg/cmd/roachtest/test_test.go @@ -27,6 +27,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/spec" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test" "github.com/cockroachdb/cockroach/pkg/roachprod/logger" + "github.com/cockroachdb/cockroach/pkg/roachprod/vm" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/quotapool" "github.com/cockroachdb/cockroach/pkg/util/stop" @@ -98,8 +99,8 @@ func alwaysFailingClusterAllocator( alloc *quotapool.IntAlloc, artifactsDir string, wStatus *workerStatus, -) (*clusterImpl, error) { - return nil, errors.New("cluster creation failed") +) (*clusterImpl, *vm.CreateOpts, error) { + return nil, nil, errors.New("cluster creation failed") } func TestRunnerRun(t *testing.T) { diff --git a/pkg/cmd/roachtest/tests/multitenant_fairness.go b/pkg/cmd/roachtest/tests/multitenant_fairness.go index a5fe0fca414b..f447b93399f8 100644 --- a/pkg/cmd/roachtest/tests/multitenant_fairness.go +++ b/pkg/cmd/roachtest/tests/multitenant_fairness.go @@ -310,7 +310,7 @@ func runMultiTenantFairness( c.Run(ctx, c.Node(1), cmd) // get cluster timeseries data into artifacts - err := c.FetchTimeseriesData(ctx, t) + err := c.FetchTimeseriesData(ctx, t.L()) require.NoError(t, err) }