Skip to content

Commit

Permalink
Merge #129771
Browse files Browse the repository at this point in the history
129771: roachtest: add operation for pausing ldr job. r=vidit-bhat,nameisbhaskar a=shailendra-patel

On DRT clusters we run chaos operation. Adding support for a new operation, this operation will pause a LDR job running for sometime. This will help uncover issue like replication lag and time ldr takes to recover from it.

Epic: none

Release note: None

Co-authored-by: Shailendra Patel <[email protected]>
  • Loading branch information
craig[bot] and shailendra-patel committed Sep 3, 2024
2 parents 6318706 + 3a2d5c8 commit 1d35377
Show file tree
Hide file tree
Showing 5 changed files with 105 additions and 0 deletions.
1 change: 1 addition & 0 deletions pkg/cmd/roachtest/operations/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@ go_library(
"manual_compaction.go",
"network_partition.go",
"node_kill.go",
"pause_job.go",
"register.go",
"resize.go",
"utils.go",
Expand Down
88 changes: 88 additions & 0 deletions pkg/cmd/roachtest/operations/pause_job.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,88 @@
// Copyright 2024 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 operations

import (
"context"
"fmt"
"time"

"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/cluster"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/operation"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/option"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/registry"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestflags"
"github.com/cockroachdb/cockroach/pkg/util/randutil"
)

type resumePausedJob struct {
jobId string
}

func (r *resumePausedJob) Cleanup(ctx context.Context, o operation.Operation, c cluster.Cluster) {
conn := c.Conn(ctx, o.L(), 1, option.VirtualClusterName(roachtestflags.VirtualCluster))
defer conn.Close()

resumeJobStmt := fmt.Sprintf("RESUME JOB %s", r.jobId)
_, err := conn.ExecContext(ctx, resumeJobStmt)
if err != nil {
o.Fatal(err)
}
}

func pauseLDRJob(
ctx context.Context, o operation.Operation, c cluster.Cluster,
) registry.OperationCleanup {
conn := c.Conn(ctx, o.L(), 1, option.VirtualClusterName(roachtestflags.VirtualCluster))
defer conn.Close()

//fetch running ldr jobs
jobs, err := conn.QueryContext(ctx, "(WITH x AS (SHOW JOBS) SELECT job_id FROM x WHERE job_type = 'LOGICAL REPLICATION' AND status = 'running')")
if err != nil {
o.Fatal(err)
}

var jobIds []string
for jobs.Next() {
var jobId string
if err := jobs.Scan(&jobId); err != nil {
o.Fatal(err)
}
jobIds = append(jobIds, jobId)
}

//pick a random ldr job
rng, _ := randutil.NewPseudoRand()
jobId := jobIds[rng.Intn(len(jobIds))]

o.Status(fmt.Sprintf("pausing LDR job %s", jobId))
pauseJobStmt := fmt.Sprintf("PAUSE JOB %s WITH REASON = 'roachtest operation'", jobId)
_, err = conn.ExecContext(ctx, pauseJobStmt)
if err != nil {
o.Fatal(err)
}

o.Status(fmt.Sprintf("paused LDR job %s", jobId))
return &resumePausedJob{
jobId: jobId,
}
}

func registerPauseLDRJob(r registry.Registry) {
r.AddOperation(registry.OperationSpec{
Name: "pause-ldr",
Owner: registry.OwnerDisasterRecovery,
Timeout: 15 * time.Minute,
CompatibleClouds: registry.AllClouds,
Dependencies: []registry.OperationDependency{registry.OperationRequiresLDRJobRunning},
Run: pauseLDRJob,
})
}
1 change: 1 addition & 0 deletions pkg/cmd/roachtest/operations/register.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,4 +23,5 @@ func RegisterOperations(r registry.Registry) {
registerBackupRestore(r)
registerManualCompaction(r)
registerResize(r)
registerPauseLDRJob(r)
}
1 change: 1 addition & 0 deletions pkg/cmd/roachtest/registry/operation_spec.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ const (
OperationRequiresPopulatedDatabase
OperationRequiresZeroUnavailableRanges
OperationRequiresZeroUnderreplicatedRanges
OperationRequiresLDRJobRunning
)

// OperationCleanup specifies an operation that
Expand Down
14 changes: 14 additions & 0 deletions pkg/cmd/roachtest/roachtestutil/operations/dependency.go
Original file line number Diff line number Diff line change
Expand Up @@ -80,6 +80,20 @@ func CheckDependencies(
if count != 0 {
return false, nil
}
case registry.OperationRequiresLDRJobRunning:
conn := c.Conn(ctx, l, 1, option.VirtualClusterName("system"))
defer conn.Close()

jobsCur, err := conn.QueryContext(ctx, "(WITH x AS (SHOW JOBS) SELECT job_id FROM x WHERE job_type = 'LOGICAL REPLICATION' AND status = 'running' limit 1)")
if err != nil {
return false, err
}
jobsCur.Next()
var jobId string
_ = jobsCur.Scan(&jobId)
if jobId == "" {
return false, nil
}
default:
panic(fmt.Sprintf("unknown operation dependency %d", dep))
}
Expand Down

0 comments on commit 1d35377

Please sign in to comment.