Skip to content

Commit

Permalink
roachtest: run cluster backup schedule by default
Browse files Browse the repository at this point in the history
This patch creates a cluster backup schedule by default on roachtests. A
roachtest writer can override this default behavior by passing
option.DefaultStartOptsNoBackups() at cluster.Start().

Roachtests are supposed to mirror a customer env. Like a customer's cluster,
roachtests currently run background jobs like gc, sql stats computation,
rebalancing etc. But unlike most customer clusters, roachtests do not run
scheduled backup jobs by default. Any workload on a roachtest should run just
as well with or without with background jobs, but we currently do not test
this, at least wrt to scheduled backups. This patch addresses this lack of test
coverage.

This new feature is disabled on performance sentive roachtests for which an
additional background job will make it harder to debug a perf regression. This
includes:
- kv0*
- decomissionbench*
- ycsb*
- admission-control*
- backup*
- restore*
- kvbench*

Fixes #86045

Release note: None
  • Loading branch information
msbutler committed Feb 6, 2023
1 parent 6f38bdd commit eec5a47
Show file tree
Hide file tree
Showing 24 changed files with 88 additions and 47 deletions.
8 changes: 8 additions & 0 deletions pkg/cmd/roachtest/option/options.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,14 @@ type StartOpts struct {

// DefaultStartOpts returns a StartOpts populated with default values.
func DefaultStartOpts() StartOpts {
startOpts := StartOpts{RoachprodOpts: roachprod.DefaultStartOpts()}
startOpts.RoachprodOpts.ScheduleBackups = true
return startOpts
}

// DefaultStartOptsNoBackups returns a StartOpts with default values,
// but a scheduled backup will not begin at the start of the roachtest.
func DefaultStartOptsNoBackups() StartOpts {
return StartOpts{RoachprodOpts: roachprod.DefaultStartOpts()}
}

Expand Down
15 changes: 8 additions & 7 deletions pkg/cmd/roachtest/tests/admission_control_elastic_backup.go
Original file line number Diff line number Diff line change
Expand Up @@ -73,13 +73,14 @@ func registerElasticControlForBackups(r registry.Registry) {
}

runTPCC(ctx, t, c, tpccOptions{
Warehouses: numWarehouses,
Duration: workloadDuration,
SetupType: usingImport,
EstimatedSetupTime: estimatedSetupTime,
SkipPostRunCheck: true,
ExtraSetupArgs: "--checks=false",
PrometheusConfig: promCfg,
Warehouses: numWarehouses,
Duration: workloadDuration,
SetupType: usingImport,
EstimatedSetupTime: estimatedSetupTime,
SkipPostRunCheck: true,
ExtraSetupArgs: "--checks=false",
PrometheusConfig: promCfg,
DisableDefaultScheduledBackup: true,
During: func(ctx context.Context) error {
db := c.Conn(ctx, t.L(), crdbNodes)
defer db.Close()
Expand Down
15 changes: 8 additions & 7 deletions pkg/cmd/roachtest/tests/admission_control_elastic_cdc.go
Original file line number Diff line number Diff line change
Expand Up @@ -76,13 +76,14 @@ func registerElasticControlForCDC(r registry.Registry) {
}

runTPCC(ctx, t, c, tpccOptions{
Warehouses: numWarehouses,
Duration: workloadDuration,
SetupType: usingImport,
EstimatedSetupTime: estimatedSetupTime,
SkipPostRunCheck: true,
ExtraSetupArgs: "--checks=false",
PrometheusConfig: promCfg,
Warehouses: numWarehouses,
Duration: workloadDuration,
SetupType: usingImport,
EstimatedSetupTime: estimatedSetupTime,
SkipPostRunCheck: true,
ExtraSetupArgs: "--checks=false",
PrometheusConfig: promCfg,
DisableDefaultScheduledBackup: true,
During: func(ctx context.Context) error {
db := c.Conn(ctx, t.L(), crdbNodes)
defer db.Close()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,7 @@ func registerIndexOverload(r registry.Registry) {
workloadNode := c.Spec().NodeCount

c.Put(ctx, t.Cockroach(), "./cockroach", c.All())
c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings(), c.Range(1, crdbNodes))
c.Start(ctx, t.L(), option.DefaultStartOptsNoBackups(), install.MakeClusterSettings(), c.Range(1, crdbNodes))

{
promCfg := &prometheus.Config{}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@ func registerMultiStoreOverload(r registry.Registry) {
nodes := c.Spec().NodeCount - 1
c.Put(ctx, t.Cockroach(), "./cockroach", c.Range(1, nodes))
c.Put(ctx, t.DeprecatedWorkload(), "./workload", c.Node(nodes+1))
startOpts := option.DefaultStartOpts()
startOpts := option.DefaultStartOptsNoBackups()
startOpts.RoachprodOpts.StoreCount = 2
c.Start(ctx, t.L(), startOpts, install.MakeClusterSettings(), c.Range(1, nodes))

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -136,7 +136,7 @@ func runMultiTenantFairness(
t.L().Printf("starting cockroach securely (<%s)", time.Minute)
c.Put(ctx, t.Cockroach(), "./cockroach")
c.Start(ctx, t.L(),
option.DefaultStartOpts(),
option.DefaultStartOptsNoBackups(),
install.MakeClusterSettings(install.SecureOption(true)),
crdbNode,
)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,7 @@ func registerSnapshotOverload(r registry.Registry) {
crdbNodes := c.Spec().NodeCount - 1
workloadNode := crdbNodes + 1
for i := 1; i <= crdbNodes; i++ {
startOpts := option.DefaultStartOpts()
startOpts := option.DefaultStartOptsNoBackups()
startOpts.RoachprodOpts.ExtraArgs = append(startOpts.RoachprodOpts.ExtraArgs, fmt.Sprintf("--attrs=n%d", i))
c.Start(ctx, t.L(), startOpts, install.MakeClusterSettings(), c.Node(i))
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/admission_control_tpcc_overload.go
Original file line number Diff line number Diff line change
Expand Up @@ -170,7 +170,7 @@ func registerTPCCSevereOverload(r registry.Registry) {
workloadNode := c.Spec().NodeCount

c.Put(ctx, t.Cockroach(), "./cockroach", c.All())
c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings(), roachNodes)
c.Start(ctx, t.L(), option.DefaultStartOptsNoBackups(), install.MakeClusterSettings(), roachNodes)

t.Status("initializing (~1h)")
c.Run(ctx, c.Node(workloadNode), "./cockroach workload fixtures import tpcc --checks=false --warehouses=10000 {pgurl:1}")
Expand Down
3 changes: 2 additions & 1 deletion pkg/cmd/roachtest/tests/allocation_bench.go
Original file line number Diff line number Diff line change
Expand Up @@ -289,7 +289,8 @@ func setupAllocationBench(
c.Put(ctx, t.DeprecatedWorkload(), "./workload", c.Node(workloadNode))
t.Status("starting cluster")
for i := 1; i <= spec.nodes; i++ {
startOpts := option.DefaultStartOpts()
// Don't start a backup schedule as this test reports to roachperf.
startOpts := option.DefaultStartOptsNoBackups()
if attr, ok := spec.nodeAttrs[i]; ok {
startOpts.RoachprodOpts.ExtraArgs = append(startOpts.RoachprodOpts.ExtraArgs,
fmt.Sprintf("--attrs=%s", attr))
Expand Down
4 changes: 3 additions & 1 deletion pkg/cmd/roachtest/tests/allocator.go
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,9 @@ func registerAllocator(r registry.Registry) {

// Put away one node to be the stats collector.
nodes := c.Spec().NodeCount - 1
startOpts := option.DefaultStartOpts()

// Don't start scheduled backups in this perf sensitive test that reports to roachperf
startOpts := option.DefaultStartOptsNoBackups()
startOpts.RoachprodOpts.ExtraArgs = []string{"--vmodule=store_rebalancer=5,allocator=5,allocator_scorer=5,replicate_queue=5"}
c.Start(ctx, t.L(), startOpts, install.MakeClusterSettings(), c.Range(1, start))
db := c.Conn(ctx, t.L(), 1)
Expand Down
6 changes: 3 additions & 3 deletions pkg/cmd/roachtest/tests/backup.go
Original file line number Diff line number Diff line change
Expand Up @@ -99,7 +99,7 @@ func importBankDataSplit(

// NB: starting the cluster creates the logs dir as a side effect,
// needed below.
c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings())
c.Start(ctx, t.L(), option.DefaultStartOptsNoBackups(), install.MakeClusterSettings())
runImportBankDataSplit(ctx, rows, ranges, t, c)
return dest
}
Expand Down Expand Up @@ -906,7 +906,7 @@ func registerBackup(r registry.Registry) {
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
c.Put(ctx, t.Cockroach(), "./cockroach")
c.Put(ctx, t.DeprecatedWorkload(), "./workload")
c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings())
c.Start(ctx, t.L(), option.DefaultStartOptsNoBackups(), install.MakeClusterSettings())
conn := c.Conn(ctx, t.L(), 1)

duration := 5 * time.Minute
Expand Down Expand Up @@ -1135,7 +1135,7 @@ func registerBackup(r registry.Registry) {
func runBackupMVCCRangeTombstones(ctx context.Context, t test.Test, c cluster.Cluster) {
c.Put(ctx, t.Cockroach(), "./cockroach")
c.Put(ctx, t.DeprecatedWorkload(), "./workload") // required for tpch
c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings())
c.Start(ctx, t.L(), option.DefaultStartOptsNoBackups(), install.MakeClusterSettings())
t.Status("starting csv servers")
c.Run(ctx, c.All(), `./cockroach workload csv-server --port=8081 &> logs/workload-csv-server.log < /dev/null &`)

Expand Down
3 changes: 2 additions & 1 deletion pkg/cmd/roachtest/tests/connection_latency.go
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,8 @@ func runConnectionLatencyTest(
require.NoError(t, err)

settings := install.MakeClusterSettings(install.SecureOption(true))
err = c.StartE(ctx, t.L(), option.DefaultStartOpts(), settings)
// Don't start a backup schedule as this roachtest reports roachperf results.
err = c.StartE(ctx, t.L(), option.DefaultStartOptsNoBackups(), settings)
require.NoError(t, err)

var passwordFlag string
Expand Down
3 changes: 2 additions & 1 deletion pkg/cmd/roachtest/tests/decommission.go
Original file line number Diff line number Diff line change
Expand Up @@ -707,7 +707,8 @@ func runDecommissionRandomized(ctx context.Context, t test.Test, c cluster.Clust
t.L().Printf("expected to fail: restarting [n%d,n%d] and attempting to recommission through n%d\n",
targetNodeA, targetNodeB, runNode)
c.Stop(ctx, t.L(), option.DefaultStopOpts(), c.Nodes(targetNodeA, targetNodeB))
c.Start(ctx, t.L(), option.DefaultStartOpts(), settings, c.Nodes(targetNodeA, targetNodeB))
// The node is in a decomissioned state, so don't attempt to run scheduled backups.
c.Start(ctx, t.L(), option.DefaultStartOptsNoBackups(), settings, c.Nodes(targetNodeA, targetNodeB))

if _, err := h.recommission(ctx, c.Nodes(targetNodeA, targetNodeB), runNode); err == nil {
t.Fatalf("expected recommission to fail")
Expand Down
3 changes: 2 additions & 1 deletion pkg/cmd/roachtest/tests/decommissionbench.go
Original file line number Diff line number Diff line change
Expand Up @@ -371,7 +371,8 @@ func setupDecommissionBench(
c.Put(ctx, t.Cockroach(), "./cockroach", c.All())
c.Put(ctx, t.DeprecatedWorkload(), "./workload", c.Node(workloadNode))
for i := 1; i <= benchSpec.nodes; i++ {
startOpts := option.DefaultStartOpts()
// Don't start a scheduled backup as this roachtest reports to roachperf.
startOpts := option.DefaultStartOptsNoBackups()
startOpts.RoachprodOpts.ExtraArgs = append(startOpts.RoachprodOpts.ExtraArgs,
fmt.Sprintf("--attrs=node%d", i),
"--vmodule=store_rebalancer=5,allocator=5,allocator_scorer=5,replicate_queue=5")
Expand Down
4 changes: 2 additions & 2 deletions pkg/cmd/roachtest/tests/failover.go
Original file line number Diff line number Diff line change
Expand Up @@ -258,8 +258,8 @@ func runFailoverLiveness(

rng, _ := randutil.NewTestRand()

// Create cluster.
opts := option.DefaultStartOpts()
// Create cluster. Don't schedule a backup as this roachtest reports to roachperf.
opts := option.DefaultStartOptsNoBackups()
settings := install.MakeClusterSettings()

failer := makeFailer(t, c, failureMode, opts, settings)
Expand Down
4 changes: 3 additions & 1 deletion pkg/cmd/roachtest/tests/kv.go
Original file line number Diff line number Diff line change
Expand Up @@ -83,7 +83,9 @@ func registerKV(r registry.Registry) {
nodes := c.Spec().NodeCount - 1
c.Put(ctx, t.Cockroach(), "./cockroach", c.Range(1, nodes))
c.Put(ctx, t.DeprecatedWorkload(), "./workload", c.Node(nodes+1))
startOpts := option.DefaultStartOpts()

// Don't start a scheduled backup on this perf sensitive roachtest that reports to roachperf.
startOpts := option.DefaultStartOptsNoBackups()
if opts.ssds > 1 && !opts.raid0 {
startOpts.RoachprodOpts.StoreCount = opts.ssds
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/kvbench.go
Original file line number Diff line number Diff line change
Expand Up @@ -230,7 +230,7 @@ func runKVBench(ctx context.Context, t test.Test, c cluster.Cluster, b kvBenchSp
// splitting can significantly change the underlying layout of the table and
// affect benchmark results.
c.Wipe(ctx, roachNodes)
c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings(), roachNodes)
c.Start(ctx, t.L(), option.DefaultStartOptsNoBackups(), install.MakeClusterSettings(), roachNodes)
time.Sleep(restartWait)

// We currently only support one loadGroup.
Expand Down
4 changes: 3 additions & 1 deletion pkg/cmd/roachtest/tests/ledger.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,9 @@ func registerLedger(r registry.Registry) {

c.Put(ctx, t.Cockroach(), "./cockroach", roachNodes)
c.Put(ctx, t.DeprecatedWorkload(), "./workload", loadNode)
c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings(), roachNodes)

// Don't start a scheduled backup on this perf sensitive roachtest that reports to roachperf.
c.Start(ctx, t.L(), option.DefaultStartOptsNoBackups(), install.MakeClusterSettings(), roachNodes)

t.Status("running workload")
m := c.NewMonitor(ctx, roachNodes)
Expand Down
3 changes: 2 additions & 1 deletion pkg/cmd/roachtest/tests/mvcc_gc.go
Original file line number Diff line number Diff line change
Expand Up @@ -77,7 +77,8 @@ func runMVCCGC(ctx context.Context, t test.Test, c cluster.Cluster) {
c.Put(ctx, t.Cockroach(), "./cockroach")
s := install.MakeClusterSettings()
s.Env = append(s.Env, "COCKROACH_SCAN_INTERVAL=30s")
c.Start(ctx, t.L(), option.DefaultStartOpts(), s)
// Disable an automatic scheduled backup as it would mess with the gc ttl this test relies on.
c.Start(ctx, t.L(), option.DefaultStartOptsNoBackups(), s)

conn := c.Conn(ctx, t.L(), 1)
defer conn.Close()
Expand Down
12 changes: 9 additions & 3 deletions pkg/cmd/roachtest/tests/network.go
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,9 @@ func runNetworkAuthentication(ctx context.Context, t test.Test, c cluster.Cluste
// we don't want. Starting all nodes at once ensures
// that they use coherent certs.
settings := install.MakeClusterSettings(install.SecureOption(true))
c.Start(ctx, t.L(), option.DefaultStartOpts(), settings, serverNodes)

// Don't create a backup schedule as this test shuts the cluster down immediately.
c.Start(ctx, t.L(), option.DefaultStartOptsNoBackups(), settings, serverNodes)
require.NoError(t, c.StopE(ctx, t.L(), option.DefaultStopOpts(), serverNodes))

t.L().Printf("restarting nodes...")
Expand All @@ -58,14 +60,18 @@ func runNetworkAuthentication(ctx context.Context, t test.Test, c cluster.Cluste
// in a way that is unrealistically fast.
// "--env=COCKROACH_SCAN_INTERVAL=200ms",
// "--env=COCKROACH_SCAN_MAX_IDLE_TIME=20ms",
startOpts := option.DefaultStartOpts()
//
// Currently, creating a scheduled backup at start fails, potentially due to
// the induced network partition. Further investigation required to allow scheduled backups
// to run on this test.
startOpts := option.DefaultStartOptsNoBackups()
startOpts.RoachprodOpts.ExtraArgs = append(startOpts.RoachprodOpts.ExtraArgs, "--locality=node=1", "--accept-sql-without-tls")
c.Start(ctx, t.L(), startOpts, settings, c.Node(1))

// See comment above about env vars.
// "--env=COCKROACH_SCAN_INTERVAL=200ms",
// "--env=COCKROACH_SCAN_MAX_IDLE_TIME=20ms",
startOpts = option.DefaultStartOpts()
startOpts = option.DefaultStartOptsNoBackups()
startOpts.RoachprodOpts.ExtraArgs = append(startOpts.RoachprodOpts.ExtraArgs, "--locality=node=other", "--accept-sql-without-tls")
c.Start(ctx, t.L(), startOpts, settings, c.Range(2, n-1))

Expand Down
10 changes: 5 additions & 5 deletions pkg/cmd/roachtest/tests/restore.go
Original file line number Diff line number Diff line change
Expand Up @@ -304,7 +304,7 @@ func registerRestoreNodeShutdown(r registry.Registry) {
gatewayNode := 2
nodeToShutdown := 3
c.Put(ctx, t.Cockroach(), "./cockroach")
c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings())
c.Start(ctx, t.L(), option.DefaultStartOptsNoBackups(), install.MakeClusterSettings())

jobSurvivesNodeShutdown(ctx, t, c, nodeToShutdown, makeRestoreStarter(ctx, t, c, gatewayNode))
},
Expand All @@ -318,7 +318,7 @@ func registerRestoreNodeShutdown(r registry.Registry) {
gatewayNode := 2
nodeToShutdown := 2
c.Put(ctx, t.Cockroach(), "./cockroach")
c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings())
c.Start(ctx, t.L(), option.DefaultStartOptsNoBackups(), install.MakeClusterSettings())

jobSurvivesNodeShutdown(ctx, t, c, nodeToShutdown, makeRestoreStarter(ctx, t, c, gatewayNode))
},
Expand Down Expand Up @@ -453,7 +453,7 @@ func registerRestore(r registry.Registry) {
EncryptionSupport: registry.EncryptionMetamorphic,
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
c.Put(ctx, t.Cockroach(), "./cockroach")
c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings())
c.Start(ctx, t.L(), option.DefaultStartOptsNoBackups(), install.MakeClusterSettings())
m := c.NewMonitor(ctx)

// Run the disk usage logger in the monitor to guarantee its
Expand Down Expand Up @@ -523,7 +523,7 @@ func registerRestore(r registry.Registry) {
Timeout: withPauseTimeout,
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
c.Put(ctx, t.Cockroach(), "./cockroach")
c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings())
c.Start(ctx, t.L(), option.DefaultStartOptsNoBackups(), install.MakeClusterSettings())
m := c.NewMonitor(ctx)

// Run the disk usage logger in the monitor to guarantee its
Expand Down Expand Up @@ -728,7 +728,7 @@ func registerRestore(r registry.Registry) {
t.Skip("test configured to run on %s", sp.hardware.cloud)
}
c.Put(ctx, t.Cockroach(), "./cockroach")
c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings())
c.Start(ctx, t.L(), option.DefaultStartOptsNoBackups(), install.MakeClusterSettings())
m := c.NewMonitor(ctx)

// Run the disk usage logger in the monitor to guarantee its
Expand Down
7 changes: 5 additions & 2 deletions pkg/cmd/roachtest/tests/tpcc.go
Original file line number Diff line number Diff line change
Expand Up @@ -100,7 +100,8 @@ type tpccOptions struct {
// TODO(tbg): remove this once https://github.com/cockroachdb/cockroach/issues/74705 is completed.
EnableCircuitBreakers bool
// SkipPostRunCheck, if set, skips post TPC-C run checks.
SkipPostRunCheck bool
SkipPostRunCheck bool
DisableDefaultScheduledBackup bool
}

type workloadInstance struct {
Expand Down Expand Up @@ -155,7 +156,9 @@ func setupTPCC(
settings.Env = append(settings.Env, "COCKROACH_SCAN_INTERVAL=200ms")
settings.Env = append(settings.Env, "COCKROACH_SCAN_MAX_IDLE_TIME=5ms")
}
c.Start(ctx, t.L(), option.DefaultStartOpts(), settings, crdbNodes)
startOpts := option.DefaultStartOpts()
startOpts.RoachprodOpts.ScheduleBackups = !opts.DisableDefaultScheduledBackup
c.Start(ctx, t.L(), startOpts, settings, crdbNodes)
}
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/ycsb.go
Original file line number Diff line number Diff line change
Expand Up @@ -67,7 +67,7 @@ func registerYCSB(r registry.Registry) {

c.Put(ctx, t.Cockroach(), "./cockroach", c.Range(1, nodes))
c.Put(ctx, t.DeprecatedWorkload(), "./workload", c.Node(nodes+1))
c.Start(ctx, t.L(), option.DefaultStartOpts(), settings, c.Range(1, nodes))
c.Start(ctx, t.L(), option.DefaultStartOptsNoBackups(), settings, c.Range(1, nodes))
err := WaitFor3XReplication(ctx, t, c.Conn(ctx, t.L(), 1))
require.NoError(t, err)

Expand Down
17 changes: 14 additions & 3 deletions pkg/roachprod/install/cockroach.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachprod/config"
"github.com/cockroachdb/cockroach/pkg/roachprod/logger"
"github.com/cockroachdb/cockroach/pkg/roachprod/ssh"
"github.com/cockroachdb/cockroach/pkg/roachprod/vm/gce"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/errors"
)
Expand Down Expand Up @@ -225,7 +226,9 @@ func (c *SyncedCluster) Start(ctx context.Context, l *logger.Logger, startOpts S
}, DefaultSSHRetryOpts); err != nil {
return err
}
if startOpts.ScheduleBackups {

// Only after a successful cluster initialization should we attempt to schedule backups.
if startOpts.ScheduleBackups && !startOpts.SkipInit {
return c.createFixedBackupSchedule(ctx, l, startOpts.ScheduleBackupArgs)
}
return nil
Expand Down Expand Up @@ -773,6 +776,7 @@ func (c *SyncedCluster) shouldAdvertisePublicIP() bool {
// `roachprod create`, the user can provide a different recurrence using the
// 'schedule-backup-args' flag. If roachprod is local, the backups get stored in
// nodelocal, and otherwise in 'gs://cockroachdb-backup-testing'.
// This cmd also ensures that only one schedule will be created for the cluster.
func (c *SyncedCluster) createFixedBackupSchedule(
ctx context.Context, l *logger.Logger, scheduledBackupArgs string,
) error {
Expand All @@ -781,10 +785,17 @@ func (c *SyncedCluster) createFixedBackupSchedule(
if c.IsLocal() {
externalStoragePath = `nodelocal://1`
}
for _, cloud := range c.Clouds() {
if !strings.Contains(cloud, gce.ProviderName) {
l.Printf(`no scheduled backup created as there exists a vm not on google cloud`)
return nil
}
}
l.Printf("%s: creating backup schedule", c.Name)
auth := "AUTH=implicit"

collectionPath := fmt.Sprintf(`%s/roachprod-scheduled-backups/%s/%v`,
externalStoragePath, c.Name, timeutil.Now().UnixNano())
collectionPath := fmt.Sprintf(`%s/roachprod-scheduled-backups/%s/%v?%s`,
externalStoragePath, c.Name, timeutil.Now().UnixNano(), auth)

// Default scheduled backup runs a full backup every hour and an incremental
// every 15 minutes.
Expand Down

0 comments on commit eec5a47

Please sign in to comment.