diff --git a/build/teamcity-local-roachtest.sh b/build/teamcity-local-roachtest.sh index 56d7d0afa2cc..e96930b527cd 100755 --- a/build/teamcity-local-roachtest.sh +++ b/build/teamcity-local-roachtest.sh @@ -24,8 +24,9 @@ tc_start_block "Run local roachtests" run build/builder.sh env \ COCKROACH_DEV_LICENSE="$COCKROACH_DEV_LICENSE" \ stdbuf -oL -eL \ - ./bin/roachtest run '(acceptance|kv/splits|cdc/bank)' \ + ./bin/roachtest run acceptance kv/splits cdc/bank \ --local \ + --parallelism=1 \ --cockroach "cockroach" \ --roachprod "bin/roachprod" \ --workload "bin/workload" \ diff --git a/pkg/cmd/roachtest/acceptance.go b/pkg/cmd/roachtest/acceptance.go index ab5d09bd55a8..dcb0f4c325c6 100644 --- a/pkg/cmd/roachtest/acceptance.go +++ b/pkg/cmd/roachtest/acceptance.go @@ -12,25 +12,15 @@ package main import ( "context" - "fmt" "time" - - "github.com/cockroachdb/cockroach/pkg/util/version" ) -func registerAcceptance(r *registry) { - // The acceptance tests all share a cluster and run sequentially. In - // local mode the acceptance tests should be configured to run within a - // minute or so as these tests are run on every merge to master. - +func registerAcceptance(r *testRegistry) { testCases := []struct { - name string - fn func(ctx context.Context, t *test, c *cluster) - skip string - // roachtest needs to be taught about MinVersion for subtests. - // See https://github.com/cockroachdb/cockroach/issues/36752. - // - // minVersion string + name string + fn func(ctx context.Context, t *test, c *cluster) + skip string + minVersion string }{ // Sorted. Please keep it that way. {name: "bank/cluster-recovery", fn: runBankClusterRecovery}, @@ -53,16 +43,11 @@ func registerAcceptance(r *registry) { {name: "gossip/locality-address", fn: runCheckLocalityIPAddress}, {name: "rapid-restart", fn: runRapidRestart}, {name: "status-server", fn: runStatusServer}, - { - name: "version-upgrade", - fn: runVersionUpgrade, - // NB: this is hacked back in below. - // minVersion: "v19.2.0", - }, + {name: "version-upgrade", fn: runVersionUpgrade, minVersion: "v19.1.0"}, } tags := []string{"default", "quick"} const numNodes = 4 - spec := testSpec{ + specTemplate := testSpec{ // NB: teamcity-post-failures.py relies on the acceptance tests // being named acceptance/ and will avoid posting a // blank issue for the "acceptance" parent test. Make sure to @@ -70,26 +55,18 @@ func registerAcceptance(r *registry) { // this naming scheme ever change (or issues such as #33519) // will be posted. Name: "acceptance", + Timeout: 10 * time.Minute, Tags: tags, Cluster: makeClusterSpec(numNodes), } for _, tc := range testCases { - tc := tc - minV := "v19.2.0-0" - if tc.name == "version-upgrade" && !r.buildVersion.AtLeast(version.MustParse(minV)) { - tc.skip = fmt.Sprintf("skipped on %s (want at least %s)", r.buildVersion, minV) + tc := tc // copy for closure + spec := specTemplate + spec.Name = specTemplate.Name + "/" + tc.name + spec.Run = func(ctx context.Context, t *test, c *cluster) { + tc.fn(ctx, t, c) } - spec.SubTests = append(spec.SubTests, testSpec{ - Skip: tc.skip, - Name: tc.name, - Timeout: 10 * time.Minute, - Tags: tags, - Run: func(ctx context.Context, t *test, c *cluster) { - c.Wipe(ctx) - tc.fn(ctx, t, c) - }, - }) + r.Add(spec) } - r.Add(spec) } diff --git a/pkg/cmd/roachtest/allocator.go b/pkg/cmd/roachtest/allocator.go index 9af73395ad83..a095e8237349 100644 --- a/pkg/cmd/roachtest/allocator.go +++ b/pkg/cmd/roachtest/allocator.go @@ -22,7 +22,7 @@ import ( "github.com/pkg/errors" ) -func registerAllocator(r *registry) { +func registerAllocator(r *testRegistry) { runAllocator := func(ctx context.Context, t *test, c *cluster, start int, maxStdDev float64) { const fixturePath = `gs://cockroach-fixtures/workload/tpch/scalefactor=10/backup` c.Put(ctx, cockroach, "./cockroach") @@ -45,10 +45,10 @@ func registerAllocator(r *registry) { m.Wait() // Start the remaining nodes to kick off upreplication/rebalancing. - c.Start(ctx, t, c.Range(start+1, c.nodes), args) + c.Start(ctx, t, c.Range(start+1, c.spec.NodeCount), args) c.Run(ctx, c.Node(1), `./workload init kv --drop`) - for node := 1; node <= c.nodes; node++ { + for node := 1; node <= c.spec.NodeCount; node++ { node := node // TODO(dan): Ideally, the test would fail if this queryload failed, // but we can't put it in monitor as-is because the test deadlocks. @@ -247,7 +247,7 @@ func waitForRebalance(ctx context.Context, l *logger, db *gosql.DB, maxStdDev fl } func runWideReplication(ctx context.Context, t *test, c *cluster) { - nodes := c.nodes + nodes := c.spec.NodeCount if nodes != 9 { t.Fatalf("9-node cluster required") } diff --git a/pkg/cmd/roachtest/backup.go b/pkg/cmd/roachtest/backup.go index 99341b955879..34cac7b9d7aa 100644 --- a/pkg/cmd/roachtest/backup.go +++ b/pkg/cmd/roachtest/backup.go @@ -20,7 +20,7 @@ import ( "github.com/pkg/errors" ) -func registerBackup(r *registry) { +func registerBackup(r *testRegistry) { backup2TBSpec := makeClusterSpec(10) r.Add(testSpec{ Name: fmt.Sprintf("backup2TB/%s", backup2TBSpec), @@ -100,7 +100,7 @@ func registerBackup(r *registry) { t.Status(`workload initialization`) cmd := fmt.Sprintf( "./workload init tpcc --warehouses=%d {pgurl:1-%d}", - warehouses, c.nodes, + warehouses, c.spec.NodeCount, ) c.Run(ctx, c.Node(1), cmd) @@ -122,7 +122,7 @@ func registerBackup(r *registry) { go func() { cmd := fmt.Sprintf( "./workload run tpcc --warehouses=%d {pgurl:1-%d}", - warehouses, c.nodes, + warehouses, c.spec.NodeCount, ) cmdDone <- c.RunE(ctx, c.Node(1), cmd) diff --git a/pkg/cmd/roachtest/bank.go b/pkg/cmd/roachtest/bank.go index 6b7e79733b54..f149bfc80a22 100644 --- a/pkg/cmd/roachtest/bank.go +++ b/pkg/cmd/roachtest/bank.go @@ -217,7 +217,7 @@ func (s *bankState) startChaosMonkey( // Don't begin the chaos monkey until all nodes are serving SQL connections. // This ensures that we don't test cluster initialization under chaos. - for i := 1; i <= c.nodes; i++ { + for i := 1; i <= c.spec.NodeCount; i++ { db := c.Conn(ctx, i) var res int err := db.QueryRowContext(ctx, `SELECT 1`).Scan(&res) @@ -296,9 +296,9 @@ func (s *bankState) startSplitMonkey(ctx context.Context, d time.Duration, c *cl defer s.waitGroup.Done() r := newRand() - nodes := make([]string, c.nodes) + nodes := make([]string, c.spec.NodeCount) - for i := 0; i < c.nodes; i++ { + for i := 0; i < c.spec.NodeCount; i++ { nodes[i] = strconv.Itoa(i + 1) } @@ -413,7 +413,7 @@ func (s *bankState) waitClientsStop( curRound, strings.Join(strCounts, ", ")) } else { newOutput = fmt.Sprintf("test finished, waiting for shutdown of %d clients", - c.nodes-doneClients) + c.spec.NodeCount-doneClients) } // This just stops the logs from being a bit too spammy. if newOutput != prevOutput { @@ -431,14 +431,14 @@ func runBankClusterRecovery(ctx context.Context, t *test, c *cluster) { // TODO(peter): Run for longer when !local. start := timeutil.Now() s := &bankState{ - errChan: make(chan error, c.nodes), + errChan: make(chan error, c.spec.NodeCount), deadline: start.Add(time.Minute), - clients: make([]bankClient, c.nodes), + clients: make([]bankClient, c.spec.NodeCount), } s.initBank(ctx, t, c) defer s.waitGroup.Wait() - for i := 0; i < c.nodes; i++ { + for i := 0; i < c.spec.NodeCount; i++ { s.clients[i].Lock() s.initClient(ctx, c, i+1) s.clients[i].Unlock() @@ -449,7 +449,7 @@ func runBankClusterRecovery(ctx context.Context, t *test, c *cluster) { rnd, seed := randutil.NewPseudoRand() t.l.Printf("monkey starts (seed %d)\n", seed) pickNodes := func() []int { - nodes := rnd.Perm(c.nodes)[:rnd.Intn(c.nodes)+1] + nodes := rnd.Perm(c.spec.NodeCount)[:rnd.Intn(c.spec.NodeCount)+1] for i := range nodes { nodes[i]++ } @@ -485,7 +485,7 @@ func runBankNodeRestart(ctx context.Context, t *test, c *cluster) { s.initBank(ctx, t, c) defer s.waitGroup.Wait() - clientIdx := c.nodes + clientIdx := c.spec.NodeCount client := &s.clients[0] client.db = c.Conn(ctx, clientIdx) @@ -515,14 +515,14 @@ func runBankNodeZeroSum(ctx context.Context, t *test, c *cluster) { start := timeutil.Now() s := &bankState{ - errChan: make(chan error, c.nodes), + errChan: make(chan error, c.spec.NodeCount), deadline: start.Add(time.Minute), - clients: make([]bankClient, c.nodes), + clients: make([]bankClient, c.spec.NodeCount), } s.initBank(ctx, t, c) defer s.waitGroup.Wait() - for i := 0; i < c.nodes; i++ { + for i := 0; i < c.spec.NodeCount; i++ { s.clients[i].Lock() s.initClient(ctx, c, i+1) s.clients[i].Unlock() @@ -551,14 +551,14 @@ func runBankZeroSumRestart(ctx context.Context, t *test, c *cluster) { start := timeutil.Now() s := &bankState{ - errChan: make(chan error, c.nodes), + errChan: make(chan error, c.spec.NodeCount), deadline: start.Add(time.Minute), - clients: make([]bankClient, c.nodes), + clients: make([]bankClient, c.spec.NodeCount), } s.initBank(ctx, t, c) defer s.waitGroup.Wait() - for i := 0; i < c.nodes; i++ { + for i := 0; i < c.spec.NodeCount; i++ { s.clients[i].Lock() s.initClient(ctx, c, i+1) s.clients[i].Unlock() @@ -568,7 +568,7 @@ func runBankZeroSumRestart(ctx context.Context, t *test, c *cluster) { rnd, seed := randutil.NewPseudoRand() c.l.Printf("monkey starts (seed %d)\n", seed) pickNodes := func() []int { - nodes := rnd.Perm(c.nodes)[:rnd.Intn(c.nodes)+1] + nodes := rnd.Perm(c.spec.NodeCount)[:rnd.Intn(c.spec.NodeCount)+1] for i := range nodes { nodes[i]++ } diff --git a/pkg/cmd/roachtest/canary.go b/pkg/cmd/roachtest/canary.go index e0f04ce0a8c4..af0a37fd9f82 100644 --- a/pkg/cmd/roachtest/canary.go +++ b/pkg/cmd/roachtest/canary.go @@ -154,7 +154,7 @@ func repeatRunWithBuffer( // repeatGitCloneE is the same function as c.GitCloneE but with an automatic // retry loop. func repeatGitCloneE( - ctx context.Context, c *cluster, src, dest, branch string, node nodeListOption, + ctx context.Context, l *logger, c *cluster, src, dest, branch string, node nodeListOption, ) error { var lastError error for attempt, r := 0, retry.StartWithCtx(ctx, canaryRetryOptions); r.Next(); { @@ -165,8 +165,8 @@ func repeatGitCloneE( return fmt.Errorf("test has failed") } attempt++ - c.l.Printf("attempt %d - clone %s", attempt, src) - lastError = c.GitCloneE(ctx, src, dest, branch, node) + l.Printf("attempt %d - clone %s", attempt, src) + lastError = c.GitClone(ctx, l, src, dest, branch, node) if lastError != nil { c.l.Printf("error - retrying: %s", lastError) continue diff --git a/pkg/cmd/roachtest/cancel.go b/pkg/cmd/roachtest/cancel.go index 0a5681106d38..ef10ec03c9cc 100644 --- a/pkg/cmd/roachtest/cancel.go +++ b/pkg/cmd/roachtest/cancel.go @@ -32,7 +32,7 @@ import ( // // Once DistSQL queries provide more testing knobs, these tests can likely be // replaced with unit tests. -func registerCancel(r *registry) { +func registerCancel(r *testRegistry) { runCancel := func(ctx context.Context, t *test, c *cluster, queries []string, warehouses int, useDistsql bool) { c.Put(ctx, cockroach, "./cockroach", c.All()) diff --git a/pkg/cmd/roachtest/cdc.go b/pkg/cmd/roachtest/cdc.go index aa7437dc6055..00c66d8d49e1 100644 --- a/pkg/cmd/roachtest/cdc.go +++ b/pkg/cmd/roachtest/cdc.go @@ -57,13 +57,13 @@ type cdcTestArgs struct { func cdcBasicTest(ctx context.Context, t *test, c *cluster, args cdcTestArgs) { // Skip the poller test on v19.2. After 19.2 is out, we should likely delete // the test entirely. - if !args.rangefeed && t.registry.buildVersion.Compare(version.MustParse(`v19.1.0-0`)) > 0 { + if !args.rangefeed && t.buildVersion.Compare(version.MustParse(`v19.1.0-0`)) > 0 { t.Skip("no poller in >= v19.2.0", "") } - crdbNodes := c.Range(1, c.nodes-1) - workloadNode := c.Node(c.nodes) - kafkaNode := c.Node(c.nodes) + crdbNodes := c.Range(1, c.spec.NodeCount-1) + workloadNode := c.Node(c.spec.NodeCount) + kafkaNode := c.Node(c.spec.NodeCount) c.Put(ctx, cockroach, "./cockroach") c.Put(ctx, workload, "./workload", workloadNode) c.Start(ctx, t, crdbNodes) @@ -237,7 +237,7 @@ func runCDCBank(ctx context.Context, t *test, c *cluster) { // spam. c.Run(ctx, c.All(), `mkdir -p logs`) - crdbNodes, workloadNode, kafkaNode := c.Range(1, c.nodes-1), c.Node(c.nodes), c.Node(c.nodes) + crdbNodes, workloadNode, kafkaNode := c.Range(1, c.spec.NodeCount-1), c.Node(c.spec.NodeCount), c.Node(c.spec.NodeCount) c.Put(ctx, cockroach, "./cockroach", crdbNodes) c.Put(ctx, workload, "./workload", workloadNode) c.Start(ctx, t, crdbNodes) @@ -460,7 +460,7 @@ func runCDCSchemaRegistry(ctx context.Context, t *test, c *cluster) { } } -func registerCDC(r *registry) { +func registerCDC(r *testRegistry) { useRangeFeed := true if r.buildVersion.Compare(version.MustParse(`v2.2.0-0`)) < 0 { // RangeFeed is not production ready in 2.1, so run the tests with the diff --git a/pkg/cmd/roachtest/clearrange.go b/pkg/cmd/roachtest/clearrange.go index ee418533a286..ea8522d1c4eb 100644 --- a/pkg/cmd/roachtest/clearrange.go +++ b/pkg/cmd/roachtest/clearrange.go @@ -18,7 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/timeutil" ) -func registerClearRange(r *registry) { +func registerClearRange(r *testRegistry) { for _, checks := range []bool{true, false} { checks := checks r.Add(testSpec{ @@ -27,7 +27,10 @@ func registerClearRange(r *registry) { // to <3:30h but it varies. Timeout: 5*time.Hour + 90*time.Minute, MinVersion: `v2.2.0`, - Cluster: makeClusterSpec(10), + // This test reformats a drive to ZFS, so we don't want it reused. + // TODO(andrei): Can the test itself reuse the cluster (under --count=2)? + // In other words, would a OnlyTagged("clearrange") policy be good? + Cluster: makeClusterSpec(10, reuseNone()), Run: func(ctx context.Context, t *test, c *cluster) { runClearRange(ctx, t, c, checks) }, diff --git a/pkg/cmd/roachtest/clock_jump_crash.go b/pkg/cmd/roachtest/clock_jump_crash.go index b69d0ec42d09..70315935aeda 100644 --- a/pkg/cmd/roachtest/clock_jump_crash.go +++ b/pkg/cmd/roachtest/clock_jump_crash.go @@ -21,13 +21,15 @@ import ( func runClockJump(ctx context.Context, t *test, c *cluster, tc clockJumpTestCase) { // Test with a single node so that the node does not crash due to MaxOffset // violation when injecting offset - if c.nodes != 1 { - t.Fatalf("Expected num nodes to be 1, got: %d", c.nodes) + if c.spec.NodeCount != 1 { + t.Fatalf("Expected num nodes to be 1, got: %d", c.spec.NodeCount) } t.Status("deploying offset injector") offsetInjector := newOffsetInjector(c) - offsetInjector.deploy(ctx) + if err := offsetInjector.deploy(ctx, t.l); err != nil { + t.Fatal(err) + } if err := c.RunE(ctx, c.Node(1), "test -x ./cockroach"); err != nil { c.Put(ctx, cockroach, "./cockroach", c.All()) @@ -35,7 +37,7 @@ func runClockJump(ctx context.Context, t *test, c *cluster, tc clockJumpTestCase c.Wipe(ctx) c.Start(ctx, t) - db := c.Conn(ctx, c.nodes) + db := c.Conn(ctx, c.spec.NodeCount) defer db.Close() if _, err := db.Exec( fmt.Sprintf( @@ -61,8 +63,8 @@ func runClockJump(ctx context.Context, t *test, c *cluster, tc clockJumpTestCase c.Start(ctx, t, c.Node(1)) } }() - defer offsetInjector.recover(ctx, c.nodes) - offsetInjector.offset(ctx, c.nodes, tc.offset) + defer offsetInjector.recover(ctx, c.spec.NodeCount) + offsetInjector.offset(ctx, c.spec.NodeCount, tc.offset) t.Status("validating health") aliveAfterOffset = isAlive(db) @@ -78,7 +80,7 @@ type clockJumpTestCase struct { aliveAfterOffset bool } -func makeClockJumpTests() testSpec { +func registerClockJumpTests(r *testRegistry) { testCases := []clockJumpTestCase{ { name: "large_forward_enabled", @@ -115,30 +117,17 @@ func makeClockJumpTests() testSpec { }, } - spec := testSpec{ - Name: "jump", - } - for i := range testCases { tc := testCases[i] - spec.SubTests = append(spec.SubTests, testSpec{ - Name: tc.name, + spec := testSpec{ + Name: "clock/jump/" + tc.name, + // These tests muck with NTP, therefore we don't want the cluster reused + // by others. + Cluster: makeClusterSpec(1, reuseTagged("offset-injector")), Run: func(ctx context.Context, t *test, c *cluster) { runClockJump(ctx, t, c, tc) }, - }) + } + r.Add(spec) } - - return spec -} - -func registerClock(r *registry) { - r.Add(testSpec{ - Name: "clock", - Cluster: makeClusterSpec(1), - SubTests: []testSpec{ - makeClockJumpTests(), - makeClockMonotonicTests(), - }, - }) } diff --git a/pkg/cmd/roachtest/clock_monotonic.go b/pkg/cmd/roachtest/clock_monotonic.go index 5b6cebcd64b4..35959b82c8bd 100644 --- a/pkg/cmd/roachtest/clock_monotonic.go +++ b/pkg/cmd/roachtest/clock_monotonic.go @@ -21,13 +21,15 @@ import ( func runClockMonotonicity(ctx context.Context, t *test, c *cluster, tc clockMonotonicityTestCase) { // Test with a single node so that the node does not crash due to MaxOffset // violation when introducing offset - if c.nodes != 1 { - t.Fatalf("Expected num nodes to be 1, got: %d", c.nodes) + if c.spec.NodeCount != 1 { + t.Fatalf("Expected num nodes to be 1, got: %d", c.spec.NodeCount) } t.Status("deploying offset injector") offsetInjector := newOffsetInjector(c) - offsetInjector.deploy(ctx) + if err := offsetInjector.deploy(ctx, t.l); err != nil { + t.Fatal(err) + } if err := c.RunE(ctx, c.Node(1), "test -x ./cockroach"); err != nil { c.Put(ctx, cockroach, "./cockroach", c.All()) @@ -35,7 +37,7 @@ func runClockMonotonicity(ctx context.Context, t *test, c *cluster, tc clockMono c.Wipe(ctx) c.Start(ctx, t) - db := c.Conn(ctx, c.nodes) + db := c.Conn(ctx, c.spec.NodeCount) defer db.Close() if _, err := db.Exec( fmt.Sprintf(`SET CLUSTER SETTING server.clock.persist_upper_bound_interval = '%v'`, @@ -62,12 +64,12 @@ func runClockMonotonicity(ctx context.Context, t *test, c *cluster, tc clockMono } // Stop cockroach node before recovering from clock offset as this clock // jump can crash the node. - c.Stop(ctx, c.Node(c.nodes)) + c.Stop(ctx, c.Node(c.spec.NodeCount)) t.l.Printf("recovering from injected clock offset") - offsetInjector.recover(ctx, c.nodes) + offsetInjector.recover(ctx, c.spec.NodeCount) - c.Start(ctx, t, c.Node(c.nodes)) + c.Start(ctx, t, c.Node(c.spec.NodeCount)) if !isAlive(db) { t.Fatal("Node unexpectedly crashed") } @@ -75,11 +77,11 @@ func runClockMonotonicity(ctx context.Context, t *test, c *cluster, tc clockMono // Inject a clock offset after stopping a node t.Status("stopping cockroach") - c.Stop(ctx, c.Node(c.nodes)) + c.Stop(ctx, c.Node(c.spec.NodeCount)) t.Status("injecting offset") - offsetInjector.offset(ctx, c.nodes, tc.offset) + offsetInjector.offset(ctx, c.spec.NodeCount, tc.offset) t.Status("starting cockroach post offset") - c.Start(ctx, t, c.Node(c.nodes)) + c.Start(ctx, t, c.Node(c.spec.NodeCount)) if !isAlive(db) { t.Fatal("Node unexpectedly crashed") @@ -114,7 +116,7 @@ type clockMonotonicityTestCase struct { expectIncreasingWallTime bool } -func makeClockMonotonicTests() testSpec { +func registerClockMonotonicTests(r *testRegistry) { testCases := []clockMonotonicityTestCase{ { name: "persistent", @@ -124,19 +126,17 @@ func makeClockMonotonicTests() testSpec { }, } - spec := testSpec{ - Name: "monotonic", - } - for i := range testCases { tc := testCases[i] - spec.SubTests = append(spec.SubTests, testSpec{ - Name: tc.name, + spec := testSpec{ + Name: "clock/monotonic/" + tc.name, + // These tests muck with NTP, therefor we don't want the cluster reused by + // others. + Cluster: makeClusterSpec(1, reuseTagged("offset-injector")), Run: func(ctx context.Context, t *test, c *cluster) { runClockMonotonicity(ctx, t, c, tc) }, - }) + } + r.Add(spec) } - - return spec } diff --git a/pkg/cmd/roachtest/clock_util.go b/pkg/cmd/roachtest/clock_util.go index fc2894cf5142..35b8e8049eaa 100644 --- a/pkg/cmd/roachtest/clock_util.go +++ b/pkg/cmd/roachtest/clock_util.go @@ -39,21 +39,35 @@ type offsetInjector struct { } // deploy installs ntp and downloads / compiles bumptime used to create a clock offset -func (oi *offsetInjector) deploy(ctx context.Context) { - if err := oi.c.RunE(ctx, oi.c.All(), "test -x ./bumptime"); err != nil { - oi.c.Install(ctx, oi.c.All(), "ntp") - oi.c.Install(ctx, oi.c.All(), "gcc") - oi.c.Run(ctx, oi.c.All(), "sudo", "service", "ntp", "stop") - oi.c.Run(ctx, - oi.c.All(), - "curl", - "-kO", - "https://raw.githubusercontent.com/cockroachdb/jepsen/master/cockroachdb/resources/bumptime.c", - ) - oi.c.Run(ctx, oi.c.All(), "gcc", "bumptime.c", "-o", "bumptime", "&&", "rm bumptime.c") +func (oi *offsetInjector) deploy(ctx context.Context, l *logger) error { + if err := oi.c.RunE(ctx, oi.c.All(), "test -x ./bumptime"); err == nil { + return nil } + if err := oi.c.Install(ctx, l, oi.c.All(), "ntp"); err != nil { + return err + } + if err := oi.c.Install(ctx, l, oi.c.All(), "gcc"); err != nil { + return err + } + if err := oi.c.RunL(ctx, l, oi.c.All(), "sudo", "service", "ntp", "stop"); err != nil { + return err + } + if err := oi.c.RunL(ctx, l, + oi.c.All(), + "curl", + "-kO", + "https://raw.githubusercontent.com/cockroachdb/jepsen/master/cockroachdb/resources/bumptime.c", + ); err != nil { + return err + } + if err := oi.c.RunL(ctx, l, + oi.c.All(), "gcc", "bumptime.c", "-o", "bumptime", "&&", "rm bumptime.c", + ); err != nil { + return err + } oi.deployed = true + return nil } // offset injects a offset of s into the node with the given nodeID diff --git a/pkg/cmd/roachtest/cluster.go b/pkg/cmd/roachtest/cluster.go index e72b021871f5..767ed0c29fdb 100644 --- a/pkg/cmd/roachtest/cluster.go +++ b/pkg/cmd/roachtest/cluster.go @@ -37,6 +37,7 @@ import ( "github.com/armon/circbuf" "github.com/cockroachdb/cockroach/pkg/util/contextutil" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/quotapool" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" _ "github.com/lib/pq" @@ -53,7 +54,6 @@ var ( roachprod string buildTag string clusterName string - clusterID string clusterWipe bool zonesF string teamCity bool @@ -185,18 +185,22 @@ func initBinaries() { } } -var interrupted int32 - type clusterRegistry struct { mu struct { syncutil.Mutex clusters map[string]*cluster + tagCount map[string]int + // savedClusters keeps track of clusters that have been saved for further + // debugging. Each cluster comes with a message about the test failure + // causing it to be saved for debugging. + savedClusters map[*cluster]string } } func newClusterRegistry() *clusterRegistry { cr := &clusterRegistry{} cr.mu.clusters = make(map[string]*cluster) + cr.mu.savedClusters = make(map[*cluster]string) return cr } @@ -212,30 +216,68 @@ func (r *clusterRegistry) registerCluster(c *cluster) error { func (r *clusterRegistry) unregisterCluster(c *cluster) bool { r.mu.Lock() + defer r.mu.Unlock() _, exists := r.mu.clusters[c.name] - if exists { - delete(r.mu.clusters, c.name) + if !exists { + panic(fmt.Sprintf("trying to unregister cluster not in registry: %s", c)) + } + delete(r.mu.clusters, c.name) + if c.tag != "" { + if _, ok := r.mu.tagCount[c.tag]; !ok { + panic(fmt.Sprintf("tagged cluster not accounted for: %s", c)) + } + r.mu.tagCount[c.tag]-- } - r.mu.Unlock() return exists } -// destroyAllClusters destroys all the clusters. It responds to context -// cancelation. -func (r *clusterRegistry) destroyAllClusters(ctx context.Context) { - // No new clusters can be created. - atomic.StoreInt32(&interrupted, 1) +func (r *clusterRegistry) countForTag(tag string) int { + r.mu.Lock() + defer r.mu.Unlock() + return r.mu.tagCount[tag] +} + +// markClusterAsSaved marks c such that it will not be destroyed by +// destroyAllClusters. +// msg is a message recording the reason why the cluster is being saved (i.e. +// generally a test failure error). +func (r *clusterRegistry) markClusterAsSaved(c *cluster, msg string) { + r.mu.Lock() + r.mu.savedClusters[c] = msg + r.mu.Unlock() +} + +// savedClusters returns the list of clusters that have been saved for +// debugging. +func (r *clusterRegistry) savedClusters() map[*cluster]string { + r.mu.Lock() + defer r.mu.Unlock() + res := make(map[*cluster]string, len(r.mu.savedClusters)) + for c, msg := range r.mu.savedClusters { + res[c] = msg + } + return res +} +// destroyAllClusters destroys all the clusters (except for "saved" ones) and +// blocks until they're destroyed. It responds to context cancelation by +// interrupting the waiting; the cluster destruction itself does not inherit the +// cancelation. +func (r *clusterRegistry) destroyAllClusters(ctx context.Context, l *logger) { // Fire off a goroutine to destroy all of the clusters. done := make(chan struct{}) go func() { defer close(done) var clusters []*cluster + savedClusters := make(map[*cluster]struct{}) r.mu.Lock() for _, c := range r.mu.clusters { clusters = append(clusters, c) } + for c := range r.mu.savedClusters { + savedClusters[c] = struct{}{} + } r.mu.Unlock() var wg sync.WaitGroup @@ -243,9 +285,11 @@ func (r *clusterRegistry) destroyAllClusters(ctx context.Context) { for _, c := range clusters { go func(c *cluster) { defer wg.Done() - // We don't close the logger here since the cluster may be still in use - // by a test, and so the logger might still be needed. - c.Destroy(ctx, dontCloseLogger) + if _, ok := savedClusters[c]; !ok { + // We don't close the logger here since the cluster may be still in use + // by a test, and so the logger might still be needed. + c.Destroy(ctx, dontCloseLogger, l) + } }(c) } @@ -340,6 +384,9 @@ func execCmd(ctx context.Context, l *logger, args ...string) error { return nil } +// execCmdWithBuffer executes the given command and returns its stdout/stderr +// output. If the return code is not 0, an error is also returned. +// l is used to log the command before running it. No output is logged. func execCmdWithBuffer(ctx context.Context, l *logger, args ...string) ([]byte, error) { l.Printf("> %s\n", strings.Join(args, " ")) cmd := exec.CommandContext(ctx, args[0], args[1:]...) @@ -528,23 +575,28 @@ func (n nodeListOption) String() string { type clusterSpec struct { NodeCount int // CPUs is the number of CPUs per node. - CPUs int - Zones string - Geo bool - Lifetime time.Duration + CPUs int + Zones string + Geo bool + Lifetime time.Duration + ReusePolicy clusterReusePolicy } func makeClusterSpec(nodeCount int, opts ...createOption) clusterSpec { - spec := clusterSpec{ - NodeCount: nodeCount, - } - cpu(4).apply(&spec) - for _, o := range opts { + spec := clusterSpec{NodeCount: nodeCount} + defaultOpts := []createOption{cpu(4), nodeLifetimeOption(12 * time.Hour), reuseAny()} + for _, o := range append(defaultOpts, opts...) { o.apply(&spec) } return spec } +func clustersCompatible(s1, s2 clusterSpec) bool { + s1.Lifetime = 0 + s2.Lifetime = 0 + return s1 == s2 +} + func (s clusterSpec) String() string { str := fmt.Sprintf("n%dcpu%d", s.NodeCount, s.CPUs) if s.Geo { @@ -643,15 +695,76 @@ func (o nodeLifetimeOption) apply(spec *clusterSpec) { spec.Lifetime = time.Duration(o) } +// clusterReusePolicy indicates what clusters a particular test can run on and +// who (if anybody) can reuse the cluster after the test has finished running +// (either passing or failing). See the individual policies for details. +// +// Only tests whose cluster spec matches can ever run on the same +// cluster, regardless of this policy. +// +// Clean clusters (freshly-created clusters or cluster on which a test with the +// Any policy ran) are accepted by all policies. +// +// Note that not all combinations of "what cluster can I accept" and "how am I +// soiling this cluster" can be expressed. For example, there's no way to +// express that I'll accept a cluster that was tagged a certain way but after me +// nobody else can reuse the cluster at all. +type clusterReusePolicy interface { + clusterReusePolicy() +} + +// reusePolicyAny means that only clean clusters are accepted and the cluster +// can be used by any other test (i.e. the cluster remains "clean"). +type reusePolicyAny struct{} + +// reusePolicyNone means that only clean clusters are accepted and the cluster +// cannot be reused afterwards. +type reusePolicyNone struct{} + +// reusePolicyTagged means that clusters left over by similarly-tagged tests are +// accepted in addition to clean cluster and, regardless of how the cluster +// started up, it will be tagged with the given tag at the end (so only +// similarly-tagged tests can use it afterwards). +// +// The idea is that a tag identifies a particular way in which a test is soiled, +// since it's common for groups of tests to mess clusters up in similar ways and +// to also be able to reset the cluster when the test starts. It's like a virus +// - if you carry it, you infect a clean host and can otherwise intermingle with +// other hosts that are already infected. Note that using this policy assumes +// that the way in which every test soils the cluster is idempotent. +type reusePolicyTagged struct{ tag string } + +func (reusePolicyAny) clusterReusePolicy() {} +func (reusePolicyNone) clusterReusePolicy() {} +func (reusePolicyTagged) clusterReusePolicy() {} + +type clusterReusePolicyOption struct { + p clusterReusePolicy +} + +func reuseAny() clusterReusePolicyOption { + return clusterReusePolicyOption{p: reusePolicyAny{}} +} +func reuseNone() clusterReusePolicyOption { + return clusterReusePolicyOption{p: reusePolicyNone{}} +} +func reuseTagged(tag string) clusterReusePolicyOption { + return clusterReusePolicyOption{p: reusePolicyTagged{tag: tag}} +} + +func (p clusterReusePolicyOption) apply(spec *clusterSpec) { + spec.ReusePolicy = p.p +} + // cluster provides an interface for interacting with a set of machines, // starting and stopping a cockroach cluster on a subset of those machines, and // running load generators and other operations on the machines. // -// A cluster is intended to be shared between all the subtests under a root. // A cluster is safe for concurrent use by multiple goroutines. type cluster struct { name string - nodes int + tag string + spec clusterSpec status func(...interface{}) t testI // r is the registry tracking this cluster. Destroying the cluster will @@ -667,13 +780,12 @@ type cluster struct { // enabled or disabled by options passed to Start. encryptDefault bool - // destroyState is nil when the cluster should not be destroyed (i.e. when - // Destroy should not be called) - for example it is set to nil for results of - // c.clone(). - // - // NB: destroyState is a pointer to allow for copying of this struct in - // cluster.clone(). - destroyState *destroyState + // destroyState contains state related to the cluster's destruction. + destroyState destroyState +} + +func (c *cluster) String() string { + return fmt.Sprintf("%s [tag:%s] (%d nodes)", c.name, c.tag, c.spec.NodeCount) } type destroyState struct { @@ -683,6 +795,10 @@ type destroyState struct { // If not set, Destroy() only wipes the cluster. owned bool + // alloc is set if owned is set. If set, it represents resources in a + // QuotaPool that need to be released when the cluster is destroyed. + alloc *quotapool.IntAlloc + mu struct { syncutil.Mutex loggerClosed bool @@ -690,6 +806,13 @@ type destroyState struct { // destroy a cluster. It is set once the destroy process starts. It it // closed when the destruction is complete. destroyed chan struct{} + // saved is set if this cluster should not be wiped or destroyed. It should + // be left alone for further debugging. This is kept in sync with the + // clusterRegistry which maintains a list of all saved clusters. + saved bool + // savedMsg records a message describing the reason why the cluster is being + // saved. + savedMsg string } } @@ -705,84 +828,161 @@ func (c *cluster) closeLogger() { } type clusterConfig struct { - // name must be empty if localCluster is specified. - name string - nodes clusterSpec + spec clusterSpec // artifactsDir is the path where log file will be stored. artifactsDir string localCluster bool - teeOpt teeOptType - user string useIOBarrier bool + alloc *quotapool.IntAlloc +} + +// clusterFactory is a creator of clusters. +type clusterFactory struct { + // namePrefix is prepended to all cluster names. + namePrefix string + // counter is incremented with every new cluster. It's used as part of the cluster's name. + // Accessed atomically. + counter uint64 + // The registry with whom all clustered will be registered. + r *clusterRegistry + // artifactsDir is the directory in which the cluster creation log file will be placed. + artifactsDir string + // sem is a semaphore throttling the creation of clusters (because AWS has + // ridiculous API calls limits). + sem chan struct{} +} + +func newClusterFactory( + user string, clustersID string, artifactsDir string, r *clusterRegistry, concurrentCreations int, +) *clusterFactory { + secs := timeutil.Now().Unix() + var prefix string + if clustersID != "" { + prefix = fmt.Sprintf("%s-%s-%d-", user, clustersID, secs) + } else { + prefix = fmt.Sprintf("%s-%d-", user, secs) + } + return &clusterFactory{ + sem: make(chan struct{}, concurrentCreations), + namePrefix: prefix, + artifactsDir: artifactsDir, + r: r, + } +} + +// acquireSem blocks until the semaphore allows a new cluster creation. The +// returned function needs to be called when cluster creation finished. +func (f *clusterFactory) acquireSem() func() { + f.sem <- struct{}{} + return f.releaseSem +} + +func (f *clusterFactory) releaseSem() { + <-f.sem } // newCluster creates a new roachprod cluster. // -// NOTE: setTest() needs to be called before a test can use this cluster. -// -// TODO(peter): Should set the lifetime of clusters to 2x the expected test -// duration. The default lifetime of 12h is too long for some tests and will be -// too short for others. +// setStatus is called with status messages indicating the stage of cluster +// creation. // -// TODO(peter): The nodes spec should really contain a nodeSpec per node. Need -// to figure out how to make that work with `roachprod create`. Perhaps one -// invocation of `roachprod create` per unique node-spec. Are there guarantees -// we're making here about the mapping of nodeSpecs to node IDs? -func newCluster( - ctx context.Context, l *logger, cfg clusterConfig, r *clusterRegistry, +// 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 teeOptType, ) (*cluster, error) { - if atomic.LoadInt32(&interrupted) == 1 { - return nil, fmt.Errorf("newCluster interrupted") + if ctx.Err() != nil { + return nil, errors.Wrap(ctx.Err(), "newCluster") } var name string if cfg.localCluster { - if cfg.name != "" { - log.Fatalf(ctx, "can't specify name %q with local flag", cfg.name) - } name = "local" // The roachprod tool understands this magic name. } else { - name = makeClusterName(cfg.user + "-" + cfg.name) + count := atomic.AddUint64(&f.counter, 1) + name = makeClusterName( + fmt.Sprintf("%s-%02d-%s", f.namePrefix, count, cfg.spec.String())) } - if cfg.nodes.NodeCount == 0 { + if cfg.spec.NodeCount == 0 { // For tests. Return the minimum that makes them happy. - return &cluster{ + c := &cluster{ + name: name, expiration: timeutil.Now().Add(24 * time.Hour), - }, nil + status: func(...interface{}) {}, + r: f.r, + } + if err := f.r.registerCluster(c); err != nil { + return nil, err + } + return c, nil } + exp := cfg.spec.expiration() + if cfg.localCluster { + // Local clusters never expire. + exp = timeutil.Now().Add(100000 * time.Hour) + } c := &cluster{ name: name, - nodes: cfg.nodes.NodeCount, + spec: cfg.spec, status: func(...interface{}) {}, - l: l, - expiration: cfg.nodes.expiration(), + expiration: exp, encryptDefault: encrypt.asBool(), - r: r, - destroyState: &destroyState{ + r: f.r, + destroyState: destroyState{ owned: true, + alloc: cfg.alloc, }, } - if err := r.registerCluster(c); err != nil { - return nil, err - } - sargs := []string{roachprod, "create", c.name, "-n", fmt.Sprint(c.nodes)} - sargs = append(sargs, cfg.nodes.args()...) - if !local && zonesF != "" && cfg.nodes.Zones == "" { + sargs := []string{roachprod, "create", c.name, "-n", fmt.Sprint(c.spec.NodeCount)} + sargs = append(sargs, cfg.spec.args()...) + if !local && zonesF != "" && cfg.spec.Zones == "" { sargs = append(sargs, "--gce-zones="+zonesF) } if !cfg.useIOBarrier { sargs = append(sargs, "--local-ssd-no-ext4-barrier") } + setStatus("acquring cluster creation semaphore") + release := f.acquireSem() + defer release() + setStatus("roachprod create") c.status("creating cluster") - if err := execCmd(ctx, l, sargs...); err != nil { + + // Logs for creating a new cluster go to a dedicated log file. + logPath := filepath.Join(f.artifactsDir, "cluster-create", name+".log") + l, err := rootLogger(logPath, teeOpt) + if err != nil { + log.Fatal(ctx, err) + } + + success := false + // Attempt to create a cluster several times, cause them clouds be flaky that + // my phone says it's snowing. + for i := 0; i < 3; i++ { + err = execCmd(ctx, l, sargs...) + if err == nil { + success = true + break + } + l.PrintfCtx(ctx, "Failed to create cluster.") + if !strings.Contains(err.Error(), "already exists") { + l.PrintfCtx(ctx, "Cleaning up in case it was partially created.") + c.Destroy(ctx, closeLogger, l) + } else { + break + } + } + if !success { + return nil, err + } + + if err := f.r.registerCluster(c); err != nil { return nil, err } - c.status("running test") + c.status("idle") return c, nil } @@ -798,16 +998,20 @@ type attachOpt struct { // // NOTE: setTest() needs to be called before a test can use this cluster. func attachToExistingCluster( - ctx context.Context, name string, l *logger, nodes clusterSpec, opt attachOpt, r *clusterRegistry, + ctx context.Context, name string, l *logger, spec clusterSpec, opt attachOpt, r *clusterRegistry, ) (*cluster, error) { + exp := spec.expiration() + if name == "local" { + exp = timeutil.Now().Add(100000 * time.Hour) + } c := &cluster{ name: name, - nodes: nodes.NodeCount, + spec: spec, status: func(...interface{}) {}, l: l, - expiration: nodes.expiration(), + expiration: exp, encryptDefault: encrypt.asBool(), - destroyState: &destroyState{ + destroyState: destroyState{ // If we're attaching to an existing cluster, we're not going to destoy it. owned: false, }, @@ -819,7 +1023,7 @@ func attachToExistingCluster( } if !opt.skipValidation { - if err := c.validate(ctx, nodes, l); err != nil { + if err := c.validate(ctx, spec, l); err != nil { return nil, err } } @@ -831,7 +1035,7 @@ func attachToExistingCluster( } if !opt.skipWipe { if clusterWipe { - if err := c.WipeE(ctx, c.All()); err != nil { + if err := c.WipeE(ctx, l, c.All()); err != nil { return nil, err } } else { @@ -840,7 +1044,7 @@ func attachToExistingCluster( } } - c.status("running test") + c.status("idle") return c, nil } @@ -855,6 +1059,18 @@ func (c *cluster) setTest(t testI) { } } +// Save marks the cluster as "saved" so that it doesn't get destroyed. +func (c *cluster) Save(ctx context.Context, msg string, l *logger) { + l.PrintfCtx(ctx, "saving cluster %s for debugging (--debug specified)", c) + // TODO(andrei): should we extend the cluster here? For how long? + c.destroyState.alloc.Freeze() + c.r.markClusterAsSaved(c, msg) + c.destroyState.mu.Lock() + c.destroyState.mu.saved = true + c.destroyState.mu.savedMsg = msg + c.destroyState.mu.Unlock() +} + // validateCluster takes a cluster and checks that the reality corresponds to // the cluster's spec. It's intended to be used with clusters created by // attachToExistingCluster(); otherwise, clusters create with newCluster() are @@ -886,8 +1102,8 @@ func (c *cluster) validate(ctx context.Context, nodes clusterSpec, l *logger) er if !ok { return fmt.Errorf("cluster %q not found", c.name) } - if len(cDetails.VMs) < c.nodes { - return fmt.Errorf("cluster has %d nodes, test requires at least %d", len(cDetails.VMs), c.nodes) + if len(cDetails.VMs) < c.spec.NodeCount { + return fmt.Errorf("cluster has %d nodes, test requires at least %d", len(cDetails.VMs), c.spec.NodeCount) } if cpus := nodes.CPUs; cpus != 0 { for i, vm := range cDetails.VMs { @@ -900,31 +1116,15 @@ func (c *cluster) validate(ctx context.Context, nodes clusterSpec, l *logger) er return nil } -// clone creates a non-owned handle on the same cluster. -// -// NOTE: If the cloning has been done for a subtest, setTest() needs to be -// called on the returned cluster. -// -// TODO(andrei): Get rid of the funky concept of cloning once we implement a -// more principled aproach to wiping and destroying cluster. -func (c *cluster) clone() *cluster { - cpy := *c - // This cloned cluster is not taking ownership. The parent retains it. - cpy.destroyState = nil - - cpy.encryptDefault = encrypt.asBool() - return &cpy -} - // All returns a node list containing all of the nodes in the cluster. func (c *cluster) All() nodeListOption { - return c.Range(1, c.nodes) + return c.Range(1, c.spec.NodeCount) } // All returns a node list containing the nodes [begin,end]. func (c *cluster) Range(begin, end int) nodeListOption { - if begin < 1 || end > c.nodes { - c.t.Fatalf("invalid node range: %d-%d (1-%d)", begin, end, c.nodes) + if begin < 1 || end > c.spec.NodeCount { + c.t.Fatalf("invalid node range: %d-%d (1-%d)", begin, end, c.spec.NodeCount) } r := make(nodeListOption, 0, 1+end-begin) for i := begin; i <= end; i++ { @@ -941,7 +1141,7 @@ func (c *cluster) Node(i int) nodeListOption { // FetchLogs downloads the logs from the cluster using `roachprod get`. // The logs will be placed in the test's artifacts dir. func (c *cluster) FetchLogs(ctx context.Context) error { - if c.nodes == 0 { + if c.spec.NodeCount == 0 { // No nodes can happen during unit tests and implies nothing to do. return nil } @@ -963,7 +1163,7 @@ func (c *cluster) FetchLogs(ctx context.Context) error { // CopyRoachprodState copies the roachprod state directory in to the test // artifacts. func (c *cluster) CopyRoachprodState(ctx context.Context) error { - if c.nodes == 0 { + if c.spec.NodeCount == 0 { // No nodes can happen during unit tests and implies nothing to do. return nil } @@ -984,7 +1184,7 @@ func (c *cluster) CopyRoachprodState(ctx context.Context) error { // FetchDebugZip downloads the debug zip from the cluster using `roachprod ssh`. // The logs will be placed in the test's artifacts dir. func (c *cluster) FetchDebugZip(ctx context.Context) error { - if c.nodes == 0 { + if c.spec.NodeCount == 0 { // No nodes can happen during unit tests and implies nothing to do. return nil } @@ -1013,7 +1213,7 @@ func (c *cluster) FetchDebugZip(ctx context.Context) error { // FailOnDeadNodes fails the test if nodes that have a populated data dir are // found to be not running. It prints both to t.l and the test output. func (c *cluster) FailOnDeadNodes(ctx context.Context, t *test) { - if c.nodes == 0 { + if c.spec.NodeCount == 0 { // No nodes can happen during unit tests and implies nothing to do. return } @@ -1030,7 +1230,7 @@ func (c *cluster) FailOnDeadNodes(ctx context.Context, t *test) { // Don't fail if we timed out. return nil } - t.Fatalf("dead node detection: %s %s", err, output) + t.printfAndFail(0 /* skip */, "dead node detection: %s %s", err, output) } return nil }) @@ -1079,13 +1279,13 @@ WHERE t.status NOT IN ('RANGE_CONSISTENT', 'RANGE_INDETERMINATE')`) // replicas are inconsistent with each other. It uses the first node that // is up to run the query. func (c *cluster) FailOnReplicaDivergence(ctx context.Context, t *test) { - if c.nodes < 1 { + if c.spec.NodeCount < 1 { return // unit tests } // Find a live node to run against, if one exists. var db *gosql.DB - for i := 1; i <= c.nodes; i++ { + for i := 1; i <= c.spec.NodeCount; i++ { // Don't hang forever. if err := contextutil.RunWithTimeout( ctx, "find live node", 5*time.Second, @@ -1117,7 +1317,7 @@ func (c *cluster) FailOnReplicaDivergence(ctx context.Context, t *test) { // FetchDmesg grabs the dmesg logs if possible. This requires being able to run // `sudo dmesg` on the remote nodes. func (c *cluster) FetchDmesg(ctx context.Context) error { - if c.nodes == 0 || c.isLocal() { + 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 @@ -1148,7 +1348,7 @@ func (c *cluster) FetchDmesg(ctx context.Context) error { // FetchJournalctl grabs the journalctl logs if possible. This requires being // able to run `sudo journalctl` on the remote nodes. func (c *cluster) FetchJournalctl(ctx context.Context) error { - if c.nodes == 0 || c.isLocal() { + 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 @@ -1178,7 +1378,7 @@ func (c *cluster) FetchJournalctl(ctx context.Context) error { // FetchCores fetches any core files on the cluster. func (c *cluster) FetchCores(ctx context.Context) error { - if c.nodes == 0 || c.isLocal() { + 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 @@ -1205,34 +1405,48 @@ const ( // Destroy calls `roachprod destroy` or `roachprod wipe` on the cluster. // If called while another Destroy() or destroyInner() is in progress, the call // blocks until that first call finishes. -func (c *cluster) Destroy(ctx context.Context, lo closeLoggerOpt) { - if c.destroyState == nil { - c.l.Errorf("Destroy() called on cluster copy") +// +// If c.Save() had previously been called, then Destroy() will not actually +// touch the cluster. It might still close c.l, though. +// +// Cluster destruction errors are swallowed. +// +// lo specifies if c.l should be closed or not. If c.l may still be in use by a +// test (i.e. if this Destroy is happening because of a timeout or a signal), +// then we don't want to close the logger. +// l is the logger that will log this destroy operation. +// +// This method generally does not react to ctx cancelation. +func (c *cluster) Destroy(ctx context.Context, lo closeLoggerOpt, l *logger) { + if ctx.Err() != nil { return } - - if c.nodes == 0 { - // No nodes can happen during unit tests and implies nothing to do. + if c.spec.NodeCount == 0 { + // No nodes can happen during unit tests and implies not much to do. + c.r.unregisterCluster(c) return } - ch := c.doDestroy(ctx) + ch := c.doDestroy(ctx, l) <-ch // NB: Closing the logger without waiting on c.destroyState.destroyed above // would be bad because we might cause the ongoing `roachprod destroy` to fail // by closing its stdout/stderr. - if lo == closeLogger { + if lo == closeLogger && c.l != nil { c.closeLogger() } } -// doDestroy calls `roachprod destroy` or `roachprod wipe` on the cluster. It -// returns a chan that will be closed when the destruction complete. If there's -// no other doDestroy() in flight, the call is synchronous and the channel is -// closed upon return. -func (c *cluster) doDestroy(ctx context.Context) <-chan struct{} { +func (c *cluster) doDestroy(ctx context.Context, l *logger) <-chan struct{} { var inFlight <-chan struct{} c.destroyState.mu.Lock() + if c.destroyState.mu.saved { + // Nothing to do. Short-circuit. + c.destroyState.mu.Unlock() + ch := make(chan struct{}) + close(ch) + return ch + } if c.destroyState.mu.destroyed == nil { c.destroyState.mu.destroyed = make(chan struct{}) } else { @@ -1245,18 +1459,25 @@ func (c *cluster) doDestroy(ctx context.Context) <-chan struct{} { if clusterWipe { if c.destroyState.owned { + l.PrintfCtx(ctx, "destroying cluster %s...", c) c.status("destroying cluster") - if err := execCmd(ctx, c.l, roachprod, "destroy", c.name); err != nil { - c.l.Errorf("%s", err) + // We use a non-cancelable context for running this command. Once we got + // here, the cluster cannot be destroyed again, so we really want this command to succeed. + if err := execCmd(context.Background(), l, roachprod, "destroy", c.name); err != nil { + l.ErrorfCtx(ctx, "error destroying cluster %s: %s", c, err) + } else { + l.PrintfCtx(ctx, "destroying cluster %s... done", c) } + c.destroyState.alloc.Release() } else { + l.PrintfCtx(ctx, "wiping cluster %s", c) c.status("wiping cluster") - if err := execCmd(ctx, c.l, roachprod, "wipe", c.name); err != nil { - c.l.Errorf("%s", err) + if err := execCmd(ctx, l, roachprod, "wipe", c.name); err != nil { + l.Errorf("%s", err) } } } else { - c.l.Printf("skipping cluster wipe\n") + l.Printf("skipping cluster wipe\n") } c.r.unregisterCluster(c) c.destroyState.mu.Lock() @@ -1277,43 +1498,48 @@ func (c *cluster) LoggedCommand(ctx context.Context, arg0 string, args ...string } // Put a local file to all of the machines in a cluster. +// Put is DEPRECATED. Use PutE instead. func (c *cluster) Put(ctx context.Context, src, dest string, opts ...option) { - if c.t.Failed() { - // If the test has failed, don't try to limp along. - return + if err := c.PutE(ctx, c.l, src, dest, opts...); err != nil { + c.t.Fatal(err) } - if atomic.LoadInt32(&interrupted) == 1 { - c.t.Fatal("interrupted") +} + +// PutE puts a local file to all of the machines in a cluster. +func (c *cluster) PutE(ctx context.Context, l *logger, src, dest string, opts ...option) error { + if ctx.Err() != nil { + return errors.Wrap(ctx.Err(), "cluster.Put") } + c.status("uploading binary") + defer c.status("") + err := execCmd(ctx, c.l, roachprod, "put", c.makeNodes(opts...), src, dest) if err != nil { - c.t.Fatal(err) + return errors.Wrap(ctx.Err(), "cluster.Put") } + return nil } // Get gets files from remote hosts. -func (c *cluster) Get(ctx context.Context, src, dest string, opts ...option) { - if c.t.Failed() { - // If the test has failed, don't try to limp along. - return - } - if atomic.LoadInt32(&interrupted) == 1 { - c.t.Fatal("interrupted") +func (c *cluster) Get(ctx context.Context, l *logger, src, dest string, opts ...option) error { + if ctx.Err() != nil { + return errors.Wrap(ctx.Err(), "cluster.Get error") } c.status(fmt.Sprintf("getting %v", src)) - err := execCmd(ctx, c.l, roachprod, "get", c.makeNodes(opts...), src, dest) - if err != nil { - c.t.Fatal(err) - } + defer c.status("") + return errors.Wrap( + execCmd(ctx, l, roachprod, "get", c.makeNodes(opts...), src, dest), + "cluster.Get error") + } // Put a string into the specified file on the remote(s). func (c *cluster) PutString( ctx context.Context, l *logger, content, dest string, mode os.FileMode, opts ...option, ) error { - if atomic.LoadInt32(&interrupted) == 1 { - return errors.New("PutString: interrupted") + if ctx.Err() != nil { + return errors.Wrap(ctx.Err(), "cluster.PutString error") } c.status("uploading string") defer c.status("") @@ -1340,14 +1566,13 @@ func (c *cluster) PutString( return nil } -// GitCloneE clones a git repo from src into dest and checks out origin's +// GitClone clones a git repo from src into dest and checks out origin's // version of the given branch. The src, dest, and branch arguments must not -// contain shell special characters. GitCloneE unlike GitClone returns an -// error. -func (c *cluster) GitCloneE( - ctx context.Context, src, dest, branch string, node nodeListOption, +// contain shell special characters. +func (c *cluster) GitClone( + ctx context.Context, l *logger, src, dest, branch string, node nodeListOption, ) error { - return c.RunE(ctx, node, "bash", "-e", "-c", fmt.Sprintf(`' + return c.RunL(ctx, l, node, "bash", "-e", "-c", fmt.Sprintf(`' if ! test -d %s; then git clone -b %s --depth 1 %s %s else @@ -1361,15 +1586,6 @@ fi branch)) } -// GitClone clones a git repo from src into dest and checks out origin's -// version of the given branch. The src, dest, and branch arguments must not -// contain shell special characters. -func (c *cluster) GitClone(ctx context.Context, src, dest, branch string, node nodeListOption) { - if err := c.GitCloneE(ctx, src, dest, branch, node); err != nil { - c.t.Fatal(err) - } -} - // startArgs specifies extra arguments that are passed to `roachprod` during `c.Start`. func startArgs(extraArgs ...string) option { return roachprodArgOption(extraArgs) @@ -1411,8 +1627,8 @@ func roachprodArgs(opts []option) []string { // can either be a specific node, empty (to indicate all nodes), or a pair of // nodes indicating a range. func (c *cluster) StartE(ctx context.Context, opts ...option) error { - if atomic.LoadInt32(&interrupted) == 1 { - return fmt.Errorf("cluster.Start() interrupted") + if ctx.Err() != nil { + return errors.Wrap(ctx.Err(), "cluster.StartE") } // If the test failed (indicated by a canceled ctx), short-circuit. if ctx.Err() != nil { @@ -1450,7 +1666,7 @@ func argExists(args []string, target string) bool { // for a description of the nodes parameter. func (c *cluster) StopE(ctx context.Context, opts ...option) error { if ctx.Err() != nil { - return ctx.Err() + return errors.Wrap(ctx.Err(), "cluster.StopE") } args := []string{ roachprod, @@ -1458,9 +1674,6 @@ func (c *cluster) StopE(ctx context.Context, opts ...option) error { } args = append(args, roachprodArgs(opts)...) args = append(args, c.makeNodes(opts...)) - if atomic.LoadInt32(&interrupted) == 1 { - return fmt.Errorf("interrupted") - } c.status("stopping cluster") defer c.status() return execCmd(ctx, c.l, args...) @@ -1480,26 +1693,26 @@ func (c *cluster) Stop(ctx context.Context, opts ...option) { // WipeE wipes a subset of the nodes in a cluster. See cluster.Start() for a // description of the nodes parameter. -func (c *cluster) WipeE(ctx context.Context, opts ...option) error { +func (c *cluster) WipeE(ctx context.Context, l *logger, opts ...option) error { if ctx.Err() != nil { - return ctx.Err() + return errors.Wrap(ctx.Err(), "cluster.WipeE") } - if atomic.LoadInt32(&interrupted) == 1 { - return fmt.Errorf("interrupted") + if c.spec.NodeCount == 0 { + // For tests. + return nil } c.status("wiping cluster") defer c.status() - return execCmd(ctx, c.l, roachprod, "wipe", c.makeNodes(opts...)) + return execCmd(ctx, l, roachprod, "wipe", c.makeNodes(opts...)) } // Wipe is like WipeE, except instead of returning an error, it does // c.t.Fatal(). c.t needs to be set. func (c *cluster) Wipe(ctx context.Context, opts ...option) { - if c.t.Failed() { - // If the test has failed, don't try to limp along. + if ctx.Err() != nil { return } - if err := c.WipeE(ctx, opts...); err != nil { + if err := c.WipeE(ctx, c.l, opts...); err != nil { c.t.Fatal(err) } } @@ -1522,12 +1735,11 @@ func (c *cluster) Reformat(ctx context.Context, node nodeListOption, args ...str } // Install a package in a node -func (c *cluster) Install(ctx context.Context, node nodeListOption, args ...string) { - err := execCmd(ctx, c.l, +func (c *cluster) Install( + ctx context.Context, l *logger, node nodeListOption, args ...string, +) error { + return execCmd(ctx, l, append([]string{roachprod, "install", c.makeNodes(node), "--"}, args...)...) - if err != nil { - c.t.Fatal(err) - } } // RunE runs a command on the specified node, returning an error. @@ -1537,32 +1749,19 @@ func (c *cluster) RunE(ctx context.Context, node nodeListOption, args ...string) // RunL runs a command on the specified node, returning an error. func (c *cluster) RunL(ctx context.Context, l *logger, node nodeListOption, args ...string) error { - if err := c.preRunChecks(); err != nil { + if err := errors.Wrap(ctx.Err(), "cluster.RunL"); err != nil { return err } return execCmd(ctx, l, append([]string{roachprod, "run", c.makeNodes(node), "--"}, args...)...) } -// preRunChecks runs checks to see if it makes sense to run a command. -func (c *cluster) preRunChecks() error { - if c.t.Failed() { - // If the test has failed, don't try to limp along. - return errors.New("test already failed") - } - if atomic.LoadInt32(&interrupted) == 1 { - return errors.New("interrupted") - } - - return nil -} - // RunWithBuffer runs a command on the specified node, returning the resulting combined stderr // and stdout or an error. func (c *cluster) RunWithBuffer( ctx context.Context, l *logger, node nodeListOption, args ...string, ) ([]byte, error) { - if err := c.preRunChecks(); err != nil { + if err := errors.Wrap(ctx.Err(), "cluster.RunWithBuffer"); err != nil { return nil, err } return execCmdWithBuffer(ctx, l, @@ -1739,6 +1938,25 @@ func (c *cluster) isLocal() bool { return c.name == "local" } +// Extend extends the cluster's expiration by d, after truncating d to minute +// granularity. +func (c *cluster) Extend(ctx context.Context, d time.Duration, l *logger) error { + if ctx.Err() != nil { + return errors.Wrap(ctx.Err(), "cluster.Extend") + } + minutes := int(d.Minutes()) + l.PrintfCtx(ctx, "extending cluster by %d minutes", minutes) + if out, err := execCmdWithBuffer(ctx, l, roachprod, "extend", c.name, + fmt.Sprintf("--lifetime=%dm", minutes), + ); err != nil { + l.PrintfCtx(ctx, "roachprod extend failed: %s", out) + return errors.Wrap(err, "roachprod extend failed") + } + // Update c.expiration. Keep it under the real expiration. + c.expiration = c.expiration.Add(time.Duration((minutes - 1)) * time.Minute) + return nil +} + // getDiskUsageInBytes does what's on the tin. nodeIdx starts at one. func getDiskUsageInBytes( ctx context.Context, c *cluster, logger *logger, nodeIdx int, diff --git a/pkg/cmd/roachtest/cluster_init.go b/pkg/cmd/roachtest/cluster_init.go index 5207ad7e77ff..04d098e7d76a 100644 --- a/pkg/cmd/roachtest/cluster_init.go +++ b/pkg/cmd/roachtest/cluster_init.go @@ -49,7 +49,7 @@ func runClusterInit(ctx context.Context, t *test, c *cluster) { `--listen-addr=:{pgport:1} --http-port=$[{pgport:1}+1] `+ `> {log-dir}/cockroach.stdout 2> {log-dir}/cockroach.stderr`) }) - for i := 2; i <= c.nodes; i++ { + for i := 2; i <= c.spec.NodeCount; i++ { i := i g.Go(func() error { return c.RunE(ctx, c.Node(i), @@ -78,7 +78,7 @@ func runClusterInit(ctx context.Context, t *test, c *cluster) { func() { var g errgroup.Group - for i := 1; i <= c.nodes; i++ { + for i := 1; i <= c.spec.NodeCount; i++ { i := i g.Go(func() error { return c.RunE(ctx, c.Node(i), @@ -99,7 +99,7 @@ func runClusterInit(ctx context.Context, t *test, c *cluster) { // Wait for the servers to bind their ports. if err := retry.ForDuration(10*time.Second, func() error { - for i := 1; i <= c.nodes; i++ { + for i := 1; i <= c.spec.NodeCount; i++ { resp, err := http.Get(urlMap[i] + "/health") if err != nil { return err @@ -112,7 +112,7 @@ func runClusterInit(ctx context.Context, t *test, c *cluster) { } var dbs []*gosql.DB - for i := 1; i <= c.nodes; i++ { + for i := 1; i <= c.spec.NodeCount; i++ { db := c.Conn(ctx, i) defer db.Close() dbs = append(dbs, db) diff --git a/pkg/cmd/roachtest/cluster_test.go b/pkg/cmd/roachtest/cluster_test.go index 4eae1c1b1f39..2d04694bd7ea 100644 --- a/pkg/cmd/roachtest/cluster_test.go +++ b/pkg/cmd/roachtest/cluster_test.go @@ -25,7 +25,7 @@ import ( ) func TestClusterNodes(t *testing.T) { - c := &cluster{nodes: 10} + c := &cluster{spec: makeClusterSpec(10)} opts := func(opts ...option) []option { return opts } @@ -302,7 +302,7 @@ func TestLoadGroups(t *testing.T) { } { t.Run(fmt.Sprintf("%d/%d/%d", tc.numZones, tc.numRoachNodes, tc.numLoadNodes), func(t *testing.T) { - c := &cluster{t: testWrapper{t}, l: logger, nodes: tc.numRoachNodes + tc.numLoadNodes} + c := &cluster{t: testWrapper{t}, l: logger, spec: makeClusterSpec(tc.numRoachNodes + tc.numLoadNodes)} lg := makeLoadGroups(c, tc.numZones, tc.numRoachNodes, tc.numLoadNodes) require.EqualValues(t, lg, tc.loadGroups) }) diff --git a/pkg/cmd/roachtest/copy.go b/pkg/cmd/roachtest/copy.go index bbb51a5f0ce3..d83230d9bfc5 100644 --- a/pkg/cmd/roachtest/copy.go +++ b/pkg/cmd/roachtest/copy.go @@ -20,7 +20,7 @@ import ( "github.com/pkg/errors" ) -func registerCopy(r *registry) { +func registerCopy(r *testRegistry) { // This test imports a fully-populated Bank table. It then creates an empty // Bank schema. Finally, it performs a series of `INSERT ... SELECT ...` // statements to copy all data from the first table into the second table. diff --git a/pkg/cmd/roachtest/decommission.go b/pkg/cmd/roachtest/decommission.go index 3e8484639359..87d417afb68b 100644 --- a/pkg/cmd/roachtest/decommission.go +++ b/pkg/cmd/roachtest/decommission.go @@ -234,7 +234,7 @@ func runDecommission(t *test, c *cluster, nodes int, duration time.Duration) { } } -func registerDecommission(r *registry) { +func registerDecommission(r *testRegistry) { const numNodes = 4 duration := time.Hour diff --git a/pkg/cmd/roachtest/disk_full.go b/pkg/cmd/roachtest/disk_full.go index 27f303471004..47879627628f 100644 --- a/pkg/cmd/roachtest/disk_full.go +++ b/pkg/cmd/roachtest/disk_full.go @@ -19,7 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/timeutil" ) -func registerDiskFull(r *registry) { +func registerDiskFull(r *testRegistry) { r.Add(testSpec{ Name: "disk-full", MinVersion: `v2.1.0`, @@ -31,7 +31,7 @@ func registerDiskFull(r *registry) { return } - nodes := c.nodes - 1 + nodes := c.spec.NodeCount - 1 c.Put(ctx, cockroach, "./cockroach", c.Range(1, nodes)) c.Put(ctx, workload, "./workload", c.Node(nodes+1)) c.Start(ctx, t, c.Range(1, nodes)) diff --git a/pkg/cmd/roachtest/disk_stall.go b/pkg/cmd/roachtest/disk_stall.go index 4aaf0be63111..dcf3e3ea1fc2 100644 --- a/pkg/cmd/roachtest/disk_stall.go +++ b/pkg/cmd/roachtest/disk_stall.go @@ -22,7 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/timeutil" ) -func registerDiskStalledDetection(r *registry) { +func registerDiskStalledDetection(r *testRegistry) { for _, affectsLogDir := range []bool{false, true} { for _, affectsDataDir := range []bool{false, true} { // Grab copies of the args because we'll pass them into a closure. diff --git a/pkg/cmd/roachtest/drop.go b/pkg/cmd/roachtest/drop.go index 6ecf9c8f0a90..f113a2bdf5a2 100644 --- a/pkg/cmd/roachtest/drop.go +++ b/pkg/cmd/roachtest/drop.go @@ -21,7 +21,7 @@ import ( _ "github.com/lib/pq" ) -func registerDrop(r *registry) { +func registerDrop(r *testRegistry) { // TODO(tschottdorf): rearrange all tests so that their synopses are available // via godoc and (some variation on) `roachtest run --help`. diff --git a/pkg/cmd/roachtest/election.go b/pkg/cmd/roachtest/election.go index ff451f8d7617..0c4de82bdeb5 100644 --- a/pkg/cmd/roachtest/election.go +++ b/pkg/cmd/roachtest/election.go @@ -17,7 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/timeutil" ) -func registerElectionAfterRestart(r *registry) { +func registerElectionAfterRestart(r *testRegistry) { r.Add(testSpec{ Name: "election-after-restart", Skip: "https://github.com/cockroachdb/cockroach/issues/35047", diff --git a/pkg/cmd/roachtest/encryption.go b/pkg/cmd/roachtest/encryption.go index 7c56f9a89a47..e40941832507 100644 --- a/pkg/cmd/roachtest/encryption.go +++ b/pkg/cmd/roachtest/encryption.go @@ -17,12 +17,12 @@ import ( "github.com/pkg/errors" ) -func registerEncryption(r *registry) { +func registerEncryption(r *testRegistry) { // Note that no workload is run in this roachtest because kv roachtest // ideally runs with encryption turned on to see the performance impact and // to test the correctness of encryption at rest. runEncryption := func(ctx context.Context, t *test, c *cluster) { - nodes := c.nodes + nodes := c.spec.NodeCount c.Put(ctx, cockroach, "./cockroach", c.Range(1, nodes)) c.Start(ctx, t, c.Range(1, nodes), startArgs("--encrypt")) diff --git a/pkg/cmd/roachtest/event_log.go b/pkg/cmd/roachtest/event_log.go index e99b9d2394d4..edd8bfbbd673 100644 --- a/pkg/cmd/roachtest/event_log.go +++ b/pkg/cmd/roachtest/event_log.go @@ -87,9 +87,9 @@ func runEventLog(ctx context.Context, t *test, c *cluster) { if err := rows.Err(); err != nil { t.Fatal(err) } - if c.nodes != len(seenIds) { + if c.spec.NodeCount != len(seenIds) { return fmt.Errorf("expected %d node join messages, found %d: %v", - c.nodes, len(seenIds), seenIds) + c.spec.NodeCount, len(seenIds), seenIds) } return nil }) diff --git a/pkg/cmd/roachtest/flowable.go b/pkg/cmd/roachtest/flowable.go index 08b3fc0a0487..5f1e9db9c07e 100644 --- a/pkg/cmd/roachtest/flowable.go +++ b/pkg/cmd/roachtest/flowable.go @@ -19,7 +19,7 @@ var flowableReleaseTagRegex = regexp.MustCompile(`^flowable-(?P\d+)\.(?P< // This test runs Flowable test suite against a single cockroach node. -func registerFlowable(r *registry) { +func registerFlowable(r *testRegistry) { runFlowable := func( ctx context.Context, t *test, @@ -40,7 +40,7 @@ func registerFlowable(r *registry) { if err != nil { t.Fatal(err) } - c.l.Printf("Latest Flowable release is %s.", latestTag) + t.l.Printf("Latest Flowable release is %s.", latestTag) if err := repeatRunE( ctx, c, node, "update apt-get", `sudo apt-get -qq update`, @@ -66,6 +66,7 @@ func registerFlowable(r *registry) { if err := repeatGitCloneE( ctx, + t.l, c, "https://github.com/flowable/flowable-engine.git", "/mnt/data1/flowable-engine", diff --git a/pkg/cmd/roachtest/follower_reads.go b/pkg/cmd/roachtest/follower_reads.go index 89ec17338de5..19cbf6604932 100644 --- a/pkg/cmd/roachtest/follower_reads.go +++ b/pkg/cmd/roachtest/follower_reads.go @@ -27,7 +27,7 @@ import ( "golang.org/x/sync/errgroup" ) -func registerFollowerReads(r *registry) { +func registerFollowerReads(r *testRegistry) { r.Add(testSpec{ Name: "follower-reads/nodes=3", Cluster: makeClusterSpec(3 /* nodeCount */, cpu(2), geo()), @@ -55,13 +55,13 @@ func registerFollowerReads(r *registry) { // time are under 10ms which implies that no WAN RPCs occurred. // func runFollowerReadsTest(ctx context.Context, t *test, c *cluster) { - crdbNodes := c.Range(1, c.nodes) + crdbNodes := c.Range(1, c.spec.NodeCount) c.Put(ctx, cockroach, "./cockroach", crdbNodes) c.Wipe(ctx, crdbNodes) c.Start(ctx, t, crdbNodes) var conns []*gosql.DB - for i := 0; i < c.nodes; i++ { + for i := 0; i < c.spec.NodeCount; i++ { conns = append(conns, c.Conn(ctx, i+1)) defer conns[i].Close() } @@ -160,7 +160,7 @@ func runFollowerReadsTest(ctx context.Context, t *test, c *cluster) { } // Verify error on immediate read. g, gCtx = errgroup.WithContext(ctx) - for i := 1; i <= c.nodes; i++ { + for i := 1; i <= c.spec.NodeCount; i++ { // Expect an error performing a historical read at first because the table // won't have been created yet. g.Go(verifySelect(gCtx, i, 0, true, 0)) @@ -188,7 +188,7 @@ func runFollowerReadsTest(ctx context.Context, t *test, c *cluster) { // Perform reads at follower_timestamp() and ensure we get the expected value. g, gCtx = errgroup.WithContext(ctx) k, v := chooseKV() - for i := 1; i <= c.nodes; i++ { + for i := 1; i <= c.spec.NodeCount; i++ { g.Go(verifySelect(gCtx, i, k, false, v)) } if err := g.Wait(); err != nil { @@ -216,7 +216,7 @@ func runFollowerReadsTest(ctx context.Context, t *test, c *cluster) { defer cancel() g, gCtx = errgroup.WithContext(timeoutCtx) for i := 0; i < concurrency; i++ { - g.Go(doSelects(gCtx, rand.Intn(c.nodes)+1)) + g.Go(doSelects(gCtx, rand.Intn(c.spec.NodeCount)+1)) } start := timeutil.Now() if err := g.Wait(); err != nil && timeoutCtx.Err() == nil { @@ -300,7 +300,7 @@ const followerReadsMetric = "follower_reads_success_count" // getFollowerReadCounts returns a slice from node to follower read count // according to the metric. func getFollowerReadCounts(ctx context.Context, c *cluster) ([]int, error) { - followerReadCounts := make([]int, c.nodes) + followerReadCounts := make([]int, c.spec.NodeCount) getFollowerReadCount := func(ctx context.Context, node int) func() error { return func() error { url := "http://" + c.ExternalAdminUIAddr(ctx, c.Node(node))[0] + "/_status/vars" @@ -329,7 +329,7 @@ func getFollowerReadCounts(ctx context.Context, c *cluster) ([]int, error) { } } g, gCtx := errgroup.WithContext(ctx) - for i := 1; i <= c.nodes; i++ { + for i := 1; i <= c.spec.NodeCount; i++ { g.Go(getFollowerReadCount(gCtx, i)) } if err := g.Wait(); err != nil { diff --git a/pkg/cmd/roachtest/gossip.go b/pkg/cmd/roachtest/gossip.go index bd772c7459f8..2e76553f7fd8 100644 --- a/pkg/cmd/roachtest/gossip.go +++ b/pkg/cmd/roachtest/gossip.go @@ -30,7 +30,7 @@ import ( "github.com/pkg/errors" ) -func registerGossip(r *registry) { +func registerGossip(r *testRegistry) { runGossipChaos := func(ctx context.Context, t *test, c *cluster) { c.Put(ctx, cockroach, "./cockroach", c.All()) c.Start(ctx, t, c.All()) @@ -57,7 +57,7 @@ SELECT string_agg(source_id::TEXT || ':' || target_id::TEXT, ',') gossipOK := func(start time.Time) bool { var expected string var initialized bool - for i := 1; i <= c.nodes; i++ { + for i := 1; i <= c.spec.NodeCount; i++ { if elapsed := timeutil.Since(start); elapsed >= 20*time.Second { t.Fatalf("gossip did not stabilize in %.1fs", elapsed.Seconds()) } @@ -148,7 +148,7 @@ type checkGossipFunc func(map[string]gossip.Info) error func (g *gossipUtil) check(ctx context.Context, c *cluster, f checkGossipFunc) error { return retry.ForDuration(g.waitTime, func() error { var infoStatus gossip.InfoStatus - for i := 1; i <= c.nodes; i++ { + for i := 1; i <= c.spec.NodeCount; i++ { url := g.urlMap[i] + `/_status/gossip/local` if err := httputil.GetJSON(http.Client{}, url, &infoStatus); err != nil { return errors.Wrapf(err, "failed to get gossip status from node %d", i) @@ -197,7 +197,7 @@ func (gossipUtil) hasClusterID(infos map[string]gossip.Info) error { func (g *gossipUtil) checkConnectedAndFunctional(ctx context.Context, t *test, c *cluster) { t.l.Printf("waiting for gossip to be connected\n") - if err := g.check(ctx, c, g.hasPeers(c.nodes)); err != nil { + if err := g.check(ctx, c, g.hasPeers(c.spec.NodeCount)); err != nil { t.Fatal(err) } if err := g.check(ctx, c, g.hasClusterID); err != nil { @@ -207,7 +207,7 @@ func (g *gossipUtil) checkConnectedAndFunctional(ctx context.Context, t *test, c t.Fatal(err) } - for i := 1; i <= c.nodes; i++ { + for i := 1; i <= c.spec.NodeCount; i++ { db := g.conn(ctx, i) defer db.Close() if i == 1 { @@ -251,7 +251,7 @@ func runGossipPeerings(ctx context.Context, t *test, c *cluster) { deadline := timeutil.Now().Add(time.Minute) for i := 1; timeutil.Now().Before(deadline); i++ { - if err := g.check(ctx, c, g.hasPeers(c.nodes)); err != nil { + if err := g.check(ctx, c, g.hasPeers(c.spec.NodeCount)); err != nil { t.Fatal(err) } if err := g.check(ctx, c, g.hasClusterID); err != nil { @@ -298,7 +298,7 @@ func runGossipRestartNodeOne(ctx context.Context, t *test, c *cluster) { args := startArgs("--env=COCKROACH_SCAN_MAX_IDLE_TIME=5ms", "--encrypt=false") c.Put(ctx, cockroach, "./cockroach") // Reduce the scan max idle time to speed up evacuation of node 1. - c.Start(ctx, t, racks(c.nodes), args) + c.Start(ctx, t, racks(c.spec.NodeCount), args) db := c.Conn(ctx, 1) defer db.Close() @@ -393,7 +393,7 @@ SELECT count(replicas) // Restart the other nodes. These nodes won't be able to talk to node 1 until // node 1 talks to it (they have out of date address info). Node 1 needs // incoming gossip info in order to determine where range 1 is. - c.Start(ctx, t, c.Range(2, c.nodes), args) + c.Start(ctx, t, c.Range(2, c.spec.NodeCount), args) // We need to override DB connection creation to use the correct port for // node 1. This is more complicated than it should be and a limitation of the @@ -435,9 +435,9 @@ SELECT count(replicas) func runCheckLocalityIPAddress(ctx context.Context, t *test, c *cluster) { c.Put(ctx, cockroach, "./cockroach") - externalIP := c.ExternalIP(ctx, c.Range(1, c.nodes)) + externalIP := c.ExternalIP(ctx, c.Range(1, c.spec.NodeCount)) - for i := 1; i <= c.nodes; i++ { + for i := 1; i <= c.spec.NodeCount; i++ { if local { externalIP[i-1] = "localhost" } @@ -449,7 +449,7 @@ func runCheckLocalityIPAddress(ctx context.Context, t *test, c *cluster) { rowCount := 0 - for i := 1; i <= c.nodes; i++ { + for i := 1; i <= c.spec.NodeCount; i++ { db := c.Conn(ctx, 1) defer db.Close() diff --git a/pkg/cmd/roachtest/hibernate.go b/pkg/cmd/roachtest/hibernate.go index 11ba92206460..d76a4277de58 100644 --- a/pkg/cmd/roachtest/hibernate.go +++ b/pkg/cmd/roachtest/hibernate.go @@ -24,7 +24,7 @@ var hibernateReleaseTagRegex = regexp.MustCompile(`^(?P\d+)\.(?P\d // This test runs hibernate-core's full test suite against a single cockroach // node. -func registerHibernate(r *registry) { +func registerHibernate(r *testRegistry) { runHibernate := func( ctx context.Context, t *test, @@ -71,6 +71,7 @@ func registerHibernate(r *registry) { if err := repeatGitCloneE( ctx, + t.l, c, "https://github.com/hibernate/hibernate-orm.git", "/mnt/data1/hibernate", diff --git a/pkg/cmd/roachtest/hotspotsplits.go b/pkg/cmd/roachtest/hotspotsplits.go index 1208f8c03174..dac267ce6be7 100644 --- a/pkg/cmd/roachtest/hotspotsplits.go +++ b/pkg/cmd/roachtest/hotspotsplits.go @@ -22,13 +22,13 @@ import ( "golang.org/x/sync/errgroup" ) -func registerHotSpotSplits(r *registry) { +func registerHotSpotSplits(r *testRegistry) { // This test sets up a cluster and runs kv on it with high concurrency and a large block size // to force a large range. We then make sure that the largest range isn't larger than a threshold and // that backpressure is working correctly. runHotSpot := func(ctx context.Context, t *test, c *cluster, duration time.Duration, concurrency int) { - roachNodes := c.Range(1, c.nodes-1) - appNode := c.Node(c.nodes) + roachNodes := c.Range(1, c.spec.NodeCount-1) + appNode := c.Node(c.spec.NodeCount) c.Put(ctx, cockroach, "./cockroach", roachNodes) c.Start(ctx, t, roachNodes) diff --git a/pkg/cmd/roachtest/import.go b/pkg/cmd/roachtest/import.go index 451b19763103..1d8609df970a 100644 --- a/pkg/cmd/roachtest/import.go +++ b/pkg/cmd/roachtest/import.go @@ -20,7 +20,7 @@ import ( "github.com/pkg/errors" ) -func registerImportTPCC(r *registry) { +func registerImportTPCC(r *testRegistry) { runImportTPCC := func(ctx context.Context, t *test, c *cluster, warehouses int) { c.Put(ctx, cockroach, "./cockroach") c.Put(ctx, workload, "./workload") @@ -107,7 +107,7 @@ func registerImportTPCC(r *registry) { }) } -func registerImportTPCH(r *registry) { +func registerImportTPCH(r *testRegistry) { for _, item := range []struct { nodes int timeout time.Duration diff --git a/pkg/cmd/roachtest/indexes.go b/pkg/cmd/roachtest/indexes.go index 2c7e1f65f7ca..1ff852018027 100644 --- a/pkg/cmd/roachtest/indexes.go +++ b/pkg/cmd/roachtest/indexes.go @@ -17,7 +17,7 @@ import ( "strings" ) -func registerNIndexes(r *registry, secondaryIndexes int) { +func registerNIndexes(r *testRegistry, secondaryIndexes int) { const nodes = 6 geoZones := []string{"us-west1-b", "us-east1-b", "us-central1-a"} geoZonesStr := strings.Join(geoZones, ",") @@ -66,11 +66,11 @@ func registerNIndexes(r *registry, secondaryIndexes int) { }) } -func registerIndexes(r *registry) { +func registerIndexes(r *testRegistry) { registerNIndexes(r, 2) } -func registerIndexesBench(r *registry) { +func registerIndexesBench(r *testRegistry) { for i := 0; i <= 10; i++ { registerNIndexes(r, i) } diff --git a/pkg/cmd/roachtest/interleavedpartitioned.go b/pkg/cmd/roachtest/interleavedpartitioned.go index fbcfc8d6b388..d726ce7a5a58 100644 --- a/pkg/cmd/roachtest/interleavedpartitioned.go +++ b/pkg/cmd/roachtest/interleavedpartitioned.go @@ -15,7 +15,7 @@ import ( "fmt" ) -func registerInterleaved(r *registry) { +func registerInterleaved(r *testRegistry) { type config struct { eastName string westName string diff --git a/pkg/cmd/roachtest/inverted_index.go b/pkg/cmd/roachtest/inverted_index.go index f8b65c3a58c6..8a032f56bf0b 100644 --- a/pkg/cmd/roachtest/inverted_index.go +++ b/pkg/cmd/roachtest/inverted_index.go @@ -18,7 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/timeutil" ) -func registerSchemaChangeInvertedIndex(r *registry) { +func registerSchemaChangeInvertedIndex(r *testRegistry) { r.Add(testSpec{ Name: "schemachange/invertedindex", Cluster: makeClusterSpec(5), @@ -31,8 +31,8 @@ func registerSchemaChangeInvertedIndex(r *registry) { // runInvertedIndex tests the correctness and performance of building an // inverted index on randomly generated JSON data (from the JSON workload). func runSchemaChangeInvertedIndex(ctx context.Context, t *test, c *cluster) { - crdbNodes := c.Range(1, c.nodes-1) - workloadNode := c.Node(c.nodes) + crdbNodes := c.Range(1, c.spec.NodeCount-1) + workloadNode := c.Node(c.spec.NodeCount) c.Put(ctx, cockroach, "./cockroach", crdbNodes) c.Put(ctx, workload, "./workload", workloadNode) @@ -55,7 +55,7 @@ func runSchemaChangeInvertedIndex(ctx context.Context, t *test, c *cluster) { cmdWrite := fmt.Sprintf( "./workload run json --read-percent=0 --duration %s {pgurl:1-%d} --batch 1000 --sequential", - initialDataDuration.String(), c.nodes-1, + initialDataDuration.String(), c.spec.NodeCount-1, ) m.Go(func(ctx context.Context) error { c.Run(ctx, workloadNode, cmdWrite) @@ -79,7 +79,7 @@ func runSchemaChangeInvertedIndex(ctx context.Context, t *test, c *cluster) { cmdWriteAndRead := fmt.Sprintf( "./workload run json --read-percent=50 --duration %s {pgurl:1-%d} --sequential", - indexDuration.String(), c.nodes-1, + indexDuration.String(), c.spec.NodeCount-1, ) m.Go(func(ctx context.Context) error { c.Run(ctx, workloadNode, cmdWriteAndRead) diff --git a/pkg/cmd/roachtest/jepsen.go b/pkg/cmd/roachtest/jepsen.go index 3db34ff50800..1aba96a74469 100644 --- a/pkg/cmd/roachtest/jepsen.go +++ b/pkg/cmd/roachtest/jepsen.go @@ -51,12 +51,17 @@ func initJepsen(ctx context.Context, t *test, c *cluster) { return } - controller := c.Node(c.nodes) - workers := c.Range(1, c.nodes-1) + controller := c.Node(c.spec.NodeCount) + workers := c.Range(1, c.spec.NodeCount-1) // Install jepsen. This part is fast if the repo is already there, // so do it before the initialization check for ease of iteration. - c.GitClone(ctx, "https://github.com/cockroachdb/jepsen", "/mnt/data1/jepsen", "tc-nightly", controller) + if err := c.GitClone( + ctx, t.l, + "https://github.com/cockroachdb/jepsen", "/mnt/data1/jepsen", "tc-nightly", controller, + ); err != nil { + t.Fatal(err) + } // Check to see if the cluster has already been initialized. if err := c.RunE(ctx, c.Node(1), "test -e jepsen_initialized"); err == nil { @@ -132,11 +137,11 @@ func initJepsen(ctx context.Context, t *test, c *cluster) { func runJepsen(ctx context.Context, t *test, c *cluster, testName, nemesis string) { initJepsen(ctx, t, c) - controller := c.Node(c.nodes) + controller := c.Node(c.spec.NodeCount) // Get the IP addresses for all our workers. var nodeFlags []string - for _, ip := range c.InternalIP(ctx, c.Range(1, c.nodes-1)) { + for _, ip := range c.InternalIP(ctx, c.Range(1, c.spec.NodeCount-1)) { nodeFlags = append(nodeFlags, "-n "+ip) } nodesStr := strings.Join(nodeFlags, " ") @@ -286,45 +291,32 @@ cd /mnt/data1/jepsen/cockroachdb && set -eo pipefail && \ } } -func registerJepsen(r *registry) { - // We're splitting the tests arbitrarily into a number of "batches" - top - // level tests. We do this so that we can different groups can run in parallel - // (as subtests don't run concurrently with each other). We put more than one - // test in a group so that Jepsen's lengthy cluster initialization step can be - // amortized (the individual tests are smart enough to not do it if it has - // been done already). - // +func registerJepsen(r *testRegistry) { // NB: the "comments" test is not included because it requires // linearizability. - // NB: the "multi-register" test takes about twice as long as the other - // tests, so it is included the group of two. - groups := [][]string{ - {"bank", "bank-multitable", "g2"}, - {"register", "sequential", "sets"}, - {"multi-register", "monotonic"}, - } - - for i := range groups { - spec := testSpec{ - Name: fmt.Sprintf("jepsen-batch%d", i+1), - Cluster: makeClusterSpec(6), - } - - for _, testName := range groups[i] { - testName := testName - sub := testSpec{Name: testName} - for _, nemesis := range jepsenNemeses { - nemesis := nemesis - sub.SubTests = append(sub.SubTests, testSpec{ - Name: nemesis.name, - Run: func(ctx context.Context, t *test, c *cluster) { - runJepsen(ctx, t, c, testName, nemesis.config) - }, - }) + tests := []string{ + "bank", "bank-multitable", "g2", + "monotonic", "register", "sequential", + "sets", "multi-register"} + for _, testName := range tests { + testName := testName + for _, nemesis := range jepsenNemeses { + nemesis := nemesis // copy for closure + spec := testSpec{ + Name: fmt.Sprintf("jepsen/%s/%s", testName, nemesis.name), + // The Jepsen tests do funky things to machines, like muck with the + // system clock; therefore, their clusters cannot be reused other tests + // except the Jepsen ones themselves which reset all this state when + // they start. It is important, however, that the Jepsen tests reuses + // clusters because they have a lengthy setup step, but avoid doing it + // if they detect that the machines have already been properly + // initialized. + Cluster: makeClusterSpec(6, reuseTagged("jepsen")), + Run: func(ctx context.Context, t *test, c *cluster) { + runJepsen(ctx, t, c, testName, nemesis.config) + }, } - spec.SubTests = append(spec.SubTests, sub) + r.Add(spec) } - - r.Add(spec) } } diff --git a/pkg/cmd/roachtest/kv.go b/pkg/cmd/roachtest/kv.go index c7bf58e1f3c4..84ab90fd1115 100644 --- a/pkg/cmd/roachtest/kv.go +++ b/pkg/cmd/roachtest/kv.go @@ -24,7 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/timeutil" ) -func registerKV(r *registry) { +func registerKV(r *testRegistry) { type kvOptions struct { nodes int cpus int @@ -35,7 +35,7 @@ func registerKV(r *registry) { sequential bool } runKV := func(ctx context.Context, t *test, c *cluster, opts kvOptions) { - nodes := c.nodes - 1 + nodes := c.spec.NodeCount - 1 c.Put(ctx, cockroach, "./cockroach", c.Range(1, nodes)) c.Put(ctx, workload, "./workload", c.Node(nodes+1)) c.Start(ctx, t, c.Range(1, nodes), startArgs(fmt.Sprintf("--encrypt=%t", opts.encryption))) @@ -149,7 +149,7 @@ func registerKV(r *registry) { } } -func registerKVContention(r *registry) { +func registerKVContention(r *testRegistry) { const nodes = 4 r.Add(testSpec{ Name: fmt.Sprintf("kv/contention/nodes=%d", nodes), @@ -200,13 +200,13 @@ func registerKVContention(r *registry) { }) } -func registerKVQuiescenceDead(r *registry) { +func registerKVQuiescenceDead(r *testRegistry) { r.Add(testSpec{ Name: "kv/quiescence/nodes=3", Cluster: makeClusterSpec(4), MinVersion: "v2.1.0", Run: func(ctx context.Context, t *test, c *cluster) { - nodes := c.nodes - 1 + nodes := c.spec.NodeCount - 1 c.Put(ctx, cockroach, "./cockroach", c.Range(1, nodes)) c.Put(ctx, workload, "./workload", c.Node(nodes+1)) c.Start(ctx, t, c.Range(1, nodes)) @@ -281,13 +281,13 @@ func registerKVQuiescenceDead(r *registry) { }) } -func registerKVGracefulDraining(r *registry) { +func registerKVGracefulDraining(r *testRegistry) { r.Add(testSpec{ Skip: "https://github.com/cockroachdb/cockroach/issues/33501", Name: "kv/gracefuldraining/nodes=3", Cluster: makeClusterSpec(4), Run: func(ctx context.Context, t *test, c *cluster) { - nodes := c.nodes - 1 + nodes := c.spec.NodeCount - 1 c.Put(ctx, cockroach, "./cockroach", c.Range(1, nodes)) c.Put(ctx, workload, "./workload", c.Node(nodes+1)) c.Start(ctx, t, c.Range(1, nodes)) @@ -392,7 +392,7 @@ func registerKVGracefulDraining(r *registry) { }) } -func registerKVSplits(r *registry) { +func registerKVSplits(r *testRegistry) { for _, item := range []struct { quiesce bool splits int @@ -415,7 +415,7 @@ func registerKVSplits(r *registry) { Timeout: item.timeout, Cluster: makeClusterSpec(4), Run: func(ctx context.Context, t *test, c *cluster) { - nodes := c.nodes - 1 + nodes := c.spec.NodeCount - 1 c.Put(ctx, cockroach, "./cockroach", c.Range(1, nodes)) c.Put(ctx, workload, "./workload", c.Node(nodes+1)) c.Start(ctx, t, c.Range(1, nodes), @@ -445,9 +445,9 @@ func registerKVSplits(r *registry) { } } -func registerKVScalability(r *registry) { +func registerKVScalability(r *testRegistry) { runScalability := func(ctx context.Context, t *test, c *cluster, percent int) { - nodes := c.nodes - 1 + nodes := c.spec.NodeCount - 1 c.Put(ctx, cockroach, "./cockroach", c.Range(1, nodes)) c.Put(ctx, workload, "./workload", c.Node(nodes+1)) diff --git a/pkg/cmd/roachtest/ledger.go b/pkg/cmd/roachtest/ledger.go index 75a1f2f38c6d..2548082300c9 100644 --- a/pkg/cmd/roachtest/ledger.go +++ b/pkg/cmd/roachtest/ledger.go @@ -15,7 +15,7 @@ import ( "fmt" ) -func registerLedger(r *registry) { +func registerLedger(r *testRegistry) { const nodes = 6 const azs = "us-central1-a,us-central1-b,us-central1-c" r.Add(testSpec{ diff --git a/pkg/cmd/roachtest/log.go b/pkg/cmd/roachtest/log.go index 05d2259fa552..dce350d0ac5b 100644 --- a/pkg/cmd/roachtest/log.go +++ b/pkg/cmd/roachtest/log.go @@ -11,12 +11,15 @@ package main import ( - "fmt" + "context" "io" "io/ioutil" "log" "os" "path/filepath" + + crdblog "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" ) // The flags used by the internal loggers. @@ -79,6 +82,11 @@ type logger struct { // They can be used directly by clients when a writer is required (e.g. when // piping output from a subcommand). stdout, stderr io.Writer + + mu struct { + syncutil.Mutex + closed bool + } } // newLogger constructs a new logger object. Not intended for direct @@ -159,12 +167,27 @@ func rootLogger(path string, teeOpt teeOptType) (*logger, error) { return cfg.newLogger(path) } +// close closes the logger. It is idempotent. func (l *logger) close() { + l.mu.Lock() + defer l.mu.Unlock() + if l.mu.closed { + return + } + l.mu.closed = true if l.file != nil { l.file.Close() + l.file = nil } } +// closed returns true if close() was previously called. +func (l *logger) closed() bool { + l.mu.Lock() + defer l.mu.Unlock() + return l.mu.closed +} + // ChildLogger constructs a new logger which logs to the specified file. The // prefix and teeing of stdout/stdout can be controlled by logger options. // If the parent logger was logging to a file, the new logger will log to a file @@ -207,18 +230,47 @@ func (l *logger) ChildLogger(name string, opts ...loggerOption) (*logger, error) return cfg.newLogger(path) } +// PrintfCtx prints a message to the logger's stdout. The context's log tags, if +// any, will be prepended to the message. A newline is appended if the last +// character is not already a newline. +func (l *logger) PrintfCtx(ctx context.Context, f string, args ...interface{}) { + l.PrintfCtxDepth(ctx, 2 /* depth */, f, args...) +} + +// Printf is like PrintfCtx, except it doesn't take a ctx and thus no log tags +// can be passed. func (l *logger) Printf(f string, args ...interface{}) { - if err := l.stdoutL.Output(2 /* calldepth */, fmt.Sprintf(f, args...)); err != nil { + l.PrintfCtxDepth(context.Background(), 2 /* depth */, f, args...) +} + +// PrintfCtxDepth is like PrintfCtx, except that it allows the caller to control +// which stack frame is reported as the file:line in the message. depth=1 is +// equivalent to PrintfCtx. E.g. pass 2 to ignore the caller's frame. +func (l *logger) PrintfCtxDepth(ctx context.Context, depth int, f string, args ...interface{}) { + msg := crdblog.MakeMessage(ctx, f, args) + if err := l.stdoutL.Output(depth+1, msg); err != nil { // Changing our interface to return an Error from a logging method seems too // onerous. Let's just crash. panic(err) } } -func (l *logger) Errorf(f string, args ...interface{}) { - if err := l.stderrL.Output(2 /* calldepth */, fmt.Sprintf(f, args...)); err != nil { +// ErrorfCtx is like PrintfCtx, except the logger outputs to its stderr. +func (l *logger) ErrorfCtx(ctx context.Context, f string, args ...interface{}) { + l.ErrorfCtxDepth(ctx, 2 /* depth */, f, args...) +} + +func (l *logger) ErrorfCtxDepth(ctx context.Context, depth int, f string, args ...interface{}) { + msg := crdblog.MakeMessage(ctx, f, args) + if err := l.stderrL.Output(depth+1, msg); err != nil { // Changing our interface to return an Error from a logging method seems too // onerous. Let's just crash. panic(err) } } + +// Errorf is like ErrorfCtx, except it doesn't take a ctx and thus no log tags +// can be passed. +func (l *logger) Errorf(f string, args ...interface{}) { + l.ErrorfCtxDepth(context.Background(), 2 /* depth */, f, args...) +} diff --git a/pkg/cmd/roachtest/main.go b/pkg/cmd/roachtest/main.go index d4f4eb2bd8ae..fe689c5ca25c 100644 --- a/pkg/cmd/roachtest/main.go +++ b/pkg/cmd/roachtest/main.go @@ -11,10 +11,14 @@ package main import ( + "context" "fmt" "math/rand" "os" + "os/signal" "os/user" + "path/filepath" + "time" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/spf13/cobra" @@ -24,9 +28,14 @@ func main() { rand.Seed(timeutil.Now().UnixNano()) username := os.Getenv("ROACHPROD_USER") parallelism := 10 + var cpuQuota int // Path to a local dir where the test logs and artifacts collected from // cluster will be placed. var artifacts string + var httpPort int + var debugEnabled bool + var clusterID string + var count = 1 cobra.EnableCommandSorting = false @@ -43,7 +52,10 @@ func main() { } if clusterName != "" && local { - return fmt.Errorf("cannot specify both an existing cluster (%s) and --local", clusterName) + return fmt.Errorf( + "cannot specify both an existing cluster (%s) and --local. However, if a local cluster "+ + "already exists, --clusters=local will use it", + clusterName) } switch cmd.Name() { case "run", "bench", "store-gen": @@ -54,7 +66,10 @@ func main() { } rootCmd.PersistentFlags().StringVarP( - &clusterName, "cluster", "c", "", "name of an existing cluster to use for running tests") + &clusterName, "cluster", "c", "", + "Comma-separated list of names existing cluster to use for running tests. "+ + "If fewer than --parallelism names are specified, then the parallelism "+ + "is capped to the number of clusters specified.") rootCmd.PersistentFlags().BoolVarP( &local, "local", "l", local, "run tests locally") rootCmd.PersistentFlags().StringVarP( @@ -92,14 +107,17 @@ Examples: roachtest list tag:weekly `, RunE: func(_ *cobra.Command, args []string) error { - r := newRegistry(setBuildVersion) + r, err := makeTestRegistry() + if err != nil { + return err + } if !listBench { - registerTests(r) + registerTests(&r) } else { - registerBenchmarks(r) + registerBenchmarks(&r) } - names := r.ListAll(args) + names := r.List(context.Background(), args) for _, name := range names { fmt.Println(name) } @@ -110,8 +128,10 @@ Examples: &listBench, "bench", false, "list benchmarks instead of tests") var runCmd = &cobra.Command{ - Use: "run [tests]", - Short: "run automated tests on cockroach cluster", + // Don't display usage when tests fail. + SilenceUsage: true, + Use: "run [tests]", + Short: "run automated tests on cockroach cluster", Long: `Run automated tests on existing or ephemeral cockroach clusters. roachtest run takes a list of regex patterns and runs all the matching tests. @@ -119,13 +139,16 @@ If no pattern is given, all tests are run. See "help list" for more details on the test tags. `, RunE: func(_ *cobra.Command, args []string) error { - if count <= 0 { - return fmt.Errorf("--count (%d) must by greater than 0", count) - } - r := newRegistry(setBuildVersion) - registerTests(r) - os.Exit(r.Run(args, parallelism, artifacts, getUser(username))) - return nil + return runTests(registerTests, cliCfg{ + args: args, + count: count, + cpuQuota: cpuQuota, + debugEnabled: debugEnabled, + httpPort: httpPort, + parallelism: parallelism, + artifactsDir: artifacts, + user: username, + }) }, } @@ -137,17 +160,22 @@ the test tags. &teamCity, "teamcity", false, "include teamcity-specific markers in output") var benchCmd = &cobra.Command{ - Use: "bench [benchmarks]", - Short: "run automated benchmarks on cockroach cluster", - Long: `Run automated benchmarks on existing or ephemeral cockroach clusters.`, + // Don't display usage when tests fail. + SilenceUsage: true, + Use: "bench [benchmarks]", + Short: "run automated benchmarks on cockroach cluster", + Long: `Run automated benchmarks on existing or ephemeral cockroach clusters.`, RunE: func(_ *cobra.Command, args []string) error { - if count <= 0 { - return fmt.Errorf("--count (%d) must by greater than 0", count) - } - r := newRegistry(setBuildVersion) - registerBenchmarks(r) - os.Exit(r.Run(args, parallelism, artifacts, getUser(username))) - return nil + return runTests(registerBenchmarks, cliCfg{ + args: args, + count: count, + cpuQuota: cpuQuota, + debugEnabled: debugEnabled, + httpPort: httpPort, + parallelism: parallelism, + artifactsDir: artifacts, + user: username, + }) }, } @@ -172,6 +200,11 @@ the test tags. "wipe existing cluster before starting test (for use with --cluster)") cmd.Flags().StringVar( &zonesF, "zones", "", "Zones for the cluster (use roachprod defaults if empty)") + cmd.Flags().IntVar( + &cpuQuota, "cpu-quota", 300, + "The number of cloud CPUs roachtest is allowed to use at any one time.") + cmd.Flags().IntVar( + &httpPort, "port", 8080, "the port on which to serve the HTTP interface") } rootCmd.AddCommand(listCmd) @@ -184,7 +217,85 @@ the test tags. } } -// user takes the value passed on the command line and comes up with the +type cliCfg struct { + args []string + count int + cpuQuota int + debugEnabled bool + httpPort int + parallelism int + artifactsDir string + user string +} + +func runTests(register func(*testRegistry), cfg cliCfg) error { + if cfg.count <= 0 { + return fmt.Errorf("--count (%d) must by greater than 0", cfg.count) + } + r, err := makeTestRegistry() + if err != nil { + return err + } + register(&r) + cr := newClusterRegistry() + runner := newTestRunner(cr, r.buildVersion) + + filter := newFilter(cfg.args) + clusterType := roachprodCluster + if local { + clusterType = localCluster + if cfg.parallelism != 1 { + fmt.Printf("--local specified. Overriding --parallelism to 1.\n") + cfg.parallelism = 1 + } + } + opt := clustersOpt{ + typ: clusterType, + clusterName: clusterName, + user: getUser(cfg.user), + cpuQuota: cfg.cpuQuota, + keepClustersOnTestFailure: cfg.debugEnabled, + } + if err := runner.runHTTPServer(cfg.httpPort, os.Stdout); err != nil { + return err + } + + tests := testsToRun(context.Background(), r, filter) + n := len(tests) + if n*cfg.count < cfg.parallelism { + // Don't spin up more workers than necessary. This has particular + // implications for the common case of running a single test once: if + // parallelism is set to 1, we'll use teeToStdout below to get logs to + // stdout/stderr. + cfg.parallelism = n * cfg.count + } + l, tee := testRunnerLogger(context.Background(), cfg.parallelism, cfg.artifactsDir) + lopt := loggingOpt{ + l: l, + tee: tee, + stdout: os.Stdout, + stderr: os.Stderr, + artifactsDir: cfg.artifactsDir, + } + + // We're going to run all the workers (and thus all the tests) in a context + // that gets canceled when the Interrupt signal is received. + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + CtrlC(ctx, l, cancel, cr) + err = runner.Run(ctx, tests, cfg.count, cfg.parallelism, opt, cfg.artifactsDir, lopt) + + // Make sure we attempt to clean up. We run with a non-canceled ctx; the + // ctx above might be canceled in case a signal was received. If that's + // the case, we're running under a 5s timeout until the CtrlC() goroutine + // kills the process. + l.PrintfCtx(ctx, "runTests destroying all clusters") + cr.destroyAllClusters(context.Background(), l) + return err + +} + +// getUser takes the value passed on the command line and comes up with the // username to use. func getUser(userFlag string) string { if userFlag != "" { @@ -196,3 +307,92 @@ func getUser(userFlag string) string { } return usr.Username } + +// CtrlC spawns a goroutine that sits around waiting for SIGINT. Once the first +// signal is received, it calls cancel(), waits 5 seconds, and then calls +// cr.destroyAllClusters(). The expectation is that the main goroutine will +// respond to the cancelation and return, and so the process will be dead by the +// time the 5s elapse. +// If a 2nd signal is received, it calls os.Exit(2). +func CtrlC(ctx context.Context, l *logger, cancel func(), cr *clusterRegistry) { + // Shut down test clusters when interrupted (for example CTRL-C). + sig := make(chan os.Signal, 1) + signal.Notify(sig, os.Interrupt) + go func() { + <-sig + shout(ctx, l, os.Stderr, + "Signaled received. Canceling workers and waiting up to 5s for them.") + // Signal runner.Run() to stop. + cancel() + <-time.After(5 * time.Second) + shout(ctx, l, os.Stderr, "5s elapsed. Will brutally destroy all clusters.") + // Make sure there are no leftover clusters. + destroyCh := make(chan struct{}) + go func() { + // Destroy all clusters. Don't wait more than 5 min for that though. + destroyCtx, cancel := context.WithTimeout(context.Background(), 5*time.Minute) + l.PrintfCtx(ctx, "CtrlC handler destroying all clusters") + cr.destroyAllClusters(destroyCtx, l) + cancel() + close(destroyCh) + }() + // If we get a second CTRL-C, exit immediately. + select { + case <-sig: + shout(ctx, l, os.Stderr, "Second SIGINT received. Quitting.") + os.Exit(2) + case <-destroyCh: + shout(ctx, l, os.Stderr, "Done destroying all clusters.") + } + }() +} + +// testRunnerLogger returns a logger to be used by the test runner and a tee +// option for the test logs. +func testRunnerLogger( + ctx context.Context, parallelism int, artifactsDir string, +) (*logger, teeOptType) { + teeOpt := noTee + if parallelism == 1 { + teeOpt = teeToStdout + } + + var l *logger + runnerLogPath := filepath.Join( + artifactsDir, fmt.Sprintf("test_runner-%d.log", timeutil.Now().Unix())) + if teeOpt == teeToStdout { + verboseCfg := loggerConfig{stdout: os.Stdout, stderr: os.Stderr} + var err error + l, err = verboseCfg.newLogger(runnerLogPath) + if err != nil { + panic(err) + } + } else { + verboseCfg := loggerConfig{} + var err error + l, err = verboseCfg.newLogger(runnerLogPath) + if err != nil { + panic(err) + } + } + shout(ctx, l, os.Stdout, "test runner logs in: %s", runnerLogPath) + return l, teeOpt +} + +func testsToRun(ctx context.Context, r testRegistry, filter *testFilter) []testSpec { + tests := r.GetTests(ctx, filter) + + var notSkipped []testSpec + for _, s := range tests { + if s.Skip == "" { + notSkipped = append(notSkipped, s) + } else { + if teamCity { + fmt.Fprintf(os.Stdout, "##teamcity[testIgnored name='%s' message='%s']\n", + s.Name, s.Skip) + } + fmt.Fprintf(os.Stdout, "--- SKIP: %s (%s)\n\t%s\n", s.Name, "0.00s", s.Skip) + } + } + return notSkipped +} diff --git a/pkg/cmd/roachtest/main_test.go b/pkg/cmd/roachtest/main_test.go deleted file mode 100644 index f27749fc386d..000000000000 --- a/pkg/cmd/roachtest/main_test.go +++ /dev/null @@ -1,21 +0,0 @@ -// Copyright 2018 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 main - -import ( - "os" - "testing" -) - -func TestMain(m *testing.M) { - postIssues = false - os.Exit(m.Run()) -} diff --git a/pkg/cmd/roachtest/network.go b/pkg/cmd/roachtest/network.go index 480fa31d5668..db530df121c3 100644 --- a/pkg/cmd/roachtest/network.go +++ b/pkg/cmd/roachtest/network.go @@ -84,7 +84,7 @@ insert into test.commit values(3,1000), (1,1000), (2,1000); select age, message from [ show trace for session ]; `) - for i := 1; i < origC.nodes; i++ { + for i := 1; i < origC.spec.NodeCount; i++ { if dur := c.Measure(ctx, i, `SELECT 1`); dur > latency { t.Fatalf("node %d unexpectedly affected by latency: select 1 took %.2fs", i, dur.Seconds()) } @@ -97,7 +97,8 @@ select age, message from [ show trace for session ]; } func runNetworkTPCC(ctx context.Context, t *test, origC *cluster, nodes int) { - serverNodes, workerNode := origC.Range(1, origC.nodes-1), origC.Node(origC.nodes) + n := origC.spec.NodeCount + serverNodes, workerNode := origC.Range(1, n-1), origC.Node(n) origC.Put(ctx, cockroach, "./cockroach", origC.All()) origC.Put(ctx, workload, "./workload", origC.All()) @@ -136,7 +137,7 @@ func runNetworkTPCC(ctx context.Context, t *test, origC *cluster, nodes int) { cmd := fmt.Sprintf( "./workload run tpcc --warehouses=%d --wait=false --histograms=logs/stats.json --duration=%s {pgurl:2-%d}", - warehouses, duration, c.nodes-1) + warehouses, duration, c.spec.NodeCount-1) return c.RunL(ctx, tpccL, workerNode, cmd) }) @@ -232,7 +233,7 @@ func runNetworkTPCC(ctx context.Context, t *test, origC *cluster, nodes int) { m.Wait() } -func registerNetwork(r *registry) { +func registerNetwork(r *testRegistry) { const numNodes = 4 r.Add(testSpec{ diff --git a/pkg/cmd/roachtest/psycopg.go b/pkg/cmd/roachtest/psycopg.go index 185c2a977fa3..8635be827698 100644 --- a/pkg/cmd/roachtest/psycopg.go +++ b/pkg/cmd/roachtest/psycopg.go @@ -25,7 +25,7 @@ var psycopgReleaseTagRegex = regexp.MustCompile(`^(?P\d+)(?:_(?P\d // This test runs psycopg full test suite against a single cockroach node. -func registerPsycopg(r *registry) { +func registerPsycopg(r *testRegistry) { runPsycopg := func( ctx context.Context, t *test, @@ -70,6 +70,7 @@ func registerPsycopg(r *registry) { if err := repeatGitCloneE( ctx, + t.l, c, "https://github.com/psycopg/psycopg2.git", "/mnt/data1/psycopg", diff --git a/pkg/cmd/roachtest/queue.go b/pkg/cmd/roachtest/queue.go index 4e0d1215a36d..b8e88f797bb5 100644 --- a/pkg/cmd/roachtest/queue.go +++ b/pkg/cmd/roachtest/queue.go @@ -19,7 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/timeutil" ) -func registerQueue(r *registry) { +func registerQueue(r *testRegistry) { // One node runs the workload generator, all other nodes host CockroachDB. const numNodes = 2 r.Add(testSpec{ @@ -33,8 +33,8 @@ func registerQueue(r *registry) { } func runQueue(ctx context.Context, t *test, c *cluster) { - dbNodeCount := c.nodes - 1 - workloadNode := c.nodes + dbNodeCount := c.spec.NodeCount - 1 + workloadNode := c.spec.NodeCount // Distribute programs to the correct nodes and start CockroachDB. c.Put(ctx, cockroach, "./cockroach", c.Range(1, dbNodeCount)) diff --git a/pkg/cmd/roachtest/rebalance_load.go b/pkg/cmd/roachtest/rebalance_load.go index 47b3743223f8..bcbcb48face7 100644 --- a/pkg/cmd/roachtest/rebalance_load.go +++ b/pkg/cmd/roachtest/rebalance_load.go @@ -22,7 +22,7 @@ import ( "golang.org/x/sync/errgroup" ) -func registerRebalanceLoad(r *registry) { +func registerRebalanceLoad(r *testRegistry) { // This test creates a single table for kv to use and splits the table to // have one range for every node in the cluster. Because even brand new // clusters start with 20+ ranges in them, the number of new ranges in kv's @@ -46,8 +46,8 @@ func registerRebalanceLoad(r *registry) { maxDuration time.Duration, concurrency int, ) { - roachNodes := c.Range(1, c.nodes-1) - appNode := c.Node(c.nodes) + roachNodes := c.Range(1, c.spec.NodeCount-1) + appNode := c.Node(c.spec.NodeCount) c.Put(ctx, cockroach, "./cockroach", roachNodes) args := startArgs( diff --git a/pkg/cmd/roachtest/registry.go b/pkg/cmd/roachtest/registry.go index 72a3f1b4d2ff..fb4936916a28 100644 --- a/pkg/cmd/roachtest/registry.go +++ b/pkg/cmd/roachtest/registry.go @@ -10,7 +10,7 @@ package main -func registerTests(r *registry) { +func registerTests(r *testRegistry) { // Helpful shell pipeline to generate the list below: // // grep -h -E 'func register[^(]+\(.*registry\) {' *.go | grep -E -o 'register[^(]+' | grep -E -v '^register(Tests|Benchmarks)$' | grep -v '^\w*Bench$' | sort -f | awk '{printf "\t%s(r)\n", $0}' @@ -21,7 +21,8 @@ func registerTests(r *registry) { registerCancel(r) registerCDC(r) registerClearRange(r) - registerClock(r) + registerClockJumpTests(r) + registerClockMonotonicTests(r) registerCopy(r) registerDecommission(r) registerDiskFull(r) @@ -74,7 +75,7 @@ func registerTests(r *registry) { registerTPCHBench(r) } -func registerBenchmarks(r *registry) { +func registerBenchmarks(r *testRegistry) { // Helpful shell pipeline to generate the list below: // // grep -h -E 'func register[^(]+\(.*registry\) {' *.go | grep -E -o 'register[^(]+' | grep -v '^registerTests$' | grep '^\w*Bench$' | sort | awk '{printf "\t%s(r)\n", $0}' diff --git a/pkg/cmd/roachtest/replicagc.go b/pkg/cmd/roachtest/replicagc.go index fe30f44275e8..8ba0fa0dd1f1 100644 --- a/pkg/cmd/roachtest/replicagc.go +++ b/pkg/cmd/roachtest/replicagc.go @@ -17,7 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/timeutil" ) -func registerReplicaGC(r *registry) { +func registerReplicaGC(r *testRegistry) { r.Add(testSpec{ Name: "replicagc-changed-peers/withRestart", @@ -36,7 +36,7 @@ func registerReplicaGC(r *registry) { } func runReplicaGCChangedPeers(ctx context.Context, t *test, c *cluster, withRestart bool) { - if c.nodes != 6 { + if c.spec.NodeCount != 6 { t.Fatal("test needs to be run with 6 nodes") } diff --git a/pkg/cmd/roachtest/restart.go b/pkg/cmd/roachtest/restart.go index ae0c2fe70ccf..6194e9d7fe3b 100644 --- a/pkg/cmd/roachtest/restart.go +++ b/pkg/cmd/roachtest/restart.go @@ -19,7 +19,7 @@ import ( ) func runRestart(ctx context.Context, t *test, c *cluster, downDuration time.Duration) { - crdbNodes := c.Range(1, c.nodes) + crdbNodes := c.Range(1, c.spec.NodeCount) workloadNode := c.Node(1) const restartNode = 3 @@ -82,7 +82,7 @@ func runRestart(ctx context.Context, t *test, c *cluster, downDuration time.Dura } } -func registerRestart(r *registry) { +func registerRestart(r *testRegistry) { r.Add(testSpec{ Name: fmt.Sprintf("restart/down-for-2m"), Cluster: makeClusterSpec(3), diff --git a/pkg/cmd/roachtest/restore.go b/pkg/cmd/roachtest/restore.go index 591e5a01e427..536e34cfaf51 100644 --- a/pkg/cmd/roachtest/restore.go +++ b/pkg/cmd/roachtest/restore.go @@ -186,7 +186,7 @@ func (dul *DiskUsageLogger) Runner(ctx context.Context) error { } var bytesUsed []usage - for i := 1; i <= dul.c.nodes; i++ { + for i := 1; i <= dul.c.spec.NodeCount; i++ { cur, err := getDiskUsageInBytes(ctx, dul.c, quietLogger, i) if err != nil { // This can trigger spuriously as compactions remove files out from under `du`. @@ -209,7 +209,7 @@ func (dul *DiskUsageLogger) Runner(ctx context.Context) error { } } -func registerRestore(r *registry) { +func registerRestore(r *testRegistry) { for _, item := range []struct { nodes int timeout time.Duration diff --git a/pkg/cmd/roachtest/roachmart.go b/pkg/cmd/roachtest/roachmart.go index a3efdb39ef6a..71942e75767d 100644 --- a/pkg/cmd/roachtest/roachmart.go +++ b/pkg/cmd/roachtest/roachmart.go @@ -15,7 +15,7 @@ import ( "fmt" ) -func registerRoachmart(r *registry) { +func registerRoachmart(r *testRegistry) { runRoachmart := func(ctx context.Context, t *test, c *cluster, partition bool) { c.Put(ctx, cockroach, "./cockroach") c.Put(ctx, workload, "./workload") diff --git a/pkg/cmd/roachtest/scaledata.go b/pkg/cmd/roachtest/scaledata.go index 4e5733804d9c..5bda94374579 100644 --- a/pkg/cmd/roachtest/scaledata.go +++ b/pkg/cmd/roachtest/scaledata.go @@ -20,7 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/binfetcher" ) -func registerScaleData(r *registry) { +func registerScaleData(r *testRegistry) { // apps is a suite of Sqlapp applications designed to be used to check the // consistency of a database under load. Each Sqlapp application launches a // set of workers who perform database operations while another worker @@ -53,9 +53,9 @@ func registerScaleData(r *registry) { } func runSqlapp(ctx context.Context, t *test, c *cluster, app, flags string, dur time.Duration) { - roachNodeCount := c.nodes - 1 + roachNodeCount := c.spec.NodeCount - 1 roachNodes := c.Range(1, roachNodeCount) - appNode := c.Node(c.nodes) + appNode := c.Node(c.spec.NodeCount) if local && runtime.GOOS != "linux" { t.Fatalf("must run on linux os, found %s", runtime.GOOS) diff --git a/pkg/cmd/roachtest/schemachange.go b/pkg/cmd/roachtest/schemachange.go index 7ef80269790a..40782f336d5b 100644 --- a/pkg/cmd/roachtest/schemachange.go +++ b/pkg/cmd/roachtest/schemachange.go @@ -20,7 +20,7 @@ import ( "github.com/pkg/errors" ) -func registerSchemaChangeKV(r *registry) { +func registerSchemaChangeKV(r *testRegistry) { r.Add(testSpec{ Name: `schemachange/mixed/kv`, Cluster: makeClusterSpec(5), @@ -45,7 +45,7 @@ func registerSchemaChangeKV(r *registry) { m.Wait() c.Run(ctx, c.Node(1), `./workload init kv --drop --db=test`) - for node := 1; node <= c.nodes; node++ { + for node := 1; node <= c.spec.NodeCount; node++ { node := node // TODO(dan): Ideally, the test would fail if this queryload failed, // but we can't put it in monitor as-is because the test deadlocks. @@ -283,11 +283,11 @@ func findIndexProblem( return nil } -func registerSchemaChangeIndexTPCC1000(r *registry) { +func registerSchemaChangeIndexTPCC1000(r *testRegistry) { r.Add(makeIndexAddTpccTest(makeClusterSpec(5, cpu(16)), 1000, time.Hour*2)) } -func registerSchemaChangeIndexTPCC100(r *registry) { +func registerSchemaChangeIndexTPCC100(r *testRegistry) { r.Add(makeIndexAddTpccTest(makeClusterSpec(5), 100, time.Minute*15)) } @@ -314,7 +314,7 @@ func makeIndexAddTpccTest(spec clusterSpec, warehouses int, length time.Duration } } -func registerSchemaChangeBulkIngest(r *registry) { +func registerSchemaChangeBulkIngest(r *testRegistry) { r.Add(makeSchemaChangeBulkIngestTest(5, 100000000, time.Minute*20)) } @@ -336,8 +336,8 @@ func makeSchemaChangeBulkIngestTest(numNodes, numRows int, length time.Duration) cNum := 1 payloadBytes := 4 - crdbNodes := c.Range(1, c.nodes-1) - workloadNode := c.Node(c.nodes) + crdbNodes := c.Range(1, c.spec.NodeCount-1) + workloadNode := c.Node(c.spec.NodeCount) c.Put(ctx, cockroach, "./cockroach") c.Put(ctx, workload, "./workload", workloadNode) @@ -362,7 +362,7 @@ func makeSchemaChangeBulkIngestTest(numNodes, numRows int, length time.Duration) } cmdWriteAndRead := fmt.Sprintf( "./workload run bulkingest --duration %s {pgurl:1-%d} --a %d --b %d --c %d --payload-bytes %d", - indexDuration.String(), c.nodes-1, aNum, bNum, cNum, payloadBytes, + indexDuration.String(), c.spec.NodeCount-1, aNum, bNum, cNum, payloadBytes, ) m.Go(func(ctx context.Context) error { c.Run(ctx, workloadNode, cmdWriteAndRead) diff --git a/pkg/cmd/roachtest/scrub.go b/pkg/cmd/roachtest/scrub.go index 8aa83226977f..8001099aff84 100644 --- a/pkg/cmd/roachtest/scrub.go +++ b/pkg/cmd/roachtest/scrub.go @@ -19,12 +19,12 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/timeutil" ) -func registerScrubIndexOnlyTPCC(r *registry) { +func registerScrubIndexOnlyTPCC(r *testRegistry) { // numScrubRuns is set assuming a single SCRUB run (index only) takes ~1 min r.Add(makeScrubTPCCTest(5, 100, 30*time.Minute, "index-only", 20)) } -func registerScrubAllChecksTPCC(r *registry) { +func registerScrubAllChecksTPCC(r *testRegistry) { // numScrubRuns is set assuming a single SCRUB run (all checks) takes ~2 min r.Add(makeScrubTPCCTest(5, 100, 30*time.Minute, "all-checks", 10)) } diff --git a/pkg/cmd/roachtest/split.go b/pkg/cmd/roachtest/split.go index 4fe0cc91e803..118174705fd1 100644 --- a/pkg/cmd/roachtest/split.go +++ b/pkg/cmd/roachtest/split.go @@ -36,7 +36,7 @@ type splitParams struct { waitDuration time.Duration // Duration the workload should run for. } -func registerLoadSplits(r *registry) { +func registerLoadSplits(r *testRegistry) { const numNodes = 3 r.Add(testSpec{ @@ -155,7 +155,7 @@ func runLoadSplits(ctx context.Context, t *test, c *cluster, params splitParams) setRangeMaxBytes(params.maxSize) t.Status("running uniform kv workload") - c.Run(ctx, c.Node(1), fmt.Sprintf("./workload init kv {pgurl:1-%d}", c.nodes)) + c.Run(ctx, c.Node(1), fmt.Sprintf("./workload init kv {pgurl:1-%d}", c.spec.NodeCount)) t.Status("checking initial range count") rangeCount := func() int { @@ -185,7 +185,7 @@ func runLoadSplits(ctx context.Context, t *test, c *cluster, params splitParams) } c.Run(ctx, c.Node(1), fmt.Sprintf("./workload run kv "+ "--init --concurrency=%d --read-percent=%d --span-percent=%d %s {pgurl:1-%d} --duration='%s'", - params.concurrency, params.readPercent, params.spanPercent, extraFlags, c.nodes, + params.concurrency, params.readPercent, params.spanPercent, extraFlags, c.spec.NodeCount, params.waitDuration.String())) t.Status(fmt.Sprintf("waiting for splits")) @@ -198,7 +198,7 @@ func runLoadSplits(ctx context.Context, t *test, c *cluster, params splitParams) m.Wait() } -func registerLargeRange(r *registry) { +func registerLargeRange(r *testRegistry) { const size = 10 << 30 // 10 GB // TODO(nvanbenschoten): Snapshots currently hold the entirety of a range in // memory on the receiving side. This is dangerous when we grow a range to @@ -283,7 +283,7 @@ func runLargeRangeSplits(ctx context.Context, t *test, c *cluster, size int) { // schema but before populating it. This is ok because upreplication // occurs much faster than we can actually create a large range. c.Run(ctx, c.Node(1), fmt.Sprintf("./workload init bank "+ - "--rows=%d --payload-bytes=%d --ranges=1 {pgurl:1-%d}", rows, payload, c.nodes)) + "--rows=%d --payload-bytes=%d --ranges=1 {pgurl:1-%d}", rows, payload, c.spec.NodeCount)) t.Status("checking for single range") rangeCount := func() int { diff --git a/pkg/cmd/roachtest/status_server.go b/pkg/cmd/roachtest/status_server.go index dde2e7e0d571..456f726a847d 100644 --- a/pkg/cmd/roachtest/status_server.go +++ b/pkg/cmd/roachtest/status_server.go @@ -93,7 +93,7 @@ func runStatusServer(ctx context.Context, t *test, c *cluster) { } // Check local response for the every node. - for i := 1; i <= c.nodes; i++ { + for i := 1; i <= c.spec.NodeCount; i++ { id := idMap[i] checkNode(urlMap[i], id, id, id) get(urlMap[i], "/_status/nodes") @@ -101,7 +101,7 @@ func runStatusServer(ctx context.Context, t *test, c *cluster) { // Proxy from the first node to the last node. firstNode := 1 - lastNode := c.nodes + lastNode := c.spec.NodeCount firstID := idMap[firstNode] lastID := idMap[lastNode] checkNode(urlMap[firstNode], firstID, lastID, lastID) diff --git a/pkg/cmd/roachtest/synctest.go b/pkg/cmd/roachtest/synctest.go index 030efad9f622..fd731dc591af 100644 --- a/pkg/cmd/roachtest/synctest.go +++ b/pkg/cmd/roachtest/synctest.go @@ -17,7 +17,7 @@ import ( "path/filepath" ) -func registerSyncTest(r *registry) { +func registerSyncTest(r *testRegistry) { const nemesisScript = `#!/usr/bin/env bash if [[ $1 == "on" ]]; then @@ -30,7 +30,8 @@ fi r.Add(testSpec{ Name: "synctest", MinVersion: `v2.2.0`, - Cluster: makeClusterSpec(1), + // This test sets up a custom file system; we don't want the cluster reused. + Cluster: makeClusterSpec(1, reuseNone()), Run: func(ctx context.Context, t *test, c *cluster) { n := c.Node(1) tmpDir, err := ioutil.TempDir("", "synctest") diff --git a/pkg/cmd/roachtest/sysbench.go b/pkg/cmd/roachtest/sysbench.go index c803384b00e3..5dc5e2b24409 100644 --- a/pkg/cmd/roachtest/sysbench.go +++ b/pkg/cmd/roachtest/sysbench.go @@ -78,21 +78,25 @@ func (o *sysbenchOptions) cmd() string { } func runSysbench(ctx context.Context, t *test, c *cluster, opts sysbenchOptions) { - allNodes := c.Range(1, c.nodes) - roachNodes := c.Range(1, c.nodes-1) - loadNode := c.Node(c.nodes) + allNodes := c.Range(1, c.spec.NodeCount) + roachNodes := c.Range(1, c.spec.NodeCount-1) + loadNode := c.Node(c.spec.NodeCount) t.Status("installing cockroach") c.Put(ctx, cockroach, "./cockroach", allNodes) c.Start(ctx, t, roachNodes) t.Status("installing haproxy") - c.Install(ctx, loadNode, "haproxy") + if err := c.Install(ctx, t.l, loadNode, "haproxy"); err != nil { + t.Fatal(err) + } c.Run(ctx, loadNode, "./cockroach gen haproxy --insecure --url {pgurl:1}") c.Run(ctx, loadNode, "haproxy -f haproxy.cfg -D") t.Status("installing sysbench") - c.Install(ctx, loadNode, "sysbench") + if err := c.Install(ctx, t.l, loadNode, "sysbench"); err != nil { + t.Fatal(err) + } m := newMonitor(ctx, c, roachNodes) m.Go(func(ctx context.Context) error { @@ -107,7 +111,7 @@ func runSysbench(ctx context.Context, t *test, c *cluster, opts sysbenchOptions) m.Wait() } -func registerSysbench(r *registry) { +func registerSysbench(r *testRegistry) { for w := sysbenchWorkload(0); w < numSysbenchWorkloads; w++ { const n = 3 const cpus = 16 diff --git a/pkg/cmd/roachtest/test.go b/pkg/cmd/roachtest/test.go index 898fd086e6c4..00150512b850 100644 --- a/pkg/cmd/roachtest/test.go +++ b/pkg/cmd/roachtest/test.go @@ -15,90 +15,27 @@ import ( "context" "fmt" "io" - "os" + // For the debug http handlers. + _ "net/http/pprof" "os/exec" - "os/signal" - "path/filepath" "regexp" "runtime" - "sort" - "strconv" "strings" - "sync" "time" - "github.com/cockroachdb/cockroach/pkg/cmd/internal/issues" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/version" "github.com/petermattis/goid" - "github.com/pkg/errors" ) -var ( - count = 1 - debugEnabled = false - postIssues = true - gceNameRE = regexp.MustCompile(`^[a-z](?:[-a-z0-9]{0,61}[a-z0-9])?$`) -) - -// testFilter holds the name and tag filters for filtering tests. -type testFilter struct { - name *regexp.Regexp - tag *regexp.Regexp - rawTag []string -} - -func newFilter(filter []string) *testFilter { - var name []string - var tag []string - var rawTag []string - for _, v := range filter { - if strings.HasPrefix(v, "tag:") { - tag = append(tag, strings.TrimPrefix(v, "tag:")) - rawTag = append(rawTag, v) - } else { - name = append(name, v) - } - } - - if len(tag) == 0 { - tag = []string{"default"} - rawTag = []string{"tag:default"} - } - - makeRE := func(strs []string) *regexp.Regexp { - switch len(strs) { - case 0: - return regexp.MustCompile(`.`) - case 1: - return regexp.MustCompile(strs[0]) - default: - for i := range strs { - strs[i] = "(" + strs[i] + ")" - } - return regexp.MustCompile(strings.Join(strs, "|")) - } - } - - return &testFilter{ - name: makeRE(name), - tag: makeRE(tag), - rawTag: rawTag, - } -} - type testSpec struct { Skip string // if non-empty, test will be skipped // When Skip is set, this can contain more text to be printed in the logs // after the "--- SKIP" line. SkipDetails string - // For subtests, Name is supposed to originally be assigned to the name of the - // subtest when constructing the spec and then, once added to the registry, it - // will automatically be expanded to contain all the parents' names. At that - // point, subtestName will be populated to the original value of Name. - Name string - subtestName string + + Name string // The maximum duration the test is allowed to run before it is considered // failed. If not specified, the default timeout is 10m before the test's // associated cluster expires. The timeout is always truncated to 10m before @@ -114,9 +51,7 @@ type testSpec struct { // tests. If no tags are specified, the set ["default"] is automatically // given. Tags []string - // Cluster provides the specification for the cluster to use for the test. Only - // a top-level testSpec may contain a nodes specification. The cluster is - // shared by all subtests. + // Cluster provides the specification for the cluster to use for the test. Cluster clusterSpec // UseIOBarrier controls the local-ssd-no-ext4-barrier flag passed to @@ -131,12 +66,8 @@ type testSpec struct { // care about. UseIOBarrier bool - // A testSpec must specify only one of Run or SubTests. All subtests run in - // the same cluster, without concurrency between them. Subtest should not - // assume any particular state for the cluster as the SubTest may be run in - // isolation. - Run func(ctx context.Context, t *test, c *cluster) - SubTests []testSpec + // Run is the test function. + Run func(ctx context.Context, t *test, c *cluster) } // matchOrSkip returns true if the filter matches the test. If the filter does @@ -161,493 +92,6 @@ func (t *testSpec) matchOrSkip(filter *testFilter) bool { return true } -// matchRegex returns true if the regex matches the test's name or any of the -// subtest names. -func (t *testSpec) matchRegex(filter *testFilter) bool { - if t.matchOrSkip(filter) { - return true - } - for i := range t.SubTests { - if t.SubTests[i].matchRegex(filter) { - return true - } - } - return false -} - -func (t *testSpec) matchRegexRecursive(filter *testFilter) []testSpec { - var res []testSpec - if t.matchOrSkip(filter) { - res = append(res, *t) - } - for i := range t.SubTests { - res = append(res, t.SubTests[i].matchRegexRecursive(filter)...) - } - return res -} - -type registry struct { - m map[string]*testSpec - clusters map[string]string - out io.Writer - statusInterval time.Duration - buildVersion *version.Version - - config struct { - // skipClusterValidationOnAttach skips validation on existing clusters that - // the registry uses for running tests. - skipClusterValidationOnAttach bool - // skipClusterStopOnAttach skips stopping existing clusters that - // the registry uses for running tests. It implies skipClusterWipeOnAttach. - skipClusterStopOnAttach bool - skipClusterWipeOnAttach bool - } - - status struct { - syncutil.Mutex - running map[*test]struct{} - pass map[*test]struct{} - fail map[*test]struct{} - skip map[*test]struct{} - } -} - -type registryOpt func(r *registry) error - -var ( - // setBuildVersion sets the build version based on the flag variable or loads - // the version from git if the flag is not set. - setBuildVersion registryOpt = func(r *registry) error { - if buildTag != "" { - return r.setBuildVersion(buildTag) - } - return r.loadBuildVersion() - } -) - -// newRegistry constructs a registry and configures it with opts. If any opt -// returns an error then the function will log about the error and exit the -// process with os.Exit(1). -func newRegistry(opts ...registryOpt) *registry { - r := ®istry{ - m: make(map[string]*testSpec), - clusters: make(map[string]string), - out: os.Stdout, - } - r.config.skipClusterWipeOnAttach = !clusterWipe - for _, opt := range opts { - if err := opt(r); err != nil { - fmt.Fprintf(os.Stderr, "failed to construct registry: %v\n", err) - os.Exit(1) - } - } - return r -} - -func (r *registry) setBuildVersion(buildTag string) error { - var err error - r.buildVersion, err = version.Parse(buildTag) - return err -} - -func (r *registry) loadBuildVersion() error { - getLatestTag := func() (string, error) { - cmd := exec.Command("git", "describe", "--abbrev=0", "--tags", "--match=v[0-9]*") - out, err := cmd.CombinedOutput() - if err != nil { - return "", errors.Wrapf(err, "failed to get version tag from git. Are you running in the "+ - "cockroach repo directory? err=%s, out=%s", err, out) - } - return strings.TrimSpace(string(out)), nil - } - buildTag, err := getLatestTag() - if err != nil { - return err - } - return r.setBuildVersion(buildTag) -} - -// PredecessorVersion returns a recent predecessor of the build version (i.e. -// the build tag of the main binary). For example, if the running binary is from -// the master branch prior to releasing 19.2.0, this will return a recent -// (ideally though not necessarily the latest) 19.1 patch release. -func (r *registry) PredecessorVersion() (string, error) { - if r.buildVersion == nil { - return "", errors.Errorf("buildVersion not set") - } - - buildVersionMajorMinor := fmt.Sprintf("%d.%d", r.buildVersion.Major(), r.buildVersion.Minor()) - - verMap := map[string]string{ - "19.2": "19.1.0-rc.4", - "19.1": "2.1.6", - "2.2": "2.1.6", - "2.1": "2.0.7", - } - v, ok := verMap[buildVersionMajorMinor] - if !ok { - return "", errors.Errorf("prev version not set for version: %s", buildVersionMajorMinor) - } - return v, nil -} - -// verifyValidClusterName verifies that the test name can be turned into a cluster -// name when run by TeamCity. Outside of TeamCity runs, depending on the user -// running it and the "cluster id" component of a cluster name, the name may -// still be invalid; however, this method is designed to catch test names -// that will cause errors on TeamCity but not in a developer's local test -// environment. -func (r *registry) verifyValidClusterName(testName string) error { - // Both the name of the cluster, and the names of the individual nodes in the - // cluster, must be valid identifiers in GCE when running on TeamCity. An - // identifier can be tested using a regular expression. Also note that, due to - // the specifics of the regular expression, we cannot assume that a valid - // cluster name implies valid node names, or vice-versa; we therefore - // construct both a TeamCity cluster name and a TeamCity node name and - // validate both. - - // The name of a cluster is constructed as "[cluster ID][test name]" - // In TeamCity runs, the cluster ID is currently a prefix with 6 digits, but - // we use 7 here for a bit of breathing room. - teamcityClusterName := makeGCEClusterName("teamcity-1234567-" + testName) - if !gceNameRE.MatchString(teamcityClusterName) { - return fmt.Errorf( - "test name '%s' results in invalid cluster name"+ - " (generated cluster name '%s' must match regex '%s')."+ - " The test name may be too long or have invalid characters", - testName, - teamcityClusterName, - gceNameRE, - ) - } - - // The node names are constructed using the cluster name, plus a 4 digit node - // ID. - teamcityNodeName := makeGCEClusterName("teamcity-1234567-" + testName + "-1234") - if !gceNameRE.MatchString(teamcityNodeName) { - return fmt.Errorf( - "test name '%s' results in invalid cluster node names"+ - " (generated node name '%s' must match regex '%s')."+ - " The test name may be too long or have invalid characters", - testName, - teamcityNodeName, - gceNameRE, - ) - } - - // Verify that the cluster name is not shared with an existing test. - if t, ok := r.clusters[teamcityClusterName]; ok { - return fmt.Errorf("test %s and test %s have equivalent nightly cluster names: %s", - testName, t, teamcityClusterName) - } - r.clusters[teamcityClusterName] = testName - return nil -} - -func (r *registry) prepareSpec(spec *testSpec, depth int) error { - if depth == 0 { - spec.subtestName = spec.Name - // Only top-level tests can create clusters, so those are the only ones for - // which we need to verify the cluster name. - if err := r.verifyValidClusterName(spec.Name); err != nil { - return err - } - } - - if (spec.Run != nil) == (len(spec.SubTests) > 0) { - return fmt.Errorf("%s: must specify only one of Run or SubTests", spec.Name) - } - - if spec.Run == nil && spec.Timeout > 0 { - return fmt.Errorf("%s: timeouts only apply to tests specifying Run", spec.Name) - } - - if depth > 0 && spec.Cluster.NodeCount > 0 { - return fmt.Errorf("%s: subtest may not provide cluster specification", spec.Name) - } - - for i := range spec.SubTests { - spec.SubTests[i].subtestName = spec.SubTests[i].Name - spec.SubTests[i].Name = spec.Name + "/" + spec.SubTests[i].Name - if err := r.prepareSpec(&spec.SubTests[i], depth+1); err != nil { - return err - } - } - - if spec.MinVersion != "" { - v, err := version.Parse(spec.MinVersion) - if err != nil { - return fmt.Errorf("%s: unable to parse min-version: %s", spec.Name, err) - } - if v.PreRelease() != "" { - // Specifying a prerelease version as a MinVersion is too confusing - // to be useful. The comparison is not straightforward. - return fmt.Errorf("invalid version %s, cannot specify a prerelease (-xxx)", v) - } - // We append "-0" to the min-version spec so that we capture all - // prereleases of the specified version. Otherwise, "v2.1.0" would compare - // greater than "v2.1.0-alpha.x". - spec.minVersion = version.MustParse(spec.MinVersion + "-0") - } - return nil -} - -func (r *registry) Add(spec testSpec) { - if _, ok := r.m[spec.Name]; ok { - fmt.Fprintf(os.Stderr, "test %s already registered\n", spec.Name) - os.Exit(1) - } - if err := r.prepareSpec(&spec, 0); err != nil { - fmt.Fprintf(os.Stderr, "%s\n", err) - os.Exit(1) - } - r.m[spec.Name] = &spec -} - -// ListTopLevel lists the top level tests that match re, or that have a subtests -// that matches re. -func (r *registry) ListTopLevel(filter *testFilter) []*testSpec { - var results []*testSpec - for _, t := range r.m { - if t.matchRegex(filter) { - results = append(results, t) - } - } - - sort.Slice(results, func(i, j int) bool { - return results[i].Name < results[j].Name - }) - return results -} - -// ListAll lists all tests that match one of the filters. If a subtest matches -// but a parent doesn't, only the subtest is returned. If a parent matches, all -// subtests are returned. -func (r *registry) ListAll(filters []string) []string { - filter := newFilter(filters) - var tests []testSpec - for _, t := range r.m { - tests = append(tests, t.matchRegexRecursive(filter)...) - } - var names []string - for _, t := range tests { - if t.Skip == "" && t.minVersion != nil { - if !r.buildVersion.AtLeast(t.minVersion) { - t.Skip = fmt.Sprintf("build-version (%s) < min-version (%s)", - r.buildVersion, t.minVersion) - } - } - name := t.Name - if t.Skip != "" { - name += " (skipped: " + t.Skip + ")" - } - - names = append(names, name) - } - sort.Strings(names) - return names -} - -// Run runs the tests that match the filter. -// -// Args: -// artifactsDir: The path to the dir where log files will be put. If empty, all -// logging will go to stdout/stderr. -func (r *registry) Run(filters []string, parallelism int, artifactsDir string, user string) int { - filter := newFilter(filters) - // Find the top-level tests to run. - tests := r.ListTopLevel(filter) - if len(tests) == 0 { - fmt.Fprintf(r.out, "warning: no tests to run %s\n", filters) - fmt.Fprintf(r.out, "FAIL\n") - return 1 - } - - // Skip any tests for which the min-version is less than the build-version. - for _, t := range tests { - if t.Skip == "" && t.minVersion != nil { - if !r.buildVersion.AtLeast(t.minVersion) { - t.Skip = fmt.Sprintf("build-version (%s) < min-version (%s)", - r.buildVersion, t.minVersion) - } - } - } - - wg := &sync.WaitGroup{} - wg.Add(count * len(tests)) - - // We can't run tests in parallel on local clusters or on an existing - // cluster. - if local || clusterName != "" { - parallelism = 1 - } - // Limit the parallelism to the number of tests. The primary effect this has - // is that we'll log to stdout/stderr if only one test is being run. - if parallelism > len(tests) { - parallelism = len(tests) - } - - r.status.running = make(map[*test]struct{}) - r.status.pass = make(map[*test]struct{}) - r.status.fail = make(map[*test]struct{}) - r.status.skip = make(map[*test]struct{}) - - cr := newClusterRegistry() - - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - go func() { - sem := make(chan struct{}, parallelism) - for j := 0; j < count; j++ { - for i := range tests { - sem <- struct{}{} - runNum := j + 1 - if count == 1 { - runNum = 0 - } - // Log to stdout/stderr if we're not running tests in parallel. - teeOpt := noTee - if parallelism == 1 { - teeOpt = teeToStdout - } - - artifactsSuffix := "" - if runNum != 0 { - artifactsSuffix = "run_" + strconv.Itoa(runNum) - } - var runDir string - if artifactsDir != "" { - runDir = filepath.Join( - artifactsDir, teamCityNameEscape(tests[i].subtestName), artifactsSuffix) - } - - r.runAsync( - ctx, tests[i], filter, nil /* parent */, nil, /* cluster */ - runNum, teeOpt, runDir, user, cr, func(failed bool) { - wg.Done() - <-sem - }) - } - } - }() - - done := make(chan struct{}) - go func() { - wg.Wait() - close(done) - }() - - // Periodically output test status to give an indication of progress. - if r.statusInterval == 0 { - r.statusInterval = time.Minute - } - ticker := time.NewTicker(r.statusInterval) - defer ticker.Stop() - - // Shut down test clusters when interrupted (for example CTRL+C). - sig := make(chan os.Signal, 1) - signal.Notify(sig, os.Interrupt) - - for i := 1; ; i++ { - select { - case <-done: - r.status.Lock() - defer r.status.Unlock() - postSlackReport(r.status.pass, r.status.fail, r.status.skip) - - if len(r.status.fail) > 0 { - fmt.Fprintln(r.out, "FAIL") - return 1 - } - fmt.Fprintf(r.out, "PASS\n") - return 0 - - case <-ticker.C: - r.status.Lock() - runningTests := make([]*test, 0, len(r.status.running)) - for t := range r.status.running { - runningTests = append(runningTests, t) - } - sort.Slice(runningTests, func(i, j int) bool { - return runningTests[i].Name() < runningTests[j].Name() - }) - var buf bytes.Buffer - for _, t := range runningTests { - if t.spec.Run == nil { - // Ignore tests with subtests. - continue - } - - t.mu.Lock() - done := t.mu.done - var status map[int64]testStatus - if !done { - status = make(map[int64]testStatus, len(t.mu.status)) - for k, v := range t.mu.status { - status[k] = v - } - if len(status) == 0 { - // If we have no other status messages display this unknown state. - status[0] = testStatus{ - msg: "???", - time: timeutil.Now(), - } - } - } - t.mu.Unlock() - if !done { - ids := make([]int64, 0, len(status)) - for id := range status { - ids = append(ids, id) - } - sort.Slice(ids, func(i, j int) bool { - // Force the goroutine ID for the main test goroutine to sort to - // the front. NB: goroutine IDs are not monotonically increasing - // because each thread has a small cache of IDs for allocation. - if ids[j] == t.runnerID { - return false - } - if ids[i] == t.runnerID { - return true - } - return ids[i] < ids[j] - }) - - fmt.Fprintf(&buf, "[%4d] %s: ", i, t.Name()) - - for j := range ids { - s := status[ids[j]] - duration := timeutil.Now().Sub(s.time) - progStr := "" - if s.progress > 0 { - progStr = fmt.Sprintf("%.1f%%|", 100*s.progress) - } - if j > 0 { - buf.WriteString(", ") - } - fmt.Fprintf(&buf, "%s (%s%s)", s.msg, progStr, - time.Duration(duration.Seconds()+0.5)*time.Second) - } - - fmt.Fprintf(&buf, "\n") - } - } - fmt.Fprint(r.out, buf.String()) - r.status.Unlock() - - case <-sig: - if !debugEnabled { - cancel() - // Destroy all clusters. Don't wait more than 5 min for that though. - ctx, cancel := context.WithTimeout(context.Background(), 5*time.Minute) - cr.destroyAllClusters(ctx) - cancel() - } - } - } -} - type testStatus struct { msg string time time.Time @@ -655,23 +99,22 @@ type testStatus struct { } type test struct { - spec *testSpec - registry *registry + spec *testSpec + + // buildVersion is the version of the Cockroach binary that the test will run + // against. + buildVersion version.Version + // l is the logger that the test will use for its output. - l *logger + l *logger + runner string runnerID int64 start time.Time end time.Time - // debugEnabled is a test scoped value which enables automated tests to - // enable debugging without enabling debugging for all tests. - // It is a bit of a hack added to help debug #34458. - debugEnabled bool - // artifactsDir is the path to the directory holding all the artifacts for - // this test. It will contain a test.log file, cluster logs, and - // subdirectories for subtests. + // this test. It will contain a test.log file and cluster logs. artifactsDir string mu struct { syncutil.RWMutex @@ -686,8 +129,9 @@ type test struct { file string line int } - status map[int64]testStatus - output []byte + failureMsg string + status map[int64]testStatus + output []byte } } @@ -699,7 +143,7 @@ func (t *test) logger() *logger { return t.l } -func (t *test) status(id int64, args ...interface{}) { +func (t *test) status(ctx context.Context, id int64, args ...interface{}) { t.mu.Lock() defer t.mu.Unlock() @@ -710,10 +154,18 @@ func (t *test) status(id int64, args ...interface{}) { delete(t.mu.status, id) return } + msg := fmt.Sprint(args...) t.mu.status[id] = testStatus{ - msg: fmt.Sprint(args...), + msg: msg, time: timeutil.Now(), } + if !t.l.closed() { + if id == t.runnerID { + t.l.PrintfCtx(ctx, "test status: %s", msg) + } else { + t.l.PrintfCtx(ctx, "test worker status: %s", msg) + } + } } // Status sets the main status message for the test. When called from the main @@ -721,14 +173,25 @@ func (t *test) status(id int64, args ...interface{}) { // is equivalent to calling WorkerStatus. If no arguments are specified, the // status message is erased. func (t *test) Status(args ...interface{}) { - t.status(t.runnerID, args...) + t.status(context.TODO(), t.runnerID, args...) +} + +// GetStatus returns the status of the tests's main goroutine. +func (t *test) GetStatus() string { + t.mu.Lock() + defer t.mu.Unlock() + status, ok := t.mu.status[t.runnerID] + if ok { + return fmt.Sprintf("%s (set %s ago)", status.msg, timeutil.Now().Sub(status.time)) + } + return "N/A" } // WorkerStatus sets the status message for a worker goroutine associated with // the test. The status message should be cleared before the goroutine exits by // calling WorkerStatus with no arguments. func (t *test) WorkerStatus(args ...interface{}) { - t.status(goid.Get(), args...) + t.status(context.TODO(), goid.Get(), args...) } func (t *test) progress(id int64, frac float64) { @@ -765,10 +228,16 @@ func (t *test) Skip(msg string, details string) { runtime.Goexit() } +// Fatal marks the test as failed, prints the args to t.l, and calls +// runtime.GoExit(). It can be called multiple times. +// +// ATTENTION: Since this calls runtime.GoExit(), it should only be called from a +// test's closure. The test runner itself should never call this. func (t *test) Fatal(args ...interface{}) { t.fatalfInner("" /* format */, args...) } +// Fatalf is like Fatal, but takes a format string. func (t *test) Fatalf(format string, args ...interface{}) { t.fatalfInner(format, args...) } @@ -805,12 +274,23 @@ func (t *test) printAndFail(skip int, args ...interface{}) { func (t *test) printfAndFail(skip int, format string, args ...interface{}) { msg := t.decorate(skip+1, fmt.Sprintf(format, args...)) - t.l.Printf("test failure: " + msg) t.mu.Lock() defer t.mu.Unlock() - t.mu.output = append(t.mu.output, msg...) + + prefix := "" + if t.mu.failed { + prefix = "[not the first failure] " + } + t.l.Printf("%stest failure: %s", prefix, msg) + + if t.mu.failed { + return + } + t.mu.failed = true + t.mu.failureMsg = msg + t.mu.output = append(t.mu.output, msg...) if t.mu.cancel != nil { t.mu.cancel() } @@ -888,9 +368,14 @@ func (t *test) duration() time.Duration { func (t *test) Failed() bool { t.mu.RLock() - failed := t.mu.failed - t.mu.RUnlock() - return failed + defer t.mu.RUnlock() + return t.mu.failed +} + +func (t *test) FailureMsg() string { + t.mu.RLock() + defer t.mu.RUnlock() + return t.mu.failureMsg } func (t *test) ArtifactsDir() string { @@ -914,342 +399,11 @@ func (t *test) IsBuildVersion(minVersion string) bool { // prereleases of the specified version. Otherwise, "v2.1.0" would compare // greater than "v2.1.0-alpha.x". vers = version.MustParse(minVersion + "-0") - return t.registry.buildVersion.AtLeast(vers) + return t.buildVersion.AtLeast(vers) } var _ = (*test)(nil).IsBuildVersion // avoid unused lint -// runAsync starts a goroutine that runs a test. If the test has subtests, -// runAsync will be invoked recursively, but in a blocking manner. -// -// Args: -// parent: The test's parent. Nil if the test is not a subtest. -// c: The cluster on which the test (and all subtests) will run. If nil, a new -// cluster will be created. -// runNum: The 1-based index of this test run, if --count > 1. Otherwise (if -// there's a single run), runNum is 0. -func (r *registry) runAsync( - ctx context.Context, - spec *testSpec, - filter *testFilter, - parent *test, - c *cluster, - runNum int, - teeOpt teeOptType, - artifactsDir string, - user string, - cr *clusterRegistry, - done func(failed bool), -) { - t := &test{ - spec: spec, - registry: r, - artifactsDir: artifactsDir, - } - var logPath string - if artifactsDir != "" { - logPath = filepath.Join(artifactsDir, "test.log") - } - l, err := rootLogger(logPath, teeOpt) - FatalIfErr(t, err) - t.l = l - out := io.MultiWriter(r.out, t.l.file) - - if teamCity { - fmt.Printf("##teamcity[testStarted name='%s' flowId='%s']\n", t.Name(), t.Name()) - } else { - var details []string - if t.spec.Skip != "" { - details = append(details, "skip") - } - var detail string - if len(details) > 0 { - detail = fmt.Sprintf(" [%s]", strings.Join(details, ",")) - } - fmt.Fprintf(out, "=== RUN %s%s\n", t.Name(), detail) - } - r.status.Lock() - r.status.running[t] = struct{}{} - r.status.Unlock() - - callerName := func() string { - // Make room for the skip PC. - var pc [2]uintptr - n := runtime.Callers(2, pc[:]) // skip + runtime.Callers + callerName - if n == 0 { - panic("zero callers found") - } - frames := runtime.CallersFrames(pc[:n]) - frame, _ := frames.Next() - return frame.Function - } - - go func() { - t.runner = callerName() - t.runnerID = goid.Get() - - defer func() { - t.end = timeutil.Now() - - if err := recover(); err != nil { - t.mu.Lock() - t.mu.failed = true - t.mu.output = append(t.mu.output, t.decorate(0 /* skip */, fmt.Sprint(err))...) - t.mu.Unlock() - } - - t.mu.Lock() - t.mu.done = true - t.mu.Unlock() - - dstr := fmt.Sprintf("%.2fs", t.duration().Seconds()) - - if t.Failed() { - t.mu.Lock() - output := t.mu.output - failLoc := t.mu.failLoc - t.mu.Unlock() - - if teamCity { - fmt.Fprintf( - r.out, "##teamcity[testFailed name='%s' details='%s' flowId='%s']\n", - t.Name(), teamCityEscape(string(output)), t.Name(), - ) - } - - fmt.Fprintf(out, "--- FAIL: %s (%s)\n%s", t.Name(), dstr, output) - if postIssues && issues.CanPost() && t.spec.Run != nil { - authorEmail := getAuthorEmail(failLoc.file, failLoc.line) - branch := "" - if b := os.Getenv("TC_BUILD_BRANCH"); b != "" { - branch = b - } - msg := fmt.Sprintf("The test failed on branch=%s, cloud=%s:\n%s", - branch, cloud, output) - if err := issues.Post( - context.Background(), - fmt.Sprintf("roachtest: %s failed", t.Name()), - "roachtest", t.Name(), msg, authorEmail, - []string{"O-roachtest"}, - ); err != nil { - fmt.Fprintf(out, "failed to post issue: %s\n", err) - } - } - } else if t.spec.Skip == "" { - fmt.Fprintf(out, "--- PASS: %s (%s)\n", t.Name(), dstr) - // If `##teamcity[testFailed ...]` is not present before `##teamCity[testFinished ...]`, - // TeamCity regards the test as successful. - } else { - if teamCity { - fmt.Fprintf(r.out, "##teamcity[testIgnored name='%s' message='%s']\n", - t.Name(), teamCityEscape(t.spec.Skip)) - } - fmt.Fprintf(out, "--- SKIP: %s (%s)\n\t%s\n", t.Name(), dstr, t.spec.Skip) - if t.spec.SkipDetails != "" { - fmt.Fprintf(out, "Details: %s\n", t.spec.SkipDetails) - } - } - - if teamCity { - fmt.Fprintf(r.out, "##teamcity[testFinished name='%s' flowId='%s']\n", t.Name(), t.Name()) - - // Only publish artifacts for failed tests. At the time of writing, a full roachtest - // suite results in ~6gb of artifacts which we can't retain for more than a few days - // (and this in turn delays the resolution of failures). - if t.Failed() && artifactsDir != "" { - escapedTestName := teamCityNameEscape(t.Name()) - artifactsGlobPath := filepath.Join(artifactsDir, "**") - artifactsSpec := fmt.Sprintf("%s => %s", artifactsGlobPath, escapedTestName) - fmt.Fprintf(r.out, "##teamcity[publishArtifacts '%s']\n", artifactsSpec) - } - } - - r.status.Lock() - delete(r.status.running, t) - // Only include tests with a Run function in the summary output. - if t.spec.Run != nil { - if t.Failed() { - r.status.fail[t] = struct{}{} - } else if t.spec.Skip == "" { - r.status.pass[t] = struct{}{} - } else { - r.status.skip[t] = struct{}{} - } - } - r.status.Unlock() - - done(t.Failed()) - }() - - t.start = timeutil.Now() - - if t.spec.Skip != "" { - return - } - - if c == nil { - if clusterName == "" { - var name string - if !local { - name = clusterID - if name == "" { - name = fmt.Sprintf("%d", timeutil.Now().Unix()) - } - name += "-" + t.Name() - } - cfg := clusterConfig{ - name: name, - nodes: t.spec.Cluster, - useIOBarrier: t.spec.UseIOBarrier, - artifactsDir: t.ArtifactsDir(), - localCluster: local, - teeOpt: teeOpt, - user: user, - } - var err error - c, err = newCluster(ctx, t.l, cfg, cr) - if err != nil { - t.Skip("failed to created cluster", err.Error()) - } - } else { - opt := attachOpt{ - skipValidation: r.config.skipClusterValidationOnAttach, - skipStop: r.config.skipClusterStopOnAttach, - skipWipe: r.config.skipClusterWipeOnAttach, - } - var err error - c, err = attachToExistingCluster(ctx, clusterName, t.l, t.spec.Cluster, opt, cr) - FatalIfErr(t, err) - } - if c != nil { - defer func() { - if (!debugEnabled && !t.debugEnabled) || !t.Failed() { - c.Destroy(ctx, closeLogger) - } else { - c.l.Printf("not destroying cluster to allow debugging\n") - } - }() - } - } else { - c = c.clone() - } - c.setTest(t) - - // If we have subtests, handle them here and return. - if t.spec.Run == nil { - for i := range t.spec.SubTests { - childSpec := t.spec.SubTests[i] - if childSpec.matchRegex(filter) { - var wg sync.WaitGroup - wg.Add(1) - - // Each subtest gets its own subdir in the parent's artifacts dir. - var childDir string - if t.ArtifactsDir() != "" { - childDir = filepath.Join(t.ArtifactsDir(), teamCityNameEscape(childSpec.subtestName)) - } - - r.runAsync(ctx, &childSpec, filter, t, c, - runNum, teeOpt, childDir, user, cr, func(failed bool) { - if failed { - // Mark the parent test as failed since one of the subtests - // failed. - t.mu.Lock() - t.mu.failed = true - t.mu.Unlock() - } - if failed && debugEnabled { - // The test failed and debugging is enabled. Don't try to stumble - // forward running another test or subtest, just exit - // immediately. - os.Exit(1) - } - wg.Done() - }) - wg.Wait() - } - } - return - } - - // No subtests, so this is a leaf test. - - timeout := c.expiration.Add(-10 * time.Minute).Sub(timeutil.Now()) - if timeout <= 0 { - t.spec.Skip = fmt.Sprintf("cluster expired (%s)", timeout) - return - } - - if t.spec.Timeout > 0 && timeout > t.spec.Timeout { - timeout = t.spec.Timeout - } - - done := make(chan struct{}) - defer close(done) // closed only after we've grabbed the debug info below - - defer func() { - if t.Failed() { - if err := c.FetchDebugZip(ctx); err != nil { - c.l.Printf("failed to download debug zip: %s", err) - } - if err := c.FetchDmesg(ctx); err != nil { - c.l.Printf("failed to fetch dmesg: %s", err) - } - if err := c.FetchJournalctl(ctx); err != nil { - c.l.Printf("failed to fetch journalctl: %s", err) - } - if err := c.FetchCores(ctx); err != nil { - c.l.Printf("failed to fetch cores: %s", err) - } - if err := c.CopyRoachprodState(ctx); err != nil { - c.l.Printf("failed to copy roachprod state: %s", err) - } - } - // 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 - // duplication is acceptable. - if err := c.FetchLogs(ctx); err != nil { - c.l.Printf("failed to download logs: %s", err) - } - }() - // Detect replica divergence (i.e. ranges in which replicas have arrived - // at the same log position with different states). - defer c.FailOnReplicaDivergence(ctx, t) - // Detect dead nodes in an inner defer. Note that this will call t.Fatal - // when appropriate, which will cause the closure above to enter the - // t.Failed() branch. - defer c.FailOnDeadNodes(ctx, t) - - runCtx, cancel := context.WithCancel(ctx) - t.mu.Lock() - // t.Fatal() will cancel this context. - t.mu.cancel = cancel - t.mu.Unlock() - - go func() { - defer cancel() - - select { - case <-time.After(timeout): - t.printfAndFail(0 /* skip */, "test timed out (%s)\n", timeout) - if err := c.FetchDebugZip(ctx); err != nil { - c.l.Printf("failed to download logs: %s", err) - } - // NB: c.destroyState is nil for cloned clusters (i.e. in subtests). - if !debugEnabled && c.destroyState != nil { - // We don't close the logger here because the cluster may still be in - // use by the test. - c.Destroy(ctx, dontCloseLogger) - } - case <-done: - } - }() - - t.spec.Run(runCtx, t, c) - }() -} - // teamCityEscape escapes a string for use as in a key='' attribute // in TeamCity build output marker. // Documentation here: https://confluence.jetbrains.com/display/TCD10/Build+Script+Interaction+with+TeamCity#BuildScriptInteractionwithTeamCity-Escapedvalues @@ -1294,3 +448,96 @@ func getAuthorEmail(file string, line int) string { } return string(matches[1]) } + +type testWithCount struct { + spec testSpec + // count maintains the number of runs remaining for a test. + count int +} + +type clusterType int + +const ( + localCluster clusterType = iota + roachprodCluster +) + +type loggingOpt struct { + // l is the test runner logger. + // Note that individual test runs will use a different logger. + l *logger + // tee controls whether test logs (not test runner logs) also go to stdout or + // not. + tee teeOptType + stdout, stderr io.Writer + artifactsDir string +} + +type workerStatus struct { + // name is the worker's identifier. + name string + mu struct { + syncutil.Mutex + + // status is presented in the HTML progress page. + status string + + ttr testToRunRes + t *test + c *cluster + } +} + +func (w *workerStatus) Status() string { + w.mu.Lock() + defer w.mu.Unlock() + return w.mu.status +} + +func (w *workerStatus) SetStatus(status string) { + w.mu.Lock() + w.mu.status = status + w.mu.Unlock() +} + +func (w *workerStatus) Cluster() *cluster { + w.mu.Lock() + defer w.mu.Unlock() + return w.mu.c +} + +func (w *workerStatus) SetCluster(c *cluster) { + w.mu.Lock() + w.mu.c = c + w.mu.Unlock() +} + +func (w *workerStatus) TestToRun() testToRunRes { + w.mu.Lock() + defer w.mu.Unlock() + return w.mu.ttr +} + +func (w *workerStatus) Test() *test { + w.mu.Lock() + defer w.mu.Unlock() + return w.mu.t +} + +func (w *workerStatus) SetTest(t *test, ttr testToRunRes) { + w.mu.Lock() + w.mu.t = t + w.mu.ttr = ttr + w.mu.Unlock() +} + +// shout logs a message both to a logger and to an io.Writer. +// If format doesn't end with a new line, one will be automatically added. +func shout(ctx context.Context, l *logger, stdout io.Writer, format string, args ...interface{}) { + if len(format) == 0 || format[len(format)-1] != '\n' { + format += "\n" + } + msg := fmt.Sprintf(format, args...) + l.PrintfCtxDepth(ctx, 2 /* depth */, msg) + fmt.Fprint(stdout, msg) +} diff --git a/pkg/cmd/roachtest/test_registry.go b/pkg/cmd/roachtest/test_registry.go new file mode 100644 index 000000000000..59878495fdb0 --- /dev/null +++ b/pkg/cmd/roachtest/test_registry.go @@ -0,0 +1,198 @@ +// Copyright 2018 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 main + +import ( + "context" + "fmt" + "os" + "os/exec" + "regexp" + "sort" + "strings" + + "github.com/cockroachdb/cockroach/pkg/util/version" + "github.com/pkg/errors" +) + +type testRegistry struct { + m map[string]*testSpec + // buildVersion is the version of the Cockroach binary that tests will run against. + buildVersion version.Version +} + +// makeTestRegistry constructs a testRegistry and configures it with opts. +func makeTestRegistry() (testRegistry, error) { + r := testRegistry{ + m: make(map[string]*testSpec), + } + v := buildTag + if v == "" { + var err error + v, err = loadBuildVersion() + if err != nil { + return testRegistry{}, err + } + } + if err := r.setBuildVersion(v); err != nil { + return testRegistry{}, err + } + return r, nil +} + +// Add adds a test to the registry. +func (r *testRegistry) Add(spec testSpec) { + if _, ok := r.m[spec.Name]; ok { + fmt.Fprintf(os.Stderr, "test %s already registered\n", spec.Name) + os.Exit(1) + } + if err := r.prepareSpec(&spec); err != nil { + fmt.Fprintf(os.Stderr, "%s\n", err) + os.Exit(1) + } + r.m[spec.Name] = &spec +} + +// prepareSpec validates a spec and does minor massaging of its fields. +func (r *testRegistry) prepareSpec(spec *testSpec) error { + if spec.Run == nil { + return fmt.Errorf("%s: must specify Run", spec.Name) + } + + if spec.Cluster.ReusePolicy == nil { + return fmt.Errorf("%s: must specify a ClusterReusePolicy", spec.Name) + } + + if spec.MinVersion != "" { + v, err := version.Parse(spec.MinVersion) + if err != nil { + return fmt.Errorf("%s: unable to parse min-version: %s", spec.Name, err) + } + if v.PreRelease() != "" { + // Specifying a prerelease version as a MinVersion is too confusing + // to be useful. The comparison is not straightforward. + return fmt.Errorf("invalid version %s, cannot specify a prerelease (-xxx)", v) + } + // We append "-0" to the min-version spec so that we capture all + // prereleases of the specified version. Otherwise, "v2.1.0" would compare + // greater than "v2.1.0-alpha.x". + spec.minVersion = version.MustParse(spec.MinVersion + "-0") + } + return nil +} + +// GetTests returns all the tests that match the given regexp. +// Skipped tests are included, and tests that don't match their minVersion spec +// are also included but marked as skipped. +func (r testRegistry) GetTests(ctx context.Context, filter *testFilter) []testSpec { + var tests []testSpec + for _, t := range r.m { + if !t.matchOrSkip(filter) { + continue + } + if t.Skip == "" && t.minVersion != nil { + if !r.buildVersion.AtLeast(t.minVersion) { + t.Skip = fmt.Sprintf("build-version (%s) < min-version (%s)", + r.buildVersion, t.minVersion) + } + } + tests = append(tests, *t) + } + sort.Slice(tests, func(i, j int) bool { + return tests[i].Name < tests[j].Name + }) + return tests +} + +// List lists tests that match one of the filters. +func (r testRegistry) List(ctx context.Context, filters []string) []string { + filter := newFilter(filters) + tests := r.GetTests(ctx, filter) + var names []string + for _, t := range tests { + name := t.Name + if t.Skip != "" { + name += " (skipped: " + t.Skip + ")" + } + + names = append(names, name) + } + sort.Strings(names) + return names +} + +func (r *testRegistry) setBuildVersion(buildTag string) error { + v, err := version.Parse(buildTag) + if err != nil { + return err + } + r.buildVersion = *v + return err +} + +func loadBuildVersion() (string, error) { + cmd := exec.Command("git", "describe", "--abbrev=0", "--tags", "--match=v[0-9]*") + out, err := cmd.CombinedOutput() + if err != nil { + return "", errors.Wrapf( + err, "failed to get version tag from git. Are you running in the "+ + "cockroach repo directory? err=%s, out=%s", + err, out) + } + return strings.TrimSpace(string(out)), nil +} + +// testFilter holds the name and tag filters for filtering tests. +type testFilter struct { + name *regexp.Regexp + tag *regexp.Regexp + // rawTag is the string representation of the regexps in tag + rawTag []string +} + +func newFilter(filter []string) *testFilter { + var name []string + var tag []string + var rawTag []string + for _, v := range filter { + if strings.HasPrefix(v, "tag:") { + tag = append(tag, strings.TrimPrefix(v, "tag:")) + rawTag = append(rawTag, v) + } else { + name = append(name, v) + } + } + + if len(tag) == 0 { + tag = []string{"default"} + rawTag = []string{"tag:default"} + } + + makeRE := func(strs []string) *regexp.Regexp { + switch len(strs) { + case 0: + return regexp.MustCompile(`.`) + case 1: + return regexp.MustCompile(strs[0]) + default: + for i := range strs { + strs[i] = "(" + strs[i] + ")" + } + return regexp.MustCompile(strings.Join(strs, "|")) + } + } + + return &testFilter{ + name: makeRE(name), + tag: makeRE(tag), + rawTag: rawTag, + } +} diff --git a/pkg/cmd/roachtest/test_runner.go b/pkg/cmd/roachtest/test_runner.go new file mode 100644 index 000000000000..d268df6b7426 --- /dev/null +++ b/pkg/cmd/roachtest/test_runner.go @@ -0,0 +1,1015 @@ +// Copyright 2018 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 main + +import ( + "context" + "fmt" + "io" + "math/rand" + "net" + "net/http" + "os" + "path/filepath" + "runtime" + "sort" + "strconv" + "strings" + "sync" + "time" + + "github.com/cockroachdb/cockroach/pkg/cmd/internal/issues" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/quotapool" + "github.com/cockroachdb/cockroach/pkg/util/stop" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/cockroachdb/cockroach/pkg/util/version" + "github.com/cockroachdb/logtags" + "github.com/petermattis/goid" + "github.com/pkg/errors" +) + +// testRunner runs tests. +type testRunner struct { + // buildVersion is the version of the Cockroach binary that tests will run against. + buildVersion version.Version + + config struct { + // skipClusterValidationOnAttach skips validation on existing clusters that + // the registry uses for running tests. + skipClusterValidationOnAttach bool + // skipClusterStopOnAttach skips stopping existing clusters that + // the registry uses for running tests. It implies skipClusterWipeOnAttach. + skipClusterStopOnAttach bool + skipClusterWipeOnAttach bool + } + + status struct { + syncutil.Mutex + running map[*test]struct{} + pass map[*test]struct{} + fail map[*test]struct{} + skip map[*test]struct{} + } + + // cr keeps track of all live clusters. + cr *clusterRegistry + + workersMu struct { + syncutil.Mutex + // workers is a map from worker name to information about each worker currently running tests. + workers map[string]*workerStatus + } + + // work maintains the remaining tests to run. + work *workPool + + completedTestsMu struct { + syncutil.Mutex + // completed maintains information on all completed test runs. + completed []completedTestInfo + } +} + +// newTestRunner constructs a testRunner. +// +// cr: The cluster registry with which all clusters will be registered. The +// caller provides this as the caller needs to be able to shut clusters down +// on Ctrl+C. +// buildVersion: The version of the Cockroach binary against which tests will run. +func newTestRunner(cr *clusterRegistry, buildVersion version.Version) *testRunner { + r := &testRunner{ + cr: cr, + buildVersion: buildVersion, + } + r.config.skipClusterWipeOnAttach = !clusterWipe + r.workersMu.workers = make(map[string]*workerStatus) + return r +} + +// clustersOpt groups options for the clusters to be used by the tests. +type clustersOpt struct { + // The type of cluster to use. If localCluster, then no other fields can be + // set. + typ clusterType + + // If set, all the tests will run against this roachprod cluster. + clusterName string + + // If set, all the clusters will use this ID as part of their name. When + // roachtests is invoked by TeamCity, this will be the build id. + clusterID string + // The name of the user running the tests. This will be part of cluster names. + user string + + // cpuQuota specifies how many CPUs can be used concurrently by the roachprod + // clusters. While there's no quota available for creating a new cluster, the + // test runner will wait for other tests to finish and their cluster to be + // destroyed (or reused). Note that this limit is global, not per zone. + cpuQuota int + // If set, clusters will not be wiped or destroyed when a test using the + // respective cluster fails. These cluster will linger around and they'll + // continue counting towards the cpuQuota. + keepClustersOnTestFailure bool +} + +func (c clustersOpt) validate() error { + if c.typ == localCluster { + if c.clusterName != "" { + return errors.New("clusterName cannot be set when typ=localCluster") + } + if c.clusterID != "" { + return errors.New("clusterID cannot be set when typ=localCluster") + } + } + return nil +} + +// Run runs tests. +// +// Args: +// tests: The tests to run. +// count: How many times to run each test selected by filter. +// parallelism: How many workers to use for running tests. Tests are run +// locally (although generally they run against remote roachprod clusters). +// parallelism bounds the maximum number of tests that run concurrently. Note +// that the concurrency is also affected by cpuQuota. +// clusterOpt: Options for the clusters to use by tests. +// artifactsDir: A directory where all the test artifacts (in particular, log +// files) will be placed. +// lopt: Options for logging. +func (r *testRunner) Run( + ctx context.Context, + tests []testSpec, + count int, + parallelism int, + clustersOpt clustersOpt, + artifactsDir string, + lopt loggingOpt, +) error { + // Validate options. + if len(tests) == 0 { + return fmt.Errorf("no test matched filters") + } + if err := clustersOpt.validate(); err != nil { + return err + } + if parallelism != 1 { + if clustersOpt.clusterName != "" { + return fmt.Errorf("--cluster incompatible with --parallelism. Use --parallelism=1") + } + if clustersOpt.typ == localCluster { + return fmt.Errorf("--local incompatible with --parallelism. Use --parallelism=1") + } + } + + if name := clustersOpt.clusterName; name != "" { + // Since we were given a cluster, check that all tests we have to run have compatible specs. + // We should also check against the spec of the cluster, but we don't + // currently have a way of doing that; we're relying on the fact that attaching to the cluster + // will fail if the cluster is incompatible. + spec := tests[0].Cluster + spec.Lifetime = 0 + for i := 1; i < len(tests); i++ { + spec2 := tests[i].Cluster + spec2.Lifetime = 0 + if spec != spec2 { + return errors.Errorf("cluster specified but found tests "+ + "with incompatible specs: %s (%s) - %s (%s)", + tests[0].Name, spec, tests[i].Name, spec2, + ) + } + } + } + + var numConcurrentClusterCreations int + if cloud == "aws" { + // AWS has ridiculous API calls limits, so we're going to create one cluster + // at a time. Internally, roachprod has throttling for the calls required to + // create a single cluster. + numConcurrentClusterCreations = 1 + } else { + numConcurrentClusterCreations = 1000 + } + clusterFactory := newClusterFactory( + clustersOpt.user, clustersOpt.clusterID, artifactsDir, r.cr, numConcurrentClusterCreations) + + // allocateCluster will be used by workers to create new clusters (or to attach + // to an existing one). + allocateCluster := func( + ctx context.Context, + t testSpec, + alloc *quotapool.IntAlloc, + artifactsDir string, + wStatus *workerStatus, + ) (*cluster, error) { + wStatus.SetStatus("creating cluster") + defer wStatus.SetStatus("") + + lopt.l.PrintfCtx(ctx, "Creating new cluster for test %s: %s", t.Name, t.Cluster) + + existingClusterName := clustersOpt.clusterName + if existingClusterName != "" { + // Logs for attaching to a cluster go to a dedicated log file. + logPath := filepath.Join(artifactsDir, "cluster-create", existingClusterName+".log") + clusterL, err := rootLogger(logPath, lopt.tee) + defer clusterL.close() + if err != nil { + return nil, err + } + opt := attachOpt{ + skipValidation: r.config.skipClusterValidationOnAttach, + skipStop: r.config.skipClusterStopOnAttach, + skipWipe: r.config.skipClusterWipeOnAttach, + } + return attachToExistingCluster(ctx, existingClusterName, clusterL, t.Cluster, opt, r.cr) + } + + cfg := clusterConfig{ + spec: t.Cluster, + artifactsDir: artifactsDir, + localCluster: clustersOpt.typ == localCluster, + alloc: alloc, + } + return clusterFactory.newCluster(ctx, cfg, wStatus.SetStatus, lopt.tee) + } + + // Seed the default rand source so that different runs get different cluster + // IDs. + rand.Seed(timeutil.Now().UnixNano()) + + n := len(tests) + if n*count < parallelism { + // Don't spin up more workers than necessary. + parallelism = n * count + } + + r.status.running = make(map[*test]struct{}) + r.status.pass = make(map[*test]struct{}) + r.status.fail = make(map[*test]struct{}) + r.status.skip = make(map[*test]struct{}) + + r.work = newWorkPool(tests, count) + stopper := stop.NewStopper() + errs := &workerErrors{} + + qp := quotapool.NewIntPool("cloud cpu", uint64(clustersOpt.cpuQuota)) + l := lopt.l + + var wg sync.WaitGroup + for i := 0; i < parallelism; i++ { + i := i // Copy for closure. + wg.Add(1) + stopper.RunWorker(ctx, func(ctx context.Context) { + defer wg.Done() + + if err := r.runWorker( + ctx, fmt.Sprintf("w%d", i) /* name */, r.work, qp, + stopper.ShouldQuiesce(), + clustersOpt.keepClustersOnTestFailure, + lopt.artifactsDir, lopt.tee, lopt.stdout, + allocateCluster, + l, + ); err != nil { + // A worker returned an error. Let's shut down. + msg := fmt.Sprintf("Worker %d returned with error. Quiescing. Error: %s", i, err) + shout(ctx, l, lopt.stdout, msg) + errs.AddErr(err) + // Quiesce the stopper. This will cause all workers to not pick up more + // tests after finishing the currently running one. + stopper.Quiesce(ctx) + // Interrupt everybody waiting for resources. + if qp != nil { + qp.Close(msg) + } + } + }) + } + + // Wait for all the workers to finish. + wg.Wait() + r.cr.destroyAllClusters(ctx, l) + + if errs.Err() != nil { + shout(ctx, l, lopt.stdout, "FAIL (err: %s)", errs.Err()) + return errs.Err() + } + passFailLine := r.generateReport() + shout(ctx, l, lopt.stdout, passFailLine) + if len(r.status.fail) > 0 { + return fmt.Errorf("some tests failed") + } + return nil +} + +type clusterAllocatorFn func( + ctx context.Context, + t testSpec, + alloc *quotapool.IntAlloc, + artifactsDir string, + wStatus *workerStatus, +) (*cluster, error) + +// runWorker runs tests in a loop until work is exhausted. +// +// Errors are returned in exceptional circumstances, like when a cluster failed +// to be created or when a test timed out and failed to react to its +// cancelation. Upon return, an attempt is performed to destroy the cluster used +// by this worker. If an error is returned, we might have "leaked" cpu quota +// because the cluster destruction might have failed but we've still released +// the quota. Also, we might have "leaked" a test goroutine (in the test +// nonresponsive to timeout case) which might still be running and doing +// arbitrary things to the cluster it was using. +// +// Args: +// name: The worker's name, to be used as a prefix for log messages. +// artifactsRootDir: The artifacts dir. Each test's logs are going to be under a +// run_ dir. If empty, test log files will not be created. +// stdout: The Writer to use for messages that need to go to stdout (e.g. the +// "=== RUN" and "--- FAIL" lines). +// teeOpt: The teeing option for future test loggers. +// l: The logger to use for more verbose messages. +func (r *testRunner) runWorker( + ctx context.Context, + name string, + work *workPool, + qp *quotapool.IntPool, + interrupt <-chan struct{}, + debug bool, + artifactsRootDir string, + teeOpt teeOptType, + stdout io.Writer, + allocateCluster clusterAllocatorFn, + l *logger, +) error { + ctx = logtags.AddTag(ctx, name, nil /* value */) + wStatus := r.addWorker(ctx, name) + defer func() { + r.removeWorker(ctx, name) + }() + + var c *cluster // The cluster currently being used. + // When this method returns we'll destroy the cluster we had at the time. + // Note that, if debug was set, c has been set to nil. + defer func() { + wStatus.SetTest(nil /* test */, testToRunRes{noWork: true}) + wStatus.SetStatus("worker done") + wStatus.SetCluster(nil) + + if c == nil { + return + } + doDestroy := ctx.Err() == nil + if doDestroy { + l.PrintfCtx(ctx, "Worker exiting; destroying cluster.") + c.Destroy(context.Background(), closeLogger, l) + } else { + l.PrintfCtx(ctx, "Worker exiting with canceled ctx. Not destroying cluster.") + } + }() + + // Loop until there's no more work in the pool, we get interrupted, or an + // error occurs. + for { + select { + case <-interrupt: + l.ErrorfCtx(ctx, "worker detected interruption") + return errors.Errorf("interrupted") + default: + if ctx.Err() != nil { + // The context has been canceled. No need to continue. + return errors.Wrap(ctx.Err(), "worker ctx done") + } + } + + if c != nil { + if _, ok := c.spec.ReusePolicy.(reusePolicyNone); ok { + wStatus.SetStatus("destroying cluster") + // We use a context that can't be canceled for the Destroy(). + c.Destroy(ctx, closeLogger, l) + c = nil + } + } + + var testToRun testToRunRes + var err error + wStatus.SetTest(nil /* test */, testToRunRes{}) + wStatus.SetStatus("getting work") + testToRun, c, err = r.getWork( + ctx, work, qp, c, interrupt, l, + getWorkCallbacks{ + createCluster: func(ctx context.Context, ttr testToRunRes) (*cluster, error) { + wStatus.SetTest(nil /* test */, ttr) + return allocateCluster(ctx, ttr.spec, ttr.alloc, artifactsRootDir, wStatus) + }, + onDestroy: func() { + wStatus.SetCluster(nil) + }, + }) + if err != nil || testToRun.noWork { + return err + } + c.status("running test") + + // Prepare the test's logger. + logPath := "" + var artifactsDir string + if artifactsRootDir != "" { + artifactsSuffix := "run_" + strconv.Itoa(testToRun.runNum) + artifactsDir = filepath.Join( + artifactsRootDir, teamCityNameEscape(testToRun.spec.Name), artifactsSuffix) + logPath = filepath.Join(artifactsDir, "test.log") + } + testL, err := rootLogger(logPath, teeOpt) + if err != nil { + return err + } + t := &test{ + spec: &testToRun.spec, + buildVersion: r.buildVersion, + artifactsDir: artifactsDir, + l: testL, + } + // Tell the cluster that, from now on, it will be run "on behalf of this + // test". + c.setTest(t) + wStatus.SetCluster(c) + wStatus.SetTest(t, testToRun) + wStatus.SetStatus("running test") + + // Now run the test. + l.PrintfCtx(ctx, "starting test: %s:%d", testToRun.spec.Name, testToRun.runNum) + var success bool + success, err = r.runTest(ctx, t, testToRun.runNum, c, t.ArtifactsDir(), stdout, testL) + testL.close() + if err != nil { + shout(ctx, l, stdout, "test returned error: %s: %s", t.Name(), err) + // Mark the test as failed if it isn't already. + if !t.Failed() { + t.printAndFail(0 /* skip */, err) + } + } else { + msg := "test passed" + if !success { + msg = fmt.Sprintf("test failed: %s (run %d)", t.Name(), testToRun.runNum) + } + l.PrintfCtx(ctx, msg) + } + // If a test failed and debug was set, we bail. + if (err != nil || t.Failed()) && debug { + failureMsg := fmt.Sprintf("%s (%d) - ", testToRun.spec.Name, testToRun.runNum) + if err != nil { + failureMsg += err.Error() + } else { + failureMsg += t.FailureMsg() + } + // Save the cluster for future debugging. + c.Save(ctx, failureMsg, l) + + if err != nil { + return err + } + + // Continue with a fresh cluster. + c = nil + } + } +} + +// An error is returned in exceptional situations. The cluster cannot be reused +// if an error is returned. +// Returns true if the test is considered to have passed, false otherwise. +// +// Args: +// c: The cluster on which the test will run. runTest() does not wipe or destroy +// the cluster. +// artifactsDir: A dir that will be collected by TeamCity. All the test's +// artifacts are expected to be there. +func (r *testRunner) runTest( + ctx context.Context, + t *test, + runNum int, + c *cluster, + artifactsDir string, + stdout io.Writer, + l *logger, +) (bool, error) { + if t.spec.Skip != "" { + return false, fmt.Errorf("can't run skipped test: %s: %s", t.Name(), t.spec.Skip) + } + + if teamCity { + shout(ctx, l, stdout, "##teamcity[testStarted name='%s' flowId='%s']", t.Name(), t.Name()) + } else { + shout(ctx, l, stdout, "=== RUN %s", t.Name()) + } + + r.status.Lock() + r.status.running[t] = struct{}{} + r.status.Unlock() + + t.runner = callerName() + t.runnerID = goid.Get() + + defer func() { + t.end = timeutil.Now() + + if err := recover(); err != nil { + t.mu.Lock() + t.mu.failed = true + t.mu.output = append(t.mu.output, t.decorate(0 /* skip */, fmt.Sprint(err))...) + t.mu.Unlock() + } + + t.mu.Lock() + t.mu.done = true + t.mu.Unlock() + + durationStr := fmt.Sprintf("%.2fs", t.duration().Seconds()) + if t.Failed() { + t.mu.Lock() + output := fmt.Sprintf("test artifacts and logs in: %s\n", t.ArtifactsDir()) + string(t.mu.output) + failLoc := t.mu.failLoc + t.mu.Unlock() + + if teamCity { + shout(ctx, l, stdout, "##teamcity[testFailed name='%s' details='%s' flowId='%s']", + t.Name(), teamCityEscape(string(output)), t.Name(), + ) + } + + shout(ctx, l, stdout, "--- FAIL: %s %s\n%s", t.Name(), durationStr, output) + if issues.CanPost() && t.spec.Run != nil { + authorEmail := getAuthorEmail(failLoc.file, failLoc.line) + branch := "" + if b := os.Getenv("TC_BUILD_BRANCH"); b != "" { + branch = b + } + msg := fmt.Sprintf("The test failed on branch=%s, cloud=%s:\n%s", + branch, cloud, output) + + if err := issues.Post( + context.Background(), + fmt.Sprintf("roachtest: %s failed", t.Name()), + "roachtest", t.Name(), msg, authorEmail, []string{"O-roachtest"}, + ); err != nil { + shout(ctx, l, stdout, "failed to post issue: %s", err) + } + } + } else { + shout(ctx, l, stdout, "--- PASS: %s %s", t.Name(), durationStr) + // If `##teamcity[testFailed ...]` is not present before `##teamCity[testFinished ...]`, + // TeamCity regards the test as successful. + } + + if teamCity { + shout(ctx, l, stdout, "##teamcity[testFinished name='%s' flowId='%s']", t.Name(), t.Name()) + + escapedTestName := teamCityNameEscape(t.Name()) + artifactsGlobPath := filepath.Join(artifactsDir, escapedTestName, "**") + artifactsSpec := fmt.Sprintf("%s => %s", artifactsGlobPath, escapedTestName) + shout(ctx, l, stdout, "##teamcity[publishArtifacts '%s']", artifactsSpec) + } + + r.recordTestFinish(completedTestInfo{ + test: t.Name(), + run: runNum, + start: t.start, + end: t.end, + pass: !t.Failed(), + failure: t.FailureMsg(), + }) + r.status.Lock() + delete(r.status.running, t) + // Only include tests with a Run function in the summary output. + if t.spec.Run != nil { + if t.Failed() { + r.status.fail[t] = struct{}{} + } else if t.spec.Skip == "" { + r.status.pass[t] = struct{}{} + } else { + r.status.skip[t] = struct{}{} + } + } + r.status.Unlock() + }() + + t.start = timeutil.Now() + + timeout := 10 * time.Hour + if t.spec.Timeout != 0 { + timeout = t.spec.Timeout + } + // Make sure the cluster has enough life left for the test plus enough headroom + // after the test finishes so that the next test can be selected. If it + // doesn't, extend it. + minExp := timeutil.Now().Add(timeout + time.Hour) + if c.expiration.Before(minExp) { + extend := minExp.Sub(c.expiration) + l.PrintfCtx(ctx, "cluster needs to survive until %s, but has expiration: %s. Extending.", + minExp, c.expiration) + if err := c.Extend(ctx, extend, l); err != nil { + t.printfAndFail(0 /* skip */, "failed to extend cluster") + return false, err + } + } + + runCtx, cancel := context.WithCancel(ctx) + defer cancel() + t.mu.Lock() + // t.Fatal() will cancel this context. + t.mu.cancel = cancel + t.mu.Unlock() + + // We run the actual test in a different goroutine because it might call + // t.Fatal() which kills the goroutine, and also because we want to enforce a + // timeout. + success := false + done := make(chan struct{}) + go func() { + defer close(done) // closed only after we've grabbed the debug info below + + defer func() { + // Detect replica divergence (i.e. ranges in which replicas have arrived + // at the same log position with different states). + c.FailOnReplicaDivergence(ctx, t) + // Detect dead nodes in an inner defer. Note that this will call + // t.printfAndFail() when appropriate, which will cause the code below to + // enter the t.Failed() branch. + c.FailOnDeadNodes(ctx, t) + + if !t.Failed() { + success = true + return + } + + if err := c.FetchDebugZip(ctx); err != nil { + t.l.Printf("failed to download logs: %s", err) + } + if err := c.FetchDmesg(ctx); err != nil { + c.l.Printf("failed to fetch dmesg: %s", err) + } + if err := c.FetchJournalctl(ctx); err != nil { + c.l.Printf("failed to fetch journalctl: %s", err) + } + if err := c.FetchCores(ctx); err != nil { + c.l.Printf("failed to fetch cores: %s", err) + } + if err := c.CopyRoachprodState(ctx); err != nil { + c.l.Printf("failed to copy roachprod state: %s", err) + } + // 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 + // duplication is acceptable. + if err := c.FetchLogs(ctx); err != nil { + c.l.Printf("failed to download logs: %s", err) + } + }() + + // This is the call to actually run the test. + t.spec.Run(runCtx, t, c) + }() + + select { + case <-done: + case <-time.After(timeout): + // We hit a timeout. We're going to mark the test as failed (which will also + // cancel its context). Then we'll wait up to 5 minutes in the hope + // that the test reacts either to the ctx cancelation or to the fact that it + // was marked as failed. If that happens, great - we return normally and so + // the cluster can be reused. It the test does not react to anything, then + // we return an error, which will cause the caller to stop everything and + // destroy this cluster (as well as all the others). The cluster + // cannot be reused since we have a runaway test goroutine that's presumably + // going to continue using the cluster. + t.printfAndFail(0 /* skip */, "test timed out (%s)", timeout) + select { + case <-done: + if success { + panic("expected success=false after a timeout") + } + case <-time.After(5 * time.Minute): + return false, fmt.Errorf("test timed out and afterwards failed to respond to cancelation") + } + } + return success, nil +} + +func callerName() string { + // Make room for the skip PC. + var pc [2]uintptr + n := runtime.Callers(2, pc[:]) // runtime.Callers + callerName + if n == 0 { + panic("zero callers found") + } + frames := runtime.CallersFrames(pc[:n]) + frame, _ := frames.Next() + return frame.Function +} + +// generateReport produces the final pass/fail line and produces a slack report +// if configured. +func (r *testRunner) generateReport() string { + r.status.Lock() + defer r.status.Unlock() + postSlackReport(r.status.pass, r.status.fail, r.status.skip) + + fails := len(r.status.fail) + var msg string + if fails > 0 { + msg = fmt.Sprintf("FAIL (%d fails)\n", fails) + } else { + msg = "PASS" + } + return msg +} + +type getWorkCallbacks struct { + createCluster func(context.Context, testToRunRes) (*cluster, error) + onDestroy func() +} + +// getWork selects the next test to run and creates a suitable cluster for it if +// need be. If a new cluster needs to be created, the method blocks until there +// are enough resources available to run it. +// getWork takes in a cluster; if not nil, tests that can reuse it are +// preferred. If a test that can reuse it is not found (or if there's no more +// work), the cluster is destroyed (and so its resources are released). +// +// If the cluster is to be reused, getWork() wipes it. +func (r *testRunner) getWork( + ctx context.Context, + work *workPool, + qp *quotapool.IntPool, + c *cluster, + interrupt <-chan struct{}, + l *logger, + callbacks getWorkCallbacks, +) (testToRunRes, *cluster, error) { + + select { + case <-interrupt: + return testToRunRes{}, nil, fmt.Errorf("interrupted") + default: + } + + testToRun, err := work.getTestToRun(ctx, c, qp, r.cr, callbacks.onDestroy, l) + if err != nil { + return testToRunRes{}, nil, err + } + if !testToRun.noWork { + l.PrintfCtx(ctx, "Selected test: %s run: %d.", testToRun.spec.Name, testToRun.runNum) + } + // Are we done? + if testToRun.noWork { + return testToRun, nil, nil + } + + // Create a cluster, if we no longer have one. + if testToRun.canReuseCluster { + l.PrintfCtx(ctx, "Using existing cluster: %s. Wiping", c.name) + if err := c.WipeE(ctx, l); err != nil { + return testToRunRes{}, nil, err + } + // Overwrite the spec of the cluster with the one coming from the test. In + // particular, this overwrites the reuse policy to reflect what the test + // intends to do with it. + c.spec = testToRun.spec.Cluster + } else { + var err error + c, err = callbacks.createCluster(ctx, testToRun) + if err != nil { + return testToRunRes{}, nil, err + } + } + return testToRun, c, nil +} + +// addWorker updates the bookkeeping for one more worker. +func (r *testRunner) addWorker(ctx context.Context, name string) *workerStatus { + r.workersMu.Lock() + defer r.workersMu.Unlock() + w := &workerStatus{name: name} + if _, ok := r.workersMu.workers[name]; ok { + log.Fatalf(ctx, "worker %q already exists", name) + } + r.workersMu.workers[name] = w + return w +} + +// removeWorker deletes the bookkepping for a worker that has finished running. +func (r *testRunner) removeWorker(ctx context.Context, name string) { + r.workersMu.Lock() + delete(r.workersMu.workers, name) + r.workersMu.Unlock() +} + +// runHTTPServer starts a server running in the background. +// +// httpPort: The port on which to serve the web interface. Pass 0 for allocating +// a port automatically (which will be printed to stdout). +func (r *testRunner) runHTTPServer(httpPort int, stdout io.Writer) error { + http.HandleFunc("/", r.serveHTTP) + // Run an http server in the background. + // We handle the case where httpPort is 0, which means we automatically + // allocate a port. + listener, err := net.Listen("tcp", fmt.Sprintf(":%d", httpPort)) + if err != nil { + return err + } + httpPort = listener.Addr().(*net.TCPAddr).Port + go func() { + if err := http.Serve(listener, nil /* handler */); err != nil { + panic(err) + } + }() + fmt.Fprintf(stdout, "HTTP server listening on all network interfaces, port %d.\n", httpPort) + return nil +} + +// serveHTTP is the handler for the test runner's web server. +func (r *testRunner) serveHTTP(wr http.ResponseWriter, req *http.Request) { + fmt.Fprintf(wr, "") + fmt.Fprintf(wr, "pprof") + fmt.Fprintf(wr, "

") + // Print the workers report. + fmt.Fprintf(wr, "

Workers:

") + fmt.Fprintf(wr, ` + + + + + + + `) + r.workersMu.Lock() + workers := make([]*workerStatus, len(r.workersMu.workers)) + i := 0 + for _, w := range r.workersMu.workers { + workers[i] = w + i++ + } + r.workersMu.Unlock() + sort.Slice(workers, func(i int, j int) bool { + l := workers[i] + r := workers[j] + return strings.Compare(l.name, r.name) < 0 + }) + for _, w := range workers { + var testName string + ttr := w.TestToRun() + clusterReused := "" + if ttr.noWork { + testName = "done" + } else if ttr.spec.Name == "" { + testName = "N/A" + } else { + testName = fmt.Sprintf("%s (run %d)", ttr.spec.Name, ttr.runNum) + if ttr.canReuseCluster { + clusterReused = "yes" + } else { + clusterReused = "no" + } + } + var clusterName string + if w.Cluster() != nil { + clusterName = w.Cluster().name + } + t := w.Test() + testStatus := "N/A" + if t != nil { + testStatus = t.GetStatus() + } + fmt.Fprintf(wr, "\n", + w.name, w.Status(), testName, clusterName, clusterReused, testStatus) + } + fmt.Fprintf(wr, "
WorkerWorker StatusTestClusterCluster reusedTest Status
%s%s%s%s%s%s
") + + // Print the finished tests report. + fmt.Fprintf(wr, "

") + fmt.Fprintf(wr, "

Finished tests:

") + fmt.Fprintf(wr, ` + + + + `) + for _, t := range r.getCompletedTests() { + name := fmt.Sprintf("%s (run %d)", t.test, t.run) + status := "PASS" + if !t.pass { + status = "FAIL " + t.failure + } + duration := fmt.Sprintf("%s (%s - %s)", t.end.Sub(t.start), t.start, t.end) + fmt.Fprintf(wr, "", name, status, duration) + } + fmt.Fprintf(wr, "
TestStatusDuration
%s%s%s
") + + // Print the saved clusters report. + fmt.Fprintf(wr, "

") + fmt.Fprintf(wr, "

Clusters left alive for further debugging "+ + "(if --debug was specified):

") + fmt.Fprintf(wr, ` + + + `) + for c, msg := range r.cr.savedClusters() { + fmt.Fprintf(wr, "", c.name, msg) + } + fmt.Fprintf(wr, "
ClusterTest
%s%s
") + + fmt.Fprintf(wr, "

") + fmt.Fprintf(wr, "

Tests left:

") + fmt.Fprintf(wr, ` + + + `) + for _, t := range r.work.workRemaining() { + fmt.Fprintf(wr, "", t.spec.Name, t.count) + } + + fmt.Fprintf(wr, "") +} + +// recordTestFinish updated bookkeeping when a test finishes. +func (r *testRunner) recordTestFinish(info completedTestInfo) { + r.completedTestsMu.Lock() + r.completedTestsMu.completed = append(r.completedTestsMu.completed, info) + r.completedTestsMu.Unlock() +} + +// getCompletedTests returns info on all tests that finished running. +func (r *testRunner) getCompletedTests() []completedTestInfo { + r.completedTestsMu.Lock() + defer r.completedTestsMu.Unlock() + res := make([]completedTestInfo, len(r.completedTestsMu.completed)) + copy(res, r.completedTestsMu.completed) + return res +} + +// completedTestInfo represents information on a completed test run. +type completedTestInfo struct { + test string + run int + start time.Time + end time.Time + pass bool + failure string +} + +// PredecessorVersion returns a recent predecessor of the build version (i.e. +// the build tag of the main binary). For example, if the running binary is from +// the master branch prior to releasing 19.2.0, this will return a recent +// (ideally though not necessarily the latest) 19.1 patch release. +func PredecessorVersion(buildVersion version.Version) (string, error) { + if buildVersion == (version.Version{}) { + return "", errors.Errorf("buildVersion not set") + } + + buildVersionMajorMinor := fmt.Sprintf("%d.%d", buildVersion.Major(), buildVersion.Minor()) + + verMap := map[string]string{ + "19.2": "19.1.0-rc.4", + "19.1": "2.1.6", + "2.2": "2.1.6", + "2.1": "2.0.7", + } + v, ok := verMap[buildVersionMajorMinor] + if !ok { + return "", errors.Errorf("prev version not set for version: %s", buildVersionMajorMinor) + } + return v, nil +} + +type workerErrors struct { + mu struct { + syncutil.Mutex + errs []error + } +} + +func (we *workerErrors) AddErr(err error) { + we.mu.Lock() + defer we.mu.Unlock() + we.mu.errs = append(we.mu.errs, err) +} + +func (we *workerErrors) Err() error { + we.mu.Lock() + defer we.mu.Unlock() + if len(we.mu.errs) == 0 { + return nil + } + // TODO(andrei): Maybe we should do something other than return the first + // error... + return we.mu.errs[0] +} diff --git a/pkg/cmd/roachtest/test_test.go b/pkg/cmd/roachtest/test_test.go index 1fe2da4dd8df..914d92081374 100644 --- a/pkg/cmd/roachtest/test_test.go +++ b/pkg/cmd/roachtest/test_test.go @@ -13,17 +13,16 @@ package main import ( "bytes" "context" - "fmt" "io/ioutil" "regexp" "sort" "strings" - "sync" "testing" "time" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/syncutil" + "github.com/cockroachdb/cockroach/pkg/util/version" "github.com/kr/pretty" ) @@ -61,37 +60,72 @@ func TestMatchOrSkip(t *testing.T) { } } -func TestRegistryRun(t *testing.T) { - r := newRegistry() - r.out = ioutil.Discard +func nilLogger() *logger { + lcfg := loggerConfig{ + stdout: ioutil.Discard, + stderr: ioutil.Discard, + } + l, err := lcfg.newLogger("" /* path */) + if err != nil { + panic(err) + } + return l +} + +func TestRunnerRun(t *testing.T) { + ctx := context.Background() + r, err := makeTestRegistry() + if err != nil { + t.Fatal(err) + } r.Add(testSpec{ - Name: "pass", - Run: func(ctx context.Context, t *test, c *cluster) { - }, + Name: "pass", + Run: func(ctx context.Context, t *test, c *cluster) {}, + Cluster: makeClusterSpec(0), }) r.Add(testSpec{ Name: "fail", Run: func(ctx context.Context, t *test, c *cluster) { t.Fatal("failed") }, + Cluster: makeClusterSpec(0), }) testCases := []struct { - filters []string - expected int + filters []string + expErr string }{ - {nil, 1}, - {[]string{"pass"}, 0}, - {[]string{"fail"}, 1}, - {[]string{"pass|fail"}, 1}, - {[]string{"pass", "fail"}, 1}, - {[]string{"notests"}, 1}, + {nil, "some tests failed"}, + {[]string{"pass"}, ""}, + {[]string{"fail"}, "some tests failed"}, + {[]string{"pass|fail"}, "some tests failed"}, + {[]string{"pass", "fail"}, "some tests failed"}, + {[]string{"notests"}, "no test"}, } for _, c := range testCases { t.Run("", func(t *testing.T) { - code := r.Run(c.filters, defaultParallelism, "" /* artifactsDir */, "myuser") - if c.expected != code { - t.Fatalf("expected code %d, but found code %d. Filters: %s", c.expected, code, c.filters) + tests := testsToRun(ctx, r, newFilter(c.filters)) + cr := newClusterRegistry() + runner := newTestRunner(cr, r.buildVersion) + + lopt := loggingOpt{ + l: nilLogger(), + tee: noTee, + stdout: ioutil.Discard, + stderr: ioutil.Discard, + artifactsDir: "", + } + copt := clustersOpt{ + typ: roachprodCluster, + user: "test_user", + cpuQuota: 1000, + keepClustersOnTestFailure: false, + } + err := runner.Run(ctx, tests, 1, /* count */ + defaultParallelism, copt, "" /* artifactsDir */, lopt) + + if !testutils.IsError(err, c.expErr) { + t.Fatalf("expected err: %q, but found %v. Filters: %s", c.expErr, err, c.filters) } }) } @@ -114,219 +148,52 @@ func (b *syncedBuffer) String() string { return b.buf.String() } -func TestRegistryStatus(t *testing.T) { - var buf syncedBuffer - waitingRE := regexp.MustCompile(`(?m)^.*status: waiting.*worker?.*worker?.*$`) - cleaningUpRE := regexp.MustCompile(`(?m)^.*status: cleaning up \(.*\)$`) +func TestRunnerTestTimeout(t *testing.T) { + ctx := context.Background() - r := newRegistry() - r.out = &buf - r.statusInterval = 20 * time.Millisecond - r.Add(testSpec{ - Name: `status`, - Run: func(ctx context.Context, t *test, c *cluster) { - t.Status("waiting") - var wg sync.WaitGroup - for i := 0; i < 2; i++ { - wg.Add(1) - go func(i int) { - defer wg.Done() - t.WorkerStatus("worker", i) - defer t.WorkerStatus() - for i := 0.0; i < 1.0; i += 0.01 { - t.WorkerProgress(i) - time.Sleep(r.statusInterval) - if waitingRE.MatchString(buf.String()) { - break - } - } - }(i) - } - wg.Wait() - t.Status("cleaning up") - for i := 0.0; i < 1.0; i += 0.01 { - t.Progress(i) - time.Sleep(r.statusInterval) - if cleaningUpRE.MatchString(buf.String()) { - break - } - } - }, - }) - r.Run([]string{"status"}, defaultParallelism, "" /* artifactsDir */, "myuser") + cr := newClusterRegistry() + runner := newTestRunner(cr, version.Version{}) - status := buf.String() - if !waitingRE.MatchString(status) { - t.Fatalf("unable to find \"waiting\" status:\n%s", status) - } - if !cleaningUpRE.MatchString(status) { - t.Fatalf("unable to find \"cleaning up\" status:\n%s", status) - } - if testing.Verbose() { - fmt.Println(status) - } -} - -func TestRegistryStatusUnknown(t *testing.T) { var buf syncedBuffer - unknownRE := regexp.MustCompile(`(?m)^.*status: \?\?\? \(.*\)$`) - - r := newRegistry() - r.out = &buf - r.statusInterval = 20 * time.Millisecond - - r.Add(testSpec{ - Name: `status`, - Run: func(ctx context.Context, t *test, c *cluster) { - for i := 0; i < 100; i++ { - time.Sleep(r.statusInterval) - if unknownRE.MatchString(buf.String()) { - break - } - } - }, - }) - r.Run([]string{"status"}, defaultParallelism, "" /* artifactsDir */, "myuser") - - status := buf.String() - if !unknownRE.MatchString(status) { - t.Fatalf("unable to find \"waiting\" status:\n%s", status) + lopt := loggingOpt{ + l: nilLogger(), + tee: noTee, + stdout: &buf, + stderr: &buf, + artifactsDir: "", } - if testing.Verbose() { - fmt.Println(status) + copt := clustersOpt{ + typ: roachprodCluster, + user: "test_user", + cpuQuota: 1000, + keepClustersOnTestFailure: false, } -} - -func TestRegistryRunTimeout(t *testing.T) { - var buf syncedBuffer - timeoutRE := regexp.MustCompile(`(?m)^.*test timed out \(.*\)$`) - - r := newRegistry() - r.out = &buf - - r.Add(testSpec{ + test := testSpec{ Name: `timeout`, Timeout: 10 * time.Millisecond, + Cluster: makeClusterSpec(0), Run: func(ctx context.Context, t *test, c *cluster) { <-ctx.Done() }, - }) - r.Run([]string{"timeout"}, defaultParallelism, "" /* artifactsDir */, "myuser") - - out := buf.String() - if !timeoutRE.MatchString(out) { - t.Fatalf("unable to find \"timed out\" message:\n%s", out) } -} - -func TestRegistryRunSubTestFailed(t *testing.T) { - var buf syncedBuffer - failedRE := regexp.MustCompile(`(?m)^.*--- FAIL: parent \(.*$`) - - r := newRegistry() - r.out = &buf - r.Add(testSpec{ - Name: "parent", - SubTests: []testSpec{{ - Name: "child", - Run: func(ctx context.Context, t *test, c *cluster) { - t.Fatal("failed") - }, - }}, - }) - - r.Run([]string{"."}, defaultParallelism, "" /* artifactsDir */, "myuser") - out := buf.String() - if !failedRE.MatchString(out) { - t.Fatalf("unable to find \"FAIL: parent\" message:\n%s", out) + err := runner.Run(ctx, []testSpec{test}, 1, /* count */ + defaultParallelism, copt, "" /* artifactsDir */, lopt) + if !testutils.IsError(err, "some tests failed") { + t.Fatalf("expected error \"some tests failed\", got: %v", err) } -} - -func TestRegistryRunNoTests(t *testing.T) { - var buf syncedBuffer - failedRE := regexp.MustCompile(`(?m)^warning: no tests to run \[notest\]\nFAIL$`) - - r := newRegistry() - r.out = &buf - r.Add(testSpec{ - Name: "some-test", - Run: func(ctx context.Context, t *test, c *cluster) { - t.Fatal("failed") - }, - }) - - r.Run([]string{"notest"}, defaultParallelism, "" /* artifactsDir */, "myuser") - out := buf.String() - if !failedRE.MatchString(out) { - t.Fatalf("unable to find \"warning: no tests to run\" message:\n%s", out) - } -} - -func TestRegistryRunClusterExpired(t *testing.T) { - defer func(l bool, n string) { - local, clusterName = l, n - }(local, clusterName) - local, clusterName = true, "local" - - var buf syncedBuffer - expiredRE := regexp.MustCompile(`(?m)^.*cluster expired \(.*\)$`) - - r := newRegistry() - r.config.skipClusterValidationOnAttach = true - r.config.skipClusterStopOnAttach = true - r.out = &buf - - r.Add(testSpec{ - Name: `expired`, - Cluster: makeClusterSpec(1, nodeLifetimeOption(time.Second)), - Run: func(ctx context.Context, t *test, c *cluster) { - panic("not reached") - }, - }) - r.Run([]string{"expired"}, defaultParallelism, "" /* artifactsDir */, "myuser") out := buf.String() - if !expiredRE.MatchString(out) { - t.Fatalf("unable to find \"cluster expired\" message:\n%s", out) - } -} - -func TestRegistryVerifyValidClusterName(t *testing.T) { - testCases := []struct { - testNames []string - expectedErr string - }{ - {[]string{"hello"}, ""}, - {[]string{"HELLO", "hello"}, "have equivalent nightly cluster names"}, - {[]string{"hel+lo", "hel++lo"}, "have equivalent nightly cluster names"}, - {[]string{"hello+"}, "must match regex"}, - {[]string{strings.Repeat("y", 41)}, ""}, - {[]string{strings.Repeat("y", 42)}, "must match regex"}, - } - for _, c := range testCases { - t.Run("", func(t *testing.T) { - r := newRegistry() - var err error - for _, n := range c.testNames { - err = r.verifyValidClusterName(n) - } - if !testutils.IsError(err, c.expectedErr) { - t.Fatalf("expected %s, but found %v", c.expectedErr, err) - } - }) + timeoutRE := regexp.MustCompile(`(?m)^.*test timed out \(.*\)$`) + if !timeoutRE.MatchString(out) { + t.Fatalf("unable to find \"timed out\" message:\n%s", out) } } func TestRegistryPrepareSpec(t *testing.T) { dummyRun := func(context.Context, *test, *cluster) {} - var listTests func(t *testSpec) []string - listTests = func(t *testSpec) []string { - r := []string{t.Name} - for i := range t.SubTests { - r = append(r, listTests(&t.SubTests[i])...) - } - return r + var listTests = func(t *testSpec) []string { + return []string{t.Name} } testCases := []struct { @@ -336,116 +203,41 @@ func TestRegistryPrepareSpec(t *testing.T) { }{ { testSpec{ - Name: "a", - Run: dummyRun, + Name: "a", + Run: dummyRun, + Cluster: makeClusterSpec(0), }, "", []string{"a"}, }, - { - testSpec{ - Name: "a", - SubTests: []testSpec{{ - Name: "b", - Run: dummyRun, - }}, - }, - "", - []string{"a", "a/b"}, - }, - { - testSpec{ - Name: "a", - Run: dummyRun, - SubTests: []testSpec{{ - Name: "b", - Run: dummyRun, - }}, - }, - "a: must specify only one of Run or SubTests", - nil, - }, - { - testSpec{ - Name: "a", - SubTests: []testSpec{{ - Name: "b", - }}, - }, - "a/b: must specify only one of Run or SubTests", - nil, - }, - { - testSpec{ - Name: "a", - SubTests: []testSpec{{ - Name: "b", - Run: dummyRun, - SubTests: []testSpec{{ - Name: "c", - Run: dummyRun, - }}, - }}, - }, - "b: must specify only one of Run or SubTests", - nil, - }, - { - testSpec{ - Name: "a", - SubTests: []testSpec{{ - Name: "b", - Cluster: makeClusterSpec(1), - Run: dummyRun, - }}, - }, - "a/b: subtest may not provide cluster specification", - nil, - }, { testSpec{ Name: "a", MinVersion: "v2.1.0", Run: dummyRun, + Cluster: makeClusterSpec(0), }, "", []string{"a"}, }, - { - testSpec{ - Name: "a", - MinVersion: "v2.1.0-foo", - Run: dummyRun, - }, - regexp.QuoteMeta(`invalid version v2.1.0-foo, cannot specify a prerelease (-xxx)`), - nil, - }, { testSpec{ Name: "a", MinVersion: "foo", Run: dummyRun, + Cluster: makeClusterSpec(0), }, "a: unable to parse min-version: invalid version string 'foo'", nil, }, - { - testSpec{ - Name: "a", - Timeout: time.Second, - SubTests: []testSpec{{ - Name: "b", - Run: dummyRun, - }}, - }, - "a: timeouts only apply to tests specifying Run", - nil, - }, } for _, c := range testCases { t.Run("", func(t *testing.T) { - r := newRegistry() - err := r.prepareSpec(&c.spec, 0) + r, err := makeTestRegistry() + if err != nil { + t.Fatal(err) + } + err = r.prepareSpec(&c.spec) if !testutils.IsError(err, c.expectedErr) { t.Fatalf("expected %q, but found %q", c.expectedErr, err.Error()) } @@ -461,24 +253,28 @@ func TestRegistryPrepareSpec(t *testing.T) { } func TestRegistryMinVersion(t *testing.T) { + ctx := context.Background() testCases := []struct { buildVersion string expectedA bool expectedB bool + expErr string }{ - {"v1.1.0", false, false}, - {"v2.0.0", true, false}, - {"v2.1.0", true, true}, + {"v1.1.0", false, false, "no test matched filters"}, + {"v2.0.0", true, false, ""}, + {"v2.1.0", true, true, ""}, } for _, c := range testCases { t.Run(c.buildVersion, func(t *testing.T) { - var buf syncedBuffer var runA, runB bool - r := newRegistry() - r.out = &buf + r, err := makeTestRegistry() + if err != nil { + t.Fatal(err) + } r.Add(testSpec{ Name: "a", MinVersion: "v2.0.0", + Cluster: makeClusterSpec(0), Run: func(ctx context.Context, t *test, c *cluster) { runA = true }, @@ -486,6 +282,7 @@ func TestRegistryMinVersion(t *testing.T) { r.Add(testSpec{ Name: "b", MinVersion: "v2.1.0", + Cluster: makeClusterSpec(0), Run: func(ctx context.Context, t *test, c *cluster) { runB = true }, @@ -493,7 +290,30 @@ func TestRegistryMinVersion(t *testing.T) { if err := r.setBuildVersion(c.buildVersion); err != nil { t.Fatal(err) } - r.Run(nil /* filter */, defaultParallelism, "" /* artifactsDir */, "myuser") + tests := testsToRun(ctx, r, newFilter(nil)) + + var buf syncedBuffer + lopt := loggingOpt{ + l: nilLogger(), + tee: noTee, + stdout: &buf, + stderr: &buf, + artifactsDir: "", + } + copt := clustersOpt{ + typ: roachprodCluster, + user: "test_user", + cpuQuota: 1000, + keepClustersOnTestFailure: false, + } + cr := newClusterRegistry() + runner := newTestRunner(cr, r.buildVersion) + err = runner.Run(ctx, tests, 1, /* count */ + defaultParallelism, copt, "" /* artifactsDir */, lopt) + if !testutils.IsError(err, c.expErr) { + t.Fatalf("expected err: %q, got: %v", c.expErr, err) + } + if c.expectedA != runA || c.expectedB != runB { t.Fatalf("expected %t,%t, but got %t,%t\n%s", c.expectedA, c.expectedB, runA, runB, buf.String()) diff --git a/pkg/cmd/roachtest/tpcc.go b/pkg/cmd/roachtest/tpcc.go index 42a60370d6d9..721ed9726842 100644 --- a/pkg/cmd/roachtest/tpcc.go +++ b/pkg/cmd/roachtest/tpcc.go @@ -41,7 +41,13 @@ type tpccOptions struct { Chaos func() Chaos // for late binding of stopper During func(context.Context) error // for running a function during the test Duration time.Duration - ZFS bool + + // ZFS, if set, will make the cluster use a ZFS volume. + // Be careful with ClusterReusePolicy when using this. + // + // TODO(andrei): move this to the test's cluster spec. + ZFS bool + // The CockroachDB versions to deploy. The first one indicates the first node, // etc. To use the main binary, specify "". When Versions is nil, it defaults // to "" for all nodes. When it is specified, len(Versions) needs to match the @@ -88,8 +94,8 @@ func tpccFixturesCmd(t *test, cloud string, warehouses int, extraArgs string) st } func runTPCC(ctx context.Context, t *test, c *cluster, opts tpccOptions) { - crdbNodes := c.Range(1, c.nodes-1) - workloadNode := c.Node(c.nodes) + crdbNodes := c.Range(1, c.spec.NodeCount-1) + workloadNode := c.Node(c.spec.NodeCount) rampDuration := 5 * time.Minute if c.isLocal() { opts.Warehouses = 1 @@ -98,9 +104,9 @@ func runTPCC(ctx context.Context, t *test, c *cluster, opts tpccOptions) { } if n := len(opts.Versions); n == 0 { - opts.Versions = make([]string, c.nodes-1) - } else if n != c.nodes-1 { - t.Fatalf("must specify Versions for all %d nodes: %v", c.nodes-1, opts.Versions) + opts.Versions = make([]string, c.spec.NodeCount-1) + } else if n != c.spec.NodeCount-1 { + t.Fatalf("must specify Versions for all %d nodes: %v", c.spec.NodeCount-1, opts.Versions) } { @@ -163,7 +169,7 @@ func runTPCC(ctx context.Context, t *test, c *cluster, opts tpccOptions) { cmd := fmt.Sprintf( "./workload run tpcc --warehouses=%d --histograms=logs/stats.json "+ opts.Extra+" --ramp=%s --duration=%s {pgurl:1-%d}", - opts.Warehouses, rampDuration, opts.Duration, c.nodes-1) + opts.Warehouses, rampDuration, opts.Duration, c.spec.NodeCount-1) c.Run(ctx, workloadNode, cmd) return nil }) @@ -204,7 +210,7 @@ var tpccSupportedWarehouses = []struct { {hardware: "gce-n5cpu16", v: version.MustParse(`v2.1.0-0`), warehouses: 1300}, } -func (r *registry) maxSupportedTPCCWarehouses(cloud string, nodes clusterSpec) int { +func maxSupportedTPCCWarehouses(buildVersion version.Version, cloud string, nodes clusterSpec) int { var v *version.Version var warehouses int hardware := fmt.Sprintf(`%s-%s`, cloud, &nodes) @@ -212,7 +218,7 @@ func (r *registry) maxSupportedTPCCWarehouses(cloud string, nodes clusterSpec) i if x.hardware != hardware { continue } - if r.buildVersion.AtLeast(x.v) && (v == nil || r.buildVersion.AtLeast(v)) { + if buildVersion.AtLeast(x.v) && (v == nil || buildVersion.AtLeast(v)) { v = x.v warehouses = x.warehouses } @@ -223,7 +229,7 @@ func (r *registry) maxSupportedTPCCWarehouses(cloud string, nodes clusterSpec) i return warehouses } -func registerTPCC(r *registry) { +func registerTPCC(r *testRegistry) { headroomSpec := makeClusterSpec(4, cpu(16)) r.Add(testSpec{ // w=headroom runs tpcc for a semi-extended period with some amount of @@ -236,7 +242,7 @@ func registerTPCC(r *registry) { Tags: []string{`default`, `release_qualification`}, Cluster: headroomSpec, Run: func(ctx context.Context, t *test, c *cluster) { - maxWarehouses := r.maxSupportedTPCCWarehouses(cloud, t.spec.Cluster) + maxWarehouses := maxSupportedTPCCWarehouses(r.buildVersion, cloud, t.spec.Cluster) headroomWarehouses := int(float64(maxWarehouses) * 0.7) t.l.Printf("computed headroom warehouses of %d\n", headroomWarehouses) runTPCC(ctx, t, c, tpccOptions{ @@ -259,9 +265,9 @@ func registerTPCC(r *registry) { Tags: []string{`default`}, Cluster: mixedHeadroomSpec, Run: func(ctx context.Context, t *test, c *cluster) { - maxWarehouses := r.maxSupportedTPCCWarehouses(cloud, t.spec.Cluster) + maxWarehouses := maxSupportedTPCCWarehouses(r.buildVersion, cloud, t.spec.Cluster) headroomWarehouses := int(float64(maxWarehouses) * 0.7) - oldV, err := r.PredecessorVersion() + oldV, err := PredecessorVersion(r.buildVersion) if err != nil { t.Fatal(err) } @@ -320,7 +326,7 @@ func registerTPCC(r *registry) { Period: 45 * time.Second, DownTime: 10 * time.Second, }, - Target: func() nodeListOption { return c.Node(1 + rand.Intn(c.nodes-1)) }, + Target: func() nodeListOption { return c.Node(1 + rand.Intn(c.spec.NodeCount-1)) }, Stopper: time.After(duration), DrainAndQuit: false, } @@ -477,7 +483,7 @@ type tpccBenchSpec struct { // change (i.e. CockroachDB gets faster!). EstimatedMax int - // Tags to pass to registry.Add. + // Tags to pass to testRegistry.Add. Tags []string } @@ -504,7 +510,7 @@ func (s tpccBenchSpec) startOpts() []option { return opts } -func registerTPCCBenchSpec(r *registry, b tpccBenchSpec) { +func registerTPCCBenchSpec(r *testRegistry, b tpccBenchSpec) { nameParts := []string{ "tpccbench", fmt.Sprintf("nodes=%d", b.Nodes), @@ -683,7 +689,9 @@ func runTPCCBench(ctx context.Context, t *test, c *cluster, b tpccBenchSpec) { t.Fatal("distributed chaos benchmarking not supported") } t.Status("installing haproxy") - c.Install(ctx, loadNodes, "haproxy") + if err := c.Install(ctx, t.l, loadNodes, "haproxy"); err != nil { + t.Fatal(err) + } c.Run(ctx, loadNodes, "./cockroach gen haproxy --insecure --url {pgurl:1}") c.Run(ctx, loadNodes, "haproxy -f haproxy.cfg -D") } @@ -785,7 +793,11 @@ func runTPCCBench(ctx context.Context, t *test, c *cluster, b tpccBenchSpec) { return errors.Wrapf(err, "error running tpcc load generator") } roachtestHistogramsPath := filepath.Join(resultsDir, fmt.Sprintf("%d.%d-stats.json", warehouses, groupIdx)) - c.Get(ctx, histogramsPath, roachtestHistogramsPath, group.loadNodes) + if err := c.Get( + ctx, t.l, histogramsPath, roachtestHistogramsPath, group.loadNodes, + ); err != nil { + t.Fatal(err) + } snapshots, err := histogram.DecodeSnapshots(roachtestHistogramsPath) if err != nil { return errors.Wrapf(err, "failed to decode histogram snapshots") @@ -830,7 +842,7 @@ func runTPCCBench(ctx context.Context, t *test, c *cluster, b tpccBenchSpec) { m.Wait() } -func registerTPCCBench(r *registry) { +func registerTPCCBench(r *testRegistry) { specs := []tpccBenchSpec{ { Nodes: 3, diff --git a/pkg/cmd/roachtest/tpcc_test.go b/pkg/cmd/roachtest/tpcc_test.go index 522f0c07023c..8c1da756a622 100644 --- a/pkg/cmd/roachtest/tpcc_test.go +++ b/pkg/cmd/roachtest/tpcc_test.go @@ -38,14 +38,14 @@ func TestTPCCSupportedWarehouses(t *testing.T) { } for _, test := range tests { t.Run("", func(t *testing.T) { - r := ®istry{buildVersion: test.buildVersion} + r := &testRunner{buildVersion: *test.buildVersion} if test.expected == expectPanic { require.Panics(t, func() { - w := r.maxSupportedTPCCWarehouses(test.cloud, test.spec) + w := maxSupportedTPCCWarehouses(r.buildVersion, test.cloud, test.spec) t.Errorf("%s %s got unexpected result %d", test.cloud, &test.spec, w) }) } else { - require.Equal(t, test.expected, r.maxSupportedTPCCWarehouses(test.cloud, test.spec)) + require.Equal(t, test.expected, maxSupportedTPCCWarehouses(r.buildVersion, test.cloud, test.spec)) } }) } diff --git a/pkg/cmd/roachtest/tpchbench.go b/pkg/cmd/roachtest/tpchbench.go index 3f248def8ead..aa843505fecc 100644 --- a/pkg/cmd/roachtest/tpchbench.go +++ b/pkg/cmd/roachtest/tpchbench.go @@ -68,8 +68,8 @@ type tpchBenchSpec struct { // This benchmark runs with a single load generator node running a single // worker. func runTPCHBench(ctx context.Context, t *test, c *cluster, b tpchBenchSpec) { - roachNodes := c.Range(1, c.nodes-1) - loadNode := c.Node(c.nodes) + roachNodes := c.Range(1, c.spec.NodeCount-1) + loadNode := c.Node(c.spec.NodeCount) t.Status("copying binaries") c.Put(ctx, cockroach, "./cockroach", roachNodes) @@ -204,7 +204,7 @@ func loadTPCHBench( // If the scale factor was smaller than the required scale factor, wipe the // cluster and restore. - m.ExpectDeaths(int32(c.nodes)) + m.ExpectDeaths(int32(c.spec.NodeCount)) c.Wipe(ctx, roachNodes) c.Start(ctx, t, roachNodes) m.ResetDeaths() @@ -220,7 +220,7 @@ func loadTPCHBench( return err } -func registerTPCHBenchSpec(r *registry, b tpchBenchSpec) { +func registerTPCHBenchSpec(r *testRegistry, b tpchBenchSpec) { nameParts := []string{ "tpchbench", b.benchType.String(), @@ -246,7 +246,7 @@ func registerTPCHBenchSpec(r *registry, b tpchBenchSpec) { }) } -func registerTPCHBench(r *registry) { +func registerTPCHBench(r *testRegistry) { specs := []tpchBenchSpec{ { Nodes: 3, diff --git a/pkg/cmd/roachtest/typeorm.go b/pkg/cmd/roachtest/typeorm.go index d9787946dcb2..015f360abe3b 100644 --- a/pkg/cmd/roachtest/typeorm.go +++ b/pkg/cmd/roachtest/typeorm.go @@ -19,7 +19,7 @@ import ( var typeORMReleaseTagRegex = regexp.MustCompile(`^(?P\d+)\.(?P\d+)\.(?P\d+)$`) // This test runs TypeORM's full test suite against a single cockroach node. -func registerTypeORM(r *registry) { +func registerTypeORM(r *testRegistry) { runTypeORM := func( ctx context.Context, t *test, @@ -87,6 +87,7 @@ func registerTypeORM(r *registry) { if err := repeatGitCloneE( ctx, + t.l, c, "https://github.com/typeorm/typeorm.git", "/mnt/data1/typeorm", diff --git a/pkg/cmd/roachtest/upgrade.go b/pkg/cmd/roachtest/upgrade.go index 1c8c4f04863e..d175ddef225b 100644 --- a/pkg/cmd/roachtest/upgrade.go +++ b/pkg/cmd/roachtest/upgrade.go @@ -26,9 +26,9 @@ import ( "github.com/pkg/errors" ) -func registerUpgrade(r *registry) { +func registerUpgrade(r *testRegistry) { runUpgrade := func(ctx context.Context, t *test, c *cluster, oldVersion string) { - nodes := c.nodes + nodes := c.spec.NodeCount goos := ifLocal(runtime.GOOS, "linux") b, err := binfetcher.Download(ctx, binfetcher.Options{ @@ -266,7 +266,7 @@ func registerUpgrade(r *registry) { MinVersion: "v2.1.0", Cluster: makeClusterSpec(5), Run: func(ctx context.Context, t *test, c *cluster) { - pred, err := r.PredecessorVersion() + pred, err := PredecessorVersion(r.buildVersion) if err != nil { t.Fatal(err) } @@ -419,7 +419,7 @@ func runVersionUpgrade(ctx context.Context, t *test, c *cluster) { t.l.Printf("%s: using workaround for upgrade\n", newVersion) } - for i := 1; i < c.nodes; i++ { + for i := 1; i < c.spec.NodeCount; i++ { err := retry.ForDuration(30*time.Second, func() error { db := c.Conn(ctx, i) defer db.Close() diff --git a/pkg/cmd/roachtest/version.go b/pkg/cmd/roachtest/version.go index f2de66dfa1a2..687e1071a2db 100644 --- a/pkg/cmd/roachtest/version.go +++ b/pkg/cmd/roachtest/version.go @@ -23,9 +23,9 @@ import ( "github.com/pkg/errors" ) -func registerVersion(r *registry) { +func registerVersion(r *testRegistry) { runVersion := func(ctx context.Context, t *test, c *cluster, version string) { - nodes := c.nodes - 1 + nodes := c.spec.NodeCount - 1 goos := ifLocal(runtime.GOOS, "linux") b, err := binfetcher.Download(ctx, binfetcher.Options{ @@ -239,7 +239,7 @@ func registerVersion(r *registry) { MinVersion: "v2.1.0", Cluster: makeClusterSpec(n + 1), Run: func(ctx context.Context, t *test, c *cluster) { - pred, err := r.PredecessorVersion() + pred, err := PredecessorVersion(r.buildVersion) if err != nil { t.Fatal(err) } diff --git a/pkg/cmd/roachtest/work_pool.go b/pkg/cmd/roachtest/work_pool.go new file mode 100644 index 000000000000..8131ecec91c7 --- /dev/null +++ b/pkg/cmd/roachtest/work_pool.go @@ -0,0 +1,290 @@ +// Copyright 2018 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 main + +import ( + "context" + "fmt" + "math" + + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/quotapool" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" +) + +// workPool keeps track of what tests still need to run and facilitates +// selecting the next test to run. +type workPool struct { + // count is the total number of times each test has to run. It is constant. + // Not to be confused with the count inside mu.tests, which tracks remaining + // runs. + count int + mu struct { + syncutil.Mutex + // tests with remaining run count. + tests []testWithCount + } +} + +func newWorkPool(tests []testSpec, count int) *workPool { + p := &workPool{count: count} + for _, spec := range tests { + p.mu.tests = append(p.mu.tests, testWithCount{spec: spec, count: count}) + } + return p +} + +// testToRunRes represents the return value of getTestToRun. It provides +// information about what test to run (if any) and what cluster to use for it. +type testToRunRes struct { + // noWork is set if the work pool was empty and thus no test was selected. No + // other fields are set. + noWork bool + // spec is the selected test. + spec testSpec + // runNum is run number. 1 if --count was not used. + runNum int + + // canReuseCluster is true if the selected test can reuse the cluster passed + // to testToRun(). Will be false if noWork is set. + canReuseCluster bool + // alloc is set if canReuseCluster is false (and noWork is not set). It + // represents the resources to use for creating a new cluster (matching spec). + // The alloc needs to be transferred to the cluster that is created, or + // otherwise Release()d. + alloc *quotapool.IntAlloc +} + +func (p *workPool) workRemaining() []testWithCount { + p.mu.Lock() + defer p.mu.Unlock() + res := make([]testWithCount, len(p.mu.tests)) + copy(res, p.mu.tests) + return res +} + +// getTestToRun selects a test. It optionally takes a cluster and will try to +// select a test that can reuse that cluster. If it succeeds, then +// testToRunRes.canReuseCluster will be set. Otherwise, the cluster is destroyed +// so its resources are released, and the result will contain a quota alloc to +// be used by the caller for creating a new cluster. +// +// If a new cluster needs to be created, the call blocks until enough resources +// are taken out of qp. +// +// If there are no more tests to run, c will be destroyed and the result will +// have noWork set. +func (p *workPool) getTestToRun( + ctx context.Context, + c *cluster, + qp *quotapool.IntPool, + cr *clusterRegistry, + onDestroy func(), + l *logger, +) (testToRunRes, error) { + // If we've been given a cluster, see if we can reuse it. + if c != nil { + ttr := p.selectTestForCluster(ctx, c.spec, cr) + if ttr.noWork { + // We failed to find a test that can take advantage of this cluster. So + // we're going to release is, which will deallocate its resources, and + // then we'll look for a test below. + l.PrintfCtx(ctx, + "No tests that can reuse cluster %s found (or there are no further tests to run). "+ + "Destroying.", c) + c.Destroy(ctx, closeLogger, l) + onDestroy() + } else { + return ttr, nil + } + } + + return p.selectTest(ctx, qp) +} + +// selectTestForCluster selects a test to run on a cluster with a given spec. +// +// Among tests that match the spec, we do the following: +// - If the cluster is already tagged, we only look at tests with the same tag. +// - Otherwise, we'll choose in the following order of preference: +// 1) tests that leave the cluster usable by anybody afterwards +// 2) tests that leave the cluster usable by some other tests +// 2.1) within this OnlyTagged category, we'll prefer the tag with the +// fewest existing clusters. +// 3) tests that leave the cluster unusable by anybody +// +// Within each of the categories, we'll give preference to tests with fewer +// runs. +// +// cr is used for its information about how many clusters with a given tag currently exist. +func (p *workPool) selectTestForCluster( + ctx context.Context, spec clusterSpec, cr *clusterRegistry, +) testToRunRes { + p.mu.Lock() + defer p.mu.Unlock() + testsWithCounts := p.findCompatibleTestsLocked(spec) + + if len(testsWithCounts) == 0 { + return testToRunRes{noWork: true} + } + + tag := "" + if p, ok := spec.ReusePolicy.(reusePolicyTagged); ok { + tag = p.tag + } + // Find the best test to run. + candidateScore := 0 + var candidate testWithCount + for _, tc := range testsWithCounts { + score := scoreTestAgainstCluster(tc, tag, cr) + if score > candidateScore { + candidateScore = score + candidate = tc + } + } + + p.decTestLocked(ctx, candidate.spec.Name) + runNum := p.count - candidate.count + 1 + return testToRunRes{ + spec: candidate.spec, + runNum: runNum, + canReuseCluster: true, + } +} + +// selectTest selects a test to run based on the available resources. If there are +// no resources available to run any test, it blocks until enough resources become available. +// +// If multiple tests are eligible to run, one with the most runs left is chosen. +// TODO(andrei): We could be smarter in guessing what kind of cluster is best to +// allocate. +func (p *workPool) selectTest(ctx context.Context, qp *quotapool.IntPool) (testToRunRes, error) { + var ttr testToRunRes + alloc, err := qp.AcquireFunc(ctx, func(ctx context.Context, pi quotapool.PoolInfo) (uint64, error) { + p.mu.Lock() + defer p.mu.Unlock() + + if len(p.mu.tests) == 0 { + ttr = testToRunRes{ + noWork: true, + } + return 0, nil + } + + candidateIdx := -1 + candidateCount := 0 + smallestTest := math.MaxInt64 + for i, t := range p.mu.tests { + cpu := t.spec.Cluster.NodeCount * t.spec.Cluster.CPUs + if cpu < smallestTest { + smallestTest = cpu + } + if uint64(cpu) > pi.Available { + continue + } + if t.count > candidateCount { + candidateIdx = i + candidateCount = t.count + } + } + + if candidateIdx == -1 { + if uint64(smallestTest) > pi.Capacity { + return 0, fmt.Errorf("not enough CPU quota to run any of the remaining tests") + } + + return 0, quotapool.ErrNotEnoughQuota + } + + tc := p.mu.tests[candidateIdx] + runNum := p.count - tc.count + 1 + p.decTestLocked(ctx, tc.spec.Name) + ttr = testToRunRes{ + spec: tc.spec, + runNum: runNum, + canReuseCluster: false, + } + cpu := tc.spec.Cluster.NodeCount * tc.spec.Cluster.CPUs + return uint64(cpu), nil + }) + if err != nil { + return testToRunRes{}, err + } + ttr.alloc = alloc + return ttr, nil +} + +// scoreTestAgainstCluster scores the suitability of running a test against a +// cluster currently tagged with tag (empty if cluster is not tagged). +// +// cr is used for its information about how many clusters with a given tag +// currently exist. +func scoreTestAgainstCluster(tc testWithCount, tag string, cr *clusterRegistry) int { + t := tc.spec + testPolicy := t.Cluster.ReusePolicy + if tag != "" && testPolicy != (reusePolicyTagged{tag: tag}) { + log.Fatalf(context.TODO(), + "incompatible test and cluster. Cluster tag: %s. Test policy: %+v", + tag, t.Cluster.ReusePolicy) + } + score := 0 + if _, ok := testPolicy.(reusePolicyAny); ok { + score = 1000000 + } else if _, ok := testPolicy.(reusePolicyTagged); ok { + score = 500000 + if tag == "" { + // We have an untagged cluster and a tagged test. Within this category of + // tests, we prefer the tags with the fewest existing clusters. + score -= 1000 * cr.countForTag(tag) + } + } else { // NoReuse policy + score = 0 + } + + // We prefer tests that have run fewer times (so, that have more runs left). + score += tc.count + + return score +} + +// findCompatibleTestsLocked returns a list of tests compatible with a cluster spec. +func (p *workPool) findCompatibleTestsLocked(clusterSpec clusterSpec) []testWithCount { + if _, ok := clusterSpec.ReusePolicy.(reusePolicyNone); ok { + panic("can't search for tests compatible with a ReuseNone policy") + } + var tests []testWithCount + for _, tc := range p.mu.tests { + if clustersCompatible(clusterSpec, tc.spec.Cluster) { + tests = append(tests, tc) + } + } + return tests +} + +// decTestLocked decrements a test's remaining count and removes it +// from the workPool if it was exhausted. +func (p *workPool) decTestLocked(ctx context.Context, name string) { + idx := -1 + for idx = range p.mu.tests { + if p.mu.tests[idx].spec.Name == name { + break + } + } + if idx == -1 { + log.Fatalf(ctx, "failed to find test: %s", name) + } + tc := &p.mu.tests[idx] + tc.count-- + if tc.count == 0 { + // We've selected the last run for a test. Take that test out of the pool. + p.mu.tests = append(p.mu.tests[:idx], p.mu.tests[idx+1:]...) + } +} diff --git a/pkg/cmd/roachtest/ycsb.go b/pkg/cmd/roachtest/ycsb.go index 2f9d42a1a2a9..1657d02816d4 100644 --- a/pkg/cmd/roachtest/ycsb.go +++ b/pkg/cmd/roachtest/ycsb.go @@ -15,9 +15,9 @@ import ( "fmt" ) -func registerYCSB(r *registry) { +func registerYCSB(r *testRegistry) { runYCSB := func(ctx context.Context, t *test, c *cluster, wl string, cpus int) { - nodes := c.nodes - 1 + nodes := c.spec.NodeCount - 1 c.Put(ctx, cockroach, "./cockroach", c.Range(1, nodes)) c.Put(ctx, workload, "./workload", c.Node(nodes+1)) diff --git a/pkg/util/quotapool/example_test.go b/pkg/util/quotapool/example_test.go index 0b35fe00c1be..e759560d3c94 100644 --- a/pkg/util/quotapool/example_test.go +++ b/pkg/util/quotapool/example_test.go @@ -28,7 +28,7 @@ func ExampleIntPool_AcquireFunc() { qp := NewIntPool("work units", quota) type job struct { name string - cost int64 + cost uint64 } jobs := []*job{ {name: "foo", cost: 3}, @@ -53,24 +53,24 @@ func ExampleIntPool_AcquireFunc() { ctx context.Context, qp *IntPool, ) (j *job, alloc *IntAlloc, err error) { alloc, err = qp.AcquireFunc(ctx, func( - ctx context.Context, v int64, - ) (fulfilled bool, took int64) { + ctx context.Context, pi PoolInfo, + ) (took uint64, err error) { sortJobs() // There are no more jobs, take 0 and return. if jobs[0] == nil { - return true, 0 + return 0, nil } // Find the largest jobs which can be run. for i := range jobs { if jobs[i] == nil { break } - if jobs[i].cost <= v { + if jobs[i].cost <= pi.Available { j, jobs[i] = jobs[i], nil - return true, j.cost + return j.cost, nil } } - return false, 0 + return 0, ErrNotEnoughQuota }) return j, alloc, err } diff --git a/pkg/util/quotapool/intpool.go b/pkg/util/quotapool/intpool.go index beb3bc0c6beb..db548a756dcf 100644 --- a/pkg/util/quotapool/intpool.go +++ b/pkg/util/quotapool/intpool.go @@ -12,8 +12,10 @@ package quotapool import ( "context" + "fmt" "strconv" "sync" + "sync/atomic" "github.com/pkg/errors" ) @@ -24,8 +26,13 @@ import ( // allows the client to provide a function which will be used to determine // whether a quantity of quota is sufficient when it becomes available. type IntPool struct { - qp *QuotaPool - max int64 + qp *QuotaPool + + // capacity maintains how much total quota there is (not necessarily available). + // Accessed atomically! + // The capacity is originally set when the IntPool is constructed, and then it + // can be decreased by IntAlloc.Freeze(). + capacity uint64 intAllocSyncPool sync.Pool intRequestSyncPool sync.Pool @@ -34,7 +41,7 @@ type IntPool struct { // IntAlloc is an allocated quantity which should be released. type IntAlloc struct { - alloc int64 + alloc uint64 p *IntPool } @@ -44,7 +51,7 @@ func (ia *IntAlloc) Release() { } // Acquired returns the quantity that this alloc has acquired. -func (ia *IntAlloc) Acquired() int64 { +func (ia *IntAlloc) Acquired() uint64 { return ia.alloc } @@ -55,16 +62,25 @@ func (ia *IntAlloc) Merge(other *IntAlloc) { if ia.p != other.p { panic("cannot merge IntAllocs from two different pools") } - ia.alloc = min(ia.p.max, ia.alloc+other.alloc) + ia.alloc = min(ia.p.Capacity(), ia.alloc+other.alloc) ia.p.putIntAlloc(other) } +// Freeze informs the quota pool that this allocation will never be Release()ed. +// Releasing it later is illegal. +// +// AcquireFunc() requests will be woken up with an updated Capacity, and Alloc() +// requests will be trimmed accordingly. +func (ia *IntAlloc) Freeze() { + ia.p.decCapacity(ia.alloc) +} + // String formats an IntAlloc as a string. func (ia *IntAlloc) String() string { if ia == nil { return strconv.Itoa(0) } - return strconv.FormatInt(ia.alloc, 10) + return strconv.FormatUint(ia.alloc, 10) } // intAlloc is used to make IntAlloc implement Resource without muddling its @@ -76,10 +92,12 @@ func (ia *intAlloc) Merge(other Resource) { (*IntAlloc)(ia).Merge((*IntAlloc)(other.(*intAlloc))) } -// NewIntPool creates a new named IntPool with a maximum quota value. -func NewIntPool(name string, max int64, options ...Option) *IntPool { +// NewIntPool creates a new named IntPool. +// +// capacity is the amount of quota initially available. +func NewIntPool(name string, capacity uint64, options ...Option) *IntPool { p := IntPool{ - max: max, + capacity: capacity, intAllocSyncPool: sync.Pool{ New: func() interface{} { return new(IntAlloc) }, }, @@ -90,24 +108,26 @@ func NewIntPool(name string, max int64, options ...Option) *IntPool { New: func() interface{} { return new(intFuncRequest) }, }, } - p.qp = New(name, (*intAlloc)(p.newIntAlloc(max)), options...) + p.qp = New(name, (*intAlloc)(p.newIntAlloc(capacity)), options...) return &p } -// Acquire acquires the specified amount of quota from the pool. On success, -// nil is returned and the caller must call add(v) or otherwise arrange for the -// quota to be returned to the pool. If 'v' is greater than the total capacity -// of the pool, we instead try to acquire quota equal to the maximum capacity. +// Acquire acquires the specified amount of quota from the pool. On success, a +// non-nil alloc is returned and Release() must be called on it to return the +// quota to the pool. +// +// If 'v' is greater than the total capacity of the pool, we instead try to +// acquire quota equal to the maximum capacity. If the maximum capacity is +// decreased while this request is ongoing, the request is again truncated to +// the maximum capacity. // // Acquisitions of 0 return immediately with no error, even if the IntPool is // closed. // // Safe for concurrent use. -func (p *IntPool) Acquire(ctx context.Context, v int64) (*IntAlloc, error) { +func (p *IntPool) Acquire(ctx context.Context, v uint64) (*IntAlloc, error) { // Special case acquisitions of size 0. - if v < 0 { - panic("cannot acquire negative quota") - } else if v == 0 { + if v == 0 { return p.newIntAlloc(v), nil } r := p.newIntRequest(v) @@ -120,7 +140,35 @@ func (p *IntPool) Acquire(ctx context.Context, v int64) (*IntAlloc, error) { // IntRequestFunc is used to request a quantity of quota determined when quota is // available rather than before requesting. -type IntRequestFunc func(ctx context.Context, v int64) (fulfilled bool, took int64) +// +// If the request is satisfied, the function returns the amount of quota +// consumed and no error. If the request is not satisfied because there's no +// enough quota currently available, ErrNotEnoughQuota is returned to cause the +// function to be called again where more quota becomes available. took has to +// be 0 (i.e. it is not allowed for the request to save some quota for later +// use). If any other error is returned, took again has to be 0. The function +// will not be called any more and the error will be returned from +// IntPool.AcquireFunc(). +type IntRequestFunc func(ctx context.Context, p PoolInfo) (took uint64, err error) + +// ErrNotEnoughQuota is returned by IntRequestFuncs when they want to be called +// again once there's new resources. +var ErrNotEnoughQuota = fmt.Errorf("not enough quota available") + +// PoolInfo represents the information that the IntRequestFunc gets about the current quota pool conditions. +type PoolInfo struct { + // Available is the amount of quota available to be consumed. This is the + // maximum value that the `took` return value from IntRequestFunc can be set + // to. + // Note that Available() can be 0. This happens when the IntRequestFunc() is + // called as a result of the pool's capacity decreasing. + Available uint64 + + // Capacity returns the maximum capacity available in the pool. This can + // decrease over time. It can be used to determine that the resources required + // by a request will never be available. + Capacity uint64 +} // AcquireFunc acquires a quantity of quota determined by a function which is // called with a quantity of available quota. @@ -131,6 +179,12 @@ func (p *IntPool) AcquireFunc(ctx context.Context, f IntRequestFunc) (*IntAlloc, if err != nil { return nil, err } + if r.err != nil { + if r.took != 0 { + panic(fmt.Sprintf("both took set (%d) and err (%s)", r.took, r.err)) + } + return nil, r.err + } return p.newIntAlloc(r.took), nil } @@ -139,7 +193,7 @@ func (p *IntPool) AcquireFunc(ctx context.Context, f IntRequestFunc) (*IntAlloc, // the return value can be up to 'v' less than actual available quota where 'v' // is the value the acquisition goroutine first in line is attempting to // acquire. -func (p *IntPool) ApproximateQuota() (q int64) { +func (p *IntPool) ApproximateQuota() (q uint64) { p.qp.ApproximateQuota(func(r Resource) { if ia, ok := r.(*intAlloc); ok { q = ia.alloc @@ -156,7 +210,7 @@ func (p *IntPool) Close(reason string) { p.qp.Close(reason) } -func (p *IntPool) newIntAlloc(v int64) *IntAlloc { +func (p *IntPool) newIntAlloc(v uint64) *IntAlloc { ia := p.intAllocSyncPool.Get().(*IntAlloc) *ia = IntAlloc{p: p, alloc: v} return ia @@ -168,9 +222,9 @@ func (p *IntPool) putIntAlloc(ia *IntAlloc) { // newIntRequest allocates an intRequest from the sync.Pool. // It should be returned with putIntRequest. -func (p *IntPool) newIntRequest(v int64) *intRequest { +func (p *IntPool) newIntRequest(v uint64) *intRequest { r := p.intRequestSyncPool.Get().(*intRequest) - r.want = min(v, p.max) + r.want = v return r } @@ -183,21 +237,39 @@ func (p *IntPool) putIntRequest(r *intRequest) { func (p *IntPool) newIntFuncRequest(f IntRequestFunc) *intFuncRequest { r := p.intFuncRequestSyncPool.Get().(*intFuncRequest) r.f = f + r.p = p return r } func (p *IntPool) putIntFuncRequest(r *intFuncRequest) { r.f = nil + r.took = 0 p.intFuncRequestSyncPool.Put(r) } +// Capacity returns the amount of quota managed by this pool. +func (p *IntPool) Capacity() uint64 { + return atomic.LoadUint64(&p.capacity) +} + +// decCapacity decrements the capacity by c. +func (p *IntPool) decCapacity(c uint64) { + // This is how you decrement from a uint64. + atomic.AddUint64(&p.capacity, ^uint64(c-1)) + // Wake up the request at the front of the queue. The decrement above may race + // with an ongoing request (which is why it's an atomic access), but in any + // case that request is evaluated again. + p.qp.Add(&intAlloc{alloc: 0, p: p}) +} + // intRequest is used to acquire a quantity from the quota known ahead of time. type intRequest struct { - want int64 + want uint64 } func (r *intRequest) Acquire(ctx context.Context, v Resource) (fulfilled bool, extra Resource) { ia := v.(*intAlloc) + r.want = min(r.want, ia.p.Capacity()) if ia.alloc < r.want { return false, nil } @@ -208,15 +280,30 @@ func (r *intRequest) Acquire(ctx context.Context, v Resource) (fulfilled bool, e // intFuncRequest is used to acquire a quantity from the pool which is not // known ahead of time. type intFuncRequest struct { + p *IntPool f IntRequestFunc - took int64 + took uint64 + // err saves the error returned by r.f, if other than ErrNotEnoughQuota. + err error } func (r *intFuncRequest) Acquire(ctx context.Context, v Resource) (fulfilled bool, extra Resource) { ia := v.(*intAlloc) - ok, took := r.f(ctx, ia.alloc) - if !ok { - return false, nil + pi := PoolInfo{ + Available: ia.alloc, + Capacity: ia.p.Capacity(), + } + took, err := r.f(ctx, pi) + if err != nil { + if took != 0 { + panic(fmt.Sprintf("IntRequestFunc returned both took: %d and err: %s", took, err)) + } + if err == ErrNotEnoughQuota { + return false, nil + } + r.err = err + // Take the request out of the queue and put all the quota back. + return true, ia } if took > ia.alloc { panic(errors.Errorf("took %d quota > %d allocated", took, ia.alloc)) @@ -226,7 +313,7 @@ func (r *intFuncRequest) Acquire(ctx context.Context, v Resource) (fulfilled boo return true, ia } -func min(a, b int64) (v int64) { +func min(a, b uint64) (v uint64) { if a < b { return a } diff --git a/pkg/util/quotapool/intpool_test.go b/pkg/util/quotapool/intpool_test.go index 3f62ce8a1a88..adbd6d774932 100644 --- a/pkg/util/quotapool/intpool_test.go +++ b/pkg/util/quotapool/intpool_test.go @@ -17,6 +17,7 @@ import ( "testing" "time" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/quotapool" "golang.org/x/sync/errgroup" @@ -28,7 +29,7 @@ import ( func TestQuotaPoolBasic(t *testing.T) { defer leaktest.AfterTest(t)() - quotas := []int64{1, 10, 100, 1000} + quotas := []uint64{1, 10, 100, 1000} goroutineCounts := []int{1, 10, 100} for _, quota := range quotas { @@ -264,7 +265,7 @@ func TestQuotaPoolMaxQuota(t *testing.T) { t.Fatal(err) } if got := alloc.Acquired(); got != quota { - t.Fatalf("expected to acquire the max quota %d, instead got %d", quota, got) + t.Fatalf("expected to acquire the capacity quota %d, instead got %d", quota, got) } alloc.Release() if q := qp.ApproximateQuota(); q != quota { @@ -345,6 +346,45 @@ func TestSlowAcquisition(t *testing.T) { } } +// Test that AcquireFunc() is called after IntAlloc.Freeze() is called - so that an ongoing acquisition gets +// the chance to observe that there's no capacity for its request. +func TestQuotaPoolCapacityDecrease(t *testing.T) { + qp := quotapool.NewIntPool("test", 100) + ctx := context.Background() + + alloc50, err := qp.Acquire(ctx, 50) + if err != nil { + t.Fatal(err) + } + + first := true + firstCh := make(chan struct{}) + doneCh := make(chan struct{}) + go func() { + _, err = qp.AcquireFunc(ctx, + func(_ context.Context, pi quotapool.PoolInfo) (took uint64, err error) { + if first { + first = false + close(firstCh) + } + if pi.Capacity < 100 { + return 0, fmt.Errorf("hopeless") + } + return 0, quotapool.ErrNotEnoughQuota + }) + close(doneCh) + }() + + // Wait for the callback to be called the first time. It should return ErrNotEnoughQuota. + <-firstCh + // Now leak the quota. This should call the callback to be called again. + alloc50.Freeze() + <-doneCh + if !testutils.IsError(err, "hopeless") { + t.Fatalf("expected hopeless error, got: %v", err) + } +} + // BenchmarkIntQuotaPool benchmarks the common case where we have sufficient // quota available in the pool and we repeatedly acquire and release quota. func BenchmarkIntQuotaPool(b *testing.B) { @@ -366,9 +406,9 @@ func BenchmarkIntQuotaPool(b *testing.B) { func BenchmarkConcurrentIntQuotaPool(b *testing.B) { // test returns the arguments to b.Run for a given number of workers and // quantity of quota. - test := func(workers, quota int) (string, func(b *testing.B)) { + test := func(workers int, quota uint64) (string, func(b *testing.B)) { return fmt.Sprintf("workers=%d,quota=%d", workers, quota), func(b *testing.B) { - qp := quotapool.NewIntPool("test", int64(quota), quotapool.LogSlowAcquisition) + qp := quotapool.NewIntPool("test", quota, quotapool.LogSlowAcquisition) g, ctx := errgroup.WithContext(context.Background()) runWorker := func(workerNum int) { g.Go(func() error { @@ -393,7 +433,8 @@ func BenchmarkConcurrentIntQuotaPool(b *testing.B) { } } for _, c := range []struct { - workers, quota int + workers int + quota uint64 }{ {1, 1}, {2, 2}, @@ -425,11 +466,11 @@ func BenchmarkIntQuotaPoolFunc(b *testing.B) { } // intRequest is a wrapper to create a IntRequestFunc from an int64. -type intRequest int64 +type intRequest uint64 -func (ir intRequest) acquire(_ context.Context, v int64) (fulfilled bool, took int64) { - if int64(ir) < v { - return false, 0 +func (ir intRequest) acquire(_ context.Context, pi quotapool.PoolInfo) (took uint64, err error) { + if uint64(ir) < pi.Available { + return 0, quotapool.ErrNotEnoughQuota } - return true, int64(ir) + return uint64(ir), nil }
TestRuns
%s%d