From 6e790fbd7b5ce5cdb5aa1fa8bd037971d53972b1 Mon Sep 17 00:00:00 2001 From: Marius Posta Date: Wed, 17 Feb 2021 14:03:56 -0500 Subject: [PATCH] sql: add catalog.Mutation interface for table mutations Previously, catalog.TableDescriptor had a method, GetMutations, which returned a slice of descpb.DescriptorMutation structs, which in turn contain descpb.IndexDescriptor and descpb.ColumnDescriptor types. This is an obstacle to our ongoing effort to virtualize column and index descriptors. This commit therefore wraps descpb.DescriptorMutation in an interface type, catalog.Mutation, and also wraps the remaining mutation descriptor field types: - ConstraintToUpdate, - PrimaryKeySwap, - ComputedColumnSwap, - MaterializedViewRefresh. Release note: None --- .../schematestutils/schema_test_utils.go | 4 +- .../schemafeed/table_event_filter.go | 12 +- .../schemafeed/table_event_filter_test.go | 5 +- .../multiregionccl/regional_by_row_test.go | 4 +- pkg/jobs/registry.go | 2 +- pkg/sql/alter_column_type_test.go | 2 +- pkg/sql/alter_table_locality.go | 4 +- pkg/sql/backfill.go | 14 +- pkg/sql/backfill/backfill.go | 33 +- pkg/sql/catalog/BUILD.bazel | 1 + pkg/sql/catalog/descriptor.go | 407 +-------------- pkg/sql/catalog/table_elements.go | 486 ++++++++++++++++++ pkg/sql/catalog/tabledesc/BUILD.bazel | 1 + pkg/sql/catalog/tabledesc/column.go | 103 ++-- pkg/sql/catalog/tabledesc/index.go | 66 +-- pkg/sql/catalog/tabledesc/mutation.go | 266 ++++++++++ pkg/sql/catalog/tabledesc/safe_format.go | 2 +- pkg/sql/catalog/tabledesc/structured.go | 4 +- pkg/sql/catalog/tabledesc/table_desc.go | 30 +- .../catalog/tabledesc/table_desc_builder.go | 9 +- pkg/sql/crdb_internal.go | 2 +- pkg/sql/descriptor_mutation_test.go | 32 +- pkg/sql/rowexec/backfiller.go | 4 +- pkg/sql/rowexec/backfiller_test.go | 2 +- pkg/sql/schema_changer.go | 28 +- pkg/sql/schema_changer_test.go | 38 +- 26 files changed, 929 insertions(+), 632 deletions(-) create mode 100644 pkg/sql/catalog/table_elements.go create mode 100644 pkg/sql/catalog/tabledesc/mutation.go diff --git a/pkg/ccl/changefeedccl/schemafeed/schematestutils/schema_test_utils.go b/pkg/ccl/changefeedccl/schemafeed/schematestutils/schema_test_utils.go index aedb49eaf12b..e8f8e368d607 100644 --- a/pkg/ccl/changefeedccl/schemafeed/schematestutils/schema_test_utils.go +++ b/pkg/ccl/changefeedccl/schemafeed/schematestutils/schema_test_utils.go @@ -57,7 +57,7 @@ func AddColumnDropBackfillMutation(desc catalog.TableDescriptor) catalog.TableDe Direction: descpb.DescriptorMutation_DROP, Descriptor_: &descpb.DescriptorMutation_Column{Column: MakeColumnDesc(desc.GetNextColumnID() - 1)}, }) - return desc + return tabledesc.NewBuilder(desc.TableDesc()).BuildImmutableTable() } // AddNewColumnBackfillMutation adds a mutation to desc to add a column. @@ -70,5 +70,5 @@ func AddNewColumnBackfillMutation(desc catalog.TableDescriptor) catalog.TableDes MutationID: 0, Rollback: false, }) - return desc + return tabledesc.NewBuilder(desc.TableDesc()).BuildImmutableTable() } diff --git a/pkg/ccl/changefeedccl/schemafeed/table_event_filter.go b/pkg/ccl/changefeedccl/schemafeed/table_event_filter.go index fa47c92224e1..8473f31daebc 100644 --- a/pkg/ccl/changefeedccl/schemafeed/table_event_filter.go +++ b/pkg/ccl/changefeedccl/schemafeed/table_event_filter.go @@ -13,7 +13,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeedbase" "github.com/cockroachdb/cockroach/pkg/sql/catalog" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/errors" ) @@ -92,12 +91,11 @@ func hasNewColumnDropBackfillMutation(e TableEvent) (res bool) { } func dropColumnMutationExists(desc catalog.TableDescriptor) bool { - for _, m := range desc.GetMutations() { - if m.GetColumn() == nil { + for _, m := range desc.AllMutations() { + if m.AsColumn() == nil { continue } - if m.Direction == descpb.DescriptorMutation_DROP && - m.State == descpb.DescriptorMutation_DELETE_AND_WRITE_ONLY { + if m.Dropped() && m.WriteAndDeleteOnly() { return true } } @@ -117,8 +115,8 @@ func newColumnNoBackfill(e TableEvent) (res bool) { } func pkChangeMutationExists(desc catalog.TableDescriptor) bool { - for _, m := range desc.GetMutations() { - if m.Direction == descpb.DescriptorMutation_ADD && m.GetPrimaryKeySwap() != nil { + for _, m := range desc.AllMutations() { + if m.Adding() && m.AsPrimaryKeySwap() != nil { return true } } diff --git a/pkg/ccl/changefeedccl/schemafeed/table_event_filter_test.go b/pkg/ccl/changefeedccl/schemafeed/table_event_filter_test.go index fde8002e8a72..76a0f759e7b3 100644 --- a/pkg/ccl/changefeedccl/schemafeed/table_event_filter_test.go +++ b/pkg/ccl/changefeedccl/schemafeed/table_event_filter_test.go @@ -15,6 +15,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/schemafeed/schematestutils" "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/gogo/protobuf/proto" "github.com/stretchr/testify/require" @@ -75,10 +76,10 @@ func TestTableEventFilter(t *testing.T) { e: TableEvent{ Before: func() catalog.TableDescriptor { td := addColBackfill(mkTableDesc(42, 4, ts(4), 1)) - col := td.GetMutations()[0].GetColumn() + col := td.TableDesc().Mutations[0].GetColumn() col.Nullable = true col.ComputeExpr = proto.String("1") - return td + return tabledesc.NewBuilder(td.TableDesc()).BuildImmutableTable() }(), After: mkTableDesc(42, 4, ts(4), 2), }, diff --git a/pkg/ccl/multiregionccl/regional_by_row_test.go b/pkg/ccl/multiregionccl/regional_by_row_test.go index 7ad674654943..af25f7c682af 100644 --- a/pkg/ccl/multiregionccl/regional_by_row_test.go +++ b/pkg/ccl/multiregionccl/regional_by_row_test.go @@ -419,10 +419,10 @@ USE t; tableDesc := catalogkv.TestingGetTableDescriptor( kvDB, keys.SystemSQLCodec, "t", "test", ) - if len(tableDesc.GetMutations()) != 0 { + if len(tableDesc.AllMutations()) != 0 { return errors.Errorf( "expected 0 mutations after cancellation, found %d", - len(tableDesc.GetMutations()), + len(tableDesc.AllMutations()), ) } if tableDesc.GetPrimaryIndex().NumColumns() != len(testCase.originalPKCols) { diff --git a/pkg/jobs/registry.go b/pkg/jobs/registry.go index cb02b5670c78..1e38a3fb3872 100644 --- a/pkg/jobs/registry.go +++ b/pkg/jobs/registry.go @@ -870,7 +870,7 @@ func (r *Registry) isOrphaned(ctx context.Context, payload *jobspb.Payload) (boo if err != nil { return err } - hasAnyMutations := len(td.GetMutations()) != 0 || len(td.GetGCMutations()) != 0 + hasAnyMutations := len(td.AllMutations()) != 0 || len(td.GetGCMutations()) != 0 hasDropJob := td.TableDesc().DropJobID != 0 pendingMutations = hasAnyMutations || hasDropJob return nil diff --git a/pkg/sql/alter_column_type_test.go b/pkg/sql/alter_column_type_test.go index 04f04e5b6462..4f0db407fb1b 100644 --- a/pkg/sql/alter_column_type_test.go +++ b/pkg/sql/alter_column_type_test.go @@ -278,7 +278,7 @@ ALTER TABLE t.test ALTER COLUMN x TYPE INT; // Ensure that the add column and column swap mutations are cleaned up. testutils.SucceedsSoon(t, func() error { desc := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "test") - if len(desc.GetMutations()) != 0 { + if len(desc.AllMutations()) != 0 { return errors.New("expected no mutations on TableDescriptor") } return nil diff --git a/pkg/sql/alter_table_locality.go b/pkg/sql/alter_table_locality.go index a13aea66e903..7ade25472104 100644 --- a/pkg/sql/alter_table_locality.go +++ b/pkg/sql/alter_table_locality.go @@ -348,7 +348,7 @@ func (n *alterTableSetLocalityNode) alterTableLocalityToRegionalByRow( } // Allow add column mutation to be on the same mutation ID in AlterPrimaryKey. - mutationIdx := len(n.tableDesc.GetMutations()) - 1 + mutationIdx := len(n.tableDesc.Mutations) - 1 mutationIdxAllowedInSameTxn = &mutationIdx newColumnName = &partColName @@ -363,7 +363,7 @@ func (n *alterTableSetLocalityNode) alterTableLocalityToRegionalByRow( // The primary_region default helps us also have a material value. // This can be removed when the default_expr can serialize user defined // functions. - col := n.tableDesc.GetMutations()[mutationIdx].GetColumn() + col := n.tableDesc.Mutations[mutationIdx].GetColumn() finalDefaultExpr, err := schemaexpr.SanitizeVarFreeExpr( params.ctx, regionalByRowGatewayRegionDefaultExpr(enumOID), diff --git a/pkg/sql/backfill.go b/pkg/sql/backfill.go index 3248d3108ac6..1acb81c43d1d 100644 --- a/pkg/sql/backfill.go +++ b/pkg/sql/backfill.go @@ -1426,19 +1426,19 @@ func (sc *SchemaChanger) validateIndexes(ctx context.Context) error { var forwardIndexes []*descpb.IndexDescriptor var invertedIndexes []*descpb.IndexDescriptor - for _, m := range tableDesc.GetMutations() { - if sc.mutationID != m.MutationID { + for _, m := range tableDesc.AllMutations() { + if sc.mutationID != m.MutationID() { break } - idx := m.GetIndex() - if idx == nil || m.Direction == descpb.DescriptorMutation_DROP { + idx := m.AsIndex() + if idx == nil || idx.Dropped() { continue } - switch idx.Type { + switch idx.GetType() { case descpb.IndexDescriptor_FORWARD: - forwardIndexes = append(forwardIndexes, idx) + forwardIndexes = append(forwardIndexes, idx.IndexDesc()) case descpb.IndexDescriptor_INVERTED: - invertedIndexes = append(invertedIndexes, idx) + invertedIndexes = append(invertedIndexes, idx.IndexDesc()) } } if len(forwardIndexes) == 0 && len(invertedIndexes) == 0 { diff --git a/pkg/sql/backfill/backfill.go b/pkg/sql/backfill/backfill.go index 769177169999..79119513eb73 100644 --- a/pkg/sql/backfill/backfill.go +++ b/pkg/sql/backfill/backfill.go @@ -38,18 +38,17 @@ import ( ) // MutationFilter is the type of a simple predicate on a mutation. -type MutationFilter func(descpb.DescriptorMutation) bool +type MutationFilter func(catalog.Mutation) bool // ColumnMutationFilter is a filter that allows mutations that add or drop // columns. -func ColumnMutationFilter(m descpb.DescriptorMutation) bool { - return m.GetColumn() != nil && - (m.Direction == descpb.DescriptorMutation_ADD || m.Direction == descpb.DescriptorMutation_DROP) +func ColumnMutationFilter(m catalog.Mutation) bool { + return m.AsColumn() != nil && (m.Adding() || m.Dropped()) } // IndexMutationFilter is a filter that allows mutations that add indexes. -func IndexMutationFilter(m descpb.DescriptorMutation) bool { - return m.GetIndex() != nil && m.Direction == descpb.DescriptorMutation_ADD +func IndexMutationFilter(m catalog.Mutation) bool { + return m.AsIndex() != nil && m.Adding() } // ColumnBackfiller is capable of running a column backfill for all @@ -72,14 +71,13 @@ type ColumnBackfiller struct { // initCols is a helper to populate some column metadata on a ColumnBackfiller. func (cb *ColumnBackfiller) initCols(desc catalog.TableDescriptor) { - for _, m := range desc.GetMutations() { + for _, m := range desc.AllMutations() { if ColumnMutationFilter(m) { - desc := *m.GetColumn() - switch m.Direction { - case descpb.DescriptorMutation_ADD: - cb.added = append(cb.added, desc) - case descpb.DescriptorMutation_DROP: - cb.dropped = append(cb.dropped, desc) + col := *m.AsColumn().ColumnDesc() + if m.Adding() { + cb.added = append(cb.added, col) + } else if m.Dropped() { + cb.dropped = append(cb.dropped, col) } } } @@ -692,16 +690,17 @@ func (ib *IndexBackfiller) initCols(desc catalog.TableDescriptor) { // fetched in order to backfill the added indexes. func (ib *IndexBackfiller) initIndexes(desc catalog.TableDescriptor) util.FastIntSet { var valNeededForCol util.FastIntSet - mutationID := desc.GetMutations()[0].MutationID + mutations := desc.AllMutations() + mutationID := mutations[0].MutationID() // Mutations in the same transaction have the same ID. Loop through the // mutations and collect all index mutations. - for _, m := range desc.GetMutations() { - if m.MutationID != mutationID { + for _, m := range mutations { + if m.MutationID() != mutationID { break } if IndexMutationFilter(m) { - idx := m.GetIndex() + idx := m.AsIndex().IndexDesc() ib.added = append(ib.added, idx) for i := range ib.cols { id := ib.cols[i].ID diff --git a/pkg/sql/catalog/BUILD.bazel b/pkg/sql/catalog/BUILD.bazel index 1987e1582366..15aa1a145860 100644 --- a/pkg/sql/catalog/BUILD.bazel +++ b/pkg/sql/catalog/BUILD.bazel @@ -11,6 +11,7 @@ go_library( "errors.go", "table_col_map.go", "table_col_set.go", + "table_elements.go", "validate.go", ], importpath = "github.com/cockroachdb/cockroach/pkg/sql/catalog", diff --git a/pkg/sql/catalog/descriptor.go b/pkg/sql/catalog/descriptor.go index adf07ae28a2a..d07bc679fa79 100644 --- a/pkg/sql/catalog/descriptor.go +++ b/pkg/sql/catalog/descriptor.go @@ -13,7 +13,6 @@ package catalog import ( "context" - "github.com/cockroachdb/cockroach/pkg/geo/geoindex" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" @@ -22,7 +21,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/hlc" - "github.com/cockroachdb/cockroach/pkg/util/iterutil" "github.com/cockroachdb/errors" "github.com/cockroachdb/redact" ) @@ -299,7 +297,7 @@ type TableDescriptor interface { HasColumnBackfillMutation() bool MakeFirstMutationPublic(includeConstraints bool) (TableDescriptor, error) - GetMutations() []descpb.DescriptorMutation + AllMutations() []Mutation GetGCMutations() []descpb.TableDescriptor_GCDescriptorMutation GetMutationJobs() []descpb.TableDescriptor_MutationJob @@ -331,217 +329,6 @@ type TableDescriptor interface { GetRegionalByRowTableRegionColumnName() (tree.Name, error) } -// Index is an interface around the index descriptor types. -type Index interface { - - // IndexDesc returns the underlying protobuf descriptor. - // Ideally, this method should be called as rarely as possible. - IndexDesc() *descpb.IndexDescriptor - - // IndexDescDeepCopy returns a deep copy of the underlying proto. - IndexDescDeepCopy() descpb.IndexDescriptor - - // Ordinal returns the ordinal of the index in its parent table descriptor. - // - // The ordinal of an index in a `tableDesc descpb.TableDescriptor` is - // defined as follows: - // - 0 is the ordinal of the primary index, - // - [1:1+len(tableDesc.Indexes)] is the range of public non-primary indexes, - // - [1+len(tableDesc.Indexes):] is the range of non-public indexes. - // - // In terms of a `table catalog.TableDescriptor` interface, it is defined - // as the catalog.Index object's position in the table.AllIndexes() slice. - Ordinal() int - - // Primary returns true iff the index is the primary index for the table - // descriptor. - Primary() bool - - // Public returns true iff the index is active, i.e. readable, in the table - // descriptor. - Public() bool - - // WriteAndDeleteOnly returns true iff the index is a mutation in the - // delete-and-write-only state in the table descriptor. - WriteAndDeleteOnly() bool - - // DeleteOnly returns true iff the index is a mutation in the delete-only - // state in the table descriptor. - DeleteOnly() bool - - // Adding returns true iff the index is an add mutation in the table - // descriptor. - Adding() bool - - // Dropped returns true iff the index is a drop mutation in the table - // descriptor. - Dropped() bool - - // The remaining methods operate on the underlying descpb.IndexDescriptor object. - - GetID() descpb.IndexID - GetName() string - IsInterleaved() bool - IsPartial() bool - IsUnique() bool - IsDisabled() bool - IsSharded() bool - IsCreatedExplicitly() bool - GetPredicate() string - GetType() descpb.IndexDescriptor_Type - GetGeoConfig() geoindex.Config - GetVersion() descpb.IndexDescriptorVersion - GetEncodingType() descpb.IndexDescriptorEncodingType - - GetSharded() descpb.ShardedDescriptor - GetShardColumnName() string - - IsValidOriginIndex(originColIDs descpb.ColumnIDs) bool - IsValidReferencedUniqueConstraint(referencedColIDs descpb.ColumnIDs) bool - - GetPartitioning() descpb.PartitioningDescriptor - FindPartitionByName(name string) descpb.PartitioningDescriptor - PartitionNames() []string - - NumInterleaveAncestors() int - GetInterleaveAncestor(ancestorOrdinal int) descpb.InterleaveDescriptor_Ancestor - - NumInterleavedBy() int - GetInterleavedBy(interleavedByOrdinal int) descpb.ForeignKeyReference - - NumColumns() int - GetColumnID(columnOrdinal int) descpb.ColumnID - GetColumnName(columnOrdinal int) string - GetColumnDirection(columnOrdinal int) descpb.IndexDescriptor_Direction - - ForEachColumnID(func(id descpb.ColumnID) error) error - ContainsColumnID(colID descpb.ColumnID) bool - InvertedColumnID() descpb.ColumnID - InvertedColumnName() string - - NumStoredColumns() int - GetStoredColumnID(storedColumnOrdinal int) descpb.ColumnID - GetStoredColumnName(storedColumnOrdinal int) string - HasOldStoredColumns() bool - - NumExtraColumns() int - GetExtraColumnID(extraColumnOrdinal int) descpb.ColumnID - - NumCompositeColumns() int - GetCompositeColumnID(compositeColumnOrdinal int) descpb.ColumnID -} - -// Column is an interface around the index descriptor types. -type Column interface { - - // ColumnDesc returns the underlying protobuf descriptor. - // Ideally, this method should be called as rarely as possible. - ColumnDesc() *descpb.ColumnDescriptor - - // ColumnDescDeepCopy returns a deep copy of the underlying proto. - ColumnDescDeepCopy() descpb.ColumnDescriptor - - // DeepCopy returns a deep copy of the receiver. - DeepCopy() Column - - // Ordinal returns the ordinal of the column in its parent table descriptor. - // - // The ordinal of a column in a `tableDesc descpb.TableDescriptor` is - // defined as follows: - // - [0:len(tableDesc.Columns)] is the range of public columns, - // - [len(tableDesc.Columns):] is the range of non-public columns. - // - // In terms of a `table catalog.TableDescriptor` interface, it is defined - // as the catalog.Column object's position in the table.AllColumns() slice. - Ordinal() int - - // Public returns true iff the column is active, i.e. readable, in the table - // descriptor. - Public() bool - - // WriteAndDeleteOnly returns true iff the column is a mutation in the - // delete-and-write-only state in the table descriptor. - WriteAndDeleteOnly() bool - - // DeleteOnly returns true iff the column is a mutation in the delete-only - // state in the table descriptor. - DeleteOnly() bool - - // Adding returns true iff the column is an add mutation in the table - // descriptor. - Adding() bool - - // Dropped returns true iff the column is a drop mutation in the table - // descriptor. - Dropped() bool - - // GetID returns the column ID. - GetID() descpb.ColumnID - - // GetName returns the column name as a string. - GetName() string - - // ColName returns the column name as a tree.Name. - ColName() tree.Name - - // HasType returns true iff the column type is set. - HasType() bool - - // GetType returns the column type. - GetType() *types.T - - // IsNullable returns true iff the column allows NULL values. - IsNullable() bool - - // HasDefault returns true iff the column has a default expression set. - HasDefault() bool - - // GetDefaultExpr returns the column default expression if it exists, - // empty string otherwise. - GetDefaultExpr() string - - // IsComputed returns true iff the column is a computed column. - IsComputed() bool - - // GetComputeExpr returns the column computed expression if it exists, - // empty string otherwise. - GetComputeExpr() string - - // IsHidden returns true iff the column is not visible. - IsHidden() bool - - // NumUsesSequences returns the number of sequences used by this column. - NumUsesSequences() int - - // GetUsesSequenceID returns the ID of a sequence used by this column. - GetUsesSequenceID(usesSequenceOrdinal int) descpb.ID - - // NumOwnsSequences returns the number of sequences owned by this column. - NumOwnsSequences() int - - // GetOwnsSequenceID returns the ID of a sequence owned by this column. - GetOwnsSequenceID(ownsSequenceOrdinal int) descpb.ID - - // IsVirtual returns true iff the column is a virtual column. - IsVirtual() bool - - // CheckCanBeInboundFKRef returns whether the given column can be on the - // referenced (target) side of a foreign key relation. - CheckCanBeInboundFKRef() error - - // CheckCanBeOutboundFKRef returns whether the given column can be on the - // referencing (origin) side of a foreign key relation. - CheckCanBeOutboundFKRef() error - - // GetPGAttributeNum returns the PGAttributeNum of the column descriptor - // if the PGAttributeNum is set (non-zero). Returns the ID of the - // column descriptor if the PGAttributeNum is not set. - GetPGAttributeNum() uint32 - - // IsSystemColumn returns true iff the column is a system column. - IsSystemColumn() bool -} - // TypeDescriptor will eventually be called typedesc.Descriptor. // It is implemented by (Imm|M)utableTypeDescriptor. type TypeDescriptor interface { @@ -637,195 +424,3 @@ func FormatSafeDescriptorProperties(w *redact.StringBuilder, desc Descriptor) { w.Printf(", NumDrainingNames: %d", len(drainingNames)) } } - -func isIndexInSearchSet(desc TableDescriptor, opts IndexOpts, idx Index) bool { - if !opts.NonPhysicalPrimaryIndex && idx.Primary() && !desc.IsPhysicalTable() { - return false - } - if !opts.AddMutations && idx.Adding() { - return false - } - if !opts.DropMutations && idx.Dropped() { - return false - } - return true -} - -// ForEachIndex runs f over each index in the table descriptor according to -// filter parameters in opts. Indexes are visited in their canonical order, -// see Index.Ordinal(). ForEachIndex supports iterutil.StopIteration(). -func ForEachIndex(desc TableDescriptor, opts IndexOpts, f func(idx Index) error) error { - for _, idx := range desc.AllIndexes() { - if !isIndexInSearchSet(desc, opts, idx) { - continue - } - if err := f(idx); err != nil { - if iterutil.Done(err) { - return nil - } - return err - } - } - return nil -} - -func forEachIndex(slice []Index, f func(idx Index) error) error { - for _, idx := range slice { - if err := f(idx); err != nil { - if iterutil.Done(err) { - return nil - } - return err - } - } - return nil -} - -// ForEachActiveIndex is like ForEachIndex over ActiveIndexes(). -func ForEachActiveIndex(desc TableDescriptor, f func(idx Index) error) error { - return forEachIndex(desc.ActiveIndexes(), f) -} - -// ForEachNonDropIndex is like ForEachIndex over NonDropIndexes(). -func ForEachNonDropIndex(desc TableDescriptor, f func(idx Index) error) error { - return forEachIndex(desc.NonDropIndexes(), f) -} - -// ForEachPartialIndex is like ForEachIndex over PartialIndexes(). -func ForEachPartialIndex(desc TableDescriptor, f func(idx Index) error) error { - return forEachIndex(desc.PartialIndexes(), f) -} - -// ForEachPublicNonPrimaryIndex is like ForEachIndex over -// PublicNonPrimaryIndexes(). -func ForEachPublicNonPrimaryIndex(desc TableDescriptor, f func(idx Index) error) error { - return forEachIndex(desc.PublicNonPrimaryIndexes(), f) -} - -// ForEachWritableNonPrimaryIndex is like ForEachIndex over -// WritableNonPrimaryIndexes(). -func ForEachWritableNonPrimaryIndex(desc TableDescriptor, f func(idx Index) error) error { - return forEachIndex(desc.WritableNonPrimaryIndexes(), f) -} - -// ForEachDeletableNonPrimaryIndex is like ForEachIndex over -// DeletableNonPrimaryIndexes(). -func ForEachDeletableNonPrimaryIndex(desc TableDescriptor, f func(idx Index) error) error { - return forEachIndex(desc.DeletableNonPrimaryIndexes(), f) -} - -// ForEachDeleteOnlyNonPrimaryIndex is like ForEachIndex over -// DeleteOnlyNonPrimaryIndexes(). -func ForEachDeleteOnlyNonPrimaryIndex(desc TableDescriptor, f func(idx Index) error) error { - return forEachIndex(desc.DeleteOnlyNonPrimaryIndexes(), f) -} - -// FindIndex returns the first index for which test returns true, nil otherwise, -// according to the parameters in opts just like ForEachIndex. -// Indexes are visited in their canonical order, see Index.Ordinal(). -func FindIndex(desc TableDescriptor, opts IndexOpts, test func(idx Index) bool) Index { - for _, idx := range desc.AllIndexes() { - if !isIndexInSearchSet(desc, opts, idx) { - continue - } - if test(idx) { - return idx - } - } - return nil -} - -func findIndex(slice []Index, test func(idx Index) bool) Index { - for _, idx := range slice { - if test(idx) { - return idx - } - } - return nil -} - -// FindActiveIndex returns the first index in ActiveIndex() for which test -// returns true. -func FindActiveIndex(desc TableDescriptor, test func(idx Index) bool) Index { - return findIndex(desc.ActiveIndexes(), test) -} - -// FindNonDropIndex returns the first index in NonDropIndex() for which test -// returns true. -func FindNonDropIndex(desc TableDescriptor, test func(idx Index) bool) Index { - return findIndex(desc.NonDropIndexes(), test) -} - -// FindPartialIndex returns the first index in PartialIndex() for which test -// returns true. -func FindPartialIndex(desc TableDescriptor, test func(idx Index) bool) Index { - return findIndex(desc.PartialIndexes(), test) -} - -// FindPublicNonPrimaryIndex returns the first index in PublicNonPrimaryIndex() -// for which test returns true. -func FindPublicNonPrimaryIndex(desc TableDescriptor, test func(idx Index) bool) Index { - return findIndex(desc.PublicNonPrimaryIndexes(), test) -} - -// FindWritableNonPrimaryIndex returns the first index in -// WritableNonPrimaryIndex() for which test returns true. -func FindWritableNonPrimaryIndex(desc TableDescriptor, test func(idx Index) bool) Index { - return findIndex(desc.WritableNonPrimaryIndexes(), test) -} - -// FindDeletableNonPrimaryIndex returns the first index in -// DeletableNonPrimaryIndex() for which test returns true. -func FindDeletableNonPrimaryIndex(desc TableDescriptor, test func(idx Index) bool) Index { - return findIndex(desc.DeletableNonPrimaryIndexes(), test) -} - -// FindDeleteOnlyNonPrimaryIndex returns the first index in -// DeleteOnlyNonPrimaryIndex() for which test returns true. -func FindDeleteOnlyNonPrimaryIndex(desc TableDescriptor, test func(idx Index) bool) Index { - return findIndex(desc.DeleteOnlyNonPrimaryIndexes(), test) -} - -// UserDefinedTypeColsHaveSameVersion returns whether one table descriptor's -// columns with user defined type metadata have the same versions of metadata -// as in the other descriptor. Note that this function is only valid on two -// descriptors representing the same table at the same version. -func UserDefinedTypeColsHaveSameVersion(desc TableDescriptor, otherDesc TableDescriptor) bool { - otherCols := otherDesc.UserDefinedTypeColumns() - for i, thisCol := range desc.UserDefinedTypeColumns() { - this, other := thisCol.GetType(), otherCols[i].GetType() - if this.TypeMeta.Version != other.TypeMeta.Version { - return false - } - } - return true -} - -// ColumnIDToOrdinalMap returns a map from Column ID to the ordinal -// position of that column. -func ColumnIDToOrdinalMap(columns []Column) TableColMap { - var m TableColMap - for _, col := range columns { - m.Set(col.GetID(), col.Ordinal()) - } - return m -} - -// ColumnTypes returns the types of the given columns -func ColumnTypes(columns []Column) []*types.T { - return ColumnTypesWithVirtualCol(columns, nil) -} - -// ColumnTypesWithVirtualCol returns the types of all given columns, -// If virtualCol is non-nil, substitutes the type of the virtual -// column instead of the column with the same ID. -func ColumnTypesWithVirtualCol(columns []Column, virtualCol Column) []*types.T { - t := make([]*types.T, len(columns)) - for i, col := range columns { - t[i] = col.GetType() - if virtualCol != nil && col.GetID() == virtualCol.GetID() { - t[i] = virtualCol.GetType() - } - } - return t -} diff --git a/pkg/sql/catalog/table_elements.go b/pkg/sql/catalog/table_elements.go new file mode 100644 index 000000000000..8a8d566aae29 --- /dev/null +++ b/pkg/sql/catalog/table_elements.go @@ -0,0 +1,486 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package catalog + +import ( + "github.com/cockroachdb/cockroach/pkg/geo/geoindex" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util/iterutil" +) + +// TableElementMaybeMutation is an interface used as a subtype for the various +// table descriptor elements which may be present in a mutation. +type TableElementMaybeMutation interface { + // IsMutation returns true iff this table element is in a mutation. + IsMutation() bool + + // IsRollback returns true iff the table element is in a rollback mutation. + IsRollback() bool + + // MutationID returns the table element's mutationID if applicable, + // descpb.InvalidMutationID otherwise. + MutationID() descpb.MutationID + + // WriteAndDeleteOnly returns true iff the table element is in a mutation in + // the delete-and-write-only state. + WriteAndDeleteOnly() bool + + // DeleteOnly returns true iff the table element is in a mutation in the + // delete-only state. + DeleteOnly() bool + + // Adding returns true iff the table element is in an add mutation. + Adding() bool + + // Dropped returns true iff the table element is in a drop mutation. + Dropped() bool +} + +// Mutation is an interface around a table descriptor mutation. +type Mutation interface { + TableElementMaybeMutation + + // AsColumn returns the corresponding Column if the mutation is on a column, + // nil otherwise. + AsColumn() Column + + // AsIndex returns the corresponding Index if the mutation is on an index, + // nil otherwise. + AsIndex() Index + + // AsConstraint returns the corresponding ConstraintToUpdate if the mutation + // is on a constraint, nil otherwise. + AsConstraint() ConstraintToUpdate + + // AsPrimaryKeySwap returns the corresponding PrimaryKeySwap if the mutation + // is a primary key swap, nil otherwise. + AsPrimaryKeySwap() PrimaryKeySwap + + // AsComputedColumnSwap returns the corresponding ComputedColumnSwap if the + // mutation is a computed column swap, nil otherwise. + AsComputedColumnSwap() ComputedColumnSwap + + // AsMaterializedViewRefresh returns the corresponding MaterializedViewRefresh + // if the mutation is a materialized view refresh, nil otherwise. + AsMaterializedViewRefresh() MaterializedViewRefresh + + // MutationOrdinal returns the ordinal of the mutation in the underlying table + // descriptor's Mutations slice. + MutationOrdinal() int +} + +// Index is an interface around the index descriptor types. +type Index interface { + TableElementMaybeMutation + + // IndexDesc returns the underlying protobuf descriptor. + // Ideally, this method should be called as rarely as possible. + IndexDesc() *descpb.IndexDescriptor + + // IndexDescDeepCopy returns a deep copy of the underlying proto. + IndexDescDeepCopy() descpb.IndexDescriptor + + // Ordinal returns the ordinal of the index in its parent table descriptor. + // + // The ordinal of an index in a `tableDesc descpb.TableDescriptor` is + // defined as follows: + // - 0 is the ordinal of the primary index, + // - [1:1+len(tableDesc.Indexes)] is the range of public non-primary indexes, + // - [1+len(tableDesc.Indexes):] is the range of non-public indexes. + // + // In terms of a `table catalog.TableDescriptor` interface, it is defined + // as the catalog.Index object's position in the table.AllIndexes() slice. + Ordinal() int + + // Primary returns true iff the index is the primary index for the table + // descriptor. + Primary() bool + + // Public returns true iff the index is active, i.e. readable, in the table + // descriptor. + Public() bool + + // The remaining methods operate on the underlying descpb.IndexDescriptor object. + + GetID() descpb.IndexID + GetName() string + IsInterleaved() bool + IsPartial() bool + IsUnique() bool + IsDisabled() bool + IsSharded() bool + IsCreatedExplicitly() bool + GetPredicate() string + GetType() descpb.IndexDescriptor_Type + GetGeoConfig() geoindex.Config + GetVersion() descpb.IndexDescriptorVersion + GetEncodingType() descpb.IndexDescriptorEncodingType + + GetSharded() descpb.ShardedDescriptor + GetShardColumnName() string + + IsValidOriginIndex(originColIDs descpb.ColumnIDs) bool + IsValidReferencedUniqueConstraint(referencedColIDs descpb.ColumnIDs) bool + + GetPartitioning() descpb.PartitioningDescriptor + FindPartitionByName(name string) descpb.PartitioningDescriptor + PartitionNames() []string + + NumInterleaveAncestors() int + GetInterleaveAncestor(ancestorOrdinal int) descpb.InterleaveDescriptor_Ancestor + + NumInterleavedBy() int + GetInterleavedBy(interleavedByOrdinal int) descpb.ForeignKeyReference + + NumColumns() int + GetColumnID(columnOrdinal int) descpb.ColumnID + GetColumnName(columnOrdinal int) string + GetColumnDirection(columnOrdinal int) descpb.IndexDescriptor_Direction + + ForEachColumnID(func(id descpb.ColumnID) error) error + ContainsColumnID(colID descpb.ColumnID) bool + InvertedColumnID() descpb.ColumnID + InvertedColumnName() string + + NumStoredColumns() int + GetStoredColumnID(storedColumnOrdinal int) descpb.ColumnID + GetStoredColumnName(storedColumnOrdinal int) string + HasOldStoredColumns() bool + + NumExtraColumns() int + GetExtraColumnID(extraColumnOrdinal int) descpb.ColumnID + + NumCompositeColumns() int + GetCompositeColumnID(compositeColumnOrdinal int) descpb.ColumnID +} + +// Column is an interface around the index descriptor types. +type Column interface { + TableElementMaybeMutation + + // ColumnDesc returns the underlying protobuf descriptor. + // Ideally, this method should be called as rarely as possible. + ColumnDesc() *descpb.ColumnDescriptor + + // ColumnDescDeepCopy returns a deep copy of the underlying proto. + ColumnDescDeepCopy() descpb.ColumnDescriptor + + // DeepCopy returns a deep copy of the receiver. + DeepCopy() Column + + // Ordinal returns the ordinal of the column in its parent table descriptor. + // + // The ordinal of a column in a `tableDesc descpb.TableDescriptor` is + // defined as follows: + // - [0:len(tableDesc.Columns)] is the range of public columns, + // - [len(tableDesc.Columns):] is the range of non-public columns. + // + // In terms of a `table catalog.TableDescriptor` interface, it is defined + // as the catalog.Column object's position in the table.AllColumns() slice. + Ordinal() int + + // Public returns true iff the column is active, i.e. readable, in the table + // descriptor. + Public() bool + + // GetID returns the column ID. + GetID() descpb.ColumnID + + // GetName returns the column name as a string. + GetName() string + + // ColName returns the column name as a tree.Name. + ColName() tree.Name + + // HasType returns true iff the column type is set. + HasType() bool + + // GetType returns the column type. + GetType() *types.T + + // IsNullable returns true iff the column allows NULL values. + IsNullable() bool + + // HasDefault returns true iff the column has a default expression set. + HasDefault() bool + + // GetDefaultExpr returns the column default expression if it exists, + // empty string otherwise. + GetDefaultExpr() string + + // IsComputed returns true iff the column is a computed column. + IsComputed() bool + + // GetComputeExpr returns the column computed expression if it exists, + // empty string otherwise. + GetComputeExpr() string + + // IsHidden returns true iff the column is not visible. + IsHidden() bool + + // NumUsesSequences returns the number of sequences used by this column. + NumUsesSequences() int + + // GetUsesSequenceID returns the ID of a sequence used by this column. + GetUsesSequenceID(usesSequenceOrdinal int) descpb.ID + + // NumOwnsSequences returns the number of sequences owned by this column. + NumOwnsSequences() int + + // GetOwnsSequenceID returns the ID of a sequence owned by this column. + GetOwnsSequenceID(ownsSequenceOrdinal int) descpb.ID + + // IsVirtual returns true iff the column is a virtual column. + IsVirtual() bool + + // CheckCanBeInboundFKRef returns whether the given column can be on the + // referenced (target) side of a foreign key relation. + CheckCanBeInboundFKRef() error + + // CheckCanBeOutboundFKRef returns whether the given column can be on the + // referencing (origin) side of a foreign key relation. + CheckCanBeOutboundFKRef() error + + // GetPGAttributeNum returns the PGAttributeNum of the column descriptor + // if the PGAttributeNum is set (non-zero). Returns the ID of the + // column descriptor if the PGAttributeNum is not set. + GetPGAttributeNum() uint32 + + // IsSystemColumn returns true iff the column is a system column. + IsSystemColumn() bool +} + +// ConstraintToUpdate is an interface around a constraint mutation. +type ConstraintToUpdate interface { + TableElementMaybeMutation + + // ConstraintToUpdateDesc returns the underlying protobuf descriptor. + ConstraintToUpdateDesc() *descpb.ConstraintToUpdate +} + +// PrimaryKeySwap is an interface around a primary key swap mutation. +type PrimaryKeySwap interface { + TableElementMaybeMutation + + // PrimaryKeySwapDesc returns the underlying protobuf descriptor. + PrimaryKeySwapDesc() *descpb.PrimaryKeySwap +} + +// ComputedColumnSwap is an interface around a computed column swap mutation. +type ComputedColumnSwap interface { + TableElementMaybeMutation + + // ComputedColumnSwapDesc returns the underlying protobuf descriptor. + ComputedColumnSwapDesc() *descpb.ComputedColumnSwap +} + +// MaterializedViewRefresh is an interface around a materialized view refresh +// mutation. +type MaterializedViewRefresh interface { + TableElementMaybeMutation + + // MaterializedViewRefreshDesc returns the underlying protobuf descriptor. + MaterializedViewRefreshDesc() *descpb.MaterializedViewRefresh +} + +func isIndexInSearchSet(desc TableDescriptor, opts IndexOpts, idx Index) bool { + if !opts.NonPhysicalPrimaryIndex && idx.Primary() && !desc.IsPhysicalTable() { + return false + } + if !opts.AddMutations && idx.Adding() { + return false + } + if !opts.DropMutations && idx.Dropped() { + return false + } + return true +} + +// ForEachIndex runs f over each index in the table descriptor according to +// filter parameters in opts. Indexes are visited in their canonical order, +// see Index.Ordinal(). ForEachIndex supports iterutil.StopIteration(). +func ForEachIndex(desc TableDescriptor, opts IndexOpts, f func(idx Index) error) error { + for _, idx := range desc.AllIndexes() { + if !isIndexInSearchSet(desc, opts, idx) { + continue + } + if err := f(idx); err != nil { + if iterutil.Done(err) { + return nil + } + return err + } + } + return nil +} + +func forEachIndex(slice []Index, f func(idx Index) error) error { + for _, idx := range slice { + if err := f(idx); err != nil { + if iterutil.Done(err) { + return nil + } + return err + } + } + return nil +} + +// ForEachActiveIndex is like ForEachIndex over ActiveIndexes(). +func ForEachActiveIndex(desc TableDescriptor, f func(idx Index) error) error { + return forEachIndex(desc.ActiveIndexes(), f) +} + +// ForEachNonDropIndex is like ForEachIndex over NonDropIndexes(). +func ForEachNonDropIndex(desc TableDescriptor, f func(idx Index) error) error { + return forEachIndex(desc.NonDropIndexes(), f) +} + +// ForEachPartialIndex is like ForEachIndex over PartialIndexes(). +func ForEachPartialIndex(desc TableDescriptor, f func(idx Index) error) error { + return forEachIndex(desc.PartialIndexes(), f) +} + +// ForEachPublicNonPrimaryIndex is like ForEachIndex over +// PublicNonPrimaryIndexes(). +func ForEachPublicNonPrimaryIndex(desc TableDescriptor, f func(idx Index) error) error { + return forEachIndex(desc.PublicNonPrimaryIndexes(), f) +} + +// ForEachWritableNonPrimaryIndex is like ForEachIndex over +// WritableNonPrimaryIndexes(). +func ForEachWritableNonPrimaryIndex(desc TableDescriptor, f func(idx Index) error) error { + return forEachIndex(desc.WritableNonPrimaryIndexes(), f) +} + +// ForEachDeletableNonPrimaryIndex is like ForEachIndex over +// DeletableNonPrimaryIndexes(). +func ForEachDeletableNonPrimaryIndex(desc TableDescriptor, f func(idx Index) error) error { + return forEachIndex(desc.DeletableNonPrimaryIndexes(), f) +} + +// ForEachDeleteOnlyNonPrimaryIndex is like ForEachIndex over +// DeleteOnlyNonPrimaryIndexes(). +func ForEachDeleteOnlyNonPrimaryIndex(desc TableDescriptor, f func(idx Index) error) error { + return forEachIndex(desc.DeleteOnlyNonPrimaryIndexes(), f) +} + +// FindIndex returns the first index for which test returns true, nil otherwise, +// according to the parameters in opts just like ForEachIndex. +// Indexes are visited in their canonical order, see Index.Ordinal(). +func FindIndex(desc TableDescriptor, opts IndexOpts, test func(idx Index) bool) Index { + for _, idx := range desc.AllIndexes() { + if !isIndexInSearchSet(desc, opts, idx) { + continue + } + if test(idx) { + return idx + } + } + return nil +} + +func findIndex(slice []Index, test func(idx Index) bool) Index { + for _, idx := range slice { + if test(idx) { + return idx + } + } + return nil +} + +// FindActiveIndex returns the first index in ActiveIndex() for which test +// returns true. +func FindActiveIndex(desc TableDescriptor, test func(idx Index) bool) Index { + return findIndex(desc.ActiveIndexes(), test) +} + +// FindNonDropIndex returns the first index in NonDropIndex() for which test +// returns true. +func FindNonDropIndex(desc TableDescriptor, test func(idx Index) bool) Index { + return findIndex(desc.NonDropIndexes(), test) +} + +// FindPartialIndex returns the first index in PartialIndex() for which test +// returns true. +func FindPartialIndex(desc TableDescriptor, test func(idx Index) bool) Index { + return findIndex(desc.PartialIndexes(), test) +} + +// FindPublicNonPrimaryIndex returns the first index in PublicNonPrimaryIndex() +// for which test returns true. +func FindPublicNonPrimaryIndex(desc TableDescriptor, test func(idx Index) bool) Index { + return findIndex(desc.PublicNonPrimaryIndexes(), test) +} + +// FindWritableNonPrimaryIndex returns the first index in +// WritableNonPrimaryIndex() for which test returns true. +func FindWritableNonPrimaryIndex(desc TableDescriptor, test func(idx Index) bool) Index { + return findIndex(desc.WritableNonPrimaryIndexes(), test) +} + +// FindDeletableNonPrimaryIndex returns the first index in +// DeletableNonPrimaryIndex() for which test returns true. +func FindDeletableNonPrimaryIndex(desc TableDescriptor, test func(idx Index) bool) Index { + return findIndex(desc.DeletableNonPrimaryIndexes(), test) +} + +// FindDeleteOnlyNonPrimaryIndex returns the first index in +// DeleteOnlyNonPrimaryIndex() for which test returns true. +func FindDeleteOnlyNonPrimaryIndex(desc TableDescriptor, test func(idx Index) bool) Index { + return findIndex(desc.DeleteOnlyNonPrimaryIndexes(), test) +} + +// UserDefinedTypeColsHaveSameVersion returns whether one table descriptor's +// columns with user defined type metadata have the same versions of metadata +// as in the other descriptor. Note that this function is only valid on two +// descriptors representing the same table at the same version. +func UserDefinedTypeColsHaveSameVersion(desc TableDescriptor, otherDesc TableDescriptor) bool { + otherCols := otherDesc.UserDefinedTypeColumns() + for i, thisCol := range desc.UserDefinedTypeColumns() { + this, other := thisCol.GetType(), otherCols[i].GetType() + if this.TypeMeta.Version != other.TypeMeta.Version { + return false + } + } + return true +} + +// ColumnIDToOrdinalMap returns a map from Column ID to the ordinal +// position of that column. +func ColumnIDToOrdinalMap(columns []Column) TableColMap { + var m TableColMap + for _, col := range columns { + m.Set(col.GetID(), col.Ordinal()) + } + return m +} + +// ColumnTypes returns the types of the given columns +func ColumnTypes(columns []Column) []*types.T { + return ColumnTypesWithVirtualCol(columns, nil) +} + +// ColumnTypesWithVirtualCol returns the types of all given columns, +// If virtualCol is non-nil, substitutes the type of the virtual +// column instead of the column with the same ID. +func ColumnTypesWithVirtualCol(columns []Column, virtualCol Column) []*types.T { + t := make([]*types.T, len(columns)) + for i, col := range columns { + t[i] = col.GetType() + if virtualCol != nil && col.GetID() == virtualCol.GetID() { + t[i] = virtualCol.GetType() + } + } + return t +} diff --git a/pkg/sql/catalog/tabledesc/BUILD.bazel b/pkg/sql/catalog/tabledesc/BUILD.bazel index 3401be6dd210..a0123ece4c6c 100644 --- a/pkg/sql/catalog/tabledesc/BUILD.bazel +++ b/pkg/sql/catalog/tabledesc/BUILD.bazel @@ -5,6 +5,7 @@ go_library( srcs = [ "column.go", "index.go", + "mutation.go", "safe_format.go", "structured.go", "table.go", diff --git a/pkg/sql/catalog/tabledesc/column.go b/pkg/sql/catalog/tabledesc/column.go index 45373168d2b8..4eb869c938f4 100644 --- a/pkg/sql/catalog/tabledesc/column.go +++ b/pkg/sql/catalog/tabledesc/column.go @@ -25,11 +25,9 @@ var _ catalog.Column = (*column)(nil) // column descriptor along with some metadata from its parent table // descriptor. type column struct { - desc *descpb.ColumnDescriptor - ordinal int - mutationID descpb.MutationID - mutationDirection descpb.DescriptorMutation_Direction - mutationState descpb.DescriptorMutation_State + maybeMutation + desc *descpb.ColumnDescriptor + ordinal int } // ColumnDesc returns the underlying protobuf descriptor. @@ -47,11 +45,9 @@ func (w column) ColumnDescDeepCopy() descpb.ColumnDescriptor { func (w column) DeepCopy() catalog.Column { desc := w.ColumnDescDeepCopy() return &column{ - desc: &desc, - ordinal: w.ordinal, - mutationID: w.mutationID, - mutationDirection: w.mutationDirection, - mutationState: w.mutationState, + maybeMutation: w.maybeMutation, + desc: &desc, + ordinal: w.ordinal, } } @@ -65,31 +61,7 @@ func (w column) Ordinal() int { // Public returns true iff the column is active, i.e. readable. func (w column) Public() bool { - return w.mutationState == descpb.DescriptorMutation_UNKNOWN && !w.IsSystemColumn() -} - -// Adding returns true iff the column is an add mutation in the table -//descriptor. -func (w column) Adding() bool { - return w.mutationDirection == descpb.DescriptorMutation_ADD -} - -// Dropped returns true iff the column is a drop mutation in the table -// descriptor. -func (w column) Dropped() bool { - return w.mutationDirection == descpb.DescriptorMutation_DROP -} - -// WriteAndDeleteOnly returns true iff the column is a mutation in the -// delete-and-write-only state. -func (w column) WriteAndDeleteOnly() bool { - return w.mutationState == descpb.DescriptorMutation_DELETE_AND_WRITE_ONLY -} - -// DeleteOnly returns true iff the column is a mutation in the delete-only -// state. -func (w column) DeleteOnly() bool { - return w.mutationState == descpb.DescriptorMutation_DELETE_ONLY + return !w.IsMutation() && !w.IsSystemColumn() } // GetID returns the column ID. @@ -219,75 +191,70 @@ type columnCache struct { // newColumnCache returns a fresh fully-populated columnCache struct for the // TableDescriptor. -func newColumnCache(desc *descpb.TableDescriptor) *columnCache { +func newColumnCache(desc *descpb.TableDescriptor, mutations *mutationCache) *columnCache { c := columnCache{} - // Build a slice of structs to back the interfaces in c.all. + // Build a slice of structs to back the public and system interfaces in c.all. // This is better than allocating memory once per struct. - backingStructs := make([]column, len(desc.Columns), len(desc.Columns)+len(desc.Mutations)+len(colinfo.AllSystemColumnDescs)) + numPublic := len(desc.Columns) + backingStructs := make([]column, numPublic, numPublic+len(colinfo.AllSystemColumnDescs)) for i := range desc.Columns { backingStructs[i] = column{desc: &desc.Columns[i], ordinal: i} } - for _, m := range desc.Mutations { - if colDesc := m.GetColumn(); colDesc != nil { - col := column{ - desc: colDesc, - ordinal: len(backingStructs), - mutationID: m.MutationID, - mutationState: m.State, - mutationDirection: m.Direction, - } - backingStructs = append(backingStructs, col) - } - } - numDeletable := len(backingStructs) + numMutations := len(mutations.columns) + numDeletable := numPublic + numMutations for i := range colinfo.AllSystemColumnDescs { col := column{ desc: &colinfo.AllSystemColumnDescs[i], - ordinal: len(backingStructs), + ordinal: numDeletable + i, } backingStructs = append(backingStructs, col) } - - // Populate the c.all slice with column interfaces. - c.all = make([]catalog.Column, len(backingStructs)) - for i := range backingStructs { - c.all[i] = &backingStructs[i] + // Populate the c.all slice with Column interfaces. + c.all = make([]catalog.Column, 0, numDeletable+len(colinfo.AllSystemColumnDescs)) + for i := range backingStructs[:numPublic] { + c.all = append(c.all, &backingStructs[i]) + } + for _, m := range mutations.columns { + c.all = append(c.all, m.AsColumn()) + } + for i := range backingStructs[numPublic:] { + c.all = append(c.all, &backingStructs[numPublic+i]) } // Populate the remaining fields. c.deletable = c.all[:numDeletable] c.system = c.all[numDeletable:] - c.public = c.all[:len(desc.Columns)] - if len(c.public) == len(c.deletable) { + c.public = c.all[:numPublic] + if numMutations == 0 { c.readable = c.public c.writable = c.public c.nonDrop = c.public } else { - readableDescs := make([]descpb.ColumnDescriptor, 0, len(c.deletable)-len(c.public)) - readableBackingStructs := make([]column, 0, cap(readableDescs)) - for i, col := range c.deletable { + readableDescs := make([]descpb.ColumnDescriptor, 0, numMutations) + readableBackingStructs := make([]column, 0, numMutations) + for _, col := range c.deletable { if !col.DeleteOnly() { - lazyAllocAppendColumn(&c.writable, col, len(c.deletable)) + lazyAllocAppendColumn(&c.writable, col, numDeletable) } if !col.Dropped() { - lazyAllocAppendColumn(&c.nonDrop, col, len(c.deletable)) + lazyAllocAppendColumn(&c.nonDrop, col, numDeletable) } if !col.Public() && !col.IsNullable() { j := len(readableDescs) readableDescs = append(readableDescs, *col.ColumnDesc()) readableDescs[j].Nullable = true - readableBackingStructs = append(readableBackingStructs, backingStructs[i]) + readableBackingStructs = append(readableBackingStructs, *col.(*column)) readableBackingStructs[j].desc = &readableDescs[j] col = &readableBackingStructs[j] } - lazyAllocAppendColumn(&c.readable, col, len(c.deletable)) + lazyAllocAppendColumn(&c.readable, col, numDeletable) } } for _, col := range c.deletable { if col.Public() && !col.IsHidden() { - lazyAllocAppendColumn(&c.visible, col, len(c.public)) + lazyAllocAppendColumn(&c.visible, col, numPublic) } if col.HasType() && col.GetType().UserDefined() { - lazyAllocAppendColumn(&c.withUDTs, col, len(c.deletable)) + lazyAllocAppendColumn(&c.withUDTs, col, numDeletable) } } return &c diff --git a/pkg/sql/catalog/tabledesc/index.go b/pkg/sql/catalog/tabledesc/index.go index bfbbd741c655..45cd1a7bdbf8 100644 --- a/pkg/sql/catalog/tabledesc/index.go +++ b/pkg/sql/catalog/tabledesc/index.go @@ -22,11 +22,9 @@ var _ catalog.Index = (*index)(nil) // index implements the catalog.Index interface by wrapping the protobuf index // descriptor along with some metadata from its parent table descriptor. type index struct { - desc *descpb.IndexDescriptor - ordinal int - mutationID descpb.MutationID - mutationDirection descpb.DescriptorMutation_Direction - mutationState descpb.DescriptorMutation_State + maybeMutation + desc *descpb.IndexDescriptor + ordinal int } // IndexDesc returns the underlying protobuf descriptor. @@ -57,28 +55,7 @@ func (w index) Primary() bool { // Public returns true iff the index is active, i.e. readable. func (w index) Public() bool { - return w.mutationState == descpb.DescriptorMutation_UNKNOWN -} - -// Adding returns true iff the index is an add mutation in the table descriptor. -func (w index) Adding() bool { - return w.mutationDirection == descpb.DescriptorMutation_ADD -} - -// Dropped returns true iff the index is a drop mutation in the table descriptor. -func (w index) Dropped() bool { - return w.mutationDirection == descpb.DescriptorMutation_DROP -} - -// WriteAndDeleteOnly returns true iff the index is a mutation in the -// delete-and-write-only state. -func (w index) WriteAndDeleteOnly() bool { - return w.mutationState == descpb.DescriptorMutation_DELETE_AND_WRITE_ONLY -} - -// DeleteOnly returns true iff the index is a mutation in the delete-only state. -func (w index) DeleteOnly() bool { - return w.mutationState == descpb.DescriptorMutation_DELETE_ONLY + return !w.IsMutation() } // GetID returns the index ID. @@ -328,43 +305,36 @@ type indexCache struct { // newIndexCache returns a fresh fully-populated indexCache struct for the // TableDescriptor. -func newIndexCache(desc *descpb.TableDescriptor) *indexCache { +func newIndexCache(desc *descpb.TableDescriptor, mutations *mutationCache) *indexCache { c := indexCache{} - // Build a slice of structs to back the interfaces in c.all. + // Build a slice of structs to back the public interfaces in c.all. // This is better than allocating memory once per struct. - backingStructs := make([]index, 1+len(desc.Indexes), 1+len(desc.Indexes)+len(desc.Mutations)) + numPublic := 1 + len(desc.Indexes) + backingStructs := make([]index, numPublic) backingStructs[0] = index{desc: &desc.PrimaryIndex} for i := range desc.Indexes { backingStructs[i+1] = index{desc: &desc.Indexes[i], ordinal: i + 1} } - for _, m := range desc.Mutations { - if idxDesc := m.GetIndex(); idxDesc != nil { - idx := index{ - desc: idxDesc, - ordinal: len(backingStructs), - mutationID: m.MutationID, - mutationState: m.State, - mutationDirection: m.Direction, - } - backingStructs = append(backingStructs, idx) - } - } - // Populate the c.all slice with index interfaces. - c.all = make([]catalog.Index, len(backingStructs)) + // Populate the c.all slice with Index interfaces. + numMutations := len(mutations.indexes) + c.all = make([]catalog.Index, numPublic, numPublic+numMutations) for i := range backingStructs { c.all[i] = &backingStructs[i] } - // Populate the remaining fields. + for _, m := range mutations.indexes { + c.all = append(c.all, m.AsIndex()) + } + // Populate the remaining fields in c. c.primary = c.all[0] - c.active = c.all[:1+len(desc.Indexes)] + c.active = c.all[:numPublic] c.publicNonPrimary = c.active[1:] c.deletableNonPrimary = c.all[1:] - if len(c.active) == len(c.all) { + if numMutations == 0 { c.writableNonPrimary = c.publicNonPrimary } else { for _, idx := range c.deletableNonPrimary { if idx.DeleteOnly() { - lazyAllocAppendIndex(&c.deleteOnlyNonPrimary, idx, len(c.all)-len(c.active)) + lazyAllocAppendIndex(&c.deleteOnlyNonPrimary, idx, numMutations) } else { lazyAllocAppendIndex(&c.writableNonPrimary, idx, len(c.deletableNonPrimary)) } diff --git a/pkg/sql/catalog/tabledesc/mutation.go b/pkg/sql/catalog/tabledesc/mutation.go new file mode 100644 index 000000000000..f472c372a816 --- /dev/null +++ b/pkg/sql/catalog/tabledesc/mutation.go @@ -0,0 +1,266 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package tabledesc + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" +) + +var _ catalog.TableElementMaybeMutation = maybeMutation{} +var _ catalog.TableElementMaybeMutation = constraintToUpdate{} +var _ catalog.TableElementMaybeMutation = primaryKeySwap{} +var _ catalog.TableElementMaybeMutation = computedColumnSwap{} +var _ catalog.TableElementMaybeMutation = materializedViewRefresh{} +var _ catalog.Mutation = mutation{} + +// maybeMutation implements the catalog.TableElementMaybeMutation interface +// and is embedded in table element interface implementations column and index +// as well as mutation. +type maybeMutation struct { + mutationID descpb.MutationID + mutationDirection descpb.DescriptorMutation_Direction + mutationState descpb.DescriptorMutation_State + mutationIsRollback bool +} + +// IsMutation returns true iff this table element is in a mutation. +func (mm maybeMutation) IsMutation() bool { + return mm.mutationState != descpb.DescriptorMutation_UNKNOWN +} + +// IsRollback returns true iff the table element is in a rollback mutation. +func (mm maybeMutation) IsRollback() bool { + return mm.mutationIsRollback +} + +// MutationID returns the table element's mutationID if applicable, +// descpb.InvalidMutationID otherwise. +func (mm maybeMutation) MutationID() descpb.MutationID { + return mm.mutationID +} + +// WriteAndDeleteOnly returns true iff the table element is in a mutation in +// the delete-and-write-only state. +func (mm maybeMutation) WriteAndDeleteOnly() bool { + return mm.mutationState == descpb.DescriptorMutation_DELETE_AND_WRITE_ONLY +} + +// DeleteOnly returns true iff the table element is in a mutation in the +// delete-only state. +func (mm maybeMutation) DeleteOnly() bool { + return mm.mutationState == descpb.DescriptorMutation_DELETE_ONLY +} + +// Adding returns true iff the table element is in an add mutation. +func (mm maybeMutation) Adding() bool { + return mm.mutationDirection == descpb.DescriptorMutation_ADD +} + +// Dropped returns true iff the table element is in a drop mutation. +func (mm maybeMutation) Dropped() bool { + return mm.mutationDirection == descpb.DescriptorMutation_DROP +} + +// constraintToUpdate implements the catalog.ConstraintToUpdate interface. +type constraintToUpdate struct { + maybeMutation + desc *descpb.ConstraintToUpdate +} + +// ConstraintToUpdateDesc returns the underlying protobuf descriptor. +func (c constraintToUpdate) ConstraintToUpdateDesc() *descpb.ConstraintToUpdate { + return c.desc +} + +// primaryKeySwap implements the catalog.PrimaryKeySwap interface. +type primaryKeySwap struct { + maybeMutation + desc *descpb.PrimaryKeySwap +} + +// PrimaryKeySwapDesc returns the underlying protobuf descriptor. +func (c primaryKeySwap) PrimaryKeySwapDesc() *descpb.PrimaryKeySwap { + return c.desc +} + +// computedColumnSwap implements the catalog.ComputedColumnSwap interface. +type computedColumnSwap struct { + maybeMutation + desc *descpb.ComputedColumnSwap +} + +// ComputedColumnSwapDesc returns the underlying protobuf descriptor. +func (c computedColumnSwap) ComputedColumnSwapDesc() *descpb.ComputedColumnSwap { + return c.desc +} + +// materializedViewRefresh implements the catalog.MaterializedViewRefresh interface. +type materializedViewRefresh struct { + maybeMutation + desc *descpb.MaterializedViewRefresh +} + +// MaterializedViewRefreshDesc returns the underlying protobuf descriptor. +func (c materializedViewRefresh) MaterializedViewRefreshDesc() *descpb.MaterializedViewRefresh { + return c.desc +} + +// mutation implements the +type mutation struct { + maybeMutation + column catalog.Column + index catalog.Index + constraint catalog.ConstraintToUpdate + pkSwap catalog.PrimaryKeySwap + ccSwap catalog.ComputedColumnSwap + mvRefresh catalog.MaterializedViewRefresh + mutationOrdinal int +} + +// AsColumn returns the corresponding Column if the mutation is on a column, +// nil otherwise. +func (m mutation) AsColumn() catalog.Column { + return m.column +} + +// AsIndex returns the corresponding Index if the mutation is on an index, +// nil otherwise. +func (m mutation) AsIndex() catalog.Index { + return m.index +} + +// AsConstraint returns the corresponding ConstraintToUpdate if the +// mutation is on a constraint, nil otherwise. +func (m mutation) AsConstraint() catalog.ConstraintToUpdate { + return m.constraint +} + +// AsPrimaryKeySwap returns the corresponding PrimaryKeySwap if the mutation +// is a primary key swap, nil otherwise. +func (m mutation) AsPrimaryKeySwap() catalog.PrimaryKeySwap { + return m.pkSwap +} + +// AsComputedColumnSwap returns the corresponding ComputedColumnSwap if the +// mutation is a computed column swap, nil otherwise. +func (m mutation) AsComputedColumnSwap() catalog.ComputedColumnSwap { + return m.ccSwap +} + +// AsMaterializedViewRefresh returns the corresponding MaterializedViewRefresh +// if the mutation is a materialized view refresh, nil otherwise. +func (m mutation) AsMaterializedViewRefresh() catalog.MaterializedViewRefresh { + return m.mvRefresh +} + +// MutationOrdinal returns the ordinal of the mutation in the underlying table +// descriptor's Mutations slice. +func (m mutation) MutationOrdinal() int { + return m.mutationOrdinal +} + +// mutationCache contains precomputed slices of catalog.Mutation interfaces. +type mutationCache struct { + all []catalog.Mutation + columns []catalog.Mutation + indexes []catalog.Mutation +} + +// newMutationCache returns a fresh fully-populated mutationCache struct for the +// TableDescriptor. +func newMutationCache(desc *descpb.TableDescriptor) *mutationCache { + c := mutationCache{} + if len(desc.Mutations) == 0 { + return &c + } + // Build slices of structs to back the interfaces in c.all. + // This is better than allocating memory once per struct. + backingStructs := make([]mutation, len(desc.Mutations)) + var columns []column + var indexes []index + var constraints []constraintToUpdate + var pkSwaps []primaryKeySwap + var ccSwaps []computedColumnSwap + var mvRefreshes []materializedViewRefresh + for i, m := range desc.Mutations { + mm := maybeMutation{ + mutationID: m.MutationID, + mutationDirection: m.Direction, + mutationState: m.State, + mutationIsRollback: m.Rollback, + } + backingStructs[i] = mutation{ + maybeMutation: mm, + mutationOrdinal: i, + } + if pb := m.GetColumn(); pb != nil { + columns = append(columns, column{ + maybeMutation: mm, + desc: pb, + ordinal: len(desc.Columns) + len(columns), + }) + backingStructs[i].column = &columns[len(columns)-1] + } else if pb := m.GetIndex(); pb != nil { + indexes = append(indexes, index{ + maybeMutation: mm, + desc: pb, + ordinal: 1 + len(desc.Indexes) + len(indexes), + }) + backingStructs[i].index = &indexes[len(indexes)-1] + } else if pb := m.GetConstraint(); pb != nil { + constraints = append(constraints, constraintToUpdate{ + maybeMutation: mm, + desc: pb, + }) + backingStructs[i].constraint = &constraints[len(constraints)-1] + } else if pb := m.GetPrimaryKeySwap(); pb != nil { + pkSwaps = append(pkSwaps, primaryKeySwap{ + maybeMutation: mm, + desc: pb, + }) + backingStructs[i].pkSwap = &pkSwaps[len(pkSwaps)-1] + } else if pb := m.GetComputedColumnSwap(); pb != nil { + ccSwaps = append(ccSwaps, computedColumnSwap{ + maybeMutation: mm, + desc: pb, + }) + backingStructs[i].ccSwap = &ccSwaps[len(ccSwaps)-1] + } else if pb := m.GetMaterializedViewRefresh(); pb != nil { + mvRefreshes = append(mvRefreshes, materializedViewRefresh{ + maybeMutation: mm, + desc: pb, + }) + backingStructs[i].mvRefresh = &mvRefreshes[len(mvRefreshes)-1] + } + } + // Populate the c.all slice with Mutation interfaces. + c.all = make([]catalog.Mutation, len(backingStructs)) + for i := range backingStructs { + c.all[i] = &backingStructs[i] + } + // Populate the remaining fields in c. + // Use nil instead of empty slices. + if len(columns) > 0 { + c.columns = make([]catalog.Mutation, 0, len(columns)) + } + if len(indexes) > 0 { + c.indexes = make([]catalog.Mutation, 0, len(indexes)) + } + for _, m := range c.all { + if col := m.AsColumn(); col != nil { + c.columns = append(c.columns, m) + } else if idx := m.AsIndex(); idx != nil { + c.indexes = append(c.indexes, m) + } + } + return &c +} diff --git a/pkg/sql/catalog/tabledesc/safe_format.go b/pkg/sql/catalog/tabledesc/safe_format.go index bc97db69e7cd..80d2406a316c 100644 --- a/pkg/sql/catalog/tabledesc/safe_format.go +++ b/pkg/sql/catalog/tabledesc/safe_format.go @@ -286,7 +286,7 @@ func formatSafeTableMutationJobs(w *redact.StringBuilder, td catalog.TableDescri } func formatSafeMutations(w *redact.StringBuilder, td catalog.TableDescriptor) { - mutations := td.GetMutations() + mutations := td.TableDesc().Mutations for i := range mutations { w.Printf(", ") m := &mutations[i] diff --git a/pkg/sql/catalog/tabledesc/structured.go b/pkg/sql/catalog/tabledesc/structured.go index f5bd5e337959..ad4b9149ccc6 100644 --- a/pkg/sql/catalog/tabledesc/structured.go +++ b/pkg/sql/catalog/tabledesc/structured.go @@ -408,7 +408,7 @@ func ForEachExprStringInTableDesc(descI catalog.TableDescriptor, f func(expr *st } // Process all non-index mutations. - for _, mut := range desc.GetMutations() { + for _, mut := range desc.Mutations { if c := mut.GetColumn(); c != nil { if err := doCol(c); err != nil { return err @@ -1162,7 +1162,7 @@ func (desc *Mutable) FindActiveOrNewColumnByName(name tree.Name) (catalog.Column currentMutationID := desc.ClusterVersion.NextMutationID for _, col := range desc.DeletableColumns() { if (col.Public() && col.ColName() == name) || - (col.Adding() && col.(*column).mutationID == currentMutationID) { + (col.Adding() && col.MutationID() == currentMutationID) { return col, nil } } diff --git a/pkg/sql/catalog/tabledesc/table_desc.go b/pkg/sql/catalog/tabledesc/table_desc.go index b17f36a65d00..97af4e94e396 100644 --- a/pkg/sql/catalog/tabledesc/table_desc.go +++ b/pkg/sql/catalog/tabledesc/table_desc.go @@ -31,11 +31,13 @@ var _ catalog.TableDescriptor = (*wrapper)(nil) type wrapper struct { descpb.TableDescriptor - // indexCache and columnCache, when not nil, respectively point to a struct - // containing precomputed catalog.Index or catalog.Column slices. - // This can therefore only be set when creating an immutable. - indexCache *indexCache - columnCache *columnCache + // mutationCache, indexCache and columnCache, when not nil, respectively point + // to a struct containing precomputed catalog.Mutation, catalog.Index or + // catalog.Column slices. + // Those can therefore only be set when creating an immutable. + mutationCache *mutationCache + indexCache *indexCache + columnCache *columnCache postDeserializationChanges PostDeserializationTableDescriptorChanges } @@ -164,7 +166,7 @@ func (desc *wrapper) getExistingOrNewIndexCache() *indexCache { if desc.indexCache != nil { return desc.indexCache } - return newIndexCache(desc.TableDesc()) + return newIndexCache(desc.TableDesc(), desc.getExistingOrNewMutationCache()) } // AllIndexes returns a slice with all indexes, public and non-public, @@ -284,7 +286,7 @@ func (desc *wrapper) getExistingOrNewColumnCache() *columnCache { if desc.columnCache != nil { return desc.columnCache } - return newColumnCache(desc.TableDesc()) + return newColumnCache(desc.TableDesc(), desc.getExistingOrNewMutationCache()) } // AllColumns returns a slice of Column interfaces containing the @@ -374,3 +376,17 @@ func (desc *wrapper) FindColumnWithName(name tree.Name) (catalog.Column, error) } return nil, colinfo.NewUndefinedColumnError(string(name)) } + +// getExistingOrNewMutationCache should be the only place where the +// mutationCache field in wrapper is ever read. +func (desc *wrapper) getExistingOrNewMutationCache() *mutationCache { + if desc.mutationCache != nil { + return desc.mutationCache + } + return newMutationCache(desc.TableDesc()) +} + +// AllMutations returns all of the table descriptor's mutations. +func (desc *wrapper) AllMutations() []catalog.Mutation { + return desc.getExistingOrNewMutationCache().all +} diff --git a/pkg/sql/catalog/tabledesc/table_desc_builder.go b/pkg/sql/catalog/tabledesc/table_desc_builder.go index 96d54d745b0b..4c63849723c5 100644 --- a/pkg/sql/catalog/tabledesc/table_desc_builder.go +++ b/pkg/sql/catalog/tabledesc/table_desc_builder.go @@ -147,11 +147,10 @@ func (tdb *tableDescriptorBuilder) BuildCreatedMutableTable() *Mutable { // makeImmutable returns an immutable from the given TableDescriptor. func makeImmutable(tbl *descpb.TableDescriptor) *immutable { - desc := immutable{wrapper: wrapper{ - TableDescriptor: *tbl, - indexCache: newIndexCache(tbl), - columnCache: newColumnCache(tbl), - }} + desc := immutable{wrapper: wrapper{TableDescriptor: *tbl}} + desc.mutationCache = newMutationCache(desc.TableDesc()) + desc.indexCache = newIndexCache(desc.TableDesc(), desc.mutationCache) + desc.columnCache = newColumnCache(desc.TableDesc(), desc.mutationCache) desc.allChecks = make([]descpb.TableDescriptor_CheckConstraint, len(tbl.Checks)) for i, c := range tbl.Checks { diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 1015e2e4759a..f5c33942ddd3 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -515,7 +515,7 @@ CREATE TABLE crdb_internal.schema_changes ( tableID := tree.NewDInt(tree.DInt(int64(table.GetID()))) parentID := tree.NewDInt(tree.DInt(int64(table.GetParentID()))) tableName := tree.NewDString(table.GetName()) - for _, mut := range table.GetMutations() { + for _, mut := range table.TableDesc().Mutations { mutType := "UNKNOWN" targetID := tree.DNull targetName := tree.DNull diff --git a/pkg/sql/descriptor_mutation_test.go b/pkg/sql/descriptor_mutation_test.go index a3210d10b2cf..6125b93f0e4e 100644 --- a/pkg/sql/descriptor_mutation_test.go +++ b/pkg/sql/descriptor_mutation_test.go @@ -1155,27 +1155,33 @@ CREATE TABLE t.test (k CHAR PRIMARY KEY, v CHAR UNIQUE); {"v", 4, descpb.DescriptorMutation_DELETE_AND_WRITE_ONLY}, } - if len(tableDesc.GetMutations()) != len(expected) { - t.Fatalf("%d mutations, instead of expected %d", len(tableDesc.GetMutations()), len(expected)) + if len(tableDesc.AllMutations()) != len(expected) { + t.Fatalf("%d mutations, instead of expected %d", len(tableDesc.AllMutations()), len(expected)) } - for i, m := range tableDesc.GetMutations() { + for i, m := range tableDesc.AllMutations() { name := expected[i].name - if col := m.GetColumn(); col != nil { - if col.Name != name { - t.Errorf("%d entry: name %s, expected %s", i, col.Name, name) + if col := m.AsColumn(); col != nil { + if col.GetName() != name { + t.Errorf("%d entry: name %s, expected %s", i, col.GetName(), name) } } - if idx := m.GetIndex(); idx != nil { - if idx.Name != name { - t.Errorf("%d entry: name %s, expected %s", i, idx.Name, name) + if idx := m.AsIndex(); idx != nil { + if idx.GetName() != name { + t.Errorf("%d entry: name %s, expected %s", i, idx.GetName(), name) } } - if id := expected[i].id; m.MutationID != id { - t.Errorf("%d entry: id %d, expected %d", i, m.MutationID, id) + if id := expected[i].id; m.MutationID() != id { + t.Errorf("%d entry: id %d, expected %d", i, m.MutationID(), id) + } + actualState := descpb.DescriptorMutation_UNKNOWN + if m.WriteAndDeleteOnly() { + actualState = descpb.DescriptorMutation_DELETE_AND_WRITE_ONLY + } else if m.DeleteOnly() { + actualState = descpb.DescriptorMutation_DELETE_ONLY } - if state := expected[i].state; m.State != state { - t.Errorf("%d entry: state %s, expected %s", i, m.State, state) + if state := expected[i].state; actualState != state { + t.Errorf("%d entry: state %s, expected %s", i, actualState, state) } } } diff --git a/pkg/sql/rowexec/backfiller.go b/pkg/sql/rowexec/backfiller.go index b83bec0047a6..5904312cfaf5 100644 --- a/pkg/sql/rowexec/backfiller.go +++ b/pkg/sql/rowexec/backfiller.go @@ -199,8 +199,8 @@ func GetResumeSpans( // Find the index of the first mutation that is being worked on. const noIndex = -1 mutationIdx := noIndex - for i, m := range tableDesc.GetMutations() { - if m.MutationID != mutationID { + for i, m := range tableDesc.AllMutations() { + if m.MutationID() != mutationID { break } if mutationIdx == noIndex && filter(m) { diff --git a/pkg/sql/rowexec/backfiller_test.go b/pkg/sql/rowexec/backfiller_test.go index 7bedd03345c8..ba09e7c91513 100644 --- a/pkg/sql/rowexec/backfiller_test.go +++ b/pkg/sql/rowexec/backfiller_test.go @@ -116,7 +116,7 @@ func TestWriteResumeSpan(t *testing.T) { t.Fatal(err) } - mutationID := tableDesc.Mutations[0].MutationID + mutationID := tableDesc.AllMutations()[0].MutationID() var jobID jobspb.JobID if len(tableDesc.MutationJobs) > 0 { diff --git a/pkg/sql/schema_changer.go b/pkg/sql/schema_changer.go index f0f97168504e..b30039c90ec4 100644 --- a/pkg/sql/schema_changer.go +++ b/pkg/sql/schema_changer.go @@ -515,8 +515,8 @@ func (sc *SchemaChanger) notFirstInLine(ctx context.Context, desc catalog.Descri // descriptor, it seems possible for a job to be resumed after the mutation // has already been removed. If there's a mutation provided, we should check // whether it actually exists on the table descriptor and exit the job if not. - for i, mutation := range tableDesc.GetMutations() { - if mutation.MutationID == sc.mutationID { + for i, mutation := range tableDesc.AllMutations() { + if mutation.MutationID() == sc.mutationID { if i != 0 { log.Infof(ctx, "schema change on %q (v%d): another change is still in progress", @@ -785,25 +785,17 @@ func (sc *SchemaChanger) initJobRunningStatus(ctx context.Context) error { } var runStatus jobs.RunningStatus - for _, mutation := range desc.GetMutations() { - if mutation.MutationID != sc.mutationID { + for _, mutation := range desc.AllMutations() { + if mutation.MutationID() != sc.mutationID { // Mutations are applied in a FIFO order. Only apply the first set of // mutations if they have the mutation ID we're looking for. break } - switch mutation.Direction { - case descpb.DescriptorMutation_ADD: - switch mutation.State { - case descpb.DescriptorMutation_DELETE_ONLY: - runStatus = RunningStatusDeleteOnly - } - - case descpb.DescriptorMutation_DROP: - switch mutation.State { - case descpb.DescriptorMutation_DELETE_AND_WRITE_ONLY: - runStatus = RunningStatusDeleteAndWriteOnly - } + if mutation.Adding() && mutation.DeleteOnly() { + runStatus = RunningStatusDeleteOnly + } else if mutation.Dropped() && mutation.WriteAndDeleteOnly() { + runStatus = RunningStatusDeleteAndWriteOnly } } if runStatus != "" && !desc.Dropped() { @@ -1679,8 +1671,8 @@ func (sc *SchemaChanger) updateJobForRollback( // Initialize refresh spans to scan the entire table. span := tableDesc.PrimaryIndexSpan(sc.execCfg.Codec) var spanList []jobspb.ResumeSpanList - for _, m := range tableDesc.GetMutations() { - if m.MutationID == sc.mutationID { + for _, m := range tableDesc.AllMutations() { + if m.MutationID() == sc.mutationID { spanList = append(spanList, jobspb.ResumeSpanList{ ResumeSpans: []roachpb.Span{span}, diff --git a/pkg/sql/schema_changer_test.go b/pkg/sql/schema_changer_test.go index 101839fcd19e..7ca52572d771 100644 --- a/pkg/sql/schema_changer_test.go +++ b/pkg/sql/schema_changer_test.go @@ -160,7 +160,7 @@ INSERT INTO t.test VALUES ('a', 'b'), ('c', 'd'); for _, direction := range []descpb.DescriptorMutation_Direction{ descpb.DescriptorMutation_ADD, descpb.DescriptorMutation_DROP, } { - tableDesc.GetMutations()[0].Direction = direction + tableDesc.Mutations[0].Direction = direction expectedVersion++ if err := kvDB.Put( ctx, @@ -186,7 +186,7 @@ INSERT INTO t.test VALUES ('a', 'b'), ('c', 'd'); if newVersion != expectedVersion { t.Fatalf("bad version; e = %d, v = %d", expectedVersion, newVersion) } - state := tableDesc.GetMutations()[0].State + state := tableDesc.Mutations[0].State if state != expectedState { t.Fatalf("bad state; e = %d, v = %d", expectedState, state) } @@ -195,7 +195,7 @@ INSERT INTO t.test VALUES ('a', 'b'), ('c', 'd'); // RunStateMachineBeforeBackfill() doesn't complete the schema change. tableDesc = catalogkv.TestingGetMutableExistingTableDescriptor( kvDB, keys.SystemSQLCodec, "t", "test") - if len(tableDesc.GetMutations()) == 0 { + if len(tableDesc.Mutations) == 0 { t.Fatalf("table expected to have an outstanding schema change: %v", tableDesc) } } @@ -1800,8 +1800,8 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v INT8); // Wait until all the mutations have been processed. testutils.SucceedsSoon(t, func() error { tableDesc = catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "test") - if len(tableDesc.GetMutations()) > 0 { - return errors.Errorf("%d mutations remaining", len(tableDesc.GetMutations())) + if len(tableDesc.AllMutations()) > 0 { + return errors.Errorf("%d mutations remaining", len(tableDesc.AllMutations())) } return nil }) @@ -2224,8 +2224,8 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v INT UNIQUE DEFAULT 23 CREATE FAMILY F3 t.Fatalf("e = %d, v = %d, columns = %+v", e, len(tableDesc.PublicColumns()), tableDesc.PublicColumns()) } else if tableDesc.PublicColumns()[0].GetName() != "k" { t.Fatalf("columns %+v", tableDesc.PublicColumns()) - } else if len(tableDesc.GetMutations()) != 2 { - t.Fatalf("mutations %+v", tableDesc.GetMutations()) + } else if len(tableDesc.AllMutations()) != 2 { + t.Fatalf("mutations %+v", tableDesc.AllMutations()) } } @@ -2795,8 +2795,8 @@ COMMIT; // Ensure that t.test doesn't have any pending mutations // after the primary key change. desc := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "test") - if len(desc.GetMutations()) != 0 { - t.Fatalf("expected to find 0 mutations, but found %d", len(desc.GetMutations())) + if len(desc.AllMutations()) != 0 { + t.Fatalf("expected to find 0 mutations, but found %d", len(desc.AllMutations())) } } @@ -3068,8 +3068,8 @@ CREATE TABLE t.test (k INT NOT NULL, v INT); // that the job did not succeed even though it was canceled. testutils.SucceedsSoon(t, func() error { tableDesc := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "test") - if len(tableDesc.GetMutations()) != 0 { - return errors.Errorf("expected 0 mutations after cancellation, found %d", len(tableDesc.GetMutations())) + if len(tableDesc.AllMutations()) != 0 { + return errors.Errorf("expected 0 mutations after cancellation, found %d", len(tableDesc.AllMutations())) } if tableDesc.GetPrimaryIndex().NumColumns() != 1 || tableDesc.GetPrimaryIndex().GetColumnName(0) != "rowid" { return errors.Errorf("expected primary key change to not succeed after cancellation") @@ -3370,7 +3370,7 @@ INSERT INTO t.test (k, v, length) VALUES (2, 3, 1); // Wait until both mutations are queued up. testutils.SucceedsSoon(t, func() error { tableDesc := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "test") - if l := len(tableDesc.GetMutations()); l != 3 { + if l := len(tableDesc.AllMutations()); l != 3 { return errors.Errorf("number of mutations = %d", l) } return nil @@ -3468,7 +3468,7 @@ INSERT INTO t.test (k, v, length) VALUES (2, 3, 1); } tableDesc := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "test") - if l := len(tableDesc.GetMutations()); l != 3 { + if l := len(tableDesc.AllMutations()); l != 3 { t.Fatalf("number of mutations = %d", l) } @@ -4356,7 +4356,7 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v INT); // Check that an outstanding schema change exists. tableDesc := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "test") oldID := tableDesc.GetID() - if lenMutations := len(tableDesc.GetMutations()); lenMutations != 3 { + if lenMutations := len(tableDesc.AllMutations()); lenMutations != 3 { t.Fatalf("%d outstanding schema change", lenMutations) } @@ -4383,7 +4383,7 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v INT); } // Col "x" is public and col "v" is dropped. - if num := len(tableDesc.GetMutations()); num > 0 { + if num := len(tableDesc.AllMutations()); num > 0 { t.Fatalf("%d outstanding mutation", num) } if lenCols := len(tableDesc.PublicColumns()); lenCols != 2 { @@ -5233,8 +5233,8 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v INT); } tableDesc = catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "test") - if len(tableDesc.PublicNonPrimaryIndexes()) > 0 || len(tableDesc.GetMutations()) > 0 { - t.Fatalf("descriptor broken %d, %d", len(tableDesc.PublicNonPrimaryIndexes()), len(tableDesc.GetMutations())) + if len(tableDesc.PublicNonPrimaryIndexes()) > 0 || len(tableDesc.AllMutations()) > 0 { + t.Fatalf("descriptor broken %d, %d", len(tableDesc.PublicNonPrimaryIndexes()), len(tableDesc.AllMutations())) } } @@ -5307,8 +5307,8 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v JSON); } tableDesc = catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "test") - if len(tableDesc.PublicNonPrimaryIndexes()) > 0 || len(tableDesc.GetMutations()) > 0 { - t.Fatalf("descriptor broken %d, %d", len(tableDesc.PublicNonPrimaryIndexes()), len(tableDesc.GetMutations())) + if len(tableDesc.PublicNonPrimaryIndexes()) > 0 || len(tableDesc.AllMutations()) > 0 { + t.Fatalf("descriptor broken %d, %d", len(tableDesc.PublicNonPrimaryIndexes()), len(tableDesc.AllMutations())) } }