Skip to content

Commit

Permalink
Merge #71877
Browse files Browse the repository at this point in the history
71877: lint: add new errwrap linter r=ajwerner,knz a=rafiss

fixes #42510

This linter checks if we don't correctly wrap errors.

The `/* nolint:errwrap */` comment can be used to disable the linter inline.

See individual commits for mistakes this linter caught.
It had already caught a few in #72353, #72352, #72351, #72350, and #72349.

Release note: None

Co-authored-by: Rafi Shamim <[email protected]>
  • Loading branch information
craig[bot] and rafiss committed Dec 15, 2021
2 parents ef789b1 + 118a446 commit 2c590e2
Show file tree
Hide file tree
Showing 106 changed files with 944 additions and 243 deletions.
1 change: 1 addition & 0 deletions BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -241,6 +241,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",
Expand Down
9 changes: 9 additions & 0 deletions build/bazelutil/nogo_config.json
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
1 change: 1 addition & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -329,6 +329,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",
Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/backupccl/backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1665,11 +1665,11 @@ func TestBackupRestoreCheckpointing(t *testing.T) {
checkpointPath = filepath.Join(ip.dir, ip.name, backupManifestCheckpointName)
checkpointDescBytes, err := ioutil.ReadFile(checkpointPath)
if err != nil {
return errors.Errorf("%+v", err)
return errors.Wrap(err, "error while reading checkpoint")
}
var checkpointDesc BackupManifest
if err := protoutil.Unmarshal(checkpointDescBytes, &checkpointDesc); err != nil {
return errors.Errorf("%+v", err)
return errors.Wrap(err, "error while unmarshalling checkpoint")
}
if len(checkpointDesc.Files) == 0 {
return errors.Errorf("empty backup checkpoint descriptor")
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/changefeedccl/sink_kafka.go
Original file line number Diff line number Diff line change
Expand Up @@ -581,7 +581,7 @@ func buildKafkaConfig(u sinkURL, opts map[string]string) (*sarama.Config, error)
if dialConfig.clientCert != nil && dialConfig.clientKey != nil {
cert, err := tls.X509KeyPair(dialConfig.clientCert, dialConfig.clientKey)
if err != nil {
return nil, errors.Errorf(`invalid client certificate data provided: %s`, err)
return nil, errors.Wrap(err, `invalid client certificate data provided`)
}
config.Net.TLS.Config.Certificates = []tls.Certificate{cert}
}
Expand Down
11 changes: 10 additions & 1 deletion pkg/ccl/importccl/read_import_mysql.go
Original file line number Diff line number Diff line change
Expand Up @@ -761,7 +761,16 @@ func mysqlColToCockroach(
exprString := mysql.String(col.Default)
expr, err := parser.ParseExpr(exprString)
if err != nil {
return nil, unimplemented.Newf("import.mysql.default", "unsupported default expression %q for column %q: %v", exprString, name, err)
// There is currently no way to wrap an error with an unimplemented
// error.
// nolint:errwrap
return nil, errors.Wrapf(
unimplemented.Newf("import.mysql.default", "unsupported default expression"),
"error parsing %q for column %q: %v",
exprString,
name,
err,
)
}
def.DefaultExpr.Expr = expr
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/serverccl/diagnosticsccl/reporter_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -121,7 +121,7 @@ func TestServerReport(t *testing.T) {
fmt.Sprintf(`ALTER %s CONFIGURE ZONE = '%s'`, cmd.resource, cmd.config),
); err != nil {
// Work around gossip asynchronicity.
return errors.Errorf("error applying zone config %q to %q: %v", cmd.config, cmd.resource, err)
return errors.Wrapf(err, "error applying zone config %q to %q", cmd.config, cmd.resource)
}
return nil
})
Expand Down
36 changes: 27 additions & 9 deletions pkg/cli/clierrorplus/decorate_error.go
Original file line number Diff line number Diff line change
Expand Up @@ -75,25 +75,35 @@ func MaybeDecorateError(
}()

connFailed := func() error {
// Avoid errors.Wrapf here so that we have more control over the
// formatting of the message with error text.
const format = "cannot dial server.\n" +
"Is the server running?\n" +
"If the server is running, check --host client-side and --advertise server-side.\n\n%v"
return errors.Errorf(format, err)
}

connSecurityHint := func() error {
// Avoid errors.Wrapf here so that we have more control over the
// formatting of the message with error text.
const format = "SSL authentication error while connecting.\n%v"
return errors.Errorf(format, err)
}

connInsecureHint := func() error {
return errors.Errorf("cannot establish secure connection to insecure server.\n"+
"Maybe use --insecure?\n\n%v", err)
// Avoid errors.Wrapf here so that we have more control over the
// formatting of the message with error text.
const format = "cannot establish secure connection to insecure server.\n" +
"Maybe use --insecure?\n\n%v"
return errors.Errorf(format, err)
}

connRefused := func() error {
return errors.Errorf("server closed the connection.\n"+
"Is this a CockroachDB node?\n%v", err)
// Avoid errors.Wrapf here so that we have more control over the
// formatting of the message with error text.
const format = "server closed the connection.\n" +
"Is this a CockroachDB node?\n%v"
return errors.Errorf(format, err)
}

// Is this an "unable to connect" type of error?
Expand All @@ -105,9 +115,11 @@ func MaybeDecorateError(
}

if wErr := (*security.Error)(nil); errors.As(err, &wErr) {
return errors.Errorf("cannot load certificates.\n"+
"Check your certificate settings, set --certs-dir, or use --insecure for insecure clusters.\n\n%v",
err)
// Avoid errors.Wrapf here so that we have more control over the
// formatting of the message with error text.
const format = "cannot load certificates.\n" +
"Check your certificate settings, set --certs-dir, or use --insecure for insecure clusters.\n\n%v"
return errors.Errorf(format, err)
}

if wErr := (*x509.UnknownAuthorityError)(nil); errors.As(err, &wErr) {
Expand Down Expand Up @@ -181,7 +193,10 @@ func MaybeDecorateError(
}

opTimeout := func() error {
return errors.Errorf("operation timed out.\n\n%v", err)
// Avoid errors.Wrapf here so that we have more control over the
// formatting of the message with error text.
const format = "operation timed out.\n\n%v"
return errors.Errorf(format, err)
}

// Is it a plain context cancellation (i.e. timeout)?
Expand All @@ -201,7 +216,10 @@ func MaybeDecorateError(
return fmt.Errorf(
"incompatible client and server versions (likely server version: v1.0, required: >=v1.1)")
} else if grpcutil.IsClosedConnection(err) {
return errors.Errorf("connection lost.\n\n%v", err)
// Avoid errors.Wrapf here so that we have more control over the
// formatting of the message with error text.
const format = "connection lost.\n\n%v"
return errors.Errorf(format, err)
}

// Does the server require GSSAPI authentication?
Expand Down
4 changes: 2 additions & 2 deletions pkg/cli/clisqlclient/conn_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -69,7 +69,7 @@ func TestConnRecover(t *testing.T) {
t.Fatal(closeErr)
}
} else if !errors.Is(err, driver.ErrBadConn) {
return errors.Newf("expected ErrBadConn, got %v", err)
return errors.Newf("expected ErrBadConn, got %v", err) // nolint:errwrap
}
return nil
})
Expand All @@ -89,7 +89,7 @@ func TestConnRecover(t *testing.T) {
// Ditto from Query().
testutils.SucceedsSoon(t, func() error {
if err := conn.Exec(`SELECT 1`, nil); !errors.Is(err, driver.ErrBadConn) {
return errors.Newf("expected ErrBadConn, got %v", err)
return errors.Newf("expected ErrBadConn, got %v", err) // nolint:errwrap
}
return nil
})
Expand Down
2 changes: 1 addition & 1 deletion pkg/cli/node.go
Original file line number Diff line number Diff line change
Expand Up @@ -305,7 +305,7 @@ func parseNodeIDs(strNodeIDs []string) ([]roachpb.NodeID, error) {
for _, str := range strNodeIDs {
i, err := strconv.ParseInt(str, 10, 32)
if err != nil {
return nil, errors.Errorf("unable to parse %s: %s", str, err)
return nil, errors.Wrapf(err, "unable to parse %s", str)
}
nodeIDs = append(nodeIDs, roachpb.NodeID(i))
}
Expand Down
10 changes: 5 additions & 5 deletions pkg/cli/testutils.go
Original file line number Diff line number Diff line change
Expand Up @@ -441,7 +441,7 @@ func GetCsvNumCols(csvStr string) (cols int, err error) {
reader := csv.NewReader(strings.NewReader(csvStr))
records, err := reader.Read()
if err != nil {
return 0, errors.Errorf("error reading csv input: \n %v\n errors:%s", csvStr, err)
return 0, errors.Wrapf(err, "error reading csv input:\n %v\n", csvStr)
}
return len(records), nil
}
Expand All @@ -451,8 +451,8 @@ func GetCsvNumCols(csvStr string) (cols int, err error) {
func MatchCSV(csvStr string, matchColRow [][]string) (err error) {
defer func() {
if err != nil {
err = errors.Errorf("csv input:\n%v\nexpected:\n%s\nerrors:%s",
csvStr, pretty.Sprint(matchColRow), err)
err = errors.Wrapf(err, "csv input:\n%v\nexpected:\n%s\n",
csvStr, pretty.Sprint(matchColRow))
}
}()

Expand Down Expand Up @@ -482,8 +482,8 @@ func MatchCSV(csvStr string, matchColRow [][]string) (err error) {
pat, str := matchColRow[i][j], records[i][j]
re := regexp.MustCompile(pat)
if !re.MatchString(str) {
err = errors.Errorf("%v\nrow #%d, col #%d: found %q which does not match %q",
err, i+1, j+1, str, pat)
err = errors.Wrapf(err, "row #%d, col #%d: found %q which does not match %q",
i+1, j+1, str, pat)
}
}
}
Expand Down
5 changes: 3 additions & 2 deletions pkg/cloud/amazon/s3_storage.go
Original file line number Diff line number Diff line change
Expand Up @@ -433,9 +433,10 @@ func (s *s3Storage) openStreamAt(
switch aerr.Code() {
// Relevant 404 errors reported by AWS.
case s3.ErrCodeNoSuchBucket, s3.ErrCodeNoSuchKey:
return nil, errors.WithMessagef(
// nolint:errwrap
return nil, errors.Wrapf(
errors.Wrap(cloud.ErrFileDoesNotExist, "s3 object does not exist"),
"%s",
"%v",
err.Error(),
)
}
Expand Down
5 changes: 3 additions & 2 deletions pkg/cloud/azure/azure_storage.go
Original file line number Diff line number Diff line change
Expand Up @@ -156,9 +156,10 @@ func (s *azureStorage) ReadFileAt(
switch azerr.ServiceCode() {
// TODO(adityamaru): Investigate whether both these conditions are required.
case azblob.ServiceCodeBlobNotFound, azblob.ServiceCodeResourceNotFound:
return nil, 0, errors.WithMessagef(
// nolint:errwrap
return nil, 0, errors.Wrapf(
errors.Wrap(cloud.ErrFileDoesNotExist, "azure blob does not exist"),
"%s",
"%v",
err.Error(),
)
}
Expand Down
5 changes: 3 additions & 2 deletions pkg/cloud/gcp/gcs_storage.go
Original file line number Diff line number Diff line change
Expand Up @@ -196,9 +196,10 @@ func (g *gcsStorage) ReadFileAt(
// if file does not exist. Regardless why we couldn't open the stream
// (whether its invalid bucket or file doesn't exist),
// return our internal ErrFileDoesNotExist.
err = errors.WithMessagef(
// nolint:errwrap
err = errors.Wrapf(
errors.Wrap(cloud.ErrFileDoesNotExist, "gcs object does not exist"),
"%s",
"%v",
err.Error(),
)
}
Expand Down
5 changes: 3 additions & 2 deletions pkg/cloud/httpsink/http_storage.go
Original file line number Diff line number Diff line change
Expand Up @@ -266,9 +266,10 @@ func (h *httpStorage) req(
_ = resp.Body.Close()
err := errors.Errorf("error response from server: %s %q", resp.Status, body)
if err != nil && resp.StatusCode == 404 {
err = errors.WithMessagef(
// nolint:errwrap
err = errors.Wrapf(
errors.Wrap(cloud.ErrFileDoesNotExist, "http storage file does not exist"),
"%s",
"%v",
err.Error(),
)
}
Expand Down
1 change: 1 addition & 0 deletions pkg/cloud/nodelocal/nodelocal_storage.go
Original file line number Diff line number Diff line change
Expand Up @@ -144,6 +144,7 @@ func (l *localFileStorage) ReadFileAt(
// The local store returns a golang native ErrNotFound, whereas the remote
// store returns a gRPC native NotFound error.
if oserror.IsNotExist(err) || status.Code(err) == codes.NotFound {
// nolint:errwrap
return nil, 0, errors.WithMessagef(
errors.Wrap(cloud.ErrFileDoesNotExist, "nodelocal storage file does not exist"),
"%s",
Expand Down
4 changes: 2 additions & 2 deletions pkg/cmd/github-post/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -503,8 +503,8 @@ func getFileLine(
// ../ccl/storageccl/export_test.go:31:func TestExportCmd(t *testing.T) {
out, err := cmd.CombinedOutput()
if err != nil {
return "", "", errors.Errorf("couldn't find test %s in %s: %s %s",
testName, packageName, err, string(out))
return "", "", errors.Wrapf(err, "couldn't find test %s in %s: %s\n",
testName, packageName, string(out))
}
re := regexp.MustCompile(`(.*):(.*):`)
// The first 2 :-delimited fields are the filename and line number.
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/encryption.go
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,7 @@ func registerEncryption(r registry.Registry) {
testCLIGenKey := func(size int) error {
// Generate encryption store key through `./cockroach gen encryption-key -s=size aes-size.key`.
if err := c.RunE(ctx, c.Node(nodes), fmt.Sprintf("./cockroach gen encryption-key -s=%[1]d aes-%[1]d.key", size)); err != nil {
return errors.Errorf("failed to generate aes key with size %d through CLI, got err %s", size, err)
return errors.Wrapf(err, "failed to generate AES key with size %d through CLI", size)
}

// Check the size of generated aes key has expected size.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/option"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/registry"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/util/version"
"github.com/cockroachdb/errors"
Expand Down Expand Up @@ -91,7 +92,7 @@ func runSchemaChangeDatabaseVersionUpgrade(
assertDatabaseNotResolvable := func(ctx context.Context, db *gosql.DB, dbName string) error {
_, err = db.ExecContext(ctx, fmt.Sprintf(`SELECT table_name FROM [SHOW TABLES FROM %s]`, dbName))
if err == nil || err.Error() != "pq: target database or schema does not exist" {
return errors.AssertionFailedf("unexpected error: %s", err)
return errors.Newf("unexpected error: %s", pgerror.FullError(err))
}
return nil
}
Expand Down
1 change: 1 addition & 0 deletions pkg/cmd/roachvet/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
2 changes: 2 additions & 0 deletions pkg/cmd/roachvet/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -65,6 +66,7 @@ func main() {
fmtsafe.Analyzer,
errcmp.Analyzer,
nilness.Analyzer,
errwrap.Analyzer,
)

// Standard go vet analyzers:
Expand Down
2 changes: 1 addition & 1 deletion pkg/col/coldatatestutils/random_testutils.go
Original file line number Diff line number Diff line change
Expand Up @@ -216,7 +216,7 @@ func setNull(rng *rand.Rand, vec coldata.Vec, i int) {
case types.DecimalFamily:
_, err := vec.Decimal()[i].SetFloat64(rng.Float64())
if err != nil {
colexecerror.InternalError(errors.AssertionFailedf("%v", err))
colexecerror.InternalError(errors.NewAssertionErrorWithWrappedErrf(err, "could not set decimal"))
}
case types.IntervalFamily:
vec.Interval()[i] = duration.MakeDuration(rng.Int63(), rng.Int63(), rng.Int63())
Expand Down
4 changes: 2 additions & 2 deletions pkg/geo/geos/geos.go
Original file line number Diff line number Diff line change
Expand Up @@ -654,11 +654,11 @@ func PrepareGeometry(a geopb.EWKB) (PreparedGeometry, error) {
func PreparedGeomDestroy(a PreparedGeometry) {
g, err := ensureInitInternal()
if err != nil {
panic(errors.AssertionFailedf("trying to destroy PreparedGeometry with no GEOS: %v", err))
panic(errors.NewAssertionErrorWithWrappedErrf(err, "trying to destroy PreparedGeometry with no GEOS"))
}
ap := (*C.CR_GEOS_PreparedGeometry)(unsafe.Pointer(a))
if err := statusToError(C.CR_GEOS_PreparedGeometryDestroy(g, ap)); err != nil {
panic(errors.AssertionFailedf("PreparedGeometryDestroy returned an error: %v", err))
panic(errors.NewAssertionErrorWithWrappedErrf(err, "PreparedGeometryDestroy returned an error"))
}
}

Expand Down
4 changes: 2 additions & 2 deletions pkg/gossip/client.go
Original file line number Diff line number Diff line change
Expand Up @@ -263,8 +263,8 @@ func (c *client) handleResponse(ctx context.Context, g *Gossip, reply *Response)
// The certificates (if any) may only be valid for the unresolved
// address.
if _, err := reply.AlternateAddr.Resolve(); err != nil {
return errors.Errorf("unable to resolve alternate address %s for n%d: %s",
reply.AlternateAddr, reply.AlternateNodeID, err)
return errors.Wrapf(err, "unable to resolve alternate address %s for n%d",
reply.AlternateAddr, reply.AlternateNodeID)
}
c.forwardAddr = reply.AlternateAddr
return errors.Errorf("received forward from n%d to n%d (%s)",
Expand Down
2 changes: 1 addition & 1 deletion pkg/jobs/job_scheduler_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -616,7 +616,7 @@ func TestJobSchedulerDaemonUsesSystemTables(t *testing.T) {
var count int
if err := db.QueryRow(
"SELECT count(*) FROM defaultdb.foo").Scan(&count); err != nil || count != 3 {
return errors.Newf("expected 3 rows, got %d (err=%+v)", count, err)
return errors.Newf("expected 3 rows, got %d (err=%+v)", count, err) // nolint:errwrap
}
return nil
})
Expand Down
Loading

0 comments on commit 2c590e2

Please sign in to comment.