From 3c49a5f22be27dd0770b9b1918b82a9e0b13d195 Mon Sep 17 00:00:00 2001 From: Lucy Zhang Date: Thu, 9 May 2019 15:51:50 -0400 Subject: [PATCH] sql: async validation of foreign keys in ADD CONSTRAINT Currently, validating an unvalidated foreign key constraint requires running `VALIDATE CONSTRAINT`, which executes a potentially long-running query in the user transaction. In this PR, `ADD CONSTRAINT` for foreign keys is now processed by the schema changer, with the validation for existing rows running in a separate transaction from the original user transaction. The foreign key mutation (represented by the new `FOREIGN_KEY` enum value in `ConstraintToUpdate`) is processed in the same way as check constraint mutations: the foreign key is in the mutations list while other columns and indexes with the same mutation ID are backfilled, then added to the appropriate index in the `Validating` state before being validated, and is finalized when the validation query for existing rows completes successfully. If validation fails, the transaction is rolled back, with the foreign key (and backreference) removed from the table descriptor(s) as part of the rollback. Adding foreign keys to columns or indexes being added is still not supported and is left for later work. Also unsupported is adding a foreign key constraint in the same transaction as `CREATE TABLE` that is either validated or that rolls back the entire transaction on failure. In this PR, the constraint is just left unvalidated; This needs a follow-up PR to queue a separate mutation for validating the constraint after it's been added. Release note (sql change): Foreign keys that are added to an existing table using `ALTER TABLE` will now be validated for existing rows, with improved performance compared to running `ADD CONSTRAINT` followed by `VALIDATE CONSTRAINT` previously. --- pkg/sql/alter_table.go | 41 +- pkg/sql/backfill.go | 232 +++++-- pkg/sql/check.go | 96 +-- pkg/sql/create_table.go | 83 ++- pkg/sql/drop_index.go | 8 + pkg/sql/drop_table.go | 18 +- .../logictest/testdata/logic_test/alter_table | 3 + pkg/sql/logictest/testdata/logic_test/fk | 394 +++--------- .../testdata/logic_test/schema_change_in_txn | 107 +++- pkg/sql/opt/exec/execbuilder/testdata/fk | 11 + pkg/sql/planner.go | 56 -- pkg/sql/schema_changer.go | 58 +- pkg/sql/show_create.go | 2 +- pkg/sql/sqlbase/structured.go | 66 +- pkg/sql/sqlbase/structured.pb.go | 564 ++++++++++-------- pkg/sql/sqlbase/structured.proto | 3 + pkg/sql/sqlbase/structured_test.go | 2 +- pkg/sql/sqlbase/table.go | 71 ++- 18 files changed, 976 insertions(+), 839 deletions(-) diff --git a/pkg/sql/alter_table.go b/pkg/sql/alter_table.go index 6c942bf434a9..1337da005289 100644 --- a/pkg/sql/alter_table.go +++ b/pkg/sql/alter_table.go @@ -491,10 +491,12 @@ func (n *alterTableNode) startExec(params runParams) error { switch constraint.Kind { case sqlbase.ConstraintTypeCheck: found := false - var idx int - for idx = range n.tableDesc.Checks { - if n.tableDesc.Checks[idx].Name == name { + var ck *sqlbase.TableDescriptor_CheckConstraint + for _, c := range n.tableDesc.Checks { + // If the constraint is still being validated, don't allow VALIDATE CONSTRAINT to run + if c.Name == name && c.Validity != sqlbase.ConstraintValidity_Validating { found = true + ck = c break } } @@ -502,45 +504,42 @@ func (n *alterTableNode) startExec(params runParams) error { return pgerror.Newf(pgerror.CodeObjectNotInPrerequisiteStateError, "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.tableDesc.TableDesc(), params.EvalContext().InternalExecutor, params.EvalContext().Txn, + if err := validateCheckInTxn( + params.ctx, params.p.LeaseMgr(), params.EvalContext(), n.tableDesc, params.EvalContext().Txn, name, ); err != nil { return err } - n.tableDesc.Checks[idx].Validity = sqlbase.ConstraintValidity_Validated - descriptorChanged = true + ck.Validity = sqlbase.ConstraintValidity_Validated case sqlbase.ConstraintTypeFK: found := false - var id sqlbase.IndexID + var fkIdx *sqlbase.IndexDescriptor for _, idx := range n.tableDesc.AllNonDropIndexes() { - if idx.ForeignKey.IsSet() && idx.ForeignKey.Name == name { + fk := &idx.ForeignKey + // If the constraint is still being validated, don't allow VALIDATE CONSTRAINT to run + if fk.IsSet() && fk.Name == name && fk.Validity != sqlbase.ConstraintValidity_Validating { found = true - id = idx.ID + fkIdx = idx break } } if !found { - return pgerror.AssertionFailedf( - "constraint returned by GetConstraintInfo not found") - } - idx, err := n.tableDesc.FindIndexByID(id) - if err != nil { - return pgerror.NewAssertionErrorWithWrappedErrf(err, "") + return pgerror.Newf(pgerror.CodeObjectNotInPrerequisiteStateError, + "constraint %q in the middle of being added, try again later", t.Constraint) } - if err := params.p.validateForeignKey(params.ctx, n.tableDesc.TableDesc(), idx); err != nil { + if err := validateFkInTxn( + params.ctx, params.p.LeaseMgr(), params.EvalContext(), n.tableDesc, params.EvalContext().Txn, name, + ); err != nil { return err } - idx.ForeignKey.Validity = sqlbase.ConstraintValidity_Validated - descriptorChanged = true + fkIdx.ForeignKey.Validity = sqlbase.ConstraintValidity_Validated default: return pgerror.Newf(pgerror.CodeWrongObjectTypeError, "constraint %q of relation %q is not a foreign key or check constraint", tree.ErrString(&t.Constraint), tree.ErrString(&n.n.Table)) } + descriptorChanged = true case tree.ColumnMutationCmd: // Column mutations diff --git a/pkg/sql/backfill.go b/pkg/sql/backfill.go index 8af0193b288b..d2c2c0c4225b 100644 --- a/pkg/sql/backfill.go +++ b/pkg/sql/backfill.go @@ -36,6 +36,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/pkg/errors" ) const ( @@ -125,8 +126,8 @@ func (sc *SchemaChanger) runBackfill( var droppedIndexDescs []sqlbase.IndexDescriptor var addedIndexSpans []roachpb.Span - var addedChecks []*sqlbase.TableDescriptor_CheckConstraint - var checksToValidate []sqlbase.ConstraintToUpdate + var constraintsToAdd []sqlbase.ConstraintToUpdate + var constraintsToValidate []sqlbase.ConstraintToUpdate tableDesc, err := sc.updateJobRunningStatus(ctx, RunningStatusBackfill) if err != nil { @@ -157,14 +158,8 @@ func (sc *SchemaChanger) runBackfill( case *sqlbase.DescriptorMutation_Index: addedIndexSpans = append(addedIndexSpans, tableDesc.IndexSpan(t.Index.ID)) case *sqlbase.DescriptorMutation_Constraint: - switch t.Constraint.ConstraintType { - case sqlbase.ConstraintToUpdate_CHECK: - addedChecks = append(addedChecks, &t.Constraint.Check) - checksToValidate = append(checksToValidate, *t.Constraint) - default: - return pgerror.AssertionFailedf( - "unsupported constraint type: %d", log.Safe(t.Constraint.ConstraintType)) - } + constraintsToAdd = append(constraintsToAdd, *t.Constraint) + constraintsToValidate = append(constraintsToValidate, *t.Constraint) default: return pgerror.AssertionFailedf( "unsupported mutation: %+v", m) @@ -216,7 +211,7 @@ func (sc *SchemaChanger) runBackfill( } } - // Add check constraints, publish the new version of the table descriptor, + // Add check and foreign key constraints, publish the new version of the table descriptor, // and wait until the entire cluster is on the new version. This is basically // a state transition for the schema change, which must happen after the // columns are backfilled and before constraint validation begins. This @@ -225,44 +220,63 @@ func (sc *SchemaChanger) runBackfill( // a constraint references both public and non-public columns), and 2) the // validation occurs only when the entire cluster is already enforcing the // constraint on insert/update. - if len(addedChecks) > 0 { - if err := sc.addChecks(ctx, addedChecks); err != nil { + if len(constraintsToAdd) > 0 { + if err := sc.AddConstraints(ctx, constraintsToAdd); err != nil { return err } } - // Validate check constraints. - if len(checksToValidate) > 0 { - if err := sc.validateChecks(ctx, evalCtx, lease, checksToValidate); err != nil { + // Validate check and foreign key constraints. + if len(constraintsToValidate) > 0 { + if err := sc.validateConstraints(ctx, evalCtx, lease, constraintsToValidate); err != nil { return err } } return nil } -// addChecks publishes a new version of the given table descriptor with the +// AddConstraints publishes a new version of the given table descriptor with the // given check constraint added to it, and waits until the entire cluster is on // the new version of the table descriptor. -func (sc *SchemaChanger) addChecks( - ctx context.Context, addedChecks []*sqlbase.TableDescriptor_CheckConstraint, +func (sc *SchemaChanger) AddConstraints( + ctx context.Context, constraints []sqlbase.ConstraintToUpdate, ) error { _, err := sc.leaseMgr.Publish(ctx, sc.tableID, func(desc *sqlbase.MutableTableDescriptor) error { - for i, added := range addedChecks { - found := false - for _, c := range desc.Checks { - if c.Name == added.Name { - log.VEventf( - ctx, 2, - "backfiller tried to add constraint %+v but found existing constraint %+v, presumably due to a retry", - added, c, - ) - found = true - break + for i, added := range constraints { + switch added.ConstraintType { + case sqlbase.ConstraintToUpdate_CHECK: + found := false + for _, c := range desc.Checks { + if c.Name == added.Name { + log.VEventf( + ctx, 2, + "backfiller tried to add constraint %+v but found existing constraint %+v, presumably due to a retry", + added, c, + ) + found = true + break + } + } + if !found { + desc.Checks = append(desc.Checks, &constraints[i].Check) + } + case sqlbase.ConstraintToUpdate_FOREIGN_KEY: + idx, err := desc.FindIndexByID(added.ForeignKeyIndex) + if err != nil { + return err + } + if idx.ForeignKey.IsSet() { + if log.V(2) { + log.VEventf( + ctx, 2, + "backfiller tried to add constraint %+v but found existing constraint %+v, presumably due to a retry", + added, idx.ForeignKey, + ) + } + } else { + idx.ForeignKey = added.ForeignKey } - } - if !found { - desc.Checks = append(desc.Checks, addedChecks[i]) } } return nil @@ -275,11 +289,11 @@ func (sc *SchemaChanger) addChecks( return sc.waitToUpdateLeases(ctx, sc.tableID) } -func (sc *SchemaChanger) validateChecks( +func (sc *SchemaChanger) validateConstraints( ctx context.Context, evalCtx *extendedEvalContext, lease *sqlbase.TableDescriptor_SchemaChangeLease, - checks []sqlbase.ConstraintToUpdate, + constraints []sqlbase.ConstraintToUpdate, ) error { if testDisableTableLeases { return nil @@ -311,10 +325,10 @@ func (sc *SchemaChanger) validateChecks( grp := ctxgroup.WithContext(ctx) // Notify when validation is finished (or has returned an error) for a check. - countDone := make(chan struct{}, len(checks)) + countDone := make(chan struct{}, len(constraints)) - for i := range checks { - c := checks[i] + for i := range constraints { + c := constraints[i] grp.GoCtx(func(ctx context.Context) error { defer func() { countDone <- struct{}{} }() @@ -333,13 +347,25 @@ func (sc *SchemaChanger) validateChecks( // Create a new eval context only because the eval context cannot be shared across many // goroutines. newEvalCtx := createSchemaChangeEvalCtx(ctx, readAsOf, evalCtx.Tracing, sc.ieFactory) - return validateCheckInTxn(ctx, sc.leaseMgr, &newEvalCtx.EvalContext, desc, txn, &c.Name) + switch c.ConstraintType { + case sqlbase.ConstraintToUpdate_CHECK: + if err := validateCheckInTxn(ctx, sc.leaseMgr, &newEvalCtx.EvalContext, desc, txn, c.Name); err != nil { + return err + } + case sqlbase.ConstraintToUpdate_FOREIGN_KEY: + if err := validateFkInTxn(ctx, sc.leaseMgr, &newEvalCtx.EvalContext, desc, txn, c.Name); err != nil { + return err + } + default: + return errors.Errorf("unsupported constraint type: %d", c.ConstraintType) + } + return nil }) } // Periodic schema change lease extension. grp.GoCtx(func(ctx context.Context) error { - count := len(checks) + count := len(constraints) refreshTimer := timeutil.NewTimer() defer refreshTimer.Stop() refreshTimer.Reset(checkpointInterval) @@ -1114,7 +1140,7 @@ func runSchemaChangesInTxn( // all column mutations. doneColumnBackfill := false // Checks are validated after all other mutations have been applied. - var checksToValidate []sqlbase.ConstraintToUpdate + var constraintsToValidate []sqlbase.ConstraintToUpdate for _, m := range tableDesc.Mutations { immutDesc := sqlbase.NewImmutableTableDescriptor(*tableDesc.TableDesc()) @@ -1139,11 +1165,17 @@ func runSchemaChangesInTxn( switch t.Constraint.ConstraintType { case sqlbase.ConstraintToUpdate_CHECK: tableDesc.Checks = append(tableDesc.Checks, &t.Constraint.Check) - checksToValidate = append(checksToValidate, *t.Constraint) + case sqlbase.ConstraintToUpdate_FOREIGN_KEY: + idx, err := tableDesc.FindIndexByID(t.Constraint.ForeignKeyIndex) + if err != nil { + return err + } + idx.ForeignKey = t.Constraint.ForeignKey default: return pgerror.AssertionFailedf( "unsupported constraint type: %d", log.Safe(t.Constraint.ConstraintType)) } + constraintsToValidate = append(constraintsToValidate, *t.Constraint) default: return pgerror.AssertionFailedf( @@ -1184,44 +1216,126 @@ func runSchemaChangesInTxn( // 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 - for _, c := range checksToValidate { - if err := validateCheckInTxn(ctx, tc.leaseMgr, evalCtx, tableDesc, txn, &c.Name); err != nil { - return err + for _, c := range constraintsToValidate { + switch c.ConstraintType { + case sqlbase.ConstraintToUpdate_CHECK: + if err := validateCheckInTxn(ctx, tc.leaseMgr, evalCtx, tableDesc, txn, c.Name); err != nil { + return err + } + case sqlbase.ConstraintToUpdate_FOREIGN_KEY: + // We can't support adding a validated foreign key constraint in the same + // transaction as the CREATE TABLE statement. This would require adding + // the backreference to the other table and then validating the constraint + // for whatever rows were inserted into the referencing table in this + // transaction, which requires multiple schema changer states across + // multiple transactions. + // TODO (lucy): Add a validation job that runs after the user transaction. + // This won't roll back the original transaction if validation fails, but + // it will at least leave the constraint in the Validated state if + // validation succeeds. + + // For now, revert the constraint to an unvalidated state. + idx, err := tableDesc.FindIndexByID(c.ForeignKeyIndex) + if err != nil { + return err + } + idx.ForeignKey.Validity = sqlbase.ConstraintValidity_Unvalidated + default: + return pgerror.AssertionFailedf( + "unsupported constraint type: %d", log.Safe(c.ConstraintType)) } } return nil } // 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. +// transaction. If the provided table descriptor version is newer than the +// cluster version, it will be used in the InternalExecutor that performs the +// validation query. +// TODO (lucy): The special case where the table descriptor version is the same +// as the cluster version only happens because the query in VALIDATE CONSTRAINT +// still runs in the user transaction instead of a step in the schema changer. +// When that's no longer true, this function should be updated. func validateCheckInTxn( ctx context.Context, leaseMgr *LeaseManager, evalCtx *tree.EvalContext, tableDesc *MutableTableDescriptor, txn *client.Txn, - checkName *string, + checkName string, ) 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 - }() + if tableDesc.Version > tableDesc.ClusterVersion.Version { + newTc := &TableCollection{leaseMgr: leaseMgr} + // pretend that the schema has been modified. + if err := newTc.addUncommittedTable(*tableDesc); err != nil { + return err + } - check, err := tableDesc.FindCheckByName(*checkName) + ie.impl.tcModifier = newTc + defer func() { + ie.impl.tcModifier = nil + }() + } + + check, err := tableDesc.FindCheckByName(checkName) if err != nil { return err } return validateCheckExpr(ctx, check.Expr, tableDesc.TableDesc(), ie, txn) } +// validateFkInTxn validates foreign key constraints within the provided +// transaction. If the provided table descriptor version is newer than the +// cluster version, it will be used in the InternalExecutor that performs the +// validation query. +// TODO (lucy): The special case where the table descriptor version is the same +// as the cluster version only happens because the query in VALIDATE CONSTRAINT +// still runs in the user transaction instead of a step in the schema changer. +// When that's no longer true, this function should be updated. +func validateFkInTxn( + ctx context.Context, + leaseMgr *LeaseManager, + evalCtx *tree.EvalContext, + tableDesc *MutableTableDescriptor, + txn *client.Txn, + fkName string, +) error { + ie := evalCtx.InternalExecutor.(*SessionBoundInternalExecutor) + if tableDesc.Version > tableDesc.ClusterVersion.Version { + newTc := &TableCollection{leaseMgr: leaseMgr} + // pretend that the schema has been modified. + if err := newTc.addUncommittedTable(*tableDesc); err != nil { + return err + } + + ie.impl.tcModifier = newTc + defer func() { + ie.impl.tcModifier = nil + }() + } + + var fkIdx *sqlbase.IndexDescriptor + if tableDesc.PrimaryIndex.ForeignKey.IsSet() && tableDesc.PrimaryIndex.ForeignKey.Name == fkName { + fkIdx = &tableDesc.PrimaryIndex + } else { + found := false + for i := range tableDesc.Indexes { + idx := &tableDesc.Indexes[i] + if idx.ForeignKey.IsSet() && idx.ForeignKey.Name == fkName { + fkIdx = idx + found = true + break + } + } + if !found { + return fmt.Errorf("foreign key %s does not exist", fkName) + } + } + + return validateForeignKey(ctx, tableDesc.TableDesc(), fkIdx, ie, txn) +} + // columnBackfillInTxn backfills columns for all mutation columns in // the mutation list. func columnBackfillInTxn( diff --git a/pkg/sql/check.go b/pkg/sql/check.go index e7c5dba71799..d22bb963853a 100644 --- a/pkg/sql/check.go +++ b/pkg/sql/check.go @@ -76,7 +76,7 @@ func validateCheckExpr( // NOT ((COALESCE(a_id, b_id) IS NULL) OR (a_id IS NOT NULL AND b_id IS NOT NULL)) // LIMIT 1; func matchFullUnacceptableKeyQuery( - prefix int, srcName *string, srcIdx *sqlbase.IndexDescriptor, + prefix int, srcID sqlbase.ID, srcIdx *sqlbase.IndexDescriptor, ) string { srcCols, srcNotNullClause := make([]string, prefix), make([]string, prefix) for i := 0; i < prefix; i++ { @@ -84,8 +84,8 @@ func matchFullUnacceptableKeyQuery( srcNotNullClause[i] = fmt.Sprintf("%s IS NOT NULL", tree.NameString(srcIdx.ColumnNames[i])) } return fmt.Sprintf( - `SELECT * FROM %s@%s WHERE NOT ((COALESCE(%s) IS NULL) OR (%s)) LIMIT 1`, - *srcName, tree.NameString(srcIdx.Name), + `SELECT * FROM [%d AS tbl]@[%d] WHERE NOT ((COALESCE(%s) IS NULL) OR (%s)) LIMIT 1`, + srcID, srcIdx.ID, strings.Join(srcCols, ", "), strings.Join(srcNotNullClause, " AND "), ) @@ -111,9 +111,9 @@ func matchFullUnacceptableKeyQuery( // LIMIT 1; func nonMatchingRowQuery( prefix int, - srcName *string, + srcID sqlbase.ID, srcIdx *sqlbase.IndexDescriptor, - targetName *string, + targetID sqlbase.ID, targetIdx *sqlbase.IndexDescriptor, ) string { srcCols, srcWhere, targetCols, on := make([]string, prefix), make([]string, prefix), make([]string, prefix), make([]string, prefix) @@ -127,21 +127,25 @@ func nonMatchingRowQuery( } return fmt.Sprintf( - `SELECT %s FROM (SELECT * FROM %s@%s WHERE %s) AS s LEFT OUTER JOIN %s@%s AS t ON %s WHERE %s IS NULL LIMIT 1`, + `SELECT %s FROM (SELECT * FROM [%d AS src]@[%d] WHERE %s) AS s LEFT OUTER JOIN [%d AS target]@[%d] AS t ON %s WHERE %s IS NULL LIMIT 1`, strings.Join(srcCols, ", "), - *srcName, tree.NameString(srcIdx.Name), + srcID, srcIdx.ID, strings.Join(srcWhere, " AND "), - *targetName, tree.NameString(targetIdx.Name), + targetID, targetIdx.ID, strings.Join(on, " AND "), // Sufficient to check the first column to see whether there was no matching row targetCols[0], ) } -func (p *planner) validateForeignKey( - ctx context.Context, srcTable *sqlbase.TableDescriptor, srcIdx *sqlbase.IndexDescriptor, +func validateForeignKey( + ctx context.Context, + srcTable *sqlbase.TableDescriptor, + srcIdx *sqlbase.IndexDescriptor, + ie tree.SessionBoundInternalExecutor, + txn *client.Txn, ) error { - targetTable, err := sqlbase.GetTableDescFromID(ctx, p.txn, srcIdx.ForeignKey.Table) + targetTable, err := sqlbase.GetTableDescFromID(ctx, txn, srcIdx.ForeignKey.Table) if err != nil { return err } @@ -150,16 +154,6 @@ func (p *planner) validateForeignKey( return err } - srcName, err := p.getQualifiedTableName(ctx, srcTable) - if err != nil { - return err - } - - targetName, err := p.getQualifiedTableName(ctx, targetTable) - if err != nil { - return err - } - prefix := len(srcIdx.ColumnNames) if p := len(targetIdx.ColumnNames); p < prefix { prefix = p @@ -169,7 +163,7 @@ func (p *planner) validateForeignKey( // null and non-null values exist. // (The matching options only matter for FKs with more than one column.) if prefix > 1 && srcIdx.ForeignKey.Match == sqlbase.ForeignKeyReference_FULL { - query := matchFullUnacceptableKeyQuery(prefix, &srcName, srcIdx) + query := matchFullUnacceptableKeyQuery(prefix, srcTable.ID, srcIdx) log.Infof(ctx, "Validating MATCH FULL FK %q (%q [%v] -> %q [%v]) with query %q", srcIdx.ForeignKey.Name, @@ -177,31 +171,18 @@ func (p *planner) validateForeignKey( query, ) - plan, err := p.delegateQuery(ctx, "ALTER TABLE VALIDATE", query, nil, nil) - if err != nil { - return err - } - - plan, err = p.optimizePlan(ctx, plan, allColumns(plan)) + rows, err := ie.QueryRow(ctx, "validate foreign key constraint", txn, query) if err != nil { return err } - defer plan.Close(ctx) - - rows, err := p.runWithDistSQL(ctx, plan) - if err != nil { - return err - } - defer rows.Close(ctx) - if rows.Len() > 0 { return pgerror.Newf(pgerror.CodeForeignKeyViolationError, "foreign key violation: MATCH FULL does not allow mixing of null and nonnull values %s for %s", - rows.At(0), srcIdx.ForeignKey.Name, + rows, srcIdx.ForeignKey.Name, ) } } - query := nonMatchingRowQuery(prefix, &srcName, srcIdx, &targetName, targetIdx) + query := nonMatchingRowQuery(prefix, srcTable.ID, srcIdx, targetTable.ID, targetIdx) log.Infof(ctx, "Validating FK %q (%q [%v] -> %q [%v]) with query %q", srcIdx.ForeignKey.Name, @@ -209,36 +190,21 @@ func (p *planner) validateForeignKey( query, ) - plan, err := p.delegateQuery(ctx, "ALTER TABLE VALIDATE", query, nil, nil) - if err != nil { - return err - } - - plan, err = p.optimizePlan(ctx, plan, allColumns(plan)) + values, err := ie.QueryRow(ctx, "validate fk constraint", txn, query) if err != nil { return err } - defer plan.Close(ctx) - - rows, err := p.runWithDistSQL(ctx, plan) - if err != nil { - return err - } - defer rows.Close(ctx) - - if rows.Len() == 0 { - return nil - } - - values := rows.At(0) - var pairs bytes.Buffer - for i := range values { - if i > 0 { - pairs.WriteString(", ") + if values.Len() > 0 { + var pairs bytes.Buffer + for i := range values { + if i > 0 { + pairs.WriteString(", ") + } + pairs.WriteString(fmt.Sprintf("%s=%v", srcIdx.ColumnNames[i], values[i])) } - pairs.WriteString(fmt.Sprintf("%s=%v", srcIdx.ColumnNames[i], values[i])) + return pgerror.Newf(pgerror.CodeForeignKeyViolationError, + "foreign key violation: %q row %s has no match in %q", + srcTable.Name, pairs.String(), targetTable.Name) } - return pgerror.Newf(pgerror.CodeForeignKeyViolationError, - "foreign key violation: %q row %s has no match in %q", - srcTable.Name, pairs.String(), targetTable.Name) + return nil } diff --git a/pkg/sql/create_table.go b/pkg/sql/create_table.go index c552a242301c..8110d72fbe7e 100644 --- a/pkg/sql/create_table.go +++ b/pkg/sql/create_table.go @@ -569,44 +569,52 @@ func ResolveFK( } if ts != NewTable { - ref.Validity = sqlbase.ConstraintValidity_Unvalidated + ref.Validity = sqlbase.ConstraintValidity_Validating } backref := sqlbase.ForeignKeyReference{Table: tbl.ID} + var idx *sqlbase.IndexDescriptor + found := false if matchesIndex(srcCols, tbl.PrimaryIndex, matchPrefix) { if tbl.PrimaryIndex.ForeignKey.IsSet() { return pgerror.Newf(pgerror.CodeInvalidForeignKeyError, "columns cannot be used by multiple foreign key constraints") } - tbl.PrimaryIndex.ForeignKey = ref - backref.Index = tbl.PrimaryIndex.ID + idx = &tbl.PrimaryIndex + found = true } else { - found := false for i := range tbl.Indexes { if matchesIndex(srcCols, tbl.Indexes[i], matchPrefix) { if tbl.Indexes[i].ForeignKey.IsSet() { return pgerror.Newf(pgerror.CodeInvalidForeignKeyError, "columns cannot be used by multiple foreign key constraints") } - tbl.Indexes[i].ForeignKey = ref - backref.Index = tbl.Indexes[i].ID + idx = &tbl.Indexes[i] found = true break } } - if !found { - // Avoid unexpected index builds from ALTER TABLE ADD CONSTRAINT. - if ts == NonEmptyTable { - return pgerror.Newf(pgerror.CodeInvalidForeignKeyError, - "foreign key requires an existing index on columns %s", colNames(srcCols)) - } - added, err := addIndexForFK(tbl, srcCols, constraintName, ref, ts) - if err != nil { - return err - } - backref.Index = added + } + if found { + if ts == NewTable { + idx.ForeignKey = ref + } else { + tbl.AddForeignKeyValidationMutation(&ref, idx.ID) } + backref.Index = idx.ID + } else { + // Avoid unexpected index builds from ALTER TABLE ADD CONSTRAINT. + if ts == NonEmptyTable { + return pgerror.Newf(pgerror.CodeInvalidForeignKeyError, + "foreign key requires an existing index on columns %s", colNames(srcCols)) + } + added, err := addIndexForFK(tbl, srcCols, constraintName, ref, ts) + if err != nil { + return err + } + backref.Index = added } + if targetIdxIndex > -1 { target.Indexes[targetIdxIndex].ReferencedBy = append(target.Indexes[targetIdxIndex].ReferencedBy, backref) } else { @@ -616,19 +624,26 @@ func ResolveFK( // Multiple FKs from the same column would potentially result in ambiguous or // unexpected behavior with conflicting CASCADE/RESTRICT/etc behaviors. colsInFKs := make(map[sqlbase.ColumnID]struct{}) - for _, idx := range tbl.Indexes { - if idx.ForeignKey.IsSet() { - numCols := len(idx.ColumnIDs) - if idx.ForeignKey.SharedPrefixLen > 0 { - numCols = int(idx.ForeignKey.SharedPrefixLen) - } - for i := 0; i < numCols; i++ { - if _, ok := colsInFKs[idx.ColumnIDs[i]]; ok { - return pgerror.Newf(pgerror.CodeInvalidForeignKeyError, - "column %q cannot be used by multiple foreign key constraints", idx.ColumnNames[i]) - } - colsInFKs[idx.ColumnIDs[i]] = struct{}{} + + fks, err := tbl.AllActiveAndInactiveForeignKeys() + if err != nil { + return err + } + for id, fk := range fks { + idx, err := tbl.FindIndexByID(id) + if err != nil { + return err + } + numCols := len(idx.ColumnIDs) + if fk.SharedPrefixLen > 0 { + numCols = int(fk.SharedPrefixLen) + } + for i := 0; i < numCols; i++ { + if _, ok := colsInFKs[idx.ColumnIDs[i]]; ok { + return pgerror.Newf(pgerror.CodeInvalidForeignKeyError, + "column %q cannot be used by multiple foreign key constraints", idx.ColumnNames[i]) } + colsInFKs[idx.ColumnIDs[i]] = struct{}{} } } @@ -649,7 +664,6 @@ func addIndexForFK( Name: fmt.Sprintf("%s_auto_index_%s", tbl.Name, constraintName), ColumnNames: make([]string, len(srcCols)), ColumnDirections: make([]sqlbase.IndexDescriptor_Direction, len(srcCols)), - ForeignKey: ref, } for i, c := range srcCols { idx.ColumnDirections[i] = sqlbase.IndexDescriptor_ASC @@ -657,6 +671,7 @@ func addIndexForFK( } if ts == NewTable { + idx.ForeignKey = ref if err := tbl.AddIndex(idx, false); err != nil { return 0, err } @@ -674,13 +689,19 @@ func addIndexForFK( return added.ID, nil } + // TODO (lucy): In the EmptyTable case, we add an index mutation, making this + // the only case where a foreign key is added to an index being added. + // Allowing FKs to be added to other indexes/columns also being added should + // be a generalization of this special case. if err := tbl.AddIndexMutation(&idx, sqlbase.DescriptorMutation_ADD); err != nil { return 0, err } if err := tbl.AllocateIDs(); err != nil { return 0, err } - return tbl.Mutations[len(tbl.Mutations)-1].GetIndex().ID, nil + id := tbl.Mutations[len(tbl.Mutations)-1].GetIndex().ID + tbl.AddForeignKeyValidationMutation(&ref, id) + return id, nil } // colNames converts a []colDesc to a human-readable string for use in error messages. diff --git a/pkg/sql/drop_index.go b/pkg/sql/drop_index.go index 82a8d3b137ca..6aed800b3463 100644 --- a/pkg/sql/drop_index.go +++ b/pkg/sql/drop_index.go @@ -151,6 +151,14 @@ func (p *planner) dropIndexByName( } } + // Check for foreign key mutations referencing this index. + for _, m := range tableDesc.Mutations { + if c := m.GetConstraint(); c != nil && c.ConstraintType == sqlbase.ConstraintToUpdate_FOREIGN_KEY && c.ForeignKeyIndex == idx.ID { + return pgerror.Newf(pgerror.CodeObjectNotInPrerequisiteStateError, + "referencing constraint %q in the middle of being added, try again later", c.ForeignKey.Name) + } + } + // Queue the mutation. var droppedViews []string if idx.ForeignKey.IsSet() { diff --git a/pkg/sql/drop_table.go b/pkg/sql/drop_table.go index 4956ff4de568..e320044c2740 100644 --- a/pkg/sql/drop_table.go +++ b/pkg/sql/drop_table.go @@ -422,16 +422,28 @@ func (p *planner) removeFKBackReference( // The referenced table is being dropped. No need to modify it further. return nil } - targetIdx, err := t.FindIndexByID(idx.ForeignKey.Index) + if err := removeFKBackReferenceFromTable(t, idx.ForeignKey.Index, tableDesc.ID, idx.ID); err != nil { + return err + } + return p.writeSchemaChange(ctx, t, sqlbase.InvalidMutationID) +} + +func removeFKBackReferenceFromTable( + targetDesc *sqlbase.MutableTableDescriptor, + referencedIdx sqlbase.IndexID, + source sqlbase.ID, + sourceIdx sqlbase.IndexID, +) error { + targetIdx, err := targetDesc.FindIndexByID(referencedIdx) if err != nil { return err } for k, ref := range targetIdx.ReferencedBy { - if ref.Table == tableDesc.ID && ref.Index == idx.ID { + if ref.Table == source && ref.Index == sourceIdx { targetIdx.ReferencedBy = append(targetIdx.ReferencedBy[:k], targetIdx.ReferencedBy[k+1:]...) } } - return p.writeSchemaChange(ctx, t, sqlbase.InvalidMutationID) + return nil } func (p *planner) removeInterleaveBackReference( diff --git a/pkg/sql/logictest/testdata/logic_test/alter_table b/pkg/sql/logictest/testdata/logic_test/alter_table index 68a8adab5730..f3f7003b0c14 100644 --- a/pkg/sql/logictest/testdata/logic_test/alter_table +++ b/pkg/sql/logictest/testdata/logic_test/alter_table @@ -128,6 +128,9 @@ t primary PRIMARY KEY PRIMARY KEY (a ASC) true statement error duplicate constraint name ALTER TABLE t ADD CONSTRAINT check_a CHECK (a > 0) +statement error duplicate constraint name +ALTER TABLE t ADD CONSTRAINT fk_f_ref_other FOREIGN KEY (a) REFERENCES other (b) + # added constraints with generated names avoid name collisions. statement ok ALTER TABLE t ADD CHECK (a > 0) diff --git a/pkg/sql/logictest/testdata/logic_test/fk b/pkg/sql/logictest/testdata/logic_test/fk index 7db66cbbd733..7b4515c43924 100644 --- a/pkg/sql/logictest/testdata/logic_test/fk +++ b/pkg/sql/logictest/testdata/logic_test/fk @@ -306,33 +306,19 @@ ALTER TABLE delivery DROP CONSTRAINT fk_item_ref_products statement ok UPDATE products SET upc = 'blah' WHERE sku = '780' -statement ok -ALTER TABLE delivery ADD FOREIGN KEY (item) REFERENCES products (upc) - -query TTTTB -SHOW CONSTRAINTS FROM delivery ----- -delivery fk_item_ref_products FOREIGN KEY FOREIGN KEY (item) REFERENCES products (upc) false -delivery fk_order_ref_orders FOREIGN KEY FOREIGN KEY ("order", shipment) REFERENCES orders (id, shipment) true - statement error pgcode 23503 foreign key violation: "delivery" row item='885155001450' has no match in "products" -ALTER TABLE delivery VALIDATE CONSTRAINT fk_item_ref_products +ALTER TABLE delivery ADD FOREIGN KEY (item) REFERENCES products (upc) query TTTTB SHOW CONSTRAINTS FROM delivery ---- -delivery fk_item_ref_products FOREIGN KEY FOREIGN KEY (item) REFERENCES products (upc) false delivery fk_order_ref_orders FOREIGN KEY FOREIGN KEY ("order", shipment) REFERENCES orders (id, shipment) true statement ok UPDATE products SET upc = '885155001450' WHERE sku = '780' -# Changing referenced non-pk index fails once again with constraint re-added. -statement error pgcode 23503 foreign key violation: values \['885155001450'\] in columns \[upc\] referenced in table "delivery" -UPDATE products SET upc = 'blah' WHERE sku = '780' - statement ok -ALTER TABLE delivery VALIDATE CONSTRAINT fk_item_ref_products +ALTER TABLE delivery ADD FOREIGN KEY (item) REFERENCES products (upc) query TTTTB SHOW CONSTRAINTS FROM delivery @@ -1365,6 +1351,56 @@ ALTER TABLE t ADD FOREIGN KEY (a, b) REFERENCES t (a, b) statement ok DROP TABLE t +subtest unvalidated_fk_plan + +# To get an unvalidated foreign key for testing, use the loophole that we +# currently don't support adding a validated FK in the same transaction as +# CREATE TABLE +# TODO (lucy): Once this is no longer true (and we support adding NOT VALID +# constraints), update this test +statement ok +BEGIN + +statement ok +CREATE TABLE a ( + x STRING NULL, + y STRING NULL, + z STRING NULL, + CONSTRAINT "primary" PRIMARY KEY (z, y, x) +) + +statement ok +CREATE TABLE b ( + a_y STRING NULL, + a_x STRING NULL, + a_z STRING NULL, + INDEX idx (a_z, a_y, a_x) +) + +statement ok +INSERT INTO b (a_x, a_y, a_z) VALUES ('x2', 'y1', 'z1') + +statement ok +ALTER TABLE b ADD CONSTRAINT fk_ref FOREIGN KEY (a_z, a_y, a_x) REFERENCES a (z, y, x) + +statement ok +COMMIT + +# Verify that the optimizer doesn't use an unvalidated constraint to simplify plans. +query TTT +SELECT + s.a_z, s.a_y, s.a_x +FROM + (SELECT * FROM b WHERE a_z IS NOT NULL AND a_y IS NOT NULL AND a_x IS NOT NULL) AS s + LEFT JOIN a AS t ON s.a_z = t.z AND s.a_y = t.y AND s.a_x = t.x +WHERE + t.z IS NULL +---- +z1 y1 x2 + +statement ok +DROP TABLE a, b + subtest Composite_Simple # Originally from 26748. @@ -1507,62 +1543,9 @@ INSERT INTO b (a_x, a_y, a_z) VALUES (NULL, 'y2', 'z2') statement ok DROP TABLE b, a -subtest Composite_Simple_Unvalidated -# Test inserting into table with an unvalidated constraint, and running VALIDATE CONSTRAINT later - -# Test composite key with two columns. -statement ok -CREATE TABLE a ( - x STRING NULL - ,y STRING NULL - ,CONSTRAINT "primary" PRIMARY KEY (y, x) -); - -statement ok -CREATE TABLE b ( - a_y STRING NULL - ,a_x STRING NULL -); - -# Add the constraint separately so that it's unvalidated, so we can test VALIDATE CONSTRAINT. -statement ok -ALTER TABLE b ADD CONSTRAINT fk_ref FOREIGN KEY (a_y, a_x) REFERENCES a (y, x) - -statement ok -INSERT INTO a (x, y) VALUES ('x1', 'y1') - -# All of these are allowed because we do composite matching using MATCH SIMPLE. -statement ok -INSERT INTO b (a_x) VALUES ('x1') - -statement ok -INSERT INTO b (a_y) VALUES ('y1') - -statement ok -INSERT INTO b (a_y, a_x) VALUES ('y1', NULL) - -statement ok -INSERT INTO b (a_y, a_x) VALUES (NULL, 'x1') - -statement ok -INSERT INTO b (a_y, a_x) VALUES ('y2', NULL) - -statement ok -INSERT INTO b (a_y, a_x) VALUES (NULL, 'x2') - -statement ok -INSERT INTO b (a_x, a_y) VALUES ('x1', 'y1') +subtest Composite_Simple_Add_Constraint_Valid +# Test ADD CONSTRAINT validation by inserting valid rows before the constraint is added. -statement ok -INSERT INTO b (a_x, a_y) VALUES (NULL, NULL) - -statement ok -ALTER TABLE b VALIDATE CONSTRAINT fk_ref - -statement ok -DROP TABLE b, a - -# Test composite key with three columns. statement ok CREATE TABLE a ( x STRING NULL @@ -1576,52 +1559,13 @@ CREATE TABLE b ( a_y STRING NULL ,a_x STRING NULL ,a_z STRING NULL + ,INDEX idx (a_z, a_y, a_x) ); -# Add the constraint separately so that it's unvalidated, so we can test VALIDATE CONSTRAINT. -statement ok -ALTER TABLE b ADD CONSTRAINT fk_ref FOREIGN KEY (a_z, a_y, a_x) REFERENCES a (z, y, x) - statement ok INSERT INTO a (x, y, z) VALUES ('x1', 'y1', 'z1') # All of these are allowed because we do composite matching using MATCH SIMPLE. -statement ok -INSERT INTO b (a_x) VALUES ('x1') - -statement ok -INSERT INTO b (a_y) VALUES ('y1') - -statement ok -INSERT INTO b (a_z) VALUES ('z1') - -statement ok -INSERT INTO b (a_x, a_y) VALUES ('x1', 'y1') - -statement ok -INSERT INTO b (a_x, a_y) VALUES (NULL, 'y1') - -statement ok -INSERT INTO b (a_x, a_y) VALUES ('x1', NULL) - -statement ok -INSERT INTO b (a_x, a_z) VALUES ('x1', 'z1') - -statement ok -INSERT INTO b (a_x, a_z) VALUES (NULL, 'z1') - -statement ok -INSERT INTO b (a_x, a_z) VALUES ('x1', NULL) - -statement ok -INSERT INTO b (a_y, a_z) VALUES ('y1', 'z1') - -statement ok -INSERT INTO b (a_y, a_z) VALUES (NULL, 'z1') - -statement ok -INSERT INTO b (a_y, a_z) VALUES ('y1', NULL) - statement ok INSERT INTO b (a_x, a_y, a_z) VALUES ('x1', NULL, NULL) @@ -1640,6 +1584,9 @@ INSERT INTO b (a_x, a_y, a_z) VALUES ('x1', NULL, 'z1') statement ok INSERT INTO b (a_x, a_y, a_z) VALUES (NULL, 'y1', 'z1') +statement ok +INSERT INTO b (a_x, a_y, a_z) VALUES (NULL, NULL, NULL) + statement ok INSERT INTO b (a_x, a_y, a_z) VALUES ('x2', NULL, NULL) @@ -1659,16 +1606,13 @@ statement ok INSERT INTO b (a_x, a_y, a_z) VALUES (NULL, 'y2', 'z2') statement ok -INSERT INTO b (a_x, a_y, a_z) VALUES (NULL, NULL, NULL) - -statement ok -ALTER TABLE b VALIDATE CONSTRAINT fk_ref +ALTER TABLE b ADD CONSTRAINT fk_ref FOREIGN KEY (a_z, a_y, a_x) REFERENCES a (z, y, x) statement ok DROP TABLE b, a -subtest Composite_Simple_Validate_Constraint_Invalid -# Test VALIDATE CONSTRAINT by inserting invalid rows before the constraint is added, one at a time. +subtest Composite_Simple_Add_Constraint_Invalid +# Test ADD CONSTRAINT validation by inserting invalid rows before the constraint is added, one at a time. statement ok CREATE TABLE a ( @@ -1689,53 +1633,26 @@ CREATE TABLE b ( statement ok INSERT INTO b (a_x, a_y, a_z) VALUES ('x2', 'y1', 'z1') -statement ok -ALTER TABLE b ADD CONSTRAINT fk_ref FOREIGN KEY (a_z, a_y, a_x) REFERENCES a (z, y, x) - -# Verify that the optimizer doesn't use an unvalidated constraint to simplify plans. -query TTT -SELECT - s.a_z, s.a_y, s.a_x -FROM - (SELECT * FROM b WHERE a_z IS NOT NULL AND a_y IS NOT NULL AND a_x IS NOT NULL) AS s - LEFT JOIN a AS t ON s.a_z = t.z AND s.a_y = t.y AND s.a_x = t.x -WHERE - t.z IS NULL ----- -z1 y1 x2 - statement error foreign key violation: "b" row a_z='z1', a_y='y1', a_x='x2' has no match in "a" -ALTER TABLE b VALIDATE CONSTRAINT fk_ref +ALTER TABLE b ADD CONSTRAINT fk_ref FOREIGN KEY (a_z, a_y, a_x) REFERENCES a (z, y, x) statement ok TRUNCATE b -statement ok -ALTER TABLE b DROP CONSTRAINT fk_ref - statement ok INSERT INTO b (a_x, a_y, a_z) VALUES ('x2', 'y2', 'z1') -statement ok -ALTER TABLE b ADD CONSTRAINT fk_ref FOREIGN KEY (a_z, a_y, a_x) REFERENCES a (z, y, x) - statement error foreign key violation: "b" row a_z='z1', a_y='y2', a_x='x2' has no match in "a" -ALTER TABLE b VALIDATE CONSTRAINT fk_ref +ALTER TABLE b ADD CONSTRAINT fk_ref FOREIGN KEY (a_z, a_y, a_x) REFERENCES a (z, y, x) statement ok TRUNCATE b -statement ok -ALTER TABLE b DROP CONSTRAINT fk_ref - statement ok INSERT INTO b (a_x, a_y, a_z) VALUES ('x2', 'y2', 'z2') -statement ok -ALTER TABLE b ADD CONSTRAINT fk_ref FOREIGN KEY (a_z, a_y, a_x) REFERENCES a (z, y, x) - statement error foreign key violation: "b" row a_z='z2', a_y='y2', a_x='x2' has no match in "a" -ALTER TABLE b VALIDATE CONSTRAINT fk_ref +ALTER TABLE b ADD CONSTRAINT fk_ref FOREIGN KEY (a_z, a_y, a_x) REFERENCES a (z, y, x) statement ok DROP TABLE b, a @@ -1866,57 +1783,9 @@ INSERT INTO b (a_x, a_y, a_z) VALUES (NULL, NULL, NULL) statement ok DROP TABLE b, a -subtest Composite_Full_Validate_Later -# Test inserting into table with an unvalidated constraint, and running VALIDATE CONSTRAINT later - -# Test composite key with two columns. -statement ok -CREATE TABLE a ( - x STRING NULL - ,y STRING NULL - ,CONSTRAINT "primary" PRIMARY KEY (y, x) -); - -statement ok -CREATE TABLE b ( - a_y STRING NULL - ,a_x STRING NULL -); - -# Add the constraint separately so that it's unvalidated, so we can test VALIDATE CONSTRAINT. -statement ok -ALTER TABLE b ADD CONSTRAINT fk_ref FOREIGN KEY (a_y, a_x) REFERENCES a (y, x) MATCH FULL - -statement ok -INSERT INTO a (x, y) VALUES ('x1', 'y1') - -# These statements should all fail because this uses MATCH FULL. -statement error missing value for column "a_y" in multi-part foreign key -INSERT INTO b (a_x) VALUES ('x1') - -statement error missing value for column "a_x" in multi-part foreign key -INSERT INTO b (a_y) VALUES ('y1') - -statement error foreign key violation: MATCH FULL does not allow mixing of null and nonnull values -INSERT INTO b (a_y, a_x) VALUES ('y1', NULL) - -statement error foreign key violation: MATCH FULL does not allow mixing of null and nonnull values -INSERT INTO b (a_y, a_x) VALUES (NULL, 'x1') - -# These next two statements should still be allowed. -statement ok -INSERT INTO b (a_x, a_y) VALUES ('x1', 'y1') - -statement ok -INSERT INTO b (a_x, a_y) VALUES (NULL, NULL) +subtest Composite_Full_Add_Constraint_Valid +# Test ADD CONSTRAINT validation by inserting valid rows before the constraint is added. -statement ok -ALTER TABLE b VALIDATE CONSTRAINT fk_ref - -statement ok -DROP TABLE b, a - -# Test composite key with three columns. statement ok CREATE TABLE a ( x STRING NULL @@ -1930,76 +1799,18 @@ CREATE TABLE b ( a_y STRING NULL ,a_x STRING NULL ,a_z STRING NULL + ,INDEX idx (a_z, a_y, a_x) ); -# Add the constraint separately so that it's unvalidated, so we can test VALIDATE CONSTRAINT. -statement ok -ALTER TABLE b ADD CONSTRAINT fk_ref FOREIGN KEY (a_z, a_y, a_x) REFERENCES a (z, y, x) MATCH FULL - statement ok INSERT INTO a (x, y, z) VALUES ('x1', 'y1', 'z1') -# These statements should all fail because this uses MATCH FULL. -statement error missing values for columns \["a_y" "a_z"\] in multi-part foreign key -INSERT INTO b (a_x) VALUES ('x1') - -statement error missing values for columns \["a_x" "a_z"\] in multi-part foreign key -INSERT INTO b (a_y) VALUES ('y1') - -statement error missing values for columns \["a_x" "a_y"\] in multi-part foreign key -INSERT INTO b (a_z) VALUES ('z1') - -statement error missing value for column "a_z" in multi-part foreign key -INSERT INTO b (a_x, a_y) VALUES ('x1', 'y1') - -statement error missing value for column "a_z" in multi-part foreign key -INSERT INTO b (a_x, a_y) VALUES (NULL, 'y1') - -statement error missing value for column "a_z" in multi-part foreign key -INSERT INTO b (a_x, a_y) VALUES ('x1', NULL) - -statement error missing value for column "a_y" in multi-part foreign key -INSERT INTO b (a_x, a_z) VALUES ('x1', 'z1') - -statement error missing value for column "a_y" in multi-part foreign key -INSERT INTO b (a_x, a_z) VALUES (NULL, 'z1') - -statement error missing value for column "a_y" in multi-part foreign key -INSERT INTO b (a_x, a_z) VALUES ('x1', NULL) - -statement error missing value for column "a_x" in multi-part foreign key -INSERT INTO b (a_y, a_z) VALUES ('y1', 'z1') - -statement error missing value for column "a_x" in multi-part foreign key -INSERT INTO b (a_y, a_z) VALUES (NULL, 'z1') - -statement error missing value for column "a_x" in multi-part foreign key -INSERT INTO b (a_y, a_z) VALUES ('y1', NULL) - -statement error foreign key violation: MATCH FULL does not allow mixing of null and nonnull values -INSERT INTO b (a_x, a_y, a_z) VALUES ('x1', NULL, NULL) - -statement error foreign key violation: MATCH FULL does not allow mixing of null and nonnull values -INSERT INTO b (a_x, a_y, a_z) VALUES (NULL, 'y1', NULL) - -statement error foreign key violation: MATCH FULL does not allow mixing of null and nonnull values -INSERT INTO b (a_x, a_y, a_z) VALUES (NULL, NULL, 'z1') - -statement error foreign key violation: MATCH FULL does not allow mixing of null and nonnull values -INSERT INTO b (a_x, a_y, a_z) VALUES ('x1', 'y1', NULL) - -statement error foreign key violation: MATCH FULL does not allow mixing of null and nonnull values -INSERT INTO b (a_x, a_y, a_z) VALUES ('x1', NULL, 'z1') - -statement error foreign key violation: MATCH FULL does not allow mixing of null and nonnull values -INSERT INTO b (a_x, a_y, a_z) VALUES (NULL, 'y1', 'z1') - # This statement should still be allowed. statement ok INSERT INTO b (a_x, a_y, a_z) VALUES (NULL, NULL, NULL) statement ok -ALTER TABLE b VALIDATE CONSTRAINT fk_ref +ALTER TABLE b ADD CONSTRAINT fk_ref FOREIGN KEY (a_z, a_y, a_x) REFERENCES a (z, y, x) statement ok DROP TABLE b, a @@ -2026,131 +1837,80 @@ CREATE TABLE b ( statement ok INSERT INTO b (a_x, a_y, a_z) VALUES ('x1', NULL, NULL) -statement ok -ALTER TABLE b ADD CONSTRAINT fk_ref FOREIGN KEY (a_z, a_y, a_x) REFERENCES a (z, y, x) MATCH FULL - statement error foreign key violation: MATCH FULL does not allow mixing of null and nonnull values -ALTER TABLE b VALIDATE CONSTRAINT fk_ref +ALTER TABLE b ADD CONSTRAINT fk_ref FOREIGN KEY (a_z, a_y, a_x) REFERENCES a (z, y, x) MATCH FULL statement ok TRUNCATE b -statement ok -ALTER TABLE b DROP CONSTRAINT fk_ref - statement ok INSERT INTO b (a_x, a_y, a_z) VALUES (NULL, 'y1', NULL) -statement ok -ALTER TABLE b ADD CONSTRAINT fk_ref FOREIGN KEY (a_z, a_y, a_x) REFERENCES a (z, y, x) MATCH FULL - statement error foreign key violation: MATCH FULL does not allow mixing of null and nonnull values -ALTER TABLE b VALIDATE CONSTRAINT fk_ref +ALTER TABLE b ADD CONSTRAINT fk_ref FOREIGN KEY (a_z, a_y, a_x) REFERENCES a (z, y, x) MATCH FULL statement ok TRUNCATE b -statement ok -ALTER TABLE b DROP CONSTRAINT fk_ref - statement ok INSERT INTO b (a_x, a_y, a_z) VALUES (NULL, NULL, 'z1') -statement ok -ALTER TABLE b ADD CONSTRAINT fk_ref FOREIGN KEY (a_z, a_y, a_x) REFERENCES a (z, y, x) MATCH FULL - statement error foreign key violation: MATCH FULL does not allow mixing of null and nonnull values -ALTER TABLE b VALIDATE CONSTRAINT fk_ref +ALTER TABLE b ADD CONSTRAINT fk_ref FOREIGN KEY (a_z, a_y, a_x) REFERENCES a (z, y, x) MATCH FULL statement ok TRUNCATE b -statement ok -ALTER TABLE b DROP CONSTRAINT fk_ref - statement ok INSERT INTO b (a_x, a_y, a_z) VALUES ('x1', 'y1', NULL) -statement ok -ALTER TABLE b ADD CONSTRAINT fk_ref FOREIGN KEY (a_z, a_y, a_x) REFERENCES a (z, y, x) MATCH FULL - statement error foreign key violation: MATCH FULL does not allow mixing of null and nonnull values -ALTER TABLE b VALIDATE CONSTRAINT fk_ref +ALTER TABLE b ADD CONSTRAINT fk_ref FOREIGN KEY (a_z, a_y, a_x) REFERENCES a (z, y, x) MATCH FULL statement ok TRUNCATE b -statement ok -ALTER TABLE b DROP CONSTRAINT fk_ref - statement ok INSERT INTO b (a_x, a_y, a_z) VALUES ('x1', NULL, 'z1') -statement ok -ALTER TABLE b ADD CONSTRAINT fk_ref FOREIGN KEY (a_z, a_y, a_x) REFERENCES a (z, y, x) MATCH FULL - statement error foreign key violation: MATCH FULL does not allow mixing of null and nonnull values -ALTER TABLE b VALIDATE CONSTRAINT fk_ref +ALTER TABLE b ADD CONSTRAINT fk_ref FOREIGN KEY (a_z, a_y, a_x) REFERENCES a (z, y, x) MATCH FULL statement ok TRUNCATE b -statement ok -ALTER TABLE b DROP CONSTRAINT fk_ref - statement ok INSERT INTO b (a_x, a_y, a_z) VALUES (NULL, 'y1', 'z1') -statement ok -ALTER TABLE b ADD CONSTRAINT fk_ref FOREIGN KEY (a_z, a_y, a_x) REFERENCES a (z, y, x) MATCH FULL - statement error foreign key violation: MATCH FULL does not allow mixing of null and nonnull values -ALTER TABLE b VALIDATE CONSTRAINT fk_ref +ALTER TABLE b ADD CONSTRAINT fk_ref FOREIGN KEY (a_z, a_y, a_x) REFERENCES a (z, y, x) MATCH FULL statement ok TRUNCATE b -statement ok -ALTER TABLE b DROP CONSTRAINT fk_ref - statement ok INSERT INTO b (a_x, a_y, a_z) VALUES ('x2', 'y1', 'z1') -statement ok -ALTER TABLE b ADD CONSTRAINT fk_ref FOREIGN KEY (a_z, a_y, a_x) REFERENCES a (z, y, x) MATCH FULL - statement error foreign key violation: "b" row a_z='z1', a_y='y1', a_x='x2' has no match in "a" -ALTER TABLE b VALIDATE CONSTRAINT fk_ref +ALTER TABLE b ADD CONSTRAINT fk_ref FOREIGN KEY (a_z, a_y, a_x) REFERENCES a (z, y, x) MATCH FULL statement ok TRUNCATE b -statement ok -ALTER TABLE b DROP CONSTRAINT fk_ref - statement ok INSERT INTO b (a_x, a_y, a_z) VALUES ('x2', 'y2', 'z1') -statement ok -ALTER TABLE b ADD CONSTRAINT fk_ref FOREIGN KEY (a_z, a_y, a_x) REFERENCES a (z, y, x) MATCH FULL - statement error foreign key violation: "b" row a_z='z1', a_y='y2', a_x='x2' has no match in "a" -ALTER TABLE b VALIDATE CONSTRAINT fk_ref +ALTER TABLE b ADD CONSTRAINT fk_ref FOREIGN KEY (a_z, a_y, a_x) REFERENCES a (z, y, x) MATCH FULL statement ok TRUNCATE b -statement ok -ALTER TABLE b DROP CONSTRAINT fk_ref - statement ok INSERT INTO b (a_x, a_y, a_z) VALUES ('x2', 'y2', 'z2') -statement ok -ALTER TABLE b ADD CONSTRAINT fk_ref FOREIGN KEY (a_z, a_y, a_x) REFERENCES a (z, y, x) MATCH FULL - statement error foreign key violation: "b" row a_z='z2', a_y='y2', a_x='x2' has no match in "a" -ALTER TABLE b VALIDATE CONSTRAINT fk_ref +ALTER TABLE b ADD CONSTRAINT fk_ref FOREIGN KEY (a_z, a_y, a_x) REFERENCES a (z, y, x) MATCH FULL statement ok DROP TABLE b, a 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 89664f2e3a29..5a1d30db0b12 100644 --- a/pkg/sql/logictest/testdata/logic_test/schema_change_in_txn +++ b/pkg/sql/logictest/testdata/logic_test/schema_change_in_txn @@ -34,6 +34,24 @@ SELECT * FROM test.child@child_auto_index_fk_child_parent_id statement ok COMMIT +# Verify that the constraint is unvalidated, which is a limitation of adding the +# constraint in the same transaction as CREATE TABLE. +# TODO (lucy): Add a job to validate the table in this situation. +query TTTTB +SHOW CONSTRAINTS FROM test.child +---- +child fk_child_parent_id FOREIGN KEY FOREIGN KEY (parent_id) REFERENCES parent (id) false +child primary PRIMARY KEY PRIMARY KEY (id ASC) true + +statement ok +ALTER TABLE test.child VALIDATE CONSTRAINT fk_child_parent_id + +query TTTTB +SHOW CONSTRAINTS FROM test.child +---- +child fk_child_parent_id FOREIGN KEY FOREIGN KEY (parent_id) REFERENCES parent (id) true +child primary PRIMARY KEY PRIMARY KEY (id ASC) true + statement ok DROP TABLE test.child, test.parent @@ -67,6 +85,13 @@ SELECT * FROM test.child@child_auto_index_fk_child_parent_id ---- 1 1 +# Verify that the constraint is validated. +query TTTTB +SHOW CONSTRAINTS FROM test.child +---- +child fk_child_parent_id FOREIGN KEY FOREIGN KEY (parent_id) REFERENCES parent (id) true +child primary PRIMARY KEY PRIMARY KEY (id ASC) true + statement ok DROP TABLE test.child, test.parent @@ -761,6 +786,12 @@ succeeded ALTER TABLE test.public.customers ADD COLUMN i INT8 DEFAULT 5;ALTER T # a schema change in the same txn #32118 subtest validate_in_schema_change_txn +# To get an unvalidated foreign key for testing, use the loophole that we +# currently don't support adding a validated FK in the same transaction as +# CREATE TABLE +statement ok +BEGIN + statement ok CREATE TABLE products (sku STRING PRIMARY KEY, upc STRING UNIQUE, vendor STRING) @@ -771,18 +802,92 @@ CREATE TABLE orders2 ( INDEX (product) ) +statement ok +ALTER TABLE orders2 ADD FOREIGN KEY (product) REFERENCES products + +statement ok +COMMIT + statement ok BEGIN +# Perform an unrelated schema change statement ok -ALTER TABLE orders2 ADD FOREIGN KEY (product) REFERENCES products +ALTER TABLE orders2 ADD CHECK (id > 0) + +statement ok +ALTER TABLE orders2 VALIDATE CONSTRAINT fk_product_ref_products + +statement ok +COMMIT + +statement ok +DROP TABLE products, orders2 + +subtest fk_constraint_being_added + +statement ok +CREATE TABLE products (sku STRING PRIMARY KEY, upc STRING UNIQUE, vendor STRING) + +statement ok +CREATE TABLE orders2 ( + id INT8 PRIMARY KEY, + product STRING DEFAULT 'sprockets', + INDEX (product) +) + +# The constraint can't be validated with VALIDATE CONSTRAINT in the same transaction +statement ok +BEGIN statement ok +ALTER TABLE orders2 ADD FOREIGN KEY (product) REFERENCES products + +statement error constraint "fk_product_ref_products" in the middle of being added, try again later ALTER TABLE orders2 VALIDATE CONSTRAINT fk_product_ref_products statement ok COMMIT +# Dependent columns can't be dropped +statement ok +BEGIN + +statement ok +ALTER TABLE orders2 ADD FOREIGN KEY (product) REFERENCES products + +statement error constraint "fk_product_ref_products" in the middle of being added, try again later +ALTER TABLE orders2 DROP COLUMN product + +statement ok +COMMIT + +# Dependent indexes can't be dropped +statement ok +BEGIN + +statement ok +ALTER TABLE orders2 ADD FOREIGN KEY (product) REFERENCES products + +statement error constraint "fk_product_ref_products" in the middle of being added, try again later +DROP INDEX orders2@orders2_product_idx + +statement ok +COMMIT + +# The constraint can't be renamed +statement ok +BEGIN + +statement ok +ALTER TABLE orders2 ADD CONSTRAINT c FOREIGN KEY (product) REFERENCES products + +statement error constraint "c" in the middle of being added, try again later +ALTER TABLE orders2 RENAME CONSTRAINT c to d + +statement ok +COMMIT + # Verify that check constraints can be added on columns being added in the same transaction subtest check_on_add_col diff --git a/pkg/sql/opt/exec/execbuilder/testdata/fk b/pkg/sql/opt/exec/execbuilder/testdata/fk index bb07ea4812e1..79b0e23ff67f 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/fk +++ b/pkg/sql/opt/exec/execbuilder/testdata/fk @@ -1,5 +1,13 @@ # LogicTest: local-opt +# To get an unvalidated foreign key for testing, use the loophole that we +# currently don't support adding a validated FK in the same transaction as +# CREATE TABLE +# TODO (lucy): Once this is no longer true (and we support adding NOT VALID +# constraints), update this test +statement ok +BEGIN + statement ok CREATE TABLE a ( x STRING NULL, @@ -19,6 +27,9 @@ CREATE TABLE b ( statement ok ALTER TABLE b ADD CONSTRAINT fk_ref FOREIGN KEY (a_z, a_y, a_x) REFERENCES a (z, y, x) +statement ok +COMMIT + # Verify that the optimizer doesn't use an unvalidated constraint to simplify plans. query TTT colnames EXPLAIN SELECT diff --git a/pkg/sql/planner.go b/pkg/sql/planner.go index 3ad6fb35b7f0..d80b64a19872 100644 --- a/pkg/sql/planner.go +++ b/pkg/sql/planner.go @@ -26,7 +26,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/querycache" "github.com/cockroachdb/cockroach/pkg/sql/row" - "github.com/cockroachdb/cockroach/pkg/sql/rowcontainer" "github.com/cockroachdb/cockroach/pkg/sql/sem/transform" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" @@ -34,7 +33,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/envutil" "github.com/cockroachdb/cockroach/pkg/util/hlc" - "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/log/logtags" "github.com/cockroachdb/cockroach/pkg/util/mon" "github.com/pkg/errors" @@ -547,60 +545,6 @@ func (p *planner) prepareForDistSQLSupportCheck() { p.setUnlimited(p.curPlan.plan) } -// runWithDistSQL runs a planNode tree synchronously via DistSQL, returning the -// results in a RowContainer. There's no streaming on this, so use sparingly. -// In general, you should always prefer to use the internal executor if you can. -func (p *planner) runWithDistSQL( - ctx context.Context, plan planNode, -) (*rowcontainer.RowContainer, error) { - params := runParams{ - ctx: ctx, - extendedEvalCtx: &p.extendedEvalCtx, - p: p, - } - // Create the DistSQL plan for the input. - planCtx := params.extendedEvalCtx.DistSQLPlanner.NewPlanningCtx(ctx, params.extendedEvalCtx, params.p.txn) - log.VEvent(ctx, 1, "creating DistSQL plan") - physPlan, err := planCtx.ExtendedEvalCtx.DistSQLPlanner.createPlanForNode(planCtx, plan) - if err != nil { - return nil, err - } - planCtx.ExtendedEvalCtx.DistSQLPlanner.FinalizePlan(planCtx, &physPlan) - columns := planColumns(plan) - - // Initialize a row container for the DistSQL execution engine to write into. - // The caller of this method will call Close on the returned RowContainer, - // which will close this account. - acc := planCtx.EvalContext().Mon.MakeBoundAccount() - ci := sqlbase.ColTypeInfoFromResCols(columns) - rows := rowcontainer.NewRowContainer(acc, ci, 0 /* rowCapacity */) - rowResultWriter := NewRowResultWriter(rows) - recv := MakeDistSQLReceiver( - ctx, - rowResultWriter, - tree.Rows, - p.ExecCfg().RangeDescriptorCache, - p.ExecCfg().LeaseHolderCache, - p.txn, - func(ts hlc.Timestamp) { - _ = p.ExecCfg().Clock.Update(ts) - }, - p.extendedEvalCtx.Tracing, - ) - defer recv.Release() - - // Copy the evalCtx, as dsp.Run() might change it. - evalCtxCopy := p.extendedEvalCtx - // Run the plan, writing to the row container we initialized earlier. - p.extendedEvalCtx.DistSQLPlanner.Run( - planCtx, p.txn, &physPlan, recv, &evalCtxCopy, nil /* finishedSetupFn */) - if rowResultWriter.Err() != nil { - rows.Close(ctx) - return nil, rowResultWriter.Err() - } - return rows, nil -} - // txnModesSetter is an interface used by SQL execution to influence the current // transaction. type txnModesSetter interface { diff --git a/pkg/sql/schema_changer.go b/pkg/sql/schema_changer.go index 7e2dfbd94557..5b0fa07a7b07 100644 --- a/pkg/sql/schema_changer.go +++ b/pkg/sql/schema_changer.go @@ -569,11 +569,13 @@ func (sc *SchemaChanger) maybeMakeAddTablePublic( ctx context.Context, table *sqlbase.TableDescriptor, ) error { if table.Adding() { - for _, idx := range table.AllNonDropIndexes() { - if idx.ForeignKey.IsSet() { - if err := sc.waitToUpdateLeases(ctx, idx.ForeignKey.Table); err != nil { - return err - } + fks, err := table.AllActiveAndInactiveForeignKeys() + if err != nil { + return err + } + for _, fk := range fks { + if err := sc.waitToUpdateLeases(ctx, fk.Table); err != nil { + return err } } @@ -1223,11 +1225,13 @@ func (sc *SchemaChanger) reverseMutations(ctx context.Context, causingError erro // where the indexes refer columns. Whenever a column schema change // is reversed, any index mutation referencing it is also reversed. var droppedMutations map[sqlbase.MutationID]struct{} + var backrefs map[sqlbase.ID][]*sqlbase.ConstraintToUpdate _, err := sc.leaseMgr.Publish(ctx, sc.tableID, func(desc *sqlbase.MutableTableDescriptor) error { // Keep track of the column mutations being reversed so that indexes // referencing them can be dropped. columns := make(map[string]struct{}) droppedMutations = nil + backrefs = make(map[sqlbase.ID][]*sqlbase.ConstraintToUpdate) for i, mutation := range desc.Mutations { if mutation.MutationID != sc.mutationID { @@ -1256,6 +1260,17 @@ func (sc *SchemaChanger) reverseMutations(ctx context.Context, causingError erro if err := sc.maybeDropValidatingConstraint(ctx, desc, constraint); err != nil { return err } + // Get the foreign key backreferences to remove, and remove them immediately if they're on the same table + if constraint.ConstraintType == sqlbase.ConstraintToUpdate_FOREIGN_KEY { + fk := &constraint.ForeignKey + if fk.Table == desc.ID { + if err := removeFKBackReferenceFromTable(desc, fk.Index, desc.ID, constraint.ForeignKeyIndex); err != nil { + return err + } + } else { + backrefs[constraint.ForeignKey.Table] = append(backrefs[constraint.ForeignKey.Table], constraint) + } + } } desc.Mutations[i].Rollback = true @@ -1314,6 +1329,23 @@ func (sc *SchemaChanger) reverseMutations(ctx context.Context, causingError erro if err != nil { return err } + + // Drop foreign key backreferences on other tables + for tbl, refs := range backrefs { + _, err = sc.leaseMgr.Publish(ctx, tbl, func(desc *sqlbase.MutableTableDescriptor) error { + for _, ref := range refs { + if err := removeFKBackReferenceFromTable(desc, ref.ForeignKey.Index, sc.tableID, ref.ForeignKeyIndex); err != nil { + return err + } + } + return nil + }, nil, + ) + if err != nil { + return err + } + } + // Only update the job if the transaction has succeeded. The schame change // job will now references the rollback job. if scJob != nil { @@ -1424,6 +1456,22 @@ func (sc *SchemaChanger) maybeDropValidatingConstraint( constraint.Name, ) } + case sqlbase.ConstraintToUpdate_FOREIGN_KEY: + idx, err := desc.FindIndexByID(constraint.ForeignKeyIndex) + if err != nil { + return err + } + if idx.ForeignKey.IsSet() { + idx.ForeignKey = sqlbase.ForeignKeyReference{} + } else { + if log.V(2) { + log.Infof( + ctx, + "attempted to drop constraint %s, but it hadn't been added to the table descriptor yet", + constraint.Name, + ) + } + } default: return pgerror.AssertionFailedf("unsupported constraint type: %d", log.Safe(constraint.ConstraintType)) } diff --git a/pkg/sql/show_create.go b/pkg/sql/show_create.go index 6739c37eac45..d476dac14697 100644 --- a/pkg/sql/show_create.go +++ b/pkg/sql/show_create.go @@ -79,7 +79,6 @@ func printForeignKeyConstraint( refNames = []string{"???"} fkTableName = tree.MakeTableName(tree.Name(""), tree.Name(fmt.Sprintf("[%d as ref]", fk.Table))) fkTableName.ExplicitSchema = false - fkTableName.ExplicitSchema = false } buf.WriteString("FOREIGN KEY (") formatQuoteNames(buf, idx.ColumnNames[0:idx.ForeignKey.SharedPrefixLen]...) @@ -172,6 +171,7 @@ func ShowCreateTable( allIdx := append(desc.Indexes, desc.PrimaryIndex) for i := range allIdx { idx := &allIdx[i] + // TODO (lucy): Possibly include FKs being validated here if fk := &idx.ForeignKey; fk.IsSet() && !ignoreFKs { f.WriteString(",\n\tCONSTRAINT ") f.FormatNameP(&fk.Name) diff --git a/pkg/sql/sqlbase/structured.go b/pkg/sql/sqlbase/structured.go index 81f28e13aec0..79373192b900 100644 --- a/pkg/sql/sqlbase/structured.go +++ b/pkg/sql/sqlbase/structured.go @@ -601,13 +601,48 @@ func (desc *TableDescriptor) AllActiveAndInactiveChecks() []*TableDescriptor_Che } } for _, m := range desc.Mutations { - if c := m.GetConstraint(); c != nil { + if c := m.GetConstraint(); c != nil && c.ConstraintType == ConstraintToUpdate_CHECK { checks = append(checks, &c.Check) } } return checks } +// AllActiveAndInactiveForeignKeys returns all foreign keys, including both +// "active" ones on the index descriptor which are being enforced for all +// writes, and "inactive" ones queued in the mutations list. An error is +// returned if multiple foreign keys (including mutations) are found for the +// same index. +func (desc *TableDescriptor) AllActiveAndInactiveForeignKeys() ( + map[IndexID]*ForeignKeyReference, + error, +) { + fks := make(map[IndexID]*ForeignKeyReference) + // While a foreign key constraint is being validated for existing rows, the + // foreign key reference is present both on the index descriptor and in the + // mutations list in the Validating state, so those FKs are excluded here to + // avoid double-counting. + if desc.PrimaryIndex.ForeignKey.IsSet() && desc.PrimaryIndex.ForeignKey.Validity != ConstraintValidity_Validating { + fks[desc.PrimaryIndex.ID] = &desc.PrimaryIndex.ForeignKey + } + for i := range desc.Indexes { + idx := &desc.Indexes[i] + if idx.ForeignKey.IsSet() && idx.ForeignKey.Validity != ConstraintValidity_Validating { + fks[idx.ID] = &idx.ForeignKey + } + } + for i := range desc.Mutations { + if c := desc.Mutations[i].GetConstraint(); c != nil && c.ConstraintType == ConstraintToUpdate_FOREIGN_KEY { + if _, ok := fks[c.ForeignKeyIndex]; ok { + return nil, pgerror.AssertionFailedf( + "foreign key mutation found for index that already has a foreign key") + } + fks[c.ForeignKeyIndex] = &c.ForeignKey + } + } + return fks, nil +} + // ForeachNonDropIndex runs a function on all indexes, including those being // added in the mutations. func (desc *TableDescriptor) ForeachNonDropIndex(f func(*IndexDescriptor) error) error { @@ -2197,6 +2232,11 @@ func (desc *MutableTableDescriptor) RenameConstraint( return desc.RenameIndexDescriptor(detail.Index, newName) case ConstraintTypeFK: + if detail.FK.Validity == ConstraintValidity_Validating { + return pgerror.Unimplementedf("rename-constraint-fk-mutation", + "constraint %q in the middle of being added, try again later", + tree.ErrNameStringP(&detail.FK.Name)) + } idx, err := desc.FindIndexByID(detail.Index.ID) if err != nil { return err @@ -2317,6 +2357,12 @@ func (desc *MutableTableDescriptor) MakeMutationComplete(m DescriptorMutation) e break } } + case ConstraintToUpdate_FOREIGN_KEY: + idx, err := desc.FindIndexByID(t.Constraint.ForeignKeyIndex) + if err != nil { + return err + } + idx.ForeignKey.Validity = ConstraintValidity_Validated default: return errors.Errorf("unsupported constraint type: %d", t.Constraint.ConstraintType) } @@ -2348,6 +2394,24 @@ func (desc *MutableTableDescriptor) AddCheckValidationMutation( desc.addMutation(m) } +// AddForeignKeyValidationMutation adds a foreign key constraint validation mutation to desc.Mutations. +func (desc *MutableTableDescriptor) AddForeignKeyValidationMutation( + fk *ForeignKeyReference, idx IndexID, +) { + m := DescriptorMutation{ + Descriptor_: &DescriptorMutation_Constraint{ + Constraint: &ConstraintToUpdate{ + ConstraintType: ConstraintToUpdate_FOREIGN_KEY, + Name: fk.Name, + ForeignKey: *fk, + ForeignKeyIndex: idx, + }, + }, + Direction: DescriptorMutation_ADD, + } + desc.addMutation(m) +} + // AddColumnMutation adds a column mutation to desc.Mutations. Callers must take // care not to further mutate the column descriptor, since this method retains // a pointer to it. diff --git a/pkg/sql/sqlbase/structured.pb.go b/pkg/sql/sqlbase/structured.pb.go index a5d9dde421db..93e270a78bbf 100644 --- a/pkg/sql/sqlbase/structured.pb.go +++ b/pkg/sql/sqlbase/structured.pb.go @@ -67,7 +67,7 @@ func (x *ConstraintValidity) UnmarshalJSON(data []byte) error { return nil } func (ConstraintValidity) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_81f3763b694c15f3, []int{0} + return fileDescriptor_structured_a76e1756e60399f7, []int{0} } type ForeignKeyReference_Action int32 @@ -112,7 +112,7 @@ func (x *ForeignKeyReference_Action) UnmarshalJSON(data []byte) error { return nil } func (ForeignKeyReference_Action) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_81f3763b694c15f3, []int{0, 0} + return fileDescriptor_structured_a76e1756e60399f7, []int{0, 0} } // Match is the algorithm used to compare composite keys. @@ -152,7 +152,7 @@ func (x *ForeignKeyReference_Match) UnmarshalJSON(data []byte) error { return nil } func (ForeignKeyReference_Match) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_81f3763b694c15f3, []int{0, 1} + return fileDescriptor_structured_a76e1756e60399f7, []int{0, 1} } // The direction of a column in the index. @@ -189,7 +189,7 @@ func (x *IndexDescriptor_Direction) UnmarshalJSON(data []byte) error { return nil } func (IndexDescriptor_Direction) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_81f3763b694c15f3, []int{5, 0} + return fileDescriptor_structured_a76e1756e60399f7, []int{5, 0} } // The type of the index. @@ -226,20 +226,23 @@ func (x *IndexDescriptor_Type) UnmarshalJSON(data []byte) error { return nil } func (IndexDescriptor_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_81f3763b694c15f3, []int{5, 1} + return fileDescriptor_structured_a76e1756e60399f7, []int{5, 1} } type ConstraintToUpdate_ConstraintType int32 const ( - ConstraintToUpdate_CHECK ConstraintToUpdate_ConstraintType = 0 + ConstraintToUpdate_CHECK ConstraintToUpdate_ConstraintType = 0 + ConstraintToUpdate_FOREIGN_KEY ConstraintToUpdate_ConstraintType = 1 ) var ConstraintToUpdate_ConstraintType_name = map[int32]string{ 0: "CHECK", + 1: "FOREIGN_KEY", } var ConstraintToUpdate_ConstraintType_value = map[string]int32{ - "CHECK": 0, + "CHECK": 0, + "FOREIGN_KEY": 1, } func (x ConstraintToUpdate_ConstraintType) Enum() *ConstraintToUpdate_ConstraintType { @@ -259,7 +262,7 @@ func (x *ConstraintToUpdate_ConstraintType) UnmarshalJSON(data []byte) error { return nil } func (ConstraintToUpdate_ConstraintType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_81f3763b694c15f3, []int{6, 0} + return fileDescriptor_structured_a76e1756e60399f7, []int{6, 0} } // A descriptor within a mutation is unavailable for reads, writes @@ -324,7 +327,7 @@ func (x *DescriptorMutation_State) UnmarshalJSON(data []byte) error { return nil } func (DescriptorMutation_State) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_81f3763b694c15f3, []int{7, 0} + return fileDescriptor_structured_a76e1756e60399f7, []int{7, 0} } // Direction of mutation. @@ -367,7 +370,7 @@ func (x *DescriptorMutation_Direction) UnmarshalJSON(data []byte) error { return nil } func (DescriptorMutation_Direction) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_81f3763b694c15f3, []int{7, 1} + return fileDescriptor_structured_a76e1756e60399f7, []int{7, 1} } // State is set if this TableDescriptor is in the process of being added or deleted. @@ -418,7 +421,7 @@ func (x *TableDescriptor_State) UnmarshalJSON(data []byte) error { return nil } func (TableDescriptor_State) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_81f3763b694c15f3, []int{8, 0} + return fileDescriptor_structured_a76e1756e60399f7, []int{8, 0} } // AuditMode indicates which auditing actions to take when this table is used. @@ -455,7 +458,7 @@ func (x *TableDescriptor_AuditMode) UnmarshalJSON(data []byte) error { return nil } func (TableDescriptor_AuditMode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_81f3763b694c15f3, []int{8, 1} + return fileDescriptor_structured_a76e1756e60399f7, []int{8, 1} } type ForeignKeyReference struct { @@ -479,7 +482,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_81f3763b694c15f3, []int{0} + return fileDescriptor_structured_a76e1756e60399f7, []int{0} } func (m *ForeignKeyReference) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -526,7 +529,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_81f3763b694c15f3, []int{1} + return fileDescriptor_structured_a76e1756e60399f7, []int{1} } func (m *ColumnDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -579,7 +582,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_81f3763b694c15f3, []int{2} + return fileDescriptor_structured_a76e1756e60399f7, []int{2} } func (m *ColumnFamilyDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -627,7 +630,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_81f3763b694c15f3, []int{3} + return fileDescriptor_structured_a76e1756e60399f7, []int{3} } func (m *InterleaveDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -673,7 +676,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_81f3763b694c15f3, []int{3, 0} + return fileDescriptor_structured_a76e1756e60399f7, []int{3, 0} } func (m *InterleaveDescriptor_Ancestor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -720,7 +723,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_81f3763b694c15f3, []int{4} + return fileDescriptor_structured_a76e1756e60399f7, []int{4} } func (m *PartitioningDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -765,7 +768,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_81f3763b694c15f3, []int{4, 0} + return fileDescriptor_structured_a76e1756e60399f7, []int{4, 0} } func (m *PartitioningDescriptor_List) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -812,7 +815,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_81f3763b694c15f3, []int{4, 1} + return fileDescriptor_structured_a76e1756e60399f7, []int{4, 1} } func (m *PartitioningDescriptor_Range) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -946,7 +949,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_81f3763b694c15f3, []int{5} + return fileDescriptor_structured_a76e1756e60399f7, []int{5} } func (m *IndexDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -988,6 +991,8 @@ type ConstraintToUpdate struct { ConstraintType ConstraintToUpdate_ConstraintType `protobuf:"varint,1,req,name=constraint_type,json=constraintType,enum=cockroach.sql.sqlbase.ConstraintToUpdate_ConstraintType" json:"constraint_type"` Name string `protobuf:"bytes,2,req,name=name" json:"name"` Check TableDescriptor_CheckConstraint `protobuf:"bytes,3,opt,name=check" json:"check"` + ForeignKey ForeignKeyReference `protobuf:"bytes,4,opt,name=foreign_key,json=foreignKey" json:"foreign_key"` + ForeignKeyIndex IndexID `protobuf:"varint,5,opt,name=foreign_key_index,json=foreignKeyIndex,casttype=IndexID" json:"foreign_key_index"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_sizecache int32 `json:"-"` } @@ -996,7 +1001,7 @@ func (m *ConstraintToUpdate) Reset() { *m = ConstraintToUpdate{} } func (m *ConstraintToUpdate) String() string { return proto.CompactTextString(m) } func (*ConstraintToUpdate) ProtoMessage() {} func (*ConstraintToUpdate) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_81f3763b694c15f3, []int{6} + return fileDescriptor_structured_a76e1756e60399f7, []int{6} } func (m *ConstraintToUpdate) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1052,7 +1057,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_81f3763b694c15f3, []int{7} + return fileDescriptor_structured_a76e1756e60399f7, []int{7} } func (m *DescriptorMutation) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1336,7 +1341,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_81f3763b694c15f3, []int{8} + return fileDescriptor_structured_a76e1756e60399f7, []int{8} } func (m *TableDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1588,7 +1593,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_81f3763b694c15f3, []int{8, 0} + return fileDescriptor_structured_a76e1756e60399f7, []int{8, 0} } func (m *TableDescriptor_SchemaChangeLease) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1627,7 +1632,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_81f3763b694c15f3, []int{8, 1} + return fileDescriptor_structured_a76e1756e60399f7, []int{8, 1} } func (m *TableDescriptor_CheckConstraint) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1732,7 +1737,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_81f3763b694c15f3, []int{8, 2} + return fileDescriptor_structured_a76e1756e60399f7, []int{8, 2} } func (m *TableDescriptor_NameInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1774,7 +1779,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_81f3763b694c15f3, []int{8, 3} + return fileDescriptor_structured_a76e1756e60399f7, []int{8, 3} } func (m *TableDescriptor_Reference) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1813,7 +1818,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_81f3763b694c15f3, []int{8, 4} + return fileDescriptor_structured_a76e1756e60399f7, []int{8, 4} } func (m *TableDescriptor_MutationJob) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1857,7 +1862,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_81f3763b694c15f3, []int{8, 5} + return fileDescriptor_structured_a76e1756e60399f7, []int{8, 5} } func (m *TableDescriptor_SequenceOpts) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1893,7 +1898,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_81f3763b694c15f3, []int{8, 6} + return fileDescriptor_structured_a76e1756e60399f7, []int{8, 6} } func (m *TableDescriptor_Replacement) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1932,7 +1937,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_81f3763b694c15f3, []int{8, 7} + return fileDescriptor_structured_a76e1756e60399f7, []int{8, 7} } func (m *TableDescriptor_GCDescriptorMutation) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1973,7 +1978,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_81f3763b694c15f3, []int{9} + return fileDescriptor_structured_a76e1756e60399f7, []int{9} } func (m *DatabaseDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2033,7 +2038,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_81f3763b694c15f3, []int{10} + return fileDescriptor_structured_a76e1756e60399f7, []int{10} } func (m *Descriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2719,6 +2724,17 @@ func (m *ConstraintToUpdate) MarshalTo(dAtA []byte) (int, error) { return 0, err } i += n6 + dAtA[i] = 0x22 + i++ + i = encodeVarintStructured(dAtA, i, uint64(m.ForeignKey.Size())) + n7, err := m.ForeignKey.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n7 + dAtA[i] = 0x28 + i++ + i = encodeVarintStructured(dAtA, i, uint64(m.ForeignKeyIndex)) return i, nil } @@ -2738,11 +2754,11 @@ func (m *DescriptorMutation) MarshalTo(dAtA []byte) (int, error) { var l int _ = l if m.Descriptor_ != nil { - nn7, err := m.Descriptor_.MarshalTo(dAtA[i:]) + nn8, err := m.Descriptor_.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += nn7 + i += nn8 } dAtA[i] = 0x18 i++ @@ -2770,11 +2786,11 @@ func (m *DescriptorMutation_Column) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintStructured(dAtA, i, uint64(m.Column.Size())) - n8, err := m.Column.MarshalTo(dAtA[i:]) + n9, err := m.Column.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n8 + i += n9 } return i, nil } @@ -2784,11 +2800,11 @@ func (m *DescriptorMutation_Index) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintStructured(dAtA, i, uint64(m.Index.Size())) - n9, err := m.Index.MarshalTo(dAtA[i:]) + n10, err := m.Index.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n9 + i += n10 } return i, nil } @@ -2798,11 +2814,11 @@ func (m *DescriptorMutation_Constraint) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x42 i++ i = encodeVarintStructured(dAtA, i, uint64(m.Constraint.Size())) - n10, err := m.Constraint.MarshalTo(dAtA[i:]) + n11, err := m.Constraint.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n10 + i += n11 } return i, nil } @@ -2837,11 +2853,11 @@ func (m *TableDescriptor) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x3a i++ i = encodeVarintStructured(dAtA, i, uint64(m.ModificationTime.Size())) - n11, err := m.ModificationTime.MarshalTo(dAtA[i:]) + n12, err := m.ModificationTime.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n11 + i += n12 if len(m.Columns) > 0 { for _, msg := range m.Columns { dAtA[i] = 0x42 @@ -2860,11 +2876,11 @@ func (m *TableDescriptor) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x52 i++ i = encodeVarintStructured(dAtA, i, uint64(m.PrimaryIndex.Size())) - n12, err := m.PrimaryIndex.MarshalTo(dAtA[i:]) + n13, err := m.PrimaryIndex.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n12 + i += n13 if len(m.Indexes) > 0 { for _, msg := range m.Indexes { dAtA[i] = 0x5a @@ -2884,11 +2900,11 @@ func (m *TableDescriptor) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x6a i++ i = encodeVarintStructured(dAtA, i, uint64(m.Privileges.Size())) - n13, err := m.Privileges.MarshalTo(dAtA[i:]) + n14, err := m.Privileges.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n13 + i += n14 } if len(m.Mutations) > 0 { for _, msg := range m.Mutations { @@ -2906,11 +2922,11 @@ func (m *TableDescriptor) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x7a i++ i = encodeVarintStructured(dAtA, i, uint64(m.Lease.Size())) - n14, err := m.Lease.MarshalTo(dAtA[i:]) + n15, err := m.Lease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n14 + i += n15 } dAtA[i] = 0x80 i++ @@ -3023,11 +3039,11 @@ func (m *TableDescriptor) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintStructured(dAtA, i, uint64(m.SequenceOpts.Size())) - n15, err := m.SequenceOpts.MarshalTo(dAtA[i:]) + n16, err := m.SequenceOpts.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n15 + i += n16 } dAtA[i] = 0xe8 i++ @@ -3039,11 +3055,11 @@ func (m *TableDescriptor) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintStructured(dAtA, i, uint64(m.ReplacementOf.Size())) - n16, err := m.ReplacementOf.MarshalTo(dAtA[i:]) + n17, err := m.ReplacementOf.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n16 + i += n17 dAtA[i] = 0xf8 i++ dAtA[i] = 0x1 @@ -3270,11 +3286,11 @@ func (m *TableDescriptor_Replacement) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintStructured(dAtA, i, uint64(m.Time.Size())) - n17, err := m.Time.MarshalTo(dAtA[i:]) + n18, err := m.Time.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n17 + i += n18 return i, nil } @@ -3331,11 +3347,11 @@ func (m *DatabaseDescriptor) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1a i++ i = encodeVarintStructured(dAtA, i, uint64(m.Privileges.Size())) - n18, err := m.Privileges.MarshalTo(dAtA[i:]) + n19, err := m.Privileges.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n18 + i += n19 } return i, nil } @@ -3356,11 +3372,11 @@ func (m *Descriptor) MarshalTo(dAtA []byte) (int, error) { var l int _ = l if m.Union != nil { - nn19, err := m.Union.MarshalTo(dAtA[i:]) + nn20, err := m.Union.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += nn19 + i += nn20 } return i, nil } @@ -3371,11 +3387,11 @@ func (m *Descriptor_Table) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintStructured(dAtA, i, uint64(m.Table.Size())) - n20, err := m.Table.MarshalTo(dAtA[i:]) + n21, err := m.Table.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n20 + i += n21 } return i, nil } @@ -3385,11 +3401,11 @@ func (m *Descriptor_Database) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintStructured(dAtA, i, uint64(m.Database.Size())) - n21, err := m.Database.MarshalTo(dAtA[i:]) + n22, err := m.Database.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n21 + i += n22 } return i, nil } @@ -3640,6 +3656,9 @@ func (m *ConstraintToUpdate) Size() (n int) { n += 1 + l + sovStructured(uint64(l)) l = m.Check.Size() n += 1 + l + sovStructured(uint64(l)) + l = m.ForeignKey.Size() + n += 1 + l + sovStructured(uint64(l)) + n += 1 + sovStructured(uint64(m.ForeignKeyIndex)) return n } @@ -6115,6 +6134,55 @@ func (m *ConstraintToUpdate) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ForeignKey", 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 + } + if err := m.ForeignKey.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ForeignKeyIndex", wireType) + } + m.ForeignKeyIndex = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowStructured + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ForeignKeyIndex |= (IndexID(b) & 0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skipStructured(dAtA[iNdEx:]) @@ -8580,187 +8648,189 @@ var ( ) func init() { - proto.RegisterFile("sql/sqlbase/structured.proto", fileDescriptor_structured_81f3763b694c15f3) -} - -var fileDescriptor_structured_81f3763b694c15f3 = []byte{ - // 2838 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x59, 0x4b, 0x73, 0xdb, 0xc8, - 0xf1, 0x17, 0xf8, 0x04, 0x9b, 0x2f, 0x68, 0xfc, 0x58, 0x98, 0xf6, 0x4a, 0x34, 0x77, 0xbd, 0x7f, - 0xfd, 0xf7, 0x41, 0x79, 0xe5, 0x6c, 0xe2, 0x4a, 0x52, 0x5b, 0xe1, 0x4b, 0x16, 0x65, 0x8a, 0x94, - 0x21, 0xca, 0xda, 0x4d, 0x6d, 0xc2, 0x82, 0x80, 0x91, 0x84, 0x35, 0x08, 0xd0, 0x00, 0xa8, 0x48, - 0xdf, 0x60, 0x8f, 0x39, 0x25, 0xa7, 0x6c, 0xa5, 0xb6, 0x72, 0x48, 0x55, 0xae, 0x39, 0xe4, 0x23, - 0xec, 0x31, 0x95, 0x53, 0x2a, 0x07, 0x55, 0xa2, 0x54, 0xae, 0xc9, 0xdd, 0x95, 0x54, 0xa5, 0x66, - 0x30, 0x03, 0x02, 0x12, 0xa9, 0x50, 0xf6, 0x8d, 0xe8, 0x99, 0xfe, 0xcd, 0x4c, 0x4f, 0xf7, 0xaf, - 0xbb, 0x87, 0x70, 0xcf, 0x7d, 0x69, 0xae, 0xba, 0x2f, 0xcd, 0x7d, 0xd5, 0xc5, 0xab, 0xae, 0xe7, - 0x8c, 0x35, 0x6f, 0xec, 0x60, 0xbd, 0x3a, 0x72, 0x6c, 0xcf, 0x46, 0xb7, 0x34, 0x5b, 0x7b, 0xe1, - 0xd8, 0xaa, 0x76, 0x54, 0x75, 0x5f, 0x9a, 0x55, 0x36, 0xaf, 0x24, 0x8f, 0x3d, 0xc3, 0x5c, 0x3d, - 0x32, 0xb5, 0x55, 0xcf, 0x18, 0x62, 0xd7, 0x53, 0x87, 0x23, 0x5f, 0xa1, 0x74, 0x37, 0x0c, 0x37, - 0x72, 0x8c, 0x63, 0xc3, 0xc4, 0x87, 0x98, 0x0d, 0xde, 0x3c, 0xb4, 0x0f, 0x6d, 0xfa, 0x73, 0x95, - 0xfc, 0xf2, 0xa5, 0x95, 0x7f, 0x27, 0xe0, 0xc6, 0xba, 0xed, 0x60, 0xe3, 0xd0, 0x7a, 0x8a, 0x4f, - 0x15, 0x7c, 0x80, 0x1d, 0x6c, 0x69, 0x18, 0x95, 0x21, 0xe9, 0xa9, 0xfb, 0x26, 0x96, 0x85, 0xb2, - 0xb0, 0x92, 0xaf, 0xc3, 0xb7, 0x67, 0xcb, 0x0b, 0xaf, 0xce, 0x96, 0x63, 0xed, 0xa6, 0xe2, 0x0f, - 0xa0, 0x07, 0x90, 0x34, 0x2c, 0x1d, 0x9f, 0xc8, 0x31, 0x3a, 0xa3, 0xc8, 0x66, 0xa4, 0xdb, 0x44, - 0x48, 0xa6, 0xd1, 0x51, 0x24, 0x43, 0xc2, 0x52, 0x87, 0x58, 0x8e, 0x97, 0x85, 0x95, 0x4c, 0x3d, - 0x41, 0x66, 0x29, 0x54, 0x82, 0x9e, 0x82, 0x78, 0xac, 0x9a, 0x86, 0x6e, 0x78, 0xa7, 0x72, 0xa2, - 0x2c, 0xac, 0x14, 0xd6, 0xfe, 0xbf, 0x3a, 0xf5, 0xc4, 0xd5, 0x86, 0x6d, 0xb9, 0x9e, 0xa3, 0x1a, - 0x96, 0xf7, 0x9c, 0x29, 0x30, 0xa0, 0x00, 0x00, 0x3d, 0x84, 0x45, 0xf7, 0x48, 0x75, 0xb0, 0x3e, - 0x18, 0x39, 0xf8, 0xc0, 0x38, 0x19, 0x98, 0xd8, 0x92, 0x93, 0x65, 0x61, 0x25, 0xc9, 0xa6, 0x16, - 0xfd, 0xe1, 0x6d, 0x3a, 0xda, 0xc1, 0x16, 0xea, 0x43, 0xc6, 0xb6, 0x06, 0x3a, 0x36, 0xb1, 0x87, - 0xe5, 0x14, 0x5d, 0xff, 0xe3, 0x19, 0xeb, 0x4f, 0x31, 0x50, 0xb5, 0xa6, 0x79, 0x86, 0x6d, 0xf1, - 0x7d, 0xd8, 0x56, 0x93, 0x02, 0x31, 0xd4, 0xf1, 0x48, 0x57, 0x3d, 0x2c, 0xa7, 0xdf, 0x18, 0x75, - 0x97, 0x02, 0xa1, 0x0e, 0x24, 0x87, 0xaa, 0xa7, 0x1d, 0xc9, 0x22, 0x45, 0x7c, 0x78, 0x0d, 0xc4, - 0x2d, 0xa2, 0xc7, 0x00, 0x7d, 0x90, 0xca, 0x1e, 0xa4, 0xfc, 0x75, 0x50, 0x1e, 0x32, 0xdd, 0xde, - 0xa0, 0xd6, 0xe8, 0xb7, 0x7b, 0x5d, 0x69, 0x01, 0xe5, 0x40, 0x54, 0x5a, 0x3b, 0x7d, 0xa5, 0xdd, - 0xe8, 0x4b, 0x02, 0xf9, 0xda, 0x69, 0xf5, 0x07, 0xdd, 0xdd, 0x4e, 0x47, 0x8a, 0xa1, 0x22, 0x64, - 0xc9, 0x57, 0xb3, 0xb5, 0x5e, 0xdb, 0xed, 0xf4, 0xa5, 0x38, 0xca, 0x42, 0xba, 0x51, 0xdb, 0x69, - 0xd4, 0x9a, 0x2d, 0x29, 0x51, 0x4a, 0xfc, 0xf6, 0x37, 0x4b, 0x0b, 0x95, 0x87, 0x90, 0xa4, 0xcb, - 0x21, 0x80, 0xd4, 0x4e, 0x7b, 0x6b, 0xbb, 0xd3, 0x92, 0x16, 0x90, 0x08, 0x89, 0x75, 0x02, 0x21, - 0x10, 0x8d, 0xed, 0x9a, 0xd2, 0x6f, 0xd7, 0x3a, 0x52, 0x8c, 0x69, 0xfc, 0x33, 0x06, 0x52, 0xc3, - 0x36, 0xc7, 0x43, 0xab, 0x89, 0x5d, 0xcd, 0x31, 0x46, 0x9e, 0xed, 0x04, 0x2e, 0x23, 0x5c, 0x72, - 0x99, 0xf7, 0x20, 0x66, 0xe8, 0xcc, 0xe1, 0x6e, 0x13, 0xf9, 0x39, 0x75, 0xc9, 0x57, 0x67, 0xcb, - 0xa2, 0x8f, 0xd2, 0x6e, 0x2a, 0x31, 0x43, 0x47, 0x1d, 0x48, 0x78, 0xa7, 0x23, 0xdf, 0xe9, 0x72, - 0xf5, 0xc7, 0x64, 0xe6, 0x5f, 0xce, 0x96, 0x1f, 0x1e, 0x1a, 0xde, 0xd1, 0x78, 0xbf, 0xaa, 0xd9, - 0xc3, 0xd5, 0xc0, 0x80, 0xfa, 0xfe, 0xe4, 0xf7, 0xea, 0xe8, 0xc5, 0x21, 0x89, 0x9f, 0x55, 0xa2, - 0xec, 0x56, 0xfb, 0x0a, 0x45, 0x41, 0x65, 0x10, 0xad, 0xb1, 0x69, 0xd2, 0x70, 0x20, 0x8e, 0x2a, - 0xf2, 0xfb, 0xe1, 0x52, 0x74, 0x1f, 0x72, 0x3a, 0x3e, 0x50, 0xc7, 0xa6, 0x37, 0xc0, 0x27, 0x23, - 0x87, 0x3a, 0x5e, 0x46, 0xc9, 0x32, 0x59, 0xeb, 0x64, 0xe4, 0xa0, 0x7b, 0x90, 0x3a, 0x32, 0x74, - 0x1d, 0x5b, 0xd4, 0xd7, 0x38, 0x04, 0x93, 0xa1, 0x35, 0x58, 0x1c, 0xbb, 0xd8, 0x1d, 0xb8, 0xf8, - 0xe5, 0x98, 0x5c, 0xdb, 0xc0, 0xd0, 0x5d, 0x19, 0xca, 0xf1, 0x95, 0x7c, 0x3d, 0xc5, 0xc2, 0xae, - 0x48, 0x26, 0xec, 0xb0, 0xf1, 0xb6, 0xee, 0x92, 0x45, 0x35, 0x7b, 0x38, 0x1a, 0x7b, 0xd8, 0x5f, - 0x34, 0xeb, 0x2f, 0xca, 0x64, 0x64, 0xd1, 0xcd, 0x84, 0x28, 0x4a, 0x99, 0xcd, 0x84, 0x98, 0x91, - 0x60, 0x33, 0x21, 0xa6, 0x25, 0xb1, 0xf2, 0x55, 0x0c, 0x6e, 0xfb, 0xa6, 0x5a, 0x57, 0x87, 0x86, - 0x79, 0xfa, 0xa6, 0x66, 0xf7, 0x51, 0x98, 0xd9, 0xe9, 0x8e, 0x08, 0xf6, 0x80, 0xa8, 0xb9, 0x72, - 0xbc, 0x1c, 0xf7, 0x77, 0x44, 0x64, 0x5d, 0x22, 0x42, 0x8f, 0x01, 0xd8, 0x14, 0x72, 0xc2, 0x04, - 0x3d, 0xe1, 0x9d, 0xf3, 0xb3, 0xe5, 0x0c, 0xbf, 0x3f, 0x37, 0x72, 0x99, 0x19, 0x7f, 0x32, 0x39, - 0x6e, 0x0f, 0x16, 0xb9, 0x8d, 0x03, 0x04, 0x6a, 0xe8, 0x7c, 0xfd, 0x1d, 0xb6, 0xa7, 0x62, 0xd3, - 0x9f, 0xc0, 0xd5, 0x23, 0x50, 0x45, 0x3d, 0x32, 0xa8, 0x57, 0x7e, 0x17, 0x83, 0x9b, 0x6d, 0xcb, - 0xc3, 0x8e, 0x89, 0xd5, 0x63, 0x1c, 0x32, 0xc4, 0x67, 0x90, 0x51, 0x2d, 0x0d, 0xbb, 0x9e, 0xed, - 0xb8, 0xb2, 0x50, 0x8e, 0xaf, 0x64, 0xd7, 0xbe, 0x33, 0x23, 0xe2, 0xa6, 0xe9, 0x57, 0x6b, 0x4c, - 0x99, 0xd9, 0x70, 0x02, 0x56, 0xfa, 0x83, 0x00, 0x22, 0x1f, 0x45, 0x0f, 0x41, 0xa4, 0x4c, 0x4a, - 0xce, 0xe1, 0xb3, 0xec, 0x2d, 0x76, 0x8e, 0x74, 0x9f, 0xc8, 0xe9, 0xfe, 0xc9, 0xcd, 0xa7, 0xe9, - 0xb4, 0xb6, 0x8e, 0x3e, 0x01, 0x91, 0x92, 0xea, 0x20, 0xb8, 0x8d, 0x12, 0xd7, 0x60, 0xac, 0x1b, - 0x26, 0xe0, 0x34, 0x9d, 0xdb, 0xd6, 0x51, 0x63, 0x1a, 0x37, 0xc6, 0xa9, 0xfe, 0x5b, 0xdc, 0x72, - 0x3b, 0x51, 0x76, 0xbc, 0x44, 0x97, 0x95, 0x7f, 0xc4, 0xe1, 0xf6, 0xb6, 0xea, 0x78, 0x06, 0x21, - 0x0e, 0xc3, 0x3a, 0x0c, 0xd9, 0xeb, 0x01, 0x64, 0xad, 0xf1, 0x90, 0xdd, 0x8a, 0xcb, 0xce, 0xe2, - 0x9f, 0x1d, 0xac, 0xf1, 0xd0, 0x37, 0xb8, 0x4b, 0x82, 0xd2, 0x34, 0x5c, 0x4f, 0x8e, 0x51, 0x8b, - 0xae, 0xcd, 0xb0, 0xe8, 0xf4, 0x35, 0xaa, 0x1d, 0xc3, 0xf5, 0xb8, 0x4f, 0x12, 0x14, 0xd4, 0x83, - 0xa4, 0xa3, 0x5a, 0x87, 0x98, 0x3a, 0x59, 0x76, 0xed, 0xd1, 0xf5, 0xe0, 0x14, 0xa2, 0xca, 0x59, - 0x91, 0xe2, 0x94, 0x7e, 0x29, 0x40, 0x82, 0xac, 0x72, 0x45, 0x1c, 0xdc, 0x86, 0xd4, 0xb1, 0x6a, - 0x8e, 0xb1, 0x4b, 0xcf, 0x90, 0x53, 0xd8, 0x17, 0xfa, 0x09, 0x14, 0xdd, 0xf1, 0xfe, 0x28, 0xb4, - 0x14, 0x35, 0x6f, 0x76, 0xed, 0xa3, 0x6b, 0xed, 0x2a, 0xc8, 0x54, 0x51, 0xac, 0xd2, 0x0b, 0x48, - 0xd2, 0xfd, 0x5e, 0xb1, 0xb3, 0xfb, 0x90, 0xf3, 0xec, 0x01, 0x3e, 0xd1, 0xcc, 0xb1, 0x6b, 0x1c, - 0x63, 0xea, 0x1d, 0x39, 0x25, 0xeb, 0xd9, 0x2d, 0x2e, 0x42, 0x0f, 0xa0, 0x70, 0xe0, 0xd8, 0xc3, - 0x81, 0x61, 0xf1, 0x49, 0x94, 0x1d, 0x95, 0x3c, 0x91, 0xb6, 0xb9, 0xb0, 0xf2, 0x1f, 0x11, 0x8a, - 0xd4, 0x83, 0xe6, 0x62, 0x86, 0x07, 0x21, 0x66, 0xb8, 0x15, 0x61, 0x86, 0xc0, 0x0d, 0x09, 0x31, - 0xdc, 0x83, 0xd4, 0xd8, 0x32, 0x5e, 0x8e, 0xfd, 0x35, 0x03, 0xf2, 0xf3, 0x65, 0x97, 0x68, 0x23, - 0x71, 0x99, 0x36, 0x3e, 0x04, 0x44, 0x62, 0x06, 0x0f, 0x22, 0x13, 0x93, 0x74, 0xa2, 0x44, 0x47, - 0x1a, 0x33, 0x49, 0x26, 0x75, 0x0d, 0x92, 0xd9, 0x00, 0x09, 0x9f, 0x78, 0x8e, 0x3a, 0x08, 0xe9, - 0xa7, 0xa9, 0xfe, 0xd2, 0xf9, 0xd9, 0x72, 0xa1, 0x45, 0xc6, 0xa6, 0x83, 0x14, 0x70, 0x68, 0x4c, - 0x27, 0x3e, 0xb1, 0xc8, 0x30, 0x74, 0xc3, 0xc1, 0x34, 0xdd, 0xba, 0xb2, 0x58, 0x8e, 0x5f, 0x91, - 0xbe, 0x2f, 0x98, 0xbd, 0xda, 0xe4, 0x8a, 0x8a, 0xe4, 0x43, 0x05, 0x02, 0x17, 0x3d, 0x83, 0xec, - 0x81, 0x9f, 0xed, 0x07, 0x2f, 0xf0, 0xa9, 0x9c, 0xa1, 0xee, 0xf6, 0xfe, 0xfc, 0x75, 0x01, 0x8f, - 0xcf, 0x83, 0x60, 0x08, 0xed, 0x42, 0xde, 0xe1, 0xc3, 0xfa, 0x60, 0xff, 0x94, 0xe6, 0x9f, 0xd7, - 0x01, 0xcd, 0x4d, 0x60, 0xea, 0xa7, 0xe8, 0x19, 0x80, 0x11, 0xb0, 0x24, 0x4d, 0x52, 0xd9, 0xb5, - 0x0f, 0xae, 0x41, 0xa7, 0x7c, 0xa7, 0x13, 0x10, 0xb4, 0x07, 0x85, 0xc9, 0x17, 0xdd, 0x6a, 0xee, - 0x35, 0xb7, 0x9a, 0x0f, 0xe1, 0xd4, 0x4f, 0x51, 0x1f, 0x6e, 0x92, 0xf4, 0x69, 0xbb, 0x86, 0x87, - 0xc3, 0x2e, 0x90, 0xa7, 0x2e, 0x50, 0x39, 0x3f, 0x5b, 0x46, 0x0d, 0x3e, 0x3e, 0xdd, 0x0d, 0x90, - 0x76, 0x61, 0xdc, 0x77, 0xaa, 0x88, 0xf3, 0x12, 0xc4, 0xc2, 0xc4, 0xa9, 0x76, 0x26, 0xee, 0x7b, - 0xc9, 0xa9, 0x42, 0xae, 0x4d, 0x90, 0xf6, 0x20, 0x17, 0x61, 0x99, 0xe2, 0xeb, 0xb3, 0x4c, 0x04, - 0x08, 0xb5, 0x58, 0xc1, 0x24, 0xd1, 0xfa, 0xf2, 0x83, 0x39, 0x1d, 0xb4, 0x7f, 0x3a, 0xe2, 0x86, - 0xa4, 0xea, 0x95, 0x25, 0xc8, 0x04, 0x3e, 0x8a, 0xd2, 0x10, 0xaf, 0xed, 0x34, 0xfc, 0x0a, 0xb0, - 0xd9, 0xda, 0x69, 0x48, 0x42, 0xe5, 0x3e, 0x24, 0x88, 0x0e, 0xa9, 0x04, 0xd7, 0x7b, 0xca, 0x5e, - 0x4d, 0x69, 0xfa, 0x55, 0x67, 0xbb, 0xfb, 0xbc, 0xa5, 0xf4, 0x5b, 0x4d, 0x49, 0x20, 0x05, 0x0a, - 0x9a, 0xd4, 0xfb, 0x7d, 0x9b, 0x55, 0xc0, 0x87, 0x50, 0xd4, 0x02, 0xe9, 0x80, 0xee, 0x55, 0x28, - 0xc7, 0x56, 0x0a, 0x6b, 0x8f, 0xff, 0x67, 0xcf, 0xc0, 0x31, 0xc2, 0xa2, 0xc9, 0xc6, 0x0b, 0x5a, - 0x44, 0x1a, 0x70, 0x5d, 0xac, 0x1c, 0xbb, 0xc0, 0x75, 0x0a, 0x24, 0xb5, 0x23, 0xac, 0xbd, 0x60, - 0xdc, 0xfe, 0xdd, 0x19, 0x0b, 0xd3, 0xdc, 0x1d, 0x32, 0x52, 0x83, 0xe8, 0x4c, 0x96, 0xe6, 0x49, - 0x87, 0x42, 0x55, 0xee, 0x42, 0x21, 0xba, 0x2b, 0x94, 0x81, 0x64, 0x63, 0xa3, 0xd5, 0x78, 0x2a, - 0x2d, 0x54, 0xfe, 0x95, 0x00, 0x34, 0x01, 0xda, 0x1a, 0x7b, 0x2a, 0xb5, 0x6b, 0x0d, 0x52, 0xbe, - 0x23, 0x51, 0x3e, 0xce, 0xae, 0xfd, 0xdf, 0x4c, 0x0b, 0x44, 0xeb, 0xea, 0x8d, 0x05, 0x85, 0x29, - 0xa2, 0x4f, 0xc3, 0xbd, 0x5b, 0x76, 0xed, 0xbd, 0xf9, 0xee, 0x7b, 0x63, 0x81, 0x37, 0x75, 0x4f, - 0x21, 0xe9, 0x7a, 0xa4, 0xc3, 0x89, 0x53, 0x7f, 0x59, 0x9d, 0xa1, 0x7f, 0x79, 0xf3, 0xd5, 0x1d, - 0xa2, 0xc6, 0x6d, 0x40, 0x31, 0xd0, 0x1e, 0x64, 0x02, 0x8a, 0x64, 0x8d, 0xe0, 0xa3, 0xf9, 0x01, - 0x03, 0x7f, 0xe3, 0xd5, 0x56, 0x80, 0x85, 0x6a, 0x90, 0x1d, 0xb2, 0x69, 0x93, 0x5a, 0xb1, 0xcc, - 0xb2, 0x14, 0x70, 0x04, 0x9a, 0xad, 0x42, 0x5f, 0x0a, 0x70, 0xa5, 0xb6, 0x4e, 0x4a, 0x7f, 0xc7, - 0x36, 0xcd, 0x7d, 0x55, 0x7b, 0x41, 0xbb, 0xb9, 0xa0, 0xf4, 0xe7, 0x52, 0xf4, 0x94, 0xe4, 0x1a, - 0x7e, 0x83, 0xb4, 0x3f, 0xcb, 0xce, 0xd1, 0xc7, 0x72, 0x9f, 0xdc, 0x58, 0x50, 0x42, 0xea, 0x95, - 0x1f, 0x41, 0x92, 0x1a, 0x88, 0x04, 0xc8, 0x6e, 0xf7, 0x69, 0xb7, 0xb7, 0x47, 0xda, 0xb2, 0x22, - 0x64, 0x9b, 0xad, 0x4e, 0xab, 0xdf, 0x1a, 0xf4, 0xba, 0x9d, 0xcf, 0x25, 0x01, 0xdd, 0x81, 0x5b, - 0x4c, 0x50, 0xeb, 0x36, 0x07, 0x7b, 0x4a, 0x9b, 0x0f, 0xc5, 0x2a, 0x2b, 0xe1, 0x08, 0x14, 0x21, - 0xd1, 0xed, 0x75, 0x49, 0x13, 0x46, 0x62, 0xb1, 0xd9, 0x94, 0x04, 0x1a, 0x8b, 0x4a, 0x6f, 0x5b, - 0x8a, 0xd5, 0x73, 0x00, 0x7a, 0x60, 0xce, 0xcd, 0x84, 0x98, 0x92, 0xd2, 0x95, 0x5f, 0xdd, 0x85, - 0xe2, 0x05, 0xff, 0xbd, 0x22, 0xf9, 0x97, 0x69, 0xf2, 0xf7, 0x0b, 0x49, 0x29, 0x92, 0xfc, 0x63, - 0x2c, 0xef, 0x3f, 0x82, 0xcc, 0x48, 0x75, 0xb0, 0xe5, 0x11, 0xfb, 0x27, 0x22, 0xfd, 0x83, 0xb8, - 0x4d, 0x07, 0x82, 0xe9, 0xa2, 0x3f, 0xb1, 0x4d, 0x94, 0xd2, 0xc7, 0xd8, 0x71, 0x89, 0x37, 0xf8, - 0x57, 0x76, 0x87, 0x3d, 0x2d, 0x2c, 0x4e, 0x76, 0xf5, 0xdc, 0x9f, 0xa0, 0xf0, 0x99, 0x68, 0x1b, - 0x16, 0x87, 0xb6, 0x6e, 0x1c, 0x18, 0x9a, 0x7f, 0xdf, 0x9e, 0x31, 0xf4, 0xfb, 0xef, 0xec, 0xda, - 0xdb, 0xa1, 0xdb, 0x18, 0x7b, 0x86, 0x59, 0x3d, 0x32, 0xb5, 0x6a, 0x9f, 0x3f, 0x9d, 0xb0, 0x13, - 0x49, 0x61, 0x6d, 0x32, 0x88, 0x9e, 0x40, 0x9a, 0x57, 0xb4, 0x22, 0xcd, 0x2e, 0xf3, 0xc6, 0x19, - 0x43, 0xe4, 0xda, 0x68, 0x1d, 0x0a, 0x16, 0x3e, 0x09, 0x77, 0x2d, 0x99, 0x88, 0x27, 0xe6, 0xba, - 0xf8, 0x64, 0x7a, 0xcb, 0x92, 0xb3, 0x26, 0x23, 0x3a, 0x7a, 0x06, 0xf9, 0x91, 0x63, 0x0c, 0x55, - 0xe7, 0x74, 0xe0, 0x07, 0x2f, 0x5c, 0x27, 0x78, 0x03, 0xda, 0xf7, 0x21, 0xe8, 0x28, 0x5a, 0x07, - 0xbf, 0x49, 0xc0, 0xae, 0x9c, 0xa5, 0x67, 0xbc, 0x1e, 0x18, 0x57, 0x46, 0x75, 0xc8, 0xd3, 0x23, - 0x06, 0xdd, 0x49, 0x8e, 0x9e, 0x70, 0x89, 0x9d, 0x30, 0x4b, 0x4e, 0x38, 0xa5, 0x43, 0xc9, 0x5a, - 0x81, 0x5c, 0x47, 0x9b, 0x00, 0xc1, 0x93, 0x15, 0xc9, 0xb8, 0x57, 0x15, 0x34, 0xdb, 0x7c, 0xe2, - 0x64, 0x4b, 0x4a, 0x48, 0x1b, 0x6d, 0x41, 0x86, 0x07, 0xb1, 0x9f, 0x6a, 0x67, 0xc7, 0xe4, 0x65, - 0x4a, 0xe1, 0x44, 0x12, 0x20, 0xa0, 0x2e, 0x24, 0x4d, 0xac, 0xba, 0x98, 0xe5, 0xdb, 0xc7, 0x73, - 0x32, 0xff, 0x8e, 0x76, 0x84, 0x87, 0x6a, 0xe3, 0x88, 0xd4, 0xee, 0x1d, 0xa2, 0xaf, 0xf8, 0x30, - 0xa8, 0x0b, 0x12, 0x35, 0x57, 0x98, 0x9d, 0x24, 0x6a, 0xb1, 0x77, 0x99, 0xc5, 0x0a, 0xc4, 0x62, - 0x33, 0x19, 0x8a, 0xfa, 0xd3, 0xd6, 0x84, 0xa5, 0x7e, 0x08, 0x85, 0x03, 0xdb, 0x19, 0xaa, 0xde, - 0x80, 0x07, 0xce, 0xe2, 0xa4, 0x22, 0x7f, 0x75, 0xb6, 0x9c, 0x5f, 0xa7, 0xa3, 0x3c, 0x68, 0xf2, - 0x07, 0xe1, 0x4f, 0xb4, 0xc1, 0xc9, 0xfc, 0x06, 0xe5, 0xde, 0x0f, 0xe7, 0x3d, 0xdd, 0x65, 0x26, - 0xef, 0x42, 0x8a, 0xa6, 0x35, 0x57, 0xbe, 0x49, 0x6d, 0xfe, 0x9a, 0x29, 0x52, 0x61, 0x28, 0xe8, - 0x0b, 0x28, 0xe8, 0x44, 0x62, 0x58, 0x87, 0xac, 0xe2, 0xbf, 0x45, 0x71, 0x57, 0xe7, 0xc4, 0x25, - 0xdd, 0x40, 0xdb, 0x3a, 0xb0, 0x79, 0xb1, 0xc7, 0xc1, 0xfc, 0x2e, 0xa1, 0x07, 0xe2, 0x81, 0x3a, - 0x34, 0x4c, 0x03, 0xbb, 0xf2, 0x6d, 0x8a, 0xfb, 0xd1, 0x95, 0x11, 0x7e, 0xf1, 0xc1, 0x84, 0xa7, - 0x02, 0x0e, 0x12, 0x04, 0x3a, 0x15, 0x9c, 0x92, 0x4b, 0x7d, 0xeb, 0x72, 0xa0, 0xf3, 0x07, 0x93, - 0xc8, 0xe3, 0x09, 0x0d, 0x74, 0xf6, 0xa5, 0xa3, 0x77, 0x00, 0x8e, 0x0d, 0xfc, 0xb3, 0xc1, 0xcb, - 0x31, 0x76, 0x4e, 0x65, 0x39, 0xc4, 0xbb, 0x19, 0x22, 0x7f, 0x46, 0xc4, 0xe8, 0x63, 0xc8, 0xe8, - 0x78, 0x84, 0x2d, 0xdd, 0xed, 0x59, 0xf2, 0x1d, 0x5a, 0x4d, 0xde, 0x20, 0x2d, 0x4e, 0x93, 0x0b, - 0x19, 0xaf, 0x4e, 0x66, 0xa1, 0x2f, 0x21, 0xe7, 0x7f, 0x60, 0xbd, 0x67, 0xd5, 0x4f, 0xe5, 0x12, - 0x3d, 0xf4, 0xc3, 0x39, 0x8d, 0x39, 0x29, 0x9d, 0x6f, 0xf2, 0xf3, 0x34, 0x43, 0x68, 0x4a, 0x04, - 0x1b, 0x7d, 0x01, 0x39, 0xee, 0xdd, 0x9b, 0xf6, 0xbe, 0x2b, 0xdf, 0xbd, 0xb2, 0xe9, 0xbf, 0xb8, - 0xd6, 0xd6, 0x44, 0x95, 0xf3, 0x56, 0x18, 0x0d, 0x7d, 0x06, 0xf9, 0xe0, 0xa5, 0xcc, 0x1e, 0x79, - 0xae, 0x7c, 0x8f, 0x06, 0xe6, 0xa3, 0x79, 0x5d, 0x97, 0xe9, 0xf6, 0x46, 0x9e, 0xab, 0xe4, 0xdc, - 0xd0, 0x17, 0xba, 0x0f, 0x19, 0xdd, 0xb1, 0x47, 0x7e, 0xfe, 0x78, 0xbb, 0x2c, 0xac, 0xc4, 0xf9, - 0x35, 0x13, 0x31, 0x4d, 0x0c, 0x03, 0x28, 0x38, 0x78, 0x64, 0xaa, 0x1a, 0x1e, 0x92, 0xcc, 0x66, - 0x1f, 0xc8, 0x4b, 0x74, 0xf5, 0xb5, 0xb9, 0x0d, 0x19, 0x28, 0x73, 0xc7, 0x0c, 0xe1, 0xf5, 0x0e, - 0xd0, 0x2e, 0x80, 0x3a, 0xd6, 0x0d, 0x6f, 0x30, 0xb4, 0x75, 0x2c, 0x2f, 0x5f, 0xf9, 0xe4, 0x7b, - 0x11, 0xbc, 0x46, 0x14, 0xb7, 0x6c, 0x1d, 0x07, 0x8f, 0x4f, 0x5c, 0x80, 0x3e, 0x86, 0x2c, 0x3d, - 0xda, 0x97, 0xf6, 0x3e, 0xf1, 0xcd, 0x32, 0x3d, 0xdc, 0x22, 0xbb, 0xcb, 0x4c, 0xd3, 0xb1, 0x47, - 0x9b, 0xf6, 0x3e, 0xf5, 0x18, 0xf6, 0x53, 0x47, 0x2e, 0xe4, 0x0e, 0xb5, 0xc1, 0x84, 0x4a, 0xef, - 0xd3, 0x5b, 0xfc, 0xc1, 0x9c, 0x7b, 0x79, 0xd2, 0x98, 0x42, 0xae, 0x37, 0x78, 0x4e, 0x78, 0xd2, - 0xe0, 0x32, 0x57, 0xc9, 0x1e, 0x6a, 0xc1, 0x47, 0xe9, 0x1b, 0x01, 0x16, 0x2f, 0x51, 0x27, 0xfa, - 0x29, 0xa4, 0x2d, 0x5b, 0x0f, 0x3d, 0x96, 0xb5, 0x18, 0x50, 0xaa, 0x6b, 0xeb, 0xfe, 0x5b, 0xd9, - 0xa3, 0xb9, 0xde, 0x77, 0xe9, 0xaf, 0xd1, 0x7e, 0xd5, 0x57, 0x53, 0x52, 0x04, 0xb5, 0xad, 0xa3, - 0x8f, 0xa0, 0x88, 0x4f, 0x46, 0x86, 0x13, 0x2a, 0x1f, 0x62, 0xa1, 0xeb, 0x2f, 0x4c, 0x06, 0x89, - 0x13, 0x94, 0xfe, 0x24, 0x40, 0xf1, 0x02, 0x6d, 0x91, 0x4a, 0x89, 0x3e, 0xc4, 0x46, 0x2a, 0x25, - 0x22, 0x09, 0x35, 0x15, 0x57, 0xfd, 0x09, 0x12, 0x7f, 0xd3, 0x3f, 0x41, 0xa2, 0xef, 0x1e, 0xc9, - 0xf9, 0xdf, 0x3d, 0x36, 0x13, 0x62, 0x42, 0x4a, 0x96, 0x3e, 0x07, 0x91, 0x53, 0x66, 0xb4, 0x74, - 0x13, 0xe6, 0x2c, 0xdd, 0x66, 0x9e, 0xb3, 0xf4, 0xb5, 0x00, 0x99, 0xf0, 0xbf, 0x4b, 0xb1, 0x00, - 0x75, 0x7a, 0xe5, 0xf8, 0x9a, 0x4f, 0x9d, 0x51, 0x0b, 0xc4, 0xe7, 0xb7, 0x40, 0xe9, 0x18, 0xb2, - 0x21, 0xd6, 0xb9, 0xd8, 0x3b, 0x08, 0xaf, 0xd1, 0x3b, 0xbc, 0x0b, 0x29, 0x16, 0x6a, 0xbe, 0x23, - 0xe5, 0x99, 0x76, 0xd2, 0x0f, 0xb3, 0xe4, 0x97, 0x24, 0xc4, 0x4a, 0xbf, 0x17, 0x20, 0x17, 0xe6, - 0x23, 0x54, 0x81, 0x8c, 0x61, 0x69, 0x0e, 0x25, 0x03, 0xba, 0x2e, 0x77, 0xc1, 0x89, 0x98, 0xb0, - 0xd4, 0xd0, 0xb0, 0x06, 0xf4, 0xf9, 0x31, 0xe2, 0xa6, 0xe2, 0xd0, 0xb0, 0x9e, 0x13, 0x29, 0x9d, - 0xa2, 0x9e, 0xb0, 0x29, 0xf1, 0xc8, 0x14, 0xf5, 0xc4, 0x9f, 0x52, 0xa2, 0x89, 0xdf, 0xf1, 0x68, - 0x65, 0x1e, 0x0f, 0xa5, 0x72, 0xc7, 0x43, 0x4b, 0x90, 0x3e, 0x36, 0x1c, 0x6f, 0xac, 0x9a, 0xb4, - 0x08, 0xe7, 0x7d, 0x0f, 0x17, 0x96, 0x8e, 0x20, 0x1b, 0xe2, 0xb1, 0x39, 0x2e, 0xf4, 0x7b, 0x90, - 0x08, 0x82, 0x6a, 0xce, 0x9a, 0x9c, 0x2a, 0x94, 0x7e, 0x21, 0xc0, 0xcd, 0x69, 0x4c, 0x12, 0x71, - 0x11, 0xdf, 0x4e, 0x73, 0xb9, 0x48, 0x84, 0xe1, 0x63, 0x53, 0x19, 0x7e, 0x72, 0x73, 0xf1, 0xd9, - 0x37, 0x57, 0xf9, 0x84, 0x37, 0x6b, 0x00, 0xa9, 0xed, 0xdd, 0x7a, 0xa7, 0xdd, 0x98, 0xda, 0x68, - 0xa1, 0x3c, 0x64, 0xda, 0x5b, 0xdb, 0x3d, 0xa5, 0xdf, 0xee, 0x3e, 0x91, 0xe2, 0xa4, 0x43, 0x0b, - 0x48, 0x1a, 0xe5, 0x40, 0x6c, 0xb6, 0x77, 0x6a, 0xf5, 0x4e, 0xab, 0x29, 0x2d, 0x90, 0x99, 0x4a, - 0xab, 0xd6, 0xa4, 0x0d, 0x9d, 0x24, 0x7c, 0x3f, 0xf1, 0xd5, 0xaf, 0x97, 0x05, 0xbf, 0x33, 0xdb, - 0x4c, 0x88, 0x48, 0xba, 0x51, 0xf9, 0x46, 0x00, 0xd4, 0x54, 0x3d, 0x95, 0x10, 0xc2, 0x35, 0x5a, - 0xb4, 0xd8, 0x15, 0xf7, 0x12, 0x2d, 0xbb, 0xe3, 0x6f, 0x52, 0x76, 0xfb, 0x1b, 0xae, 0x7c, 0x2d, - 0x00, 0x84, 0x36, 0xf7, 0x69, 0xf8, 0x9f, 0xe4, 0xd9, 0x1d, 0xc6, 0x85, 0xe4, 0xb1, 0xb1, 0xc0, - 0xff, 0x67, 0x7e, 0x02, 0xa2, 0xce, 0x8e, 0xcc, 0x9c, 0x67, 0x66, 0x29, 0x7f, 0xc9, 0x32, 0x1b, - 0xe4, 0x56, 0x99, 0xb4, 0x9e, 0x86, 0xe4, 0xd8, 0x32, 0x6c, 0xeb, 0xfd, 0x66, 0xf8, 0x85, 0x89, - 0x93, 0x29, 0x31, 0x3e, 0xfd, 0xad, 0x7a, 0x58, 0xf7, 0x9b, 0xee, 0x5d, 0xeb, 0x38, 0x10, 0x08, - 0xa8, 0x00, 0xc0, 0xc6, 0x0d, 0xeb, 0x50, 0x8a, 0xd5, 0xef, 0x7f, 0xfb, 0xb7, 0xa5, 0x85, 0x6f, - 0xcf, 0x97, 0x84, 0x3f, 0x9e, 0x2f, 0x09, 0x7f, 0x3e, 0x5f, 0x12, 0xfe, 0x7a, 0xbe, 0x24, 0xfc, - 0xfc, 0xef, 0x4b, 0x0b, 0x3f, 0x4e, 0xb3, 0x0d, 0xfd, 0x37, 0x00, 0x00, 0xff, 0xff, 0x3e, 0xad, - 0x7f, 0xbf, 0xdf, 0x1f, 0x00, 0x00, + proto.RegisterFile("sql/sqlbase/structured.proto", fileDescriptor_structured_a76e1756e60399f7) +} + +var fileDescriptor_structured_a76e1756e60399f7 = []byte{ + // 2874 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x59, 0xcd, 0x73, 0x23, 0x47, + 0x15, 0xf7, 0xe8, 0x73, 0xf4, 0xf4, 0x35, 0xee, 0xfd, 0xc8, 0xac, 0xb2, 0xb1, 0xb5, 0x4a, 0x36, + 0x98, 0x7c, 0xc8, 0x1b, 0x2f, 0x81, 0x2d, 0x42, 0xa5, 0xd0, 0x97, 0xd7, 0xf2, 0xca, 0x92, 0x77, + 0x2c, 0xaf, 0x13, 0x2a, 0x30, 0x35, 0xd6, 0xb4, 0xed, 0xc9, 0x8e, 0x66, 0xb4, 0x33, 0x23, 0x63, + 0xff, 0x07, 0x39, 0x72, 0x82, 0x13, 0x29, 0x2a, 0xc5, 0x01, 0x8a, 0x2b, 0x07, 0xfe, 0x84, 0x1c, + 0x29, 0x4e, 0x14, 0x07, 0x17, 0x98, 0xe2, 0x0a, 0xf7, 0x14, 0x54, 0x51, 0xdd, 0xd3, 0x3d, 0x9a, + 0xb1, 0x25, 0x23, 0xef, 0x72, 0xd3, 0xbc, 0x7e, 0xef, 0xd7, 0xdd, 0xaf, 0xdf, 0xfb, 0xf5, 0x7b, + 0x2d, 0xb8, 0xeb, 0xbe, 0x30, 0x57, 0xdd, 0x17, 0xe6, 0xbe, 0xe6, 0xe2, 0x55, 0xd7, 0x73, 0xc6, + 0x03, 0x6f, 0xec, 0x60, 0xbd, 0x3a, 0x72, 0x6c, 0xcf, 0x46, 0xb7, 0x06, 0xf6, 0xe0, 0xb9, 0x63, + 0x6b, 0x83, 0xa3, 0xaa, 0xfb, 0xc2, 0xac, 0x32, 0xbd, 0x92, 0x3c, 0xf6, 0x0c, 0x73, 0xf5, 0xc8, + 0x1c, 0xac, 0x7a, 0xc6, 0x10, 0xbb, 0x9e, 0x36, 0x1c, 0xf9, 0x06, 0xa5, 0xd7, 0xc3, 0x70, 0x23, + 0xc7, 0x38, 0x36, 0x4c, 0x7c, 0x88, 0xd9, 0xe0, 0xcd, 0x43, 0xfb, 0xd0, 0xa6, 0x3f, 0x57, 0xc9, + 0x2f, 0x5f, 0x5a, 0xf9, 0x77, 0x02, 0x6e, 0xac, 0xdb, 0x0e, 0x36, 0x0e, 0xad, 0x27, 0xf8, 0x54, + 0xc1, 0x07, 0xd8, 0xc1, 0xd6, 0x00, 0xa3, 0x32, 0x24, 0x3d, 0x6d, 0xdf, 0xc4, 0xb2, 0x50, 0x16, + 0x56, 0xf2, 0x75, 0xf8, 0xfa, 0x6c, 0x79, 0xe1, 0x9b, 0xb3, 0xe5, 0x58, 0xbb, 0xa9, 0xf8, 0x03, + 0xe8, 0x3e, 0x24, 0x0d, 0x4b, 0xc7, 0x27, 0x72, 0x8c, 0x6a, 0x14, 0x99, 0x46, 0xba, 0x4d, 0x84, + 0x44, 0x8d, 0x8e, 0x22, 0x19, 0x12, 0x96, 0x36, 0xc4, 0x72, 0xbc, 0x2c, 0xac, 0x64, 0xea, 0x09, + 0xa2, 0xa5, 0x50, 0x09, 0x7a, 0x02, 0xe2, 0xb1, 0x66, 0x1a, 0xba, 0xe1, 0x9d, 0xca, 0x89, 0xb2, + 0xb0, 0x52, 0x58, 0xfb, 0x76, 0x75, 0xea, 0x8e, 0xab, 0x0d, 0xdb, 0x72, 0x3d, 0x47, 0x33, 0x2c, + 0xef, 0x19, 0x33, 0x60, 0x40, 0x01, 0x00, 0x7a, 0x00, 0x8b, 0xee, 0x91, 0xe6, 0x60, 0x5d, 0x1d, + 0x39, 0xf8, 0xc0, 0x38, 0x51, 0x4d, 0x6c, 0xc9, 0xc9, 0xb2, 0xb0, 0x92, 0x64, 0xaa, 0x45, 0x7f, + 0x78, 0x9b, 0x8e, 0x76, 0xb0, 0x85, 0xfa, 0x90, 0xb1, 0x2d, 0x55, 0xc7, 0x26, 0xf6, 0xb0, 0x9c, + 0xa2, 0xf3, 0x7f, 0x30, 0x63, 0xfe, 0x29, 0x0e, 0xaa, 0xd6, 0x06, 0x9e, 0x61, 0x5b, 0x7c, 0x1d, + 0xb6, 0xd5, 0xa4, 0x40, 0x0c, 0x75, 0x3c, 0xd2, 0x35, 0x0f, 0xcb, 0xe9, 0x57, 0x46, 0xdd, 0xa5, + 0x40, 0xa8, 0x03, 0xc9, 0xa1, 0xe6, 0x0d, 0x8e, 0x64, 0x91, 0x22, 0x3e, 0xb8, 0x06, 0xe2, 0x16, + 0xb1, 0x63, 0x80, 0x3e, 0x48, 0x65, 0x0f, 0x52, 0xfe, 0x3c, 0x28, 0x0f, 0x99, 0x6e, 0x4f, 0xad, + 0x35, 0xfa, 0xed, 0x5e, 0x57, 0x5a, 0x40, 0x39, 0x10, 0x95, 0xd6, 0x4e, 0x5f, 0x69, 0x37, 0xfa, + 0x92, 0x40, 0xbe, 0x76, 0x5a, 0x7d, 0xb5, 0xbb, 0xdb, 0xe9, 0x48, 0x31, 0x54, 0x84, 0x2c, 0xf9, + 0x6a, 0xb6, 0xd6, 0x6b, 0xbb, 0x9d, 0xbe, 0x14, 0x47, 0x59, 0x48, 0x37, 0x6a, 0x3b, 0x8d, 0x5a, + 0xb3, 0x25, 0x25, 0x4a, 0x89, 0xdf, 0xfc, 0x7a, 0x69, 0xa1, 0xf2, 0x00, 0x92, 0x74, 0x3a, 0x04, + 0x90, 0xda, 0x69, 0x6f, 0x6d, 0x77, 0x5a, 0xd2, 0x02, 0x12, 0x21, 0xb1, 0x4e, 0x20, 0x04, 0x62, + 0xb1, 0x5d, 0x53, 0xfa, 0xed, 0x5a, 0x47, 0x8a, 0x31, 0x8b, 0x7f, 0xc6, 0x40, 0x6a, 0xd8, 0xe6, + 0x78, 0x68, 0x35, 0xb1, 0x3b, 0x70, 0x8c, 0x91, 0x67, 0x3b, 0x41, 0xc8, 0x08, 0x97, 0x42, 0xe6, + 0x6d, 0x88, 0x19, 0x3a, 0x0b, 0xb8, 0xdb, 0x44, 0x7e, 0x4e, 0x43, 0xf2, 0x9b, 0xb3, 0x65, 0xd1, + 0x47, 0x69, 0x37, 0x95, 0x98, 0xa1, 0xa3, 0x0e, 0x24, 0xbc, 0xd3, 0x91, 0x1f, 0x74, 0xb9, 0xfa, + 0x23, 0xa2, 0xf9, 0x97, 0xb3, 0xe5, 0x07, 0x87, 0x86, 0x77, 0x34, 0xde, 0xaf, 0x0e, 0xec, 0xe1, + 0x6a, 0xe0, 0x40, 0x7d, 0x7f, 0xf2, 0x7b, 0x75, 0xf4, 0xfc, 0x90, 0xe4, 0xcf, 0x2a, 0x31, 0x76, + 0xab, 0x7d, 0x85, 0xa2, 0xa0, 0x32, 0x88, 0xd6, 0xd8, 0x34, 0x69, 0x3a, 0x90, 0x40, 0x15, 0xf9, + 0xf9, 0x70, 0x29, 0xba, 0x07, 0x39, 0x1d, 0x1f, 0x68, 0x63, 0xd3, 0x53, 0xf1, 0xc9, 0xc8, 0xa1, + 0x81, 0x97, 0x51, 0xb2, 0x4c, 0xd6, 0x3a, 0x19, 0x39, 0xe8, 0x2e, 0xa4, 0x8e, 0x0c, 0x5d, 0xc7, + 0x16, 0x8d, 0x35, 0x0e, 0xc1, 0x64, 0x68, 0x0d, 0x16, 0xc7, 0x2e, 0x76, 0x55, 0x17, 0xbf, 0x18, + 0x93, 0x63, 0x53, 0x0d, 0xdd, 0x95, 0xa1, 0x1c, 0x5f, 0xc9, 0xd7, 0x53, 0x2c, 0xed, 0x8a, 0x44, + 0x61, 0x87, 0x8d, 0xb7, 0x75, 0x97, 0x4c, 0x3a, 0xb0, 0x87, 0xa3, 0xb1, 0x87, 0xfd, 0x49, 0xb3, + 0xfe, 0xa4, 0x4c, 0x46, 0x26, 0xdd, 0x4c, 0x88, 0xa2, 0x94, 0xd9, 0x4c, 0x88, 0x19, 0x09, 0x36, + 0x13, 0x62, 0x5a, 0x12, 0x2b, 0x5f, 0xc4, 0xe0, 0xb6, 0xef, 0xaa, 0x75, 0x6d, 0x68, 0x98, 0xa7, + 0xaf, 0xea, 0x76, 0x1f, 0x85, 0xb9, 0x9d, 0xae, 0x88, 0x60, 0xab, 0xc4, 0xcc, 0x95, 0xe3, 0xe5, + 0xb8, 0xbf, 0x22, 0x22, 0xeb, 0x12, 0x11, 0x7a, 0x04, 0xc0, 0x54, 0xc8, 0x0e, 0x13, 0x74, 0x87, + 0x77, 0xce, 0xcf, 0x96, 0x33, 0xfc, 0xfc, 0xdc, 0xc8, 0x61, 0x66, 0x7c, 0x65, 0xb2, 0xdd, 0x1e, + 0x2c, 0x72, 0x1f, 0x07, 0x08, 0xd4, 0xd1, 0xf9, 0xfa, 0x9b, 0x6c, 0x4d, 0xc5, 0xa6, 0xaf, 0xc0, + 0xcd, 0x23, 0x50, 0x45, 0x3d, 0x32, 0xa8, 0x57, 0x7e, 0x17, 0x83, 0x9b, 0x6d, 0xcb, 0xc3, 0x8e, + 0x89, 0xb5, 0x63, 0x1c, 0x72, 0xc4, 0x27, 0x90, 0xd1, 0xac, 0x01, 0x76, 0x3d, 0xdb, 0x71, 0x65, + 0xa1, 0x1c, 0x5f, 0xc9, 0xae, 0x7d, 0x67, 0x46, 0xc6, 0x4d, 0xb3, 0xaf, 0xd6, 0x98, 0x31, 0xf3, + 0xe1, 0x04, 0xac, 0xf4, 0x07, 0x01, 0x44, 0x3e, 0x8a, 0x1e, 0x80, 0x48, 0x99, 0x94, 0xec, 0xc3, + 0x67, 0xd9, 0x5b, 0x6c, 0x1f, 0xe9, 0x3e, 0x91, 0xd3, 0xf5, 0x93, 0x93, 0x4f, 0x53, 0xb5, 0xb6, + 0x8e, 0x3e, 0x04, 0x91, 0x92, 0xaa, 0x1a, 0x9c, 0x46, 0x89, 0x5b, 0x30, 0xd6, 0x0d, 0x13, 0x70, + 0x9a, 0xea, 0xb6, 0x75, 0xd4, 0x98, 0xc6, 0x8d, 0x71, 0x6a, 0xff, 0x1a, 0xf7, 0xdc, 0x4e, 0x94, + 0x1d, 0x2f, 0xd1, 0x65, 0xe5, 0x1f, 0x71, 0xb8, 0xbd, 0xad, 0x39, 0x9e, 0x41, 0x88, 0xc3, 0xb0, + 0x0e, 0x43, 0xfe, 0xba, 0x0f, 0x59, 0x6b, 0x3c, 0x64, 0xa7, 0xe2, 0xb2, 0xbd, 0xf8, 0x7b, 0x07, + 0x6b, 0x3c, 0xf4, 0x1d, 0xee, 0x92, 0xa4, 0x34, 0x0d, 0xd7, 0x93, 0x63, 0xd4, 0xa3, 0x6b, 0x33, + 0x3c, 0x3a, 0x7d, 0x8e, 0x6a, 0xc7, 0x70, 0x3d, 0x1e, 0x93, 0x04, 0x05, 0xf5, 0x20, 0xe9, 0x68, + 0xd6, 0x21, 0xa6, 0x41, 0x96, 0x5d, 0x7b, 0x78, 0x3d, 0x38, 0x85, 0x98, 0x72, 0x56, 0xa4, 0x38, + 0xa5, 0x5f, 0x08, 0x90, 0x20, 0xb3, 0x5c, 0x91, 0x07, 0xb7, 0x21, 0x75, 0xac, 0x99, 0x63, 0xec, + 0xd2, 0x3d, 0xe4, 0x14, 0xf6, 0x85, 0x7e, 0x0c, 0x45, 0x77, 0xbc, 0x3f, 0x0a, 0x4d, 0x45, 0xdd, + 0x9b, 0x5d, 0x7b, 0xff, 0x5a, 0xab, 0x0a, 0x6e, 0xaa, 0x28, 0x56, 0xe9, 0x39, 0x24, 0xe9, 0x7a, + 0xaf, 0x58, 0xd9, 0x3d, 0xc8, 0x79, 0xb6, 0x8a, 0x4f, 0x06, 0xe6, 0xd8, 0x35, 0x8e, 0x31, 0x8d, + 0x8e, 0x9c, 0x92, 0xf5, 0xec, 0x16, 0x17, 0xa1, 0xfb, 0x50, 0x38, 0x70, 0xec, 0xa1, 0x6a, 0x58, + 0x5c, 0x89, 0xb2, 0xa3, 0x92, 0x27, 0xd2, 0x36, 0x17, 0x56, 0xfe, 0x23, 0x42, 0x91, 0x46, 0xd0, + 0x5c, 0xcc, 0x70, 0x3f, 0xc4, 0x0c, 0xb7, 0x22, 0xcc, 0x10, 0x84, 0x21, 0x21, 0x86, 0xbb, 0x90, + 0x1a, 0x5b, 0xc6, 0x8b, 0xb1, 0x3f, 0x67, 0x40, 0x7e, 0xbe, 0xec, 0x12, 0x6d, 0x24, 0x2e, 0xd3, + 0xc6, 0x7b, 0x80, 0x48, 0xce, 0x60, 0x35, 0xa2, 0x98, 0xa4, 0x8a, 0x12, 0x1d, 0x69, 0xcc, 0x24, + 0x99, 0xd4, 0x35, 0x48, 0x66, 0x03, 0x24, 0x7c, 0xe2, 0x39, 0x9a, 0x1a, 0xb2, 0x4f, 0x53, 0xfb, + 0xa5, 0xf3, 0xb3, 0xe5, 0x42, 0x8b, 0x8c, 0x4d, 0x07, 0x29, 0xe0, 0xd0, 0x98, 0x4e, 0x62, 0x62, + 0x91, 0x61, 0xe8, 0x86, 0x83, 0xe9, 0x75, 0xeb, 0xca, 0x62, 0x39, 0x7e, 0xc5, 0xf5, 0x7d, 0xc1, + 0xed, 0xd5, 0x26, 0x37, 0x54, 0x24, 0x1f, 0x2a, 0x10, 0xb8, 0xe8, 0x29, 0x64, 0x0f, 0xfc, 0xdb, + 0x5e, 0x7d, 0x8e, 0x4f, 0xe5, 0x0c, 0x0d, 0xb7, 0x77, 0xe6, 0xaf, 0x0b, 0x78, 0x7e, 0x1e, 0x04, + 0x43, 0x68, 0x17, 0xf2, 0x0e, 0x1f, 0xd6, 0xd5, 0xfd, 0x53, 0x7a, 0xff, 0xbc, 0x0c, 0x68, 0x6e, + 0x02, 0x53, 0x3f, 0x45, 0x4f, 0x01, 0x8c, 0x80, 0x25, 0xe9, 0x25, 0x95, 0x5d, 0x7b, 0xf7, 0x1a, + 0x74, 0xca, 0x57, 0x3a, 0x01, 0x41, 0x7b, 0x50, 0x98, 0x7c, 0xd1, 0xa5, 0xe6, 0x5e, 0x72, 0xa9, + 0xf9, 0x10, 0x4e, 0xfd, 0x14, 0xf5, 0xe1, 0x26, 0xb9, 0x3e, 0x6d, 0xd7, 0xf0, 0x70, 0x38, 0x04, + 0xf2, 0x34, 0x04, 0x2a, 0xe7, 0x67, 0xcb, 0xa8, 0xc1, 0xc7, 0xa7, 0x87, 0x01, 0x1a, 0x5c, 0x18, + 0xf7, 0x83, 0x2a, 0x12, 0xbc, 0x04, 0xb1, 0x30, 0x09, 0xaa, 0x9d, 0x49, 0xf8, 0x5e, 0x0a, 0xaa, + 0x50, 0x68, 0x13, 0xa4, 0x3d, 0xc8, 0x45, 0x58, 0xa6, 0xf8, 0xf2, 0x2c, 0x13, 0x01, 0x42, 0x2d, + 0x56, 0x30, 0x49, 0xb4, 0xbe, 0x7c, 0x77, 0xce, 0x00, 0xed, 0x9f, 0x8e, 0xb8, 0x23, 0xa9, 0x79, + 0x65, 0x09, 0x32, 0x41, 0x8c, 0xa2, 0x34, 0xc4, 0x6b, 0x3b, 0x0d, 0xbf, 0x02, 0x6c, 0xb6, 0x76, + 0x1a, 0x92, 0x50, 0xb9, 0x07, 0x09, 0x62, 0x43, 0x2a, 0xc1, 0xf5, 0x9e, 0xb2, 0x57, 0x53, 0x9a, + 0x7e, 0xd5, 0xd9, 0xee, 0x3e, 0x6b, 0x29, 0xfd, 0x56, 0x53, 0x12, 0x2a, 0xbf, 0x8d, 0x03, 0x9a, + 0xd4, 0xfb, 0x7d, 0x9b, 0x55, 0xc0, 0x87, 0x50, 0x1c, 0x04, 0x52, 0x95, 0xae, 0x55, 0x28, 0xc7, + 0x56, 0x0a, 0x6b, 0x8f, 0xfe, 0x67, 0xcf, 0xc0, 0x31, 0xc2, 0xa2, 0xc9, 0xc2, 0x0b, 0x83, 0x88, + 0x34, 0xe0, 0xba, 0x58, 0x39, 0x76, 0x81, 0xeb, 0x14, 0x48, 0x0e, 0x8e, 0xf0, 0xe0, 0x39, 0xe3, + 0xf6, 0xef, 0xce, 0x98, 0x98, 0xde, 0xdd, 0x21, 0x27, 0x35, 0x88, 0xcd, 0x64, 0x6a, 0x7e, 0xe9, + 0x50, 0xa8, 0x8b, 0x69, 0x9c, 0xf8, 0x3f, 0xa4, 0xf1, 0x47, 0xb0, 0x18, 0x82, 0x54, 0xfd, 0x1e, + 0x2d, 0x39, 0xbd, 0x47, 0x2b, 0x4e, 0xec, 0xa8, 0xa8, 0xf2, 0x1e, 0x14, 0xa2, 0x5e, 0x42, 0x19, + 0x48, 0x36, 0x36, 0x5a, 0x8d, 0x27, 0xd2, 0x02, 0x69, 0x01, 0xd6, 0x7b, 0x4a, 0xab, 0xfd, 0xb8, + 0xab, 0x3e, 0x69, 0x7d, 0x2a, 0x09, 0x95, 0x7f, 0x25, 0x00, 0x4d, 0x76, 0xba, 0x35, 0xf6, 0x34, + 0x7a, 0xf0, 0x35, 0x48, 0xf9, 0x91, 0x4e, 0x2f, 0x8c, 0xec, 0xda, 0xb7, 0x66, 0x1e, 0x51, 0xb4, + 0xf0, 0xdf, 0x58, 0x50, 0x98, 0x21, 0xfa, 0x38, 0xdc, 0x5c, 0x66, 0xd7, 0xde, 0x9e, 0x2f, 0x20, + 0x37, 0x16, 0x78, 0xd7, 0xf9, 0x04, 0x92, 0xae, 0x47, 0x5a, 0xb0, 0x38, 0x0d, 0xe8, 0xd5, 0x19, + 0xf6, 0x97, 0x17, 0x5f, 0xdd, 0x21, 0x66, 0xfc, 0x90, 0x28, 0x06, 0xda, 0x83, 0x4c, 0xc0, 0xe1, + 0xac, 0x53, 0x7d, 0x38, 0x3f, 0x60, 0x90, 0x10, 0xbc, 0x1c, 0x0c, 0xb0, 0x50, 0x0d, 0xb2, 0x43, + 0xa6, 0x36, 0x29, 0x66, 0xcb, 0xec, 0x1a, 0x05, 0x8e, 0x40, 0xaf, 0xd3, 0xd0, 0x97, 0x02, 0xdc, + 0xa8, 0xad, 0x93, 0xde, 0xc4, 0xb1, 0x4d, 0x73, 0x5f, 0x1b, 0x3c, 0xa7, 0xed, 0x66, 0xd0, 0x9b, + 0x70, 0x29, 0x7a, 0x42, 0x2e, 0x43, 0x7e, 0xa4, 0xb4, 0x81, 0xcc, 0xce, 0xd1, 0x68, 0xf3, 0xa4, + 0xd9, 0x58, 0x50, 0x42, 0xe6, 0x95, 0x1f, 0x42, 0x92, 0x3a, 0x88, 0x64, 0xf0, 0x6e, 0xf7, 0x49, + 0xb7, 0xb7, 0xd7, 0xf5, 0x03, 0xa3, 0xd9, 0xea, 0xb4, 0xfa, 0x2d, 0xb5, 0xd7, 0xed, 0x7c, 0x2a, + 0x09, 0xe8, 0x0e, 0xdc, 0x62, 0x82, 0x5a, 0xb7, 0xa9, 0xee, 0x29, 0x6d, 0x3e, 0x14, 0xab, 0xac, + 0x84, 0x29, 0x42, 0x84, 0x44, 0xb7, 0xd7, 0x25, 0x5d, 0x22, 0x21, 0x8b, 0x66, 0x53, 0x12, 0x28, + 0x59, 0x28, 0xbd, 0x6d, 0x29, 0x56, 0xcf, 0x01, 0xe8, 0x81, 0x3b, 0x37, 0x13, 0x62, 0x4a, 0x4a, + 0x57, 0x7e, 0xf9, 0x3a, 0x14, 0x2f, 0x24, 0xd8, 0x15, 0xd5, 0x49, 0x99, 0x56, 0x27, 0x7e, 0xa5, + 0x2b, 0x45, 0xaa, 0x93, 0x18, 0x2b, 0x4c, 0x1e, 0x42, 0x66, 0xa4, 0x39, 0xd8, 0xf2, 0x88, 0xff, + 0x13, 0x91, 0x06, 0x47, 0xdc, 0xa6, 0x03, 0x81, 0xba, 0xe8, 0x2b, 0xb6, 0x89, 0x51, 0xfa, 0x18, + 0x3b, 0x2e, 0x89, 0x06, 0xff, 0xc8, 0xee, 0xb0, 0xbc, 0x5a, 0x9c, 0xac, 0xea, 0x99, 0xaf, 0xa0, + 0x70, 0x4d, 0xb4, 0x0d, 0x8b, 0x43, 0x5b, 0x37, 0x0e, 0x8c, 0x81, 0x7f, 0xde, 0x9e, 0x31, 0xf4, + 0x1f, 0x08, 0xb2, 0x6b, 0x6f, 0x84, 0x4e, 0x63, 0xec, 0x19, 0x66, 0xf5, 0xc8, 0x1c, 0x54, 0xfb, + 0xfc, 0x6d, 0x87, 0xed, 0x48, 0x0a, 0x5b, 0x93, 0x41, 0xf4, 0x18, 0xd2, 0xbc, 0xe4, 0x16, 0xe9, + 0xf5, 0x37, 0x6f, 0x9e, 0x31, 0x44, 0x6e, 0x8d, 0xd6, 0xa1, 0x60, 0xe1, 0x93, 0x70, 0x5b, 0x95, + 0x89, 0x44, 0x62, 0xae, 0x8b, 0x4f, 0xa6, 0xf7, 0x54, 0x39, 0x6b, 0x32, 0xa2, 0xa3, 0xa7, 0x90, + 0x1f, 0x39, 0xc6, 0x50, 0x73, 0x38, 0xeb, 0xc0, 0x75, 0x92, 0x37, 0xb8, 0x97, 0x7c, 0x08, 0x3a, + 0x8a, 0xd6, 0xc1, 0xef, 0x62, 0xb0, 0x2b, 0x67, 0xe9, 0x1e, 0xaf, 0x07, 0xc6, 0x8d, 0x51, 0x1d, + 0xf2, 0x74, 0x8b, 0x41, 0xfb, 0x94, 0xa3, 0x3b, 0x5c, 0x62, 0x3b, 0xcc, 0x92, 0x1d, 0x4e, 0x69, + 0xa1, 0xb2, 0x56, 0x20, 0xd7, 0xd1, 0x26, 0x40, 0xf0, 0xa6, 0x46, 0x4a, 0x82, 0xab, 0xa8, 0x7a, + 0x9b, 0x2b, 0x4e, 0x96, 0xa4, 0x84, 0xac, 0xd1, 0x16, 0x64, 0x78, 0x12, 0xfb, 0xb5, 0xc0, 0xec, + 0x9c, 0xbc, 0x4c, 0x29, 0x9c, 0x48, 0x02, 0x04, 0xd4, 0x85, 0xa4, 0x89, 0x35, 0x17, 0xb3, 0x82, + 0xe0, 0xd1, 0x9c, 0x57, 0xd3, 0xce, 0xe0, 0x08, 0x0f, 0xb5, 0xc6, 0x11, 0x69, 0x2e, 0x3a, 0xc4, + 0x5e, 0xf1, 0x61, 0x50, 0x17, 0x24, 0xea, 0xae, 0x30, 0x3b, 0x49, 0xd4, 0x63, 0x6f, 0x31, 0x8f, + 0x15, 0x88, 0xc7, 0x66, 0x32, 0x14, 0x8d, 0xa7, 0xad, 0x09, 0x4b, 0xfd, 0x00, 0x0a, 0x07, 0xb6, + 0x33, 0xd4, 0x3c, 0x95, 0x27, 0xce, 0xe2, 0xa4, 0x65, 0xf8, 0xe6, 0x6c, 0x39, 0xbf, 0x4e, 0x47, + 0x79, 0xd2, 0xe4, 0x0f, 0xc2, 0x9f, 0x68, 0x83, 0x93, 0xf9, 0x0d, 0xca, 0xbd, 0xef, 0xcd, 0xbb, + 0xbb, 0xcb, 0x4c, 0xde, 0x85, 0x14, 0xbd, 0x77, 0x5d, 0xf9, 0x26, 0xf5, 0xf9, 0x4b, 0xde, 0xe1, + 0x0a, 0x43, 0x41, 0x9f, 0x41, 0x41, 0x27, 0x12, 0xc3, 0x3a, 0x64, 0x2d, 0xc9, 0x2d, 0x8a, 0xbb, + 0x3a, 0x27, 0x2e, 0x69, 0x57, 0xda, 0xd6, 0x81, 0xcd, 0xab, 0x51, 0x0e, 0xe6, 0xb7, 0x31, 0x3d, + 0x10, 0x0f, 0xb4, 0xa1, 0x61, 0x1a, 0xd8, 0x95, 0x6f, 0x53, 0xdc, 0xf7, 0xaf, 0xcc, 0xf0, 0x8b, + 0x2f, 0x3a, 0xfc, 0x2a, 0xe0, 0x20, 0x41, 0xa2, 0x53, 0xc1, 0x29, 0x39, 0xd4, 0xd7, 0x2e, 0x27, + 0x3a, 0x7f, 0xd1, 0x89, 0xbc, 0xee, 0xd0, 0x44, 0x67, 0x5f, 0x3a, 0x7a, 0x13, 0xe0, 0xd8, 0xc0, + 0x3f, 0x55, 0x5f, 0x8c, 0xb1, 0x73, 0x2a, 0xcb, 0x21, 0xde, 0xcd, 0x10, 0xf9, 0x53, 0x22, 0x46, + 0x1f, 0x40, 0x46, 0xc7, 0x23, 0x6c, 0xe9, 0x6e, 0xcf, 0x92, 0xef, 0xd0, 0x72, 0xf7, 0x06, 0xe9, + 0xc1, 0x9a, 0x5c, 0xc8, 0x78, 0x75, 0xa2, 0x85, 0x3e, 0x87, 0x9c, 0xff, 0x81, 0xf5, 0x9e, 0x55, + 0x3f, 0x95, 0x4b, 0x74, 0xd3, 0x0f, 0xe6, 0x74, 0xe6, 0xa4, 0x28, 0xba, 0xc9, 0xf7, 0xd3, 0x0c, + 0xa1, 0x29, 0x11, 0x6c, 0xf4, 0x19, 0xe4, 0x78, 0x74, 0x6f, 0xda, 0xfb, 0xae, 0xfc, 0xfa, 0x95, + 0xaf, 0x12, 0x17, 0xe7, 0xda, 0x9a, 0x98, 0x72, 0xde, 0x0a, 0xa3, 0xa1, 0x4f, 0x20, 0x1f, 0x3c, + 0xe5, 0xd9, 0x23, 0xcf, 0x95, 0xef, 0xd2, 0xc4, 0x7c, 0x38, 0x6f, 0xe8, 0x32, 0xdb, 0xde, 0xc8, + 0x73, 0x95, 0x9c, 0x1b, 0xfa, 0x42, 0xf7, 0x20, 0xa3, 0x3b, 0xf6, 0xc8, 0xbf, 0x3f, 0xde, 0x28, + 0x0b, 0x2b, 0x71, 0x7e, 0xcc, 0x44, 0x4c, 0x2f, 0x06, 0x15, 0x0a, 0x0e, 0x1e, 0x99, 0xda, 0x00, + 0x0f, 0xc9, 0xcd, 0x66, 0x1f, 0xc8, 0x4b, 0x74, 0xf6, 0xb5, 0xb9, 0x1d, 0x19, 0x18, 0xf3, 0xc0, + 0x0c, 0xe1, 0xf5, 0x0e, 0xd0, 0x2e, 0x80, 0x36, 0xd6, 0x0d, 0x4f, 0x1d, 0xda, 0x3a, 0x96, 0x97, + 0xaf, 0x7c, 0x93, 0xbe, 0x08, 0x5e, 0x23, 0x86, 0x5b, 0xb6, 0x8e, 0x83, 0xd7, 0x31, 0x2e, 0x40, + 0x1f, 0x40, 0x96, 0x6e, 0xed, 0x73, 0x7b, 0x9f, 0xc4, 0x66, 0x99, 0x6e, 0x6e, 0x91, 0x9d, 0x65, + 0xa6, 0xe9, 0xd8, 0xa3, 0x4d, 0x7b, 0x9f, 0x46, 0x0c, 0xfb, 0xa9, 0x23, 0x17, 0x72, 0x87, 0x03, + 0x75, 0x42, 0xa5, 0xf7, 0xe8, 0x29, 0x7e, 0x34, 0xe7, 0x5a, 0x1e, 0x37, 0xa6, 0x90, 0xeb, 0x0d, + 0x7e, 0x27, 0x3c, 0x6e, 0x70, 0x99, 0xab, 0x64, 0x0f, 0x07, 0xc1, 0x47, 0xe9, 0x2b, 0x01, 0x16, + 0x2f, 0x51, 0x27, 0xfa, 0x09, 0xa4, 0x2d, 0x5b, 0x0f, 0xbd, 0xe6, 0xb5, 0x18, 0x50, 0xaa, 0x6b, + 0xeb, 0xfe, 0x63, 0xde, 0xc3, 0xb9, 0x1e, 0xa0, 0xe9, 0xaf, 0xd1, 0x7e, 0xd5, 0x37, 0x53, 0x52, + 0x04, 0xb5, 0xad, 0xa3, 0xf7, 0xa1, 0x88, 0x4f, 0x46, 0x86, 0x13, 0x2a, 0x1f, 0x62, 0xa1, 0xe3, + 0x2f, 0x4c, 0x06, 0x49, 0x10, 0x94, 0xfe, 0x24, 0x40, 0xf1, 0x02, 0x6d, 0x91, 0x4a, 0x89, 0xbe, + 0x14, 0x47, 0x2a, 0x25, 0x22, 0x09, 0x75, 0x3d, 0x57, 0xfd, 0x4b, 0x13, 0x7f, 0xd5, 0x7f, 0x69, + 0xa2, 0x0f, 0x33, 0xc9, 0xf9, 0x1f, 0x66, 0x36, 0x13, 0x62, 0x42, 0x4a, 0x96, 0x3e, 0x05, 0x91, + 0x53, 0x66, 0xb4, 0x74, 0x13, 0xe6, 0x2c, 0xdd, 0x66, 0xee, 0xb3, 0xf4, 0xa5, 0x00, 0x99, 0xf0, + 0xdf, 0x5f, 0xb1, 0x00, 0x75, 0x7a, 0xe5, 0xf8, 0x92, 0x6f, 0xb1, 0x51, 0x0f, 0xc4, 0xe7, 0xf7, + 0x40, 0xe9, 0x18, 0xb2, 0x21, 0xd6, 0xb9, 0xd8, 0x3b, 0x08, 0x2f, 0xd1, 0x3b, 0xbc, 0x05, 0x29, + 0x96, 0x6a, 0x7e, 0x20, 0xe5, 0x99, 0x75, 0xd2, 0x4f, 0xb3, 0xe4, 0xe7, 0x24, 0xc5, 0x4a, 0xbf, + 0x17, 0x20, 0x17, 0xe6, 0x23, 0x54, 0x81, 0x8c, 0x61, 0x0d, 0x1c, 0x4a, 0x06, 0x74, 0x5e, 0x1e, + 0x82, 0x13, 0x31, 0x61, 0xa9, 0xa1, 0x61, 0xa9, 0xf4, 0x7d, 0x34, 0x12, 0xa6, 0xe2, 0xd0, 0xb0, + 0x9e, 0x11, 0x29, 0x55, 0xd1, 0x4e, 0x98, 0x4a, 0x3c, 0xa2, 0xa2, 0x9d, 0xf8, 0x2a, 0x25, 0x7a, + 0xf1, 0x3b, 0x1e, 0xad, 0xcc, 0xe3, 0xa1, 0xab, 0xdc, 0xf1, 0xd0, 0x12, 0xa4, 0x8f, 0x0d, 0xc7, + 0x1b, 0x6b, 0x26, 0x2d, 0xc2, 0x79, 0xdf, 0xc3, 0x85, 0xa5, 0x23, 0xc8, 0x86, 0x78, 0x6c, 0x8e, + 0x03, 0xfd, 0x1e, 0x24, 0x82, 0xa4, 0x9a, 0xb3, 0x26, 0xa7, 0x06, 0xa5, 0x9f, 0x0b, 0x70, 0x73, + 0x1a, 0x93, 0x44, 0x42, 0xc4, 0xf7, 0xd3, 0x5c, 0x21, 0x12, 0x61, 0xf8, 0xd8, 0x54, 0x86, 0x9f, + 0x9c, 0x5c, 0x7c, 0xf6, 0xc9, 0x55, 0x3e, 0xe4, 0xcd, 0x1a, 0x40, 0x6a, 0x7b, 0xb7, 0xde, 0x69, + 0x37, 0xa6, 0x36, 0x5a, 0x28, 0x0f, 0x99, 0xf6, 0xd6, 0x76, 0x4f, 0xe9, 0xb7, 0xbb, 0x8f, 0xa5, + 0x38, 0xe9, 0xd0, 0x02, 0x92, 0x46, 0x39, 0x10, 0x9b, 0xed, 0x9d, 0x5a, 0xbd, 0xd3, 0x6a, 0x4a, + 0x0b, 0x44, 0x53, 0x69, 0xd5, 0x9a, 0xb4, 0xa1, 0x93, 0x84, 0xef, 0x27, 0xbe, 0xf8, 0xd5, 0xb2, + 0xe0, 0x77, 0x66, 0x9b, 0x09, 0x11, 0x49, 0x37, 0x2a, 0x5f, 0x09, 0x80, 0x9a, 0x9a, 0xa7, 0x11, + 0x42, 0xb8, 0x46, 0x8b, 0x16, 0xbb, 0xe2, 0x5c, 0xa2, 0x65, 0x77, 0xfc, 0x55, 0xca, 0x6e, 0x7f, + 0xc1, 0x95, 0x2f, 0x05, 0x80, 0xd0, 0xe2, 0x3e, 0x0e, 0xff, 0xd5, 0x3d, 0xbb, 0xc3, 0xb8, 0x70, + 0x79, 0x6c, 0x2c, 0xf0, 0x3f, 0xc2, 0x1f, 0x83, 0xa8, 0xb3, 0x2d, 0xb3, 0xe0, 0x99, 0x59, 0xca, + 0x5f, 0xf2, 0xcc, 0x06, 0x39, 0x55, 0x26, 0xad, 0xa7, 0x21, 0x39, 0xb6, 0x0c, 0xdb, 0x7a, 0xa7, + 0x19, 0x7e, 0x02, 0xe3, 0x64, 0x4a, 0x9c, 0x4f, 0x7f, 0x6b, 0x1e, 0xd6, 0xfd, 0xa6, 0x7b, 0xd7, + 0x3a, 0x0e, 0x04, 0x02, 0x2a, 0x00, 0xb0, 0x71, 0xc3, 0x3a, 0x94, 0x62, 0xf5, 0x7b, 0x5f, 0xff, + 0x6d, 0x69, 0xe1, 0xeb, 0xf3, 0x25, 0xe1, 0x8f, 0xe7, 0x4b, 0xc2, 0x9f, 0xcf, 0x97, 0x84, 0xbf, + 0x9e, 0x2f, 0x09, 0x3f, 0xfb, 0xfb, 0xd2, 0xc2, 0x8f, 0xd2, 0x6c, 0x41, 0xff, 0x0d, 0x00, 0x00, + 0xff, 0xff, 0xb9, 0x1f, 0xc8, 0x03, 0x80, 0x20, 0x00, 0x00, } diff --git a/pkg/sql/sqlbase/structured.proto b/pkg/sql/sqlbase/structured.proto index 5858f99820da..e249467efd02 100644 --- a/pkg/sql/sqlbase/structured.proto +++ b/pkg/sql/sqlbase/structured.proto @@ -341,10 +341,13 @@ message IndexDescriptor { message ConstraintToUpdate { enum ConstraintType { CHECK = 0; + FOREIGN_KEY = 1; } required ConstraintType constraint_type = 1 [(gogoproto.nullable) = false]; required string name = 2 [(gogoproto.nullable) = false]; optional TableDescriptor.CheckConstraint check = 3 [(gogoproto.nullable) = false]; + optional ForeignKeyReference foreign_key = 4 [(gogoproto.nullable) = false]; + optional uint32 foreign_key_index = 5 [(gogoproto.nullable) = false, (gogoproto.casttype) = "IndexID"]; } // A DescriptorMutation represents a column or an index that diff --git a/pkg/sql/sqlbase/structured_test.go b/pkg/sql/sqlbase/structured_test.go index eae7fe7e482a..9d9bb6b843ab 100644 --- a/pkg/sql/sqlbase/structured_test.go +++ b/pkg/sql/sqlbase/structured_test.go @@ -1261,7 +1261,7 @@ func TestUnvalidateConstraints(t *testing.T) { t.Fatal(err) } if c, ok := after["fk"]; !ok || !c.Unvalidated { - t.Fatalf("expected to find a unvalididated constraint fk before, found %v", c) + t.Fatalf("expected to find an unvalidated constraint fk before, found %v", c) } } diff --git a/pkg/sql/sqlbase/table.go b/pkg/sql/sqlbase/table.go index a3342c5e121d..f7655eafa2fd 100644 --- a/pkg/sql/sqlbase/table.go +++ b/pkg/sql/sqlbase/table.go @@ -351,41 +351,50 @@ func (desc *TableDescriptor) collectConstraintInfo( detail.Index = index info[index.Name] = detail } + } - if index.ForeignKey.IsSet() { - if _, ok := info[index.ForeignKey.Name]; ok { - return nil, pgerror.Newf(pgerror.CodeDuplicateObjectError, - "duplicate constraint name: %q", index.ForeignKey.Name) - } - detail := ConstraintDetail{Kind: ConstraintTypeFK} - detail.Unvalidated = index.ForeignKey.Validity == ConstraintValidity_Unvalidated - numCols := len(index.ColumnIDs) - if index.ForeignKey.SharedPrefixLen > 0 { - numCols = int(index.ForeignKey.SharedPrefixLen) - } - detail.Columns = index.ColumnNames[:numCols] - detail.Index = index - detail.FK = &index.ForeignKey + fks, err := desc.AllActiveAndInactiveForeignKeys() + if err != nil { + return nil, err + } + for id, fk := range fks { + idx, err := desc.FindIndexByID(id) + if err != nil { + return nil, err + } + if _, ok := info[fk.Name]; ok { + return nil, pgerror.Newf(pgerror.CodeDuplicateObjectError, + "duplicate constraint name: %q", fk.Name) + } + detail := ConstraintDetail{Kind: ConstraintTypeFK} + // Constraints in the Validating state are considered Unvalidated for this purpose + detail.Unvalidated = fk.Validity != ConstraintValidity_Validated + numCols := len(idx.ColumnIDs) + if fk.SharedPrefixLen > 0 { + numCols = int(fk.SharedPrefixLen) + } + detail.Columns = idx.ColumnNames[:numCols] + detail.Index = idx + detail.FK = fk - if tableLookup != nil { - other, err := tableLookup(index.ForeignKey.Table) - if err != nil { - return nil, pgerror.NewAssertionErrorWithWrappedErrf(err, - "error resolving table %d referenced in foreign key", - log.Safe(index.ForeignKey.Table)) - } - otherIdx, err := other.FindIndexByID(index.ForeignKey.Index) - if err != nil { - return nil, pgerror.NewAssertionErrorWithWrappedErrf(err, - "error resolving index %d in table %s referenced in foreign key", - log.Safe(index.ForeignKey.Index), other.Name) - } - detail.Details = fmt.Sprintf("%s.%v", other.Name, otherIdx.ColumnNames) - detail.ReferencedTable = other - detail.ReferencedIndex = otherIdx + if tableLookup != nil { + other, err := tableLookup(fk.Table) + if err != nil { + return nil, pgerror.NewAssertionErrorWithWrappedErrf(err, + "error resolving table %d referenced in foreign key", + log.Safe(fk.Table)) + } + otherIdx, err := other.FindIndexByID(fk.Index) + if err != nil { + return nil, pgerror.NewAssertionErrorWithWrappedErrf(err, + "error resolving index %d in table %s referenced in foreign key", + log.Safe(fk.Index), other.Name) } - info[index.ForeignKey.Name] = detail + detail.Details = fmt.Sprintf("%s.%v", other.Name, otherIdx.ColumnNames) + detail.ReferencedTable = other + detail.ReferencedIndex = otherIdx } + info[fk.Name] = detail } for _, c := range desc.AllActiveAndInactiveChecks() {