From 205e71ef12866e144dae757ea0c748dcdc521467 Mon Sep 17 00:00:00 2001 From: Rafi Shamim Date: Fri, 22 Oct 2021 13:26:32 -0400 Subject: [PATCH] lint: add new errwrap linter This linter checks if we don't correctly wrap errors. It checks two cases: 1. if err.Error() is used as an argument to a function that is meant to wrap an error. 2. if an error is passed as a string format parameter with a format verb of %s or %v. The /* nolint:errwrap */ comment can be used to disable the linter inline. Release note: None --- BUILD.bazel | 1 + build/bazelutil/nogo_config.json | 9 + pkg/BUILD.bazel | 1 + pkg/cmd/roachvet/BUILD.bazel | 1 + pkg/cmd/roachvet/main.go | 2 + pkg/testutils/lint/passes/errwrap/BUILD.bazel | 29 +++ pkg/testutils/lint/passes/errwrap/errwrap.go | 240 ++++++++++++++++++ .../lint/passes/errwrap/errwrap_test.go | 39 +++ .../lint/passes/errwrap/functions.go | 92 +++++++ .../lint/passes/errwrap/testdata/src/a/a.go | 65 +++++ .../passes/errwrap/testdata/src/a/a_test.go | 79 ++++++ .../passes/errwrap/testdata/src/a/embedded.go | 25 ++ .../cockroach/pkg/sql/pgwire/pgcode/pgcode.go | 15 ++ .../pkg/sql/pgwire/pgerror/errors.go | 45 ++++ .../github.com/cockroachdb/errors/errors.go | 57 +++++ pkg/testutils/lint/passes/fmtsafe/BUILD.bazel | 1 + .../lint/passes/fmtsafe/functions.go | 87 +------ 17 files changed, 711 insertions(+), 77 deletions(-) create mode 100644 pkg/testutils/lint/passes/errwrap/BUILD.bazel create mode 100644 pkg/testutils/lint/passes/errwrap/errwrap.go create mode 100644 pkg/testutils/lint/passes/errwrap/errwrap_test.go create mode 100644 pkg/testutils/lint/passes/errwrap/functions.go create mode 100644 pkg/testutils/lint/passes/errwrap/testdata/src/a/a.go create mode 100644 pkg/testutils/lint/passes/errwrap/testdata/src/a/a_test.go create mode 100644 pkg/testutils/lint/passes/errwrap/testdata/src/a/embedded.go create mode 100644 pkg/testutils/lint/passes/errwrap/testdata/src/github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode/pgcode.go create mode 100644 pkg/testutils/lint/passes/errwrap/testdata/src/github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror/errors.go create mode 100644 pkg/testutils/lint/passes/errwrap/testdata/src/github.com/cockroachdb/errors/errors.go diff --git a/BUILD.bazel b/BUILD.bazel index d2d4b590ed6c..fb998c376ad2 100644 --- a/BUILD.bazel +++ b/BUILD.bazel @@ -240,6 +240,7 @@ nogo( "//pkg/testutils/lint/passes/descriptormarshal", "//pkg/testutils/lint/passes/errcheck", "//pkg/testutils/lint/passes/errcmp", + "//pkg/testutils/lint/passes/errwrap", "//pkg/testutils/lint/passes/fmtsafe", "//pkg/testutils/lint/passes/grpcclientconnclose", "//pkg/testutils/lint/passes/grpcstatuswithdetails", diff --git a/build/bazelutil/nogo_config.json b/build/bazelutil/nogo_config.json index d9391744eaca..d3d20be48288 100644 --- a/build/bazelutil/nogo_config.json +++ b/build/bazelutil/nogo_config.json @@ -56,6 +56,15 @@ "github.com/cockroachdb/cockroach/.*$": "first-party code" } }, + "errwrap": { + "exclude_files": { + ".*\\.pb\\.go$": "generated code", + ".*\\.pb\\.gw\\.go$": "generated code" + }, + "only_files": { + "github.com/cockroachdb/cockroach/.*$": "first-party code" + } + }, "fmtsafe": { "exclude_files": { "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/logger/log\\.go$": "format argument is not a constant expression", diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index fb184e62743f..4d12616bee54 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -327,6 +327,7 @@ ALL_TESTS = [ "//pkg/storage:storage_test", "//pkg/testutils/floatcmp:floatcmp_test", "//pkg/testutils/keysutils:keysutils_test", + "//pkg/testutils/lint/passes/errwrap:errwrap_test", "//pkg/testutils/lint/passes/fmtsafe:fmtsafe_test", "//pkg/testutils/lint/passes/forbiddenmethod:forbiddenmethod_test", "//pkg/testutils/lint/passes/hash:hash_test", diff --git a/pkg/cmd/roachvet/BUILD.bazel b/pkg/cmd/roachvet/BUILD.bazel index 8936679e1f70..f9b2ca205862 100644 --- a/pkg/cmd/roachvet/BUILD.bazel +++ b/pkg/cmd/roachvet/BUILD.bazel @@ -7,6 +7,7 @@ go_library( visibility = ["//visibility:private"], deps = [ "//pkg/testutils/lint/passes/errcmp", + "//pkg/testutils/lint/passes/errwrap", "//pkg/testutils/lint/passes/fmtsafe", "//pkg/testutils/lint/passes/forbiddenmethod", "//pkg/testutils/lint/passes/hash", diff --git a/pkg/cmd/roachvet/main.go b/pkg/cmd/roachvet/main.go index 9d3257b82808..40fb3e5010d9 100644 --- a/pkg/cmd/roachvet/main.go +++ b/pkg/cmd/roachvet/main.go @@ -15,6 +15,7 @@ package main import ( "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/errcmp" + "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/errwrap" "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/fmtsafe" "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/forbiddenmethod" "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/hash" @@ -65,6 +66,7 @@ func main() { fmtsafe.Analyzer, errcmp.Analyzer, nilness.Analyzer, + errwrap.Analyzer, ) // Standard go vet analyzers: diff --git a/pkg/testutils/lint/passes/errwrap/BUILD.bazel b/pkg/testutils/lint/passes/errwrap/BUILD.bazel new file mode 100644 index 000000000000..9157e779f9cb --- /dev/null +++ b/pkg/testutils/lint/passes/errwrap/BUILD.bazel @@ -0,0 +1,29 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") + +go_library( + name = "errwrap", + srcs = [ + "errwrap.go", + "functions.go", + ], + importpath = "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/errwrap", + visibility = ["//visibility:public"], + deps = [ + "@org_golang_x_tools//go/analysis", + "@org_golang_x_tools//go/analysis/passes/inspect", + "@org_golang_x_tools//go/ast/inspector", + ], +) + +go_test( + name = "errwrap_test", + srcs = ["errwrap_test.go"], + data = glob(["testdata/**"]), + deps = [ + ":errwrap", + "//pkg/build/bazel", + "//pkg/testutils", + "//pkg/testutils/skip", + "@org_golang_x_tools//go/analysis/analysistest", + ], +) diff --git a/pkg/testutils/lint/passes/errwrap/errwrap.go b/pkg/testutils/lint/passes/errwrap/errwrap.go new file mode 100644 index 000000000000..d7f48e4e9aef --- /dev/null +++ b/pkg/testutils/lint/passes/errwrap/errwrap.go @@ -0,0 +1,240 @@ +// Copyright 2021 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 errwrap + +import ( + "fmt" + "go/ast" + "go/constant" + "go/token" + "go/types" + "regexp" + "strings" + + "golang.org/x/tools/go/analysis" + "golang.org/x/tools/go/analysis/passes/inspect" + "golang.org/x/tools/go/ast/inspector" +) + +const Doc = `checks for unwrapped errors. + +This linter checks that: + +- err.Error() is not passed as an argument to an error-creating + function. + +- the '%s', '%v', and '%+v' format verbs are not used to format + errors when creating a new error. + +In both cases, an error-wrapping function can be used to correctly +preserve the chain of errors so that user-directed hints, links to +documentation issues, and telemetry data are all propagated. + +It is possible for a call site *in a test file* to opt the format/message +string out of the linter using /* nolint:errwrap */ after the format +argument. This escape hatch is not available in non-test code.` + +var errorType = types.Universe.Lookup("error").Type().String() + +// Analyzer checks for improperly wrapped errors. +var Analyzer = &analysis.Analyzer{ + Name: "errwrap", + Doc: Doc, + Requires: []*analysis.Analyzer{inspect.Analyzer}, + Run: run, +} + +func run(pass *analysis.Pass) (interface{}, error) { + inspctr := pass.ResultOf[inspect.Analyzer].(*inspector.Inspector) + nodeFilter := []ast.Node{ + (*ast.CallExpr)(nil), + } + + inspctr.Preorder(nodeFilter, func(n ast.Node) { + // Catch-all for possible bugs in the linter code. + defer func() { + if r := recover(); r != nil { + if err, ok := r.(error); ok { + pass.Reportf(n.Pos(), "internal linter error: %v", err) + return + } + panic(r) + } + }() + + callExpr, ok := n.(*ast.CallExpr) + if !ok { + return + } + if pass.TypesInfo.TypeOf(callExpr).String() != errorType { + return + } + sel, ok := callExpr.Fun.(*ast.SelectorExpr) + if !ok { + return + } + obj, ok := pass.TypesInfo.Uses[sel.Sel] + if !ok { + return + } + fn, ok := obj.(*types.Func) + if !ok { + return + } + pkg := obj.Pkg() + if pkg == nil { + return + } + + // Skip files generated by go-bindata. + file := pass.Fset.File(n.Pos()) + if strings.HasSuffix(file.Name(), "/embedded.go") { + return + } + fnName := stripVendor(fn.FullName()) + + // Check that none of the arguments are err.Error() + if _, ok := ErrorFnFormatStringIndex[fnName]; ok { + for i := range callExpr.Args { + if isErrorStringCall(pass, callExpr.Args[i]) { + pass.Report(analysis.Diagnostic{ + Pos: n.Pos(), + Message: fmt.Sprintf( + "err.Error() is passed to %s.%s; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/"+ + "errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead", + pkg.Name(), fn.Name()), + }) + } + } + } + + // Check that the format string does not use %s or %v for an error. + formatStringIdx, ok := ErrorFnFormatStringIndex[fnName] + if !ok || formatStringIdx < 0 { + // Not an error formatting function. + return + } + + // Find all % fields in the format string. + formatVerbs, ok := getFormatStringVerbs(pass, callExpr, formatStringIdx) + if !ok { + return + } + + // For any arguments that are errors, check whether the wrapping verb + // is %s or %v. + args := callExpr.Args[formatStringIdx+1:] + for i := 0; i < len(args) && i < len(formatVerbs); i++ { + if pass.TypesInfo.TypeOf(args[i]).String() != errorType { + continue + } + + if formatVerbs[i] == "%v" || formatVerbs[i] == "%+v" || formatVerbs[i] == "%s" { + // If the argument is opting out of the linter with a special + // comment, tolerate that. + if hasNoLintComment(pass, callExpr, formatStringIdx+1+i) { + continue + } + + pass.Report(analysis.Diagnostic{ + Pos: n.Pos(), + Message: fmt.Sprintf( + "non-wrapped error is passed to %s.%s; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/"+ + "errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead", + pkg.Name(), fn.Name(), + ), + }) + } + } + }) + + return nil, nil +} + +// isErrorStringCall tests whether the expression is a string expression that +// is the result of an `(error).Error()` method call. +func isErrorStringCall(pass *analysis.Pass, expr ast.Expr) bool { + if call, ok := expr.(*ast.CallExpr); ok { + if pass.TypesInfo.TypeOf(call).String() == "string" { + if callSel, ok := call.Fun.(*ast.SelectorExpr); ok { + fun := pass.TypesInfo.Uses[callSel.Sel].(*types.Func) + return fun.Type().String() == "func() string" && fun.Name() == "Error" + } + } + } + return false +} + +// formatVerbRegexp naively matches format string verbs. This does not take +// modifiers such as padding into account. +var formatVerbRegexp = regexp.MustCompile(`%([^%+]|\+v)`) + +// getFormatStringVerbs return an array of all `%` format verbs from the format +// string argument of a function call. +// Based on https://github.com/polyfloyd/go-errorlint/blob/e4f368f0ae6983eb40821ba4f88dc84ac51aef5b/errorlint/lint.go#L88 +func getFormatStringVerbs( + pass *analysis.Pass, call *ast.CallExpr, formatStringIdx int, +) ([]string, bool) { + if len(call.Args) <= formatStringIdx { + return nil, false + } + strLit, ok := call.Args[formatStringIdx].(*ast.BasicLit) + if !ok { + // Ignore format strings that are not literals. + return nil, false + } + formatString := constant.StringVal(pass.TypesInfo.Types[strLit].Value) + + return formatVerbRegexp.FindAllString(formatString, -1), true +} + +func hasNoLintComment(pass *analysis.Pass, call *ast.CallExpr, idx int) bool { + fPos, f := findContainingFile(pass, call) + + if !strings.HasSuffix(fPos.Name(), "_test.go") { + // The nolint: escape hatch is only supported in test files. + return false + } + + startPos := call.Args[idx].End() + endPos := call.Rparen + if idx < len(call.Args)-1 { + endPos = call.Args[idx+1].Pos() + } + for _, cg := range f.Comments { + if cg.Pos() > endPos || cg.End() < startPos { + continue + } + for _, c := range cg.List { + if strings.Contains(c.Text, "nolint:errwrap") { + return true + } + } + } + return false +} + +func findContainingFile(pass *analysis.Pass, n ast.Node) (*token.File, *ast.File) { + fPos := pass.Fset.File(n.Pos()) + for _, f := range pass.Files { + if pass.Fset.File(f.Pos()) == fPos { + return fPos, f + } + } + panic(fmt.Errorf("cannot file file for %v", n)) +} + +func stripVendor(s string) string { + if i := strings.Index(s, "/vendor/"); i != -1 { + s = s[i+len("/vendor/"):] + } + return s +} diff --git a/pkg/testutils/lint/passes/errwrap/errwrap_test.go b/pkg/testutils/lint/passes/errwrap/errwrap_test.go new file mode 100644 index 000000000000..53a2a207bf6f --- /dev/null +++ b/pkg/testutils/lint/passes/errwrap/errwrap_test.go @@ -0,0 +1,39 @@ +// Copyright 2021 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 errwrap_test + +import ( + "testing" + + "github.com/cockroachdb/cockroach/pkg/build/bazel" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/errwrap" + "github.com/cockroachdb/cockroach/pkg/testutils/skip" + "golang.org/x/tools/go/analysis/analysistest" +) + +func init() { + if bazel.BuiltWithBazel() { + bazel.SetGoEnv() + } +} + +func Test(t *testing.T) { + skip.UnderStress(t) + testdata := testutils.TestDataPath(t) + analysistest.TestData = func() string { return testdata } + results := analysistest.Run(t, testdata, errwrap.Analyzer, "a") + for _, r := range results { + for _, d := range r.Diagnostics { + t.Logf("%s: %v: %s", r.Pass.Analyzer.Name, d.Pos, d.Message) + } + } +} diff --git a/pkg/testutils/lint/passes/errwrap/functions.go b/pkg/testutils/lint/passes/errwrap/functions.go new file mode 100644 index 000000000000..8c41edf62b2a --- /dev/null +++ b/pkg/testutils/lint/passes/errwrap/functions.go @@ -0,0 +1,92 @@ +// Copyright 2021 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 errwrap + +// ErrorFnFormatStringIndex contains functions that should be checked for improperly +// wrapped errors. The value is the index of the function parameter containing +// the format string. It is -1 if there is no format string parameter. +var ErrorFnFormatStringIndex = map[string]int{ + "errors.New": -1, + + "github.com/pkg/errors.New": -1, + "github.com/pkg/errors.Wrap": -1, + + "github.com/cockroachdb/errors.New": -1, + "github.com/cockroachdb/errors.Error": -1, + "github.com/cockroachdb/errors.NewWithDepth": -1, + "github.com/cockroachdb/errors.WithMessage": -1, + "github.com/cockroachdb/errors.Wrap": -1, + "github.com/cockroachdb/errors.WrapWithDepth": -1, + "github.com/cockroachdb/errors.AssertionFailed": -1, + "github.com/cockroachdb/errors.HandledWithMessage": -1, + "github.com/cockroachdb/errors.HandledInDomainWithMessage": -1, + + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror.New": -1, + + "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented.New": -1, + "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented.NewWithIssue": -1, + "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented.NewWithIssueDetail": -1, + + "github.com/cockroachdb/cockroach/pkg/sql/pgwire.newAdminShutdownErr": -1, + + "(*github.com/cockroachdb/cockroach/pkg/parser/lexer).Error": -1, + + "fmt.Errorf": 0, + + "github.com/pkg/errors.Errorf": 0, + "github.com/pkg/errors.Wrapf": 1, + + "github.com/cockroachdb/errors.Newf": 0, + "github.com/cockroachdb/errors.Errorf": 0, + "github.com/cockroachdb/errors.NewWithDepthf": 1, + "github.com/cockroachdb/errors.WithMessagef": 1, + "github.com/cockroachdb/errors.Wrapf": 1, + "github.com/cockroachdb/errors.WrapWithDepthf": 2, + "github.com/cockroachdb/errors.AssertionFailedf": 0, + "github.com/cockroachdb/errors.AssertionFailedWithDepthf": 1, + "github.com/cockroachdb/errors.NewAssertionErrorWithWrappedErrf": 1, + "github.com/cockroachdb/errors.WithSafeDetails": 1, + + "github.com/cockroachdb/cockroach/pkg/roachpb.NewErrorf": 0, + + "github.com/cockroachdb/cockroach/pkg/ccl/importccl.makeRowErr": 3, + "github.com/cockroachdb/cockroach/pkg/ccl/importccl.wrapRowErr": 4, + + "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors.NewSyntaxErrorf": 0, + "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors.NewDependentObjectErrorf": 0, + + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree.decorateTypeCheckError": 1, + + "github.com/cockroachdb/cockroach/pkg/sql/opt/optbuilder.unimplementedWithIssueDetailf": 2, + + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror.Newf": 1, + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror.NewWithDepthf": 2, + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror.DangerousStatementf": 0, + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror.Wrapf": 2, + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror.WrapWithDepthf": 3, + + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice.Newf": 0, + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice.NewWithSeverityf": 1, + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgwirebase.NewProtocolViolationErrorf": 0, + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgwirebase.NewInvalidBinaryRepresentationErrorf": 0, + + "github.com/cockroachdb/cockroach/pkg/util/errorutil.UnexpectedWithIssueErrorf": 1, + + "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented.Newf": 1, + "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented.NewWithDepthf": 2, + "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented.NewWithIssuef": 1, + "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented.NewWithIssueDetailf": 2, + "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented.unimplementedInternal": 3, + + "github.com/cockroachdb/cockroach/pkg/util/timeutil/pgdate.inputErrorf": 0, + + "github.com/cockroachdb/cockroach/pkg/ccl/sqlproxyccl.newErrorf": 1, +} diff --git a/pkg/testutils/lint/passes/errwrap/testdata/src/a/a.go b/pkg/testutils/lint/passes/errwrap/testdata/src/a/a.go new file mode 100644 index 000000000000..8a27bb38da52 --- /dev/null +++ b/pkg/testutils/lint/passes/errwrap/testdata/src/a/a.go @@ -0,0 +1,65 @@ +// Copyright 2020 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 a + +import ( + "fmt" + + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "github.com/cockroachdb/errors" +) + +var wrappedErr = fmt.Errorf("abc") +var anotherErr = fmt.Errorf("def") + +func init() { + _ = recover() + + _ = fmt.Errorf(wrappedErr.Error()) // want `err.Error\(\) is passed to fmt.Errorf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + _ = fmt.Errorf("format %s", wrappedErr.Error()) // want `err.Error\(\) is passed to fmt.Errorf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + + s := wrappedErr.Error() + _ = fmt.Errorf("format %s", s) // this way is allowed + + _ = pgerror.Wrap(anotherErr, pgcode.Warning, wrappedErr.Error()) // want `err.Error\(\) is passed to pgerror.Wrap; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + _ = pgerror.Wrapf(anotherErr, pgcode.Warning, "format %s", wrappedErr.Error()) // want `err.Error\(\) is passed to pgerror.Wrapf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + _ = pgerror.WrapWithDepthf(1, anotherErr, pgcode.Warning, "format %s", wrappedErr.Error()) // want `err.Error\(\) is passed to pgerror.WrapWithDepthf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + _ = pgerror.New(pgcode.Warning, wrappedErr.Error()) // want `err.Error\(\) is passed to pgerror.New; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + _ = pgerror.Newf(pgcode.Warning, "format %s", wrappedErr.Error()) // want `err.Error\(\) is passed to pgerror.Newf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + + _ = errors.Wrap(anotherErr, wrappedErr.Error()) // want `err.Error\(\) is passed to errors.Wrap; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + _ = errors.Wrapf(anotherErr, "format %s", wrappedErr.Error()) // want `err.Error\(\) is passed to errors.Wrapf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + _ = errors.WrapWithDepthf(1, anotherErr, "format %s", wrappedErr.Error()) // want `err.Error\(\) is passed to errors.WrapWithDepthf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + _ = errors.New(wrappedErr.Error()) // want `err.Error\(\) is passed to errors.New; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + _ = errors.Newf("format %d %s", 1, wrappedErr.Error()) // want `err.Error\(\) is passed to errors.Newf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + _ = errors.NewWithDepthf(1, "format %s", wrappedErr.Error()) // want `err.Error\(\) is passed to errors.NewWithDepthf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + _ = errors.AssertionFailedf(wrappedErr.Error()) // want `err.Error\(\) is passed to errors.AssertionFailedf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + _ = errors.AssertionFailedWithDepthf(1, "format %s", wrappedErr.Error()) // want `err.Error\(\) is passed to errors.AssertionFailedWithDepthf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + _ = errors.NewAssertionErrorWithWrappedErrf(anotherErr, "format %s", wrappedErr.Error()) // want `err.Error\(\) is passed to errors.NewAssertionErrorWithWrappedErrf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + + _ = fmt.Errorf("got %s", wrappedErr) // want `non-wrapped error is passed to fmt.Errorf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + _ = fmt.Errorf("got %v", wrappedErr) // want `non-wrapped error is passed to fmt.Errorf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + _ = fmt.Errorf("got %+v", wrappedErr) // want `non-wrapped error is passed to fmt.Errorf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + _ = fmt.Errorf("got %w", wrappedErr) // this is allowed because of the %w verb` + + _ = pgerror.Wrapf(anotherErr, pgcode.Warning, "format %s", wrappedErr) // want `non-wrapped error is passed to pgerror.Wrapf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + _ = pgerror.WrapWithDepthf(1, anotherErr, pgcode.Warning, "format %s", wrappedErr) // want `non-wrapped error is passed to pgerror.WrapWithDepthf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + _ = pgerror.Newf(pgcode.Warning, "format %s", wrappedErr) // want `non-wrapped error is passed to pgerror.Newf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + + _ = errors.Wrapf(anotherErr, "format %v", wrappedErr) // want `non-wrapped error is passed to errors.Wrapf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + _ = errors.WrapWithDepthf(1, anotherErr, "format %+v", wrappedErr) // want `non-wrapped error is passed to errors.WrapWithDepthf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + _ = errors.Newf("format %d %s", 1, wrappedErr) // want `non-wrapped error is passed to errors.Newf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + _ = errors.NewWithDepthf(1, "format %s", wrappedErr) // want `non-wrapped error is passed to errors.NewWithDepthf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + _ = errors.AssertionFailedf("format %v", wrappedErr) // want `non-wrapped error is passed to errors.AssertionFailedf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + _ = errors.AssertionFailedWithDepthf(1, "format %s", wrappedErr) // want `non-wrapped error is passed to errors.AssertionFailedWithDepthf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + _ = errors.NewAssertionErrorWithWrappedErrf(anotherErr, "format %v", wrappedErr) // want `non-wrapped error is passed to errors.NewAssertionErrorWithWrappedErrf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` +} diff --git a/pkg/testutils/lint/passes/errwrap/testdata/src/a/a_test.go b/pkg/testutils/lint/passes/errwrap/testdata/src/a/a_test.go new file mode 100644 index 000000000000..b7d9bb0be3ec --- /dev/null +++ b/pkg/testutils/lint/passes/errwrap/testdata/src/a/a_test.go @@ -0,0 +1,79 @@ +// Copyright 2021 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 a + +import ( + "fmt" + + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "github.com/cockroachdb/errors" +) + +func init() { + _ = recover() + + _ = fmt.Errorf(wrappedErr.Error()) // want `err.Error\(\) is passed to fmt.Errorf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + _ = fmt.Errorf("format %s", wrappedErr.Error()) // want `err.Error\(\) is passed to fmt.Errorf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + + s := wrappedErr.Error() + _ = fmt.Errorf("format %s", s) // this way is allowed + + _ = pgerror.Wrap(anotherErr, pgcode.Warning, wrappedErr.Error()) // want `err.Error\(\) is passed to pgerror.Wrap; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + _ = pgerror.Wrapf(anotherErr, pgcode.Warning, "format %s", wrappedErr.Error()) // want `err.Error\(\) is passed to pgerror.Wrapf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + _ = pgerror.WrapWithDepthf(1, anotherErr, pgcode.Warning, "format %s", wrappedErr.Error()) // want `err.Error\(\) is passed to pgerror.WrapWithDepthf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + _ = pgerror.New(pgcode.Warning, wrappedErr.Error()) // want `err.Error\(\) is passed to pgerror.New; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + _ = pgerror.Newf(pgcode.Warning, "format %s", wrappedErr.Error()) // want `err.Error\(\) is passed to pgerror.Newf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + + _ = errors.Wrap(anotherErr, wrappedErr.Error()) // want `err.Error\(\) is passed to errors.Wrap; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + _ = errors.Wrapf(anotherErr, "format %s", wrappedErr.Error()) // want `err.Error\(\) is passed to errors.Wrapf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + _ = errors.WrapWithDepthf(1, anotherErr, "format %s", wrappedErr.Error()) // want `err.Error\(\) is passed to errors.WrapWithDepthf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + _ = errors.New(wrappedErr.Error()) // want `err.Error\(\) is passed to errors.New; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + _ = errors.Newf("format %d %s", 1, wrappedErr.Error()) // want `err.Error\(\) is passed to errors.Newf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + _ = errors.NewWithDepthf(1, "format %s", wrappedErr.Error()) // want `err.Error\(\) is passed to errors.NewWithDepthf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + _ = errors.AssertionFailedf(wrappedErr.Error()) // want `err.Error\(\) is passed to errors.AssertionFailedf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + _ = errors.AssertionFailedWithDepthf(1, "format %s", wrappedErr.Error()) // want `err.Error\(\) is passed to errors.AssertionFailedWithDepthf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + _ = errors.NewAssertionErrorWithWrappedErrf(anotherErr, "format %s", wrappedErr.Error()) // want `err.Error\(\) is passed to errors.NewAssertionErrorWithWrappedErrf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + + _ = fmt.Errorf("got %s", wrappedErr) // want `non-wrapped error is passed to fmt.Errorf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + _ = fmt.Errorf("got %v", wrappedErr) // want `non-wrapped error is passed to fmt.Errorf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + _ = fmt.Errorf("got %+v", wrappedErr) // want `non-wrapped error is passed to fmt.Errorf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + _ = fmt.Errorf("got %w", wrappedErr) // this is allowed because of the %w verb` + + _ = pgerror.Wrapf(anotherErr, pgcode.Warning, "format %s", wrappedErr) // want `non-wrapped error is passed to pgerror.Wrapf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + _ = pgerror.WrapWithDepthf(1, anotherErr, pgcode.Warning, "format %s", wrappedErr) // want `non-wrapped error is passed to pgerror.WrapWithDepthf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + _ = pgerror.Newf(pgcode.Warning, "format %s", wrappedErr) // want `non-wrapped error is passed to pgerror.Newf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + + _ = errors.Wrapf(anotherErr, "format %v", wrappedErr) // want `non-wrapped error is passed to errors.Wrapf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + _ = errors.WrapWithDepthf(1, anotherErr, "format %+v", wrappedErr) // want `non-wrapped error is passed to errors.WrapWithDepthf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + _ = errors.Newf("format %d %s", 1, wrappedErr) // want `non-wrapped error is passed to errors.Newf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + _ = errors.NewWithDepthf(1, "format %s", wrappedErr) // want `non-wrapped error is passed to errors.NewWithDepthf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + _ = errors.AssertionFailedf("format %v", wrappedErr) // want `non-wrapped error is passed to errors.AssertionFailedf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + _ = errors.AssertionFailedWithDepthf(1, "format %s", wrappedErr) // want `non-wrapped error is passed to errors.AssertionFailedWithDepthf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + _ = errors.NewAssertionErrorWithWrappedErrf(anotherErr, "format %v", wrappedErr) // want `non-wrapped error is passed to errors.NewAssertionErrorWithWrappedErrf; use pgerror.Wrap/errors.Wrap/errors.CombineErrors/errors.WithSecondaryError/errors.NewAssertionErrorWithWrappedErrf instead` + + _ = fmt.Errorf("got %s", wrappedErr /* nolint:errwrap */) // linting is skipped. + _ = fmt.Errorf("got %v", wrappedErr /* nolint:errwrap */) // linting is skipped. + _ = fmt.Errorf("got %+v", wrappedErr /* nolint:errwrap */) // linting is skipped. + + _ = pgerror.Wrapf(anotherErr, pgcode.Warning, "format %s", wrappedErr /* nolint:errwrap */) // linting is skipped. + _ = pgerror.WrapWithDepthf(1, anotherErr, pgcode.Warning, "format %s", wrappedErr /* nolint:errwrap */) // linting is skipped. + _ = pgerror.Newf(pgcode.Warning, "format %s", wrappedErr /* nolint:errwrap */) // linting is skipped. + + _ = errors.Wrapf(anotherErr, "format %v", wrappedErr /* nolint:errwrap */) // linting is skipped. + _ = errors.WrapWithDepthf(1, anotherErr, "format %+v", wrappedErr /* nolint:errwrap */) // linting is skipped. + _ = errors.Newf("format %d %s", 1, wrappedErr /* nolint:errwrap */) // linting is skipped. + _ = errors.NewWithDepthf(1, "format %s", wrappedErr /* nolint:errwrap */) // linting is skipped. + _ = errors.AssertionFailedf("format %v", wrappedErr /* nolint:errwrap */) // linting is skipped. + _ = errors.AssertionFailedWithDepthf(1, "format %s", wrappedErr /* nolint:errwrap */) // linting is skipped. + _ = errors.NewAssertionErrorWithWrappedErrf(anotherErr, "format %v", wrappedErr /* nolint:errwrap */) // linting is skipped. + +} diff --git a/pkg/testutils/lint/passes/errwrap/testdata/src/a/embedded.go b/pkg/testutils/lint/passes/errwrap/testdata/src/a/embedded.go new file mode 100644 index 000000000000..1c6e77a26a72 --- /dev/null +++ b/pkg/testutils/lint/passes/errwrap/testdata/src/a/embedded.go @@ -0,0 +1,25 @@ +// Copyright 2021 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 a + +import "fmt" + +// In the main source code, embedded.go files are generated by go-bindata. + +func init() { + _ = recover() + + _ = fmt.Errorf(wrappedErr.Error()) // linting is skipped. + _ = fmt.Errorf("format %s", wrappedErr.Error()) // linting is skipped. + + _ = fmt.Errorf("got %s", wrappedErr) // linting is skipped. + _ = fmt.Errorf("got %v", wrappedErr) // linting is skipped. +} diff --git a/pkg/testutils/lint/passes/errwrap/testdata/src/github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode/pgcode.go b/pkg/testutils/lint/passes/errwrap/testdata/src/github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode/pgcode.go new file mode 100644 index 000000000000..2aaeabfe858c --- /dev/null +++ b/pkg/testutils/lint/passes/errwrap/testdata/src/github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode/pgcode.go @@ -0,0 +1,15 @@ +// Copyright 2021 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 pgcode + +type Code uint32 + +var Warning Code = 0 diff --git a/pkg/testutils/lint/passes/errwrap/testdata/src/github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror/errors.go b/pkg/testutils/lint/passes/errwrap/testdata/src/github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror/errors.go new file mode 100644 index 000000000000..972dd5d1347b --- /dev/null +++ b/pkg/testutils/lint/passes/errwrap/testdata/src/github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror/errors.go @@ -0,0 +1,45 @@ +// Copyright 2021 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 pgerror + +import ( + "fmt" + + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" +) + +func NewWithDepthf(depth int, code pgcode.Code, format string, args ...interface{}) error { + return fmt.Errorf(format, args) +} + +func New(code pgcode.Code, msg string) error { + return fmt.Errorf(msg) + +} + +func Newf(code pgcode.Code, format string, args ...interface{}) error { + return fmt.Errorf(format, args) +} + +func Wrapf(err error, code pgcode.Code, format string, args ...interface{}) error { + return fmt.Errorf(format, args) +} + +func WrapWithDepthf( + depth int, err error, code pgcode.Code, format string, args ...interface{}, +) error { + return fmt.Errorf(format, args) + +} + +func Wrap(err error, code pgcode.Code, msg string) error { + return fmt.Errorf(msg) +} diff --git a/pkg/testutils/lint/passes/errwrap/testdata/src/github.com/cockroachdb/errors/errors.go b/pkg/testutils/lint/passes/errwrap/testdata/src/github.com/cockroachdb/errors/errors.go new file mode 100644 index 000000000000..b9210fdc5886 --- /dev/null +++ b/pkg/testutils/lint/passes/errwrap/testdata/src/github.com/cockroachdb/errors/errors.go @@ -0,0 +1,57 @@ +// Copyright 2021 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 errors + +import "fmt" + +func New(msg string) error { + return fmt.Errorf(msg) +} + +func Newf(format string, args ...interface{}) error { + return fmt.Errorf(format, args...) +} + +func Wrap(_ error, msg string) error { + return fmt.Errorf(msg) +} + +func Wrapf(_ error, format string, args ...interface{}) error { + return fmt.Errorf(format, args...) +} + +func WrapWithDepth(depth int, err error, msg string) error { + return fmt.Errorf(msg) +} + +func WrapWithDepthf(depth int, err error, format string, args ...interface{}) error { + return fmt.Errorf(format, args...) +} + +func NewWithDepth(_ int, msg string) error { + return fmt.Errorf(msg) +} + +func NewWithDepthf(_ int, format string, args ...interface{}) error { + return fmt.Errorf(format, args...) +} + +func AssertionFailedf(format string, args ...interface{}) error { + return fmt.Errorf(format, args...) +} + +func AssertionFailedWithDepthf(_ int, format string, args ...interface{}) error { + return fmt.Errorf(format, args...) +} + +func NewAssertionErrorWithWrappedErrf(_ error, format string, args ...interface{}) error { + return fmt.Errorf(format, args...) +} diff --git a/pkg/testutils/lint/passes/fmtsafe/BUILD.bazel b/pkg/testutils/lint/passes/fmtsafe/BUILD.bazel index 6e633b128234..e364dd3e8fb8 100644 --- a/pkg/testutils/lint/passes/fmtsafe/BUILD.bazel +++ b/pkg/testutils/lint/passes/fmtsafe/BUILD.bazel @@ -9,6 +9,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/fmtsafe", visibility = ["//visibility:public"], deps = [ + "//pkg/testutils/lint/passes/errwrap", "//pkg/util/log/logpb", "@com_github_cockroachdb_errors//:errors", "@org_golang_x_tools//go/analysis", diff --git a/pkg/testutils/lint/passes/fmtsafe/functions.go b/pkg/testutils/lint/passes/fmtsafe/functions.go index 8dd98a64aa47..bebf337df550 100644 --- a/pkg/testutils/lint/passes/fmtsafe/functions.go +++ b/pkg/testutils/lint/passes/fmtsafe/functions.go @@ -13,37 +13,13 @@ package fmtsafe import ( "strings" + "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/errwrap" "github.com/cockroachdb/cockroach/pkg/util/log/logpb" ) // requireConstMsg records functions for which the last string // argument must be a constant string. var requireConstMsg = map[string]bool{ - "errors.New": true, - - "github.com/pkg/errors.New": true, - "github.com/pkg/errors.Wrap": true, - - "github.com/cockroachdb/errors.New": true, - "github.com/cockroachdb/errors.Error": true, - "github.com/cockroachdb/errors.NewWithDepth": true, - "github.com/cockroachdb/errors.WithMessage": true, - "github.com/cockroachdb/errors.Wrap": true, - "github.com/cockroachdb/errors.WrapWithDepth": true, - "github.com/cockroachdb/errors.AssertionFailed": true, - "github.com/cockroachdb/errors.HandledWithMessage": true, - "github.com/cockroachdb/errors.HandledInDomainWithMessage": true, - - "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror.New": true, - - "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented.New": true, - "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented.NewWithIssue": true, - "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented.NewWithIssueDetail": true, - - "github.com/cockroachdb/cockroach/pkg/sql/pgwire.newAdminShutdownErr": true, - - "(*github.com/cockroachdb/cockroach/pkg/parser/lexer).Error": true, - "github.com/cockroachdb/cockroach/pkg/util/log.Shout": true, "github.com/cockroachdb/cockroach/pkg/util/log.Event": true, "github.com/cockroachdb/cockroach/pkg/util/log.VEvent": true, @@ -144,66 +120,15 @@ var requireConstFmt = map[string]bool{ "(*github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl.kafkaLogAdapter).Printf": true, - // Error things. - "fmt.Errorf": true, - - "github.com/pkg/errors.Errorf": true, - "github.com/pkg/errors.Wrapf": true, - - "github.com/cockroachdb/errors.Newf": true, - "github.com/cockroachdb/errors.Errorf": true, - "github.com/cockroachdb/errors.NewWithDepthf": true, - "github.com/cockroachdb/errors.WithMessagef": true, - "github.com/cockroachdb/errors.Wrapf": true, - "github.com/cockroachdb/errors.WrapWithDepthf": true, - "github.com/cockroachdb/errors.AssertionFailedf": true, - "github.com/cockroachdb/errors.AssertionFailedWithDepthf": true, - "github.com/cockroachdb/errors.NewAssertionErrorWithWrappedErrf": true, - "github.com/cockroachdb/errors.WithSafeDetails": true, - "github.com/cockroachdb/redact.Sprintf": true, "github.com/cockroachdb/redact.Fprintf": true, "(github.com/cockroachdb/redact.SafePrinter).Printf": true, "(github.com/cockroachdb/redact.SafeWriter).Printf": true, "(*github.com/cockroachdb/redact.printer).Printf": true, - "github.com/cockroachdb/cockroach/pkg/roachpb.NewErrorf": true, - - "github.com/cockroachdb/cockroach/pkg/ccl/importccl.makeRowErr": true, - "github.com/cockroachdb/cockroach/pkg/ccl/importccl.wrapRowErr": true, - - "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors.NewSyntaxErrorf": true, - "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors.NewDependentObjectErrorf": true, - - "github.com/cockroachdb/cockroach/pkg/sql/sem/tree.newSourceNotFoundError": true, - "github.com/cockroachdb/cockroach/pkg/sql/sem/tree.decorateTypeCheckError": true, - - "github.com/cockroachdb/cockroach/pkg/sql/opt/optbuilder.unimplementedWithIssueDetailf": true, - "(*github.com/cockroachdb/cockroach/pkg/sql/pgwire.authPipe).Logf": true, - "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror.Newf": true, - "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror.NewWithDepthf": true, - "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror.DangerousStatementf": true, - "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror.Wrapf": true, - "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror.WrapWithDepthf": true, - - "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice.Newf": true, - "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice.NewWithSeverityf": true, - "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgwirebase.NewProtocolViolationErrorf": true, - "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgwirebase.NewInvalidBinaryRepresentationErrorf": true, - - "github.com/cockroachdb/cockroach/pkg/util/errorutil.UnexpectedWithIssueErrorf": true, - - "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented.Newf": true, - "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented.NewWithDepthf": true, - "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented.NewWithIssuef": true, - "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented.NewWithIssueDetailf": true, - "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented.unimplementedInternal": true, - - "github.com/cockroachdb/cockroach/pkg/util/timeutil/pgdate.inputErrorf": true, - - "github.com/cockroachdb/cockroach/pkg/ccl/sqlproxyccl.newErrorf": true, + // Error things are populated in the init() message. } func init() { @@ -237,4 +162,12 @@ func init() { // log.Ops.Shout, log.Dev.Shout, etc. requireConstMsg["(github.com/cockroachdb/cockroach/pkg/util/log.logger"+capch+").Shout"] = true } + + for errorFn, formatStringIndex := range errwrap.ErrorFnFormatStringIndex { + if formatStringIndex < 0 { + requireConstMsg[errorFn] = true + } else { + requireConstFmt[errorFn] = true + } + } }