diff --git a/pkg/sql/alter_table.go b/pkg/sql/alter_table.go index 9731d8a17c3d..08abe30c4a56 100644 --- a/pkg/sql/alter_table.go +++ b/pkg/sql/alter_table.go @@ -661,7 +661,7 @@ func (n *alterTableNode) startExec(params runParams) error { "constraint %q in the middle of being added, try again later", t.Constraint) } if err := validateCheckInTxn( - params.ctx, params.p.LeaseMgr(), params.EvalContext(), n.tableDesc, params.EvalContext().Txn, name, + params.ctx, params.p.LeaseMgr(), ¶ms.p.semaCtx, params.EvalContext(), n.tableDesc, params.EvalContext().Txn, name, ); err != nil { return err } diff --git a/pkg/sql/backfill.go b/pkg/sql/backfill.go index 99a11ba910a7..00d2916a7669 100644 --- a/pkg/sql/backfill.go +++ b/pkg/sql/backfill.go @@ -540,9 +540,16 @@ func (sc *SchemaChanger) validateConstraints( } // Each check operates at the historical timestamp. return runHistoricalTxn(ctx, func(ctx context.Context, txn *kv.Txn, evalCtx *extendedEvalContext) error { + // If the constraint is a check constraint that fails validation, we + // need a semaContext set up that can resolve types in order to pretty + // print the check expression back to the user. + evalCtx.Txn = txn + semaCtx := tree.MakeSemaContext() + // Use the DistSQLTypeResolver because we need to resolve types by ID. + semaCtx.TypeResolver = &execinfrapb.DistSQLTypeResolver{EvalContext: &evalCtx.EvalContext} switch c.ConstraintType { case sqlbase.ConstraintToUpdate_CHECK: - if err := validateCheckInTxn(ctx, sc.leaseMgr, &evalCtx.EvalContext, desc, txn, c.Check.Name); err != nil { + if err := validateCheckInTxn(ctx, sc.leaseMgr, &semaCtx, &evalCtx.EvalContext, desc, txn, c.Check.Name); err != nil { return err } case sqlbase.ConstraintToUpdate_FOREIGN_KEY: @@ -550,7 +557,7 @@ func (sc *SchemaChanger) validateConstraints( return err } case sqlbase.ConstraintToUpdate_NOT_NULL: - if err := validateCheckInTxn(ctx, sc.leaseMgr, &evalCtx.EvalContext, desc, txn, c.Check.Name); err != nil { + if err := validateCheckInTxn(ctx, sc.leaseMgr, &semaCtx, &evalCtx.EvalContext, desc, txn, c.Check.Name); err != nil { // TODO (lucy): This should distinguish between constraint // validation errors and other types of unexpected errors, and // return a different error code in the former case @@ -1517,7 +1524,7 @@ func runSchemaChangesInTxn( switch c.ConstraintType { case sqlbase.ConstraintToUpdate_CHECK, sqlbase.ConstraintToUpdate_NOT_NULL: if err := validateCheckInTxn( - ctx, planner.Tables().LeaseManager(), planner.EvalContext(), tableDesc, planner.txn, c.Check.Name, + ctx, planner.Tables().LeaseManager(), &planner.semaCtx, planner.EvalContext(), tableDesc, planner.txn, c.Check.Name, ); err != nil { return err } @@ -1564,6 +1571,7 @@ func runSchemaChangesInTxn( func validateCheckInTxn( ctx context.Context, leaseMgr *lease.Manager, + semaCtx *tree.SemaContext, evalCtx *tree.EvalContext, tableDesc *MutableTableDescriptor, txn *kv.Txn, @@ -1587,7 +1595,7 @@ func validateCheckInTxn( if err != nil { return err } - return validateCheckExpr(ctx, check.Expr, tableDesc.TableDesc(), ie, txn) + return validateCheckExpr(ctx, semaCtx, check.Expr, tableDesc.TableDesc(), ie, txn) } // validateFkInTxn validates foreign key constraints within the provided diff --git a/pkg/sql/check.go b/pkg/sql/check.go index a430648ee9f6..1860cca6d19a 100644 --- a/pkg/sql/check.go +++ b/pkg/sql/check.go @@ -18,9 +18,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" - "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "github.com/cockroachdb/cockroach/pkg/sql/schemaexpr" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/util" @@ -35,12 +35,13 @@ import ( // reuse an existing client.Txn safely. func validateCheckExpr( ctx context.Context, + semaCtx *tree.SemaContext, exprStr string, tableDesc *sqlbase.TableDescriptor, ie *InternalExecutor, txn *kv.Txn, ) error { - expr, err := parser.ParseExpr(exprStr) + expr, err := schemaexpr.DeserializeTableDescExpr(ctx, semaCtx, tableDesc, exprStr) if err != nil { return err } @@ -54,7 +55,7 @@ func validateCheckExpr( lim := &tree.Limit{Count: tree.NewDInt(1)} stmt := &tree.Select{Select: sel, Limit: lim} queryStr := tree.AsStringWithFlags(stmt, tree.FmtParsable) - log.Infof(ctx, "Validating check constraint %q with query %q", expr.String(), queryStr) + log.Infof(ctx, "Validating check constraint %q with query %q", tree.SerializeForDisplay(expr), queryStr) rows, err := ie.QueryRow(ctx, "validate check constraint", txn, queryStr) if err != nil { @@ -63,7 +64,7 @@ func validateCheckExpr( if rows.Len() > 0 { return pgerror.Newf(pgcode.CheckViolation, "validation of CHECK %q failed on row: %s", - expr.String(), labeledRowValues(tableDesc.Columns, rows)) + tree.SerializeForDisplay(expr), labeledRowValues(tableDesc.Columns, rows)) } return nil } @@ -336,7 +337,11 @@ type checkSet = util.FastIntSet // values, as ordinals into ActiveChecks(). There must be exactly one value in // checkVals for each element in checkSet. func checkMutationInput( - tabDesc *sqlbase.ImmutableTableDescriptor, checkOrds checkSet, checkVals tree.Datums, + ctx context.Context, + semaCtx *tree.SemaContext, + tabDesc *sqlbase.ImmutableTableDescriptor, + checkOrds checkSet, + checkVals tree.Datums, ) error { if len(checkVals) < checkOrds.Len() { return errors.AssertionFailedf( @@ -353,9 +358,16 @@ func checkMutationInput( if res, err := tree.GetBool(checkVals[colIdx]); err != nil { return err } else if !res && checkVals[colIdx] != tree.DNull { - // Failed to satisfy CHECK constraint. + // Failed to satisfy CHECK constraint, so unwrap the serialized + // check expression to display to the user. + expr, exprErr := schemaexpr.DeserializeTableDescExpr(ctx, semaCtx, tabDesc.TableDesc(), checks[i].Expr) + if exprErr != nil { + // If we ran into an error trying to read the check constraint, wrap it + // and return. + return errors.Wrapf(exprErr, "failed to satisfy CHECK constraint (%s)", checks[i].Expr) + } return pgerror.Newf( - pgcode.CheckViolation, "failed to satisfy CHECK constraint (%s)", checks[i].Expr, + pgcode.CheckViolation, "failed to satisfy CHECK constraint (%s)", tree.SerializeForDisplay(expr), ) } colIdx++ diff --git a/pkg/sql/insert.go b/pkg/sql/insert.go index 48ae4fe3fb29..0e4bfa501e3a 100644 --- a/pkg/sql/insert.go +++ b/pkg/sql/insert.go @@ -173,7 +173,7 @@ func (r *insertRun) processSourceRow(params runParams, rowVals tree.Datums) erro // Verify the CHECK constraint results, if any. if !r.checkOrds.Empty() { checkVals := rowVals[len(r.insertCols):] - if err := checkMutationInput(r.ti.tableDesc(), r.checkOrds, checkVals); err != nil { + if err := checkMutationInput(params.ctx, ¶ms.p.semaCtx, r.ti.tableDesc(), r.checkOrds, checkVals); err != nil { return err } rowVals = rowVals[:len(r.insertCols)] diff --git a/pkg/sql/logictest/testdata/logic_test/enums b/pkg/sql/logictest/testdata/logic_test/enums index c156495c567d..e75adb56f8fc 100644 --- a/pkg/sql/logictest/testdata/logic_test/enums +++ b/pkg/sql/logictest/testdata/logic_test/enums @@ -586,6 +586,47 @@ enum_checks CREATE TABLE enum_checks ( CONSTRAINT "check" CHECK ('hello':::test.public.greeting = 'hello':::test.public.greeting) ) +# Ensure that we can add check constraints to tables with enums. +statement ok +DROP TABLE enum_checks; +CREATE TABLE enum_checks (x greeting); +INSERT INTO enum_checks VALUES ('hi'), ('howdy'); +ALTER TABLE enum_checks ADD CHECK (x > 'hello') + +# Ensure that checks are validated on insert. +statement error pq: failed to satisfy CHECK constraint \(x > 'hello':::test.public.greeting\) +INSERT INTO enum_checks VALUES ('hello') + +# Try adding a check that fails validation. +statement error pq: validation of CHECK "x = 'hello':::test.public.greeting" failed +ALTER TABLE enum_checks ADD CHECK (x = 'hello') + +# Check the above cases, but in a transaction. +statement ok +DROP TABLE enum_checks; +BEGIN; +CREATE TABLE enum_checks (x greeting); +INSERT INTO enum_checks VALUES ('hi'), ('howdy'); +ALTER TABLE enum_checks ADD CHECK (x > 'hello') + +statement error pq: failed to satisfy CHECK constraint \(x > 'hello':::test.public.greeting\) +INSERT INTO enum_checks VALUES ('hello') + +statement ok +ROLLBACK + +statement ok +BEGIN; +CREATE TABLE enum_checks (x greeting); +INSERT INTO enum_checks VALUES ('hi'), ('howdy'); + +# Try adding a check that fails validation. +statement error pq: validation of CHECK "x = 'hello':::test.public.greeting" failed +ALTER TABLE enum_checks ADD CHECK (x = 'hello') + +statement ok +ROLLBACK + subtest schema_changes # Ensure that we can drop and create indexes on user defined type columns, diff --git a/pkg/sql/update.go b/pkg/sql/update.go index d9fd44e498d4..aa64989d1fae 100644 --- a/pkg/sql/update.go +++ b/pkg/sql/update.go @@ -300,7 +300,7 @@ func (u *updateNode) processSourceRow(params runParams, sourceVals tree.Datums) // contain the results of evaluation. if !u.run.checkOrds.Empty() { checkVals := sourceVals[len(u.run.tu.ru.FetchCols)+len(u.run.tu.ru.UpdateCols)+u.run.numPassthrough:] - if err := checkMutationInput(u.run.tu.tableDesc(), u.run.checkOrds, checkVals); err != nil { + if err := checkMutationInput(params.ctx, ¶ms.p.semaCtx, u.run.tu.tableDesc(), u.run.checkOrds, checkVals); err != nil { return err } } diff --git a/pkg/sql/upsert.go b/pkg/sql/upsert.go index e93f1584274b..d864e60d3623 100644 --- a/pkg/sql/upsert.go +++ b/pkg/sql/upsert.go @@ -161,7 +161,7 @@ func (n *upsertNode) processSourceRow(params runParams, rowVals tree.Datums) err ord++ } checkVals := rowVals[ord:] - if err := checkMutationInput(n.run.tw.tableDesc(), n.run.checkOrds, checkVals); err != nil { + if err := checkMutationInput(params.ctx, ¶ms.p.semaCtx, n.run.tw.tableDesc(), n.run.checkOrds, checkVals); err != nil { return err } rowVals = rowVals[:ord]