Skip to content

Commit

Permalink
Merge #48040 #48113
Browse files Browse the repository at this point in the history
48040: *: lint log and error formatters / constructors r=irfansharif,ajwerner,RaduBerinde a=knz

This patch adds a new `fmtsafe` linter in the roachvet
tree and fixes a couple of bugs found by the litter.

The linter checks the following:

- that the format string in `log.Infof()`, `errors.Errorf()` and
  similar calls is a constant string.

  This check is essential for correctness because format strings
  are assumed to be PII-free and always safe for reporting in
  telemetry or PII-free logs.

- that the message strings in errors.New() and similar calls that
  construct error objects is a constant string.

  This check is provided to encourage hygiene: errors
  constructed using non-constant strings are better constructed using
  a formatting function instead, which makes the construction more
  readable and encourage the provision of PII-free reportable details.

New error or logging formatter should be subsequently added to the
file `functions.go` in that linter's directory.

It is possible for a call site to opt the format/message string out of
the linter using `/* nolint:fmtsafe */` after the format argument.
This can only be done in tests, where it can be assumed that no
formatted messages/errors will ever be reported to telemetry.

Example bugs that were found:

- `log.Warningf(ctx, msg)` with a non-constant `msg` that might
  contain `%` codes.
- calls to `errors.New(fmt.Sprintf(...))` instead of
  `errors.Newf(...)`.

Release note: None

48113: colexec: fix TestRouterOutputAddBatch when BatchSize=3 r=yuzefovich a=asubiotto

The QuarterSelection test would use BatchSize/4 elements and expect a spill to
disk. With BatchSize=3, the number of elements would be 0, causing the test to
fail since no disk spill occurred.

Release note: None (testing fix).

Fixes #48088 

Co-authored-by: Raphael 'kena' Poss <[email protected]>
Co-authored-by: Alfonso Subiotto Marques <[email protected]>
  • Loading branch information
3 people committed Apr 29, 2020
3 parents a0d78db + 59af3e6 + c552a81 commit 5b1d806
Show file tree
Hide file tree
Showing 55 changed files with 796 additions and 142 deletions.
4 changes: 2 additions & 2 deletions pkg/ccl/importccl/import_processor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -10,7 +10,6 @@ package importccl

