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