diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index 99273d43f9f0..a2aacb5d34c4 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -133,6 +133,7 @@ ALL_TESTS = [ "//pkg/cmd/reduce/reduce:reduce_test", "//pkg/cmd/release:release_test", "//pkg/cmd/roachtest/clusterstats:clusterstats_test", + "//pkg/cmd/roachtest/registry:registry_test", "//pkg/cmd/roachtest/tests:tests_test", "//pkg/cmd/roachtest:roachtest_test", "//pkg/cmd/teamcity-trigger:teamcity-trigger_test", @@ -975,6 +976,7 @@ GO_TARGETS = [ "//pkg/cmd/roachtest/clusterstats:clusterstats_test", "//pkg/cmd/roachtest/option:option", "//pkg/cmd/roachtest/registry:registry", + "//pkg/cmd/roachtest/registry:registry_test", "//pkg/cmd/roachtest/roachtestutil:roachtestutil", "//pkg/cmd/roachtest/spec:spec", "//pkg/cmd/roachtest/test:test", diff --git a/pkg/cmd/roachtest/cluster.go b/pkg/cmd/roachtest/cluster.go index 339e0e813acd..e64cb832384b 100644 --- a/pkg/cmd/roachtest/cluster.go +++ b/pkg/cmd/roachtest/cluster.go @@ -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/registry" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/spec" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test" @@ -1371,7 +1372,12 @@ 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 roachtestutil.CheckReplicaDivergenceOnDB(ctx, t.L(), db) + err := roachtestutil.CheckReplicaDivergenceOnDB(ctx, t.L(), db, t.Spec().(*registry.TestSpec).FullConsistencyCheck) + if err != nil { + // Failed consistency checks are always KV's problem. + err = registry.WrapWithOwner(err, registry.OwnerKV) + } + return err }, ); err != nil { t.Errorf("consistency check failed: %v", err) diff --git a/pkg/cmd/roachtest/github.go b/pkg/cmd/roachtest/github.go index e9d019900f0a..b2719fd68491 100644 --- a/pkg/cmd/roachtest/github.go +++ b/pkg/cmd/roachtest/github.go @@ -17,6 +17,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/roachtestutil" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test" "github.com/cockroachdb/cockroach/pkg/internal/team" rperrors "github.com/cockroachdb/cockroach/pkg/roachprod/errors" @@ -33,14 +34,6 @@ type githubIssues struct { teamLoader func() (team.Map, error) } -type issueCategory int - -const ( - otherErr issueCategory = iota - clusterCreationErr - sshErr -) - func newGithubIssues( disable bool, c *clusterImpl, vmCreateOpts *vm.CreateOpts, l *logger.Logger, ) *githubIssues { @@ -68,26 +61,27 @@ func (g *githubIssues) shouldPost(t test.Test) bool { t.Spec().(*registry.TestSpec).Cluster.NodeCount > 0 } +type issueOverride struct { + owner registry.Owner + testName string + msgPrefix string +} + func (g *githubIssues) createPostRequest( - t test.Test, cat issueCategory, message string, + t test.Test, override issueOverride, message string, ) issues.PostRequest { var mention []string var projColID int issueOwner := t.Spec().(*registry.TestSpec).Owner + if o := override.owner; o != "" { + issueOwner = o + } issueName := t.Name() - - messagePrefix := "" - // Overrides to shield eng teams from potential flakes - if cat == clusterCreationErr { - issueOwner = registry.OwnerDevInf - issueName = "cluster_creation" - messagePrefix = fmt.Sprintf("test %s was skipped due to ", t.Name()) - } else if cat == sshErr { - issueOwner = registry.OwnerTestEng - issueName = "ssh_problem" - messagePrefix = fmt.Sprintf("test %s failed due to ", t.Name()) + if n := override.testName; n != "" { + issueName = n } + messagePrefix := override.msgPrefix teams, err := g.teamLoader() if err != nil { @@ -161,19 +155,36 @@ func (g *githubIssues) MaybePost(t *testImpl, message string) error { return nil } - cat := otherErr - + var o issueOverride // Overrides to shield eng teams from potential flakes firstFailure := t.firstFailure() if failureContainsError(firstFailure, errClusterProvisioningFailed) { - cat = clusterCreationErr + o.owner = registry.OwnerDevInf + o.testName = "cluster_creation" + o.msgPrefix = fmt.Sprintf("test %s was skipped due to ", t.Name()) } else if failureContainsError(firstFailure, rperrors.ErrSSH255) { - cat = sshErr + o.owner = registry.OwnerTestEng + o.testName = "ssh_problem" + o.msgPrefix = fmt.Sprintf("test %s failed due to ", t.Name()) + } else if failureContainsError(firstFailure, roachtestutil.ErrMarkConsistencyCheckFailed) { + o.owner = registry.OwnerKV + o.testName = "consistency_check" + o.msgPrefix = fmt.Sprintf("consistency check failed after running %s", t.Name()) + } else { + for _, err := range firstFailure.errors { + owner, ok := registry.OwnerFromErr(err) + if !ok { + continue + } + o.owner = owner + o.msgPrefix = "owner overridden by error" + break + } } return g.issuePoster( context.Background(), issues.UnitTestFormatter, - g.createPostRequest(t, cat, message), + g.createPostRequest(t, o, message), ) } diff --git a/pkg/cmd/roachtest/registry/BUILD.bazel b/pkg/cmd/roachtest/registry/BUILD.bazel index f3ab12d0bbc0..c6071caadca5 100644 --- a/pkg/cmd/roachtest/registry/BUILD.bazel +++ b/pkg/cmd/roachtest/registry/BUILD.bazel @@ -1,5 +1,5 @@ load("//build/bazelutil/unused_checker:unused.bzl", "get_x_data") -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "registry", @@ -17,6 +17,21 @@ go_library( "//pkg/cmd/roachtest/cluster", "//pkg/cmd/roachtest/spec", "//pkg/cmd/roachtest/test", + "@com_github_cockroachdb_errors//:errors", + "@com_github_cockroachdb_errors//errbase", + "@com_github_cockroachdb_errors//errorspb", + "@com_github_gogo_protobuf//proto", + ], +) + +go_test( + name = "registry_test", + srcs = ["owners_test.go"], + args = ["-test.timeout=295s"], + embed = [":registry"], + deps = [ + "@com_github_cockroachdb_errors//:errors", + "@com_github_stretchr_testify//require", ], ) diff --git a/pkg/cmd/roachtest/registry/owners.go b/pkg/cmd/roachtest/registry/owners.go index 2beba7b6e010..fa7871492690 100644 --- a/pkg/cmd/roachtest/registry/owners.go +++ b/pkg/cmd/roachtest/registry/owners.go @@ -10,6 +10,16 @@ package registry +import ( + "context" + "fmt" + + "github.com/cockroachdb/errors" + "github.com/cockroachdb/errors/errbase" + "github.com/cockroachdb/errors/errorspb" + "github.com/gogo/protobuf/proto" +) + // Owner is a valid entry for the Owners field of a roachtest. They should be // teams, not individuals. type Owner string @@ -31,3 +41,78 @@ const ( OwnerDevInf Owner = `dev-inf` OwnerMultiTenant Owner = `multi-tenant` ) + +func init() { + pKey := errors.GetTypeKey(&ownedError{}) + errors.RegisterWrapperEncoder(pKey, encodeOwnedError) + errors.RegisterWrapperDecoder(pKey, decodeOwnedError) +} + +func encodeOwnedError( + _ context.Context, err error, +) (msgPrefix string, safe []string, details proto.Message) { + var e *ownedError + errors.As(err, &e) + details = &errorspb.StringsPayload{ + Details: []string{string(e.owner)}, + } + return "", nil, details +} + +func decodeOwnedError( + ctx context.Context, cause error, msgPrefix string, safeDetails []string, payload proto.Message, +) error { + m, ok := payload.(*errorspb.StringsPayload) + if !ok || len(m.Details) != 1 { + return nil + } + owner := Owner(m.Details[0]) + return &ownedError{ + owner: owner, + wrapped: cause, + } +} + +type ownedError struct { + owner Owner + wrapped error +} + +// Unwrap implements errors.Wrapper. +func (e *ownedError) Unwrap() error { + return e.wrapped +} + +// Format implements fmt.Formatter. +func (e *ownedError) Format(f fmt.State, verb rune) { + errors.FormatError(e, f, verb) +} + +// FormatError implements errors.Formatter. +func (e *ownedError) FormatError(p errbase.Printer) (next error) { + p.Printf("owned by %s: %s", e.owner, e.wrapped) + return e.wrapped +} + +func (e *ownedError) Error() string { + return fmt.Sprint(e) +} + +// WrapWithOwner wraps the error with a hint at who should own the +// problem. The hint can later be retrieved via OwnerFromErr. +func WrapWithOwner(err error, owner Owner) error { + return &ownedError{ + owner: owner, + wrapped: err, + } +} + +// OwnerFromErr returns the Owner associated previously via WrapWithOwner, to the error, +// if any. +func OwnerFromErr(err error) (Owner, bool) { + var oe *ownedError + if errors.As(err, &oe) { + return oe.owner, true + } + return "", false +} diff --git a/pkg/cmd/roachtest/registry/owners_test.go b/pkg/cmd/roachtest/registry/owners_test.go new file mode 100644 index 000000000000..1564afd41b5e --- /dev/null +++ b/pkg/cmd/roachtest/registry/owners_test.go @@ -0,0 +1,43 @@ +// 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 registry + +import ( + "context" + "testing" + + "github.com/cockroachdb/errors" + "github.com/stretchr/testify/require" +) + +func TestOwnedError(t *testing.T) { + ctx := context.Background() + err := errors.DecodeError(ctx, errors.EncodeError( + ctx, WrapWithOwner(context.Canceled, OwnerTestEng), + )) + t.Log(err) + require.True(t, errors.HasType(err, (*ownedError)(nil))) + require.True(t, errors.Is(err, context.Canceled)) + { + owner, ok := OwnerFromErr(err) + require.True(t, ok) + require.Equal(t, OwnerTestEng, owner) + } + + // Last owner wins. + err = WrapWithOwner(err, OwnerKV) + { + owner, ok := OwnerFromErr(err) + require.True(t, ok) + require.Equal(t, OwnerKV, owner) + } +} diff --git a/pkg/cmd/roachtest/registry/test_spec.go b/pkg/cmd/roachtest/registry/test_spec.go index b5eceb6e6497..a91ea39a83e1 100644 --- a/pkg/cmd/roachtest/registry/test_spec.go +++ b/pkg/cmd/roachtest/registry/test_spec.go @@ -79,6 +79,11 @@ type TestSpec struct { // cannot be run with encryption enabled. EncryptionSupport EncryptionSupport + // When FullConsistencyCheck is set, after a successful run of the test + // roachtest will run a full KV consistency check and mark the test as + // failed if that check discovers any discrepancies. + FullConsistencyCheck bool + // Run is the test function. Run func(ctx context.Context, t test.Test, c cluster.Cluster) } diff --git a/pkg/cmd/roachtest/roachtestutil/consistency_check.go b/pkg/cmd/roachtest/roachtestutil/consistency_check.go index 529dbd0a9081..5bfd6f9ff44a 100644 --- a/pkg/cmd/roachtest/roachtestutil/consistency_check.go +++ b/pkg/cmd/roachtest/roachtestutil/consistency_check.go @@ -14,27 +14,37 @@ package roachtestutil import ( "context" gosql "database/sql" + "fmt" "github.com/cockroachdb/cockroach/pkg/roachprod/logger" "github.com/cockroachdb/errors" ) +// ErrMarkConsistencyCheckFailed marks errors that indicate a replica divergence +// problem detected in CheckReplicaDivergenceOnDB. +var ErrMarkConsistencyCheckFailed = errors.New("consistency check failed") + // 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 { +// provided DB. It ignores any errors running the checks and will only propagate +// an error resulting from an actual detected discrepancy. In other words, a nil +// result does not prove anything. +func CheckReplicaDivergenceOnDB( + ctx context.Context, l *logger.Logger, db *gosql.DB, statsOnly bool, +) 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, ` + // + // NB: can't use prepared statements since there are two statements here, + // so Sprintf it is. + rows, err := db.QueryContext(ctx, fmt.Sprintf(` 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')`) +crdb_internal.check_consistency(%t, '', '') as t +WHERE t.status NOT IN ('RANGE_CONSISTENT', 'RANGE_INDETERMINATE')`, statsOnly)) 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 @@ -58,5 +68,9 @@ WHERE t.status NOT IN ('RANGE_CONSISTENT', 'RANGE_INDETERMINATE')`) l.Printf("consistency check failed with %v; ignoring", err) return nil } - return finalErr + + if finalErr != nil { + return errors.Mark(finalErr, ErrMarkConsistencyCheckFailed) + } + return nil } diff --git a/pkg/cmd/roachtest/test_test.go b/pkg/cmd/roachtest/test_test.go index 4a38e9033292..4dc1e1c3b8d8 100644 --- a/pkg/cmd/roachtest/test_test.go +++ b/pkg/cmd/roachtest/test_test.go @@ -119,6 +119,14 @@ func TestRunnerRun(t *testing.T) { }, Cluster: r.MakeClusterSpec(0), }) + r.Add(registry.TestSpec{ + Name: "fail-with-owner", + Owner: OwnerUnitTest, + Run: func(ctx context.Context, t test.Test, c cluster.Cluster) { + t.Fatalf("this is your fault: %s", registry.WrapWithOwner(errors.New("explody"), registry.OwnerTestEng)) + }, + Cluster: r.MakeClusterSpec(0), + }) r.Add(registry.TestSpec{ Name: "errors", Owner: OwnerUnitTest, @@ -154,6 +162,7 @@ func TestRunnerRun(t *testing.T) { {filters: []string{"notests"}, expErr: "no test"}, {filters: []string{"errors"}, expErr: "some tests failed", expOut: "second error"}, {filters: []string{"panic"}, expErr: "some tests failed", expOut: "index out of range"}, + {filters: []string{"fail-with-owner"}, expErr: "some tests failed", expOut: "ownership of failure redirected to: test-eng"}, } for _, c := range testCases { t.Run("", func(t *testing.T) { diff --git a/pkg/cmd/roachtest/tests/clearrange.go b/pkg/cmd/roachtest/tests/clearrange.go index e17c5feed671..78666215a801 100644 --- a/pkg/cmd/roachtest/tests/clearrange.go +++ b/pkg/cmd/roachtest/tests/clearrange.go @@ -36,7 +36,10 @@ func registerClearRange(r registry.Registry) { // 5h for import, 90 for the test. The import should take closer // to <3:30h but it varies. Timeout: 5*time.Hour + 90*time.Minute, - Cluster: r.MakeClusterSpec(10, spec.CPU(16)), + // This test uses MVCC range deletions, and this gives us more + // confidence that everything works ok. + FullConsistencyCheck: true, + Cluster: r.MakeClusterSpec(10, spec.CPU(16)), Run: func(ctx context.Context, t test.Test, c cluster.Cluster) { runClearRange(ctx, t, c, checks, rangeTombstones) }, diff --git a/pkg/cmd/roachtest/tests/schemachange.go b/pkg/cmd/roachtest/tests/schemachange.go index 3b2d747e4f92..55d7910a676c 100644 --- a/pkg/cmd/roachtest/tests/schemachange.go +++ b/pkg/cmd/roachtest/tests/schemachange.go @@ -427,6 +427,9 @@ func makeSchemaChangeDuringTPCC( Owner: registry.OwnerSQLSchema, Cluster: spec, Timeout: length * 3, + // This test runs a workload and drops tables, so it has good overall coverage + // including MVCC range deletions. + FullConsistencyCheck: true, Run: func(ctx context.Context, t test.Test, c cluster.Cluster) { runTPCC(ctx, t, c, tpccOptions{ Warehouses: warehouses, diff --git a/pkg/cmd/roachtest/tests/version.go b/pkg/cmd/roachtest/tests/version.go index 5ede042f76bf..fc292f24f948 100644 --- a/pkg/cmd/roachtest/tests/version.go +++ b/pkg/cmd/roachtest/tests/version.go @@ -14,7 +14,6 @@ import ( "context" "fmt" "runtime" - "strings" "time" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/cluster" @@ -98,16 +97,8 @@ func registerVersion(r registry.Registry) { if err := rows.Close(); err != nil { return err } - // Regression test for #37425. We can't run this in 2.1 because - // 19.1 changed downstream-of-raft semantics for consistency - // checks but unfortunately our versioning story for these - // checks had been broken for a long time. See: - // - // https://github.com/cockroachdb/cockroach/issues/37737#issuecomment-496026918 - if !strings.HasPrefix(binaryVersion, "2.") { - if err := roachtestutil.CheckReplicaDivergenceOnDB(ctx, t.L(), db); err != nil { - return errors.Wrapf(err, "node %d", i) - } + if err := roachtestutil.CheckReplicaDivergenceOnDB(ctx, t.L(), db, false /* statsOnly */); err != nil { + return errors.Wrapf(err, "node %d", i) } } return nil diff --git a/pkg/testutils/lint/lint_test.go b/pkg/testutils/lint/lint_test.go index 900ce4c359bc..14c27353a63d 100644 --- a/pkg/testutils/lint/lint_test.go +++ b/pkg/testutils/lint/lint_test.go @@ -1197,6 +1197,7 @@ func TestLint(t *testing.T) { "*.go", ":!*.pb.go", ":!*.pb.gw.go", + ":!cmd/roachtest/registry/owners.go", ":!kv/kvclient/kvcoord/lock_spans_over_budget_error.go", ":!spanconfig/errors.go", ":!roachpb/replica_unavailable_error.go",