import (
"context"
"errors"
"fmt"
"io/ioutil"
"math"
Expand Down Expand Up @@ -46,6 +45,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/cockroach/pkg/util/retry"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/cockroachdb/errors"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
)
Expand Down Expand Up @@ -536,7 +536,7 @@ func queryJob(db sqlutils.DBHandle, jobID int64) (js jobState) {
payload := &jobspb.Payload{}
js.err = protoutil.Unmarshal(payloadBytes, payload)
if js.err == nil {
js.err = errors.New(payload.Error)
js.err = errors.Newf("%s", payload.Error)
}
return
}
Expand Down
15 changes: 7 additions & 8 deletions pkg/ccl/importccl/read_import_base.go
Original file line number Diff line number Diff line change
Expand Up @@ -195,13 +195,11 @@ func readInputFiles(
for s := range rejected {
countRejected++
if countRejected > 1000 { // TODO(spaskob): turn the magic constant into an option
return pgerror.New(
return pgerror.Newf(
pgcode.DataCorrupted,
fmt.Sprintf(
"too many parsing errors (%d) encountered for file %s",
countRejected,
dataFile,
),
"too many parsing errors (%d) encountered for file %s",
countRejected,
dataFile,
)
}
buf = append(buf, s...)
Expand Down Expand Up @@ -339,8 +337,9 @@ func isMultiTableFormat(format roachpb.IOFileFormat_FileFormat) bool {
}

func makeRowErr(_ string, row int64, code, format string, args ...interface{}) error {
return pgerror.NewWithDepthf(1, code,
"row %d: "+format, append([]interface{}{row}, args...)...)
err := pgerror.NewWithDepthf(1, code, format, args...)
err = errors.WrapWithDepthf(1, err, "row %d", row)
return err
}

func wrapRowErr(err error, _ string, row int64, code, format string, args ...interface{}) error {
Expand Down
14 changes: 7 additions & 7 deletions pkg/ccl/importccl/read_import_mysql.go
Original file line number Diff line number Diff line change
Expand Up @@ -676,18 +676,18 @@ func mysqlColToCockroach(
def.Type = types.Jsonb

case mysqltypes.Set:
return nil, unimplemented.NewWithIssueHint(32560,
"cannot import SET columns at this time",
return nil, errors.WithHint(
unimplemented.NewWithIssue(32560, "cannot import SET columns at this time"),
"try converting the column to a 64-bit integer before import")
case mysqltypes.Geometry:
return nil, unimplemented.NewWithIssuef(32559,
"cannot import GEOMETRY columns at this time")
return nil, unimplemented.NewWithIssue(32559, "cannot import GEOMETRY columns at this time")
case mysqltypes.Bit:
return nil, unimplemented.NewWithIssueHint(32561,
"cannot improt BIT columns at this time",
return nil, errors.WithHint(
unimplemented.NewWithIssue(32561, "cannot import BIT columns at this time"),
"try converting the column to a 64-bit integer before import")
default:
return nil, unimplemented.Newf(fmt.Sprintf("import.mysqlcoltype.%s", typ), "unsupported mysql type %q", col.Type)
return nil, unimplemented.Newf(fmt.Sprintf("import.mysqlcoltype.%s", typ),
"unsupported mysql type %q", col.Type)
}

if col.NotNull {
Expand Down
4 changes: 2 additions & 2 deletions pkg/cli/start.go
Original file line number Diff line number Diff line change
Expand Up @@ -661,7 +661,7 @@ If problems persist, please see ` + base.DocsURL("cluster-setup-troubleshooting.
} else {
// Don't shout to stderr since the server will have detached by
// the time this function gets called.
log.Warningf(ctx, msg)
log.Warning(ctx, msg)
}
}

Expand Down Expand Up @@ -1265,7 +1265,7 @@ func setupAndInitializeLoggingAndProfiling(
// We log build information to stdout (for the short summary), but also
// to stderr to coincide with the full logs.
info := build.GetInfo()
log.Infof(ctx, info.Short())
log.Info(ctx, info.Short())

initMemProfile(ctx, outputDirectory)
initCPUProfile(ctx, outputDirectory)
Expand Down
6 changes: 3 additions & 3 deletions pkg/cmd/roachprod-stress/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,6 @@ import (
"bufio"
"bytes"
"context"
"errors"
"flag"
"fmt"
"io"
Expand All @@ -33,6 +32,7 @@ import (
"time"

"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/errors"
)

var (
Expand Down Expand Up @@ -62,7 +62,7 @@ func run() error {
var b bytes.Buffer
flags.SetOutput(&b)
flags.Usage()
return errors.New(b.String())
return errors.Newf("%s", b.String())
}

cluster := os.Args[1]
Expand Down Expand Up @@ -109,7 +109,7 @@ func run() error {
var b bytes.Buffer
flags.SetOutput(&b)
flags.Usage()
return errors.New(b.String())
return errors.Newf("%s", b.String())
}
if *flagFailure != "" {
if _, err := regexp.Compile(*flagFailure); err != nil {
Expand Down
4 changes: 2 additions & 2 deletions pkg/cmd/roachtest/test_runner.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,9 +35,9 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/cockroach/pkg/util/version"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/logtags"
"github.com/petermattis/goid"
"github.com/pkg/errors"
)

// testRunner runs tests.
Expand Down Expand Up @@ -812,7 +812,7 @@ func (r *testRunner) runTest(
r.collectClusterLogs(ctx, c, t.l)
// We return an error here because the test goroutine is still running, so
// we want to alert the caller of this unusual situation.
return false, fmt.Errorf(msg)
return false, errors.New(msg)
}
}

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/descriptormarshal"
"github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/fmtsafe"
"github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/hash"
"github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/nocopy"
"github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/returnerrcheck"
Expand Down Expand Up @@ -55,6 +56,7 @@ func main() {
returnerrcheck.Analyzer,
timer.Analyzer,
unconvert.Analyzer,
fmtsafe.Analyzer,

// Standard go vet analyzers:
asmdecl.Analyzer,
Expand Down
8 changes: 4 additions & 4 deletions pkg/cmd/urlcheck/lib/urlcheck/urlcheck.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,6 @@ package urlcheck
import (
"bytes"
"crypto/tls"
"errors"
"fmt"
"log"
"net"
Expand All @@ -23,6 +22,7 @@ import (
"strings"
"time"

"github.com/cockroachdb/errors"
"github.com/ghemawat/stream"
)

Expand Down Expand Up @@ -138,7 +138,7 @@ func checkURL(client *http.Client, url string) error {
return nil
}

return errors.New(resp.Status)
return errors.Newf("%s", errors.Safe(resp.Status))
}

func checkURLWithRetries(client *http.Client, url string) error {
Expand Down Expand Up @@ -239,7 +239,7 @@ func checkURLs(uniqueURLs map[string][]string) error {
for _, loc := range locs {
fmt.Fprintln(&buf, " ", loc)
}
errChan <- errors.New(buf.String())
errChan <- errors.Newf("%s", buf.String())
} else {
errChan <- nil
}
Expand All @@ -258,7 +258,7 @@ func checkURLs(uniqueURLs map[string][]string) error {
fmt.Fprint(&buf, err)
}
fmt.Fprintf(&buf, "%d errors\n", len(errs))
return errors.New(buf.String())
return errors.Newf("%s", buf.String())
}
return nil
}
7 changes: 5 additions & 2 deletions pkg/internal/rsg/yacc/parse.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,8 @@ package yacc
import (
"fmt"
"runtime"

"github.com/pkg/errors"
)

// Tree is the representation of a single parsed file.
Expand Down Expand Up @@ -80,8 +82,9 @@ func New(name string) *Tree {

// errorf formats the error and terminates processing.
func (t *Tree) errorf(format string, args ...interface{}) {
format = fmt.Sprintf("parse: %s:%d: %s", t.Name, t.lex.lineNumber(), format)
panic(fmt.Errorf(format, args...))
err := fmt.Errorf(format, args...)
err = errors.Wrapf(err, "parse: %s:%d", t.Name, t.lex.lineNumber())
panic(err)
}

// expect consumes the next token and guarantees it has the required type.
Expand Down
6 changes: 3 additions & 3 deletions pkg/kv/kvclient/kvcoord/dist_sender.go
Original file line number Diff line number Diff line change
Expand Up @@ -1834,10 +1834,10 @@ func (ds *DistSender) sendToReplicas(

// Has the caller given up?
if ctx.Err() != nil {
errMsg := fmt.Sprintf("context done during DistSender.Send: %s", ctx.Err())
log.Eventf(ctx, errMsg)
reportedErr := errors.Wrap(ctx.Err(), "context done during DistSender.Send")
log.Eventf(ctx, "%v", reportedErr)
if ambiguousError != nil {
return nil, roachpb.NewAmbiguousResultError(errMsg)
return nil, roachpb.NewAmbiguousResultError(reportedErr.Error())
}
// Don't consider this a SendError, because SendErrors indicate that we
// were unable to reach a replica that could serve the request, and they
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvclient/kvcoord/transport_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -115,7 +115,7 @@ func TestSpanImport(t *testing.T) {
server := mockInternalClient{}
// Let's spice things up and simulate an error from the server.
expectedErr := "my expected error"
server.pErr = roachpb.NewErrorf(expectedErr)
server.pErr = roachpb.NewErrorf(expectedErr /* nolint:fmtsafe */)

recCtx, getRec, cancel := tracing.ContextWithRecordingSpan(ctx, "test")
defer cancel()
Expand Down
9 changes: 6 additions & 3 deletions pkg/kv/kvserver/batcheval/cmd_push_txn.go
Original file line number Diff line number Diff line change
Expand Up @@ -244,9 +244,12 @@ func PushTxn(
if !pusherWins {
s = "failed to push"
}
log.Infof(ctx, "%s "+s+" (push type=%s) %s: %s (pushee last active: %s)",
args.PusherTxn.Short(), pushType, args.PusheeTxn.Short(),
reason, reply.PusheeTxn.LastActive())
log.Infof(ctx, "%s %s (push type=%s) %s: %s (pushee last active: %s)",
args.PusherTxn.Short(), log.Safe(s),
log.Safe(pushType),
args.PusheeTxn.Short(),
log.Safe(reason),
reply.PusheeTxn.LastActive())
}

// If the pushed transaction is in the staging state, we can't change its
Expand Down
7 changes: 5 additions & 2 deletions pkg/kv/kvserver/queue.go
Original file line number Diff line number Diff line change
Expand Up @@ -564,18 +564,21 @@ type queueHelper interface {
// is not available, the 'wait' parameter decides whether to wait or to return
// as a noop. Note that if the system is quiescing, fn may never be called in-
// dependent of the value of 'wait'.
//
// The caller is responsible for ensuring that opName does not contain PII.
// (Best is to pass a constant string.)
func (bq *baseQueue) Async(
ctx context.Context, opName string, wait bool, fn func(ctx context.Context, h queueHelper),
) {
if log.V(3) {
log.InfofDepth(ctx, 2, opName)
log.InfofDepth(ctx, 2, "%s", log.Safe(opName))
}
opName += " (" + bq.name + ")"
if err := bq.store.stopper.RunLimitedAsyncTask(context.Background(), opName, bq.addOrMaybeAddSem, wait,
func(ctx context.Context) {
fn(ctx, baseQueueHelper{bq})
}); err != nil && bq.addLogN.ShouldLog() {
log.Infof(ctx, "rate limited in %s: %s", opName, err)
log.Infof(ctx, "rate limited in %s: %s", log.Safe(opName), err)
}
}

Expand Down
9 changes: 8 additions & 1 deletion pkg/kv/kvserver/raft_log_queue.go
Original file line number Diff line number Diff line change
Expand Up @@ -277,6 +277,10 @@ func (input truncateDecisionInput) LogTooLarge() bool {
return input.LogSize > input.MaxLogSize
}

// truncateDecision describes a truncation decision.
// Beware: when extending this struct, be sure to adjust .String()
// so that it is guaranteed to not contain any PII or confidential
// cluster data.
type truncateDecision struct {
Input truncateDecisionInput
CommitIndex uint64
Expand Down Expand Up @@ -318,6 +322,9 @@ func (td *truncateDecision) NumNewRaftSnapshots() int {
return td.raftSnapshotsForIndex(td.NewFirstIndex) - td.raftSnapshotsForIndex(td.Input.FirstIndex)
}

// String returns a representation for the decision.
// It is guaranteed to not return PII or confidential
// information from the cluster.
func (td *truncateDecision) String() string {
var buf strings.Builder
_, _ = fmt.Fprintf(&buf, "should truncate: %t [", td.ShouldTruncate())
Expand Down Expand Up @@ -612,7 +619,7 @@ func (rlq *raftLogQueue) process(ctx context.Context, r *Replica, _ *config.Syst
}
r.store.metrics.RaftLogTruncated.Inc(int64(decision.NumTruncatableIndexes()))
} else {
log.VEventf(ctx, 3, decision.String())
log.VEventf(ctx, 3, "%s", log.Safe(decision.String()))
}
return nil
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/replica_rangefeed.go
Original file line number Diff line number Diff line change
Expand Up @@ -131,7 +131,7 @@ func (r *Replica) RangeFeed(
args *roachpb.RangeFeedRequest, stream roachpb.Internal_RangeFeedServer,
) *roachpb.Error {
if !RangefeedEnabled.Get(&r.store.cfg.Settings.SV) {
return roachpb.NewErrorf("rangefeeds require the kv.rangefeed.enabled setting. See " +
return roachpb.NewErrorf("rangefeeds require the kv.rangefeed.enabled setting. See %s",
base.DocsURL(`change-data-capture.html#enable-rangefeeds-to-reduce-latency`))
}
ctx := r.AnnotateCtx(stream.Context())
Expand Down
6 changes: 3 additions & 3 deletions pkg/roachpb/errors.go
Original file line number Diff line number Diff line change
Expand Up @@ -144,10 +144,10 @@ func NewErrorWithTxn(err error, txn *Transaction) *Error {
// passthrough to fmt.Errorf, with an additional prefix containing the
// filename and line number.
func NewErrorf(format string, a ...interface{}) *Error {
// Cannot use errors.Errorf here due to cyclic dependency.
err := errors.Newf(format, a...)
file, line, _ := caller.Lookup(1)
s := fmt.Sprintf("%s:%d: ", file, line)
return NewError(fmt.Errorf(s+format, a...))
err = errors.Wrapf(err, "%s:%d", file, line)
return NewError(err)
}

// String implements fmt.Stringer.
Expand Down
8 changes: 4 additions & 4 deletions pkg/server/authentication.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,8 +31,8 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/errors"
gwruntime "github.com/grpc-ecosystem/grpc-gateway/runtime"
"github.com/pkg/errors"
"google.golang.org/grpc"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/metadata"
Expand Down Expand Up @@ -175,9 +175,9 @@ func (s *authenticationServer) UserLogout(
); err != nil {
return nil, apiInternalError(ctx, err)
} else if n == 0 {
msg := fmt.Sprintf("session with id %d nonexistent", sessionID)
log.Info(ctx, msg)
return nil, fmt.Errorf(msg)
err := errors.Newf("session with id %d nonexistent", sessionID)
log.Info(ctx, err)
return nil, err
}

// Send back a header which will cause the browser to destroy the cookie.
Expand Down
2 changes: 1 addition & 1 deletion pkg/server/status/recorder.go
Original file line number Diff line number Diff line change
Expand Up @@ -594,7 +594,7 @@ func GetTotalMemory(ctx context.Context) (int64, error) {
return 0, err
}
if warning != "" {
log.Infof(ctx, warning)
log.Infof(ctx, "%s", warning)
}
return memory, nil
}
Expand Down
4 changes: 4 additions & 0 deletions pkg/sql/colexec/routers_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -140,6 +140,10 @@ func TestRouterOutputAddBatch(t *testing.T) {
defer cleanup()

for _, tc := range testCases {
if len(tc.selection) == 0 {
// No data to work with, probably due to a low coldata.BatchSize.
continue
}
for _, mtc := range memoryTestCases {
t.Run(fmt.Sprintf("%s/memoryLimit=%s", tc.name, humanizeutil.IBytes(mtc.bytes)), func(t *testing.T) {
// Clear the testAllocator for use.
Expand Down
Loading

0 comments on commit 5b1d806

Please sign in to comment.