From cea639ae578a85e6a35b68c38fc8931f082265cc Mon Sep 17 00:00:00 2001 From: Lucy Zhang Date: Mon, 28 Jan 2019 11:41:30 -0500 Subject: [PATCH] sql: validate check constraints with the schema changer Currently, constraints are added in the `Unvalidated` state, and are not validated for existing rows until ALTER TABLE ... VALIDATE CONSTRAINT is run. With this change, check constraints will be validated asynchronously after they are added by default (and similar changes to FKs are to follow). This addresses the problematic long-running transactions caused by the current implementation of VALIDATE CONSTRAINT. This PR is a rework of #32504, and has the same tests. With this change, check constraints will be added to the table descriptor in the new `Validating` state, visible to CRUD operations, and a mutation is queued indicating that the constraint is to be validated. During the backfill step, the constraint is validated for existing rows. If validation succeeds, then the constraint moves to the `Validated` state; otherwise, it is dropped. The behavior when dropping constraints (either via DROP CONSTRAINT or indirectly when a column is dropped) is unchanged: no mutation is enqueued. As part of this change, check constraints can be added to non-public columns in the process of being added, including columns that were created earlier in the same transaction. The main difference between this PR and #32504 is that #32504 does not add the constraint to the table descriptor until it has been validated. Release note (sql change): Check constraint adds by default will validate table data with the added constraint asynchronously after the transaction commits. --- pkg/sql/alter_table.go | 26 +- pkg/sql/backfill.go | 168 +++- pkg/sql/check.go | 11 +- pkg/sql/crdb_internal.go | 3 + pkg/sql/create_table.go | 6 +- .../logictest/testdata/logic_test/alter_table | 23 +- .../testdata/logic_test/schema_change_in_txn | 214 +++++ pkg/sql/opt_catalog.go | 4 +- pkg/sql/schema_changer.go | 36 + pkg/sql/schema_changer_test.go | 37 +- pkg/sql/sqlbase/check.go | 12 +- pkg/sql/sqlbase/structured.go | 73 +- pkg/sql/sqlbase/structured.pb.go | 843 ++++++++++++------ pkg/sql/sqlbase/structured.proto | 15 + pkg/sql/sqlbase/table.go | 3 +- pkg/sql/update.go | 4 +- 16 files changed, 1174 insertions(+), 304 deletions(-) diff --git a/pkg/sql/alter_table.go b/pkg/sql/alter_table.go index bd7188c8bfca..79e4315276ab 100644 --- a/pkg/sql/alter_table.go +++ b/pkg/sql/alter_table.go @@ -213,15 +213,23 @@ func (n *alterTableNode) startExec(params runParams) error { } case *tree.CheckConstraintTableDef: + // A previous command could have added a column which the new constraint uses, + // allocate IDs now. + if err != n.tableDesc.AllocateIDs() { + return err + } + ck, err := MakeCheckConstraint(params.ctx, n.tableDesc, d, inuseNames, ¶ms.p.semaCtx, n.n.Table) if err != nil { return err } - ck.Validity = sqlbase.ConstraintValidity_Unvalidated + ck.Validity = sqlbase.ConstraintValidity_Validating n.tableDesc.Checks = append(n.tableDesc.Checks, ck) descriptorChanged = true + n.tableDesc.AddCheckValidationMutation(ck.Name) + case *tree.ForeignKeyConstraintTableDef: for _, colName := range d.FromCols { col, _, err := n.tableDesc.FindColumnByName(colName) @@ -402,7 +410,11 @@ func (n *alterTableNode) startExec(params runParams) error { for _, check := range n.tableDesc.Checks { if used, err := check.UsesColumn(n.tableDesc.TableDesc(), col.ID); err != nil { return err - } else if !used { + } else if used { + if check.Validity == sqlbase.ConstraintValidity_Validating { + return fmt.Errorf("referencing constraint %q in the middle of being added, try again later", check.Name) + } + } else { validChecks = append(validChecks, check) } } @@ -452,6 +464,9 @@ func (n *alterTableNode) startExec(params runParams) error { return fmt.Errorf("UNIQUE constraint depends on index %q, use DROP INDEX with CASCADE if you really want to drop it", t.Constraint) case sqlbase.ConstraintTypeCheck: for i := range n.tableDesc.Checks { + if n.tableDesc.Checks[i].Validity == sqlbase.ConstraintValidity_Validating { + return fmt.Errorf("constraint %q in the middle of being added, try again later", t.Constraint) + } if n.tableDesc.Checks[i].Name == name { n.tableDesc.Checks = append(n.tableDesc.Checks[:i], n.tableDesc.Checks[i+1:]...) descriptorChanged = true @@ -498,9 +513,14 @@ func (n *alterTableNode) startExec(params runParams) error { if !found { panic("constraint returned by GetConstraintInfo not found") } + + if n.tableDesc.Checks[idx].Validity == sqlbase.ConstraintValidity_Validating { + return fmt.Errorf("constraint %q in the middle of being added, try again later", t.Constraint) + } + ck := n.tableDesc.Checks[idx] if err := validateCheckExpr( - params.ctx, ck.Expr, &n.n.Table, n.tableDesc.TableDesc(), params.EvalContext(), + params.ctx, ck.Expr, n.tableDesc.TableDesc(), params.EvalContext().InternalExecutor, params.EvalContext().Txn, ); err != nil { return err } diff --git a/pkg/sql/backfill.go b/pkg/sql/backfill.go index 08306627b2de..da1daa5000d9 100644 --- a/pkg/sql/backfill.go +++ b/pkg/sql/backfill.go @@ -125,6 +125,8 @@ func (sc *SchemaChanger) runBackfill( var droppedIndexDescs []sqlbase.IndexDescriptor var addedIndexDescs []sqlbase.IndexDescriptor + var checksToValidate []sqlbase.ConstraintToValidate + var tableDesc *sqlbase.TableDescriptor if err := sc.db.Txn(ctx, func(ctx context.Context, txn *client.Txn) error { var err error @@ -156,6 +158,13 @@ func (sc *SchemaChanger) runBackfill( } case *sqlbase.DescriptorMutation_Index: addedIndexDescs = append(addedIndexDescs, *t.Index) + case *sqlbase.DescriptorMutation_Constraint: + switch t.Constraint.ConstraintType { + case sqlbase.ConstraintToValidate_CHECK: + checksToValidate = append(checksToValidate, *t.Constraint) + default: + return errors.Errorf("unsupported constraint type: %d", t.Constraint.ConstraintType) + } default: return errors.Errorf("unsupported mutation: %+v", m) } @@ -168,6 +177,8 @@ func (sc *SchemaChanger) runBackfill( if !sc.canClearRangeForDrop(t.Index) { droppedIndexDescs = append(droppedIndexDescs, *t.Index) } + case *sqlbase.DescriptorMutation_Constraint: + // no-op default: return errors.Errorf("unsupported mutation: %+v", m) } @@ -198,9 +209,113 @@ func (sc *SchemaChanger) runBackfill( } } + // Validate check constraints. + if len(checksToValidate) > 0 { + if err := sc.validateChecks(ctx, evalCtx, lease, checksToValidate); err != nil { + return err + } + } return nil } +func (sc *SchemaChanger) validateChecks( + ctx context.Context, + evalCtx *extendedEvalContext, + lease *sqlbase.TableDescriptor_SchemaChangeLease, + checks []sqlbase.ConstraintToValidate, +) error { + if testDisableTableLeases { + return nil + } + readAsOf := sc.clock.Now() + return sc.db.Txn(ctx, func(ctx context.Context, txn *client.Txn) error { + txn.SetFixedTimestamp(ctx, readAsOf) + tableDesc, err := sqlbase.GetTableDescFromID(ctx, txn, sc.tableID) + if err != nil { + return err + } + + if err := sc.ExtendLease(ctx, lease); err != nil { + return err + } + + grp := ctxgroup.WithContext(ctx) + + // Notify when validation is finished (or has returned an error) for a check. + countDone := make(chan struct{}, len(checks)) + + for _, c := range checks { + grp.GoCtx(func(ctx context.Context) error { + defer func() { countDone <- struct{}{} }() + + // Make the mutations public in a private copy of the descriptor + // and add it to the TableCollection, so that we can use SQL below to perform + // the validation. We wouldn't have needed to do this if we could have + // updated the descriptor and run validation in the same transaction. However, + // our current system is incapable of running long running schema changes + // (the validation can take many minutes). So we pretend that the schema + // has been updated and actually update it in a separate transaction that + // follows this one. + desc, err := sqlbase.NewImmutableTableDescriptor(*tableDesc).MakeFirstMutationPublic() + if err != nil { + return err + } + tc := &TableCollection{leaseMgr: sc.leaseMgr} + // pretend that the schema has been modified. + if err := tc.addUncommittedTable(*desc); err != nil { + return err + } + + // Create a new eval context only because the eval context cannot be shared across many + // goroutines. + newEvalCtx := createSchemaChangeEvalCtx(ctx, readAsOf, evalCtx.Tracing, sc.ieFactory) + // TODO(vivek): This is not a great API. Leaving #34304 open. + ie := newEvalCtx.InternalExecutor.(*SessionBoundInternalExecutor) + ie.impl.tcModifier = tc + defer func() { + ie.impl.tcModifier = nil + }() + + check, err := desc.FindCheckByName(c.Name) + if err != nil { + return err + } + + return validateCheckExpr(ctx, check.Expr, desc.TableDesc(), ie, txn) + }) + } + + // Periodic schema change lease extension. + grp.GoCtx(func(ctx context.Context) error { + count := len(checks) + refreshTimer := timeutil.NewTimer() + defer refreshTimer.Stop() + refreshTimer.Reset(checkpointInterval) + for { + select { + case <-countDone: + count-- + if count == 0 { + // Stop. + return nil + } + + case <-refreshTimer.C: + refreshTimer.Read = true + refreshTimer.Reset(checkpointInterval) + if err := sc.ExtendLease(ctx, lease); err != nil { + return err + } + + case <-ctx.Done(): + return ctx.Err() + } + } + }) + return grp.Wait() + }) +} + func (sc *SchemaChanger) getTableVersion( ctx context.Context, txn *client.Txn, tc *TableCollection, version sqlbase.DescriptorVersion, ) (*sqlbase.ImmutableTableDescriptor, error) { @@ -701,11 +816,14 @@ func runSchemaChangesInTxn( // Only needed because columnBackfillInTxn() backfills // all column mutations. doneColumnBackfill := false + // Checks are validated after all other mutations have been applied. + var checksToValidate []sqlbase.ConstraintToValidate + for _, m := range tableDesc.Mutations { immutDesc := sqlbase.NewImmutableTableDescriptor(*tableDesc.TableDesc()) switch m.Direction { case sqlbase.DescriptorMutation_ADD: - switch m.Descriptor_.(type) { + switch t := m.Descriptor_.(type) { case *sqlbase.DescriptorMutation_Column: if doneColumnBackfill || !sqlbase.ColumnNeedsBackfill(m.GetColumn()) { break @@ -720,6 +838,14 @@ func runSchemaChangesInTxn( return err } + case *sqlbase.DescriptorMutation_Constraint: + switch t.Constraint.ConstraintType { + case sqlbase.ConstraintToValidate_CHECK: + checksToValidate = append(checksToValidate, *t.Constraint) + default: + return errors.Errorf("unsupported constraint type: %d", t.Constraint.ConstraintType) + } + default: return errors.Errorf("unsupported mutation: %+v", m) } @@ -741,6 +867,9 @@ func runSchemaChangesInTxn( return err } + case *sqlbase.DescriptorMutation_Constraint: + return errors.Errorf("constraint validation mutation cannot be in the DROP state within the same transaction: %+v", m) + default: return errors.Errorf("unsupported mutation: %+v", m) } @@ -752,6 +881,43 @@ func runSchemaChangesInTxn( } tableDesc.Mutations = nil + // Now that the table descriptor is in a valid state with all column and index + // mutations applied, it can be used for validating check constraints + return validateChecksInTxn(ctx, tc.leaseMgr, evalCtx, tableDesc, txn, checksToValidate) +} + +// validateCheckInTxn validates check constraints within the provided +// transaction. The table descriptor that is passed in will be used for the +// InternalExecutor that performs the validation query. +func validateChecksInTxn( + ctx context.Context, + leaseMgr *LeaseManager, + evalCtx *tree.EvalContext, + tableDesc *MutableTableDescriptor, + txn *client.Txn, + checks []sqlbase.ConstraintToValidate, +) error { + newTc := &TableCollection{leaseMgr: leaseMgr} + // pretend that the schema has been modified. + if err := newTc.addUncommittedTable(*tableDesc); err != nil { + return err + } + + ie := evalCtx.InternalExecutor.(*SessionBoundInternalExecutor) + ie.impl.tcModifier = newTc + defer func() { + ie.impl.tcModifier = nil + }() + + for _, c := range checks { + check, err := tableDesc.FindCheckByName(c.Name) + if err != nil { + return err + } + if err := validateCheckExpr(ctx, check.Expr, tableDesc.TableDesc(), ie, txn); err != nil { + return err + } + } return nil } diff --git a/pkg/sql/check.go b/pkg/sql/check.go index 39779b952b9c..49fbaabd6909 100644 --- a/pkg/sql/check.go +++ b/pkg/sql/check.go @@ -20,6 +20,7 @@ import ( "fmt" "strings" + "github.com/cockroachdb/cockroach/pkg/internal/client" "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -31,26 +32,28 @@ import ( func validateCheckExpr( ctx context.Context, exprStr string, - tableName tree.TableExpr, tableDesc *sqlbase.TableDescriptor, - evalCtx *tree.EvalContext, + ie tree.SessionBoundInternalExecutor, + txn *client.Txn, ) error { expr, err := parser.ParseExpr(exprStr) if err != nil { return err } // Construct AST and then convert to a string, to avoid problems with escaping the check expression + tblref := tree.TableRef{TableID: int64(tableDesc.ID), As: tree.AliasClause{Alias: "t"}} sel := &tree.SelectClause{ Exprs: sqlbase.ColumnsSelectors(tableDesc.Columns, false /* forUpdateOrDelete */), - From: &tree.From{Tables: tree.TableExprs{tableName}}, + From: &tree.From{Tables: []tree.TableExpr{&tblref}}, Where: &tree.Where{Type: tree.AstWhere, Expr: &tree.NotExpr{Expr: expr}}, } 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) - rows, err := evalCtx.InternalExecutor.QueryRow(ctx, "validate check constraint", evalCtx.Txn, queryStr) + rows, err := ie.QueryRow(ctx, "validate check constraint", txn, queryStr) if err != nil { return err } diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 07b8490251d8..657870b1c497 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -301,6 +301,9 @@ CREATE TABLE crdb_internal.schema_changes ( mutType = "INDEX" targetID = tree.NewDInt(tree.DInt(int64(d.Index.ID))) targetName = tree.NewDString(d.Index.Name) + case *sqlbase.DescriptorMutation_Constraint: + mutType = "CONSTRAINT VALIDATION" + targetName = tree.NewDString(d.Constraint.Name) } if err := addRow( tableID, diff --git a/pkg/sql/create_table.go b/pkg/sql/create_table.go index 6bae62c3406b..544bb1c563c0 100644 --- a/pkg/sql/create_table.go +++ b/pkg/sql/create_table.go @@ -1484,8 +1484,8 @@ func replaceVars( return nil, true, expr } - col, err := desc.FindActiveColumnByName(string(c.ColumnName)) - if err != nil { + col, dropped, err := desc.FindColumnByName(c.ColumnName) + if err != nil || dropped { return fmt.Errorf("column %q not found for constraint %q", c.ColumnName, expr.String()), false, nil } @@ -1533,7 +1533,7 @@ func MakeCheckConstraint( sort.Sort(sqlbase.ColumnIDs(colIDs)) sourceInfo := sqlbase.NewSourceInfoForSingleTable( - tableName, sqlbase.ResultColumnsFromColDescs(desc.Columns), + tableName, sqlbase.ResultColumnsFromColDescs(desc.TableDesc().AllNonDropColumns()), ) sources := sqlbase.MultiSourceInfo{sourceInfo} diff --git a/pkg/sql/logictest/testdata/logic_test/alter_table b/pkg/sql/logictest/testdata/logic_test/alter_table index 263004e9ee9f..57737ff6efec 100644 --- a/pkg/sql/logictest/testdata/logic_test/alter_table +++ b/pkg/sql/logictest/testdata/logic_test/alter_table @@ -105,6 +105,12 @@ ALTER TABLE t DROP CONSTRAINT check_a statement ok INSERT INTO t (a, f) VALUES (-2, 9) +statement error validation of CHECK "a > 0" failed on row: a=-2, f=9, b=NULL, c=NULL +ALTER TABLE t ADD CONSTRAINT check_a CHECK (a > 0) + +statement ok +DELETE FROM t WHERE a = -2 + statement ok ALTER TABLE t ADD CONSTRAINT check_a CHECK (a > 0) @@ -114,7 +120,7 @@ INSERT INTO t (a) VALUES (-3) query TTTTB SHOW CONSTRAINTS FROM t ---- -t check_a CHECK CHECK (a > 0) false +t check_a CHECK CHECK (a > 0) true t fk_f_ref_other FOREIGN KEY FOREIGN KEY (f) REFERENCES other (b) true t foo UNIQUE UNIQUE (b ASC) true t primary PRIMARY KEY PRIMARY KEY (a ASC) true @@ -129,8 +135,8 @@ ALTER TABLE t ADD CHECK (a > 0) query TTTTB SHOW CONSTRAINTS FROM t ---- -t check_a CHECK CHECK (a > 0) false -t check_a1 CHECK CHECK (a > 0) false +t check_a CHECK CHECK (a > 0) true +t check_a1 CHECK CHECK (a > 0) true t fk_f_ref_other FOREIGN KEY FOREIGN KEY (f) REFERENCES other (b) true t foo UNIQUE UNIQUE (b ASC) true t primary PRIMARY KEY PRIMARY KEY (a ASC) true @@ -138,11 +144,12 @@ t primary PRIMARY KEY PRIMARY KEY (a ASC) true statement error constraint "typo" does not exist ALTER TABLE t VALIDATE CONSTRAINT typo -statement error validation of CHECK "a > 0" failed on row: a=-2, f=9, b=NULL, c=NULL -ALTER TABLE t VALIDATE CONSTRAINT check_a +# TODO(erik): re-enable test when unvalidated checks can be added +#statement error validation of CHECK "a > 0" failed on row: a=-2, f=9, b=NULL, c=NULL +#ALTER TABLE t VALIDATE CONSTRAINT check_a -statement ok -DELETE FROM t WHERE a = -2 +#statement ok +#DELETE FROM t WHERE a = -2 statement ok ALTER TABLE t VALIDATE CONSTRAINT check_a @@ -151,7 +158,7 @@ query TTTTB SHOW CONSTRAINTS FROM t ---- t check_a CHECK CHECK (a > 0) true -t check_a1 CHECK CHECK (a > 0) false +t check_a1 CHECK CHECK (a > 0) true t fk_f_ref_other FOREIGN KEY FOREIGN KEY (f) REFERENCES other (b) true t foo UNIQUE UNIQUE (b ASC) true t primary PRIMARY KEY PRIMARY KEY (a ASC) true diff --git a/pkg/sql/logictest/testdata/logic_test/schema_change_in_txn b/pkg/sql/logictest/testdata/logic_test/schema_change_in_txn index f85e34b5bd98..4ee0872e85b6 100644 --- a/pkg/sql/logictest/testdata/logic_test/schema_change_in_txn +++ b/pkg/sql/logictest/testdata/logic_test/schema_change_in_txn @@ -782,3 +782,217 @@ ALTER TABLE orders2 VALIDATE CONSTRAINT fk_product_ref_products statement ok COMMIT + +# Verify that check constraints can be added on columns being added in the same transaction +subtest check_on_add_col + +statement ok +CREATE TABLE check_table (k INT PRIMARY KEY) + +statement ok +INSERT INTO check_table VALUES (1) + +statement ok +BEGIN + +statement ok +ALTER TABLE check_table ADD c INT + +statement ok +ALTER TABLE check_table ADD CONSTRAINT c_0 CHECK (c > 0) + +statement ok +ALTER TABLE check_table ADD d INT DEFAULT 1 + +statement ok +ALTER TABLE check_table ADD CONSTRAINT d_0 CHECK (d > 0) + +statement ok +COMMIT + +statement ok +BEGIN + +statement ok +ALTER TABLE check_table ADD e INT DEFAULT 0 + +statement ok +ALTER TABLE check_table ADD CONSTRAINT e_0 CHECK (e > 0) + +statement error pq: validation of CHECK "e > 0" failed on row: k=1, c=NULL, d=1, e=0 +COMMIT + +# Constraint e_0 was not added +query TTTTB +SHOW CONSTRAINTS FROM check_table +---- +check_table c_0 CHECK CHECK (c > 0) true +check_table d_0 CHECK CHECK (d > 0) true +check_table primary PRIMARY KEY PRIMARY KEY (k ASC) true + +# Adding column e was rolled back +query TTBTTTB +SHOW COLUMNS FROM check_table +---- +k INT8 false NULL · {primary} false +c INT8 true NULL · {} false +d INT8 true 1:::INT8 · {} false + +statement ok +DROP TABLE check_table + +subtest check_constraint_being_added + +statement ok +CREATE TABLE check_table (k INT PRIMARY KEY) + +statement ok +BEGIN + +statement ok +ALTER TABLE check_table ADD f INT + +statement ok +ALTER TABLE check_table ADD CONSTRAINT f_0 CHECK (f > 0) + +statement error pq: constraint "f_0" in the middle of being added +ALTER TABLE check_table DROP CONSTRAINT f_0 + +statement ok +COMMIT + +statement ok +BEGIN + +statement ok +ALTER TABLE check_table ADD g INT + +statement ok +ALTER TABLE check_table ADD CONSTRAINT g_0 CHECK (g > 0) + +statement error pq: referencing constraint "g_0" in the middle of being added +ALTER TABLE check_table DROP COLUMN g + +statement ok +COMMIT + +statement ok +BEGIN + +statement ok +ALTER TABLE check_table ADD h INT + +statement ok +ALTER TABLE check_table ADD CONSTRAINT h_0 CHECK (h > 0) + +statement error pq: constraint "h_0" in the middle of being added +ALTER TABLE check_table VALIDATE CONSTRAINT h_0 + +statement ok +COMMIT + +statement ok +DROP TABLE check_table + +# Test adding a check constraint to a table that was created in the same transaction +subtest check_on_new_table + +# Test multiple successful constraint adds in the same transaction +statement ok +BEGIN + +statement ok +CREATE TABLE check_table (a INT) + +statement ok +INSERT INTO check_table VALUES (0) + +# This validates the constraint for existing rows, because it's in the same txn as CREATE TABLE +statement ok +ALTER TABLE check_table ADD CONSTRAINT ck_a CHECK (a = 0) + +statement ok +ALTER TABLE check_table ADD COLUMN b INT DEFAULT 1 + +# This validates the constraint for existing rows, because it's in the same txn as CREATE TABLE +statement ok +ALTER TABLE check_table ADD CONSTRAINT ck_b CHECK (b > 0) + +# Test ADD COLUMN and ADD CONSTRAINT in the same ALTER TABLE statement +statement ok +ALTER TABLE check_table ADD COLUMN c INT DEFAULT 2, ADD CONSTRAINT ck_c CHECK (c > b) + +statement ok +COMMIT + +# Verify that the constraints had been validated in the above txn +query TTTTB +SHOW CONSTRAINTS FROM check_table +---- +check_table ck_a CHECK CHECK (a = 0) true +check_table ck_b CHECK CHECK (b > 0) true +check_table ck_c CHECK CHECK (c > b) true + +# Also test insert/update to ensure constraint was added in a valid state (with correct column IDs, etc.) + +statement ok +INSERT INTO check_table VALUES (0, 1, 2) + +statement ok +UPDATE check_table SET b = 1 WHERE b IS NULL + +statement ok +DROP TABLE check_table + +# Test when check validation fails + +statement ok +BEGIN + +statement ok +CREATE TABLE check_table (a INT) + +statement ok +INSERT INTO check_table VALUES (0) + +# This validates the constraint for existing rows, because it's in the same txn as CREATE TABLE +statement error pq: validation of CHECK "a > 0" failed on row: a=0 +ALTER TABLE check_table ADD CONSTRAINT ck CHECK (a > 0) + +statement ok +COMMIT + +statement ok +BEGIN + +statement ok +CREATE TABLE check_table (a INT PRIMARY KEY) + +statement ok +INSERT INTO check_table VALUES (0) + +statement ok +ALTER TABLE check_table ADD COLUMN b INT DEFAULT 0 + +# This validates the constraint for existing rows, because it's in the same txn as CREATE TABLE +statement error pq: validation of CHECK "b > 0" failed on row: a=0, b=0 +ALTER TABLE check_table ADD CONSTRAINT ck CHECK (b > 0) + +statement ok +COMMIT + +statement ok +BEGIN + +statement ok +CREATE TABLE check_table (a INT PRIMARY KEY) + +statement ok +INSERT INTO check_table VALUES (0) + +# Test ADD COLUMN and ADD CONSTRAINT in the same ALTER TABLE statement +statement error pq: validation of CHECK "c > 0" failed on row: a=0, c=0 +ALTER TABLE check_table ADD COLUMN c INT DEFAULT 0, ADD CONSTRAINT ck CHECK (c > 0) + +statement ok +COMMIT diff --git a/pkg/sql/opt_catalog.go b/pkg/sql/opt_catalog.go index 64f2120869cc..7bb4b645c4b4 100644 --- a/pkg/sql/opt_catalog.go +++ b/pkg/sql/opt_catalog.go @@ -559,12 +559,12 @@ func (ot *optTable) ensureColMap() { // CheckCount is part of the cat.Table interface. func (ot *optTable) CheckCount() int { - return len(ot.desc.Checks) + return len(ot.desc.AllChecks()) } // Check is part of the cat.Table interface. func (ot *optTable) Check(i int) cat.CheckConstraint { - check := ot.desc.Checks[i] + check := ot.desc.AllChecks()[i] return cat.CheckConstraint(check.Expr) } diff --git a/pkg/sql/schema_changer.go b/pkg/sql/schema_changer.go index b5ff3b988de9..b2199395cfef 100644 --- a/pkg/sql/schema_changer.go +++ b/pkg/sql/schema_changer.go @@ -1136,6 +1136,16 @@ func (sc *SchemaChanger) reverseMutations(ctx context.Context, causingError erro log.Warningf(ctx, "reverse schema change mutation: %+v", mutation) desc.Mutations[i], columns = sc.reverseMutation(mutation, false /*notStarted*/, columns) + // If the mutation is for validating a constraint that is being added, + // drop the constraint because validation has failed + if constraint := mutation.GetConstraint(); constraint != nil && + mutation.Direction == sqlbase.DescriptorMutation_ADD { + log.Warningf(ctx, "dropping constraint %+v", constraint) + if err := sc.dropValidatingConstraint(desc, constraint); err != nil { + return err + } + } + desc.Mutations[i].Rollback = true } @@ -1284,6 +1294,32 @@ func (sc *SchemaChanger) createRollbackJob( return nil, fmt.Errorf("no job found for table %d mutation %d", sc.tableID, sc.mutationID) } +func (sc *SchemaChanger) dropValidatingConstraint( + desc *MutableTableDescriptor, constraint *sqlbase.ConstraintToValidate, +) error { + switch constraint.ConstraintType { + case sqlbase.ConstraintToValidate_CHECK: + check, err := desc.FindCheckByName(constraint.Name) + if err != nil { + return err + } + // This isn't possible right now, but will be when it becomes possible to add constraints as NOT VALID + if check.Validity == sqlbase.ConstraintValidity_Unvalidated { + break + } + + for j, c := range desc.Checks { + if c.Name == constraint.Name { + desc.Checks = append(desc.Checks[:j], desc.Checks[j+1:]...) + break + } + } + default: + return errors.Errorf("unsupported constraint type: %d", constraint.ConstraintType) + } + return nil +} + // deleteIndexMutationsWithReversedColumns deletes mutations with a // different mutationID than the schema changer and with an index that // references one of the reversed columns. Execute this as a breadth diff --git a/pkg/sql/schema_changer_test.go b/pkg/sql/schema_changer_test.go index 01ac613a3119..42fa94e30404 100644 --- a/pkg/sql/schema_changer_test.go +++ b/pkg/sql/schema_changer_test.go @@ -1223,7 +1223,8 @@ func dropIndexSchemaChange( } } -// TestDropColumn tests that dropped columns properly drop their Table's CHECK constraints +// TestDropColumn tests that dropped columns properly drop their Table's CHECK constraints, +// or an error occurs if a CHECK constraint is being added on it. func TestDropColumn(t *testing.T) { defer leaktest.AfterTest(t)() params, _ := tests.CreateTestServerParams() @@ -1267,6 +1268,22 @@ CREATE TABLE t.test ( if tableDesc.Checks[0].Name != "check_ab" { t.Fatalf("Only check_ab should remain, got: %s ", tableDesc.Checks[0].Name) } + + // Test that a constraint being added prevents the column from being dropped. + txn, err := sqlDB.Begin() + if err != nil { + t.Fatal(err) + } + if _, err := txn.Exec(`ALTER TABLE t.test ADD CONSTRAINT check_bk CHECK (b >= k)`); err != nil { + t.Fatal(err) + } + if _, err := txn.Exec(`ALTER TABLE t.test DROP b`); !testutils.IsError(err, + "referencing constraint \"check_bk\" in the middle of being added") { + t.Fatalf("err = %+v", err) + } + if err := txn.Rollback(); err != nil { + t.Fatal(err) + } } // Test schema changes are retried and complete properly. This also checks @@ -2558,6 +2575,8 @@ INSERT INTO t.kv VALUES ('a', 'b'); {`select-create`, `SELECT * FROM t.kv`, `CREATE INDEX bar ON t.kv (v)`, ``}, {`index-on-add-col`, `ALTER TABLE t.kv ADD i INT`, `CREATE INDEX foobar ON t.kv (i)`, ``}, + {`check-on-add-col`, `ALTER TABLE t.kv ADD j INT`, + `ALTER TABLE t.kv ADD CONSTRAINT ck_j CHECK (j >= 0)`, ``}, } for _, testCase := range testCases { @@ -3417,7 +3436,7 @@ CREATE DATABASE t; t.Fatal(err) } - if _, err := tx.Exec(`CREATE TABLE t.testing (k INT PRIMARY KEY, v INT, INDEX foo(v));`); err != nil { + if _, err := tx.Exec(`CREATE TABLE t.testing (k INT PRIMARY KEY, v INT, INDEX foo(v), CONSTRAINT ck_k CHECK (k >= 0));`); err != nil { t.Fatal(err) } @@ -3434,9 +3453,9 @@ CREATE DATABASE t; t.Fatal(err) } - // Run schema changes that are execute Column and Index backfills. + // Run schema changes that execute Column, Check and Index backfills. if _, err := tx.Exec(` -ALTER TABLE t.test ADD COLUMN c INT AS (v + 4) STORED, ADD COLUMN d INT DEFAULT 23, ADD CONSTRAINT bar UNIQUE (c) +ALTER TABLE t.test ADD COLUMN c INT AS (v + 4) STORED, ADD COLUMN d INT DEFAULT 23, ADD CONSTRAINT bar UNIQUE (c), DROP CONSTRAINT ck_k, ADD CONSTRAINT ck_c CHECK (c >= 4) `); err != nil { t.Fatal(err) } @@ -3488,6 +3507,16 @@ ALTER TABLE t.test ADD COLUMN c INT AS (v + 4) STORED, ADD COLUMN d INT DEFAULT t.Fatalf("read the wrong number of rows: e = %d, v = %d", eCount, count) } + // Constraint ck_k dropped, ck_c public. + if _, err := sqlDB.Exec(fmt.Sprintf("INSERT INTO t.test (k, v) VALUES (-1, %d)", maxValue+10)); err != nil { + t.Fatal(err) + } + q := fmt.Sprintf("INSERT INTO t.test (k, v) VALUES (%d, -1)", maxValue+10) + if _, err := sqlDB.Exec(q); !testutils.IsError(err, + `failed to satisfy CHECK constraint \(c >= 4\)`) { + t.Fatalf("err = %+v", err) + } + // The descriptor version hasn't changed. tableDesc := sqlbase.GetTableDescriptor(kvDB, "t", "test") if tableDesc.Version != 1 { diff --git a/pkg/sql/sqlbase/check.go b/pkg/sql/sqlbase/check.go index 322f5b108352..563f1f5f876f 100644 --- a/pkg/sql/sqlbase/check.go +++ b/pkg/sql/sqlbase/check.go @@ -74,7 +74,7 @@ type AnalyzeExprFunction func( func NewEvalCheckHelper( ctx context.Context, analyzeExpr AnalyzeExprFunction, tableDesc *ImmutableTableDescriptor, ) (*CheckHelper, error) { - if len(tableDesc.Checks) == 0 { + if len(tableDesc.AllChecks()) == 0 { return nil, nil } @@ -85,9 +85,9 @@ func NewEvalCheckHelper( ResultColumnsFromColDescs(tableDesc.Columns), ) - c.Exprs = make([]tree.TypedExpr, len(tableDesc.Checks)) - exprStrings := make([]string, len(tableDesc.Checks)) - for i, check := range tableDesc.Checks { + c.Exprs = make([]tree.TypedExpr, len(tableDesc.AllChecks())) + exprStrings := make([]string, len(tableDesc.AllChecks())) + for i, check := range tableDesc.AllChecks() { exprStrings[i] = check.Expr } exprs, err := parser.ParseExprs(exprStrings) @@ -213,7 +213,7 @@ func (c *CheckHelper) CheckInput(checkVals tree.Datums) error { "mismatched check constraint columns: expected %d, got %d", c.checkSet.Len(), len(checkVals)) } - for i := range c.tableDesc.Checks { + for i, check := range c.tableDesc.AllChecks() { if !c.checkSet.Contains(i) { continue } @@ -223,7 +223,7 @@ func (c *CheckHelper) CheckInput(checkVals tree.Datums) error { } else if !res && checkVals[i] != tree.DNull { // Failed to satisfy CHECK constraint. return pgerror.NewErrorf(pgerror.CodeCheckViolationError, - "failed to satisfy CHECK constraint (%s)", c.tableDesc.Checks[i].Expr) + "failed to satisfy CHECK constraint (%s)", check.Expr) } } return nil diff --git a/pkg/sql/sqlbase/structured.go b/pkg/sql/sqlbase/structured.go index 52d700fb36a0..e3520473f4b9 100644 --- a/pkg/sql/sqlbase/structured.go +++ b/pkg/sql/sqlbase/structured.go @@ -130,6 +130,8 @@ type ImmutableTableDescriptor struct { writeOnlyColCount int writeOnlyIndexCount int + allChecks []TableDescriptor_CheckConstraint + // ReadableColumns is a list of columns (including those undergoing a schema change) // which can be scanned. Columns in the process of a schema change // are all set to nullable while column backfilling is still in @@ -252,6 +254,11 @@ func NewImmutableTableDescriptor(tbl TableDescriptor) *ImmutableTableDescriptor desc.publicAndNonPublicCols = publicAndNonPublicCols desc.publicAndNonPublicIndexes = publicAndNonPublicIndexes + desc.allChecks = make([]TableDescriptor_CheckConstraint, len(tbl.Checks)) + for i, c := range tbl.Checks { + desc.allChecks[i] = *c + } + return desc } @@ -535,9 +542,9 @@ func (desc *TableDescriptor) KeysPerRow(indexID IndexID) int { return 1 } -// allNonDropColumns returns all the columns, including those being added +// AllNonDropColumns returns all the columns, including those being added // in the mutations. -func (desc *TableDescriptor) allNonDropColumns() []ColumnDescriptor { +func (desc *TableDescriptor) AllNonDropColumns() []ColumnDescriptor { cols := make([]ColumnDescriptor, 0, len(desc.Columns)+len(desc.Mutations)) cols = append(cols, desc.Columns...) for _, m := range desc.Mutations { @@ -1217,7 +1224,7 @@ func (desc *TableDescriptor) ValidateTable(st *cluster.Settings) error { columnNames := make(map[string]ColumnID, len(desc.Columns)) columnIDs := make(map[ColumnID]string, len(desc.Columns)) - for _, column := range desc.allNonDropColumns() { + for _, column := range desc.AllNonDropColumns() { if err := validateName(column.Name, "column"); err != nil { return err } @@ -1272,6 +1279,10 @@ func (desc *TableDescriptor) ValidateTable(st *cluster.Settings) error { idx := desc.Index return errors.Errorf("mutation in state %s, direction %s, index %s, id %v", m.State, m.Direction, idx.Name, idx.ID) } + case *DescriptorMutation_Constraint: + if unSetEnums { + return errors.Errorf("mutation in state %s, direction %s, constraint %v", m.State, m.Direction, desc.Constraint.Name) + } default: return errors.Errorf("mutation in state %s, direction %s, and no column/index descriptor", m.State, m.Direction) } @@ -1719,7 +1730,7 @@ func notIndexableError(cols []ColumnDescriptor, inverted bool) error { func checkColumnsValidForIndex(tableDesc *MutableTableDescriptor, indexColNames []string) error { invalidColumns := make([]ColumnDescriptor, 0, len(indexColNames)) for _, indexCol := range indexColNames { - for _, col := range tableDesc.allNonDropColumns() { + for _, col := range tableDesc.AllNonDropColumns() { if col.Name == indexCol { if !columnTypeIsIndexable(col.Type) { invalidColumns = append(invalidColumns, col) @@ -1741,7 +1752,7 @@ func checkColumnsValidForInvertedIndex( } invalidColumns := make([]ColumnDescriptor, 0, len(indexColNames)) for _, indexCol := range indexColNames { - for _, col := range tableDesc.allNonDropColumns() { + for _, col := range tableDesc.AllNonDropColumns() { if col.Name == indexCol { if !columnTypeIsInvertedIndexable(col.Type) { invalidColumns = append(invalidColumns, col) @@ -2052,6 +2063,18 @@ func (desc *TableDescriptor) FindIndexByName(name string) (*IndexDescriptor, boo return nil, false, fmt.Errorf("index %q does not exist", name) } +// FindCheckByName finds the check constraint with the specified name. +func (desc *TableDescriptor) FindCheckByName( + name string, +) (*TableDescriptor_CheckConstraint, error) { + for _, c := range desc.Checks { + if c.Name == name { + return c, nil + } + } + return nil, fmt.Errorf("check %q does not exist", name) +} + // RenameIndexDescriptor renames an index descriptor. func (desc *MutableTableDescriptor) RenameIndexDescriptor( index *IndexDescriptor, name string, @@ -2160,6 +2183,19 @@ func (desc *MutableTableDescriptor) MakeMutationComplete(m DescriptorMutation) e if err := desc.AddIndex(*t.Index, false); err != nil { return err } + + case *DescriptorMutation_Constraint: + switch t.Constraint.ConstraintType { + case ConstraintToValidate_CHECK: + for _, c := range desc.Checks { + if c.Name == t.Constraint.Name { + c.Validity = ConstraintValidity_Validated + break + } + } + default: + return errors.Errorf("unsupported constraint type: %d", t.Constraint.ConstraintType) + } } case DescriptorMutation_DROP: @@ -2173,6 +2209,19 @@ func (desc *MutableTableDescriptor) MakeMutationComplete(m DescriptorMutation) e return nil } +// AddCheckValidationMutation adds a check constraint mutation to desc.Mutations. +func (desc *MutableTableDescriptor) AddCheckValidationMutation(name string) { + m := DescriptorMutation{ + Descriptor_: &DescriptorMutation_Constraint{ + Constraint: &ConstraintToValidate{ + ConstraintType: ConstraintToValidate_CHECK, Name: name, + }, + }, + Direction: DescriptorMutation_ADD, + } + desc.addMutation(m) +} + // AddColumnMutation adds a column mutation to desc.Mutations. func (desc *MutableTableDescriptor) AddColumnMutation( c ColumnDescriptor, direction DescriptorMutation_Direction, @@ -2227,6 +2276,7 @@ func (desc *ImmutableTableDescriptor) MakeFirstMutationPublic() (*MutableTableDe // Clone the ImmutableTable descriptor because we want to create an Immutable one. table := NewMutableExistingTableDescriptor(*protoutil.Clone(desc.TableDesc()).(*TableDescriptor)) mutationID := desc.Mutations[0].MutationID + i := 0 for _, mutation := range desc.Mutations { if mutation.MutationID != mutationID { // Mutations are applied in a FIFO order. Only apply the first set @@ -2236,7 +2286,9 @@ func (desc *ImmutableTableDescriptor) MakeFirstMutationPublic() (*MutableTableDe if err := table.MakeMutationComplete(mutation); err != nil { return nil, err } + i++ } + table.Mutations = table.Mutations[i:] table.Version++ return table, nil } @@ -2505,8 +2557,8 @@ func (cc *TableDescriptor_CheckConstraint) ColumnsUsed(desc *TableDescriptor) ([ return err, false, nil } if c, ok := v.(*tree.ColumnItem); ok { - col, err := desc.FindActiveColumnByName(string(c.ColumnName)) - if err != nil { + col, dropped, err := desc.FindColumnByName(c.ColumnName) + if err != nil || dropped { return errors.Errorf("column %q not found for constraint %q", c.ColumnName, parsed.String()), false, nil } @@ -2728,7 +2780,7 @@ func (desc *TableDescriptor) FindAllReferences() (map[ID]struct{}, error) { return nil, err } - for _, c := range desc.allNonDropColumns() { + for _, c := range desc.AllNonDropColumns() { for _, id := range c.UsesSequenceIds { refs[id] = struct{}{} } @@ -2744,6 +2796,11 @@ func (desc *TableDescriptor) FindAllReferences() (map[ID]struct{}, error) { return refs, nil } +// AllChecks returns a list of all check constraints (including constraints being validated). +func (desc *ImmutableTableDescriptor) AllChecks() []TableDescriptor_CheckConstraint { + return desc.allChecks +} + // WritableColumns returns a list of public and write-only mutation columns. func (desc *ImmutableTableDescriptor) WritableColumns() []ColumnDescriptor { return desc.publicAndNonPublicCols[:len(desc.Columns)+desc.writeOnlyColCount] diff --git a/pkg/sql/sqlbase/structured.pb.go b/pkg/sql/sqlbase/structured.pb.go index 5c9fca900d60..a098d63475ac 100644 --- a/pkg/sql/sqlbase/structured.pb.go +++ b/pkg/sql/sqlbase/structured.pb.go @@ -12,6 +12,8 @@ import github_com_cockroachdb_cockroach_pkg_roachpb "github.com/cockroachdb/cock import io "io" +import github_com_gogo_protobuf_proto "github.com/gogo/protobuf/proto" + // Reference imports to suppress errors if they are not otherwise used. var _ = proto.Marshal var _ = fmt.Errorf @@ -26,17 +28,25 @@ const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package type ConstraintValidity int32 const ( - ConstraintValidity_Validated ConstraintValidity = 0 + // The constraint is valid for all rows. + ConstraintValidity_Validated ConstraintValidity = 0 + // The constraint has not yet been validated for all rows (and will not be + // validated until VALIDATE CONSTRAINT is used). ConstraintValidity_Unvalidated ConstraintValidity = 1 + // The constraint was just added, but the validation for existing rows is not + // yet complete. If validation fails, the constraint will be dropped. + ConstraintValidity_Validating ConstraintValidity = 2 ) var ConstraintValidity_name = map[int32]string{ 0: "Validated", 1: "Unvalidated", + 2: "Validating", } var ConstraintValidity_value = map[string]int32{ "Validated": 0, "Unvalidated": 1, + "Validating": 2, } func (x ConstraintValidity) Enum() *ConstraintValidity { @@ -56,7 +66,7 @@ func (x *ConstraintValidity) UnmarshalJSON(data []byte) error { return nil } func (ConstraintValidity) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_096aadf1685143bc, []int{0} + return fileDescriptor_structured_1bcacc6943392042, []int{0} } // These mirror the types supported by sql/coltypes. @@ -164,7 +174,7 @@ func (x *ColumnType_SemanticType) UnmarshalJSON(data []byte) error { return nil } func (ColumnType_SemanticType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_096aadf1685143bc, []int{0, 0} + return fileDescriptor_structured_1bcacc6943392042, []int{0, 0} } type ColumnType_VisibleType int32 @@ -224,7 +234,7 @@ func (x *ColumnType_VisibleType) UnmarshalJSON(data []byte) error { return nil } func (ColumnType_VisibleType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_096aadf1685143bc, []int{0, 1} + return fileDescriptor_structured_1bcacc6943392042, []int{0, 1} } type ForeignKeyReference_Action int32 @@ -269,7 +279,7 @@ func (x *ForeignKeyReference_Action) UnmarshalJSON(data []byte) error { return nil } func (ForeignKeyReference_Action) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_096aadf1685143bc, []int{1, 0} + return fileDescriptor_structured_1bcacc6943392042, []int{1, 0} } // Match is the algorithm used to compare composite keys. @@ -306,7 +316,7 @@ func (x *ForeignKeyReference_Match) UnmarshalJSON(data []byte) error { return nil } func (ForeignKeyReference_Match) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_096aadf1685143bc, []int{1, 1} + return fileDescriptor_structured_1bcacc6943392042, []int{1, 1} } // The direction of a column in the index. @@ -343,7 +353,7 @@ func (x *IndexDescriptor_Direction) UnmarshalJSON(data []byte) error { return nil } func (IndexDescriptor_Direction) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_096aadf1685143bc, []int{6, 0} + return fileDescriptor_structured_1bcacc6943392042, []int{6, 0} } // The direction of a column in the index. @@ -380,7 +390,40 @@ func (x *IndexDescriptor_Type) UnmarshalJSON(data []byte) error { return nil } func (IndexDescriptor_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_096aadf1685143bc, []int{6, 1} + return fileDescriptor_structured_1bcacc6943392042, []int{6, 1} +} + +type ConstraintToValidate_ConstraintType int32 + +const ( + ConstraintToValidate_CHECK ConstraintToValidate_ConstraintType = 0 +) + +var ConstraintToValidate_ConstraintType_name = map[int32]string{ + 0: "CHECK", +} +var ConstraintToValidate_ConstraintType_value = map[string]int32{ + "CHECK": 0, +} + +func (x ConstraintToValidate_ConstraintType) Enum() *ConstraintToValidate_ConstraintType { + p := new(ConstraintToValidate_ConstraintType) + *p = x + return p +} +func (x ConstraintToValidate_ConstraintType) String() string { + return proto.EnumName(ConstraintToValidate_ConstraintType_name, int32(x)) +} +func (x *ConstraintToValidate_ConstraintType) UnmarshalJSON(data []byte) error { + value, err := proto.UnmarshalJSONEnum(ConstraintToValidate_ConstraintType_value, data, "ConstraintToValidate_ConstraintType") + if err != nil { + return err + } + *x = ConstraintToValidate_ConstraintType(value) + return nil +} +func (ConstraintToValidate_ConstraintType) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_structured_1bcacc6943392042, []int{7, 0} } // A descriptor within a mutation is unavailable for reads, writes @@ -445,7 +488,7 @@ func (x *DescriptorMutation_State) UnmarshalJSON(data []byte) error { return nil } func (DescriptorMutation_State) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_096aadf1685143bc, []int{7, 0} + return fileDescriptor_structured_1bcacc6943392042, []int{8, 0} } // Direction of mutation. @@ -488,7 +531,7 @@ func (x *DescriptorMutation_Direction) UnmarshalJSON(data []byte) error { return nil } func (DescriptorMutation_Direction) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_096aadf1685143bc, []int{7, 1} + return fileDescriptor_structured_1bcacc6943392042, []int{8, 1} } // State is set if this TableDescriptor is in the process of being added or deleted. @@ -535,7 +578,7 @@ func (x *TableDescriptor_State) UnmarshalJSON(data []byte) error { return nil } func (TableDescriptor_State) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_096aadf1685143bc, []int{8, 0} + return fileDescriptor_structured_1bcacc6943392042, []int{9, 0} } // AuditMode indicates which auditing actions to take when this table is used. @@ -572,7 +615,7 @@ func (x *TableDescriptor_AuditMode) UnmarshalJSON(data []byte) error { return nil } func (TableDescriptor_AuditMode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_096aadf1685143bc, []int{8, 1} + return fileDescriptor_structured_1bcacc6943392042, []int{9, 1} } type ColumnType struct { @@ -604,7 +647,7 @@ func (m *ColumnType) Reset() { *m = ColumnType{} } func (m *ColumnType) String() string { return proto.CompactTextString(m) } func (*ColumnType) ProtoMessage() {} func (*ColumnType) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_096aadf1685143bc, []int{0} + return fileDescriptor_structured_1bcacc6943392042, []int{0} } func (m *ColumnType) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -650,7 +693,7 @@ func (m *ForeignKeyReference) Reset() { *m = ForeignKeyReference{} } func (m *ForeignKeyReference) String() string { return proto.CompactTextString(m) } func (*ForeignKeyReference) ProtoMessage() {} func (*ForeignKeyReference) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_096aadf1685143bc, []int{1} + return fileDescriptor_structured_1bcacc6943392042, []int{1} } func (m *ForeignKeyReference) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -697,7 +740,7 @@ func (m *ColumnDescriptor) Reset() { *m = ColumnDescriptor{} } func (m *ColumnDescriptor) String() string { return proto.CompactTextString(m) } func (*ColumnDescriptor) ProtoMessage() {} func (*ColumnDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_096aadf1685143bc, []int{2} + return fileDescriptor_structured_1bcacc6943392042, []int{2} } func (m *ColumnDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -750,7 +793,7 @@ func (m *ColumnFamilyDescriptor) Reset() { *m = ColumnFamilyDescriptor{} func (m *ColumnFamilyDescriptor) String() string { return proto.CompactTextString(m) } func (*ColumnFamilyDescriptor) ProtoMessage() {} func (*ColumnFamilyDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_096aadf1685143bc, []int{3} + return fileDescriptor_structured_1bcacc6943392042, []int{3} } func (m *ColumnFamilyDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -798,7 +841,7 @@ func (m *InterleaveDescriptor) Reset() { *m = InterleaveDescriptor{} } func (m *InterleaveDescriptor) String() string { return proto.CompactTextString(m) } func (*InterleaveDescriptor) ProtoMessage() {} func (*InterleaveDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_096aadf1685143bc, []int{4} + return fileDescriptor_structured_1bcacc6943392042, []int{4} } func (m *InterleaveDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -844,7 +887,7 @@ func (m *InterleaveDescriptor_Ancestor) Reset() { *m = InterleaveDescrip func (m *InterleaveDescriptor_Ancestor) String() string { return proto.CompactTextString(m) } func (*InterleaveDescriptor_Ancestor) ProtoMessage() {} func (*InterleaveDescriptor_Ancestor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_096aadf1685143bc, []int{4, 0} + return fileDescriptor_structured_1bcacc6943392042, []int{4, 0} } func (m *InterleaveDescriptor_Ancestor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -891,7 +934,7 @@ func (m *PartitioningDescriptor) Reset() { *m = PartitioningDescriptor{} func (m *PartitioningDescriptor) String() string { return proto.CompactTextString(m) } func (*PartitioningDescriptor) ProtoMessage() {} func (*PartitioningDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_096aadf1685143bc, []int{5} + return fileDescriptor_structured_1bcacc6943392042, []int{5} } func (m *PartitioningDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -936,7 +979,7 @@ func (m *PartitioningDescriptor_List) Reset() { *m = PartitioningDescrip func (m *PartitioningDescriptor_List) String() string { return proto.CompactTextString(m) } func (*PartitioningDescriptor_List) ProtoMessage() {} func (*PartitioningDescriptor_List) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_096aadf1685143bc, []int{5, 0} + return fileDescriptor_structured_1bcacc6943392042, []int{5, 0} } func (m *PartitioningDescriptor_List) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -983,7 +1026,7 @@ func (m *PartitioningDescriptor_Range) Reset() { *m = PartitioningDescri func (m *PartitioningDescriptor_Range) String() string { return proto.CompactTextString(m) } func (*PartitioningDescriptor_Range) ProtoMessage() {} func (*PartitioningDescriptor_Range) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_096aadf1685143bc, []int{5, 1} + return fileDescriptor_structured_1bcacc6943392042, []int{5, 1} } func (m *PartitioningDescriptor_Range) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1117,7 +1160,7 @@ func (m *IndexDescriptor) Reset() { *m = IndexDescriptor{} } func (m *IndexDescriptor) String() string { return proto.CompactTextString(m) } func (*IndexDescriptor) ProtoMessage() {} func (*IndexDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_096aadf1685143bc, []int{6} + return fileDescriptor_structured_1bcacc6943392042, []int{6} } func (m *IndexDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1142,6 +1185,42 @@ func (m *IndexDescriptor) XXX_DiscardUnknown() { var xxx_messageInfo_IndexDescriptor proto.InternalMessageInfo +type ConstraintToValidate struct { + ConstraintType ConstraintToValidate_ConstraintType `protobuf:"varint,1,req,name=constraint_type,json=constraintType,enum=cockroach.sql.sqlbase.ConstraintToValidate_ConstraintType" json:"constraint_type"` + Name string `protobuf:"bytes,2,req,name=name" json:"name"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ConstraintToValidate) Reset() { *m = ConstraintToValidate{} } +func (m *ConstraintToValidate) String() string { return proto.CompactTextString(m) } +func (*ConstraintToValidate) ProtoMessage() {} +func (*ConstraintToValidate) Descriptor() ([]byte, []int) { + return fileDescriptor_structured_1bcacc6943392042, []int{7} +} +func (m *ConstraintToValidate) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ConstraintToValidate) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ConstraintToValidate) XXX_Merge(src proto.Message) { + xxx_messageInfo_ConstraintToValidate.Merge(dst, src) +} +func (m *ConstraintToValidate) XXX_Size() int { + return m.Size() +} +func (m *ConstraintToValidate) XXX_DiscardUnknown() { + xxx_messageInfo_ConstraintToValidate.DiscardUnknown(m) +} + +var xxx_messageInfo_ConstraintToValidate proto.InternalMessageInfo + // A DescriptorMutation represents a column or an index that // has either been added or dropped and hasn't yet transitioned // into a stable state: completely backfilled and visible, or @@ -1152,6 +1231,7 @@ type DescriptorMutation struct { // Types that are valid to be assigned to Descriptor_: // *DescriptorMutation_Column // *DescriptorMutation_Index + // *DescriptorMutation_Constraint Descriptor_ isDescriptorMutation_Descriptor_ `protobuf_oneof:"descriptor"` State DescriptorMutation_State `protobuf:"varint,3,opt,name=state,enum=cockroach.sql.sqlbase.DescriptorMutation_State" json:"state"` Direction DescriptorMutation_Direction `protobuf:"varint,4,opt,name=direction,enum=cockroach.sql.sqlbase.DescriptorMutation_Direction" json:"direction"` @@ -1170,7 +1250,7 @@ func (m *DescriptorMutation) Reset() { *m = DescriptorMutation{} } func (m *DescriptorMutation) String() string { return proto.CompactTextString(m) } func (*DescriptorMutation) ProtoMessage() {} func (*DescriptorMutation) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_096aadf1685143bc, []int{7} + return fileDescriptor_structured_1bcacc6943392042, []int{8} } func (m *DescriptorMutation) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1207,9 +1287,13 @@ type DescriptorMutation_Column struct { type DescriptorMutation_Index struct { Index *IndexDescriptor `protobuf:"bytes,2,opt,name=index,oneof"` } +type DescriptorMutation_Constraint struct { + Constraint *ConstraintToValidate `protobuf:"bytes,8,opt,name=constraint,oneof"` +} -func (*DescriptorMutation_Column) isDescriptorMutation_Descriptor_() {} -func (*DescriptorMutation_Index) isDescriptorMutation_Descriptor_() {} +func (*DescriptorMutation_Column) isDescriptorMutation_Descriptor_() {} +func (*DescriptorMutation_Index) isDescriptorMutation_Descriptor_() {} +func (*DescriptorMutation_Constraint) isDescriptorMutation_Descriptor_() {} func (m *DescriptorMutation) GetDescriptor_() isDescriptorMutation_Descriptor_ { if m != nil { @@ -1232,11 +1316,19 @@ func (m *DescriptorMutation) GetIndex() *IndexDescriptor { return nil } +func (m *DescriptorMutation) GetConstraint() *ConstraintToValidate { + if x, ok := m.GetDescriptor_().(*DescriptorMutation_Constraint); ok { + return x.Constraint + } + return nil +} + // XXX_OneofFuncs is for the internal use of the proto package. func (*DescriptorMutation) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error, func(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error), func(msg proto.Message) (n int), []interface{}) { return _DescriptorMutation_OneofMarshaler, _DescriptorMutation_OneofUnmarshaler, _DescriptorMutation_OneofSizer, []interface{}{ (*DescriptorMutation_Column)(nil), (*DescriptorMutation_Index)(nil), + (*DescriptorMutation_Constraint)(nil), } } @@ -1254,6 +1346,11 @@ func _DescriptorMutation_OneofMarshaler(msg proto.Message, b *proto.Buffer) erro if err := b.EncodeMessage(x.Index); err != nil { return err } + case *DescriptorMutation_Constraint: + _ = b.EncodeVarint(8<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.Constraint); err != nil { + return err + } case nil: default: return fmt.Errorf("DescriptorMutation.Descriptor_ has unexpected type %T", x) @@ -1280,6 +1377,14 @@ func _DescriptorMutation_OneofUnmarshaler(msg proto.Message, tag, wire int, b *p err := b.DecodeMessage(msg) m.Descriptor_ = &DescriptorMutation_Index{msg} return true, err + case 8: // descriptor.constraint + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(ConstraintToValidate) + err := b.DecodeMessage(msg) + m.Descriptor_ = &DescriptorMutation_Constraint{msg} + return true, err default: return false, nil } @@ -1299,6 +1404,11 @@ func _DescriptorMutation_OneofSizer(msg proto.Message) (n int) { n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s + case *DescriptorMutation_Constraint: + s := proto.Size(x.Constraint) + n += 1 // tag and wire + n += proto.SizeVarint(uint64(s)) + n += s case nil: default: panic(fmt.Sprintf("proto: unexpected type %T in oneof", x)) @@ -1424,7 +1534,7 @@ func (m *TableDescriptor) Reset() { *m = TableDescriptor{} } func (m *TableDescriptor) String() string { return proto.CompactTextString(m) } func (*TableDescriptor) ProtoMessage() {} func (*TableDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_096aadf1685143bc, []int{8} + return fileDescriptor_structured_1bcacc6943392042, []int{9} } func (m *TableDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1676,7 +1786,7 @@ func (m *TableDescriptor_SchemaChangeLease) Reset() { *m = TableDescript func (m *TableDescriptor_SchemaChangeLease) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_SchemaChangeLease) ProtoMessage() {} func (*TableDescriptor_SchemaChangeLease) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_096aadf1685143bc, []int{8, 0} + return fileDescriptor_structured_1bcacc6943392042, []int{9, 0} } func (m *TableDescriptor_SchemaChangeLease) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1715,7 +1825,7 @@ func (m *TableDescriptor_CheckConstraint) Reset() { *m = TableDescriptor func (m *TableDescriptor_CheckConstraint) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_CheckConstraint) ProtoMessage() {} func (*TableDescriptor_CheckConstraint) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_096aadf1685143bc, []int{8, 1} + return fileDescriptor_structured_1bcacc6943392042, []int{9, 1} } func (m *TableDescriptor_CheckConstraint) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1820,7 +1930,7 @@ func (m *TableDescriptor_NameInfo) Reset() { *m = TableDescriptor_NameIn func (m *TableDescriptor_NameInfo) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_NameInfo) ProtoMessage() {} func (*TableDescriptor_NameInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_096aadf1685143bc, []int{8, 2} + return fileDescriptor_structured_1bcacc6943392042, []int{9, 2} } func (m *TableDescriptor_NameInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1862,7 +1972,7 @@ func (m *TableDescriptor_Reference) Reset() { *m = TableDescriptor_Refer func (m *TableDescriptor_Reference) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_Reference) ProtoMessage() {} func (*TableDescriptor_Reference) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_096aadf1685143bc, []int{8, 3} + return fileDescriptor_structured_1bcacc6943392042, []int{9, 3} } func (m *TableDescriptor_Reference) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1901,7 +2011,7 @@ func (m *TableDescriptor_MutationJob) Reset() { *m = TableDescriptor_Mut func (m *TableDescriptor_MutationJob) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_MutationJob) ProtoMessage() {} func (*TableDescriptor_MutationJob) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_096aadf1685143bc, []int{8, 4} + return fileDescriptor_structured_1bcacc6943392042, []int{9, 4} } func (m *TableDescriptor_MutationJob) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1945,7 +2055,7 @@ func (m *TableDescriptor_SequenceOpts) Reset() { *m = TableDescriptor_Se func (m *TableDescriptor_SequenceOpts) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_SequenceOpts) ProtoMessage() {} func (*TableDescriptor_SequenceOpts) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_096aadf1685143bc, []int{8, 5} + return fileDescriptor_structured_1bcacc6943392042, []int{9, 5} } func (m *TableDescriptor_SequenceOpts) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1981,7 +2091,7 @@ func (m *TableDescriptor_Replacement) Reset() { *m = TableDescriptor_Rep func (m *TableDescriptor_Replacement) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_Replacement) ProtoMessage() {} func (*TableDescriptor_Replacement) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_096aadf1685143bc, []int{8, 6} + return fileDescriptor_structured_1bcacc6943392042, []int{9, 6} } func (m *TableDescriptor_Replacement) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2020,7 +2130,7 @@ func (m *TableDescriptor_GCDescriptorMutation) Reset() { *m = TableDescr func (m *TableDescriptor_GCDescriptorMutation) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_GCDescriptorMutation) ProtoMessage() {} func (*TableDescriptor_GCDescriptorMutation) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_096aadf1685143bc, []int{8, 7} + return fileDescriptor_structured_1bcacc6943392042, []int{9, 7} } func (m *TableDescriptor_GCDescriptorMutation) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2061,7 +2171,7 @@ func (m *DatabaseDescriptor) Reset() { *m = DatabaseDescriptor{} } func (m *DatabaseDescriptor) String() string { return proto.CompactTextString(m) } func (*DatabaseDescriptor) ProtoMessage() {} func (*DatabaseDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_096aadf1685143bc, []int{9} + return fileDescriptor_structured_1bcacc6943392042, []int{10} } func (m *DatabaseDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2121,7 +2231,7 @@ func (m *Descriptor) Reset() { *m = Descriptor{} } func (m *Descriptor) String() string { return proto.CompactTextString(m) } func (*Descriptor) ProtoMessage() {} func (*Descriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_096aadf1685143bc, []int{10} + return fileDescriptor_structured_1bcacc6943392042, []int{11} } func (m *Descriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2268,6 +2378,7 @@ func init() { proto.RegisterType((*PartitioningDescriptor_List)(nil), "cockroach.sql.sqlbase.PartitioningDescriptor.List") proto.RegisterType((*PartitioningDescriptor_Range)(nil), "cockroach.sql.sqlbase.PartitioningDescriptor.Range") proto.RegisterType((*IndexDescriptor)(nil), "cockroach.sql.sqlbase.IndexDescriptor") + proto.RegisterType((*ConstraintToValidate)(nil), "cockroach.sql.sqlbase.ConstraintToValidate") proto.RegisterType((*DescriptorMutation)(nil), "cockroach.sql.sqlbase.DescriptorMutation") proto.RegisterType((*TableDescriptor)(nil), "cockroach.sql.sqlbase.TableDescriptor") proto.RegisterType((*TableDescriptor_SchemaChangeLease)(nil), "cockroach.sql.sqlbase.TableDescriptor.SchemaChangeLease") @@ -2287,6 +2398,7 @@ func init() { proto.RegisterEnum("cockroach.sql.sqlbase.ForeignKeyReference_Match", ForeignKeyReference_Match_name, ForeignKeyReference_Match_value) proto.RegisterEnum("cockroach.sql.sqlbase.IndexDescriptor_Direction", IndexDescriptor_Direction_name, IndexDescriptor_Direction_value) proto.RegisterEnum("cockroach.sql.sqlbase.IndexDescriptor_Type", IndexDescriptor_Type_name, IndexDescriptor_Type_value) + proto.RegisterEnum("cockroach.sql.sqlbase.ConstraintToValidate_ConstraintType", ConstraintToValidate_ConstraintType_name, ConstraintToValidate_ConstraintType_value) proto.RegisterEnum("cockroach.sql.sqlbase.DescriptorMutation_State", DescriptorMutation_State_name, DescriptorMutation_State_value) proto.RegisterEnum("cockroach.sql.sqlbase.DescriptorMutation_Direction", DescriptorMutation_Direction_name, DescriptorMutation_Direction_value) proto.RegisterEnum("cockroach.sql.sqlbase.TableDescriptor_State", TableDescriptor_State_name, TableDescriptor_State_value) @@ -2928,6 +3040,31 @@ func (m *IndexDescriptor) MarshalTo(dAtA []byte) (int, error) { return i, nil } +func (m *ConstraintToValidate) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ConstraintToValidate) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0x8 + i++ + i = encodeVarintStructured(dAtA, i, uint64(m.ConstraintType)) + dAtA[i] = 0x12 + i++ + i = encodeVarintStructured(dAtA, i, uint64(len(m.Name))) + i += copy(dAtA[i:], m.Name) + return i, nil +} + func (m *DescriptorMutation) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -2998,6 +3135,20 @@ func (m *DescriptorMutation_Index) MarshalTo(dAtA []byte) (int, error) { } return i, nil } +func (m *DescriptorMutation_Constraint) MarshalTo(dAtA []byte) (int, error) { + i := 0 + if m.Constraint != nil { + dAtA[i] = 0x42 + i++ + i = encodeVarintStructured(dAtA, i, uint64(m.Constraint.Size())) + n9, err := m.Constraint.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n9 + } + return i, nil +} func (m *TableDescriptor) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -3029,11 +3180,11 @@ func (m *TableDescriptor) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x3a i++ i = encodeVarintStructured(dAtA, i, uint64(m.ModificationTime.Size())) - n9, err := m.ModificationTime.MarshalTo(dAtA[i:]) + n10, err := m.ModificationTime.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n9 + i += n10 if len(m.Columns) > 0 { for _, msg := range m.Columns { dAtA[i] = 0x42 @@ -3052,11 +3203,11 @@ func (m *TableDescriptor) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x52 i++ i = encodeVarintStructured(dAtA, i, uint64(m.PrimaryIndex.Size())) - n10, err := m.PrimaryIndex.MarshalTo(dAtA[i:]) + n11, err := m.PrimaryIndex.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n10 + i += n11 if len(m.Indexes) > 0 { for _, msg := range m.Indexes { dAtA[i] = 0x5a @@ -3076,11 +3227,11 @@ func (m *TableDescriptor) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x6a i++ i = encodeVarintStructured(dAtA, i, uint64(m.Privileges.Size())) - n11, err := m.Privileges.MarshalTo(dAtA[i:]) + n12, err := m.Privileges.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n11 + i += n12 } if len(m.Mutations) > 0 { for _, msg := range m.Mutations { @@ -3098,11 +3249,11 @@ func (m *TableDescriptor) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x7a i++ i = encodeVarintStructured(dAtA, i, uint64(m.Lease.Size())) - n12, err := m.Lease.MarshalTo(dAtA[i:]) + n13, err := m.Lease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n12 + i += n13 } dAtA[i] = 0x80 i++ @@ -3215,11 +3366,11 @@ func (m *TableDescriptor) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintStructured(dAtA, i, uint64(m.SequenceOpts.Size())) - n13, err := m.SequenceOpts.MarshalTo(dAtA[i:]) + n14, err := m.SequenceOpts.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n13 + i += n14 } dAtA[i] = 0xe8 i++ @@ -3231,11 +3382,11 @@ func (m *TableDescriptor) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintStructured(dAtA, i, uint64(m.ReplacementOf.Size())) - n14, err := m.ReplacementOf.MarshalTo(dAtA[i:]) + n15, err := m.ReplacementOf.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n14 + i += n15 dAtA[i] = 0xf8 i++ dAtA[i] = 0x1 @@ -3462,11 +3613,11 @@ func (m *TableDescriptor_Replacement) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintStructured(dAtA, i, uint64(m.Time.Size())) - n15, err := m.Time.MarshalTo(dAtA[i:]) + n16, err := m.Time.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n15 + i += n16 return i, nil } @@ -3523,11 +3674,11 @@ func (m *DatabaseDescriptor) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1a i++ i = encodeVarintStructured(dAtA, i, uint64(m.Privileges.Size())) - n16, err := m.Privileges.MarshalTo(dAtA[i:]) + n17, err := m.Privileges.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n16 + i += n17 } return i, nil } @@ -3548,11 +3699,11 @@ func (m *Descriptor) MarshalTo(dAtA []byte) (int, error) { var l int _ = l if m.Union != nil { - nn17, err := m.Union.MarshalTo(dAtA[i:]) + nn18, err := m.Union.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += nn17 + i += nn18 } return i, nil } @@ -3563,11 +3714,11 @@ func (m *Descriptor_Table) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintStructured(dAtA, i, uint64(m.Table.Size())) - n18, err := m.Table.MarshalTo(dAtA[i:]) + n19, err := m.Table.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n18 + i += n19 } return i, nil } @@ -3577,11 +3728,11 @@ func (m *Descriptor_Database) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintStructured(dAtA, i, uint64(m.Database.Size())) - n19, err := m.Database.MarshalTo(dAtA[i:]) + n20, err := m.Database.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n19 + i += n20 } return i, nil } @@ -3858,6 +4009,18 @@ func (m *IndexDescriptor) Size() (n int) { return n } +func (m *ConstraintToValidate) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += 1 + sovStructured(uint64(m.ConstraintType)) + l = len(m.Name) + n += 1 + l + sovStructured(uint64(l)) + return n +} + func (m *DescriptorMutation) Size() (n int) { if m == nil { return 0 @@ -3898,6 +4061,18 @@ func (m *DescriptorMutation_Index) Size() (n int) { } return n } +func (m *DescriptorMutation_Constraint) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Constraint != nil { + l = m.Constraint.Size() + n += 1 + l + sovStructured(uint64(l)) + } + return n +} func (m *TableDescriptor) Size() (n int) { if m == nil { return 0 @@ -6517,6 +6692,113 @@ func (m *IndexDescriptor) Unmarshal(dAtA []byte) error { } return nil } +func (m *ConstraintToValidate) Unmarshal(dAtA []byte) error { + var hasFields [1]uint64 + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowStructured + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ConstraintToValidate: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ConstraintToValidate: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ConstraintType", wireType) + } + m.ConstraintType = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowStructured + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ConstraintType |= (ConstraintToValidate_ConstraintType(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + hasFields[0] |= uint64(0x00000001) + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowStructured + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthStructured + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Name = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + hasFields[0] |= uint64(0x00000002) + default: + iNdEx = preIndex + skippy, err := skipStructured(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthStructured + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + if hasFields[0]&uint64(0x00000001) == 0 { + return github_com_gogo_protobuf_proto.NewRequiredNotSetError("constraint_type") + } + if hasFields[0]&uint64(0x00000002) == 0 { + return github_com_gogo_protobuf_proto.NewRequiredNotSetError("name") + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *DescriptorMutation) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 @@ -6687,6 +6969,38 @@ func (m *DescriptorMutation) Unmarshal(dAtA []byte) error { } } m.Rollback = bool(v != 0) + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Constraint", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowStructured + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthStructured + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + v := &ConstraintToValidate{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Descriptor_ = &DescriptorMutation_Constraint{v} + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipStructured(dAtA[iNdEx:]) @@ -8923,208 +9237,213 @@ var ( ) func init() { - proto.RegisterFile("sql/sqlbase/structured.proto", fileDescriptor_structured_096aadf1685143bc) -} - -var fileDescriptor_structured_096aadf1685143bc = []byte{ - // 3170 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x59, 0xcd, 0x6f, 0x1b, 0x47, - 0x96, 0x57, 0x93, 0x4d, 0xb2, 0xf9, 0xf8, 0xd5, 0x2a, 0xcb, 0x0e, 0xcd, 0x38, 0x92, 0xcc, 0xc4, - 0x59, 0xe5, 0x8b, 0x72, 0xe4, 0xec, 0x6e, 0x90, 0x2c, 0x82, 0x25, 0xd9, 0x2d, 0xbb, 0x65, 0x8a, - 0x94, 0x9b, 0x94, 0x1c, 0x07, 0xd9, 0x25, 0x5a, 0xec, 0x92, 0xd4, 0x71, 0xb3, 0x9b, 0xee, 0x6e, - 0x2a, 0xd2, 0x7f, 0x90, 0xd3, 0x62, 0x31, 0x87, 0x99, 0x5b, 0x10, 0x04, 0x73, 0x08, 0x30, 0xd7, - 0x39, 0xcc, 0x9f, 0xe0, 0xb9, 0x0d, 0x72, 0x9a, 0xcb, 0x18, 0x33, 0x1a, 0x0c, 0x30, 0xb7, 0xb9, - 0x07, 0x18, 0x60, 0x50, 0xd5, 0x55, 0xcd, 0xa6, 0xbe, 0x42, 0xd9, 0x37, 0xf6, 0xab, 0xf7, 0x7e, - 0x55, 0xef, 0xd5, 0x7b, 0xbf, 0x7a, 0x55, 0x84, 0x5b, 0xfe, 0x33, 0x7b, 0xd5, 0x7f, 0x66, 0xef, - 0x1a, 0x3e, 0x5e, 0xf5, 0x03, 0x6f, 0x3c, 0x08, 0xc6, 0x1e, 0x36, 0x6b, 0x23, 0xcf, 0x0d, 0x5c, - 0x74, 0x7d, 0xe0, 0x0e, 0x9e, 0x7a, 0xae, 0x31, 0x38, 0xa8, 0xf9, 0xcf, 0xec, 0x1a, 0xd3, 0xab, - 0x94, 0xc7, 0x81, 0x65, 0xaf, 0x1e, 0xd8, 0x83, 0xd5, 0xc0, 0x1a, 0x62, 0x3f, 0x30, 0x86, 0xa3, - 0xd0, 0xa0, 0xf2, 0x7a, 0x1c, 0x6e, 0xe4, 0x59, 0x87, 0x96, 0x8d, 0xf7, 0x31, 0x1b, 0x5c, 0xd8, - 0x77, 0xf7, 0x5d, 0xfa, 0x73, 0x95, 0xfc, 0x0a, 0xa5, 0xd5, 0x1f, 0x33, 0x00, 0x4d, 0xd7, 0x1e, - 0x0f, 0x9d, 0xde, 0xf1, 0x08, 0xa3, 0x27, 0x50, 0xf0, 0xf1, 0xd0, 0x70, 0x02, 0x6b, 0xd0, 0x0f, - 0x8e, 0x47, 0xb8, 0x2c, 0x2c, 0x0b, 0x2b, 0xc5, 0xb5, 0x5a, 0xed, 0xdc, 0xa5, 0xd4, 0x26, 0x96, - 0xb5, 0x2e, 0x33, 0x23, 0x1f, 0x0d, 0xf1, 0xf9, 0x8b, 0xa5, 0x39, 0x3d, 0xef, 0xc7, 0x64, 0xa8, - 0x02, 0xa9, 0xaf, 0x2d, 0x33, 0x38, 0x28, 0x27, 0x96, 0x85, 0x95, 0x14, 0x53, 0x09, 0x45, 0xa8, - 0x0a, 0xd9, 0x91, 0x87, 0x07, 0x96, 0x6f, 0xb9, 0x4e, 0x39, 0x19, 0x1b, 0x9f, 0x88, 0xd1, 0x3b, - 0x20, 0x1b, 0x9e, 0x67, 0x1c, 0xf7, 0x4d, 0x6b, 0x88, 0x1d, 0x22, 0xf2, 0xcb, 0xe2, 0x72, 0x72, - 0x25, 0xa5, 0x97, 0xa8, 0x5c, 0x89, 0xc4, 0xe8, 0x06, 0xa4, 0x6d, 0x77, 0x60, 0xd8, 0xb8, 0x9c, - 0x5a, 0x16, 0x56, 0xb2, 0x3a, 0xfb, 0x42, 0x3b, 0x90, 0x3f, 0xb4, 0x7c, 0x6b, 0xd7, 0xc6, 0xa1, - 0x73, 0x69, 0xea, 0xdc, 0x07, 0x3f, 0xef, 0xdc, 0x4e, 0x68, 0x15, 0xf3, 0x2d, 0x77, 0x38, 0x11, - 0xa1, 0x6d, 0x28, 0x86, 0x4b, 0x1b, 0xb8, 0x4e, 0x80, 0x9d, 0xc0, 0x2f, 0x67, 0x5e, 0x26, 0x6c, - 0x7a, 0x81, 0xa2, 0x34, 0x19, 0x08, 0x6a, 0x43, 0x31, 0x18, 0x8f, 0x6c, 0x3c, 0x81, 0x95, 0x96, - 0x93, 0x2b, 0xb9, 0xb5, 0xdb, 0x3f, 0x0b, 0xcb, 0x16, 0x59, 0xa0, 0xe6, 0x11, 0xde, 0x6d, 0xc8, - 0x87, 0x78, 0xb6, 0xb1, 0x8b, 0x6d, 0xbf, 0x9c, 0x5d, 0x4e, 0xae, 0x64, 0xf5, 0x1c, 0x95, 0xb5, - 0xa8, 0xa8, 0xfa, 0x43, 0x02, 0xf2, 0xf1, 0x25, 0x21, 0x09, 0xc4, 0x46, 0xa7, 0xd3, 0x92, 0xe7, - 0x50, 0x06, 0x92, 0x5a, 0xbb, 0x27, 0x0b, 0x28, 0x0b, 0xa9, 0xf5, 0x56, 0xa7, 0xde, 0x93, 0x13, - 0x28, 0x07, 0x19, 0x45, 0x6d, 0x6a, 0x9b, 0xf5, 0x96, 0x9c, 0x24, 0xaa, 0x4a, 0xbd, 0xa7, 0xca, - 0x22, 0x2a, 0x40, 0xb6, 0xa7, 0x6d, 0xaa, 0xdd, 0x5e, 0x7d, 0x73, 0x4b, 0x4e, 0xa1, 0x3c, 0x48, - 0x5a, 0xbb, 0xa7, 0xea, 0x3b, 0xf5, 0x96, 0x9c, 0x46, 0x00, 0xe9, 0x6e, 0x4f, 0xd7, 0xda, 0xf7, - 0xe5, 0x0c, 0x81, 0x6a, 0x3c, 0xe9, 0xa9, 0x5d, 0x59, 0x42, 0x25, 0xc8, 0x45, 0x36, 0xbd, 0x2f, - 0xe4, 0x2c, 0x42, 0x50, 0x6c, 0x76, 0x5a, 0xad, 0x7a, 0x4f, 0x55, 0x98, 0x3e, 0x90, 0x29, 0xda, - 0xf5, 0x4d, 0x55, 0xce, 0x91, 0xd5, 0x74, 0x34, 0x45, 0xce, 0x53, 0xd1, 0x76, 0xab, 0x25, 0x17, - 0xc8, 0xaf, 0xed, 0x6d, 0x4d, 0x91, 0x8b, 0x04, 0xb6, 0xae, 0xeb, 0xf5, 0x27, 0x72, 0x89, 0x08, - 0xb5, 0xb6, 0xda, 0x93, 0x65, 0xf2, 0x8b, 0x4c, 0x20, 0xcf, 0x93, 0xe1, 0x8d, 0x6e, 0xa7, 0xdd, - 0x90, 0x11, 0xf9, 0xd9, 0xdb, 0xde, 0x6a, 0xa9, 0xf2, 0x02, 0x41, 0x6c, 0x68, 0x3d, 0xf9, 0x3a, - 0x2a, 0x01, 0x68, 0xed, 0xde, 0xda, 0x8e, 0xda, 0xec, 0x75, 0x74, 0xf9, 0xb9, 0x80, 0x8a, 0x90, - 0xed, 0x68, 0x0a, 0xfb, 0xfe, 0xbd, 0x50, 0x15, 0xa5, 0x6b, 0xf2, 0xb5, 0xea, 0x2f, 0x04, 0xc8, - 0xc5, 0xf2, 0x82, 0x2e, 0xa4, 0xd3, 0x56, 0xe5, 0x39, 0x12, 0x15, 0xe2, 0xef, 0x7d, 0x55, 0x97, - 0x05, 0xe2, 0x7c, 0x77, 0xb3, 0xde, 0x6a, 0x91, 0xd8, 0x25, 0x88, 0xf3, 0x0d, 0xed, 0x3e, 0xf9, - 0x4d, 0xe3, 0xa5, 0xab, 0xf5, 0x96, 0x9c, 0x42, 0x0b, 0x20, 0x2b, 0x9d, 0xed, 0x46, 0x4b, 0xed, - 0x6f, 0xe9, 0x6a, 0x53, 0xeb, 0x6a, 0x9d, 0xb6, 0x9c, 0x26, 0x30, 0x3b, 0x75, 0xbd, 0xf9, 0xa0, - 0xae, 0xcb, 0x19, 0xa2, 0x4c, 0x7f, 0x49, 0x64, 0xc9, 0x8f, 0xe8, 0xcf, 0x2c, 0x41, 0xdb, 0xa9, - 0xeb, 0x64, 0xd5, 0x50, 0x15, 0x25, 0x51, 0x16, 0x3f, 0x11, 0xff, 0xfe, 0xdd, 0x92, 0x50, 0xfd, - 0x87, 0x08, 0xd7, 0xd6, 0x5d, 0x0f, 0x5b, 0xfb, 0xce, 0x43, 0x7c, 0xac, 0xe3, 0x3d, 0xec, 0x61, - 0x67, 0x80, 0xd1, 0x32, 0xa4, 0x02, 0x63, 0xd7, 0x0e, 0xab, 0xba, 0xd0, 0x00, 0x92, 0x24, 0x3f, - 0xbd, 0x58, 0x4a, 0x68, 0x8a, 0x1e, 0x0e, 0xa0, 0x3b, 0x90, 0xb2, 0x1c, 0x13, 0x1f, 0xd1, 0x22, - 0x2d, 0x34, 0x4a, 0x4c, 0x23, 0xa3, 0x11, 0x21, 0x51, 0xa3, 0xa3, 0xa8, 0x0c, 0xa2, 0x63, 0x0c, - 0x31, 0x2d, 0xd5, 0x2c, 0x4b, 0x36, 0x2a, 0x41, 0x0f, 0x41, 0x3a, 0x34, 0x6c, 0xcb, 0xb4, 0x82, - 0xe3, 0xb2, 0x48, 0x8b, 0xe0, 0x9d, 0x0b, 0xb3, 0xd5, 0xf1, 0x03, 0xcf, 0xb0, 0x9c, 0x60, 0x87, - 0x19, 0x30, 0xa0, 0x08, 0x00, 0xdd, 0x85, 0x79, 0xff, 0xc0, 0xf0, 0xb0, 0xd9, 0x1f, 0x79, 0x78, - 0xcf, 0x3a, 0xea, 0xdb, 0xd8, 0xa1, 0x25, 0xcd, 0xe9, 0xa1, 0x14, 0x0e, 0x6f, 0xd1, 0xd1, 0x16, - 0x76, 0x50, 0x0f, 0xb2, 0xae, 0xd3, 0x37, 0xb1, 0x8d, 0x03, 0x5e, 0xde, 0x1f, 0x5e, 0x30, 0xff, - 0x39, 0x01, 0xaa, 0xd5, 0x07, 0x81, 0xe5, 0x3a, 0x7c, 0x1d, 0xae, 0xa3, 0x50, 0x20, 0x86, 0x3a, - 0x1e, 0x99, 0x46, 0x80, 0x59, 0x69, 0xbf, 0x0a, 0xea, 0x36, 0x05, 0x42, 0x2d, 0x48, 0x0d, 0x8d, - 0x60, 0x70, 0x50, 0x96, 0x28, 0xe2, 0xdd, 0x2b, 0x20, 0x6e, 0x12, 0x3b, 0x4e, 0xa1, 0x14, 0xa4, - 0xfa, 0x18, 0xd2, 0xe1, 0x3c, 0xa4, 0xfa, 0xda, 0x9d, 0x7e, 0xbd, 0xd9, 0x23, 0x69, 0x34, 0x47, - 0x12, 0x50, 0x57, 0x49, 0x05, 0x35, 0x7b, 0x2c, 0x1d, 0xd5, 0x5e, 0x9f, 0x96, 0x4c, 0x82, 0x14, - 0x1d, 0xf9, 0x52, 0xd4, 0xf5, 0xfa, 0x76, 0x8b, 0xe4, 0x64, 0x0e, 0x32, 0xcd, 0x7a, 0xb7, 0x59, - 0x57, 0x54, 0x59, 0xac, 0x88, 0x3f, 0xfc, 0x7a, 0x71, 0xae, 0xfa, 0x26, 0xa4, 0xe8, 0x74, 0xb4, - 0x70, 0xb5, 0x4d, 0x52, 0x2c, 0x73, 0x24, 0x1d, 0xd7, 0x09, 0x84, 0xc0, 0x94, 0xfe, 0x94, 0x00, - 0x39, 0xa4, 0x1f, 0x05, 0xfb, 0x03, 0xcf, 0x1a, 0x05, 0xae, 0x17, 0x65, 0x89, 0x70, 0x26, 0x4b, - 0xde, 0x86, 0x84, 0x65, 0xb2, 0x1c, 0xbb, 0x41, 0xe4, 0x27, 0x34, 0x0b, 0x7f, 0x7a, 0xb1, 0x24, - 0x85, 0x28, 0x9a, 0xa2, 0x27, 0x2c, 0x13, 0x7d, 0x0a, 0x22, 0x25, 0x6a, 0x92, 0x67, 0x57, 0xe0, - 0x3d, 0x6a, 0x84, 0x96, 0x41, 0x72, 0xc6, 0xb6, 0x4d, 0x13, 0x9e, 0xa4, 0xa2, 0xc4, 0x77, 0x80, - 0x4b, 0x09, 0x21, 0x9a, 0x78, 0xcf, 0x18, 0xdb, 0x41, 0x1f, 0x1f, 0x8d, 0x3c, 0x76, 0x5a, 0xe4, - 0x98, 0x4c, 0x3d, 0x1a, 0x79, 0xe8, 0x16, 0xa4, 0x0f, 0x2c, 0xd3, 0xc4, 0x0e, 0xcd, 0x26, 0x0e, - 0xc1, 0x64, 0x68, 0x0d, 0xe6, 0xc7, 0x3e, 0xf6, 0xfb, 0x3e, 0x7e, 0x36, 0x26, 0x1b, 0xd3, 0xb7, - 0x4c, 0xbf, 0x0c, 0xcb, 0xc9, 0x95, 0x42, 0x23, 0xcd, 0x0a, 0xab, 0x44, 0x14, 0xba, 0x6c, 0x5c, - 0x33, 0x29, 0x0b, 0x0f, 0xdc, 0xe1, 0x68, 0x1c, 0xe0, 0x70, 0xd2, 0x5c, 0x38, 0x29, 0x93, 0x91, - 0x49, 0x37, 0x44, 0x49, 0x92, 0xb3, 0x1b, 0xa2, 0x94, 0x95, 0x61, 0x43, 0x94, 0x32, 0xb2, 0x54, - 0xfd, 0x26, 0x01, 0x37, 0x42, 0x37, 0xd7, 0x8d, 0xa1, 0x65, 0x1f, 0xbf, 0x6a, 0x94, 0x43, 0x14, - 0x16, 0x65, 0xba, 0x22, 0x82, 0xdd, 0x27, 0x66, 0x7e, 0x39, 0x19, 0x9e, 0x0b, 0xa1, 0xac, 0x4d, - 0x44, 0xe8, 0x63, 0x00, 0xa6, 0x42, 0x3c, 0x14, 0xa9, 0x87, 0x37, 0x4f, 0x5e, 0x2c, 0x65, 0xf9, - 0x76, 0xf9, 0x53, 0x7b, 0x97, 0x0d, 0x95, 0x89, 0xbb, 0x1d, 0x98, 0xe7, 0x31, 0x8e, 0x10, 0x68, - 0xa0, 0x0b, 0x8d, 0x37, 0xd9, 0x9a, 0x4a, 0x4a, 0xa8, 0xc0, 0xcd, 0xa7, 0xa0, 0x4a, 0xe6, 0xd4, - 0xa0, 0x59, 0xfd, 0x4d, 0x02, 0x16, 0x34, 0x27, 0xc0, 0x9e, 0x8d, 0x8d, 0x43, 0x1c, 0x0b, 0xc4, - 0xe7, 0x90, 0x35, 0x9c, 0x01, 0xf6, 0x03, 0xd7, 0xf3, 0xcb, 0x02, 0x3d, 0x29, 0x3f, 0xba, 0x20, - 0x63, 0xce, 0xb3, 0xaf, 0xd5, 0x99, 0x31, 0x6f, 0x3d, 0x22, 0xb0, 0xca, 0xef, 0x04, 0x90, 0xf8, - 0x28, 0xba, 0x0b, 0x12, 0xe5, 0x4a, 0xe2, 0x47, 0xc8, 0xa3, 0xd7, 0x99, 0x1f, 0x99, 0x1e, 0x91, - 0xd3, 0xf5, 0x93, 0x9d, 0xcf, 0x50, 0x35, 0xcd, 0x44, 0xff, 0x0e, 0x12, 0xa5, 0xcd, 0x7e, 0xb4, - 0x1b, 0x15, 0x6e, 0xc1, 0x78, 0x35, 0x4e, 0xb1, 0x19, 0xaa, 0xab, 0x99, 0xa8, 0x79, 0x1e, 0xfb, - 0x25, 0xa9, 0xfd, 0x6b, 0x3c, 0x72, 0xdd, 0x69, 0xfe, 0x3b, 0x43, 0x88, 0xd5, 0xbf, 0x25, 0xe1, - 0xc6, 0x96, 0xe1, 0x05, 0x16, 0xa1, 0x06, 0xcb, 0xd9, 0x8f, 0xc5, 0xeb, 0x0e, 0xe4, 0x9c, 0xf1, - 0x90, 0xed, 0x8a, 0xcf, 0x7c, 0x09, 0x7d, 0x07, 0x67, 0x3c, 0x0c, 0x03, 0xee, 0xa3, 0x16, 0x88, - 0xb6, 0xe5, 0x07, 0xe5, 0x04, 0x8d, 0xe8, 0xda, 0x05, 0x11, 0x3d, 0x7f, 0x8e, 0x5a, 0xcb, 0xf2, - 0x03, 0x9e, 0x93, 0x04, 0x05, 0x75, 0x20, 0xe5, 0x19, 0xce, 0x3e, 0xa6, 0x49, 0x96, 0x5b, 0xbb, - 0x77, 0x35, 0x38, 0x9d, 0x98, 0x72, 0xde, 0xa3, 0x38, 0x95, 0x5f, 0x09, 0x20, 0x92, 0x59, 0x2e, - 0xa9, 0x83, 0x1b, 0x90, 0x3e, 0x34, 0xec, 0x31, 0xf6, 0xa9, 0x0f, 0x79, 0x9d, 0x7d, 0xa1, 0xff, - 0x81, 0x92, 0x3f, 0xde, 0x1d, 0xc5, 0xa6, 0x62, 0x44, 0xf3, 0xc1, 0x95, 0x56, 0x15, 0x9d, 0x45, - 0xd3, 0x58, 0x95, 0xa7, 0x90, 0xa2, 0xeb, 0xbd, 0x64, 0x65, 0xa4, 0x23, 0x73, 0xfb, 0xf8, 0x68, - 0x60, 0x8f, 0x7d, 0xeb, 0x10, 0xd3, 0xec, 0xc8, 0xeb, 0xb9, 0xc0, 0x55, 0xb9, 0x08, 0xdd, 0x81, - 0xe2, 0x9e, 0xe7, 0x0e, 0xfb, 0x96, 0xc3, 0x95, 0x92, 0x54, 0xa9, 0x40, 0xa4, 0x1a, 0x17, 0x56, - 0xff, 0x29, 0x41, 0x89, 0x66, 0xd0, 0x4c, 0xcc, 0x70, 0x27, 0xc6, 0x0c, 0xd7, 0xa7, 0x98, 0x21, - 0x4a, 0x43, 0x42, 0x0c, 0xb7, 0x20, 0x3d, 0x76, 0xac, 0x67, 0xe3, 0x70, 0xce, 0x88, 0xfc, 0x42, - 0xd9, 0x19, 0xda, 0x10, 0xcf, 0xd2, 0xc6, 0xfb, 0x80, 0x48, 0xcd, 0xe0, 0xfe, 0x94, 0x62, 0x8a, - 0x2a, 0xca, 0x74, 0xa4, 0x79, 0x21, 0xc9, 0xa4, 0xaf, 0x40, 0x32, 0x0f, 0x40, 0xc6, 0x47, 0x81, - 0x67, 0xf4, 0x63, 0xf6, 0x19, 0x6a, 0xbf, 0x78, 0xf2, 0x62, 0xa9, 0xa8, 0x92, 0xb1, 0xf3, 0x41, - 0x8a, 0x38, 0x36, 0x66, 0x92, 0x9c, 0x98, 0x67, 0x18, 0xa6, 0xe5, 0x61, 0x7a, 0xa0, 0x86, 0x6d, - 0xf7, 0xc5, 0x07, 0xf4, 0xa9, 0xb0, 0xd7, 0x14, 0x6e, 0xa8, 0xcb, 0x21, 0x54, 0x24, 0xf0, 0xd1, - 0x23, 0xc8, 0xed, 0x85, 0xe7, 0x79, 0xff, 0x29, 0x3e, 0x2e, 0x67, 0x69, 0xba, 0xbd, 0x3b, 0xfb, - 0xc9, 0xcf, 0xeb, 0x73, 0x2f, 0x1a, 0x42, 0xdb, 0x50, 0xf0, 0xf8, 0xb0, 0xd9, 0xdf, 0x3d, 0xa6, - 0xe7, 0xcf, 0xcb, 0x80, 0xe6, 0x27, 0x30, 0x8d, 0x63, 0xf4, 0x08, 0xc0, 0x8a, 0x58, 0x92, 0x1e, - 0x52, 0xb9, 0xb5, 0xf7, 0xae, 0x40, 0xa7, 0x7c, 0xa5, 0x13, 0x10, 0xf4, 0x18, 0x8a, 0x93, 0x2f, - 0xba, 0xd4, 0xfc, 0x4b, 0x2e, 0xb5, 0x10, 0xc3, 0x69, 0x1c, 0xa3, 0x1e, 0x2c, 0x90, 0xe3, 0xd3, - 0xf5, 0xad, 0x00, 0xc7, 0x53, 0xa0, 0x40, 0x53, 0xa0, 0x7a, 0xf2, 0x62, 0x09, 0x35, 0xf9, 0xf8, - 0xf9, 0x69, 0x80, 0x06, 0xa7, 0xc6, 0xc3, 0xa4, 0x9a, 0x4a, 0x5e, 0x82, 0x58, 0x9c, 0x24, 0x55, - 0x77, 0x92, 0xbe, 0x67, 0x92, 0x2a, 0x96, 0xda, 0x04, 0xe9, 0x31, 0xe4, 0xa7, 0x58, 0xa6, 0xf4, - 0xf2, 0x2c, 0x33, 0x05, 0x84, 0x54, 0xd6, 0x1f, 0xc9, 0xb4, 0x83, 0x7c, 0x6f, 0xc6, 0x04, 0x3d, - 0xdd, 0x29, 0x55, 0x17, 0x21, 0x1b, 0xe5, 0x28, 0xb9, 0x07, 0xd5, 0xbb, 0xcd, 0xb0, 0xc7, 0x53, - 0xd4, 0x6e, 0x53, 0x16, 0xaa, 0xb7, 0x41, 0xa4, 0x57, 0x9c, 0x1c, 0x64, 0xd6, 0x3b, 0xfa, 0xe3, - 0xba, 0xae, 0x84, 0x7d, 0xa5, 0xd6, 0xde, 0x51, 0xf5, 0x9e, 0xaa, 0xc8, 0x42, 0xf5, 0x7b, 0x11, - 0xd0, 0x64, 0x8a, 0xcd, 0x71, 0x60, 0x50, 0xb0, 0x3a, 0xa4, 0xc3, 0xe8, 0x51, 0x12, 0xca, 0xad, - 0xfd, 0xdb, 0xa5, 0x2d, 0xdc, 0x04, 0xe0, 0xc1, 0x9c, 0xce, 0x0c, 0xd1, 0x67, 0xf1, 0x2b, 0x49, - 0x6e, 0xed, 0xed, 0xd9, 0x9c, 0x7c, 0x30, 0xc7, 0xef, 0x2a, 0x0f, 0x21, 0xe5, 0x07, 0xa4, 0x71, - 0x4f, 0xd2, 0x20, 0xad, 0x5e, 0x60, 0x7f, 0x76, 0xf1, 0xb5, 0x2e, 0x31, 0xe3, 0xa7, 0x0d, 0xc5, - 0x40, 0x8f, 0x21, 0x1b, 0xf1, 0x02, 0xbb, 0xdf, 0xdc, 0x9b, 0x1d, 0x30, 0x0a, 0x32, 0x6f, 0x31, - 0x22, 0x2c, 0x54, 0x87, 0xdc, 0x90, 0xa9, 0x4d, 0x1a, 0xa4, 0x65, 0x46, 0xcd, 0xc0, 0x11, 0x28, - 0x45, 0xc7, 0xbe, 0x74, 0xe0, 0x46, 0x9a, 0x49, 0xfa, 0x5d, 0xcf, 0xb5, 0xed, 0x5d, 0x63, 0xf0, - 0x94, 0x5e, 0x52, 0xa2, 0x7e, 0x97, 0x4b, 0xab, 0xff, 0x0d, 0x29, 0xea, 0x13, 0xd9, 0xc8, 0xed, - 0xf6, 0xc3, 0x76, 0xe7, 0x31, 0xb9, 0x20, 0x94, 0x20, 0xa7, 0xa8, 0x2d, 0xb5, 0xa7, 0xf6, 0x3b, - 0xed, 0xd6, 0x13, 0x59, 0x40, 0x37, 0xe1, 0x3a, 0x13, 0xd4, 0xdb, 0x4a, 0xff, 0xb1, 0xae, 0xf1, - 0xa1, 0x44, 0x75, 0x25, 0x9e, 0x29, 0x93, 0x1b, 0x2f, 0xc9, 0x19, 0x45, 0x91, 0x05, 0x9a, 0x33, - 0x7a, 0x67, 0x4b, 0x4e, 0x34, 0xf2, 0x00, 0x66, 0x14, 0x81, 0x0d, 0x51, 0x4a, 0xcb, 0x99, 0xea, - 0xff, 0xbd, 0x0e, 0x25, 0xda, 0x23, 0xcd, 0x74, 0x48, 0x2d, 0xd3, 0x43, 0x2a, 0x6c, 0x78, 0xe4, - 0xa9, 0x43, 0x2a, 0xc1, 0xce, 0xa7, 0x7b, 0x90, 0x1d, 0x19, 0x1e, 0x76, 0x02, 0x12, 0x32, 0x71, - 0xaa, 0xcf, 0x95, 0xb6, 0xe8, 0x40, 0xa4, 0x2e, 0x85, 0x8a, 0x1a, 0x31, 0xca, 0x1c, 0x62, 0x8f, - 0xbe, 0x34, 0x85, 0x51, 0xbe, 0xc9, 0x2e, 0xb9, 0xf3, 0x93, 0x55, 0xed, 0x84, 0x0a, 0x3a, 0xd7, - 0x44, 0x5b, 0x30, 0x3f, 0x74, 0x4d, 0x6b, 0xcf, 0x1a, 0x84, 0x5b, 0x14, 0x58, 0xc3, 0xf0, 0x26, - 0x98, 0x5b, 0x7b, 0x23, 0xb6, 0xff, 0xe3, 0xc0, 0xb2, 0x6b, 0x07, 0xf6, 0xa0, 0xd6, 0xe3, 0x2f, - 0x73, 0xcc, 0x23, 0x39, 0x6e, 0x4d, 0x06, 0xd1, 0x7d, 0xc8, 0xf0, 0xce, 0x2b, 0x7c, 0xd5, 0x99, - 0xb5, 0x34, 0x18, 0x22, 0xb7, 0x46, 0xeb, 0x50, 0x74, 0xf0, 0x51, 0xbc, 0xbb, 0xce, 0x4e, 0x25, - 0x4f, 0xbe, 0x8d, 0x8f, 0xce, 0x6f, 0xad, 0xf3, 0xce, 0x64, 0xc4, 0x44, 0x8f, 0xa0, 0x30, 0xf2, - 0xac, 0xa1, 0xe1, 0x1d, 0xf7, 0xc3, 0x7a, 0x83, 0xab, 0xd4, 0x5b, 0x44, 0x4f, 0x21, 0x04, 0x1d, - 0x45, 0xeb, 0x10, 0x36, 0xb3, 0xd8, 0x2f, 0xe7, 0xa8, 0x8f, 0x57, 0x03, 0xe3, 0xc6, 0xa8, 0x01, - 0x05, 0xea, 0x62, 0xd4, 0x45, 0xe7, 0xa9, 0x87, 0x8b, 0xcc, 0xc3, 0x1c, 0xf1, 0xf0, 0x9c, 0x4e, - 0x3a, 0xe7, 0x44, 0x72, 0x13, 0x6d, 0x00, 0x44, 0x2f, 0xa2, 0xe4, 0x64, 0xb8, 0xec, 0xe0, 0xdd, - 0xe2, 0x8a, 0x93, 0x25, 0xe9, 0x31, 0x6b, 0xb4, 0x09, 0x59, 0x5e, 0x77, 0xe1, 0x91, 0x90, 0xbb, - 0xf0, 0x95, 0xe3, 0x2c, 0x0b, 0xf0, 0xda, 0x8f, 0x10, 0x50, 0x1b, 0x52, 0x36, 0x36, 0x7c, 0xcc, - 0xce, 0x85, 0x8f, 0x2f, 0x80, 0x3a, 0x55, 0x39, 0xb5, 0xee, 0xe0, 0x00, 0x0f, 0x8d, 0xe6, 0x01, - 0xe9, 0x31, 0x5b, 0xc4, 0x5e, 0x0f, 0x61, 0x50, 0x1b, 0x64, 0x1a, 0xae, 0x38, 0xa1, 0xc8, 0x34, - 0x62, 0x6f, 0xb1, 0x88, 0x15, 0x49, 0xc4, 0x2e, 0x24, 0x15, 0x9a, 0x4f, 0x9b, 0x13, 0x62, 0xf9, - 0x2f, 0x28, 0xee, 0xb9, 0xde, 0xd0, 0x08, 0xfa, 0xbc, 0x70, 0xe6, 0x27, 0x9d, 0xe3, 0x4f, 0x2f, - 0x96, 0x0a, 0xeb, 0x74, 0x94, 0x17, 0x4d, 0x61, 0x2f, 0xfe, 0x89, 0x1e, 0x70, 0xfe, 0xbd, 0x46, - 0xe9, 0xf2, 0xfd, 0x59, 0xbd, 0x3b, 0x4b, 0xbe, 0x6d, 0x48, 0x0f, 0x0e, 0xf0, 0xe0, 0xa9, 0x5f, - 0x5e, 0xa0, 0x31, 0xff, 0x8f, 0x19, 0xa1, 0x9a, 0xc4, 0x68, 0xf2, 0xdc, 0xa4, 0x33, 0x14, 0xf4, - 0x25, 0x14, 0x4d, 0x22, 0xb1, 0x9c, 0x7d, 0xd6, 0x99, 0x5e, 0xa7, 0xb8, 0xab, 0x33, 0xe2, 0x92, - 0xae, 0x55, 0x73, 0xf6, 0x5c, 0xde, 0x94, 0x70, 0xb0, 0xb0, 0x9b, 0xed, 0x80, 0xb4, 0x47, 0x6e, - 0xd9, 0x16, 0xf6, 0xcb, 0x37, 0x28, 0xee, 0xe5, 0x0f, 0xcd, 0xa7, 0x2f, 0xf6, 0x9c, 0xbd, 0x39, - 0x48, 0x54, 0xe8, 0x54, 0x70, 0x4c, 0x36, 0xf5, 0xb5, 0xb3, 0x85, 0xce, 0x2f, 0xf6, 0x53, 0x97, - 0x7c, 0x5a, 0xe8, 0xec, 0xcb, 0x44, 0x6f, 0x02, 0x1c, 0x5a, 0xf8, 0xeb, 0xfe, 0xb3, 0x31, 0xf6, - 0x8e, 0xcb, 0xe5, 0x18, 0xef, 0x66, 0x89, 0xfc, 0x11, 0x11, 0xa3, 0x0f, 0x21, 0x6b, 0xe2, 0x11, - 0x76, 0x4c, 0xbf, 0xe3, 0x94, 0x6f, 0xd2, 0xae, 0xe7, 0x1a, 0x69, 0xc5, 0x15, 0x2e, 0x64, 0xbc, - 0x3a, 0xd1, 0x42, 0x5f, 0x41, 0x3e, 0xfc, 0xc0, 0x66, 0xc7, 0x69, 0x1c, 0x97, 0x2b, 0xd4, 0xe9, - 0xbb, 0x33, 0x06, 0x73, 0xd2, 0xe2, 0x2d, 0x70, 0x7f, 0x94, 0x18, 0x9a, 0x3e, 0x85, 0x8d, 0xbe, - 0x84, 0x3c, 0xcf, 0xee, 0x0d, 0x77, 0xd7, 0x2f, 0xbf, 0x7e, 0xe9, 0xe5, 0xf4, 0xf4, 0x5c, 0x9b, - 0x13, 0x53, 0xce, 0x5b, 0x71, 0x34, 0xf4, 0x39, 0x14, 0xa2, 0x17, 0x1d, 0x77, 0x14, 0xf8, 0xe5, - 0x5b, 0xb4, 0x30, 0xef, 0xcd, 0x9a, 0xba, 0xcc, 0xb6, 0x33, 0x0a, 0x7c, 0x3d, 0xef, 0xc7, 0xbe, - 0xd0, 0x6d, 0xc8, 0x9a, 0x9e, 0x3b, 0x0a, 0xcf, 0x8f, 0x37, 0x96, 0x85, 0x95, 0x24, 0xdf, 0x66, - 0x22, 0xa6, 0x07, 0x43, 0x1f, 0x8a, 0x1e, 0x1e, 0xd9, 0xc6, 0x00, 0x0f, 0xc9, 0xc9, 0xe6, 0xee, - 0x95, 0x17, 0xe9, 0xec, 0x6b, 0x33, 0x07, 0x32, 0x32, 0xe6, 0x89, 0x19, 0xc3, 0xeb, 0xec, 0xa1, - 0x6d, 0x00, 0x63, 0x6c, 0x5a, 0x41, 0x7f, 0xe8, 0x9a, 0xb8, 0xbc, 0x74, 0xe9, 0xe3, 0xe3, 0x69, - 0xf0, 0x3a, 0x31, 0xdc, 0x74, 0x4d, 0x1c, 0x3d, 0x92, 0x70, 0x01, 0xfa, 0x10, 0x72, 0xd4, 0xb5, - 0xaf, 0xdc, 0x5d, 0x92, 0x9b, 0xcb, 0xd4, 0xb9, 0x79, 0xb6, 0x97, 0x59, 0xc5, 0x73, 0x47, 0x1b, - 0xee, 0x2e, 0xcd, 0x18, 0xf6, 0xd3, 0x44, 0x3e, 0xe4, 0xf7, 0x07, 0xfd, 0x09, 0x95, 0xde, 0xa6, - 0xbb, 0xf8, 0xe9, 0x8c, 0x6b, 0xb9, 0xdf, 0x3c, 0x87, 0x5c, 0xaf, 0xf1, 0x33, 0xe1, 0x7e, 0x93, - 0xcb, 0x7c, 0x3d, 0xb7, 0x3f, 0x88, 0x3e, 0x2a, 0xdf, 0x0b, 0x30, 0x7f, 0x86, 0x3a, 0xd1, 0xff, - 0x42, 0xc6, 0x71, 0xcd, 0xd8, 0xa3, 0x8e, 0xca, 0x80, 0xd2, 0x6d, 0xd7, 0x0c, 0xdf, 0x74, 0xee, - 0xed, 0x5b, 0xc1, 0xc1, 0x78, 0xb7, 0x36, 0x70, 0x87, 0xab, 0xd1, 0x0a, 0xcd, 0xdd, 0xc9, 0xef, - 0xd5, 0xd1, 0xd3, 0xfd, 0x55, 0xfa, 0x6b, 0xb4, 0x5b, 0x0b, 0xcd, 0xf4, 0x34, 0x41, 0xd5, 0x4c, - 0xf4, 0x01, 0x94, 0xf0, 0xd1, 0xc8, 0xf2, 0x62, 0xed, 0x43, 0x22, 0xb6, 0xfd, 0xc5, 0xc9, 0x20, - 0x49, 0x82, 0xca, 0x8f, 0x02, 0x94, 0x4e, 0xd1, 0x16, 0xe9, 0x94, 0xe8, 0x83, 0xe1, 0x54, 0xa7, - 0x44, 0x24, 0x51, 0x0f, 0x95, 0xb8, 0xf4, 0x39, 0x3e, 0xf9, 0xaa, 0xcf, 0xf1, 0xd3, 0xf7, 0xf3, - 0xd4, 0xec, 0xf7, 0xf3, 0x0d, 0x51, 0x12, 0xe5, 0x54, 0xe5, 0x09, 0x48, 0x9c, 0x32, 0xa7, 0x5b, - 0x37, 0x61, 0xc6, 0xd6, 0xed, 0x42, 0x3f, 0x2b, 0xdf, 0x0a, 0x90, 0x8d, 0xff, 0xcf, 0x91, 0x88, - 0x50, 0xcf, 0xef, 0x1c, 0x5f, 0xf2, 0x49, 0x6e, 0x3a, 0x02, 0xc9, 0xd9, 0x23, 0x50, 0x39, 0x84, - 0x5c, 0x8c, 0x75, 0x4e, 0xb7, 0xfb, 0xc2, 0x4b, 0xb4, 0xfb, 0x6f, 0x41, 0x9a, 0x95, 0x5a, 0x98, - 0x48, 0x05, 0x66, 0x9d, 0x0a, 0xcb, 0x2c, 0xf5, 0x15, 0x29, 0xb1, 0xca, 0x6f, 0x05, 0xc8, 0xc7, - 0xf9, 0x08, 0x55, 0x21, 0x6b, 0x39, 0x03, 0x8f, 0x92, 0x01, 0x9d, 0x97, 0xa7, 0xe0, 0x44, 0x4c, - 0x58, 0x6a, 0x68, 0x39, 0x7d, 0xfa, 0x4c, 0x36, 0x95, 0xa6, 0xd2, 0xd0, 0x72, 0x76, 0x88, 0x94, - 0xaa, 0x18, 0x47, 0x4c, 0x25, 0x39, 0xa5, 0x62, 0x1c, 0x85, 0x2a, 0x15, 0x7a, 0xf0, 0x7b, 0x01, - 0xed, 0xcc, 0x93, 0xb1, 0xa3, 0xdc, 0x0b, 0xd0, 0x22, 0x64, 0x0e, 0x2d, 0x2f, 0x18, 0x1b, 0x36, - 0x6d, 0xc2, 0xf9, 0x55, 0x85, 0x0b, 0x2b, 0x07, 0x90, 0x8b, 0xf1, 0xd8, 0x0c, 0x1b, 0xfa, 0x9f, - 0x20, 0x46, 0x45, 0x35, 0x63, 0x4f, 0x4e, 0x0d, 0x2a, 0xbf, 0x14, 0x60, 0xe1, 0x3c, 0x26, 0x99, - 0x4a, 0x91, 0x30, 0x4e, 0x33, 0xa5, 0xc8, 0x14, 0xc3, 0x27, 0xce, 0x65, 0xf8, 0xc9, 0xce, 0x25, - 0x2f, 0xde, 0xb9, 0xea, 0xdb, 0xfc, 0xb2, 0x06, 0x90, 0xde, 0xda, 0x6e, 0xb4, 0xb4, 0xe6, 0xb9, - 0x17, 0x2d, 0x72, 0x25, 0x8b, 0x58, 0x99, 0x5c, 0xca, 0x15, 0xad, 0x5b, 0x6f, 0xb4, 0x54, 0x72, - 0x45, 0x2f, 0x40, 0x56, 0x57, 0xeb, 0x0a, 0xbd, 0xc1, 0xc9, 0xc2, 0x27, 0xe2, 0x37, 0xdf, 0x2d, - 0x09, 0xe1, 0x55, 0x6c, 0x43, 0x94, 0x90, 0x7c, 0xad, 0xfa, 0xbd, 0x00, 0x48, 0x31, 0x02, 0x83, - 0x30, 0xc0, 0x15, 0xee, 0x64, 0x89, 0x4b, 0x36, 0x62, 0xba, 0xcf, 0x4e, 0xbe, 0x4a, 0x9f, 0x1d, - 0x2e, 0xb8, 0xfa, 0xad, 0x00, 0x10, 0x5b, 0xdc, 0x67, 0xf1, 0x3f, 0x31, 0x2f, 0xbe, 0x52, 0x9c, - 0x3a, 0x2d, 0x1e, 0xcc, 0xf1, 0xbf, 0x38, 0xef, 0x83, 0x64, 0x32, 0x97, 0x59, 0xb6, 0x5c, 0xd8, - 0xbb, 0x9f, 0x89, 0xcc, 0x03, 0xb2, 0x8d, 0x4c, 0xda, 0xc8, 0x40, 0x6a, 0xec, 0x58, 0xae, 0xf3, - 0xee, 0x47, 0x80, 0xce, 0xb2, 0x27, 0x09, 0x3e, 0xfd, 0x6d, 0x04, 0xd8, 0x0c, 0x6f, 0xd9, 0xdb, - 0xce, 0x61, 0x24, 0x10, 0x1a, 0xb7, 0x9f, 0xff, 0x65, 0x71, 0xee, 0xf9, 0xc9, 0xa2, 0xf0, 0x87, - 0x93, 0x45, 0xe1, 0x8f, 0x27, 0x8b, 0xc2, 0x9f, 0x4f, 0x16, 0x85, 0xff, 0xff, 0xeb, 0xe2, 0xdc, - 0x17, 0x19, 0xb6, 0x80, 0x7f, 0x05, 0x00, 0x00, 0xff, 0xff, 0x7d, 0x1f, 0x69, 0xb2, 0x1f, 0x22, - 0x00, 0x00, + proto.RegisterFile("sql/sqlbase/structured.proto", fileDescriptor_structured_1bcacc6943392042) +} + +var fileDescriptor_structured_1bcacc6943392042 = []byte{ + // 3260 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x59, 0xcd, 0x6f, 0xe3, 0xd6, + 0x76, 0x37, 0x25, 0x4a, 0xa2, 0x8e, 0xbe, 0xe8, 0x3b, 0x9e, 0x89, 0x46, 0x33, 0xb1, 0x3d, 0x4a, + 0x26, 0x75, 0xbe, 0xe4, 0x89, 0xa7, 0x1f, 0x41, 0x52, 0x04, 0x95, 0x44, 0x7a, 0x4c, 0x8f, 0x2c, + 0x79, 0x28, 0xd9, 0x93, 0x09, 0xd2, 0x0a, 0xb4, 0x78, 0x6d, 0x33, 0x43, 0x91, 0x1a, 0x92, 0x72, + 0xec, 0xff, 0x20, 0xab, 0xa2, 0xe8, 0xa2, 0xdd, 0x05, 0x41, 0xd0, 0x45, 0x80, 0x6e, 0xbb, 0xc8, + 0x9f, 0x30, 0xdd, 0x15, 0x59, 0x75, 0xd3, 0x41, 0xeb, 0xe2, 0x01, 0x6f, 0xf7, 0xf0, 0xb6, 0x01, + 0x1e, 0xf0, 0x70, 0x2f, 0xef, 0xa5, 0x28, 0x7f, 0x45, 0x9e, 0xd9, 0x91, 0xe7, 0xde, 0xf3, 0xe3, + 0xbd, 0xe7, 0x9e, 0xf3, 0x3b, 0xe7, 0x5c, 0xc2, 0x5d, 0xff, 0x85, 0xbd, 0xea, 0xbf, 0xb0, 0xf7, + 0x0c, 0x1f, 0xaf, 0xfa, 0x81, 0x37, 0x1e, 0x04, 0x63, 0x0f, 0x9b, 0xb5, 0x91, 0xe7, 0x06, 0x2e, + 0xba, 0x39, 0x70, 0x07, 0xcf, 0x3d, 0xd7, 0x18, 0x1c, 0xd6, 0xfc, 0x17, 0x76, 0x8d, 0xcd, 0xab, + 0x94, 0xc7, 0x81, 0x65, 0xaf, 0x1e, 0xda, 0x83, 0xd5, 0xc0, 0x1a, 0x62, 0x3f, 0x30, 0x86, 0xa3, + 0x50, 0xa1, 0x72, 0x27, 0x0e, 0x37, 0xf2, 0xac, 0x23, 0xcb, 0xc6, 0x07, 0x98, 0x0d, 0x2e, 0x1c, + 0xb8, 0x07, 0x2e, 0x7d, 0x5c, 0x25, 0x4f, 0xa1, 0xb4, 0xfa, 0x4b, 0x06, 0xa0, 0xe9, 0xda, 0xe3, + 0xa1, 0xd3, 0x3b, 0x19, 0x61, 0xf4, 0x0c, 0x0a, 0x3e, 0x1e, 0x1a, 0x4e, 0x60, 0x0d, 0xfa, 0xc1, + 0xc9, 0x08, 0x97, 0x85, 0x65, 0x61, 0xa5, 0xb8, 0x56, 0xab, 0x5d, 0xb8, 0x94, 0xda, 0x44, 0xb3, + 0xd6, 0x65, 0x6a, 0xe4, 0xa5, 0x21, 0xbe, 0x7c, 0xb5, 0x34, 0xa7, 0xe7, 0xfd, 0x98, 0x0c, 0x55, + 0x20, 0xf5, 0xad, 0x65, 0x06, 0x87, 0xe5, 0xc4, 0xb2, 0xb0, 0x92, 0x62, 0x53, 0x42, 0x11, 0xaa, + 0x42, 0x76, 0xe4, 0xe1, 0x81, 0xe5, 0x5b, 0xae, 0x53, 0x4e, 0xc6, 0xc6, 0x27, 0x62, 0xf4, 0x3e, + 0xc8, 0x86, 0xe7, 0x19, 0x27, 0x7d, 0xd3, 0x1a, 0x62, 0x87, 0x88, 0xfc, 0xb2, 0xb8, 0x9c, 0x5c, + 0x49, 0xe9, 0x25, 0x2a, 0x57, 0x22, 0x31, 0xba, 0x05, 0x69, 0xdb, 0x1d, 0x18, 0x36, 0x2e, 0xa7, + 0x96, 0x85, 0x95, 0xac, 0xce, 0xde, 0xd0, 0x2e, 0xe4, 0x8f, 0x2c, 0xdf, 0xda, 0xb3, 0x71, 0xb8, + 0xb9, 0x34, 0xdd, 0xdc, 0xc7, 0xbf, 0xbd, 0xb9, 0xdd, 0x50, 0x2b, 0xb6, 0xb7, 0xdc, 0xd1, 0x44, + 0x84, 0x76, 0xa0, 0x18, 0x2e, 0x6d, 0xe0, 0x3a, 0x01, 0x76, 0x02, 0xbf, 0x9c, 0x79, 0x1d, 0xb3, + 0xe9, 0x05, 0x8a, 0xd2, 0x64, 0x20, 0xa8, 0x0d, 0xc5, 0x60, 0x3c, 0xb2, 0xf1, 0x04, 0x56, 0x5a, + 0x4e, 0xae, 0xe4, 0xd6, 0xee, 0xfd, 0x26, 0x2c, 0x5b, 0x64, 0x81, 0xaa, 0x47, 0x78, 0xf7, 0x20, + 0x1f, 0xe2, 0xd9, 0xc6, 0x1e, 0xb6, 0xfd, 0x72, 0x76, 0x39, 0xb9, 0x92, 0xd5, 0x73, 0x54, 0xd6, + 0xa2, 0xa2, 0xea, 0x4f, 0x09, 0xc8, 0xc7, 0x97, 0x84, 0x24, 0x10, 0x1b, 0x9d, 0x4e, 0x4b, 0x9e, + 0x43, 0x19, 0x48, 0x6a, 0xed, 0x9e, 0x2c, 0xa0, 0x2c, 0xa4, 0xd6, 0x5b, 0x9d, 0x7a, 0x4f, 0x4e, + 0xa0, 0x1c, 0x64, 0x14, 0xb5, 0xa9, 0x6d, 0xd5, 0x5b, 0x72, 0x92, 0x4c, 0x55, 0xea, 0x3d, 0x55, + 0x16, 0x51, 0x01, 0xb2, 0x3d, 0x6d, 0x4b, 0xed, 0xf6, 0xea, 0x5b, 0xdb, 0x72, 0x0a, 0xe5, 0x41, + 0xd2, 0xda, 0x3d, 0x55, 0xdf, 0xad, 0xb7, 0xe4, 0x34, 0x02, 0x48, 0x77, 0x7b, 0xba, 0xd6, 0x7e, + 0x24, 0x67, 0x08, 0x54, 0xe3, 0x59, 0x4f, 0xed, 0xca, 0x12, 0x2a, 0x41, 0x2e, 0xd2, 0xe9, 0x7d, + 0x25, 0x67, 0x11, 0x82, 0x62, 0xb3, 0xd3, 0x6a, 0xd5, 0x7b, 0xaa, 0xc2, 0xe6, 0x03, 0xf9, 0x44, + 0xbb, 0xbe, 0xa5, 0xca, 0x39, 0xb2, 0x9a, 0x8e, 0xa6, 0xc8, 0x79, 0x2a, 0xda, 0x69, 0xb5, 0xe4, + 0x02, 0x79, 0xda, 0xd9, 0xd1, 0x14, 0xb9, 0x48, 0x60, 0xeb, 0xba, 0x5e, 0x7f, 0x26, 0x97, 0x88, + 0x50, 0x6b, 0xab, 0x3d, 0x59, 0x26, 0x4f, 0xe4, 0x03, 0xf2, 0x3c, 0x19, 0xde, 0xec, 0x76, 0xda, + 0x0d, 0x19, 0x91, 0xc7, 0xde, 0xce, 0x76, 0x4b, 0x95, 0x17, 0x08, 0x62, 0x43, 0xeb, 0xc9, 0x37, + 0x51, 0x09, 0x40, 0x6b, 0xf7, 0xd6, 0x76, 0xd5, 0x66, 0xaf, 0xa3, 0xcb, 0x2f, 0x05, 0x54, 0x84, + 0x6c, 0x47, 0x53, 0xd8, 0xfb, 0x7f, 0x0a, 0x55, 0x51, 0xba, 0x21, 0xdf, 0xa8, 0xfe, 0xb3, 0x00, + 0xb9, 0x98, 0x5f, 0xd0, 0x85, 0x74, 0xda, 0xaa, 0x3c, 0x47, 0xac, 0x42, 0xf6, 0xfb, 0x48, 0xd5, + 0x65, 0x81, 0x6c, 0xbe, 0xbb, 0x55, 0x6f, 0xb5, 0x88, 0xed, 0x12, 0x64, 0xf3, 0x0d, 0xed, 0x11, + 0x79, 0xa6, 0xf6, 0xd2, 0xd5, 0x7a, 0x4b, 0x4e, 0xa1, 0x05, 0x90, 0x95, 0xce, 0x4e, 0xa3, 0xa5, + 0xf6, 0xb7, 0x75, 0xb5, 0xa9, 0x75, 0xb5, 0x4e, 0x5b, 0x4e, 0x13, 0x98, 0xdd, 0xba, 0xde, 0xdc, + 0xa8, 0xeb, 0x72, 0x86, 0x4c, 0xa6, 0x4f, 0x12, 0x59, 0xf2, 0x13, 0xfa, 0x98, 0x25, 0x68, 0xbb, + 0x75, 0x9d, 0xac, 0x1a, 0xaa, 0xa2, 0x24, 0xca, 0xe2, 0x67, 0xe2, 0xef, 0x7f, 0x58, 0x12, 0xaa, + 0x7f, 0x10, 0xe1, 0xc6, 0xba, 0xeb, 0x61, 0xeb, 0xc0, 0x79, 0x8c, 0x4f, 0x74, 0xbc, 0x8f, 0x3d, + 0xec, 0x0c, 0x30, 0x5a, 0x86, 0x54, 0x60, 0xec, 0xd9, 0x61, 0x54, 0x17, 0x1a, 0x40, 0x9c, 0xe4, + 0xd7, 0x57, 0x4b, 0x09, 0x4d, 0xd1, 0xc3, 0x01, 0x74, 0x1f, 0x52, 0x96, 0x63, 0xe2, 0x63, 0x1a, + 0xa4, 0x85, 0x46, 0x89, 0xcd, 0xc8, 0x68, 0x44, 0x48, 0xa6, 0xd1, 0x51, 0x54, 0x06, 0xd1, 0x31, + 0x86, 0x98, 0x86, 0x6a, 0x96, 0x39, 0x1b, 0x95, 0xa0, 0xc7, 0x20, 0x1d, 0x19, 0xb6, 0x65, 0x5a, + 0xc1, 0x49, 0x59, 0xa4, 0x41, 0xf0, 0xfe, 0xa5, 0xde, 0xea, 0xf8, 0x81, 0x67, 0x58, 0x4e, 0xb0, + 0xcb, 0x14, 0x18, 0x50, 0x04, 0x80, 0x1e, 0xc0, 0xbc, 0x7f, 0x68, 0x78, 0xd8, 0xec, 0x8f, 0x3c, + 0xbc, 0x6f, 0x1d, 0xf7, 0x6d, 0xec, 0xd0, 0x90, 0xe6, 0xf4, 0x50, 0x0a, 0x87, 0xb7, 0xe9, 0x68, + 0x0b, 0x3b, 0xa8, 0x07, 0x59, 0xd7, 0xe9, 0x9b, 0xd8, 0xc6, 0x01, 0x0f, 0xef, 0x4f, 0x2e, 0xf9, + 0xfe, 0x05, 0x06, 0xaa, 0xd5, 0x07, 0x81, 0xe5, 0x3a, 0x7c, 0x1d, 0xae, 0xa3, 0x50, 0x20, 0x86, + 0x3a, 0x1e, 0x99, 0x46, 0x80, 0x59, 0x68, 0xbf, 0x09, 0xea, 0x0e, 0x05, 0x42, 0x2d, 0x48, 0x0d, + 0x8d, 0x60, 0x70, 0x58, 0x96, 0x28, 0xe2, 0x83, 0x6b, 0x20, 0x6e, 0x11, 0x3d, 0x4e, 0xa1, 0x14, + 0xa4, 0xfa, 0x14, 0xd2, 0xe1, 0x77, 0x48, 0xf4, 0xb5, 0x3b, 0xfd, 0x7a, 0xb3, 0x47, 0xdc, 0x68, + 0x8e, 0x38, 0xa0, 0xae, 0x92, 0x08, 0x6a, 0xf6, 0x98, 0x3b, 0xaa, 0xbd, 0x3e, 0x0d, 0x99, 0x04, + 0x09, 0x3a, 0xf2, 0xa6, 0xa8, 0xeb, 0xf5, 0x9d, 0x16, 0xf1, 0xc9, 0x1c, 0x64, 0x9a, 0xf5, 0x6e, + 0xb3, 0xae, 0xa8, 0xb2, 0x58, 0x11, 0x7f, 0xfa, 0xb7, 0xc5, 0xb9, 0xea, 0x3b, 0x90, 0xa2, 0x9f, + 0xa3, 0x81, 0xab, 0x6d, 0x91, 0x60, 0x99, 0x23, 0xee, 0xb8, 0x4e, 0x20, 0x04, 0x36, 0xe9, 0x7f, + 0x12, 0x20, 0x87, 0xf4, 0xa3, 0x60, 0x7f, 0xe0, 0x59, 0xa3, 0xc0, 0xf5, 0x22, 0x2f, 0x11, 0xce, + 0x79, 0xc9, 0x7b, 0x90, 0xb0, 0x4c, 0xe6, 0x63, 0xb7, 0x88, 0xfc, 0x94, 0x7a, 0xe1, 0xaf, 0xaf, + 0x96, 0xa4, 0x10, 0x45, 0x53, 0xf4, 0x84, 0x65, 0xa2, 0xcf, 0x41, 0xa4, 0x44, 0x4d, 0xfc, 0xec, + 0x1a, 0xbc, 0x47, 0x95, 0xd0, 0x32, 0x48, 0xce, 0xd8, 0xb6, 0xa9, 0xc3, 0x13, 0x57, 0x94, 0xf8, + 0x09, 0x70, 0x29, 0x21, 0x44, 0x13, 0xef, 0x1b, 0x63, 0x3b, 0xe8, 0xe3, 0xe3, 0x91, 0xc7, 0xb2, + 0x45, 0x8e, 0xc9, 0xd4, 0xe3, 0x91, 0x87, 0xee, 0x42, 0xfa, 0xd0, 0x32, 0x4d, 0xec, 0x50, 0x6f, + 0xe2, 0x10, 0x4c, 0x86, 0xd6, 0x60, 0x7e, 0xec, 0x63, 0xbf, 0xef, 0xe3, 0x17, 0x63, 0x72, 0x30, + 0x7d, 0xcb, 0xf4, 0xcb, 0xb0, 0x9c, 0x5c, 0x29, 0x34, 0xd2, 0x2c, 0xb0, 0x4a, 0x64, 0x42, 0x97, + 0x8d, 0x6b, 0x26, 0x65, 0xe1, 0x81, 0x3b, 0x1c, 0x8d, 0x03, 0x1c, 0x7e, 0x34, 0x17, 0x7e, 0x94, + 0xc9, 0xc8, 0x47, 0x37, 0x45, 0x49, 0x92, 0xb3, 0x9b, 0xa2, 0x94, 0x95, 0x61, 0x53, 0x94, 0x32, + 0xb2, 0x54, 0xfd, 0x2e, 0x01, 0xb7, 0xc2, 0x6d, 0xae, 0x1b, 0x43, 0xcb, 0x3e, 0x79, 0x53, 0x2b, + 0x87, 0x28, 0xcc, 0xca, 0x74, 0x45, 0x04, 0xbb, 0x4f, 0xd4, 0xfc, 0x72, 0x32, 0xcc, 0x0b, 0xa1, + 0xac, 0x4d, 0x44, 0xe8, 0x53, 0x00, 0x36, 0x85, 0xec, 0x50, 0xa4, 0x3b, 0xbc, 0x7d, 0xfa, 0x6a, + 0x29, 0xcb, 0x8f, 0xcb, 0x9f, 0x3a, 0xbb, 0x6c, 0x38, 0x99, 0x6c, 0xb7, 0x03, 0xf3, 0xdc, 0xc6, + 0x11, 0x02, 0x35, 0x74, 0xa1, 0xf1, 0x0e, 0x5b, 0x53, 0x49, 0x09, 0x27, 0x70, 0xf5, 0x29, 0xa8, + 0x92, 0x39, 0x35, 0x68, 0x56, 0xff, 0x3d, 0x01, 0x0b, 0x9a, 0x13, 0x60, 0xcf, 0xc6, 0xc6, 0x11, + 0x8e, 0x19, 0xe2, 0x4b, 0xc8, 0x1a, 0xce, 0x00, 0xfb, 0x81, 0xeb, 0xf9, 0x65, 0x81, 0x66, 0xca, + 0xbf, 0xbc, 0xc4, 0x63, 0x2e, 0xd2, 0xaf, 0xd5, 0x99, 0x32, 0x2f, 0x3d, 0x22, 0xb0, 0xca, 0xcf, + 0x02, 0x48, 0x7c, 0x14, 0x3d, 0x00, 0x89, 0x72, 0x25, 0xd9, 0x47, 0xc8, 0xa3, 0x37, 0xd9, 0x3e, + 0x32, 0x3d, 0x22, 0xa7, 0xeb, 0x27, 0x27, 0x9f, 0xa1, 0xd3, 0x34, 0x13, 0xfd, 0x15, 0x48, 0x94, + 0x36, 0xfb, 0xd1, 0x69, 0x54, 0xb8, 0x06, 0xe3, 0xd5, 0x38, 0xc5, 0x66, 0xe8, 0x5c, 0xcd, 0x44, + 0xcd, 0x8b, 0xd8, 0x2f, 0x49, 0xf5, 0xdf, 0xe2, 0x96, 0xeb, 0x4e, 0xf3, 0xdf, 0x39, 0x42, 0xac, + 0xfe, 0x2e, 0x09, 0xb7, 0xb6, 0x0d, 0x2f, 0xb0, 0x08, 0x35, 0x58, 0xce, 0x41, 0xcc, 0x5e, 0xf7, + 0x21, 0xe7, 0x8c, 0x87, 0xec, 0x54, 0x7c, 0xb6, 0x97, 0x70, 0xef, 0xe0, 0x8c, 0x87, 0xa1, 0xc1, + 0x7d, 0xd4, 0x02, 0xd1, 0xb6, 0xfc, 0xa0, 0x9c, 0xa0, 0x16, 0x5d, 0xbb, 0xc4, 0xa2, 0x17, 0x7f, + 0xa3, 0xd6, 0xb2, 0xfc, 0x80, 0xfb, 0x24, 0x41, 0x41, 0x1d, 0x48, 0x79, 0x86, 0x73, 0x80, 0xa9, + 0x93, 0xe5, 0xd6, 0x1e, 0x5e, 0x0f, 0x4e, 0x27, 0xaa, 0x9c, 0xf7, 0x28, 0x4e, 0xe5, 0x5f, 0x05, + 0x10, 0xc9, 0x57, 0xae, 0x88, 0x83, 0x5b, 0x90, 0x3e, 0x32, 0xec, 0x31, 0xf6, 0xe9, 0x1e, 0xf2, + 0x3a, 0x7b, 0x43, 0x7f, 0x0f, 0x25, 0x7f, 0xbc, 0x37, 0x8a, 0x7d, 0x8a, 0x11, 0xcd, 0xc7, 0xd7, + 0x5a, 0x55, 0x94, 0x8b, 0xa6, 0xb1, 0x2a, 0xcf, 0x21, 0x45, 0xd7, 0x7b, 0xc5, 0xca, 0x48, 0x45, + 0xe6, 0xf6, 0xf1, 0xf1, 0xc0, 0x1e, 0xfb, 0xd6, 0x11, 0xa6, 0xde, 0x91, 0xd7, 0x73, 0x81, 0xab, + 0x72, 0x11, 0xba, 0x0f, 0xc5, 0x7d, 0xcf, 0x1d, 0xf6, 0x2d, 0x87, 0x4f, 0x4a, 0xd2, 0x49, 0x05, + 0x22, 0xd5, 0xb8, 0xb0, 0xfa, 0x27, 0x09, 0x4a, 0xd4, 0x83, 0x66, 0x62, 0x86, 0xfb, 0x31, 0x66, + 0xb8, 0x39, 0xc5, 0x0c, 0x91, 0x1b, 0x12, 0x62, 0xb8, 0x0b, 0xe9, 0xb1, 0x63, 0xbd, 0x18, 0x87, + 0xdf, 0x8c, 0xc8, 0x2f, 0x94, 0x9d, 0xa3, 0x0d, 0xf1, 0x3c, 0x6d, 0x7c, 0x04, 0x88, 0xc4, 0x0c, + 0xee, 0x4f, 0x4d, 0x4c, 0xd1, 0x89, 0x32, 0x1d, 0x69, 0x5e, 0x4a, 0x32, 0xe9, 0x6b, 0x90, 0xcc, + 0x06, 0xc8, 0xf8, 0x38, 0xf0, 0x8c, 0x7e, 0x4c, 0x3f, 0x43, 0xf5, 0x17, 0x4f, 0x5f, 0x2d, 0x15, + 0x55, 0x32, 0x76, 0x31, 0x48, 0x11, 0xc7, 0xc6, 0x4c, 0xe2, 0x13, 0xf3, 0x0c, 0xc3, 0xb4, 0x3c, + 0x4c, 0x13, 0x6a, 0x58, 0x76, 0x5f, 0x9e, 0xa0, 0xcf, 0x98, 0xbd, 0xa6, 0x70, 0x45, 0x5d, 0x0e, + 0xa1, 0x22, 0x81, 0x8f, 0x9e, 0x40, 0x6e, 0x3f, 0xcc, 0xe7, 0xfd, 0xe7, 0xf8, 0xa4, 0x9c, 0xa5, + 0xee, 0xf6, 0xc1, 0xec, 0x99, 0x9f, 0xc7, 0xe7, 0x7e, 0x34, 0x84, 0x76, 0xa0, 0xe0, 0xf1, 0x61, + 0xb3, 0xbf, 0x77, 0x42, 0xf3, 0xcf, 0xeb, 0x80, 0xe6, 0x27, 0x30, 0x8d, 0x13, 0xf4, 0x04, 0xc0, + 0x8a, 0x58, 0x92, 0x26, 0xa9, 0xdc, 0xda, 0x87, 0xd7, 0xa0, 0x53, 0xbe, 0xd2, 0x09, 0x08, 0x7a, + 0x0a, 0xc5, 0xc9, 0x1b, 0x5d, 0x6a, 0xfe, 0x35, 0x97, 0x5a, 0x88, 0xe1, 0x34, 0x4e, 0x50, 0x0f, + 0x16, 0x48, 0xfa, 0x74, 0x7d, 0x2b, 0xc0, 0x71, 0x17, 0x28, 0x50, 0x17, 0xa8, 0x9e, 0xbe, 0x5a, + 0x42, 0x4d, 0x3e, 0x7e, 0xb1, 0x1b, 0xa0, 0xc1, 0x99, 0xf1, 0xd0, 0xa9, 0xa6, 0x9c, 0x97, 0x20, + 0x16, 0x27, 0x4e, 0xd5, 0x9d, 0xb8, 0xef, 0x39, 0xa7, 0x8a, 0xb9, 0x36, 0x41, 0x7a, 0x0a, 0xf9, + 0x29, 0x96, 0x29, 0xbd, 0x3e, 0xcb, 0x4c, 0x01, 0x21, 0x95, 0xd5, 0x47, 0x32, 0xad, 0x20, 0x3f, + 0x9c, 0xd1, 0x41, 0xcf, 0x56, 0x4a, 0xd5, 0x45, 0xc8, 0x46, 0x3e, 0x4a, 0xfa, 0xa0, 0x7a, 0xb7, + 0x19, 0xd6, 0x78, 0x8a, 0xda, 0x6d, 0xca, 0x42, 0xf5, 0x1e, 0x88, 0xb4, 0xc5, 0xc9, 0x41, 0x66, + 0xbd, 0xa3, 0x3f, 0xad, 0xeb, 0x4a, 0x58, 0x57, 0x6a, 0xed, 0x5d, 0x55, 0xef, 0xa9, 0x8a, 0x2c, + 0x54, 0x7f, 0x16, 0x60, 0x61, 0x52, 0xd1, 0xf7, 0x5c, 0x5a, 0xd3, 0x93, 0x2a, 0xd7, 0x82, 0xd2, + 0x20, 0x92, 0xf3, 0x3b, 0x85, 0xc4, 0x4a, 0x71, 0xed, 0xb3, 0xdf, 0xec, 0x0b, 0x26, 0x28, 0x71, + 0xe1, 0x64, 0xf1, 0xc5, 0xc1, 0x94, 0x34, 0xe2, 0xbb, 0xc4, 0x72, 0x62, 0x9a, 0xef, 0xaa, 0x77, + 0xa0, 0x38, 0x8d, 0x40, 0xba, 0xa8, 0xe6, 0x86, 0xda, 0x7c, 0x2c, 0xcf, 0x55, 0xff, 0x28, 0x02, + 0x9a, 0x58, 0x67, 0x6b, 0x1c, 0x18, 0xd4, 0x0e, 0x75, 0x48, 0x87, 0x07, 0x4f, 0xf9, 0x33, 0xb7, + 0xf6, 0x17, 0x57, 0x56, 0x9f, 0x13, 0x80, 0x8d, 0x39, 0x9d, 0x29, 0xa2, 0x2f, 0xe2, 0xdd, 0x54, + 0x6e, 0xed, 0xbd, 0xd9, 0xce, 0x67, 0x63, 0x8e, 0xb7, 0x59, 0x8f, 0x21, 0xe5, 0x07, 0xa4, 0xe7, + 0x48, 0xd2, 0xf3, 0x5d, 0xbd, 0x44, 0xff, 0xfc, 0xe2, 0x6b, 0x5d, 0xa2, 0xc6, 0x13, 0x25, 0xc5, + 0x40, 0x4f, 0x21, 0x1b, 0x51, 0x1a, 0x6b, 0xcd, 0x1e, 0xce, 0x0e, 0x18, 0xf9, 0x07, 0xaf, 0x8e, + 0x22, 0x2c, 0x54, 0x87, 0xdc, 0x90, 0x4d, 0x9b, 0xd4, 0x76, 0xcb, 0x2c, 0xab, 0x00, 0x47, 0xa0, + 0xd9, 0x25, 0xf6, 0xa6, 0x03, 0x57, 0xd2, 0x4c, 0x52, 0xaa, 0x7b, 0xae, 0x6d, 0xef, 0x19, 0x83, + 0xe7, 0xb4, 0xbf, 0x8a, 0x4a, 0x75, 0x2e, 0x45, 0x5b, 0x24, 0x37, 0xf0, 0x13, 0xa4, 0x1d, 0xd3, + 0xe5, 0x74, 0x74, 0x91, 0x07, 0x6d, 0xcc, 0xe9, 0x31, 0x80, 0xea, 0xdf, 0x41, 0x8a, 0x9a, 0x88, + 0xb8, 0xf4, 0x4e, 0xfb, 0x71, 0xbb, 0xf3, 0x94, 0xb4, 0x4a, 0x25, 0xc8, 0x29, 0x6a, 0x4b, 0xed, + 0xa9, 0xfd, 0x4e, 0xbb, 0xf5, 0x4c, 0x16, 0xd0, 0x6d, 0xb8, 0xc9, 0x04, 0xf5, 0xb6, 0xd2, 0x7f, + 0xaa, 0x6b, 0x7c, 0x28, 0x51, 0x5d, 0x89, 0xc7, 0xcc, 0xa4, 0xf7, 0x27, 0xd1, 0xa3, 0x28, 0xb2, + 0x40, 0xa3, 0x47, 0xef, 0x6c, 0xcb, 0x89, 0x46, 0x1e, 0xc0, 0x8c, 0x0c, 0xba, 0x29, 0x4a, 0x69, + 0x39, 0x53, 0xfd, 0xc7, 0x3b, 0x50, 0xa2, 0xd5, 0xe2, 0x4c, 0xe9, 0x7a, 0x99, 0xa6, 0xeb, 0xb0, + 0xf4, 0x93, 0xa7, 0xd2, 0x75, 0x82, 0x65, 0xea, 0x87, 0x90, 0x1d, 0x19, 0x1e, 0x76, 0x02, 0x72, + 0x02, 0xe2, 0x54, 0xc5, 0x2f, 0x6d, 0xd3, 0x81, 0x68, 0xba, 0x14, 0x4e, 0xd4, 0x88, 0x52, 0xe6, + 0x08, 0x7b, 0xf4, 0xce, 0x2d, 0x3c, 0xb4, 0xdb, 0xac, 0xdd, 0x9f, 0x9f, 0xac, 0x6a, 0x37, 0x9c, + 0xa0, 0xf3, 0x99, 0x68, 0x1b, 0xe6, 0x87, 0xae, 0x69, 0xed, 0x5b, 0x83, 0xf0, 0xc4, 0x03, 0x6b, + 0x18, 0xf6, 0xc4, 0xb9, 0xb5, 0xb7, 0x63, 0xe7, 0x31, 0x0e, 0x2c, 0xbb, 0x76, 0x68, 0x0f, 0x6a, + 0x3d, 0x7e, 0x47, 0xc9, 0x76, 0x24, 0xc7, 0xb5, 0xc9, 0x20, 0x7a, 0x04, 0x19, 0x5e, 0x83, 0x86, + 0xf7, 0x5b, 0xb3, 0x46, 0x1a, 0x43, 0xe4, 0xda, 0x68, 0x1d, 0x8a, 0x0e, 0x3e, 0x8e, 0xf7, 0x19, + 0xd9, 0x29, 0x5f, 0xcc, 0xb7, 0xf1, 0xf1, 0xc5, 0x4d, 0x46, 0xde, 0x99, 0x8c, 0x98, 0xe8, 0x09, + 0x14, 0x46, 0x9e, 0x35, 0x34, 0xbc, 0x93, 0x7e, 0x18, 0xbe, 0x70, 0x9d, 0xf0, 0x8d, 0x88, 0x3a, + 0x84, 0xa0, 0xa3, 0x68, 0x1d, 0xc2, 0xb2, 0x1e, 0xfb, 0xe5, 0x1c, 0xdd, 0xe3, 0xf5, 0xc0, 0xb8, + 0x32, 0x6a, 0x40, 0x81, 0x6e, 0x31, 0xea, 0x27, 0xf2, 0x74, 0x87, 0x8b, 0x6c, 0x87, 0x39, 0xb2, + 0xc3, 0x0b, 0x7a, 0x8a, 0x9c, 0x13, 0xc9, 0x4d, 0xb4, 0x09, 0x10, 0xdd, 0x0d, 0x93, 0x1c, 0x79, + 0x55, 0x09, 0xb2, 0xcd, 0x27, 0x4e, 0x96, 0xa4, 0xc7, 0xb4, 0xd1, 0x16, 0x64, 0x79, 0x18, 0x87, + 0xc9, 0x31, 0x77, 0xe9, 0x7d, 0xcf, 0x79, 0x52, 0xe1, 0x54, 0x12, 0x21, 0xa0, 0x36, 0xa4, 0x6c, + 0x6c, 0xf8, 0x98, 0x65, 0xc8, 0x4f, 0x2f, 0x81, 0x3a, 0x13, 0x39, 0xb5, 0xee, 0xe0, 0x10, 0x0f, + 0x8d, 0xe6, 0x21, 0xa9, 0xb6, 0x5b, 0x44, 0x5f, 0x0f, 0x61, 0x50, 0x1b, 0x64, 0x6a, 0xae, 0x38, + 0x3f, 0xc9, 0xd4, 0x62, 0xef, 0x32, 0x8b, 0x15, 0x89, 0xc5, 0x2e, 0xe5, 0x28, 0xea, 0x4f, 0x5b, + 0x13, 0x9e, 0xfa, 0x5b, 0x28, 0xee, 0xbb, 0xde, 0xd0, 0x08, 0xfa, 0x3c, 0x70, 0xe6, 0x27, 0x35, + 0xf4, 0xaf, 0xaf, 0x96, 0x0a, 0xeb, 0x74, 0x94, 0x07, 0x4d, 0x61, 0x3f, 0xfe, 0x8a, 0x36, 0x38, + 0x9d, 0xdf, 0xa0, 0xec, 0xfb, 0xd1, 0xac, 0xbb, 0x3b, 0xcf, 0xe5, 0x6d, 0x48, 0x0f, 0x0e, 0xf1, + 0xe0, 0xb9, 0x5f, 0x5e, 0xa0, 0x36, 0xff, 0xeb, 0x19, 0xa1, 0x9a, 0x44, 0x69, 0x42, 0x8f, 0x3a, + 0x43, 0x41, 0x5f, 0x43, 0xd1, 0x24, 0x12, 0xcb, 0x39, 0x60, 0x35, 0xfa, 0x4d, 0x8a, 0xbb, 0x3a, + 0x23, 0x2e, 0xa9, 0xdf, 0x35, 0x67, 0xdf, 0xe5, 0xe5, 0x19, 0x07, 0x0b, 0xeb, 0xfa, 0x0e, 0x48, + 0xfb, 0xc6, 0xd0, 0xb2, 0x2d, 0xec, 0x97, 0x6f, 0x51, 0xdc, 0xab, 0xaf, 0xdc, 0xcf, 0x5e, 0x71, + 0xf0, 0x64, 0xc0, 0x41, 0xa2, 0x40, 0xa7, 0x82, 0x13, 0x72, 0xa8, 0x6f, 0x9d, 0x0f, 0x74, 0x7e, + 0xc5, 0x31, 0x75, 0xdd, 0x41, 0x03, 0x9d, 0xbd, 0x99, 0xe8, 0x1d, 0x80, 0x23, 0x0b, 0x7f, 0xdb, + 0x7f, 0x31, 0xc6, 0xde, 0x49, 0xb9, 0x1c, 0xe3, 0xdd, 0x2c, 0x91, 0x3f, 0x21, 0x62, 0xf4, 0x09, + 0x64, 0x4d, 0x3c, 0xc2, 0x8e, 0xe9, 0x77, 0x9c, 0xf2, 0x6d, 0x5a, 0xff, 0xdd, 0x20, 0x4d, 0x89, + 0xc2, 0x85, 0x8c, 0x57, 0x27, 0xb3, 0xd0, 0x37, 0x90, 0x0f, 0x5f, 0xb0, 0xd9, 0x71, 0x1a, 0x27, + 0xe5, 0x0a, 0xdd, 0xf4, 0x83, 0x19, 0x8d, 0x39, 0x29, 0x76, 0x17, 0xf8, 0x7e, 0x94, 0x18, 0x9a, + 0x3e, 0x85, 0x8d, 0xbe, 0x86, 0x3c, 0xf7, 0xee, 0x4d, 0x77, 0xcf, 0x2f, 0xdf, 0xb9, 0xb2, 0x4d, + 0x3f, 0xfb, 0xad, 0xad, 0x89, 0x2a, 0xe7, 0xad, 0x38, 0x1a, 0xfa, 0x12, 0x0a, 0xd1, 0xdd, 0x96, + 0x3b, 0x0a, 0xfc, 0xf2, 0x5d, 0x1a, 0x98, 0x0f, 0x67, 0x75, 0x5d, 0xa6, 0xdb, 0x19, 0x05, 0xbe, + 0x9e, 0xf7, 0x63, 0x6f, 0xe8, 0x1e, 0x64, 0x4d, 0xcf, 0x1d, 0x85, 0xf9, 0xe3, 0xed, 0x65, 0x61, + 0x25, 0xc9, 0x8f, 0x99, 0x88, 0x69, 0x62, 0xe8, 0x43, 0xd1, 0xc3, 0x23, 0xdb, 0x18, 0xe0, 0x21, + 0xc9, 0x6c, 0xee, 0x7e, 0x79, 0x91, 0x7e, 0x7d, 0x6d, 0x66, 0x43, 0x46, 0xca, 0xdc, 0x31, 0x63, + 0x78, 0x9d, 0x7d, 0xb4, 0x03, 0x60, 0x8c, 0x4d, 0x2b, 0xe8, 0x0f, 0x5d, 0x13, 0x97, 0x97, 0xae, + 0xbc, 0x86, 0x3d, 0x0b, 0x5e, 0x27, 0x8a, 0x5b, 0xae, 0x89, 0xa3, 0xeb, 0x22, 0x2e, 0x40, 0x9f, + 0x40, 0x8e, 0x6e, 0xed, 0x1b, 0x77, 0x8f, 0xf8, 0xe6, 0x32, 0xdd, 0xdc, 0x3c, 0x3b, 0xcb, 0xac, + 0xe2, 0xb9, 0xa3, 0x4d, 0x77, 0x8f, 0x7a, 0x0c, 0x7b, 0x34, 0x91, 0x0f, 0xf9, 0x83, 0x41, 0x7f, + 0x42, 0xa5, 0xf7, 0xe8, 0x29, 0x7e, 0x3e, 0xe3, 0x5a, 0x1e, 0x35, 0x2f, 0x20, 0xd7, 0x1b, 0x3c, + 0x27, 0x3c, 0x6a, 0x72, 0x99, 0xaf, 0xe7, 0x0e, 0x06, 0xd1, 0x4b, 0xe5, 0x47, 0x01, 0xe6, 0xcf, + 0x51, 0x27, 0xfa, 0x07, 0xc8, 0x38, 0xae, 0x19, 0xbb, 0xde, 0x52, 0x19, 0x50, 0xba, 0xed, 0x9a, + 0xe1, 0xed, 0xd6, 0xc3, 0x03, 0x2b, 0x38, 0x1c, 0xef, 0xd5, 0x06, 0xee, 0x70, 0x35, 0x5a, 0xa1, + 0xb9, 0x37, 0x79, 0x5e, 0x1d, 0x3d, 0x3f, 0x58, 0xa5, 0x4f, 0xa3, 0xbd, 0x5a, 0xa8, 0xa6, 0xa7, + 0x09, 0xaa, 0x66, 0xa2, 0x8f, 0xa1, 0x84, 0x8f, 0x47, 0x96, 0x17, 0x2b, 0x1f, 0x12, 0xb1, 0xe3, + 0x2f, 0x4e, 0x06, 0x89, 0x13, 0x54, 0x7e, 0x11, 0xa0, 0x74, 0x86, 0xb6, 0x48, 0xa5, 0x44, 0xaf, + 0x4e, 0xa7, 0x2a, 0x25, 0x22, 0x89, 0xb5, 0x00, 0x57, 0xfd, 0x98, 0x48, 0xbe, 0xe9, 0x8f, 0x89, + 0xe9, 0x9b, 0x8a, 0xd4, 0xec, 0x37, 0x15, 0x9b, 0xa2, 0x24, 0xca, 0xa9, 0xca, 0x33, 0x90, 0x38, + 0x65, 0x4e, 0x97, 0x6e, 0xc2, 0x8c, 0xa5, 0xdb, 0xa5, 0xfb, 0xac, 0x7c, 0x2f, 0x40, 0x36, 0xfe, + 0xc7, 0x27, 0x11, 0xa1, 0x5e, 0x5c, 0x39, 0xbe, 0xe6, 0xe5, 0xe4, 0xb4, 0x05, 0x92, 0xb3, 0x5b, + 0xa0, 0x72, 0x04, 0xb9, 0x18, 0xeb, 0x9c, 0xed, 0x1e, 0x84, 0xd7, 0xe8, 0x1e, 0xde, 0x85, 0x34, + 0x0b, 0xb5, 0xd0, 0x91, 0x0a, 0x4c, 0x3b, 0x15, 0x86, 0x59, 0xea, 0x1b, 0x12, 0x62, 0x95, 0xff, + 0x10, 0x20, 0x1f, 0xe7, 0x23, 0x54, 0x85, 0xac, 0xe5, 0x0c, 0x3c, 0x4a, 0x06, 0xf4, 0xbb, 0xdc, + 0x05, 0x27, 0x62, 0xc2, 0x52, 0x43, 0xcb, 0xe9, 0xd3, 0x0b, 0xc3, 0x29, 0x37, 0x95, 0x86, 0x96, + 0xb3, 0x4b, 0xa4, 0x74, 0x8a, 0x71, 0xcc, 0xa6, 0x24, 0xa7, 0xa6, 0x18, 0xc7, 0xe1, 0x94, 0x0a, + 0x4d, 0xfc, 0x5e, 0x40, 0x2b, 0xf3, 0x64, 0x2c, 0x95, 0x7b, 0x01, 0x5a, 0x84, 0xcc, 0x91, 0xe5, + 0x05, 0x63, 0xc3, 0xa6, 0x45, 0x38, 0xef, 0x7c, 0xb8, 0xb0, 0x72, 0x08, 0xb9, 0x18, 0x8f, 0xcd, + 0x70, 0xa0, 0x7f, 0x03, 0x62, 0x14, 0x54, 0x33, 0xd6, 0xe4, 0x54, 0xa1, 0xf2, 0x2f, 0x02, 0x2c, + 0x5c, 0xc4, 0x24, 0x53, 0x2e, 0x12, 0xda, 0x69, 0x26, 0x17, 0x99, 0x62, 0xf8, 0xc4, 0x85, 0x0c, + 0x3f, 0x39, 0xb9, 0xe4, 0xe5, 0x27, 0x57, 0x7d, 0x8f, 0x37, 0x6b, 0x00, 0xe9, 0xed, 0x9d, 0x46, + 0x4b, 0x6b, 0x5e, 0xd8, 0x68, 0x91, 0x96, 0x2c, 0x62, 0x65, 0x94, 0x07, 0x49, 0xd1, 0xba, 0xf5, + 0x46, 0x4b, 0x55, 0xe4, 0x39, 0x54, 0x80, 0xac, 0xae, 0xd6, 0x15, 0xda, 0xc1, 0xc9, 0xc2, 0x67, + 0xe2, 0x77, 0x3f, 0x2c, 0x09, 0x61, 0x2b, 0xb6, 0x29, 0x4a, 0x48, 0xbe, 0x51, 0xfd, 0x51, 0x00, + 0xa4, 0x18, 0x81, 0x41, 0x18, 0xe0, 0x1a, 0x3d, 0x59, 0xe2, 0x8a, 0x83, 0x98, 0xae, 0xb3, 0x93, + 0x6f, 0x52, 0x67, 0x87, 0x0b, 0xae, 0x7e, 0x2f, 0x00, 0xc4, 0x16, 0xf7, 0x45, 0xfc, 0x77, 0xee, + 0xe5, 0x2d, 0xc5, 0x99, 0x6c, 0xb1, 0x31, 0xc7, 0x7f, 0xf6, 0x3e, 0x02, 0xc9, 0x64, 0x5b, 0x66, + 0xde, 0x72, 0x69, 0xed, 0x7e, 0xce, 0x32, 0x1b, 0xe4, 0x18, 0x99, 0xb4, 0x91, 0x81, 0xd4, 0xd8, + 0xb1, 0x5c, 0xe7, 0x03, 0x05, 0xd0, 0x79, 0xf6, 0x24, 0xc6, 0xe7, 0x6d, 0xb8, 0x19, 0x76, 0xd9, + 0x3b, 0xce, 0x51, 0x24, 0x10, 0x50, 0x11, 0x80, 0x8d, 0x5b, 0xce, 0x81, 0x9c, 0x68, 0xdc, 0x7b, + 0xf9, 0x7f, 0x8b, 0x73, 0x2f, 0x4f, 0x17, 0x85, 0xff, 0x3a, 0x5d, 0x14, 0xfe, 0xfb, 0x74, 0x51, + 0xf8, 0xdf, 0xd3, 0x45, 0xe1, 0x9f, 0xfe, 0x7f, 0x71, 0xee, 0xab, 0x0c, 0x5b, 0xd0, 0x9f, 0x03, + 0x00, 0x00, 0xff, 0xff, 0x4b, 0x20, 0x83, 0xc3, 0x39, 0x23, 0x00, 0x00, } diff --git a/pkg/sql/sqlbase/structured.proto b/pkg/sql/sqlbase/structured.proto index 8324dc1805f2..ae2129a5aad2 100644 --- a/pkg/sql/sqlbase/structured.proto +++ b/pkg/sql/sqlbase/structured.proto @@ -191,8 +191,14 @@ message ColumnType { } enum ConstraintValidity { + // The constraint is valid for all rows. Validated = 0; + // The constraint has not yet been validated for all rows (and will not be + // validated until VALIDATE CONSTRAINT is used). Unvalidated = 1; + // The constraint was just added, but the validation for existing rows is not + // yet complete. If validation fails, the constraint will be dropped. + Validating = 2; } message ForeignKeyReference { @@ -487,6 +493,14 @@ message IndexDescriptor { optional Type type = 16 [(gogoproto.nullable)=false]; } +message ConstraintToValidate { + enum ConstraintType { + CHECK = 0; + } + required ConstraintType constraint_type = 1 [(gogoproto.nullable) = false]; + required string name = 2 [(gogoproto.nullable) = false]; +} + // A DescriptorMutation represents a column or an index that // has either been added or dropped and hasn't yet transitioned // into a stable state: completely backfilled and visible, or @@ -497,6 +511,7 @@ message DescriptorMutation { oneof descriptor { ColumnDescriptor column = 1; IndexDescriptor index = 2; + ConstraintToValidate constraint = 8; } // A descriptor within a mutation is unavailable for reads, writes // and deletes. It is only available for implicit (internal to diff --git a/pkg/sql/sqlbase/table.go b/pkg/sql/sqlbase/table.go index 12969bc09a9c..6e00229ba4ea 100644 --- a/pkg/sql/sqlbase/table.go +++ b/pkg/sql/sqlbase/table.go @@ -346,7 +346,8 @@ func (desc *TableDescriptor) collectConstraintInfo( return nil, errors.Errorf("duplicate constraint name: %q", c.Name) } detail := ConstraintDetail{Kind: ConstraintTypeCheck} - detail.Unvalidated = c.Validity == ConstraintValidity_Unvalidated + // Constraints in the Validating state are considered Unvalidated for this purpose + detail.Unvalidated = c.Validity != ConstraintValidity_Validated if tableLookup != nil { detail.Details = c.Expr detail.CheckConstraint = c diff --git a/pkg/sql/update.go b/pkg/sql/update.go index 4b8468faf28d..34ef96c33262 100644 --- a/pkg/sql/update.go +++ b/pkg/sql/update.go @@ -173,7 +173,7 @@ func (p *planner) Update( // TODO(dan): This could be made tighter, just the rows needed for RETURNING // exprs. requestedCols = desc.Columns - } else if len(desc.Checks) > 0 { + } else if len(desc.AllChecks()) > 0 { // Request any columns we'll need when validating check constraints. We // could be smarter and only validate check constraints which depend on // columns that are being modified in the UPDATE statement, in which @@ -188,7 +188,7 @@ func (p *planner) Update( for _, col := range requestedCols { requestedColSet.Add(int(col.ID)) } - for _, ck := range desc.Checks { + for _, ck := range desc.AllChecks() { cols, err := ck.ColumnsUsed(desc.TableDesc()) if err != nil { return nil, err