diff --git a/pkg/cmd/roachtest/cluster.go b/pkg/cmd/roachtest/cluster.go index 4ac37b1a01ff..17019820c89e 100644 --- a/pkg/cmd/roachtest/cluster.go +++ b/pkg/cmd/roachtest/cluster.go @@ -622,10 +622,11 @@ type nodeSelector interface { // It is safe for concurrent use by multiple goroutines. type clusterImpl struct { - name string - tag string - spec spec.ClusterSpec - t test.Test + name string + tag string + cloud string + spec spec.ClusterSpec + t test.Test // r is the registry tracking this cluster. Destroying the cluster will // unregister it. r *clusterRegistry @@ -864,7 +865,9 @@ func (f *clusterFactory) newCluster( providerOptsContainer := vm.CreateProviderOptionsContainer() + cloud := roachtestflags.Cloud params := spec.RoachprodClusterConfig{ + Cloud: cloud, UseIOBarrierOnLocalSSD: cfg.useIOBarrier, PreferredArch: cfg.arch, } @@ -878,8 +881,8 @@ func (f *clusterFactory) newCluster( if err != nil { return nil, nil, err } - if cfg.spec.Cloud != spec.Local { - providerOptsContainer.SetProviderOpts(cfg.spec.Cloud, providerOpts) + if cloud != spec.Local { + providerOptsContainer.SetProviderOpts(cloud, providerOpts) } if cfg.spec.UbuntuVersion.IsOverridden() { @@ -902,6 +905,7 @@ func (f *clusterFactory) newCluster( // loop assumes maxAttempts is atleast (1). for i := 1; ; i++ { c := &clusterImpl{ + cloud: cloud, // NB: this intentionally avoids re-using the name across iterations in // the loop. See: // @@ -2525,7 +2529,7 @@ func (c *clusterImpl) MakeNodes(opts ...option.Option) string { } func (c *clusterImpl) Cloud() string { - return c.spec.Cloud + return c.cloud } func (c *clusterImpl) IsLocal() bool { @@ -2642,7 +2646,7 @@ func archForTest(ctx context.Context, l *logger.Logger, testSpec registry.TestSp return testSpec.Cluster.Arch } - if testSpec.Benchmark && testSpec.Cluster.Cloud != spec.Local { + if testSpec.Benchmark && roachtestflags.Cloud != spec.Local { // TODO(srosenberg): enable after https://github.com/cockroachdb/cockroach/issues/104213 arch := vm.ArchAMD64 l.PrintfCtx(ctx, "Disabling arch randomization for benchmark; arch=%q, %s", arch, testSpec.Name) diff --git a/pkg/cmd/roachtest/cluster_test.go b/pkg/cmd/roachtest/cluster_test.go index eabfde9d1b52..7ac81270cc0a 100644 --- a/pkg/cmd/roachtest/cluster_test.go +++ b/pkg/cmd/roachtest/cluster_test.go @@ -27,7 +27,7 @@ import ( ) func TestClusterNodes(t *testing.T) { - c := &clusterImpl{spec: spec.MakeClusterSpec(spec.GCE, 10)} + c := &clusterImpl{spec: spec.MakeClusterSpec(10)} opts := func(opts ...option.Option) []option.Option { return opts } diff --git a/pkg/cmd/roachtest/github.go b/pkg/cmd/roachtest/github.go index e6caabb1a3ce..6fc91a18feac 100644 --- a/pkg/cmd/roachtest/github.go +++ b/pkg/cmd/roachtest/github.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/cmd/internal/issues" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/registry" + "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestflags" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/spec" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/tests" @@ -204,7 +205,7 @@ func (g *githubIssues) createPostRequest( artifacts := fmt.Sprintf("/%s", testName) clusterParams := map[string]string{ - roachtestPrefix("cloud"): spec.Cluster.Cloud, + roachtestPrefix("cloud"): roachtestflags.Cloud, roachtestPrefix("cpu"): fmt.Sprintf("%d", spec.Cluster.CPUs), roachtestPrefix("ssd"): fmt.Sprintf("%d", spec.Cluster.SSDs), roachtestPrefix("metamorphicBuild"): fmt.Sprintf("%t", metamorphicBuild), @@ -240,7 +241,7 @@ func (g *githubIssues) createPostRequest( Artifacts: artifacts, ExtraLabels: labels, ExtraParams: clusterParams, - HelpCommand: generateHelpCommand(testName, issueClusterName, spec.Cluster.Cloud, start, end), + HelpCommand: generateHelpCommand(testName, issueClusterName, roachtestflags.Cloud, start, end), }, nil } diff --git a/pkg/cmd/roachtest/github_test.go b/pkg/cmd/roachtest/github_test.go index 248f5312dd41..eb5c6823dec2 100644 --- a/pkg/cmd/roachtest/github_test.go +++ b/pkg/cmd/roachtest/github_test.go @@ -83,7 +83,7 @@ func TestShouldPost(t *testing.T) { {false, 1, "token", "master", true, ""}, } - reg := makeTestRegistry(spec.GCE) + reg := makeTestRegistry() for _, c := range testCases { t.Setenv("GITHUB_API_TOKEN", c.envGithubAPIToken) t.Setenv("TC_BUILD_BRANCH", c.envTcBuildBranch) @@ -184,7 +184,7 @@ func TestCreatePostRequest(t *testing.T) { {true, false, false, false, false, "", createFailure(gce.ErrDNSOperation), true, false, true, nil}, } - reg := makeTestRegistry(spec.GCE) + reg := makeTestRegistry() for idx, c := range testCases { t.Run(fmt.Sprintf("%d", idx+1), func(t *testing.T) { clusterSpec := reg.MakeClusterSpec(1, spec.Arch(c.arch)) diff --git a/pkg/cmd/roachtest/main.go b/pkg/cmd/roachtest/main.go index 35141c8308b6..39c59f65df09 100644 --- a/pkg/cmd/roachtest/main.go +++ b/pkg/cmd/roachtest/main.go @@ -98,7 +98,7 @@ Examples: roachtest list --suite weekly --owner kv `, RunE: func(cmd *cobra.Command, args []string) error { - r := makeTestRegistry(roachtestflags.Cloud) + r := makeTestRegistry() tests.RegisterTests(&r) filter, err := makeTestFilter(args) diff --git a/pkg/cmd/roachtest/main_test.go b/pkg/cmd/roachtest/main_test.go index 4a00c2762ee7..e7a62bf6a809 100644 --- a/pkg/cmd/roachtest/main_test.go +++ b/pkg/cmd/roachtest/main_test.go @@ -30,7 +30,7 @@ func init() { } func makeRegistry(names ...string) testRegistryImpl { - r := makeTestRegistry(spec.GCE) + r := makeTestRegistry() dummyRun := func(context.Context, test.Test, cluster.Cluster) {} for _, name := range names { @@ -38,7 +38,7 @@ func makeRegistry(names ...string) testRegistryImpl { Name: name, Owner: OwnerUnitTest, Run: dummyRun, - Cluster: spec.MakeClusterSpec(spec.GCE, 0), + Cluster: spec.MakeClusterSpec(0), CompatibleClouds: registry.AllExceptAWS, Suites: registry.Suites(registry.Nightly), }) diff --git a/pkg/cmd/roachtest/registry/registry_interface.go b/pkg/cmd/roachtest/registry/registry_interface.go index f74bfda5b808..06c59c02d0cf 100644 --- a/pkg/cmd/roachtest/registry/registry_interface.go +++ b/pkg/cmd/roachtest/registry/registry_interface.go @@ -21,5 +21,4 @@ type Registry interface { MakeClusterSpec(nodeCount int, opts ...spec.Option) spec.ClusterSpec Add(TestSpec) PromFactory() promauto.Factory - Cloud() string } diff --git a/pkg/cmd/roachtest/run.go b/pkg/cmd/roachtest/run.go index 7def1899fb8c..b19d44534405 100644 --- a/pkg/cmd/roachtest/run.go +++ b/pkg/cmd/roachtest/run.go @@ -43,7 +43,7 @@ import ( func runTests(register func(registry.Registry), filter *registry.TestFilter) error { rand.Seed(roachtestflags.GlobalSeed) - r := makeTestRegistry(roachtestflags.Cloud) + r := makeTestRegistry() // actual registering of tests // TODO: don't register if we can't run on the specified registry cloud diff --git a/pkg/cmd/roachtest/spec/cluster_spec.go b/pkg/cmd/roachtest/spec/cluster_spec.go index bc7971a507f5..dd0110b583d6 100644 --- a/pkg/cmd/roachtest/spec/cluster_spec.go +++ b/pkg/cmd/roachtest/spec/cluster_spec.go @@ -80,10 +80,6 @@ const ( // ClusterSpec represents a test's description of what its cluster needs to // look like. It becomes part of a clusterConfig when the cluster is created. type ClusterSpec struct { - // TODO(#104029): We should remove the Cloud field; the tests now specify - // their compatible clouds. - Cloud string - Arch vm.CPUArch // CPU architecture; auto-chosen if left empty NodeCount int // CPUs is the number of CPUs per node. @@ -125,8 +121,8 @@ type ClusterSpec struct { } // MakeClusterSpec makes a ClusterSpec. -func MakeClusterSpec(cloud string, nodeCount int, opts ...Option) ClusterSpec { - spec := ClusterSpec{Cloud: cloud, NodeCount: nodeCount} +func MakeClusterSpec(nodeCount int, opts ...Option) ClusterSpec { + spec := ClusterSpec{NodeCount: nodeCount} defaultOpts := []Option{CPU(4), nodeLifetime(12 * time.Hour), ReuseAny()} for _, o := range append(defaultOpts, opts...) { o(&spec) @@ -240,6 +236,8 @@ func getAzureOpts(machineType string, zones []string) vm.ProviderOpts { // does not depend on the test. It is used in conjunction with ClusterSpec to // determine the final configuration. type RoachprodClusterConfig struct { + Cloud string + // UseIOBarrierOnLocalSSD is set if we don't want to mount local SSDs with the // `-o nobarrier` flag. UseIOBarrierOnLocalSSD bool @@ -292,25 +290,26 @@ func (s *ClusterSpec) RoachprodOpts( if s.Lifetime != 0 { createVMOpts.Lifetime = s.Lifetime } - switch s.Cloud { + cloud := params.Cloud + switch cloud { case Local: - createVMOpts.VMProviders = []string{s.Cloud} + createVMOpts.VMProviders = []string{cloud} // remaining opts are not applicable to local clusters return createVMOpts, nil, nil case AWS, GCE, Azure: - createVMOpts.VMProviders = []string{s.Cloud} + createVMOpts.VMProviders = []string{cloud} default: - return vm.CreateOpts{}, nil, errors.Errorf("unsupported cloud %v", s.Cloud) + return vm.CreateOpts{}, nil, errors.Errorf("unsupported cloud %v", cloud) } - if s.Cloud != GCE && s.Cloud != AWS { + if cloud != GCE && cloud != AWS { if s.VolumeSize != 0 { - return vm.CreateOpts{}, nil, errors.Errorf("specifying volume size is not yet supported on %s", s.Cloud) + return vm.CreateOpts{}, nil, errors.Errorf("specifying volume size is not yet supported on %s", cloud) } } - if s.Cloud != GCE { + if cloud != GCE { if s.SSDs != 0 { - return vm.CreateOpts{}, nil, errors.Errorf("specifying SSD count is not yet supported on %s", s.Cloud) + return vm.CreateOpts{}, nil, errors.Errorf("specifying SSD count is not yet supported on %s", cloud) } } @@ -319,7 +318,7 @@ func (s *ClusterSpec) RoachprodOpts( ssdCount := s.SSDs machineType := params.Defaults.MachineType - switch s.Cloud { + switch cloud { case AWS: if s.AWS.MachineType != "" { machineType = s.AWS.MachineType @@ -338,7 +337,7 @@ func (s *ClusterSpec) RoachprodOpts( if machineType == "" { // If no machine type was specified, choose one // based on the cloud and CPU count. - switch s.Cloud { + switch cloud { case AWS: machineType, selectedArch = SelectAWSMachineType(s.CPUs, s.Mem, preferLocalSSD && s.VolumeSize == 0, arch) case GCE: @@ -359,8 +358,8 @@ func (s *ClusterSpec) RoachprodOpts( // - if no particular volume size is requested, and, // - on AWS, if the machine type supports it. // - on GCE, if the machine type is not ARM64. - if preferLocalSSD && s.VolumeSize == 0 && (s.Cloud != AWS || awsMachineSupportsSSD(machineType)) && - (s.Cloud != GCE || selectedArch != vm.ArchARM64) { + if preferLocalSSD && s.VolumeSize == 0 && (cloud != AWS || awsMachineSupportsSSD(machineType)) && + (cloud != GCE || selectedArch != vm.ArchARM64) { // Ensure SSD count is at least 1 if UseLocalSSD is true. if ssdCount == 0 { ssdCount = 1 @@ -373,13 +372,13 @@ func (s *ClusterSpec) RoachprodOpts( } if s.FileSystem == Zfs { - if s.Cloud != GCE { + if cloud != GCE { return vm.CreateOpts{}, nil, errors.Errorf( - "node creation with zfs file system not yet supported on %s", s.Cloud, + "node creation with zfs file system not yet supported on %s", cloud, ) } createVMOpts.SSDOpts.FileSystem = vm.Zfs - } else if s.RandomlyUseZfs && s.Cloud == GCE { + } else if s.RandomlyUseZfs && cloud == GCE { rng, _ := randutil.NewPseudoRand() if rng.Float64() <= 0.2 { createVMOpts.SSDOpts.FileSystem = vm.Zfs @@ -387,7 +386,7 @@ func (s *ClusterSpec) RoachprodOpts( } zonesStr := params.Defaults.Zones - switch s.Cloud { + switch cloud { case AWS: if s.AWS.Zones != "" { zonesStr = s.AWS.Zones @@ -405,13 +404,13 @@ func (s *ClusterSpec) RoachprodOpts( } } - if createVMOpts.Arch == string(vm.ArchFIPS) && !(s.Cloud == GCE || s.Cloud == AWS) { + if createVMOpts.Arch == string(vm.ArchFIPS) && !(cloud == GCE || cloud == AWS) { return vm.CreateOpts{}, nil, errors.Errorf( - "FIPS not yet supported on %s", s.Cloud, + "FIPS not yet supported on %s", cloud, ) } var providerOpts vm.ProviderOpts - switch s.Cloud { + switch cloud { case AWS: providerOpts = getAWSOpts(machineType, zones, s.VolumeSize, s.AWS.VolumeThroughput, createVMOpts.SSDOpts.UseLocalSSD) diff --git a/pkg/cmd/roachtest/spec/option.go b/pkg/cmd/roachtest/spec/option.go index b0e2a0adecd8..d378073edaa4 100644 --- a/pkg/cmd/roachtest/spec/option.go +++ b/pkg/cmd/roachtest/spec/option.go @@ -19,13 +19,6 @@ import ( // Option for MakeClusterSpec. type Option func(spec *ClusterSpec) -// Cloud controls what cloud is used to create the cluster. -func Cloud(s string) Option { - return func(spec *ClusterSpec) { - spec.Cloud = s - } -} - // Arch requests a specific CPU architecture. // // Note that it is not guaranteed that this architecture will be used (e.g. if diff --git a/pkg/cmd/roachtest/test_registry.go b/pkg/cmd/roachtest/test_registry.go index c4f064bba936..10466f5a084b 100644 --- a/pkg/cmd/roachtest/test_registry.go +++ b/pkg/cmd/roachtest/test_registry.go @@ -24,8 +24,7 @@ import ( ) type testRegistryImpl struct { - m map[string]*registry.TestSpec - cloud string + m map[string]*registry.TestSpec promRegistry *prometheus.Registry } @@ -33,9 +32,8 @@ type testRegistryImpl struct { var _ registry.Registry = (*testRegistryImpl)(nil) // makeTestRegistry constructs a testRegistryImpl and configures it with opts. -func makeTestRegistry(cloud string) testRegistryImpl { +func makeTestRegistry() testRegistryImpl { return testRegistryImpl{ - cloud: cloud, m: make(map[string]*registry.TestSpec), promRegistry: prometheus.NewRegistry(), } @@ -59,7 +57,7 @@ func (r *testRegistryImpl) Add(spec registry.TestSpec) { // MakeClusterSpec makes a cluster spec. It should be used over `spec.MakeClusterSpec` // because this method also adds options baked into the registry. func (r *testRegistryImpl) MakeClusterSpec(nodeCount int, opts ...spec.Option) spec.ClusterSpec { - return spec.MakeClusterSpec(r.cloud, nodeCount, opts...) + return spec.MakeClusterSpec(nodeCount, opts...) } const testNameRE = "^[a-zA-Z0-9-_=/,]+$" @@ -129,7 +127,3 @@ func (r testRegistryImpl) AllTests() []registry.TestSpec { }) return tests } - -func (r testRegistryImpl) Cloud() string { - return r.cloud -} diff --git a/pkg/cmd/roachtest/test_registry_test.go b/pkg/cmd/roachtest/test_registry_test.go index bba1af7685fa..396c97f697d3 100644 --- a/pkg/cmd/roachtest/test_registry_test.go +++ b/pkg/cmd/roachtest/test_registry_test.go @@ -21,8 +21,7 @@ import ( ) func TestMakeTestRegistry(t *testing.T) { - r := makeTestRegistry(spec.AWS) - require.Equal(t, spec.AWS, r.cloud) + r := makeTestRegistry() s := r.MakeClusterSpec(100, spec.Geo(), spec.CPU(12), spec.PreferLocalSSD()) @@ -52,7 +51,7 @@ func TestMakeTestRegistry(t *testing.T) { // TestPrometheusMetricParser tests that the registry.PromSub() // helper properly converts a string into a metric name that Prometheus can read. func TestPrometheusMetricParser(t *testing.T) { - r := makeTestRegistry(spec.AWS) + r := makeTestRegistry() f := r.PromFactory() rawName := "restore/nodes=4/duration" diff --git a/pkg/cmd/roachtest/test_runner.go b/pkg/cmd/roachtest/test_runner.go index 7063ef0b8a7c..3efe33a3bcf4 100644 --- a/pkg/cmd/roachtest/test_runner.go +++ b/pkg/cmd/roachtest/test_runner.go @@ -943,7 +943,7 @@ func (r *testRunner) runTest( s := t.Spec().(*registry.TestSpec) - grafanaAvailable := s.Cluster.Cloud == spec.GCE + grafanaAvailable := roachtestflags.Cloud == spec.GCE if err := c.addLabels(map[string]string{VmLabelTestName: testRunID}); err != nil { shout(ctx, l, stdout, "failed to add label to cluster [%s] - %s", c.Name(), err) grafanaAvailable = false diff --git a/pkg/cmd/roachtest/test_test.go b/pkg/cmd/roachtest/test_test.go index 6216c2c5d298..d26e8c879038 100644 --- a/pkg/cmd/roachtest/test_test.go +++ b/pkg/cmd/roachtest/test_test.go @@ -44,7 +44,7 @@ const defaultParallelism = 10 func mkReg(t *testing.T) testRegistryImpl { t.Helper() - return makeTestRegistry(spec.GCE) + return makeTestRegistry() } func nilLogger() *logger.Logger { @@ -319,7 +319,7 @@ func TestRunnerTestTimeout(t *testing.T) { Name: `timeout`, Owner: OwnerUnitTest, Timeout: 10 * time.Millisecond, - Cluster: spec.MakeClusterSpec(spec.GCE, 0), + Cluster: spec.MakeClusterSpec(0), CompatibleClouds: registry.AllExceptAWS, Suites: registry.Suites(registry.Nightly), Run: func(ctx context.Context, t test.Test, c cluster.Cluster) { @@ -356,7 +356,7 @@ func TestRegistryPrepareSpec(t *testing.T) { Name: "a", Owner: OwnerUnitTest, Run: dummyRun, - Cluster: spec.MakeClusterSpec(spec.GCE, 0), + Cluster: spec.MakeClusterSpec(0), CompatibleClouds: registry.AllExceptAWS, Suites: registry.Suites(registry.Nightly), }, @@ -368,7 +368,7 @@ func TestRegistryPrepareSpec(t *testing.T) { Name: "illegal *[]", Owner: OwnerUnitTest, Run: dummyRun, - Cluster: spec.MakeClusterSpec(spec.GCE, 0), + Cluster: spec.MakeClusterSpec(0), CompatibleClouds: registry.AllExceptAWS, Suites: registry.Suites(registry.Nightly), }, @@ -378,7 +378,7 @@ func TestRegistryPrepareSpec(t *testing.T) { } for _, c := range testCases { t.Run("", func(t *testing.T) { - r := makeTestRegistry(spec.GCE) + r := makeTestRegistry() err := r.prepareSpec(&c.spec) if !testutils.IsError(err, c.expectedErr) { t.Fatalf("expected %q, but found %q", c.expectedErr, err.Error()) @@ -405,7 +405,7 @@ func runExitCodeTest(t *testing.T, injectedError error) error { r.Add(registry.TestSpec{ Name: "boom", Owner: OwnerUnitTest, - Cluster: spec.MakeClusterSpec(spec.GCE, 0), + Cluster: spec.MakeClusterSpec(0), CompatibleClouds: registry.AllExceptAWS, Suites: registry.Suites(registry.Nightly), Run: func(ctx context.Context, t test.Test, c cluster.Cluster) { diff --git a/pkg/cmd/roachtest/tests/backup_restore_roundtrip.go b/pkg/cmd/roachtest/tests/backup_restore_roundtrip.go index 77cdd5610a57..0534a4936184 100644 --- a/pkg/cmd/roachtest/tests/backup_restore_roundtrip.go +++ b/pkg/cmd/roachtest/tests/backup_restore_roundtrip.go @@ -73,7 +73,7 @@ func registerBackupRestoreRoundTrip(r registry.Registry) { Cluster: r.MakeClusterSpec(4), EncryptionSupport: registry.EncryptionMetamorphic, RequiresLicense: true, - CompatibleClouds: registry.AllExceptAWS, + CompatibleClouds: registry.OnlyGCE, Suites: registry.Suites(registry.Nightly), Run: func(ctx context.Context, t test.Test, c cluster.Cluster) { backupRestoreRoundTrip(ctx, t, c, sp.metamorphicRangeSize) @@ -87,7 +87,7 @@ func registerBackupRestoreRoundTrip(r registry.Registry) { func backupRestoreRoundTrip( ctx context.Context, t test.Test, c cluster.Cluster, metamorphicRangeSize bool, ) { - if c.Spec().Cloud != spec.GCE { + if c.Cloud() != spec.GCE { t.Skip("uses gs://cockroachdb-backup-testing; see https://github.com/cockroachdb/cockroach/issues/105968") } pauseProbability := 0.2 diff --git a/pkg/cmd/roachtest/tests/cdc.go b/pkg/cmd/roachtest/tests/cdc.go index 5ababa222566..d41583844e3d 100644 --- a/pkg/cmd/roachtest/tests/cdc.go +++ b/pkg/cmd/roachtest/tests/cdc.go @@ -946,14 +946,6 @@ func runCDCKafkaAuth(ctx context.Context, t test.Test, c cluster.Cluster) { } } -func skipLocalUnderArm64(cloud string) string { - if cloud == spec.Local && runtime.GOARCH == "arm64" { - // N.B. we also have to skip locally since amd64 emulation may not be available everywhere. - return "Skip under ARM64." - } - return "" -} - func registerCDC(r registry.Registry) { r.Add(registry.TestSpec{ Name: "cdc/initial-scan-only", @@ -1349,13 +1341,18 @@ func registerCDC(r registry.Registry) { Owner: `cdc`, Benchmark: true, // Only Kafka 3 supports Arm64, but the broker setup for Oauth used only works with Kafka 2 - Skip: skipLocalUnderArm64(r.Cloud()), Cluster: r.MakeClusterSpec(4, spec.Arch(vm.ArchAMD64)), Leases: registry.MetamorphicLeases, CompatibleClouds: registry.AllExceptAWS, Suites: registry.Suites(registry.Nightly), RequiresLicense: true, Run: func(ctx context.Context, t test.Test, c cluster.Cluster) { + if c.Cloud() == spec.Local && runtime.GOARCH == "arm64" { + // N.B. We have to skip locally since amd64 emulation may not be available everywhere. + t.L().PrintfCtx(ctx, "Skipping test under ARM64") + return + } + ct := newCDCTester(ctx, t, c) defer ct.Close() diff --git a/pkg/cmd/roachtest/tests/import_cancellation.go b/pkg/cmd/roachtest/tests/import_cancellation.go index 9cc2feceaf5b..c2a8db2c0ae0 100644 --- a/pkg/cmd/roachtest/tests/import_cancellation.go +++ b/pkg/cmd/roachtest/tests/import_cancellation.go @@ -36,13 +36,10 @@ func registerImportCancellation(r registry.Registry) { Benchmark: true, Timeout: 4 * time.Hour, Cluster: r.MakeClusterSpec(6, spec.CPU(32)), - CompatibleClouds: registry.AllExceptAWS, + CompatibleClouds: registry.OnlyGCE, Suites: registry.Suites(registry.Nightly), Leases: registry.MetamorphicLeases, Run: func(ctx context.Context, t test.Test, c cluster.Cluster) { - if c.Spec().Cloud != spec.GCE { - t.Skip("uses gs://cockroach-fixtures-us-east1; see https://github.com/cockroachdb/cockroach/issues/105968") - } runImportCancellation(ctx, t, c, rangeTombstones) }, }) diff --git a/pkg/cmd/roachtest/tests/tpcc_test.go b/pkg/cmd/roachtest/tests/tpcc_test.go index 886332e7f131..aff2928a2fe8 100644 --- a/pkg/cmd/roachtest/tests/tpcc_test.go +++ b/pkg/cmd/roachtest/tests/tpcc_test.go @@ -26,16 +26,16 @@ func TestTPCCSupportedWarehouses(t *testing.T) { buildVersion *version.Version expected int }{ - {"gce", spec.MakeClusterSpec(spec.GCE, 4, spec.CPU(16)), version.MustParse(`v2.1.0`), 1300}, - {"gce", spec.MakeClusterSpec(spec.GCE, 4, spec.CPU(16)), version.MustParse(`v19.1.0-rc.1`), 1250}, - {"gce", spec.MakeClusterSpec(spec.GCE, 4, spec.CPU(16)), version.MustParse(`v19.1.0`), 1250}, + {"gce", spec.MakeClusterSpec(4, spec.CPU(16)), version.MustParse(`v2.1.0`), 1300}, + {"gce", spec.MakeClusterSpec(4, spec.CPU(16)), version.MustParse(`v19.1.0-rc.1`), 1250}, + {"gce", spec.MakeClusterSpec(4, spec.CPU(16)), version.MustParse(`v19.1.0`), 1250}, - {"aws", spec.MakeClusterSpec(spec.GCE, 4, spec.CPU(16)), version.MustParse(`v19.1.0-rc.1`), 2100}, - {"aws", spec.MakeClusterSpec(spec.GCE, 4, spec.CPU(16)), version.MustParse(`v19.1.0`), 2100}, + {"aws", spec.MakeClusterSpec(4, spec.CPU(16)), version.MustParse(`v19.1.0-rc.1`), 2100}, + {"aws", spec.MakeClusterSpec(4, spec.CPU(16)), version.MustParse(`v19.1.0`), 2100}, - {"nope", spec.MakeClusterSpec(spec.GCE, 4, spec.CPU(16)), version.MustParse(`v2.1.0`), expectPanic}, - {"gce", spec.MakeClusterSpec(spec.GCE, 5, spec.CPU(160)), version.MustParse(`v2.1.0`), expectPanic}, - {"gce", spec.MakeClusterSpec(spec.GCE, 4, spec.CPU(16)), version.MustParse(`v1.0.0`), expectPanic}, + {"nope", spec.MakeClusterSpec(4, spec.CPU(16)), version.MustParse(`v2.1.0`), expectPanic}, + {"gce", spec.MakeClusterSpec(5, spec.CPU(160)), version.MustParse(`v2.1.0`), expectPanic}, + {"gce", spec.MakeClusterSpec(4, spec.CPU(16)), version.MustParse(`v1.0.0`), expectPanic}, } for _, test := range tests { t.Run("", func(t *testing.T) {