From d8893d76d4ea3b510f7266c9be9690aba7ab4c41 Mon Sep 17 00:00:00 2001 From: richardjcai Date: Mon, 25 Oct 2021 21:21:14 -0400 Subject: [PATCH 1/4] sql: fix comment on constraint for tables in a schema Release note (sql change): Previously if you did comment on constraint on a table in a schema the command would succeed but the comment would not actually created. Now the comment is successfully created. --- pkg/sql/comment_on_constraint.go | 15 ++++++++------- pkg/sql/comment_on_constraint_test.go | 9 ++++++++- 2 files changed, 16 insertions(+), 8 deletions(-) diff --git a/pkg/sql/comment_on_constraint.go b/pkg/sql/comment_on_constraint.go index ff498b3e9951..2feea4f12e21 100644 --- a/pkg/sql/comment_on_constraint.go +++ b/pkg/sql/comment_on_constraint.go @@ -17,7 +17,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemadesc" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/privilege" @@ -62,8 +61,10 @@ func (n *commentOnConstraintNode) startExec(params runParams) error { if err != nil { return err } - cSchema, ok := schemadesc.GetVirtualSchemaByID(n.tableDesc.GetParentSchemaID()) - if !ok { + schema, err := params.p.Descriptors().GetImmutableSchemaByID( + params.ctx, params.extendedEvalCtx.Txn, n.tableDesc.GetParentSchemaID(), tree.SchemaLookupFlags{}, + ) + if err != nil { return err } @@ -78,16 +79,16 @@ func (n *commentOnConstraintNode) startExec(params runParams) error { switch kind := constraint.Kind; kind { case descpb.ConstraintTypePK: constraintDesc := constraint.Index - n.oid = hasher.PrimaryKeyConstraintOid(n.tableDesc.GetParentID(), cSchema.GetName(), n.tableDesc.GetID(), constraintDesc) + n.oid = hasher.PrimaryKeyConstraintOid(n.tableDesc.GetParentID(), schema.GetName(), n.tableDesc.GetID(), constraintDesc) case descpb.ConstraintTypeFK: constraintDesc := constraint.FK - n.oid = hasher.ForeignKeyConstraintOid(n.tableDesc.GetParentID(), cSchema.GetName(), n.tableDesc.GetID(), constraintDesc) + n.oid = hasher.ForeignKeyConstraintOid(n.tableDesc.GetParentID(), schema.GetName(), n.tableDesc.GetID(), constraintDesc) case descpb.ConstraintTypeUnique: constraintDesc := constraint.Index.ID - n.oid = hasher.UniqueConstraintOid(n.tableDesc.GetParentID(), cSchema.GetName(), n.tableDesc.GetID(), constraintDesc) + n.oid = hasher.UniqueConstraintOid(n.tableDesc.GetParentID(), schema.GetName(), n.tableDesc.GetID(), constraintDesc) case descpb.ConstraintTypeCheck: constraintDesc := constraint.CheckConstraint - n.oid = hasher.CheckConstraintOid(n.tableDesc.GetParentID(), cSchema.GetName(), n.tableDesc.GetID(), constraintDesc) + n.oid = hasher.CheckConstraintOid(n.tableDesc.GetParentID(), schema.GetName(), n.tableDesc.GetID(), constraintDesc) } // Setting the comment to NULL is the diff --git a/pkg/sql/comment_on_constraint_test.go b/pkg/sql/comment_on_constraint_test.go index 9e0265db231c..4ea597c3d18e 100644 --- a/pkg/sql/comment_on_constraint_test.go +++ b/pkg/sql/comment_on_constraint_test.go @@ -33,7 +33,9 @@ func TestCommentOnConstraint(t *testing.T) { CREATE DATABASE d; SET DATABASE = d; CREATE TABLE t ( a int UNIQUE, b numeric CONSTRAINT positive_price CHECK (b > 0), c int CHECK (b > c), CONSTRAINT pkey PRIMARY KEY (a,c)); - CREATE TABLE t2 (a UUID PRIMARY KEY, b int NOT NULL REFERENCES t (a)) + CREATE TABLE t2 (a UUID PRIMARY KEY, b int NOT NULL REFERENCES t (a)); + CREATE SCHEMA s; + CREATE TABLE s.t ( a int UNIQUE, b numeric CONSTRAINT positive_price CHECK (b > 0), c int CHECK (b > c), CONSTRAINT pkey PRIMARY KEY (a,c)); `); err != nil { t.Fatal(err) } @@ -48,6 +50,11 @@ func TestCommentOnConstraint(t *testing.T) { `SELECT obj_description(oid, 'pg_constraint') FROM pg_constraint WHERE conname='t_a_key'`, gosql.NullString{String: `unique_comment`, Valid: true}, }, + { + `COMMENT ON CONSTRAINT t_a_key ON s.t IS 'unique_comment'`, + `SELECT obj_description(oid, 'pg_constraint') FROM pg_constraint WHERE conname='t_a_key'`, + gosql.NullString{String: `unique_comment`, Valid: true}, + }, { `COMMENT ON CONSTRAINT positive_price ON t IS 'check_comment'`, `SELECT obj_description(oid, 'pg_constraint') FROM pg_constraint WHERE conname='positive_price'`, From 2ccf86bf28fe1eaad7ab04910dc66f87ab30a1c7 Mon Sep 17 00:00:00 2001 From: Rafi Shamim Date: Tue, 2 Nov 2021 16:49:13 -0400 Subject: [PATCH 2/4] sql/*: fix improperly wrapped errors I'm working on a linter that detects errors that are not wrapped correctly, and it discovered these. Release note: None --- pkg/geo/geos/geos.go | 6 ++--- pkg/sql/colflow/colrpc/inbox.go | 10 ++++---- pkg/sql/exec_util.go | 2 +- pkg/sql/flowinfra/inbound.go | 2 +- .../logictest/testdata/logic_test/drop_index | 2 +- pkg/sql/opt/optgen/cmd/langgen/main.go | 2 +- pkg/sql/opt/optgen/cmd/optgen/main.go | 2 +- pkg/sql/opt/optgen/lang/compiler.go | 2 +- pkg/sql/opt/props/func_dep_rand_test.go | 6 ++--- pkg/sql/pg_metadata_test.go | 11 +++++---- pkg/sql/pgwire/server.go | 24 ++++++++++++------- pkg/sql/region_util.go | 12 +++++----- pkg/sql/row/errors.go | 8 +++---- pkg/sql/sem/builtins/builtins.go | 4 ++-- pkg/sql/sem/tree/eval.go | 12 +++++----- pkg/sql/sem/tree/interval.go | 8 +++---- pkg/sql/set_zone_config.go | 12 ++++------ pkg/sql/sqlerrors/errors.go | 6 +---- pkg/sql/sqlstats/sslocal/BUILD.bazel | 1 + pkg/sql/sqlstats/sslocal/sslocal_provider.go | 4 ++-- .../sqlstats/ssmemstorage/ss_mem_storage.go | 2 +- pkg/sql/type_change_test.go | 2 +- 22 files changed, 71 insertions(+), 69 deletions(-) diff --git a/pkg/geo/geos/geos.go b/pkg/geo/geos/geos.go index 2d928af8a81f..7176892f17aa 100644 --- a/pkg/geo/geos/geos.go +++ b/pkg/geo/geos/geos.go @@ -194,10 +194,10 @@ func initGEOS(dirs []string) (*C.CR_GEOS, string, error) { } err = errors.CombineErrors( err, - errors.Newf( - "geos: cannot load GEOS from dir %q: %s", - dir, + errors.Wrapf( newErr, + "geos: cannot load GEOS from dir %q", + dir, ), ) } diff --git a/pkg/sql/colflow/colrpc/inbox.go b/pkg/sql/colflow/colrpc/inbox.go index 2958ca8b63b5..6962014a357b 100644 --- a/pkg/sql/colflow/colrpc/inbox.go +++ b/pkg/sql/colflow/colrpc/inbox.go @@ -12,7 +12,6 @@ package colrpc import ( "context" - "fmt" "io" "math" "sync/atomic" @@ -33,6 +32,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/log/logcrash" "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/cockroachdb/errors" "github.com/cockroachdb/logtags" ) @@ -208,7 +208,7 @@ func (i *Inbox) RunWithStream( case readerCtx = <-i.contextCh: log.VEvent(streamCtx, 2, "Inbox reader arrived") case <-streamCtx.Done(): - return fmt.Errorf("%s: streamCtx while waiting for reader (remote client canceled)", streamCtx.Err()) + return errors.Wrap(streamCtx.Err(), "streamCtx error while waiting for reader (remote client canceled)") case <-flowCtxDone: // The flow context of the inbox host has been canceled. This can occur // e.g. when the query is canceled, or when another stream encountered @@ -233,7 +233,7 @@ func (i *Inbox) RunWithStream( return nil case <-streamCtx.Done(): // The client canceled the stream. - return fmt.Errorf("%s: streamCtx in Inbox stream handler (remote client canceled)", streamCtx.Err()) + return errors.Wrap(streamCtx.Err(), "streamCtx error in Inbox stream handler (remote client canceled)") } } @@ -258,7 +258,7 @@ func (i *Inbox) Init(ctx context.Context) { select { case i.stream = <-i.streamCh: case err := <-i.timeoutCh: - i.errCh <- fmt.Errorf("%s: remote stream arrived too late", err) + i.errCh <- errors.Wrap(err, "remote stream arrived too late") return err case <-i.Ctx.Done(): // Our reader canceled the context meaning that it no longer needs @@ -325,7 +325,7 @@ func (i *Inbox) Next() coldata.Batch { // Regardless of the cause we want to propagate such an error as // expected one in all cases so that the caller could decide on how // to handle it. - err = pgerror.Newf(pgcode.InternalConnectionFailure, "inbox communication error: %s", err) + err = pgerror.Wrap(err, pgcode.InternalConnectionFailure, "inbox communication error") i.errCh <- err colexecerror.ExpectedError(err) } diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index 2dd9b178dacf..ce65bdf351a8 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -1798,7 +1798,7 @@ type registrySession interface { func (r *SessionRegistry) CancelQuery(queryIDStr string) (bool, error) { queryID, err := StringToClusterWideID(queryIDStr) if err != nil { - return false, fmt.Errorf("query ID %s malformed: %s", queryID, err) + return false, errors.Wrapf(err, "query ID %s malformed", queryID) } r.Lock() diff --git a/pkg/sql/flowinfra/inbound.go b/pkg/sql/flowinfra/inbound.go index b752b25ba96d..d478c7d770b5 100644 --- a/pkg/sql/flowinfra/inbound.go +++ b/pkg/sql/flowinfra/inbound.go @@ -137,7 +137,7 @@ func processInboundStreamHelper( if err != nil { if err != io.EOF { // Communication error. - err = pgerror.Newf(pgcode.InternalConnectionFailure, "inbox communication error: %s", err) + err = pgerror.Wrap(err, pgcode.InternalConnectionFailure, "inbox communication error") sendErrToConsumer(err) errChan <- err return diff --git a/pkg/sql/logictest/testdata/logic_test/drop_index b/pkg/sql/logictest/testdata/logic_test/drop_index index 2439f8206e71..34a5cdd22bdc 100644 --- a/pkg/sql/logictest/testdata/logic_test/drop_index +++ b/pkg/sql/logictest/testdata/logic_test/drop_index @@ -327,7 +327,7 @@ DROP INDEX t_secondary CASCADE; ALTER TABLE t DROP COLUMN b; INSERT INTO t SELECT a + 1 FROM t; -statement error pgcode 23505 duplicate key value: decoding err=column-id "2" does not exist +statement error pgcode 23505 duplicate key value got decoding error: column-id "2" does not exist UPSERT INTO t SELECT a + 1 FROM t; statement ok diff --git a/pkg/sql/opt/optgen/cmd/langgen/main.go b/pkg/sql/opt/optgen/cmd/langgen/main.go index 8993d8eff825..7771a162b1cc 100644 --- a/pkg/sql/opt/optgen/cmd/langgen/main.go +++ b/pkg/sql/opt/optgen/cmd/langgen/main.go @@ -130,7 +130,7 @@ func generate(compiled *lang.CompiledExpr, out string, genFunc genFunc) error { if err != nil { // Write out incorrect source for easier debugging. b = buf.Bytes() - err = fmt.Errorf("code formatting failed with Go parse error\n%s:%s", out, err) + err = errors.Wrapf(err, "code formatting failed with Go parse error\n%s", out) } } else { b = buf.Bytes() diff --git a/pkg/sql/opt/optgen/cmd/optgen/main.go b/pkg/sql/opt/optgen/cmd/optgen/main.go index cbc19b3e83a9..7e13322f6178 100644 --- a/pkg/sql/opt/optgen/cmd/optgen/main.go +++ b/pkg/sql/opt/optgen/cmd/optgen/main.go @@ -218,7 +218,7 @@ func (g *optgen) generate(compiled *lang.CompiledExpr, genFunc genFunc) error { // Write out incorrect source for easier debugging. b = buf.Bytes() out := g.cmdLine.Lookup("out").Value.String() - err = fmt.Errorf("code formatting failed with Go parse error\n%s:%s", out, err) + err = errors.Wrapf(err, "code formatting failed with Go parse error\n%s", out) } } else { b = buf.Bytes() diff --git a/pkg/sql/opt/optgen/lang/compiler.go b/pkg/sql/opt/optgen/lang/compiler.go index 1b0d203dc30b..bf9b18aab8a4 100644 --- a/pkg/sql/opt/optgen/lang/compiler.go +++ b/pkg/sql/opt/optgen/lang/compiler.go @@ -200,7 +200,7 @@ func (c *Compiler) compileRules(rules RuleSetExpr) bool { func (c *Compiler) addErr(src *SourceLoc, err error) { if src != nil { - err = fmt.Errorf("%s: %s", src, err.Error()) + err = errors.Wrapf(err, "%s", src) } c.errors = append(c.errors, err) } diff --git a/pkg/sql/opt/props/func_dep_rand_test.go b/pkg/sql/opt/props/func_dep_rand_test.go index c8361fbc5548..8fb8f4f76651 100644 --- a/pkg/sql/opt/props/func_dep_rand_test.go +++ b/pkg/sql/opt/props/func_dep_rand_test.go @@ -432,7 +432,7 @@ func (tc *testConfig) checkAPIs(fd *FuncDepSet, tr testRelation) error { to: closure, strict: true, }); err != nil { - return fmt.Errorf("ComputeClosure%s incorrectly returns %s: %s", cols, closure, err) + return errors.Wrapf(err, "ComputeClosure%s incorrectly returns %s", cols, closure) } reduced := fd.ReduceCols(cols) @@ -441,7 +441,7 @@ func (tc *testConfig) checkAPIs(fd *FuncDepSet, tr testRelation) error { to: cols, strict: true, }); err != nil { - return fmt.Errorf("ReduceCols%s incorrectly returns %s: %s", cols, reduced, err) + return errors.Wrapf(err, "ReduceCols%s incorrectly returns %s", cols, reduced) } var proj FuncDepSet @@ -449,7 +449,7 @@ func (tc *testConfig) checkAPIs(fd *FuncDepSet, tr testRelation) error { proj.ProjectCols(cols) // The FDs after projection should still hold on the table. if err := tr.checkFDs(&proj); err != nil { - return fmt.Errorf("ProjectCols%s incorrectly returns %s: %s", cols, proj.String(), err) + return errors.Wrapf(err, "ProjectCols%s incorrectly returns %s", cols, proj.String()) } } diff --git a/pkg/sql/pg_metadata_test.go b/pkg/sql/pg_metadata_test.go index 3c418f24af37..14a1a7c3aecb 100644 --- a/pkg/sql/pg_metadata_test.go +++ b/pkg/sql/pg_metadata_test.go @@ -123,6 +123,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/errors" "github.com/cockroachdb/errors/oserror" "github.com/lib/pq/oid" ) @@ -678,7 +679,7 @@ type outputFile struct { // appendString calls WriteString and panics on error. func (o outputFile) appendString(s string) { if _, err := o.f.WriteString(s); err != nil { - panic(fmt.Errorf("error while writing string: %s: %v", s, err)) + panic(errors.Wrapf(err, "error while writing string: %s", s)) } } @@ -703,7 +704,7 @@ func rewriteFile(fileName string, f func(*os.File, outputFile)) { updateFile(tmpName, fileName, func(input *os.File, output outputFile) { if _, err := io.Copy(output.f, input); err != nil { - panic(fmt.Errorf("problem at rewriting file %s into %s: %v", tmpName, fileName, err)) + panic(errors.Wrapf(err, "problem at rewriting file %s into %s", tmpName, fileName)) } }) } @@ -711,13 +712,13 @@ func rewriteFile(fileName string, f func(*os.File, outputFile)) { func updateFile(inputFileName, outputFileName string, f func(input *os.File, output outputFile)) { input, err := os.Open(inputFileName) if err != nil { - panic(fmt.Errorf("error opening file %s: %v", inputFileName, err)) + panic(errors.Wrapf(err, "error opening file %s", inputFileName)) } defer dClose(input) output, err := os.OpenFile(outputFileName, os.O_CREATE|os.O_WRONLY|os.O_TRUNC, 0644) if err != nil { - panic(fmt.Errorf("error opening file %s: %v", outputFileName, err)) + panic(errors.Wrapf(err, "error opening file %s", outputFileName)) } defer dClose(output) @@ -895,7 +896,7 @@ func (scf schemaCodeFixer) getTableDefinitionsText(unimplementedTables PGMetadat maxLength := 0 f, err := os.Open(fileName) if err != nil { - panic(fmt.Errorf("could not open file %s: %v", fileName, err)) + panic(errors.Wrapf(err, "could not open file %s", fileName)) } defer dClose(f) reader := bufio.NewScanner(f) diff --git a/pkg/sql/pgwire/server.go b/pkg/sql/pgwire/server.go index ec6450a411a8..34321a3e88d3 100644 --- a/pkg/sql/pgwire/server.go +++ b/pkg/sql/pgwire/server.go @@ -739,8 +739,10 @@ func parseClientProvidedSessionParameters( // Read a key-value pair from the client. key, err := buf.GetString() if err != nil { - return sql.SessionArgs{}, pgerror.Newf(pgcode.ProtocolViolation, - "error reading option key: %s", err) + return sql.SessionArgs{}, pgerror.Wrap( + err, pgcode.ProtocolViolation, + "error reading option key", + ) } if len(key) == 0 { // End of parameter list. @@ -748,8 +750,10 @@ func parseClientProvidedSessionParameters( } value, err := buf.GetString() if err != nil { - return sql.SessionArgs{}, pgerror.Newf(pgcode.ProtocolViolation, - "error reading option value: %s", err) + return sql.SessionArgs{}, pgerror.Wrapf( + err, pgcode.ProtocolViolation, + "error reading option value for key %q", key, + ) } // Case-fold for the key for easier comparison. @@ -788,13 +792,17 @@ func parseClientProvidedSessionParameters( hostS, portS, err := net.SplitHostPort(value) if err != nil { - return sql.SessionArgs{}, pgerror.Newf(pgcode.ProtocolViolation, - "invalid address format: %v", err) + return sql.SessionArgs{}, pgerror.Wrap( + err, pgcode.ProtocolViolation, + "invalid address format", + ) } port, err := strconv.Atoi(portS) if err != nil { - return sql.SessionArgs{}, pgerror.Newf(pgcode.ProtocolViolation, - "remote port is not numeric: %v", err) + return sql.SessionArgs{}, pgerror.Wrap( + err, pgcode.ProtocolViolation, + "remote port is not numeric", + ) } ip := net.ParseIP(hostS) if ip == nil { diff --git a/pkg/sql/region_util.go b/pkg/sql/region_util.go index ff161f9e0f74..939d6074fd39 100644 --- a/pkg/sql/region_util.go +++ b/pkg/sql/region_util.go @@ -663,17 +663,17 @@ func prepareZoneConfigForMultiRegionTable( return nil, nil } if err := newZoneConfig.Validate(); err != nil { - return nil, pgerror.Newf( - pgcode.CheckViolation, - "could not validate zone config: %v", + return nil, pgerror.Wrap( err, + pgcode.CheckViolation, + "could not validate zone config", ) } if err := newZoneConfig.ValidateTandemFields(); err != nil { - return nil, pgerror.Newf( - pgcode.CheckViolation, - "could not validate zone config: %v", + return nil, pgerror.Wrap( err, + pgcode.CheckViolation, + "could not validate zone config", ) } return prepareZoneConfigWrites( diff --git a/pkg/sql/row/errors.go b/pkg/sql/row/errors.go index dfbb137ad7d0..b21aa8cd70c4 100644 --- a/pkg/sql/row/errors.go +++ b/pkg/sql/row/errors.go @@ -115,8 +115,8 @@ func NewUniquenessConstraintViolationError( ) error { index, names, values, err := DecodeRowInfo(ctx, tableDesc, key, value, false) if err != nil { - return pgerror.Newf(pgcode.UniqueViolation, - "duplicate key value: decoding err=%s", err) + return pgerror.Wrap(err, pgcode.UniqueViolation, + "duplicate key value got decoding error") } // Exclude implicit partitioning columns and hash sharded index columns from @@ -156,8 +156,8 @@ func NewLockNotAvailableError( index, colNames, values, err := DecodeRowInfo(ctx, tableDesc, key, nil, false) if err != nil { - return pgerror.Newf(pgcode.LockNotAvailable, - "%s: decoding err=%s", baseMsg, err) + return pgerror.Wrapf(err, pgcode.LockNotAvailable, + "%s: got decoding error", baseMsg) } return pgerror.Newf(pgcode.LockNotAvailable, diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go index a3b9fd24c2c7..f51a1e3a1922 100644 --- a/pkg/sql/sem/builtins/builtins.go +++ b/pkg/sql/sem/builtins/builtins.go @@ -4895,7 +4895,7 @@ value if you rely on the HLC for accuracy.`, }, }) if err := ctx.Txn.Run(ctx.Context, b); err != nil { - return nil, pgerror.Newf(pgcode.InvalidParameterValue, "message: %s", err) + return nil, pgerror.Wrap(err, pgcode.InvalidParameterValue, "error fetching leaseholder") } resp := b.RawResponse().Responses[0].GetInner().(*roachpb.LeaseInfoResponse) @@ -4990,7 +4990,7 @@ value if you rely on the HLC for accuracy.`, }, }) if err := ctx.Txn.Run(ctx.Context, b); err != nil { - return nil, pgerror.Newf(pgcode.InvalidParameterValue, "message: %s", err) + return nil, pgerror.Wrap(err, pgcode.InvalidParameterValue, "error fetching range stats") } resp := b.RawResponse().Responses[0].GetInner().(*roachpb.RangeStatsResponse).MVCCStats jsonStr, err := gojson.Marshal(&resp) diff --git a/pkg/sql/sem/tree/eval.go b/pkg/sql/sem/tree/eval.go index 806d9f63716c..e66e71eabf6e 100644 --- a/pkg/sql/sem/tree/eval.go +++ b/pkg/sql/sem/tree/eval.go @@ -2983,8 +2983,8 @@ func MatchLikeEscape( like, err := optimizedLikeFunc(pattern, caseInsensitive, escapeRune) if err != nil { - return DBoolFalse, pgerror.Newf( - pgcode.InvalidRegularExpression, "LIKE regexp compilation failed: %v", err) + return DBoolFalse, pgerror.Wrap( + err, pgcode.InvalidRegularExpression, "LIKE regexp compilation failed") } if like == nil { @@ -3008,8 +3008,8 @@ func ConvertLikeToRegexp( key := likeKey{s: pattern, caseInsensitive: caseInsensitive, escape: escape} re, err := ctx.ReCache.GetRegexp(key) if err != nil { - return nil, pgerror.Newf( - pgcode.InvalidRegularExpression, "LIKE regexp compilation failed: %v", err) + return nil, pgerror.Wrap( + err, pgcode.InvalidRegularExpression, "LIKE regexp compilation failed") } return re, nil } @@ -3033,8 +3033,8 @@ func matchLike(ctx *EvalContext, left, right Datum, caseInsensitive bool) (Datum like, err := optimizedLikeFunc(pattern, caseInsensitive, '\\') if err != nil { - return DBoolFalse, pgerror.Newf( - pgcode.InvalidRegularExpression, "LIKE regexp compilation failed: %v", err) + return DBoolFalse, pgerror.Wrap( + err, pgcode.InvalidRegularExpression, "LIKE regexp compilation failed") } if like == nil { diff --git a/pkg/sql/sem/tree/interval.go b/pkg/sql/sem/tree/interval.go index 4b3bea2a627a..4831930137aa 100644 --- a/pkg/sql/sem/tree/interval.go +++ b/pkg/sql/sem/tree/interval.go @@ -67,8 +67,8 @@ func (l *intervalLexer) consumeNum() (int64, bool, float64) { // Try to convert. value, err := strconv.ParseFloat(l.str[start:l.offset], 64) if err != nil { - l.err = pgerror.Newf( - pgcode.InvalidDatetimeFormat, "interval: %v", err) + l.err = pgerror.Wrap( + err, pgcode.InvalidDatetimeFormat, "interval") return 0, false, 0 } decPart = value @@ -108,8 +108,8 @@ func (l *intervalLexer) consumeInt() int64 { x, err := strconv.ParseInt(l.str[start:l.offset], 10, 64) if err != nil { - l.err = pgerror.Newf( - pgcode.InvalidDatetimeFormat, "interval: %v", err) + l.err = pgerror.Wrap( + err, pgcode.InvalidDatetimeFormat, "interval") return 0 } if start == l.offset { diff --git a/pkg/sql/set_zone_config.go b/pkg/sql/set_zone_config.go index 97c80849886c..10b060d268f8 100644 --- a/pkg/sql/set_zone_config.go +++ b/pkg/sql/set_zone_config.go @@ -641,14 +641,12 @@ func (n *setZoneConfigNode) startExec(params runParams) error { // empty, in which case the unmarshaling will be a no-op. This is // innocuous. if err := yaml.UnmarshalStrict([]byte(yamlConfig), &newZone); err != nil { - return pgerror.Newf(pgcode.CheckViolation, - "could not parse zone config: %v", err) + return pgerror.Wrap(err, pgcode.CheckViolation, "could not parse zone config") } // Load settings from YAML into the partial zone as well. if err := yaml.UnmarshalStrict([]byte(yamlConfig), &finalZone); err != nil { - return pgerror.Newf(pgcode.CheckViolation, - "could not parse zone config: %v", err) + return pgerror.Wrap(err, pgcode.CheckViolation, "could not parse zone config") } // Load settings from var = val assignments. If there were no such @@ -739,8 +737,7 @@ func (n *setZoneConfigNode) startExec(params runParams) error { // Finally revalidate everything. Validate only the completeZone config. if err := completeZone.Validate(); err != nil { - return pgerror.Newf(pgcode.CheckViolation, - "could not validate zone config: %v", err) + return pgerror.Wrap(err, pgcode.CheckViolation, "could not validate zone config") } // Finally check for the extra protection partial zone configs would @@ -1064,8 +1061,7 @@ func prepareZoneConfigWrites( } buf, err := protoutil.Marshal(zone) if err != nil { - return nil, pgerror.Newf(pgcode.CheckViolation, - "could not marshal zone config: %v", err) + return nil, pgerror.Wrap(err, pgcode.CheckViolation, "could not marshal zone config") } return &zoneConfigUpdate{id: targetID, value: buf}, nil } diff --git a/pkg/sql/sqlerrors/errors.go b/pkg/sql/sqlerrors/errors.go index d7d6c1631462..c59838f94442 100644 --- a/pkg/sql/sqlerrors/errors.go +++ b/pkg/sql/sqlerrors/errors.go @@ -217,11 +217,7 @@ func NewDependentObjectErrorf(format string, args ...interface{}) error { // NewRangeUnavailableError creates an unavailable range error. func NewRangeUnavailableError(rangeID roachpb.RangeID, origErr error) error { - // TODO(knz): This could should really use errors.Wrap or - // errors.WithSecondaryError. - return pgerror.Newf(pgcode.RangeUnavailable, - "key range id:%d is unavailable. Original error: %v", - rangeID, origErr) + return pgerror.Wrapf(origErr, pgcode.RangeUnavailable, "key range id:%d is unavailable", rangeID) } // NewWindowInAggError creates an error for the case when a window function is diff --git a/pkg/sql/sqlstats/sslocal/BUILD.bazel b/pkg/sql/sqlstats/sslocal/BUILD.bazel index 749d8100d77c..368e1de8744e 100644 --- a/pkg/sql/sqlstats/sslocal/BUILD.bazel +++ b/pkg/sql/sqlstats/sslocal/BUILD.bazel @@ -29,6 +29,7 @@ go_library( "//pkg/util/stop", "//pkg/util/syncutil", "//pkg/util/timeutil", + "@com_github_cockroachdb_errors//:errors", ], ) diff --git a/pkg/sql/sqlstats/sslocal/sslocal_provider.go b/pkg/sql/sqlstats/sslocal/sslocal_provider.go index 2615a67cadde..691ed72982fc 100644 --- a/pkg/sql/sqlstats/sslocal/sslocal_provider.go +++ b/pkg/sql/sqlstats/sslocal/sslocal_provider.go @@ -12,7 +12,6 @@ package sslocal import ( "context" - "fmt" "sort" "time" @@ -28,6 +27,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/mon" "github.com/cockroachdb/cockroach/pkg/util/stop" "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/cockroachdb/errors" ) // New returns an instance of SQLStats. @@ -185,7 +185,7 @@ func (s *SQLStats) IterateAggregatedTransactionStats( err := statsContainer.IterateAggregatedTransactionStats(ctx, options, visitor) if err != nil { - return fmt.Errorf("sql stats iteration abort: %s", err) + return errors.Wrap(err, "sql stats iteration abort") } } diff --git a/pkg/sql/sqlstats/ssmemstorage/ss_mem_storage.go b/pkg/sql/sqlstats/ssmemstorage/ss_mem_storage.go index 7a426e39eea0..30bf14fb08e2 100644 --- a/pkg/sql/sqlstats/ssmemstorage/ss_mem_storage.go +++ b/pkg/sql/sqlstats/ssmemstorage/ss_mem_storage.go @@ -167,7 +167,7 @@ func (s *Container) IterateAggregatedTransactionStats( err := visitor(s.appName, &txnStat) if err != nil { - return fmt.Errorf("sql stats iteration abort: %s", err) + return errors.Wrap(err, "sql stats iteration abort") } return nil diff --git a/pkg/sql/type_change_test.go b/pkg/sql/type_change_test.go index c77bb6f077de..612f95f4c4f8 100644 --- a/pkg/sql/type_change_test.go +++ b/pkg/sql/type_change_test.go @@ -582,7 +582,7 @@ WHERE return errors.New("expected error, found none") } if !testutils.IsError(err, "invalid input value for enum") { - return errors.Newf("expected invalid input for enum error, found %v", err) + return errors.NewAssertionErrorWithWrappedErrf(err, "expected invalid input for enum error") } return nil }) From 2d67843e9b17383f0b54b2a4267070d5e16beb5b Mon Sep 17 00:00:00 2001 From: Steven Danna Date: Wed, 3 Nov 2021 10:51:53 +0000 Subject: [PATCH 3/4] build: fix bazel invocation of stress in github-pull-request-make Every run of the stress and stressrace bazel CI jobs were failing with: [17:16:00][Run stress tests] /bin/bash: stress: command not found I haven't dug into the Git history enough to know why this was working before. Rather, I've just copied what the `dev` tool does for me and checked that the constructed `bazci` command does in fact work. Release note: None --- pkg/cmd/github-pull-request-make/main.go | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/pkg/cmd/github-pull-request-make/main.go b/pkg/cmd/github-pull-request-make/main.go index 6c8ad8f58159..dffabb1e5b46 100644 --- a/pkg/cmd/github-pull-request-make/main.go +++ b/pkg/cmd/github-pull-request-make/main.go @@ -50,6 +50,8 @@ const targetEnv = "TARGET" // We don't want TesticularCancer. const goTestStr = `func (Test[^a-z]\w*)\(.*\*testing\.TB?\) {$` +const bazelStressTarget = "@com_github_cockroachdb_stress//:stress" + var currentGoTestRE = regexp.MustCompile(`.*` + goTestStr) var newGoTestRE = regexp.MustCompile(`^\+\s*` + goTestStr) @@ -254,13 +256,13 @@ func main() { args = append(args, "--test_arg=-test.timeout", fmt.Sprintf("--test_arg=%s", timeout)) // Give the entire test 1 more minute than the duration to wrap up. args = append(args, fmt.Sprintf("--test_timeout=%d", int((duration+1*time.Minute).Seconds()))) - // NB: stress and bazci are expected to be put in `PATH` by the caller. - args = append(args, "--run_under", fmt.Sprintf("stress -stderr -maxfails 1 -maxtime %s -p %d", duration, parallelism)) + args = append(args, "--run_under", fmt.Sprintf("%s -stderr -maxfails 1 -maxtime %s -p %d", bazelStressTarget, duration, parallelism)) if target == "stressrace" { args = append(args, "--config=race") } else { args = append(args, "--test_sharding_strategy=disabled") } + // NB: bazci is expected to be put in `PATH` by the caller. cmd := exec.Command("bazci", args...) cmd.Stdout = os.Stdout cmd.Stderr = os.Stderr From 19402850ad83f1f358c34ae394aa1ce0501cfbf5 Mon Sep 17 00:00:00 2001 From: Steven Danna Date: Wed, 3 Nov 2021 11:30:14 +0000 Subject: [PATCH 4/4] github-pull-request-make: allow testing command generation This adds the ability to feed in package and test names via an environment variable rather than depending on GitHub. It also allows one to force a bazel build. I found this helpful when debugging a problem with this command. Release note: None --- pkg/cmd/github-pull-request-make/main.go | 89 ++++++++++++++++++------ 1 file changed, 68 insertions(+), 21 deletions(-) diff --git a/pkg/cmd/github-pull-request-make/main.go b/pkg/cmd/github-pull-request-make/main.go index dffabb1e5b46..5552dc0440c5 100644 --- a/pkg/cmd/github-pull-request-make/main.go +++ b/pkg/cmd/github-pull-request-make/main.go @@ -31,6 +31,7 @@ import ( "os/exec" "path/filepath" "regexp" + "strconv" "strings" "time" @@ -40,9 +41,16 @@ import ( "golang.org/x/oauth2" ) -const githubAPITokenEnv = "GITHUB_API_TOKEN" -const teamcityVCSNumberEnv = "BUILD_VCS_NUMBER" -const targetEnv = "TARGET" +const ( + githubAPITokenEnv = "GITHUB_API_TOKEN" + teamcityVCSNumberEnv = "BUILD_VCS_NUMBER" + targetEnv = "TARGET" + // The following environment variables are for testing and are + // prefixed with GHM_ to help prevent accidentally triggering + // test code inside the CI pipeline. + packageEnv = "GHM_PACKAGES" + forceBazelEnv = "GHM_FORCE_BAZEL" +) // https://github.com/golang/go/blob/go1.7.3/src/cmd/go/test.go#L1260:L1262 // @@ -59,6 +67,24 @@ type pkg struct { tests []string } +func pkgsFromGithubPRForSHA( + ctx context.Context, org string, repo string, sha string, +) (map[string]pkg, error) { + client := ghClient(ctx) + currentPull := findPullRequest(ctx, client, org, repo, sha) + if currentPull == nil { + log.Printf("SHA %s not found in open pull requests, skipping stress", sha) + return nil, nil + } + + diff, err := getDiff(ctx, client, org, repo, *currentPull.Number) + if err != nil { + return nil, err + } + + return pkgsFromDiff(strings.NewReader(diff)) +} + // pkgsFromDiff parses a git-style diff and returns a mapping from directories // to tests added in those directories in the given diff. func pkgsFromDiff(r io.Reader) (map[string]pkg, error) { @@ -160,6 +186,25 @@ func getDiff( return diff, err } +func parsePackagesFromEnvironment(input string) (map[string]pkg, error) { + const expectedFormat = "PACKAGE_NAME=TEST_NAME[,TEST_NAME...][;PACKAGE_NAME=...]" + pkgTestStrs := strings.Split(input, ";") + pkgs := make(map[string]pkg, len(pkgTestStrs)) + for _, pts := range pkgTestStrs { + ptsParts := strings.Split(pts, "=") + if len(ptsParts) < 2 { + return nil, fmt.Errorf("invalid format for package environment variable: %q (expected format: %s)", + input, expectedFormat) + } + pkgName := ptsParts[0] + tests := ptsParts[1] + pkgs[pkgName] = pkg{ + tests: strings.Split(tests, ","), + } + } + return pkgs, nil +} + func main() { sha, ok := os.LookupEnv(teamcityVCSNumberEnv) if !ok { @@ -174,32 +219,34 @@ func main() { log.Fatalf("environment variable %s is %s; expected 'stress' or 'stressrace'", targetEnv, target) } - const org = "cockroachdb" - const repo = "cockroach" + forceBazel := false + if forceBazelStr, ok := os.LookupEnv(forceBazelEnv); ok { + forceBazel, _ = strconv.ParseBool(forceBazelStr) + } crdb, err := os.Getwd() if err != nil { log.Fatal(err) } - ctx := context.Background() - client := ghClient(ctx) - - currentPull := findPullRequest(ctx, client, org, repo, sha) - if currentPull == nil { - log.Printf("SHA %s not found in open pull requests, skipping stress", sha) - return - } + var pkgs map[string]pkg + if pkgStr, ok := os.LookupEnv(packageEnv); ok { + log.Printf("Using packages from environment variable %s", packageEnv) + pkgs, err = parsePackagesFromEnvironment(pkgStr) + if err != nil { + log.Fatal(err) + } - diff, err := getDiff(ctx, client, org, repo, *currentPull.Number) - if err != nil { - log.Fatal(err) + } else { + ctx := context.Background() + const org = "cockroachdb" + const repo = "cockroach" + pkgs, err = pkgsFromGithubPRForSHA(ctx, org, repo, sha) + if err != nil { + log.Fatal(err) + } } - pkgs, err := pkgsFromDiff(strings.NewReader(diff)) - if err != nil { - log.Fatal(err) - } if len(pkgs) > 0 { for name, pkg := range pkgs { // 20 minutes total seems OK, but at least 2 minutes per test. @@ -224,7 +271,7 @@ func main() { } var args []string - if bazel.BuiltWithBazel() { + if bazel.BuiltWithBazel() || forceBazel { args = append(args, "test") // NB: We use a pretty dumb technique to list the bazel test // targets: we ask bazel query to enumerate all the tests in this