diff --git a/docs/generated/sql/bnf/stmt_block.bnf b/docs/generated/sql/bnf/stmt_block.bnf index aabe3117d50e..3431f5046f99 100644 --- a/docs/generated/sql/bnf/stmt_block.bnf +++ b/docs/generated/sql/bnf/stmt_block.bnf @@ -558,6 +558,7 @@ unreserved_keyword ::= | 'DAY' | 'DEALLOCATE' | 'DELETE' + | 'DEFERRED' | 'DISCARD' | 'DOMAIN' | 'DOUBLE' @@ -589,6 +590,7 @@ unreserved_keyword ::= | 'HIGH' | 'HISTOGRAM' | 'HOUR' + | 'IMMEDIATE' | 'IMPORT' | 'INCREMENT' | 'INCREMENTAL' @@ -634,6 +636,7 @@ unreserved_keyword ::= | 'OF' | 'OFF' | 'OID' + | 'OIDS' | 'OIDVECTOR' | 'OPERATOR' | 'OPTION' diff --git a/pkg/server/telemetry/features.go b/pkg/server/telemetry/features.go index e2d891106bfb..cefdff349502 100644 --- a/pkg/server/telemetry/features.go +++ b/pkg/server/telemetry/features.go @@ -18,6 +18,8 @@ import ( "fmt" "sync/atomic" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/syncutil" ) @@ -123,3 +125,36 @@ func GetAndResetFeatureCounts(quantize bool) map[string]int32 { } return m } + +// RecordError takes an error and increments the corresponding count +// for its error code, and, if it is an unimplemented or internal +// error, the count for that feature or the internal error's shortened +// stack trace. +func RecordError(err error) { + if err == nil { + return + } + + if pgErr, ok := pgerror.GetPGCause(err); ok { + Count("errorcodes." + pgErr.Code) + + if details := pgErr.InternalCommand; details != "" { + var prefix string + switch pgErr.Code { + case pgerror.CodeFeatureNotSupportedError: + prefix = "unimplemented." + case pgerror.CodeInternalError: + prefix = "internalerror." + default: + prefix = "othererror." + pgErr.Code + "." + } + Count(prefix + details) + } + } else { + typ := log.ErrorSource(err) + if typ == "" { + typ = "unknown" + } + Count("othererror." + typ) + } +} diff --git a/pkg/server/updates_test.go b/pkg/server/updates_test.go index 4bb2b15fb643..44bb7ef6920e 100644 --- a/pkg/server/updates_test.go +++ b/pkg/server/updates_test.go @@ -296,6 +296,11 @@ func TestReportUsage(t *testing.T) { ) { t.Fatal(err) } + if _, err := db.Exec(`SELECT crdb_internal.set_vmodule('invalid')`); !testutils.IsError( + err, "comma-separated list", + ) { + t.Fatal(err) + } // If the vtable ever gets supported, change to pick one that is not supported yet. if _, err := db.Exec(`SELECT * FROM pg_catalog.pg_stat_wal_receiver`); !testutils.IsError( err, "virtual schema table not implemented", @@ -528,7 +533,11 @@ func TestReportUsage(t *testing.T) { "unimplemented.#9148": 10, "internalerror.": 10, "othererror.builtins.go": 10, + "othererror." + + pgerror.CodeDataExceptionError + + ".crdb_internal.set_vmodule()": 10, "errorcodes.blah": 10, + "errorcodes." + pgerror.CodeDataExceptionError: 10, "errorcodes." + pgerror.CodeInternalError: 10, "errorcodes." + pgerror.CodeSyntaxError: 10, "errorcodes." + pgerror.CodeFeatureNotSupportedError: 10, @@ -665,6 +674,7 @@ func TestReportUsage(t *testing.T) { `[true,false,true] SELECT _ / _`, `[true,false,true] SELECT crdb_internal.force_assertion_error(_)`, `[true,false,true] SELECT crdb_internal.force_error(_, $1)`, + `[true,false,true] SELECT crdb_internal.set_vmodule(_)`, `[true,true,false] SELECT * FROM _ WHERE (_ = _) AND (_ = _)`, `[true,true,false] SELECT * FROM _ WHERE (_ = length($1::STRING)) OR (_ = $2)`, `[true,true,false] SELECT _ FROM _ WHERE (_ = _) AND (lower(_) = lower(_))`, @@ -709,6 +719,7 @@ func TestReportUsage(t *testing.T) { `SELECT _ / _`, `SELECT crdb_internal.force_assertion_error(_)`, `SELECT crdb_internal.force_error(_, $1)`, + `SELECT crdb_internal.set_vmodule(_)`, `SET CLUSTER SETTING "server.time_until_store_dead" = _`, `SET CLUSTER SETTING "diagnostics.reporting.send_crash_reports" = _`, `SET application_name = _`, diff --git a/pkg/sql/conn_executor.go b/pkg/sql/conn_executor.go index 95516964d4e1..64915ef2795d 100644 --- a/pkg/sql/conn_executor.go +++ b/pkg/sql/conn_executor.go @@ -302,37 +302,6 @@ func (s *Server) Start(ctx context.Context, stopper *stop.Stopper) { s.PeriodicallyClearStmtStats(ctx, stopper) } -// recordError takes an error and increments the corresponding count -// for its error code, and, if it is an unimplemented or internal -// error, the count for that feature or the internal error's shortened -// stack trace. -func (s *Server) recordError(err error) { - if err == nil { - return - } - - if pgErr, ok := pgerror.GetPGCause(err); ok { - telemetry.Count("errorcodes." + pgErr.Code) - - switch pgErr.Code { - case pgerror.CodeFeatureNotSupportedError: - if feature := pgErr.InternalCommand; feature != "" { - telemetry.Count("unimplemented." + feature) - } - case pgerror.CodeInternalError: - if trace := pgErr.InternalCommand; trace != "" { - telemetry.Count("internalerror." + trace) - } - } - } else { - typ := log.ErrorSource(err) - if typ == "" { - typ = "unknown" - } - telemetry.Count("othererror." + typ) - } -} - // ResetStatementStats resets the executor's collected statement statistics. func (s *Server) ResetStatementStats(ctx context.Context) { s.sqlStats.resetStats(ctx) @@ -1290,12 +1259,12 @@ func (ex *connExecutor) run( ex.sessionEventf(ex.Ctx(), "execution error: %s", pe.errorCause()) } if resErr == nil && ok { - ex.server.recordError(pe.errorCause()) + telemetry.RecordError(pe.errorCause()) // Depending on whether the result has the error already or not, we have // to call either Close or CloseWithErr. res.CloseWithErr(pe.errorCause()) } else { - ex.server.recordError(resErr) + telemetry.RecordError(resErr) res.Close(stateToTxnStatusIndicator(ex.machine.CurState())) } } else { diff --git a/pkg/sql/conn_executor_exec.go b/pkg/sql/conn_executor_exec.go index 8c5b3220907f..08318c114775 100644 --- a/pkg/sql/conn_executor_exec.go +++ b/pkg/sql/conn_executor_exec.go @@ -26,6 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/internal/client" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/sql/coltypes" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -1207,7 +1208,7 @@ func (ex *connExecutor) runShowSyntax( commErr = res.AddRow(ctx, tree.Datums{tree.NewDString(field), tree.NewDString(msg)}) return nil }, - ex.server.recordError, /* reportErr */ + telemetry.RecordError, /* reportErr */ ); err != nil { res.SetError(err) } diff --git a/pkg/sql/parser/parse_test.go b/pkg/sql/parser/parse_test.go index ded37e5c67d0..e9dcd4befc8f 100644 --- a/pkg/sql/parser/parse_test.go +++ b/pkg/sql/parser/parse_test.go @@ -2548,6 +2548,7 @@ func TestUnimplementedSyntax(t *testing.T) { {`CREATE OPERATOR a`, 0, `create operator`}, {`CREATE PUBLICATION a`, 0, `create publication`}, {`CREATE RULE a`, 0, `create rule`}, + {`CREATE SCHEMA a`, 26443, `create`}, {`CREATE SERVER a`, 0, `create server`}, {`CREATE SUBSCRIPTION a`, 0, `create subscription`}, {`CREATE TEXT SEARCH a`, 7821, `create text`}, @@ -2566,6 +2567,7 @@ func TestUnimplementedSyntax(t *testing.T) { {`DROP OPERATOR a`, 0, `drop operator`}, {`DROP PUBLICATION a`, 0, `drop publication`}, {`DROP RULE a`, 0, `drop rule`}, + {`DROP SCHEMA a`, 26443, `drop`}, {`DROP SERVER a`, 0, `drop server`}, {`DROP SUBSCRIPTION a`, 0, `drop subscription`}, {`DROP TEXT SEARCH a`, 7821, `drop text`}, @@ -2586,11 +2588,26 @@ func TestUnimplementedSyntax(t *testing.T) { {`CREATE TEMP VIEW a AS SELECT b`, 5807, ``}, {`CREATE TEMP SEQUENCE a`, 5807, ``}, + {`CREATE TABLE a(LIKE b)`, 30840, ``}, + + {`CREATE TABLE a(b INT) WITH OIDS`, 0, `create table with oids`}, + {`CREATE TABLE a(b INT) WITH foo = bar`, 0, `create table with foo`}, + + {`CREATE TABLE a AS SELECT b WITH NO DATA`, 0, `create table as with no data`}, + {`CREATE TABLE a(b INT AS (123) VIRTUAL)`, 0, `virtual computed columns`}, {`CREATE TABLE a(b INT REFERENCES c(x) MATCH FULL`, 0, `references match full`}, {`CREATE TABLE a(b INT REFERENCES c(x) MATCH PARTIAL`, 0, `references match partial`}, {`CREATE TABLE a(b INT REFERENCES c(x) MATCH SIMPLE`, 0, `references match simple`}, + {`CREATE TABLE a(b INT, FOREIGN KEY (b) REFERENCES c(x) DEFERRABLE)`, 31632, `deferrable`}, + {`CREATE TABLE a(b INT, FOREIGN KEY (b) REFERENCES c(x) INITIALLY DEFERRED)`, 31632, `initially deferred`}, + {`CREATE TABLE a(b INT, FOREIGN KEY (b) REFERENCES c(x) INITIALLY IMMEDIATE)`, 31632, `initially immediate`}, + {`CREATE TABLE a(b INT, FOREIGN KEY (b) REFERENCES c(x) DEFERRABLE INITIALLY DEFERRED)`, 31632, `initially deferred`}, + {`CREATE TABLE a(b INT, FOREIGN KEY (b) REFERENCES c(x) DEFERRABLE INITIALLY IMMEDIATE)`, 31632, `initially immediate`}, + {`CREATE TABLE a(b INT, UNIQUE (b) DEFERRABLE)`, 31632, `deferrable`}, + {`CREATE TABLE a(b INT, CHECK (b > 0) DEFERRABLE)`, 31632, `deferrable`}, + {`CREATE SEQUENCE a AS DOUBLE PRECISION`, 25110, `FLOAT8`}, {`CREATE SEQUENCE a OWNED BY b`, 26382, ``}, diff --git a/pkg/sql/parser/sql.y b/pkg/sql/parser/sql.y index d9abe58018f2..7b5f0b6da328 100644 --- a/pkg/sql/parser/sql.y +++ b/pkg/sql/parser/sql.y @@ -485,7 +485,7 @@ func newNameFromStr(s string) *tree.Name { %token CURRENT_USER CYCLE %token DATA DATABASE DATABASES DATE DAY DEC DECIMAL DEFAULT -%token DEALLOCATE DEFERRABLE DELETE DESC +%token DEALLOCATE DEFERRABLE DEFERRED DELETE DESC %token DISCARD DISTINCT DO DOMAIN DOUBLE DROP %token ELSE ENCODING END ENUM ESCAPE EXCEPT @@ -502,7 +502,7 @@ func newNameFromStr(s string) *tree.Name { %token HAVING HIGH HISTOGRAM HOUR -%token IMPORT INCREMENT INCREMENTAL IF IFERROR IFNULL ILIKE IN ISERROR +%token IMMEDIATE IMPORT INCREMENT INCREMENTAL IF IFERROR IFNULL ILIKE IN ISERROR %token INET INET_CONTAINED_BY_OR_EQUALS INET_CONTAINS_OR_CONTAINED_BY %token INET_CONTAINS_OR_EQUALS INDEX INDEXES INJECT INTERLEAVE INITIALLY %token INNER INSERT INT INT2VECTOR INT2 INT4 INT8 INT64 INTEGER @@ -521,7 +521,7 @@ func newNameFromStr(s string) *tree.Name { %token NAN NAME NAMES NATURAL NEXT NO NO_INDEX_JOIN NORMAL %token NOT NOTHING NOTNULL NULL NULLIF NUMERIC -%token OF OFF OFFSET OID OIDVECTOR ON ONLY OPTION OPTIONS OR +%token OF OFF OFFSET OID OIDS OIDVECTOR ON ONLY OPTION OPTIONS OR %token ORDER ORDINALITY OUT OUTER OVER OVERLAPS OVERLAY OWNED OPERATOR %token PARENT PARTIAL PARTITION PASSWORD PAUSE PHYSICAL PLACING @@ -1975,6 +1975,7 @@ create_unsupported: | CREATE OPERATOR error { return unimplemented(sqllex, "create operator") } | CREATE PUBLICATION error { return unimplemented(sqllex, "create publication") } | CREATE opt_or_replace RULE error { return unimplemented(sqllex, "create rule") } +| CREATE SCHEMA error { return unimplementedWithIssueDetail(sqllex, 26443, "create") } | CREATE SERVER error { return unimplemented(sqllex, "create server") } | CREATE SUBSCRIPTION error { return unimplemented(sqllex, "create subscription") } | CREATE TEXT error { return unimplementedWithIssueDetail(sqllex, 7821, "create text") } @@ -2007,6 +2008,7 @@ drop_unsupported: | DROP OPERATOR error { return unimplemented(sqllex, "drop operator") } | DROP PUBLICATION error { return unimplemented(sqllex, "drop publication") } | DROP RULE error { return unimplemented(sqllex, "drop rule") } +| DROP SCHEMA error { return unimplementedWithIssueDetail(sqllex, 26443, "drop") } | DROP SERVER error { return unimplemented(sqllex, "drop server") } | DROP SUBSCRIPTION error { return unimplemented(sqllex, "drop subscription") } | DROP TEXT error { return unimplementedWithIssueDetail(sqllex, 7821, "drop text") } @@ -3657,7 +3659,7 @@ pause_stmt: // WEBDOCS/create-table.html // WEBDOCS/create-table-as.html create_table_stmt: - CREATE opt_temp TABLE table_name '(' opt_table_elem_list ')' opt_interleave opt_partition_by + CREATE opt_temp TABLE table_name '(' opt_table_elem_list ')' opt_interleave opt_partition_by opt_table_with { $$.val = &tree.CreateTable{ Table: $4.normalizableTableNameFromUnresolvedName(), @@ -3669,7 +3671,7 @@ create_table_stmt: PartitionBy: $9.partitionBy(), } } -| CREATE opt_temp TABLE IF NOT EXISTS table_name '(' opt_table_elem_list ')' opt_interleave opt_partition_by +| CREATE opt_temp TABLE IF NOT EXISTS table_name '(' opt_table_elem_list ')' opt_interleave opt_partition_by opt_table_with { $$.val = &tree.CreateTable{ Table: $7.normalizableTableNameFromUnresolvedName(), @@ -3682,30 +3684,40 @@ create_table_stmt: } } +opt_table_with: + /* EMPTY */ { /* no error */ } +| WITHOUT OIDS { /* SKIP DOC */ /* this is also the default in CockroachDB */ } +| WITH name error { return unimplemented(sqllex, "create table with " + $2) } + create_table_as_stmt: - CREATE opt_temp TABLE table_name opt_column_list AS select_stmt + CREATE opt_temp TABLE table_name opt_column_list opt_table_with AS select_stmt opt_create_as_data { $$.val = &tree.CreateTable{ Table: $4.normalizableTableNameFromUnresolvedName(), IfNotExists: false, Interleave: nil, Defs: nil, - AsSource: $7.slct(), + AsSource: $8.slct(), AsColumnNames: $5.nameList(), } } -| CREATE opt_temp TABLE IF NOT EXISTS table_name opt_column_list AS select_stmt +| CREATE opt_temp TABLE IF NOT EXISTS table_name opt_column_list opt_table_with AS select_stmt opt_create_as_data { $$.val = &tree.CreateTable{ Table: $7.normalizableTableNameFromUnresolvedName(), IfNotExists: true, Interleave: nil, Defs: nil, - AsSource: $10.slct(), + AsSource: $11.slct(), AsColumnNames: $8.nameList(), } } +opt_create_as_data: + /* EMPTY */ { /* no error */ } +| WITH DATA { /* SKIP DOC */ /* This is the default */ } +| WITH NO DATA { return unimplemented(sqllex, "create table as with no data") } + /* * Redundancy here is needed to avoid shift/reduce conflicts, * since TEMP is not a reserved word. See also OptTempTableName. @@ -3755,6 +3767,7 @@ table_elem: { $$.val = $1.constraintDef() } +| LIKE table_name error { return unimplementedWithIssue(sqllex, 30840) } opt_interleave: INTERLEAVE IN PARENT table_name '(' name_list ')' opt_interleave_drop_behavior @@ -4034,13 +4047,13 @@ table_constraint: } constraint_elem: - CHECK '(' a_expr ')' + CHECK '(' a_expr ')' opt_deferrable { $$.val = &tree.CheckConstraintTableDef{ Expr: $3.expr(), } } -| UNIQUE '(' index_params ')' opt_storing opt_interleave opt_partition_by +| UNIQUE '(' index_params ')' opt_storing opt_interleave opt_partition_by opt_deferrable { $$.val = &tree.UniqueConstraintTableDef{ IndexTableDef: tree.IndexTableDef{ @@ -4061,7 +4074,7 @@ constraint_elem: } } | FOREIGN KEY '(' name_list ')' REFERENCES table_name - opt_column_list key_match reference_actions + opt_column_list key_match reference_actions opt_deferrable { $$.val = &tree.ForeignKeyConstraintTableDef{ Table: $7.normalizableTableNameFromUnresolvedName(), @@ -4071,6 +4084,14 @@ constraint_elem: } } +opt_deferrable: + /* EMPTY */ { /* no error */ } +| DEFERRABLE { return unimplementedWithIssueDetail(sqllex, 31632, "deferrable") } +| DEFERRABLE INITIALLY DEFERRED { return unimplementedWithIssueDetail(sqllex, 31632, "def initially deferred") } +| DEFERRABLE INITIALLY IMMEDIATE { return unimplementedWithIssueDetail(sqllex, 31632, "def initially immediate") } +| INITIALLY DEFERRED { return unimplementedWithIssueDetail(sqllex, 31632, "initially deferred") } +| INITIALLY IMMEDIATE { return unimplementedWithIssueDetail(sqllex, 31632, "initially immediate") } + storing: COVERING | STORING @@ -8355,6 +8376,7 @@ unreserved_keyword: | DAY | DEALLOCATE | DELETE +| DEFERRED | DISCARD | DOMAIN | DOUBLE @@ -8386,6 +8408,7 @@ unreserved_keyword: | HIGH | HISTOGRAM | HOUR +| IMMEDIATE | IMPORT | INCREMENT | INCREMENTAL @@ -8431,6 +8454,7 @@ unreserved_keyword: | OF | OFF | OID +| OIDS | OIDVECTOR | OPERATOR | OPTION diff --git a/pkg/sql/pgwire/command_result.go b/pkg/sql/pgwire/command_result.go index 990a928fb4cb..e0474cbb6700 100644 --- a/pkg/sql/pgwire/command_result.go +++ b/pkg/sql/pgwire/command_result.go @@ -18,13 +18,14 @@ import ( "context" "fmt" + "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgwirebase" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/lib/pq/oid" - "github.com/pkg/errors" ) type completionMsgType int @@ -124,9 +125,11 @@ func (r *commandResult) Close(t sql.TransactionStatusIndicator) { r.typ == commandComplete && r.stmtType == tree.Rows { - r.err = errors.Errorf("execute row count limits not supported: %d of %d", + r.err = pgerror.UnimplementedWithIssueErrorf(4035, + "execute row count limits not supported: %d of %d", r.limit, r.rowsAffected) - r.conn.bufferErr(convertToErrWithPGCode(r.err)) + telemetry.RecordError(r.err) + r.conn.bufferErr(r.err) } // Send a completion message, specific to the type of result. diff --git a/pkg/sql/pgwire/pgerror/errors.go b/pkg/sql/pgwire/pgerror/errors.go index 6c5c82b66f1a..a62ef8a1c0c9 100644 --- a/pkg/sql/pgwire/pgerror/errors.go +++ b/pkg/sql/pgwire/pgerror/errors.go @@ -23,6 +23,7 @@ import ( "github.com/lib/pq" "github.com/cockroachdb/cockroach/pkg/util/caller" + "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/pkg/errors" ) @@ -224,3 +225,38 @@ func UnimplementedWithDepth(depth int, feature, msg string, args ...interface{}) err.Hint = unimplementedErrorHint return err } + +// Wrap wraps an error into a pgerror. The code is used +// if the original error was not a pgerror already. The errContext +// string is used to populate the InternalCommand. If InternalCommand +// already exists, the errContext is prepended. +func Wrap(err error, code, errContext string) error { + var pgErr Error + origErr, ok := GetPGCause(err) + if ok { + // Copy the error. We can't use the existing error directly + // because it may be a global (const) object and we want to modify + // it below. + pgErr = *origErr + } else { + pgErr = Error{ + Code: code, + // Keep the stack trace if one was available in the original + // non-Error error (e.g. when constructed via errors.Wrap). + InternalCommand: log.ErrorSource(err), + } + } + + // Prepend the context to the existing message. + prefix := errContext + ": " + pgErr.Message = prefix + err.Error() + + // Prepend the context also to the internal command, to ensure it + // goes to telemetry. + if pgErr.InternalCommand != "" { + pgErr.InternalCommand = prefix + pgErr.InternalCommand + } else { + pgErr.InternalCommand = errContext + } + return &pgErr +} diff --git a/pkg/sql/sem/tree/eval.go b/pkg/sql/sem/tree/eval.go index 0053192fb30d..6f44cfa8d4d1 100644 --- a/pkg/sql/sem/tree/eval.go +++ b/pkg/sql/sem/tree/eval.go @@ -3418,7 +3418,7 @@ func (expr *FuncExpr) Eval(ctx *EvalContext) (Datum, error) { if fName == `crdb_internal.force_error` { return nil, err } - return nil, errors.Wrapf(err, "%s()", fName) + return nil, pgerror.Wrap(err, pgerror.CodeDataExceptionError, fName+"()") } if ctx.TestingKnobs.AssertFuncExprReturnTypes { if err := ensureExpectedType(expr.fn.FixedReturnType(), res); err != nil {