Skip to content

Commit ff1faaa

Browse files
craig[bot]knz
craig[bot]
andcommitted
Merge #38710
38710: errors: fix the formatting with %+v r=knz a=knz (found by @RaduBerinde; needed to complete #38570) The new library `github.com/cockroachdb/errors` was not implementing `%+v` formatting properly for assertion and unimplemented errors. The wrong implementation was hiding the details of the cause of these two error types from the formatting logic. Fixing this bug comprehensively required completing the investigation of the Go 2 / `xerrors` error proposal. This revealed that the implementation of `fmt.Formatter` for wrapper errors (a `Format()` method) is required in all cases, at least until Go's stdlib learns about `errors.Formatter`. More details at golang/go#29934 and this commit message: cockroachdb/errors@78b6caa. This patch bumps the dependency `github.com/cockroachdb/errors` to pick up the fixes to assertion failures and unimplemented errors. The new definition of `errors.FormatError()` subsequently required re-implemening `Format)` for `pgerros.withCandidateCode`, which is also done here. Finally, this patch also picks up `errors.As()` and the new streamlined `fmt.Formatter` / `errors.Formatter` interaction, so this patch also simplifies a few custom error types in CockroachDB accordingly. Release note: None Co-authored-by: Raphael 'kena' Poss <[email protected]>
2 parents 712c2db + 3f7f688 commit ff1faaa

25 files changed

+238
-120
lines changed

Gopkg.lock

Lines changed: 4 additions & 7 deletions
Some generated files are not rendered by default. Learn more about customizing how changed files appear on GitHub.

pkg/cli/cli.go

Lines changed: 18 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -33,7 +33,7 @@ import (
3333
_ "github.com/cockroachdb/cockroach/pkg/workload/movr" // registers workloads
3434
_ "github.com/cockroachdb/cockroach/pkg/workload/tpcc" // registers workloads
3535
_ "github.com/cockroachdb/cockroach/pkg/workload/ycsb" // registers workloads
36-
"github.com/pkg/errors"
36+
"github.com/cockroachdb/errors"
3737
"github.com/spf13/cobra"
3838
)
3939

@@ -67,8 +67,9 @@ func Main() {
6767
if err := Run(os.Args[1:]); err != nil {
6868
fmt.Fprintf(stderr, "Failed running %q\n", cmdName)
6969
errCode = 1
70-
if ec, ok := errors.Cause(err).(*cliError); ok {
71-
errCode = ec.exitCode
70+
var cliErr *cliError
71+
if errors.As(err, &cliErr) {
72+
errCode = cliErr.exitCode
7273
}
7374
}
7475
os.Exit(errCode)
@@ -96,6 +97,20 @@ type cliError struct {
9697

9798
func (e *cliError) Error() string { return e.cause.Error() }
9899

100+
// Cause implements causer.
101+
func (e *cliError) Cause() error { return e.cause }
102+
103+
// Format implements fmt.Formatter.
104+
func (e *cliError) Format(s fmt.State, verb rune) { errors.FormatError(e, s, verb) }
105+
106+
// FormatError implements errors.Formatter.
107+
func (e *cliError) FormatError(p errors.Printer) error {
108+
if p.Detail() {
109+
p.Printf("error with exit code: %d", e.exitCode)
110+
}
111+
return e.cause
112+
}
113+
99114
// stderr aliases log.OrigStderr; we use an alias here so that tests
100115
// in this package can redirect the output of CLI commands to stdout
101116
// to be captured.

pkg/cli/error.go

Lines changed: 22 additions & 21 deletions
Original file line numberDiff line numberDiff line change
@@ -23,8 +23,8 @@ import (
2323
"github.com/cockroachdb/cockroach/pkg/util/grpcutil"
2424
"github.com/cockroachdb/cockroach/pkg/util/log"
2525
"github.com/cockroachdb/cockroach/pkg/util/netutil"
26+
"github.com/cockroachdb/errors"
2627
"github.com/lib/pq"
27-
"github.com/pkg/errors"
2828
"github.com/spf13/cobra"
2929
"google.golang.org/grpc/codes"
3030
"google.golang.org/grpc/status"
@@ -101,32 +101,32 @@ func MaybeDecorateGRPCError(
101101
}
102102

103103
// Is this an "unable to connect" type of error?
104-
unwrappedErr := errors.Cause(err)
105-
106-
if unwrappedErr == pq.ErrSSLNotSupported {
104+
if errors.Is(err, pq.ErrSSLNotSupported) {
107105
// SQL command failed after establishing a TCP connection
108106
// successfully, but discovering that it cannot use TLS while it
109107
// expected the server supports TLS.
110108
return connInsecureHint()
111109
}
112110

113-
switch wErr := unwrappedErr.(type) {
114-
case *security.Error:
111+
if wErr := (*security.Error)(nil); errors.As(err, &wErr) {
115112
return errors.Errorf("cannot load certificates.\n"+
116113
"Check your certificate settings, set --certs-dir, or use --insecure for insecure clusters.\n\n%v",
117-
unwrappedErr)
114+
err)
115+
}
118116

119-
case *x509.UnknownAuthorityError:
117+
if wErr := (*x509.UnknownAuthorityError)(nil); errors.As(err, &wErr) {
120118
// A SQL connection was attempted with an incorrect CA.
121119
return connSecurityHint()
120+
}
122121

123-
case *initialSQLConnectionError:
122+
if wErr := (*initialSQLConnectionError)(nil); errors.As(err, &wErr) {
124123
// SQL handshake failed after establishing a TCP connection
125124
// successfully, something else than CockroachDB responded, was
126125
// confused and closed the door on us.
127126
return connRefused()
127+
}
128128

129-
case *pq.Error:
129+
if wErr := (*pq.Error)(nil); errors.As(err, &wErr) {
130130
// SQL commands will fail with a pq error but only after
131131
// establishing a TCP connection successfully. So if we got
132132
// here, there was a TCP connection already.
@@ -137,8 +137,9 @@ func MaybeDecorateGRPCError(
137137
}
138138
// Otherwise, there was a regular SQL error. Just report that.
139139
return wErr
140+
}
140141

141-
case *net.OpError:
142+
if wErr := (*net.OpError)(nil); errors.As(err, &wErr) {
142143
// A non-RPC client command was used (e.g. a SQL command) and an
143144
// error occurred early while establishing the TCP connection.
144145

@@ -150,8 +151,9 @@ func MaybeDecorateGRPCError(
150151
return connSecurityHint()
151152
}
152153
return connFailed()
154+
}
153155

154-
case *netutil.InitialHeartbeatFailedError:
156+
if wErr := (*netutil.InitialHeartbeatFailedError)(nil); errors.As(err, &wErr) {
155157
// A GRPC TCP connection was established but there was an early failure.
156158
// Try to distinguish the cases.
157159
msg := wErr.Error()
@@ -177,28 +179,27 @@ func MaybeDecorateGRPCError(
177179
}
178180

179181
// Is it a plain context cancellation (i.e. timeout)?
180-
switch unwrappedErr {
181-
case context.DeadlineExceeded:
182-
return opTimeout()
183-
case context.Canceled:
182+
if errors.IsAny(err,
183+
context.DeadlineExceeded,
184+
context.Canceled) {
184185
return opTimeout()
185186
}
186187

187188
// Is it a GRPC-observed context cancellation (i.e. timeout), a GRPC
188189
// connection error, or a known indication of a too-old server?
189-
if code := status.Code(unwrappedErr); code == codes.DeadlineExceeded {
190+
if code := status.Code(errors.Cause(err)); code == codes.DeadlineExceeded {
190191
return opTimeout()
191192
} else if code == codes.Unimplemented &&
192-
strings.Contains(unwrappedErr.Error(), "unknown method Decommission") ||
193-
strings.Contains(unwrappedErr.Error(), "unknown service cockroach.server.serverpb.Init") {
193+
strings.Contains(err.Error(), "unknown method Decommission") ||
194+
strings.Contains(err.Error(), "unknown service cockroach.server.serverpb.Init") {
194195
return fmt.Errorf(
195196
"incompatible client and server versions (likely server version: v1.0, required: >=v1.1)")
196-
} else if grpcutil.IsClosedConnection(unwrappedErr) {
197+
} else if grpcutil.IsClosedConnection(err) {
197198
return errors.Errorf("connection lost.\n\n%v", err)
198199
}
199200

200201
// Does the server require GSSAPI authentication?
201-
if strings.Contains(unwrappedErr.Error(), "pq: unknown authentication response: 7") {
202+
if strings.Contains(err.Error(), "pq: unknown authentication response: 7") {
202203
return fmt.Errorf(
203204
"server requires GSSAPI authentication for this user.\n" +
204205
"The CockroachDB CLI does not support GSSAPI authentication; use 'psql' instead")

pkg/cli/sql_util.go

Lines changed: 15 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -34,8 +34,8 @@ import (
3434
"github.com/cockroachdb/cockroach/pkg/util/log"
3535
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
3636
"github.com/cockroachdb/cockroach/pkg/util/version"
37+
"github.com/cockroachdb/errors"
3738
"github.com/lib/pq"
38-
"github.com/pkg/errors"
3939
)
4040

4141
type sqlConnI interface {
@@ -74,6 +74,20 @@ type initialSQLConnectionError struct {
7474
// Error implements the error interface.
7575
func (i *initialSQLConnectionError) Error() string { return i.err.Error() }
7676

77+
// Cause implements causer.
78+
func (i *initialSQLConnectionError) Cause() error { return i.err }
79+
80+
// Format implements fmt.Formatter.
81+
func (i *initialSQLConnectionError) Format(s fmt.State, verb rune) { errors.FormatError(i, s, verb) }
82+
83+
// FormatError implements errors.Formatter.
84+
func (i *initialSQLConnectionError) FormatError(p errors.Printer) error {
85+
if p.Detail() {
86+
p.Print("error while establishing the SQL session")
87+
}
88+
return i.err
89+
}
90+
7791
// wrapConnError detects TCP EOF errors during the initial SQL handshake.
7892
// These are translated to a message "perhaps this is not a CockroachDB node"
7993
// at the top level.

pkg/sql/distsql_running.go

Lines changed: 3 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -35,9 +35,8 @@ import (
3535
"github.com/cockroachdb/cockroach/pkg/util/mon"
3636
"github.com/cockroachdb/cockroach/pkg/util/tracing"
3737
"github.com/cockroachdb/cockroach/pkg/util/uuid"
38-
"github.com/cockroachdb/errors/markers"
39-
"github.com/opentracing/opentracing-go"
40-
"github.com/pkg/errors"
38+
"github.com/cockroachdb/errors"
39+
opentracing "github.com/opentracing/opentracing-go"
4140
)
4241

4342
// To allow queries to send out flow RPCs in parallel, we use a pool of workers
@@ -169,7 +168,7 @@ func (dsp *DistSQLPlanner) setupFlows(
169168
// into the local flow.
170169
}
171170
if firstErr != nil {
172-
if _, ok := markers.If(firstErr, func(err error) (v interface{}, ok bool) {
171+
if _, ok := errors.If(firstErr, func(err error) (v interface{}, ok bool) {
173172
v, ok = err.(*distsqlrun.VectorizedSetupError)
174173
return v, ok
175174
}); ok && evalCtx.SessionData.Vectorize == sessiondata.VectorizeOn {

pkg/sql/distsqlrun/column_exec_setup.go

Lines changed: 18 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -32,7 +32,6 @@ import (
3232
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
3333
"github.com/cockroachdb/cockroach/pkg/util/tracing"
3434
"github.com/cockroachdb/errors"
35-
"github.com/cockroachdb/errors/errbase"
3635
opentracing "github.com/opentracing/opentracing-go"
3736
)
3837

@@ -956,13 +955,24 @@ type VectorizedSetupError struct {
956955
cause error
957956
}
958957

959-
// Error is part of the error interface.
960-
func (e *VectorizedSetupError) Error() string {
961-
return e.cause.Error()
962-
}
958+
var _ error = (*VectorizedSetupError)(nil)
959+
var _ fmt.Formatter = (*VectorizedSetupError)(nil)
960+
var _ errors.Formatter = (*VectorizedSetupError)(nil)
961+
962+
// Error implemented the error interface.
963+
func (e *VectorizedSetupError) Error() string { return e.cause.Error() }
964+
965+
// Cause implements the causer interface.
966+
func (e *VectorizedSetupError) Cause() error { return e.cause }
963967

964-
// Unwrap is part of the Wrapper interface.
965-
func (e *VectorizedSetupError) Unwrap() error {
968+
// Format implements the fmt.Formatter interface.
969+
func (e *VectorizedSetupError) Format(s fmt.State, verb rune) { errors.FormatError(e, s, verb) }
970+
971+
// FormatError implements the errors.Formatter interface.
972+
func (e *VectorizedSetupError) FormatError(p errors.Printer) error {
973+
if p.Detail() {
974+
p.Print("error while setting up columnar execution")
975+
}
966976
return e.cause
967977
}
968978

@@ -973,7 +983,7 @@ func decodeVectorizedSetupError(
973983
}
974984

975985
func init() {
976-
errors.RegisterWrapperDecoder(errbase.GetTypeKey((*VectorizedSetupError)(nil)), decodeVectorizedSetupError)
986+
errors.RegisterWrapperDecoder(errors.GetTypeKey((*VectorizedSetupError)(nil)), decodeVectorizedSetupError)
977987
}
978988

979989
func (f *Flow) setupVectorized(ctx context.Context) error {

pkg/sql/distsqlrun/scrub_tablereader.go

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -23,7 +23,7 @@ import (
2323
"github.com/cockroachdb/cockroach/pkg/sql/types"
2424
"github.com/cockroachdb/cockroach/pkg/util"
2525
"github.com/cockroachdb/cockroach/pkg/util/log"
26-
"github.com/pkg/errors"
26+
"github.com/cockroachdb/errors"
2727
)
2828

2929
// ScrubTypes is the schema for TableReaders that are doing a SCRUB
@@ -250,8 +250,8 @@ func (tr *scrubTableReader) Next() (sqlbase.EncDatumRow, *distsqlpb.ProducerMeta
250250
//
251251
// NB: Cases 3 and 4 are handled further below, in the standard
252252
// table scanning code path.
253-
err = errors.Cause(err)
254-
if v, ok := err.(*scrub.Error); ok {
253+
var v *scrub.Error
254+
if errors.As(err, &v) {
255255
row, err = tr.generateScrubErrorRow(row, v)
256256
} else if err == nil && row != nil {
257257
continue

pkg/sql/opt/optbuilder/builder.go

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -12,6 +12,7 @@ package optbuilder
1212

1313
import (
1414
"context"
15+
"fmt"
1516
"runtime"
1617

1718
"github.com/cockroachdb/cockroach/pkg/sql/delegate"
@@ -21,6 +22,7 @@ import (
2122
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
2223
"github.com/cockroachdb/cockroach/pkg/sql/types"
2324
"github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented"
25+
"github.com/cockroachdb/errors"
2426
)
2527

2628
// Builder holds the context needed for building a memo structure from a SQL
@@ -182,6 +184,9 @@ func (b builderError) Error() string { return b.error.Error() }
182184
// can be peeked through by the common error facilities.
183185
func (b builderError) Cause() error { return b.error }
184186

187+
// Format implements the fmt.Formatter interface.
188+
func (b builderError) Format(s fmt.State, verb rune) { errors.FormatError(b, s, verb) }
189+
185190
// unimplementedWithIssueDetailf formats according to a format
186191
// specifier and returns a Postgres error with the
187192
// pg code FeatureNotSupported, wrapped in a

pkg/sql/pgwire/pgerror/errors.go

Lines changed: 6 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -23,7 +23,11 @@ import (
2323
"github.com/lib/pq"
2424
)
2525

26-
var _ error = &Error{}
26+
var _ error = (*Error)(nil)
27+
var _ errors.ErrorHinter = (*Error)(nil)
28+
var _ errors.ErrorDetailer = (*Error)(nil)
29+
var _ errors.SafeDetailer = (*Error)(nil)
30+
var _ fmt.Formatter = (*Error)(nil)
2731

2832
// Error implements the error interface.
2933
func (pg *Error) Error() string { return pg.Message }
@@ -119,7 +123,7 @@ var _ fmt.Formatter = &Error{}
119123

120124
// Format implements the fmt.Formatter interface.
121125
//
122-
// %v/%s prints the rror as usual.
126+
// %v/%s prints the error as usual.
123127
// %#v adds the pg error code at the beginning.
124128
// %+v prints all the details, including the embedded stack traces.
125129
func (pg *Error) Format(s fmt.State, verb rune) {

0 commit comments

Comments
 (0)