Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
87383: roachtest: move CheckReplicaDivergenceOnDB to util r=renatolabs a=tbg

This had no business sitting on `Cluster`.

Release note: None
Release justification: testing code

Co-authored-by: Tobias Grieger <[email protected]>
  • Loading branch information
craig[bot] and tbg committed Sep 8, 2022
2 parents 01eb6ab + ecaf9c8 commit d33e93f
Show file tree
Hide file tree
Showing 8 changed files with 85 additions and 48 deletions.
2 changes: 2 additions & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -944,6 +944,7 @@ GO_TARGETS = [
"//pkg/cmd/roachtest/clusterstats:clusterstats_test",
"//pkg/cmd/roachtest/option:option",
"//pkg/cmd/roachtest/registry:registry",
"//pkg/cmd/roachtest/roachtestutil:roachtestutil",
"//pkg/cmd/roachtest/spec:spec",
"//pkg/cmd/roachtest/test:test",
"//pkg/cmd/roachtest/tests:tests",
Expand Down Expand Up @@ -2301,6 +2302,7 @@ GET_X_DATA_TARGETS = [
"//pkg/cmd/roachtest/clusterstats:get_x_data",
"//pkg/cmd/roachtest/option:get_x_data",
"//pkg/cmd/roachtest/registry:get_x_data",
"//pkg/cmd/roachtest/roachtestutil:get_x_data",
"//pkg/cmd/roachtest/spec:get_x_data",
"//pkg/cmd/roachtest/test:get_x_data",
"//pkg/cmd/roachtest/tests:get_x_data",
Expand Down
1 change: 1 addition & 0 deletions pkg/cmd/roachtest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ go_library(
"//pkg/cmd/roachtest/cluster",
"//pkg/cmd/roachtest/option",
"//pkg/cmd/roachtest/registry",
"//pkg/cmd/roachtest/roachtestutil",
"//pkg/cmd/roachtest/spec",
"//pkg/cmd/roachtest/test",
"//pkg/cmd/roachtest/tests",
Expand Down
48 changes: 2 additions & 46 deletions pkg/cmd/roachtest/cluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ import (
"github.com/armon/circbuf"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/cluster"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/option"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/spec"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test"
"github.com/cockroachdb/cockroach/pkg/roachprod"
Expand Down Expand Up @@ -1303,51 +1304,6 @@ func (c *clusterImpl) assertNoDeadNode(ctx context.Context, t test.Test) {
}
}

// CheckReplicaDivergenceOnDB runs a fast consistency check of the whole keyspace
// against the provided db. If an inconsistency is found, it returns it in the
// error. Note that this will swallow errors returned directly from the consistency
// check since we know that such spurious errors are possibly without any relation
// to the check having failed.
func (c *clusterImpl) CheckReplicaDivergenceOnDB(
ctx context.Context, l *logger.Logger, db *gosql.DB,
) error {
// NB: we set a statement_timeout since context cancellation won't work here,
// see:
// https://github.com/cockroachdb/cockroach/pull/34520
//
// We've seen the consistency checks hang indefinitely in some cases.
rows, err := db.QueryContext(ctx, `
SET statement_timeout = '5m';
SELECT t.range_id, t.start_key_pretty, t.status, t.detail
FROM
crdb_internal.check_consistency(true, '', '') as t
WHERE t.status NOT IN ('RANGE_CONSISTENT', 'RANGE_INDETERMINATE')`)
if err != nil {
// TODO(tbg): the checks can fail for silly reasons like missing gossiped
// descriptors, etc. -- not worth failing the test for. Ideally this would
// be rock solid.
l.Printf("consistency check failed with %v; ignoring", err)
return nil
}
defer rows.Close()
var finalErr error
for rows.Next() {
var rangeID int32
var prettyKey, status, detail string
if scanErr := rows.Scan(&rangeID, &prettyKey, &status, &detail); scanErr != nil {
l.Printf("consistency check failed with %v; ignoring", scanErr)
return nil
}
finalErr = errors.CombineErrors(finalErr,
errors.Newf("r%d (%s) is inconsistent: %s %s\n", rangeID, prettyKey, status, detail))
}
if err := rows.Err(); err != nil {
l.Printf("consistency check failed with %v; ignoring", err)
return nil
}
return finalErr
}

// FailOnReplicaDivergence fails the test if
// crdb_internal.check_consistency(true, ”, ”) indicates that any ranges'
// replicas are inconsistent with each other. It uses the first node that
Expand Down Expand Up @@ -1385,7 +1341,7 @@ func (c *clusterImpl) FailOnReplicaDivergence(ctx context.Context, t *testImpl)
if err := contextutil.RunWithTimeout(
ctx, "consistency check", 5*time.Minute,
func(ctx context.Context) error {
return c.CheckReplicaDivergenceOnDB(ctx, t.L(), db)
return roachtestutil.CheckReplicaDivergenceOnDB(ctx, t.L(), db)
},
); err != nil {
t.Errorf("consistency check failed: %v", err)
Expand Down
1 change: 0 additions & 1 deletion pkg/cmd/roachtest/cluster/cluster_interface.go
Original file line number Diff line number Diff line change
Expand Up @@ -125,7 +125,6 @@ type Cluster interface {
// These should be removed over time.

MakeNodes(opts ...option.Option) string
CheckReplicaDivergenceOnDB(context.Context, *logger.Logger, *gosql.DB) error
GitClone(
ctx context.Context, l *logger.Logger, src, dest, branch string, node option.NodeListOption,
) error
Expand Down
15 changes: 15 additions & 0 deletions pkg/cmd/roachtest/roachtestutil/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -0,0 +1,15 @@
load("//build/bazelutil/unused_checker:unused.bzl", "get_x_data")
load("@io_bazel_rules_go//go:def.bzl", "go_library")

go_library(
name = "roachtestutil",
srcs = ["consistency_check.go"],
importpath = "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil",
visibility = ["//visibility:public"],
deps = [
"//pkg/roachprod/logger",
"@com_github_cockroachdb_errors//:errors",
],
)

get_x_data(name = "get_x_data")
62 changes: 62 additions & 0 deletions pkg/cmd/roachtest/roachtestutil/consistency_check.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,62 @@
// Copyright 2022 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 roachtestutil

import (
"context"
gosql "database/sql"

"github.com/cockroachdb/cockroach/pkg/roachprod/logger"
"github.com/cockroachdb/errors"
)

// CheckReplicaDivergenceOnDB runs a stats-only consistency check via the
// provided DB. It ignores transient errors that can result from the
// implementation of crdb_internal.check_consistency, so a nil result
// does not prove anything.
func CheckReplicaDivergenceOnDB(ctx context.Context, l *logger.Logger, db *gosql.DB) error {
// NB: we set a statement_timeout since context cancellation won't work here,
// see:
// https://github.com/cockroachdb/cockroach/pull/34520
//
// We've seen the consistency checks hang indefinitely in some cases.
rows, err := db.QueryContext(ctx, `
SET statement_timeout = '5m';
SELECT t.range_id, t.start_key_pretty, t.status, t.detail
FROM
crdb_internal.check_consistency(true, '', '') as t
WHERE t.status NOT IN ('RANGE_CONSISTENT', 'RANGE_INDETERMINATE')`)
if err != nil {
// TODO(tbg): the checks can fail for silly reasons like missing gossiped
// descriptors, etc. -- not worth failing the test for. Ideally this would
// be rock solid.
l.Printf("consistency check failed with %v; ignoring", err)
return nil
}
defer rows.Close()
var finalErr error
for rows.Next() {
var rangeID int32
var prettyKey, status, detail string
if scanErr := rows.Scan(&rangeID, &prettyKey, &status, &detail); scanErr != nil {
l.Printf("consistency check failed with %v; ignoring", scanErr)
return nil
}
finalErr = errors.CombineErrors(finalErr,
errors.Newf("r%d (%s) is inconsistent: %s %s\n", rangeID, prettyKey, status, detail))
}
if err := rows.Err(); err != nil {
l.Printf("consistency check failed with %v; ignoring", err)
return nil
}
return finalErr
}
1 change: 1 addition & 0 deletions pkg/cmd/roachtest/tests/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -166,6 +166,7 @@ go_library(
"//pkg/cmd/roachtest/clusterstats",
"//pkg/cmd/roachtest/option",
"//pkg/cmd/roachtest/registry",
"//pkg/cmd/roachtest/roachtestutil",
"//pkg/cmd/roachtest/spec",
"//pkg/cmd/roachtest/test",
"//pkg/gossip",
Expand Down
3 changes: 2 additions & 1 deletion pkg/cmd/roachtest/tests/version.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/cluster"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/option"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/registry"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test"
"github.com/cockroachdb/cockroach/pkg/roachprod/install"
"github.com/cockroachdb/cockroach/pkg/util/version"
Expand Down Expand Up @@ -103,7 +104,7 @@ func registerVersion(r registry.Registry) {
//
// https://github.com/cockroachdb/cockroach/issues/37737#issuecomment-496026918
if !strings.HasPrefix(binaryVersion, "2.") {
if err := c.CheckReplicaDivergenceOnDB(ctx, t.L(), db); err != nil {
if err := roachtestutil.CheckReplicaDivergenceOnDB(ctx, t.L(), db); err != nil {
return errors.Wrapf(err, "node %d", i)
}
}
Expand Down

0 comments on commit d33e93f

Please sign in to comment.