From 45159232962b48253fbf8cb9292b6811afe7ddcb Mon Sep 17 00:00:00 2001 From: Faizan Qazi Date: Mon, 7 Jun 2021 14:43:59 -0400 Subject: [PATCH] sql: clean up drop table / sequence inside new schema changer Previous, drop sequence inside the new schema changer used to refer to descriptor ID's as table_id's when it would be cleaner to refer to sequence ID. Also clean up the code for drop table at the same time. This isn't a functional change and only cleans up the code to be more readable. These changes also removed things from schema change execution interfaces that shouldn't be shared across boundaries for schema change execution. Release note: None. --- pkg/sql/schemachanger/scbuild/sequence.go | 20 +- pkg/sql/schemachanger/scbuild/table.go | 306 ++++++++++-------- .../scbuild/testdata/drop_sequence | 16 +- .../schemachanger/scbuild/testdata/drop_table | 4 +- .../scexec/scmutationexec/scmutationexec.go | 4 - pkg/sql/schemachanger/scpb/elements.go | 8 +- pkg/sql/schemachanger/scpb/scpb.pb.go | 242 +++++++------- pkg/sql/schemachanger/scpb/scpb.proto | 4 +- pkg/sql/schemachanger/scplan/rules.go | 20 +- 9 files changed, 319 insertions(+), 305 deletions(-) diff --git a/pkg/sql/schemachanger/scbuild/sequence.go b/pkg/sql/schemachanger/scbuild/sequence.go index 8d62e7b17cb1..cde6b566c7e8 100644 --- a/pkg/sql/schemachanger/scbuild/sequence.go +++ b/pkg/sql/schemachanger/scbuild/sequence.go @@ -25,15 +25,15 @@ import ( // dropSequenceDesc builds targets and transformations using a descriptor. func (b *buildContext) dropSequenceDesc( - ctx context.Context, table catalog.TableDescriptor, cascade tree.DropBehavior, + ctx context.Context, seq catalog.TableDescriptor, cascade tree.DropBehavior, ) { // Check if there are dependencies. - err := table.ForeachDependedOnBy(func(dep *descpb.TableDescriptor_Reference) error { + err := seq.ForeachDependedOnBy(func(dep *descpb.TableDescriptor_Reference) error { if cascade != tree.DropCascade { return pgerror.Newf( pgcode.DependentObjectsStillExist, "cannot drop sequence %s because other objects depend on it", - table.GetName(), + seq.GetName(), ) } desc, err := b.Descs.GetImmutableTableByID(ctx, b.EvalCtx.Txn, dep.ID, tree.ObjectLookupFlagsWithRequired()) @@ -62,15 +62,17 @@ func (b *buildContext) dropSequenceDesc( } // Add a node to drop the sequence - sequenceNode := &scpb.Sequence{TableID: table.GetID()} + sequenceNode := &scpb.Sequence{SequenceID: seq.GetID()} if exists, _ := b.checkIfNodeExists(scpb.Target_DROP, sequenceNode); !exists { b.addNode(scpb.Target_DROP, sequenceNode) } - sequenceOwnedBy := &scpb.SequenceOwnedBy{ - TableID: table.GetID(), - OwnerTableID: table.GetSequenceOpts().SequenceOwner.OwnerTableID} - if exists, _ := b.checkIfNodeExists(scpb.Target_DROP, sequenceOwnedBy); !exists { - b.addNode(scpb.Target_DROP, sequenceOwnedBy) + if seq.GetSequenceOpts().SequenceOwner.OwnerTableID != descpb.InvalidID { + sequenceOwnedBy := &scpb.SequenceOwnedBy{ + SequenceID: seq.GetID(), + OwnerTableID: seq.GetSequenceOpts().SequenceOwner.OwnerTableID} + if exists, _ := b.checkIfNodeExists(scpb.Target_DROP, sequenceOwnedBy); !exists { + b.addNode(scpb.Target_DROP, sequenceOwnedBy) + } } } diff --git a/pkg/sql/schemachanger/scbuild/table.go b/pkg/sql/schemachanger/scbuild/table.go index ca6decb430e9..7f243593b765 100644 --- a/pkg/sql/schemachanger/scbuild/table.go +++ b/pkg/sql/schemachanger/scbuild/table.go @@ -574,160 +574,188 @@ func isTypeSupportedInVersion(v clusterversion.ClusterVersion, t *types.T) bool return v.IsActive(minVersion) } -func (b *buildContext) dropTable(ctx context.Context, n *tree.DropTable) { - // Find the table first. - for _, name := range n.Names { - table, err := resolver.ResolveExistingTableObject(ctx, b.Res, &name, - tree.ObjectLookupFlagsWithRequired()) - if err != nil { - if errors.Is(err, catalog.ErrDescriptorNotFound) && n.IfExists { - return - } - panic(err) - } - if table == nil { - panic(errors.AssertionFailedf("Unable to resolve table %s", - name.FQString())) - } - // Interleaved tables not supported in new schema changer. - if table.IsInterleaved() { - panic(¬ImplementedError{n: n, detail: "drop on interleaved table"}) - } - - // Drop dependent views - err = table.ForeachDependedOnBy(func(dep *descpb.TableDescriptor_Reference) error { - dependentDesc, err := b.Descs.GetImmutableTableByID(ctx, b.EvalCtx.Txn, dep.ID, tree.ObjectLookupFlagsWithRequired()) +func (b *buildContext) maybeCleanTableSequenceRefs( + ctx context.Context, table catalog.TableDescriptor, behavior tree.DropBehavior, +) { + // Setup nodes for dropping sequences + // and cleaning up default expressions. + for _, col := range table.PublicColumns() { + // Loop over owned sequences + for seqIdx := 0; seqIdx < col.NumOwnsSequences(); seqIdx++ { + seqID := col.GetOwnsSequenceID(seqIdx) + table, err := b.Descs.GetMutableTableByID(ctx, b.EvalCtx.Txn, seqID, tree.ObjectLookupFlagsWithRequiredTableKind(tree.ResolveRequireSequenceDesc)) if err != nil { panic(err) } - if n.DropBehavior != tree.DropCascade { - return pgerror.Newf( - pgcode.DependentObjectsStillExist, "cannot drop table %q because view %q depends on it", - table.GetName(), dependentDesc.GetName()) + if behavior != tree.DropCascade { + panic(pgerror.Newf( + pgcode.DependentObjectsStillExist, + "cannot drop table %s because other objects depend on it", + table.GetName(), + )) } - err = b.AuthAccessor.CheckPrivilege(ctx, dependentDesc, privilege.DROP) + err = b.AuthAccessor.CheckPrivilege(ctx, table, privilege.DROP) if err != nil { panic(err) } - b.maybeDropViewAndDependents(ctx, dependentDesc, n.DropBehavior) - return nil - }) + b.dropSequenceDesc(ctx, table, tree.DropCascade) + } + // Setup logic to clean up the default expression, + // only if sequences are depending on it. + if col.NumUsesSequences() > 0 { + b.addNode(scpb.Target_DROP, + &scpb.DefaultExpression{ + DefaultExpr: col.GetDefaultExpr(), + TableID: table.GetID(), + UsesSequenceIDs: col.ColumnDesc().UsesSequenceIds, + ColumnID: col.GetID()}) + // Drop the depends on within the sequence side. + for seqOrd := 0; seqOrd < col.NumUsesSequences(); seqOrd++ { + seqID := col.GetUsesSequenceID(seqOrd) + // Remove dependencies to this sequences. + dropDep := &scpb.RelationDependedOnBy{TableID: seqID, + DependedOnBy: table.GetID()} + if exists, _ := b.checkIfNodeExists(scpb.Target_DROP, dropDep); !exists { + b.addNode(scpb.Target_DROP, dropDep) + } + } + } + } +} + +func (b *buildContext) maybeCleanTableFKs( + ctx context.Context, table catalog.TableDescriptor, behavior tree.DropBehavior, +) { + // Loop through and update inbound and outbound + // foreign key references. + for _, fk := range table.GetInboundFKs() { + dependentTable, err := b.Descs.GetImmutableTableByID(ctx, b.EvalCtx.Txn, fk.OriginTableID, tree.ObjectLookupFlagsWithRequired()) if err != nil { panic(err) } + if behavior != tree.DropCascade { + panic(pgerror.Newf( + pgcode.DependentObjectsStillExist, + "%q is referenced by foreign key from table %q", fk.Name, dependentTable.GetName())) + } + err = b.AuthAccessor.CheckPrivilege(ctx, dependentTable, privilege.DROP) + if err != nil { + panic(err) + } + outFkNode := &scpb.OutboundForeignKey{ + OriginID: fk.OriginTableID, + OriginColumns: fk.OriginColumnIDs, + ReferenceID: fk.ReferencedTableID, + ReferenceColumns: fk.ReferencedColumnIDs, + Name: fk.Name, + } + inFkNode := &scpb.InboundForeignKey{ + OriginID: fk.ReferencedTableID, + OriginColumns: fk.ReferencedColumnIDs, + ReferenceID: fk.OriginTableID, + ReferenceColumns: fk.OriginColumnIDs, + Name: fk.Name, + } + if exists, _ := b.checkIfNodeExists(scpb.Target_DROP, outFkNode); !exists { + b.addNode(scpb.Target_DROP, + outFkNode) + } + if exists, _ := b.checkIfNodeExists(scpb.Target_DROP, inFkNode); !exists { + b.addNode(scpb.Target_DROP, + inFkNode) + } + } - // Loop through and update inbound and outbound - // foreign key references. - for _, fk := range table.GetInboundFKs() { - dependentTable, err := b.Descs.GetImmutableTableByID(ctx, b.EvalCtx.Txn, fk.OriginTableID, tree.ObjectLookupFlagsWithRequired()) - if err != nil { - panic(err) - } - if n.DropBehavior != tree.DropCascade { - panic(pgerror.Newf( - pgcode.DependentObjectsStillExist, - "%q is referenced by foreign key from table %q", fk.Name, dependentTable.GetName())) - } - err = b.AuthAccessor.CheckPrivilege(ctx, dependentTable, privilege.DROP) - if err != nil { - panic(err) - } - outFkNode := &scpb.OutboundForeignKey{ - OriginID: fk.OriginTableID, - OriginColumns: fk.OriginColumnIDs, - ReferenceID: fk.ReferencedTableID, - ReferenceColumns: fk.ReferencedColumnIDs, - Name: fk.Name, - } - inFkNode := &scpb.InboundForeignKey{ - OriginID: fk.ReferencedTableID, - OriginColumns: fk.ReferencedColumnIDs, - ReferenceID: fk.OriginTableID, - ReferenceColumns: fk.OriginColumnIDs, - Name: fk.Name, - } - if exists, _ := b.checkIfNodeExists(scpb.Target_DROP, outFkNode); !exists { - b.addNode(scpb.Target_DROP, - outFkNode) - } - if exists, _ := b.checkIfNodeExists(scpb.Target_DROP, inFkNode); !exists { - b.addNode(scpb.Target_DROP, - inFkNode) - } + for _, fk := range table.GetOutboundFKs() { + outFkNode := &scpb.OutboundForeignKey{ + OriginID: fk.OriginTableID, + OriginColumns: fk.OriginColumnIDs, + ReferenceID: fk.ReferencedTableID, + ReferenceColumns: fk.ReferencedColumnIDs, + Name: fk.Name, + } + inFkNode := &scpb.InboundForeignKey{ + OriginID: fk.ReferencedTableID, + OriginColumns: fk.ReferencedColumnIDs, + ReferenceID: fk.OriginTableID, + ReferenceColumns: fk.OriginColumnIDs, + Name: fk.Name, + } + if exists, _ := b.checkIfNodeExists(scpb.Target_DROP, outFkNode); !exists { + b.addNode(scpb.Target_DROP, + outFkNode) } + if exists, _ := b.checkIfNodeExists(scpb.Target_DROP, inFkNode); !exists { + b.addNode(scpb.Target_DROP, + inFkNode) + } + } +} - for _, fk := range table.GetOutboundFKs() { - outFkNode := &scpb.OutboundForeignKey{ - OriginID: fk.OriginTableID, - OriginColumns: fk.OriginColumnIDs, - ReferenceID: fk.ReferencedTableID, - ReferenceColumns: fk.ReferencedColumnIDs, - Name: fk.Name, - } - inFkNode := &scpb.InboundForeignKey{ - OriginID: fk.ReferencedTableID, - OriginColumns: fk.ReferencedColumnIDs, - ReferenceID: fk.OriginTableID, - ReferenceColumns: fk.OriginColumnIDs, - Name: fk.Name, - } - if exists, _ := b.checkIfNodeExists(scpb.Target_DROP, outFkNode); !exists { - b.addNode(scpb.Target_DROP, - outFkNode) - } - if exists, _ := b.checkIfNodeExists(scpb.Target_DROP, inFkNode); !exists { - b.addNode(scpb.Target_DROP, - inFkNode) - } +func (b *buildContext) dropTableDesc( + ctx context.Context, table catalog.TableDescriptor, behavior tree.DropBehavior, +) { + // Interleaved tables not supported in new schema changer. + if table.IsInterleaved() { + panic(¬ImplementedError{ + n: &tree.DropTable{ + Names: []tree.TableName{ + tree.MakeUnqualifiedTableName(tree.Name(table.GetName())), + }, + }, + detail: "drop on interleaved table"}) + } + + // Drop dependent views + err := table.ForeachDependedOnBy(func(dep *descpb.TableDescriptor_Reference) error { + dependentDesc, err := b.Descs.GetImmutableTableByID(ctx, b.EvalCtx.Txn, dep.ID, tree.ObjectLookupFlagsWithRequired()) + if err != nil { + panic(err) } - // Setup nodes for dropping sequences - // and cleaning up default expressions. - for _, col := range table.PublicColumns() { - // Loop over owned sequences - for seqIdx := 0; seqIdx < col.NumOwnsSequences(); seqIdx++ { - seqID := col.GetOwnsSequenceID(seqIdx) - table, err := b.Descs.GetMutableTableByID(ctx, b.EvalCtx.Txn, seqID, tree.ObjectLookupFlagsWithRequiredTableKind(tree.ResolveRequireSequenceDesc)) - if err != nil { - panic(err) - } - if n.DropBehavior != tree.DropCascade { - panic(pgerror.Newf( - pgcode.DependentObjectsStillExist, - "cannot drop table %s because other objects depend on it", - table.GetName(), - )) - } - err = b.AuthAccessor.CheckPrivilege(ctx, table, privilege.DROP) - if err != nil { - panic(err) - } - b.dropSequenceDesc(ctx, table, tree.DropCascade) - } - // Setup logic to clean up the default expression, - // only if sequences are depending on it. - if col.NumUsesSequences() > 0 { - b.addNode(scpb.Target_DROP, - &scpb.DefaultExpression{ - DefaultExpr: col.GetDefaultExpr(), - TableID: table.GetID(), - UsesSequenceIDs: col.ColumnDesc().UsesSequenceIds, - ColumnID: col.GetID()}) - // Drop the depends on within the sequence side. - for seqOrd := 0; seqOrd < col.NumUsesSequences(); seqOrd++ { - seqID := col.GetUsesSequenceID(seqOrd) - // Remove dependencies to this sequences. - dropDep := &scpb.RelationDependedOnBy{TableID: seqID, - DependedOnBy: table.GetID()} - if exists, _ := b.checkIfNodeExists(scpb.Target_DROP, dropDep); !exists { - b.addNode(scpb.Target_DROP, dropDep) - } - } - } + if behavior != tree.DropCascade { + return pgerror.Newf( + pgcode.DependentObjectsStillExist, "cannot drop table %q because view %q depends on it", + table.GetName(), dependentDesc.GetName()) + } + err = b.AuthAccessor.CheckPrivilege(ctx, dependentDesc, privilege.DROP) + if err != nil { + panic(err) } + b.maybeDropViewAndDependents(ctx, dependentDesc, behavior) + return nil + }) + if err != nil { + panic(err) + } + + // Clean up foreign key references (both inbound + // and out bound). + b.maybeCleanTableFKs(ctx, table, behavior) - // Clean up type back references - b.removeTypeBackRefDeps(ctx, table) - b.addNode(scpb.Target_DROP, - &scpb.Table{TableID: table.GetID()}) + // Clean up sequence references and ownerships. + b.maybeCleanTableSequenceRefs(ctx, table, behavior) + + // Clean up type back references + b.removeTypeBackRefDeps(ctx, table) + b.addNode(scpb.Target_DROP, + &scpb.Table{TableID: table.GetID()}) +} + +func (b *buildContext) dropTable(ctx context.Context, n *tree.DropTable) { + // Find the table first. + for _, name := range n.Names { + table, err := resolver.ResolveExistingTableObject(ctx, b.Res, &name, + tree.ObjectLookupFlagsWithRequired()) + if err != nil { + if errors.Is(err, catalog.ErrDescriptorNotFound) && n.IfExists { + return + } + panic(err) + } + if table == nil { + panic(errors.AssertionFailedf("Unable to resolve table %s", + name.FQString())) + } + b.dropTableDesc(ctx, table, n.DropBehavior) } } diff --git a/pkg/sql/schemachanger/scbuild/testdata/drop_sequence b/pkg/sql/schemachanger/scbuild/testdata/drop_sequence index 1403aa044082..42ee78ee85e1 100644 --- a/pkg/sql/schemachanger/scbuild/testdata/drop_sequence +++ b/pkg/sql/schemachanger/scbuild/testdata/drop_sequence @@ -9,13 +9,7 @@ DROP SEQUENCE defaultdb.SQ1 CASCADE direction: DROP elementProto: sequence: - tableId: 52 - state: PUBLIC -- target: - direction: DROP - elementProto: - sequenceOwner: - tableId: 52 + sequenceId: 52 state: PUBLIC create-table @@ -51,11 +45,5 @@ DROP SEQUENCE defaultdb.SQ1 CASCADE direction: DROP elementProto: sequence: - tableId: 52 - state: PUBLIC -- target: - direction: DROP - elementProto: - sequenceOwner: - tableId: 52 + sequenceId: 52 state: PUBLIC diff --git a/pkg/sql/schemachanger/scbuild/testdata/drop_table b/pkg/sql/schemachanger/scbuild/testdata/drop_table index 168d45365a67..7064701d736f 100644 --- a/pkg/sql/schemachanger/scbuild/testdata/drop_table +++ b/pkg/sql/schemachanger/scbuild/testdata/drop_table @@ -99,14 +99,14 @@ DROP TABLE defaultdb.shipments CASCADE; direction: DROP elementProto: sequence: - tableId: 56 + sequenceId: 56 state: PUBLIC - target: direction: DROP elementProto: sequenceOwner: ownerTableId: 55 - tableId: 56 + sequenceId: 56 state: PUBLIC - target: direction: DROP diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/scmutationexec.go b/pkg/sql/schemachanger/scexec/scmutationexec/scmutationexec.go index fac78a0bf74c..a6f187f45c40 100644 --- a/pkg/sql/schemachanger/scexec/scmutationexec/scmutationexec.go +++ b/pkg/sql/schemachanger/scexec/scmutationexec/scmutationexec.go @@ -16,8 +16,6 @@ import ( "sort" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" - "github.com/cockroachdb/cockroach/pkg/keys" - "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" @@ -43,7 +41,6 @@ type DescriptorReader interface { // namespaces. type NamespaceWriter interface { AddDrainedName(id descpb.ID, nameInfo descpb.NameInfo) - SubmitDrainedNames(ctx context.Context, codec keys.SQLCodec, ba *kv.Batch) error } // CommentWriter encapsulates operations used to manipulate @@ -64,7 +61,6 @@ type Catalog interface { // of jobs. type MutationJobs interface { AddNewGCJob(job jobspb.SchemaChangeGCDetails, description string) - SubmitAllJobs(ctx context.Context, txn *kv.Txn) (bool, error) } // NewMutationVisitor creates a new scop.MutationVisitor. diff --git a/pkg/sql/schemachanger/scpb/elements.go b/pkg/sql/schemachanger/scpb/elements.go index d6fd263f0b31..7c489741db54 100644 --- a/pkg/sql/schemachanger/scpb/elements.go +++ b/pkg/sql/schemachanger/scpb/elements.go @@ -160,13 +160,13 @@ func (e *CheckConstraint) GetAttributes() Attributes { } // DescriptorID implements the Element interface. -func (e *Sequence) DescriptorID() descpb.ID { return e.TableID } +func (e *Sequence) DescriptorID() descpb.ID { return e.SequenceID } // GetAttributes implements the Element interface func (e *Sequence) GetAttributes() Attributes { return makeAttributes([]attributeValue{ {key: AttributeType, value: ElementType(e)}, - {key: AttributeDescID, value: DescID(e.TableID)}, + {key: AttributeDescID, value: DescID(e.SequenceID)}, }) } @@ -258,10 +258,10 @@ func (e *RelationDependedOnBy) DescriptorID() descpb.ID { return e.TableID } func (e *SequenceOwnedBy) GetAttributes() Attributes { return makeAttributes([]attributeValue{ {key: AttributeType, value: ElementType(e)}, - {key: AttributeDescID, value: DescID(e.TableID)}, + {key: AttributeDescID, value: DescID(e.SequenceID)}, {key: AttributeDepID, value: DescID(e.OwnerTableID)}, }) } // DescriptorID implements the Element interface. -func (e *SequenceOwnedBy) DescriptorID() descpb.ID { return e.TableID } +func (e *SequenceOwnedBy) DescriptorID() descpb.ID { return e.SequenceID } diff --git a/pkg/sql/schemachanger/scpb/scpb.pb.go b/pkg/sql/schemachanger/scpb/scpb.pb.go index ef6bad1df1b0..fa7c8fdbb038 100644 --- a/pkg/sql/schemachanger/scpb/scpb.pb.go +++ b/pkg/sql/schemachanger/scpb/scpb.pb.go @@ -424,7 +424,7 @@ func (m *CheckConstraint) XXX_DiscardUnknown() { var xxx_messageInfo_CheckConstraint proto.InternalMessageInfo type Sequence struct { - TableID github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID `protobuf:"varint,1,opt,name=table_id,json=tableId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ID" json:"table_id,omitempty"` + SequenceID github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID `protobuf:"varint,1,opt,name=sequence_id,json=sequenceId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ID" json:"sequence_id,omitempty"` } func (m *Sequence) Reset() { *m = Sequence{} } @@ -670,7 +670,7 @@ func (m *InboundForeignKey) XXX_DiscardUnknown() { var xxx_messageInfo_InboundForeignKey proto.InternalMessageInfo type SequenceOwnedBy struct { - TableID github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID `protobuf:"varint,1,opt,name=table_id,json=tableId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ID" json:"table_id,omitempty"` + SequenceID github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID `protobuf:"varint,1,opt,name=sequence_id,json=sequenceId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ID" json:"sequence_id,omitempty"` OwnerTableID github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID `protobuf:"varint,2,opt,name=owner_table_id,json=ownerTableId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ID" json:"owner_table_id,omitempty"` } @@ -763,110 +763,110 @@ func init() { func init() { proto.RegisterFile("sql/schemachanger/scpb/scpb.proto", fileDescriptor_5413c88842564e28) } var fileDescriptor_5413c88842564e28 = []byte{ - // 1636 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x58, 0xcf, 0x6f, 0xdb, 0x46, - 0x16, 0x36, 0x25, 0x4a, 0x96, 0x9e, 0x7e, 0xd1, 0x13, 0xef, 0x42, 0x1b, 0xec, 0x8a, 0x8e, 0x17, - 0x48, 0x8c, 0x2c, 0x20, 0xed, 0x26, 0x8b, 0x05, 0xd6, 0xc0, 0x62, 0x63, 0x99, 0x32, 0x96, 0x6b, - 0x47, 0x72, 0x29, 0x3b, 0x6e, 0x82, 0x16, 0x02, 0x45, 0x4e, 0x64, 0x36, 0x12, 0x29, 0x93, 0x54, - 0x1c, 0xa1, 0x3d, 0xf4, 0xd2, 0x6b, 0xd3, 0x5b, 0x7b, 0xec, 0xb9, 0xf7, 0xfe, 0x0f, 0x06, 0x7a, - 0x49, 0x6f, 0x01, 0x0a, 0x08, 0xad, 0x02, 0xb4, 0xbd, 0xf4, 0x52, 0xa0, 0x97, 0x9e, 0x8a, 0x19, - 0xfe, 0x94, 0xe4, 0x42, 0x42, 0xa4, 0x0a, 0x68, 0xda, 0x8b, 0x41, 0x3f, 0xf1, 0x7d, 0xdf, 0xbc, - 0x37, 0x8f, 0xdf, 0x7b, 0x33, 0x70, 0xcd, 0x3a, 0x6b, 0x97, 0x2c, 0xe5, 0x14, 0x77, 0x64, 0xe5, - 0x54, 0xd6, 0x5b, 0xd8, 0x2c, 0x59, 0x4a, 0xb7, 0x49, 0xff, 0x14, 0xbb, 0xa6, 0x61, 0x1b, 0x68, - 0x43, 0x31, 0x94, 0x47, 0xa6, 0x21, 0x2b, 0xa7, 0x45, 0xeb, 0xac, 0x5d, 0x1c, 0x79, 0xb9, 0x48, - 0xde, 0xbb, 0xfa, 0x57, 0x02, 0xa2, 0xc8, 0xb6, 0xdc, 0x36, 0x5a, 0x25, 0x15, 0x3b, 0x00, 0xb6, - 0xd9, 0x53, 0xec, 0x9e, 0x89, 0x55, 0x07, 0xe6, 0xea, 0x7a, 0xcb, 0x68, 0x19, 0xf4, 0xb1, 0x44, - 0x9e, 0x1c, 0xeb, 0xe6, 0x47, 0x00, 0xe9, 0x4a, 0x1b, 0x77, 0xb0, 0x6e, 0x1f, 0x52, 0xb6, 0x3b, - 0x10, 0x57, 0x8c, 0x76, 0xaf, 0xa3, 0xe7, 0x99, 0x0d, 0x66, 0x2b, 0x75, 0x6b, 0xab, 0x38, 0x8d, - 0xbe, 0xb8, 0x4b, 0xdf, 0x97, 0x5c, 0x3f, 0x54, 0x87, 0x4c, 0xd7, 0xd4, 0x3a, 0xb2, 0xd9, 0x6f, - 0x68, 0xba, 0x8a, 0x9f, 0xe4, 0x23, 0x14, 0xa8, 0x38, 0x1d, 0xe8, 0xd0, 0x71, 0x13, 0x89, 0x97, - 0x94, 0xee, 0x86, 0xfe, 0x43, 0xf7, 0x21, 0x67, 0x61, 0xc5, 0xd0, 0xd5, 0x00, 0x36, 0x4a, 0x61, - 0xff, 0x3e, 0x1d, 0xb6, 0xee, 0x39, 0x3a, 0xc0, 0x59, 0x6b, 0xe4, 0x7f, 0x84, 0xe1, 0x8a, 0x85, - 0xcf, 0x7a, 0x58, 0x57, 0x70, 0x43, 0xc5, 0x5d, 0xac, 0xab, 0x58, 0x57, 0xfa, 0x79, 0x96, 0xc2, - 0xff, 0x73, 0x16, 0x78, 0xc7, 0x59, 0xf0, 0x7d, 0x25, 0x64, 0x4d, 0xd8, 0x50, 0x03, 0xd6, 0x7a, - 0xba, 0x76, 0xd6, 0xc3, 0x0d, 0xc5, 0xd0, 0x2d, 0xdb, 0x94, 0x35, 0xdd, 0xce, 0xc7, 0x28, 0xc9, - 0xad, 0xe9, 0x24, 0xc7, 0xd4, 0x75, 0xd7, 0xf7, 0x94, 0xb8, 0xde, 0x98, 0x05, 0xbd, 0x01, 0x9c, - 0x72, 0x8a, 0x95, 0x47, 0x61, 0xfc, 0x38, 0xc5, 0xff, 0xc7, 0x0c, 0x7b, 0x48, 0x3c, 0x43, 0xf0, - 0x39, 0x65, 0xd4, 0x80, 0xf6, 0x20, 0xe1, 0x05, 0x95, 0x5f, 0xa5, 0xa8, 0x37, 0x67, 0x4f, 0x8d, - 0xe4, 0xfb, 0xa2, 0x26, 0x20, 0x15, 0x3f, 0x94, 0x7b, 0x6d, 0xbb, 0x81, 0x9f, 0x74, 0x4d, 0x6c, - 0x59, 0x9a, 0xa1, 0xe7, 0x13, 0x14, 0xf1, 0xf6, 0x74, 0x44, 0xc1, 0xf1, 0xad, 0xf8, 0xae, 0xd2, - 0x9a, 0x3a, 0x6e, 0x42, 0xdb, 0xc0, 0x3e, 0xd6, 0xf0, 0x79, 0x3e, 0x49, 0x51, 0xaf, 0x4f, 0x47, - 0xbd, 0xa7, 0xe1, 0x73, 0x89, 0xfa, 0x20, 0x11, 0x56, 0xed, 0x7e, 0x17, 0x4b, 0xf8, 0x61, 0x1e, - 0xa8, 0x7b, 0x69, 0xba, 0xfb, 0x91, 0xe3, 0x80, 0x4d, 0x1a, 0xab, 0xe7, 0x8f, 0xfe, 0x03, 0x31, - 0x5b, 0x6e, 0xb6, 0x71, 0x3e, 0x45, 0x81, 0x6e, 0xcc, 0x00, 0x44, 0x5e, 0x97, 0x1c, 0x2f, 0xf4, - 0x00, 0x32, 0x46, 0xcf, 0xde, 0x33, 0x4c, 0xac, 0xb5, 0xf4, 0x7d, 0xdc, 0xcf, 0xa7, 0x67, 0xad, - 0xc8, 0x5a, 0xcf, 0x6e, 0x1a, 0x3d, 0x5d, 0x0d, 0x7c, 0xa5, 0x51, 0x28, 0x74, 0x02, 0x69, 0x4d, - 0x0f, 0x41, 0x67, 0x66, 0xcd, 0xbf, 0xa8, 0x8f, 0x23, 0x8f, 0x00, 0xa1, 0xb7, 0x60, 0xdd, 0xc4, - 0x6d, 0xd9, 0xd6, 0x0c, 0xdd, 0xad, 0x7d, 0xb5, 0xa6, 0x97, 0xfb, 0xf9, 0x2c, 0x25, 0xf8, 0xd7, - 0x74, 0x02, 0xe9, 0x12, 0x6f, 0xe9, 0x52, 0x4c, 0x74, 0x02, 0x19, 0xaf, 0xac, 0x6a, 0xe7, 0x3a, - 0x36, 0xf3, 0xb9, 0x59, 0xab, 0xbd, 0x1e, 0x72, 0x53, 0xcb, 0x7d, 0x69, 0x14, 0x67, 0x9b, 0xbd, - 0xf8, 0x98, 0x67, 0x36, 0xbf, 0x66, 0x20, 0x7e, 0x24, 0x9b, 0x2d, 0x6c, 0xa3, 0x37, 0x21, 0x83, - 0x1d, 0x91, 0x6c, 0x50, 0xd9, 0x74, 0xb5, 0x71, 0x06, 0x49, 0x0b, 0x6b, 0x6b, 0x39, 0x71, 0x31, - 0xe0, 0x57, 0x9e, 0x0d, 0x78, 0x46, 0x4a, 0xe3, 0xb0, 0xe6, 0x1e, 0x42, 0x52, 0xd5, 0x4c, 0xac, - 0x90, 0x08, 0xa9, 0x5a, 0x66, 0x67, 0x91, 0x04, 0x67, 0x6d, 0x45, 0xc1, 0xf3, 0x94, 0x02, 0x90, - 0xcd, 0xbf, 0x41, 0xd2, 0xb7, 0xa3, 0x14, 0xac, 0x1e, 0x57, 0xf7, 0xab, 0xb5, 0x93, 0x2a, 0xb7, - 0x82, 0x56, 0x21, 0xba, 0x23, 0x08, 0x1c, 0x83, 0x12, 0xc0, 0x0a, 0x52, 0xed, 0x90, 0x8b, 0x6c, - 0x7e, 0x16, 0x81, 0xb8, 0xa3, 0xe1, 0x48, 0x85, 0x04, 0x2d, 0xbe, 0x86, 0xa6, 0xd2, 0x18, 0x33, - 0x65, 0x71, 0x38, 0xe0, 0x57, 0x69, 0x5d, 0x8a, 0xc2, 0x8f, 0x03, 0x7e, 0xbb, 0xa5, 0xd9, 0xa7, - 0xbd, 0x66, 0x51, 0x31, 0x3a, 0x25, 0x7f, 0x85, 0x6a, 0x33, 0x78, 0x2e, 0x75, 0x1f, 0xb5, 0x4a, - 0x93, 0xfd, 0xa8, 0x28, 0x0a, 0xd2, 0x2a, 0x85, 0x16, 0x55, 0xd4, 0x81, 0xe4, 0x43, 0xb9, 0xa3, - 0xb5, 0xfb, 0x84, 0x26, 0x42, 0x69, 0x0e, 0x87, 0x03, 0x3e, 0xb1, 0x47, 0x8d, 0x94, 0xe7, 0xce, - 0xcb, 0xf2, 0x78, 0x18, 0x52, 0xc2, 0xa1, 0x10, 0x55, 0xc4, 0x43, 0xca, 0xa5, 0xd3, 0xe5, 0x0e, - 0xa6, 0x7d, 0x23, 0x29, 0x81, 0x63, 0xaa, 0xca, 0x1d, 0x8c, 0x2a, 0x7e, 0xcf, 0x63, 0x2f, 0xff, - 0x52, 0xcf, 0xda, 0x4d, 0xd9, 0xc2, 0x6e, 0xa3, 0x13, 0xb0, 0xa5, 0x98, 0x5a, 0xd7, 0x36, 0xcc, - 0x32, 0x4b, 0x36, 0xd4, 0x6b, 0x7c, 0xdb, 0xec, 0xb7, 0xa4, 0x6c, 0x3e, 0x64, 0x21, 0x1d, 0x6e, - 0x64, 0x4b, 0xca, 0x69, 0x19, 0x62, 0xe1, 0x6e, 0x7b, 0xfd, 0x67, 0x42, 0xa0, 0x4b, 0x9a, 0x88, - 0xc0, 0x71, 0x45, 0x4f, 0x19, 0xf8, 0xa3, 0x61, 0x9f, 0x62, 0xb3, 0x31, 0xd2, 0xc0, 0xc9, 0xc2, - 0xa3, 0x74, 0xe1, 0x0f, 0x86, 0x03, 0xfe, 0x4a, 0x8d, 0xbc, 0x11, 0x8e, 0x90, 0x06, 0xf1, 0xdf, - 0x97, 0x0e, 0xc2, 0x81, 0x90, 0xae, 0x18, 0x13, 0xb8, 0x2a, 0x7a, 0x97, 0x01, 0xce, 0xb2, 0x0d, - 0x93, 0x34, 0x4d, 0x92, 0xe3, 0x86, 0xa6, 0x5a, 0x79, 0x76, 0x23, 0xba, 0x95, 0x29, 0xdf, 0x1b, - 0x0e, 0xf8, 0x6c, 0x9d, 0xfc, 0xe6, 0x6c, 0x8b, 0x28, 0x58, 0xf3, 0xd4, 0x8d, 0x07, 0x22, 0x65, - 0xad, 0x10, 0xa6, 0x6a, 0xa1, 0x32, 0xa0, 0x91, 0x15, 0x90, 0x1a, 0xb2, 0xf2, 0xb1, 0x8d, 0xe8, - 0x56, 0xb2, 0xbc, 0x3e, 0x1c, 0xf0, 0x5c, 0x68, 0x0d, 0xa4, 0x9a, 0x2c, 0x89, 0xb3, 0xc6, 0x2c, - 0x6e, 0x65, 0x7c, 0x12, 0x81, 0xec, 0xe8, 0x2c, 0xf2, 0x2b, 0xaa, 0x8d, 0xd3, 0xf1, 0xa9, 0xce, - 0xa9, 0x88, 0xdd, 0x45, 0x6c, 0xfd, 0xc8, 0xa8, 0xe7, 0x26, 0xeb, 0x87, 0x28, 0xa0, 0xc9, 0xc9, - 0x6a, 0x79, 0x02, 0xe5, 0xd7, 0x5b, 0x58, 0xa0, 0xbc, 0x22, 0x59, 0x48, 0xa1, 0x25, 0x14, 0xb7, - 0xc6, 0x50, 0x07, 0x52, 0xfe, 0x04, 0xea, 0x7f, 0x6b, 0x07, 0xc3, 0x01, 0x0f, 0x5e, 0x06, 0xe6, - 0x0e, 0x0d, 0x3c, 0x02, 0x51, 0x45, 0x77, 0x81, 0x25, 0x23, 0x0a, 0x15, 0xbb, 0xec, 0xad, 0x7f, - 0xbf, 0xcc, 0x84, 0xeb, 0x8c, 0x3c, 0x14, 0x06, 0xfd, 0x05, 0x62, 0x4d, 0xaa, 0xe4, 0x64, 0x98, - 0x4d, 0x94, 0x13, 0xc3, 0x01, 0xcf, 0x96, 0x89, 0x02, 0xb3, 0xcd, 0xbe, 0xa8, 0x6e, 0xde, 0x00, - 0x96, 0xbc, 0x3c, 0xda, 0x85, 0x12, 0xc0, 0x1e, 0xd7, 0x2b, 0x75, 0xa7, 0x0d, 0xd5, 0x4e, 0xaa, - 0x75, 0x2e, 0xe2, 0xee, 0xfb, 0x17, 0x11, 0xe0, 0xc6, 0x87, 0xdd, 0x25, 0xed, 0xba, 0x06, 0x09, - 0x5f, 0xef, 0x9c, 0x4d, 0xaf, 0x12, 0x96, 0x05, 0x6a, 0xdc, 0xaa, 0xe6, 0xea, 0x5a, 0x17, 0x20, - 0x24, 0x68, 0x51, 0x2a, 0x68, 0xaf, 0x0d, 0x07, 0x7c, 0x72, 0xb1, 0x5a, 0x96, 0xf4, 0x4a, 0xcc, - 0x93, 0xa0, 0x4f, 0x23, 0x90, 0x1b, 0x1b, 0xf5, 0x97, 0x94, 0x5c, 0x04, 0x2c, 0xed, 0xbe, 0x11, - 0xda, 0x7d, 0xe9, 0x33, 0xb1, 0x91, 0x33, 0x80, 0xdb, 0x91, 0xe9, 0xf3, 0x58, 0x66, 0xd8, 0x5f, - 0x3e, 0x33, 0xe8, 0xcf, 0x90, 0x7c, 0x2c, 0xb7, 0x35, 0x55, 0xb6, 0xb1, 0x5b, 0xc3, 0x52, 0x60, - 0x70, 0xf3, 0xf6, 0x18, 0x12, 0xde, 0x47, 0xb0, 0x9c, 0x7c, 0xb9, 0xbc, 0xef, 0x47, 0x61, 0x6d, - 0xe2, 0xc8, 0xf3, 0x6a, 0x8a, 0xe0, 0xdb, 0x90, 0xeb, 0x59, 0xd8, 0x0a, 0x14, 0x2f, 0xfc, 0x5d, - 0xe4, 0x8e, 0x47, 0x7f, 0x9a, 0x33, 0xc6, 0x71, 0x26, 0x74, 0x0d, 0xd2, 0xe1, 0x53, 0x29, 0x95, - 0xc6, 0xa4, 0x94, 0x0a, 0x1d, 0x2d, 0xdd, 0x0d, 0xf9, 0x3c, 0x02, 0x2c, 0x39, 0x2d, 0x2e, 0x69, - 0x0f, 0x4c, 0xb2, 0xae, 0xd0, 0x31, 0x2a, 0x42, 0x33, 0x42, 0x64, 0x29, 0x1d, 0x3e, 0x0a, 0xcd, - 0x49, 0x37, 0xc2, 0x81, 0x34, 0x48, 0x3a, 0xff, 0x5b, 0x35, 0xdd, 0xdd, 0x82, 0x7d, 0xf2, 0x01, - 0x0a, 0x9e, 0x71, 0x4e, 0xb6, 0x00, 0xdd, 0xcd, 0xe9, 0x37, 0x0c, 0xc4, 0x68, 0xde, 0x5e, 0xdd, - 0xa4, 0x06, 0x91, 0x66, 0x46, 0x2e, 0x0b, 0x90, 0xec, 0x5c, 0x37, 0x04, 0x01, 0xff, 0x6f, 0x38, - 0xe0, 0xe3, 0xe4, 0x9d, 0xb9, 0xe3, 0x8d, 0x13, 0x60, 0xfa, 0x1d, 0x67, 0x54, 0x7f, 0xa8, 0x0b, - 0xbe, 0x65, 0x4a, 0x44, 0xa6, 0xbd, 0xb9, 0x89, 0xd2, 0x01, 0xbc, 0x2f, 0x5c, 0x4f, 0x59, 0x40, - 0x93, 0xd7, 0x10, 0xa8, 0x05, 0x49, 0xc3, 0xd4, 0x5a, 0x9a, 0x1e, 0x04, 0xfc, 0x7f, 0xa2, 0x29, - 0x35, 0x6a, 0x9c, 0x7b, 0x25, 0x09, 0x07, 0x5c, 0x54, 0xd1, 0x3b, 0x90, 0x75, 0x89, 0x1c, 0x81, - 0xf1, 0xc4, 0xe4, 0x78, 0x38, 0xe0, 0x33, 0x0e, 0x9b, 0xa3, 0x41, 0x8b, 0x69, 0x27, 0x19, 0x23, - 0x0c, 0x89, 0xba, 0x90, 0x36, 0xbd, 0x2d, 0x26, 0x91, 0xb2, 0x34, 0xd2, 0xbb, 0xc3, 0x01, 0x9f, - 0xf2, 0xb7, 0x7e, 0xee, 0x60, 0x53, 0x3e, 0x85, 0xa8, 0xa2, 0xf7, 0x18, 0x58, 0x0b, 0x28, 0xbd, - 0x98, 0x63, 0x34, 0xe6, 0xd7, 0xc9, 0x29, 0xc5, 0xe7, 0x5d, 0x64, 0xd8, 0x9c, 0x39, 0x86, 0xea, - 0xb7, 0xf9, 0x78, 0xd0, 0xe6, 0xbd, 0x56, 0xc6, 0xc2, 0xda, 0xc4, 0xed, 0xd1, 0xef, 0x05, 0xf1, - 0xdb, 0x2d, 0x88, 0xef, 0x18, 0xc8, 0x8d, 0x5d, 0xc4, 0x2d, 0xa9, 0x01, 0xd8, 0x90, 0x35, 0xce, - 0x75, 0x6c, 0x36, 0x7c, 0xae, 0x60, 0xdc, 0x4f, 0xd3, 0x2b, 0xc0, 0xc5, 0x10, 0xa6, 0x8d, 0x00, - 0x4b, 0xdd, 0xfc, 0x9e, 0x81, 0xf5, 0xcb, 0x6e, 0x37, 0x97, 0x14, 0xb4, 0x0e, 0x10, 0x74, 0xa4, - 0x70, 0xc0, 0x0b, 0xec, 0x79, 0x21, 0x06, 0x67, 0x93, 0x6f, 0x9e, 0x43, 0xac, 0x6e, 0xcb, 0xf6, - 0xd8, 0xf1, 0x0f, 0x20, 0xbe, 0x53, 0xae, 0x57, 0xaa, 0x47, 0x1c, 0x83, 0x72, 0x90, 0x12, 0x2a, - 0x07, 0x95, 0xa3, 0x4a, 0xa3, 0x56, 0x3d, 0xb8, 0xcf, 0x45, 0xd0, 0x9f, 0xe0, 0x0f, 0xae, 0x61, - 0xa7, 0x2a, 0x34, 0x4e, 0x24, 0xd1, 0xfb, 0x29, 0x8a, 0xb2, 0x00, 0xe5, 0x9d, 0xdd, 0xfd, 0x3d, - 0xf1, 0xe0, 0xa0, 0x22, 0x70, 0x2c, 0xca, 0x40, 0xf2, 0xde, 0xce, 0x81, 0x28, 0xec, 0x1c, 0x55, - 0x04, 0x2e, 0x46, 0x60, 0x0f, 0x8f, 0xcb, 0x07, 0xe2, 0x2e, 0x17, 0x2f, 0x5f, 0xbf, 0xf8, 0xaa, - 0xb0, 0x72, 0x31, 0x2c, 0x30, 0xcf, 0x86, 0x05, 0xe6, 0xf9, 0xb0, 0xc0, 0x7c, 0x39, 0x2c, 0x30, - 0x1f, 0xbc, 0x28, 0xac, 0x3c, 0x7b, 0x51, 0x58, 0x79, 0xfe, 0xa2, 0xb0, 0xf2, 0x80, 0x25, 0xab, - 0x6e, 0xc6, 0xe9, 0x85, 0xee, 0xed, 0x9f, 0x02, 0x00, 0x00, 0xff, 0xff, 0x06, 0xe2, 0x3b, 0xbf, - 0x89, 0x1b, 0x00, 0x00, + // 1641 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x58, 0xcd, 0x6f, 0xdb, 0x46, + 0x16, 0x37, 0x25, 0x4a, 0x96, 0x9e, 0xbe, 0xe8, 0x89, 0x77, 0xa1, 0x0d, 0x76, 0x45, 0xc7, 0x0b, + 0x24, 0x46, 0x16, 0x90, 0x76, 0x93, 0xc5, 0x02, 0x6b, 0xa0, 0x68, 0x2c, 0x53, 0x46, 0x59, 0x3b, + 0x92, 0x4b, 0xd9, 0x71, 0x13, 0xb4, 0x10, 0x28, 0x72, 0x22, 0xb3, 0x91, 0x48, 0x99, 0xa4, 0xe2, + 0x08, 0x2d, 0xd0, 0x5e, 0x7a, 0x6d, 0x7a, 0x6b, 0x8f, 0x3d, 0xf7, 0xde, 0xff, 0xc1, 0x40, 0x2f, + 0xe9, 0x2d, 0x40, 0x01, 0xa1, 0x55, 0x80, 0xb6, 0xe7, 0x02, 0xed, 0xa1, 0xa7, 0x62, 0x86, 0x9f, + 0x92, 0x5c, 0x48, 0x88, 0x14, 0x03, 0x4d, 0x7b, 0x31, 0xe8, 0xa7, 0x79, 0xbf, 0x37, 0xef, 0x63, + 0x7e, 0xef, 0xcd, 0xc0, 0x15, 0xeb, 0xa4, 0x5d, 0xb2, 0x94, 0x63, 0xdc, 0x91, 0x95, 0x63, 0x59, + 0x6f, 0x61, 0xb3, 0x64, 0x29, 0xdd, 0x26, 0xfd, 0x53, 0xec, 0x9a, 0x86, 0x6d, 0xa0, 0x35, 0xc5, + 0x50, 0x1e, 0x98, 0x86, 0xac, 0x1c, 0x17, 0xad, 0x93, 0x76, 0x71, 0x64, 0x71, 0x91, 0xac, 0xbb, + 0xfc, 0x4f, 0x02, 0xa2, 0xc8, 0xb6, 0xdc, 0x36, 0x5a, 0x25, 0x15, 0x3b, 0x00, 0xb6, 0xd9, 0x53, + 0xec, 0x9e, 0x89, 0x55, 0x07, 0xe6, 0xf2, 0x6a, 0xcb, 0x68, 0x19, 0xf4, 0xb3, 0x44, 0xbe, 0x1c, + 0xe9, 0xfa, 0xa7, 0x00, 0xe9, 0x4a, 0x1b, 0x77, 0xb0, 0x6e, 0xef, 0x53, 0x6b, 0xb7, 0x20, 0xae, + 0x18, 0xed, 0x5e, 0x47, 0xcf, 0x33, 0x6b, 0xcc, 0x46, 0xea, 0xc6, 0x46, 0x71, 0x9a, 0xf9, 0xe2, + 0x36, 0x5d, 0x2f, 0xb9, 0x7a, 0xa8, 0x0e, 0x99, 0xae, 0xa9, 0x75, 0x64, 0xb3, 0xdf, 0xd0, 0x74, + 0x15, 0x3f, 0xca, 0x47, 0x28, 0x50, 0x71, 0x3a, 0xd0, 0xbe, 0xa3, 0x26, 0x12, 0x2d, 0x29, 0xdd, + 0x0d, 0xfd, 0x87, 0xee, 0x42, 0xce, 0xc2, 0x8a, 0xa1, 0xab, 0x01, 0x6c, 0x94, 0xc2, 0xfe, 0x7b, + 0x3a, 0x6c, 0xdd, 0x53, 0x74, 0x80, 0xb3, 0xd6, 0xc8, 0xff, 0x08, 0xc3, 0x25, 0x0b, 0x9f, 0xf4, + 0xb0, 0xae, 0xe0, 0x86, 0x8a, 0xbb, 0x58, 0x57, 0xb1, 0xae, 0xf4, 0xf3, 0x2c, 0x85, 0xff, 0xef, + 0x2c, 0xf0, 0x8e, 0xb2, 0xe0, 0xeb, 0x4a, 0xc8, 0x9a, 0x90, 0xa1, 0x06, 0xac, 0xf4, 0x74, 0xed, + 0xa4, 0x87, 0x1b, 0x8a, 0xa1, 0x5b, 0xb6, 0x29, 0x6b, 0xba, 0x9d, 0x8f, 0x51, 0x23, 0x37, 0xa6, + 0x1b, 0x39, 0xa4, 0xaa, 0xdb, 0xbe, 0xa6, 0xc4, 0xf5, 0xc6, 0x24, 0xe8, 0x2d, 0xe0, 0x94, 0x63, + 0xac, 0x3c, 0x08, 0xe3, 0xc7, 0x29, 0xfe, 0x7f, 0x66, 0xc8, 0x21, 0xd1, 0x0c, 0xc1, 0xe7, 0x94, + 0x51, 0x01, 0xda, 0x81, 0x84, 0xe7, 0x54, 0x7e, 0x99, 0xa2, 0x5e, 0x9f, 0x3d, 0x34, 0x92, 0xaf, + 0x8b, 0x9a, 0x80, 0x54, 0x7c, 0x5f, 0xee, 0xb5, 0xed, 0x06, 0x7e, 0xd4, 0x35, 0xb1, 0x65, 0x69, + 0x86, 0x9e, 0x4f, 0x50, 0xc4, 0x9b, 0xd3, 0x11, 0x05, 0x47, 0xb7, 0xe2, 0xab, 0x4a, 0x2b, 0xea, + 0xb8, 0x08, 0x6d, 0x02, 0xfb, 0x50, 0xc3, 0xa7, 0xf9, 0x24, 0x45, 0xbd, 0x3a, 0x1d, 0xf5, 0x8e, + 0x86, 0x4f, 0x25, 0xaa, 0x83, 0x44, 0x58, 0xb6, 0xfb, 0x5d, 0x2c, 0xe1, 0xfb, 0x79, 0xa0, 0xea, + 0xa5, 0xe9, 0xea, 0x07, 0x8e, 0x02, 0x36, 0xa9, 0xaf, 0x9e, 0x3e, 0x7a, 0x05, 0x62, 0xb6, 0xdc, + 0x6c, 0xe3, 0x7c, 0x8a, 0x02, 0x5d, 0x9b, 0x01, 0x88, 0x2c, 0x97, 0x1c, 0x2d, 0x74, 0x0f, 0x32, + 0x46, 0xcf, 0xde, 0x31, 0x4c, 0xac, 0xb5, 0xf4, 0x5d, 0xdc, 0xcf, 0xa7, 0x67, 0xad, 0xc8, 0x5a, + 0xcf, 0x6e, 0x1a, 0x3d, 0x5d, 0x0d, 0x74, 0xa5, 0x51, 0x28, 0x74, 0x04, 0x69, 0x4d, 0x0f, 0x41, + 0x67, 0x66, 0x8d, 0xbf, 0xa8, 0x8f, 0x23, 0x8f, 0x00, 0xa1, 0x77, 0x60, 0xd5, 0xc4, 0x6d, 0xd9, + 0xd6, 0x0c, 0xdd, 0xad, 0x7d, 0xb5, 0xa6, 0x97, 0xfb, 0xf9, 0x2c, 0x35, 0xf0, 0xbf, 0xe9, 0x06, + 0xa4, 0x73, 0xb4, 0xa5, 0x73, 0x31, 0xd1, 0x11, 0x64, 0xbc, 0xb2, 0xaa, 0x9d, 0xea, 0xd8, 0xcc, + 0xe7, 0x66, 0xad, 0xf6, 0x7a, 0x48, 0x4d, 0x2d, 0xf7, 0xa5, 0x51, 0x9c, 0x4d, 0xf6, 0xec, 0x33, + 0x9e, 0x59, 0xff, 0x8e, 0x81, 0xf8, 0x81, 0x6c, 0xb6, 0xb0, 0x8d, 0xde, 0x86, 0x0c, 0x76, 0x48, + 0xb2, 0x41, 0x69, 0xd3, 0xe5, 0xc6, 0x19, 0x28, 0x2d, 0xcc, 0xad, 0xe5, 0xc4, 0xd9, 0x80, 0x5f, + 0x7a, 0x32, 0xe0, 0x19, 0x29, 0x8d, 0xc3, 0x9c, 0xbb, 0x0f, 0x49, 0x55, 0x33, 0xb1, 0x42, 0x3c, + 0xa4, 0x6c, 0x99, 0x9d, 0x85, 0x12, 0x9c, 0xbd, 0x15, 0x05, 0x4f, 0x53, 0x0a, 0x40, 0xd6, 0xff, + 0x05, 0x49, 0x5f, 0x8e, 0x52, 0xb0, 0x7c, 0x58, 0xdd, 0xad, 0xd6, 0x8e, 0xaa, 0xdc, 0x12, 0x5a, + 0x86, 0xe8, 0x96, 0x20, 0x70, 0x0c, 0x4a, 0x00, 0x2b, 0x48, 0xb5, 0x7d, 0x2e, 0xb2, 0xfe, 0x65, + 0x04, 0xe2, 0x0e, 0x87, 0x23, 0x15, 0x12, 0xb4, 0xf8, 0x1a, 0x9a, 0x4a, 0x7d, 0xcc, 0x94, 0xc5, + 0xe1, 0x80, 0x5f, 0xa6, 0x75, 0x29, 0x0a, 0xbf, 0x0c, 0xf8, 0xcd, 0x96, 0x66, 0x1f, 0xf7, 0x9a, + 0x45, 0xc5, 0xe8, 0x94, 0xfc, 0x1d, 0xaa, 0xcd, 0xe0, 0xbb, 0xd4, 0x7d, 0xd0, 0x2a, 0x4d, 0xf6, + 0xa3, 0xa2, 0x28, 0x48, 0xcb, 0x14, 0x5a, 0x54, 0x51, 0x07, 0x92, 0xf7, 0xe5, 0x8e, 0xd6, 0xee, + 0x13, 0x33, 0x11, 0x6a, 0x66, 0x7f, 0x38, 0xe0, 0x13, 0x3b, 0x54, 0x48, 0xed, 0xdc, 0x7a, 0x5e, + 0x3b, 0x1e, 0x86, 0x94, 0x70, 0x4c, 0x88, 0x2a, 0xe2, 0x21, 0xe5, 0x9a, 0xd3, 0xe5, 0x0e, 0xa6, + 0x7d, 0x23, 0x29, 0x81, 0x23, 0xaa, 0xca, 0x1d, 0x8c, 0x2a, 0x7e, 0xcf, 0x63, 0xcf, 0x3f, 0xa9, + 0x27, 0xed, 0xa6, 0x6c, 0x61, 0xb7, 0xd1, 0x09, 0xd8, 0x52, 0x4c, 0xad, 0x6b, 0x1b, 0x66, 0x99, + 0x25, 0x09, 0xf5, 0x1a, 0xdf, 0x26, 0xfb, 0x03, 0x29, 0x9b, 0x4f, 0x58, 0x48, 0x87, 0x1b, 0xd9, + 0x05, 0xc5, 0xb4, 0x0c, 0xb1, 0x70, 0xb7, 0xbd, 0xfa, 0x1b, 0x2e, 0xd0, 0x2d, 0x4d, 0x78, 0xe0, + 0xa8, 0xa2, 0xc7, 0x0c, 0xfc, 0xd5, 0xb0, 0x8f, 0xb1, 0xd9, 0x18, 0x69, 0xe0, 0x64, 0xe3, 0x51, + 0xba, 0xf1, 0x7b, 0xc3, 0x01, 0x7f, 0xa9, 0x46, 0x56, 0x84, 0x3d, 0xa4, 0x4e, 0xbc, 0xfa, 0xdc, + 0x4e, 0x38, 0x10, 0xd2, 0x25, 0x63, 0x02, 0x57, 0x45, 0x1f, 0x30, 0xc0, 0x59, 0xb6, 0x61, 0x92, + 0xa6, 0x49, 0x62, 0xdc, 0xd0, 0x54, 0x2b, 0xcf, 0xae, 0x45, 0x37, 0x32, 0xe5, 0x3b, 0xc3, 0x01, + 0x9f, 0xad, 0x93, 0xdf, 0x9c, 0xb4, 0x88, 0x82, 0x35, 0x4f, 0xdd, 0x78, 0x20, 0x52, 0xd6, 0x0a, + 0x61, 0xaa, 0x16, 0x2a, 0x03, 0x1a, 0xd9, 0x01, 0xa9, 0x21, 0x2b, 0x1f, 0x5b, 0x8b, 0x6e, 0x24, + 0xcb, 0xab, 0xc3, 0x01, 0xcf, 0x85, 0xf6, 0x40, 0xaa, 0xc9, 0x92, 0x38, 0x6b, 0x4c, 0xe2, 0x56, + 0xc6, 0xe7, 0x11, 0xc8, 0x8e, 0xce, 0x22, 0xbf, 0xa3, 0xda, 0x38, 0x1e, 0x9f, 0xea, 0x9c, 0x8a, + 0xd8, 0x5e, 0x44, 0xea, 0x47, 0x46, 0x3d, 0x37, 0x58, 0x3f, 0x45, 0x01, 0x4d, 0x4e, 0x56, 0x17, + 0x47, 0x50, 0x7e, 0xbd, 0x85, 0x09, 0xca, 0x2b, 0x92, 0x85, 0x14, 0x5a, 0x42, 0x71, 0x6b, 0x0c, + 0x75, 0x20, 0xe5, 0x4f, 0xa0, 0xfe, 0x59, 0xdb, 0x1b, 0x0e, 0x78, 0xf0, 0x22, 0x30, 0xb7, 0x6b, + 0xe0, 0x19, 0x10, 0x55, 0x74, 0x1b, 0x58, 0x32, 0xa2, 0x50, 0xb2, 0xcb, 0xde, 0xf8, 0xff, 0xf3, + 0x4c, 0xb8, 0xce, 0xc8, 0x43, 0x61, 0xd0, 0x3f, 0x20, 0xd6, 0xa4, 0x4c, 0x4e, 0x86, 0xd9, 0x44, + 0x39, 0x31, 0x1c, 0xf0, 0x6c, 0x99, 0x30, 0x30, 0xdb, 0xec, 0x8b, 0xea, 0xfa, 0x35, 0x60, 0xc9, + 0xe2, 0xd1, 0x2e, 0x94, 0x00, 0xf6, 0xb0, 0x5e, 0xa9, 0x3b, 0x6d, 0xa8, 0x76, 0x54, 0xad, 0x73, + 0x11, 0x37, 0xef, 0x5f, 0x47, 0x80, 0x1b, 0x1f, 0x76, 0x2f, 0x28, 0xeb, 0x1a, 0x24, 0x7c, 0xbe, + 0x73, 0x92, 0x5e, 0x25, 0x56, 0x16, 0xc8, 0x71, 0xcb, 0x9a, 0xcb, 0x6b, 0x5d, 0x80, 0x10, 0xa1, + 0x45, 0x29, 0xa1, 0xbd, 0x31, 0x1c, 0xf0, 0xc9, 0xc5, 0x72, 0x59, 0xd2, 0x2b, 0x31, 0x8f, 0x82, + 0xbe, 0x88, 0x40, 0x6e, 0x6c, 0xd4, 0xbf, 0xa0, 0xe0, 0x22, 0x60, 0x69, 0xf7, 0x8d, 0xd0, 0xee, + 0x4b, 0xbf, 0x89, 0x8c, 0xdc, 0x01, 0xdc, 0x8e, 0x4c, 0xbf, 0xc7, 0x22, 0xc3, 0xbe, 0xf8, 0xc8, + 0xa0, 0xbf, 0x43, 0xf2, 0xa1, 0xdc, 0xd6, 0x54, 0xd9, 0xc6, 0x6e, 0x0d, 0x4b, 0x81, 0xc0, 0x8d, + 0xdb, 0xfb, 0x90, 0xf0, 0x0e, 0xc1, 0xf8, 0x69, 0x65, 0x5e, 0xec, 0x69, 0x75, 0x37, 0xf0, 0x51, + 0x14, 0x56, 0x26, 0xee, 0x3e, 0x2f, 0x27, 0x1b, 0xbe, 0x0b, 0xb9, 0x9e, 0x85, 0xad, 0x20, 0x98, + 0xe1, 0x03, 0x92, 0x3b, 0x1c, 0xfd, 0x69, 0x4e, 0x1f, 0xc7, 0x2d, 0xa1, 0x2b, 0x90, 0x0e, 0x5f, + 0x4f, 0x29, 0x47, 0x26, 0xa5, 0x54, 0xe8, 0x8e, 0xe9, 0x26, 0xe4, 0xab, 0x08, 0xb0, 0xe4, 0xda, + 0x78, 0x41, 0x39, 0x30, 0xc9, 0xbe, 0x42, 0xf7, 0xa9, 0x08, 0x8d, 0x08, 0xe1, 0xa7, 0x74, 0xf8, + 0x4e, 0x34, 0xa7, 0xb9, 0x11, 0x1b, 0x48, 0x83, 0xa4, 0xf3, 0xbf, 0x55, 0xd3, 0xdd, 0x14, 0xec, + 0x92, 0x93, 0x28, 0x78, 0xc2, 0x39, 0xad, 0x05, 0xe8, 0x6e, 0x4c, 0xbf, 0x67, 0x20, 0x46, 0xe3, + 0xf6, 0xf2, 0x06, 0x35, 0xf0, 0x34, 0x33, 0xf2, 0x6a, 0x80, 0x64, 0xe7, 0xdd, 0x21, 0x70, 0xf8, + 0xb5, 0xe1, 0x80, 0x8f, 0x93, 0x35, 0x73, 0xfb, 0x1b, 0x27, 0xc0, 0xf4, 0x1c, 0x67, 0x54, 0x7f, + 0xba, 0x0b, 0xce, 0x32, 0x35, 0x44, 0xc6, 0xbe, 0xb9, 0x0d, 0xa5, 0x03, 0x78, 0x9f, 0xb8, 0x1e, + 0xb3, 0x80, 0x26, 0xdf, 0x23, 0x50, 0x0b, 0x92, 0x86, 0xa9, 0xb5, 0x34, 0x3d, 0x70, 0xf8, 0x75, + 0xc2, 0x29, 0x35, 0x2a, 0x9c, 0x7b, 0x27, 0x09, 0x07, 0x5c, 0x54, 0xd1, 0x7b, 0x90, 0x75, 0x0d, + 0x39, 0x04, 0xe3, 0x91, 0xc9, 0xe1, 0x70, 0xc0, 0x67, 0x1c, 0x6b, 0x0e, 0x07, 0x2d, 0xa6, 0xaf, + 0x64, 0x8c, 0x30, 0x24, 0xea, 0x42, 0xda, 0xf4, 0x52, 0x4c, 0x3c, 0x65, 0xa9, 0xa7, 0xb7, 0x87, + 0x03, 0x3e, 0xe5, 0xa7, 0x7e, 0x6e, 0x67, 0x53, 0xbe, 0x09, 0x51, 0x45, 0x1f, 0x32, 0xb0, 0x12, + 0x98, 0xf4, 0x7c, 0x8e, 0x51, 0x9f, 0xdf, 0x24, 0xd7, 0x15, 0xdf, 0xee, 0x22, 0xdd, 0xe6, 0xcc, + 0x31, 0x54, 0xbf, 0xdf, 0xc7, 0x83, 0x7e, 0xef, 0xb5, 0x32, 0x16, 0x56, 0x26, 0x9e, 0x91, 0xfe, + 0x2c, 0x88, 0x3f, 0x6e, 0x41, 0xfc, 0xcc, 0x40, 0x6e, 0xec, 0x45, 0xee, 0x82, 0x87, 0x2c, 0x64, + 0x43, 0xd6, 0x38, 0xd5, 0xb1, 0xd9, 0xf0, 0xbb, 0x4e, 0x70, 0x01, 0x48, 0xd3, 0x47, 0xc1, 0xc5, + 0xb4, 0x9e, 0xb4, 0x11, 0x60, 0xa9, 0xeb, 0x3f, 0x32, 0xb0, 0x7a, 0xde, 0x7b, 0xe7, 0x05, 0xb5, + 0x3f, 0x1d, 0x20, 0x68, 0x4d, 0x61, 0x87, 0x17, 0xd8, 0xfc, 0x42, 0x16, 0x9c, 0x6c, 0x5f, 0x3f, + 0x85, 0x58, 0xdd, 0x96, 0xed, 0xb1, 0x0b, 0x21, 0x40, 0x7c, 0xab, 0x5c, 0xaf, 0x54, 0x0f, 0x38, + 0x06, 0xe5, 0x20, 0x25, 0x54, 0xf6, 0x2a, 0x07, 0x95, 0x46, 0xad, 0xba, 0x77, 0x97, 0x8b, 0xa0, + 0xbf, 0xc1, 0x5f, 0x5c, 0xc1, 0x56, 0x55, 0x68, 0x1c, 0x49, 0xa2, 0xf7, 0x53, 0x14, 0x65, 0x01, + 0xca, 0x5b, 0xdb, 0xbb, 0x3b, 0xe2, 0xde, 0x5e, 0x45, 0xe0, 0x58, 0x94, 0x81, 0xe4, 0x9d, 0xad, + 0x3d, 0x51, 0xd8, 0x3a, 0xa8, 0x08, 0x5c, 0x8c, 0xc0, 0xee, 0x1f, 0x96, 0xf7, 0xc4, 0x6d, 0x2e, + 0x5e, 0xbe, 0x7a, 0xf6, 0x6d, 0x61, 0xe9, 0x6c, 0x58, 0x60, 0x9e, 0x0c, 0x0b, 0xcc, 0xd3, 0x61, + 0x81, 0xf9, 0x66, 0x58, 0x60, 0x3e, 0x7e, 0x56, 0x58, 0x7a, 0xf2, 0xac, 0xb0, 0xf4, 0xf4, 0x59, + 0x61, 0xe9, 0x1e, 0x4b, 0x76, 0xdd, 0x8c, 0xd3, 0x27, 0xde, 0x9b, 0xbf, 0x06, 0x00, 0x00, 0xff, + 0xff, 0x86, 0x8b, 0x96, 0x2f, 0x9b, 0x1b, 0x00, 0x00, } func (this *Column) Equal(that interface{}) bool { @@ -1109,7 +1109,7 @@ func (this *Sequence) Equal(that interface{}) bool { } else if this == nil { return false } - if this.TableID != that1.TableID { + if this.SequenceID != that1.SequenceID { return false } return true @@ -1968,8 +1968,8 @@ func (m *Sequence) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l - if m.TableID != 0 { - i = encodeVarintScpb(dAtA, i, uint64(m.TableID)) + if m.SequenceID != 0 { + i = encodeVarintScpb(dAtA, i, uint64(m.SequenceID)) i-- dAtA[i] = 0x8 } @@ -2354,8 +2354,8 @@ func (m *SequenceOwnedBy) MarshalToSizedBuffer(dAtA []byte) (int, error) { i-- dAtA[i] = 0x10 } - if m.TableID != 0 { - i = encodeVarintScpb(dAtA, i, uint64(m.TableID)) + if m.SequenceID != 0 { + i = encodeVarintScpb(dAtA, i, uint64(m.SequenceID)) i-- dAtA[i] = 0x8 } @@ -2639,8 +2639,8 @@ func (m *Sequence) Size() (n int) { } var l int _ = l - if m.TableID != 0 { - n += 1 + sovScpb(uint64(m.TableID)) + if m.SequenceID != 0 { + n += 1 + sovScpb(uint64(m.SequenceID)) } return n } @@ -2803,8 +2803,8 @@ func (m *SequenceOwnedBy) Size() (n int) { } var l int _ = l - if m.TableID != 0 { - n += 1 + sovScpb(uint64(m.TableID)) + if m.SequenceID != 0 { + n += 1 + sovScpb(uint64(m.SequenceID)) } if m.OwnerTableID != 0 { n += 1 + sovScpb(uint64(m.OwnerTableID)) @@ -4684,9 +4684,9 @@ func (m *Sequence) Unmarshal(dAtA []byte) error { switch fieldNum { case 1: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field TableID", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field SequenceID", wireType) } - m.TableID = 0 + m.SequenceID = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowScpb @@ -4696,7 +4696,7 @@ func (m *Sequence) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.TableID |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift + m.SequenceID |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift if b < 0x80 { break } @@ -5947,9 +5947,9 @@ func (m *SequenceOwnedBy) Unmarshal(dAtA []byte) error { switch fieldNum { case 1: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field TableID", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field SequenceID", wireType) } - m.TableID = 0 + m.SequenceID = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowScpb @@ -5959,7 +5959,7 @@ func (m *SequenceOwnedBy) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.TableID |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift + m.SequenceID |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift if b < 0x80 { break } diff --git a/pkg/sql/schemachanger/scpb/scpb.proto b/pkg/sql/schemachanger/scpb/scpb.proto index f3d7f065f6d2..fb9ca544d391 100644 --- a/pkg/sql/schemachanger/scpb/scpb.proto +++ b/pkg/sql/schemachanger/scpb/scpb.proto @@ -117,7 +117,7 @@ message CheckConstraint { message Sequence { option (gogoproto.equal) = true; - uint32 table_id = 1 [(gogoproto.customname) = "TableID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ID"]; + uint32 sequence_id = 1 [(gogoproto.customname) = "SequenceID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ID"]; } message DefaultExpression { @@ -166,7 +166,7 @@ message InboundForeignKey { } message SequenceOwnedBy { - uint32 table_id = 1 [(gogoproto.customname) = "TableID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ID"]; + uint32 sequence_id = 1 [(gogoproto.customname) = "SequenceID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ID"]; uint32 owner_table_id = 2 [(gogoproto.customname) = "OwnerTableID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ID"]; } diff --git a/pkg/sql/schemachanger/scplan/rules.go b/pkg/sql/schemachanger/scplan/rules.go index 9399a5f33319..1837677a93d2 100644 --- a/pkg/sql/schemachanger/scplan/rules.go +++ b/pkg/sql/schemachanger/scplan/rules.go @@ -79,7 +79,7 @@ func seqOwnedByReferencesTable(this *scpb.Table, that *scpb.SequenceOwnedBy) boo } func seqOwnedByReferencesSeq(this *scpb.SequenceOwnedBy, that *scpb.Sequence) bool { - return this.TableID == that.TableID + return this.SequenceID == that.SequenceID } func tableReferencesDefaultExpression(this *scpb.Table, that *scpb.DefaultExpression) bool { @@ -92,7 +92,7 @@ func tableReferencedByDependedOnBy(this *scpb.Table, that *scpb.RelationDepended func defaultExprReferencesColumn(this *scpb.Sequence, that *scpb.DefaultExpression) bool { for _, seq := range that.UsesSequenceIDs { - if seq == this.TableID { + if seq == this.SequenceID { return true } } @@ -136,14 +136,14 @@ var rules = map[scpb.Element]targetRules{ { predicate: func(this *scpb.SequenceOwnedBy, flags Params) bool { return flags.ExecutionPhase == StatementPhase && - !flags.CreatedDescriptorIDs.Contains(this.TableID) + !flags.CreatedDescriptorIDs.Contains(this.SequenceID) }, }, { nextState: scpb.State_ABSENT, op: func(this *scpb.SequenceOwnedBy) scop.Op { return &scop.RemoveSequenceOwnedBy{ - TableID: this.TableID, + TableID: this.SequenceID, } }, }, @@ -242,7 +242,7 @@ var rules = map[scpb.Element]targetRules{ { predicate: func(this *scpb.Sequence, flags Params) bool { return flags.ExecutionPhase == StatementPhase && - !flags.CreatedDescriptorIDs.Contains(this.TableID) + !flags.CreatedDescriptorIDs.Contains(this.SequenceID) }, }, { @@ -250,10 +250,10 @@ var rules = map[scpb.Element]targetRules{ op: func(this *scpb.Sequence) []scop.Op { ops := []scop.Op{ &scop.MarkDescriptorAsDropped{ - TableID: this.TableID, + TableID: this.SequenceID, }, &scop.CreateGcJobForDescriptor{ - DescID: this.TableID, + DescID: this.SequenceID, }, } return ops @@ -264,7 +264,7 @@ var rules = map[scpb.Element]targetRules{ { predicate: func(this *scpb.Sequence, flags Params) bool { return flags.ExecutionPhase == StatementPhase && - !flags.CreatedDescriptorIDs.Contains(this.TableID) + !flags.CreatedDescriptorIDs.Contains(this.SequenceID) }, }, { @@ -272,9 +272,9 @@ var rules = map[scpb.Element]targetRules{ op: func(this *scpb.Sequence) []scop.Op { ops := []scop.Op{ &scop.CreateGcJobForDescriptor{ - DescID: this.TableID, + DescID: this.SequenceID, }, - &scop.DrainDescriptorName{TableID: this.TableID}, + &scop.DrainDescriptorName{TableID: this.SequenceID}, } return ops },