From adf0ec19829375911cb972d5713b8e4a4b1106ab Mon Sep 17 00:00:00 2001 From: Marius Posta Date: Mon, 4 Jan 2021 11:11:42 -0500 Subject: [PATCH] sql: rename PrimaryIndexInterface method to GetPrimaryIndex Previously, the name PrimaryIndexInterface had been chosen because a deprecated GetPrimaryIndex() method already existed and because PrimaryIndex is an existing field in descpb.IndexDescriptor which is embedded in the structs implementing the catalog.TableDescriptor interface. This patch renames the method to GetPrimaryIndex: although its return type differs it has the same meaning as the recently-removed method of the same name. Release note: None --- pkg/ccl/changefeedccl/avro_test.go | 2 +- pkg/ccl/changefeedccl/encoder.go | 4 +- pkg/ccl/changefeedccl/rowfetcher_cache.go | 4 +- pkg/server/settingsworker.go | 2 +- pkg/sql/alter_primary_key.go | 12 +-- pkg/sql/alter_table.go | 8 +- pkg/sql/backfill/backfill.go | 4 +- pkg/sql/catalog/descriptor.go | 2 +- pkg/sql/catalog/tabledesc/table.go | 4 +- pkg/sql/catalog/tabledesc/table_desc.go | 6 +- pkg/sql/check.go | 8 +- pkg/sql/create_index.go | 2 +- pkg/sql/create_stats.go | 6 +- pkg/sql/create_table.go | 14 ++-- pkg/sql/delete_range.go | 4 +- pkg/sql/drop_index.go | 2 +- pkg/sql/information_schema.go | 8 +- pkg/sql/opt_catalog.go | 4 +- pkg/sql/opt_exec_factory.go | 4 +- pkg/sql/partition_test.go | 2 +- .../physicalplan/fake_span_resolver_test.go | 2 +- pkg/sql/refresh_materialized_view.go | 2 +- pkg/sql/row/deleter.go | 4 +- pkg/sql/row/fetcher_mvcc_test.go | 2 +- pkg/sql/row/fetcher_test.go | 4 +- pkg/sql/row/helper.go | 8 +- pkg/sql/row/inserter.go | 6 +- pkg/sql/row/updater.go | 8 +- pkg/sql/rowenc/client_index_encoding_test.go | 82 +++++++++---------- pkg/sql/rowenc/index_encoding_test.go | 16 ++-- pkg/sql/rowenc/testutils.go | 2 +- pkg/sql/rowexec/zigzagjoiner.go | 4 +- pkg/sql/scan.go | 2 +- pkg/sql/schema_changer.go | 2 +- pkg/sql/schema_changer_test.go | 2 +- pkg/sql/scrub.go | 6 +- pkg/sql/scrub_fk.go | 14 ++-- pkg/sql/scrub_index.go | 4 +- pkg/sql/scrub_test.go | 10 +-- pkg/sql/show_create.go | 12 +-- pkg/sql/tablewriter_delete.go | 4 +- pkg/sql/zone_config.go | 2 +- 42 files changed, 150 insertions(+), 150 deletions(-) diff --git a/pkg/ccl/changefeedccl/avro_test.go b/pkg/ccl/changefeedccl/avro_test.go index d5e49f0edf11..612f2e6853aa 100644 --- a/pkg/ccl/changefeedccl/avro_test.go +++ b/pkg/ccl/changefeedccl/avro_test.go @@ -285,7 +285,7 @@ func TestAvroSchema(t *testing.T) { `{"type":["null","long"],"name":"_u0001f366_","default":null,`+ `"__crdb__":"🍦 INT8 NOT NULL"}]}`, tableSchema.codec.Schema()) - indexSchema, err := indexToAvroSchema(tableDesc, tableDesc.PrimaryIndexInterface().IndexDesc()) + indexSchema, err := indexToAvroSchema(tableDesc, tableDesc.GetPrimaryIndex().IndexDesc()) require.NoError(t, err) require.Equal(t, `{"type":"record","name":"_u2603_","fields":[`+ diff --git a/pkg/ccl/changefeedccl/encoder.go b/pkg/ccl/changefeedccl/encoder.go index afd75d2e8254..e49f13a3a8d5 100644 --- a/pkg/ccl/changefeedccl/encoder.go +++ b/pkg/ccl/changefeedccl/encoder.go @@ -142,7 +142,7 @@ func (e *jsonEncoder) EncodeKey(_ context.Context, row encodeRow) ([]byte, error func (e *jsonEncoder) encodeKeyRaw(row encodeRow) ([]interface{}, error) { colIdxByID := row.tableDesc.ColumnIdxMap() - primaryIndex := row.tableDesc.PrimaryIndexInterface() + primaryIndex := row.tableDesc.GetPrimaryIndex() jsonEntries := make([]interface{}, primaryIndex.NumColumns()) for i := 0; i < primaryIndex.NumColumns(); i++ { colID := primaryIndex.GetColumnID(i) @@ -344,7 +344,7 @@ func (e *confluentAvroEncoder) EncodeKey(ctx context.Context, row encodeRow) ([] registered, ok := e.keyCache[cacheKey] if !ok { var err error - registered.schema, err = indexToAvroSchema(row.tableDesc, row.tableDesc.PrimaryIndexInterface().IndexDesc()) + registered.schema, err = indexToAvroSchema(row.tableDesc, row.tableDesc.GetPrimaryIndex().IndexDesc()) if err != nil { return nil, err } diff --git a/pkg/ccl/changefeedccl/rowfetcher_cache.go b/pkg/ccl/changefeedccl/rowfetcher_cache.go index 56472572daf6..15ec48f390e1 100644 --- a/pkg/ccl/changefeedccl/rowfetcher_cache.go +++ b/pkg/ccl/changefeedccl/rowfetcher_cache.go @@ -121,7 +121,7 @@ func (c *rowFetcherCache) TableDescForKey( } // Skip over the column data. - for ; skippedCols < tableDesc.PrimaryIndexInterface().NumColumns(); skippedCols++ { + for ; skippedCols < tableDesc.GetPrimaryIndex().NumColumns(); skippedCols++ { l, err := encoding.PeekLength(remaining) if err != nil { return nil, err @@ -173,7 +173,7 @@ func (c *rowFetcherCache) RowFetcherForTableDesc( row.FetcherTableArgs{ Spans: tableDesc.AllIndexSpans(c.codec), Desc: tableDesc, - Index: tableDesc.PrimaryIndexInterface().IndexDesc(), + Index: tableDesc.GetPrimaryIndex().IndexDesc(), ColIdxMap: colIdxMap, IsSecondaryIndex: false, Cols: tableDesc.Columns, diff --git a/pkg/server/settingsworker.go b/pkg/server/settingsworker.go index 968a0d57e4c8..9d099ee31457 100644 --- a/pkg/server/settingsworker.go +++ b/pkg/server/settingsworker.go @@ -50,7 +50,7 @@ func processSystemConfigKVs( { types := []*types.T{tbl.Columns[0].Type} nameRow := make([]rowenc.EncDatum, 1) - _, matches, _, err := rowenc.DecodeIndexKey(codec, tbl, tbl.PrimaryIndexInterface().IndexDesc(), types, nameRow, nil, kv.Key) + _, matches, _, err := rowenc.DecodeIndexKey(codec, tbl, tbl.GetPrimaryIndex().IndexDesc(), types, nameRow, nil, kv.Key) if err != nil { return errors.Wrap(err, "failed to decode key") } diff --git a/pkg/sql/alter_primary_key.go b/pkg/sql/alter_primary_key.go index 3a1ddc516716..9f4a2777418d 100644 --- a/pkg/sql/alter_primary_key.go +++ b/pkg/sql/alter_primary_key.go @@ -91,12 +91,12 @@ func (p *planner) AlterPrimaryKey( } // Disable primary key changes on tables that are interleaved parents. - if tableDesc.PrimaryIndexInterface().NumInterleavedBy() != 0 { + if tableDesc.GetPrimaryIndex().NumInterleavedBy() != 0 { var sb strings.Builder sb.WriteString("[") comma := ", " - for i := 0; i < tableDesc.PrimaryIndexInterface().NumInterleavedBy(); i++ { - interleaveTableID := tableDesc.PrimaryIndexInterface().GetInterleavedBy(i).Table + for i := 0; i < tableDesc.GetPrimaryIndex().NumInterleavedBy(); i++ { + interleaveTableID := tableDesc.GetPrimaryIndex().GetInterleavedBy(i).Table if i != 0 { sb.WriteString(comma) } @@ -222,7 +222,7 @@ func (p *planner) AlterPrimaryKey( // Create a new index that indexes everything the old primary index // does, but doesn't store anything. if shouldCopyPrimaryKey(tableDesc, newPrimaryIndexDesc) { - oldPrimaryIndexCopy := tableDesc.PrimaryIndexInterface().IndexDescDeepCopy() + oldPrimaryIndexCopy := tableDesc.GetPrimaryIndex().IndexDescDeepCopy() // Clear the name of the index so that it gets generated by AllocateIDs. oldPrimaryIndexCopy.Name = "" oldPrimaryIndexCopy.StoreColumnIDs = nil @@ -314,7 +314,7 @@ func (p *planner) AlterPrimaryKey( // Mark the primary key of the table as valid. { - primaryIndex := *tableDesc.PrimaryIndexInterface().IndexDesc() + primaryIndex := *tableDesc.GetPrimaryIndex().IndexDesc() primaryIndex.Disabled = false tableDesc.SetPrimaryIndex(primaryIndex) } @@ -344,7 +344,7 @@ func (p *planner) AlterPrimaryKey( // * The new primary key isn't the same hash sharded old primary key with a // different bucket count. func shouldCopyPrimaryKey(desc *tabledesc.Mutable, newPK *descpb.IndexDescriptor) bool { - oldPK := desc.PrimaryIndexInterface() + oldPK := desc.GetPrimaryIndex() if !desc.HasPrimaryKey() { return false } diff --git a/pkg/sql/alter_table.go b/pkg/sql/alter_table.go index 8874f181cfb9..46fed26abea7 100644 --- a/pkg/sql/alter_table.go +++ b/pkg/sql/alter_table.go @@ -436,7 +436,7 @@ func (n *alterTableNode) startExec(params runParams) error { return err } - if n.tableDesc.PrimaryIndexInterface().ContainsColumnID(colToDrop.ID) { + if n.tableDesc.GetPrimaryIndex().ContainsColumnID(colToDrop.ID) { return pgerror.Newf(pgcode.InvalidColumnReference, "column %q is referenced by the primary key", colToDrop.Name) } @@ -459,7 +459,7 @@ func (n *alterTableNode) startExec(params runParams) error { if !containsThisColumn { for j := 0; j < idx.NumExtraColumns(); j++ { id := idx.GetExtraColumnID(j) - if n.tableDesc.PrimaryIndexInterface().ContainsColumnID(id) { + if n.tableDesc.GetPrimaryIndex().ContainsColumnID(id) { // All secondary indices necessary contain the PK // columns, too. (See the comments on the definition of // IndexDescriptor). The presence of a PK column in the @@ -740,7 +740,7 @@ func (n *alterTableNode) startExec(params runParams) error { descriptorChanged = true case *tree.AlterTablePartitionBy: - primaryIndex := n.tableDesc.PrimaryIndexInterface() + primaryIndex := n.tableDesc.GetPrimaryIndex() newPartitioning, err := CreatePartitioning( params.ctx, params.p.ExecCfg().Settings, params.EvalContext(), @@ -1035,7 +1035,7 @@ func applyColumnMutation( } // Prevent a column in a primary key from becoming non-null. - if tableDesc.PrimaryIndexInterface().ContainsColumnID(col.ID) { + if tableDesc.GetPrimaryIndex().ContainsColumnID(col.ID) { return pgerror.Newf(pgcode.InvalidTableDefinition, `column "%s" is in a primary index`, col.Name) } diff --git a/pkg/sql/backfill/backfill.go b/pkg/sql/backfill/backfill.go index adbe355bae97..f48e639fd0f2 100644 --- a/pkg/sql/backfill/backfill.go +++ b/pkg/sql/backfill/backfill.go @@ -124,7 +124,7 @@ func (cb *ColumnBackfiller) init( tableArgs := row.FetcherTableArgs{ Desc: desc, - Index: desc.PrimaryIndexInterface().IndexDesc(), + Index: desc.GetPrimaryIndex().IndexDesc(), ColIdxMap: desc.ColumnIdxMap(), Cols: desc.Columns, ValNeededForCol: valNeededForCol, @@ -601,7 +601,7 @@ func (ib *IndexBackfiller) init( tableArgs := row.FetcherTableArgs{ Desc: desc, - Index: desc.PrimaryIndexInterface().IndexDesc(), + Index: desc.GetPrimaryIndex().IndexDesc(), ColIdxMap: ib.colIdxMap, Cols: ib.cols, ValNeededForCol: valNeededForCol, diff --git a/pkg/sql/catalog/descriptor.go b/pkg/sql/catalog/descriptor.go index 884e1fc8957a..e83724143271 100644 --- a/pkg/sql/catalog/descriptor.go +++ b/pkg/sql/catalog/descriptor.go @@ -117,7 +117,7 @@ type TableDescriptor interface { GetIndexMutationCapabilities(id descpb.IndexID) (isMutation, isWriteOnly bool) KeysPerRow(id descpb.IndexID) (int, error) - PrimaryIndexInterface() Index + GetPrimaryIndex() Index AllIndexes() []Index ActiveIndexes() []Index NonDropIndexes() []Index diff --git a/pkg/sql/catalog/tabledesc/table.go b/pkg/sql/catalog/tabledesc/table.go index d82fd5ba480e..2deaa3bdf681 100644 --- a/pkg/sql/catalog/tabledesc/table.go +++ b/pkg/sql/catalog/tabledesc/table.go @@ -339,7 +339,7 @@ func FindFKReferencedIndex( ) (*descpb.IndexDescriptor, error) { // Search for a unique index on the referenced table that matches our foreign // key columns. - primaryIndex := referencedTable.PrimaryIndexInterface() + primaryIndex := referencedTable.GetPrimaryIndex() if primaryIndex.IsValidReferencedIndex(referencedColIDs) { return primaryIndex.IndexDesc(), nil } @@ -363,7 +363,7 @@ func FindFKOriginIndex( ) (*descpb.IndexDescriptor, error) { // Search for an index on the origin table that matches our foreign // key columns. - if primaryIndex := originTable.PrimaryIndexInterface(); primaryIndex.IsValidOriginIndex(originColIDs) { + if primaryIndex := originTable.GetPrimaryIndex(); primaryIndex.IsValidOriginIndex(originColIDs) { return primaryIndex.IndexDesc(), nil } // If the PK doesn't match, find the index corresponding to the origin column. diff --git a/pkg/sql/catalog/tabledesc/table_desc.go b/pkg/sql/catalog/tabledesc/table_desc.go index 70f6893ce004..bd4fe92153f9 100644 --- a/pkg/sql/catalog/tabledesc/table_desc.go +++ b/pkg/sql/catalog/tabledesc/table_desc.go @@ -252,9 +252,9 @@ func (desc *Mutable) SetPublicNonPrimaryIndex(indexOrdinal int, index descpb.Ind desc.Indexes[indexOrdinal-1] = index } -// PrimaryIndexInterface returns the primary index in the form of a -// catalog.Index interface. -func (desc *wrapper) PrimaryIndexInterface() catalog.Index { +// GetPrimaryIndex returns the primary index in the form of a catalog.Index +// interface. +func (desc *wrapper) GetPrimaryIndex() catalog.Index { return index{desc: &desc.PrimaryIndex} } diff --git a/pkg/sql/check.go b/pkg/sql/check.go index 13cf97dc9f3c..a2541655aa78 100644 --- a/pkg/sql/check.go +++ b/pkg/sql/check.go @@ -96,8 +96,8 @@ func matchFullUnacceptableKeyQuery( srcNotNullExistsClause[i] = fmt.Sprintf("%s IS NOT NULL", srcCols[i]) } - for i := 0; i < srcTbl.PrimaryIndexInterface().NumColumns(); i++ { - id := srcTbl.PrimaryIndexInterface().GetColumnID(i) + for i := 0; i < srcTbl.GetPrimaryIndex().NumColumns(); i++ { + id := srcTbl.GetPrimaryIndex().GetColumnID(i) alreadyPresent := false for _, otherID := range fk.OriginColumnIDs { if id == otherID { @@ -160,8 +160,8 @@ func nonMatchingRowQuery( return "", nil, err } // Get primary key columns not included in the FK - for i := 0; i < srcTbl.PrimaryIndexInterface().NumColumns(); i++ { - pkColID := srcTbl.PrimaryIndexInterface().GetColumnID(i) + for i := 0; i < srcTbl.GetPrimaryIndex().NumColumns(); i++ { + pkColID := srcTbl.GetPrimaryIndex().GetColumnID(i) found := false for _, id := range fk.OriginColumnIDs { if pkColID == id { diff --git a/pkg/sql/create_index.go b/pkg/sql/create_index.go index 9089b94f6254..1a649bea41c1 100644 --- a/pkg/sql/create_index.go +++ b/pkg/sql/create_index.go @@ -391,7 +391,7 @@ func (n *createIndexNode) startExec(params runParams) error { // Warn against creating a non-partitioned index on a partitioned table, // which is undesirable in most cases. - if n.n.PartitionBy == nil && n.tableDesc.PrimaryIndexInterface().GetPartitioning().NumColumns > 0 { + if n.n.PartitionBy == nil && n.tableDesc.GetPrimaryIndex().GetPartitioning().NumColumns > 0 { params.p.BufferClientNotice( params.ctx, errors.WithHint( diff --git a/pkg/sql/create_stats.go b/pkg/sql/create_stats.go index fefc08be11f8..4f94b3e6fe8d 100644 --- a/pkg/sql/create_stats.go +++ b/pkg/sql/create_stats.go @@ -350,9 +350,9 @@ func createStatsDefaultColumns( } // Add column stats for the primary key. - for i := 0; i < desc.PrimaryIndexInterface().NumColumns(); i++ { + for i := 0; i < desc.GetPrimaryIndex().NumColumns(); i++ { // Generate stats for each column in the primary key. - addIndexColumnStatsIfNotExists(desc.PrimaryIndexInterface().GetColumnID(i), false /* isInverted */) + addIndexColumnStatsIfNotExists(desc.GetPrimaryIndex().GetColumnID(i), false /* isInverted */) // Only collect multi-column stats if enabled. if i == 0 || !multiColEnabled { @@ -361,7 +361,7 @@ func createStatsDefaultColumns( colIDs := make([]descpb.ColumnID, i+1) for j := 0; j <= i; j++ { - colIDs[j] = desc.PrimaryIndexInterface().GetColumnID(j) + colIDs[j] = desc.GetPrimaryIndex().GetColumnID(j) } // Remember the requested stats so we don't request duplicates. diff --git a/pkg/sql/create_table.go b/pkg/sql/create_table.go index 89e0ce6f9c5d..8aeda556ad4e 100644 --- a/pkg/sql/create_table.go +++ b/pkg/sql/create_table.go @@ -785,9 +785,9 @@ func ResolveFK( referencedColNames := d.ToCols // If no columns are specified, attempt to default to PK. if len(referencedColNames) == 0 { - referencedColNames = make(tree.NameList, target.PrimaryIndexInterface().NumColumns()) + referencedColNames = make(tree.NameList, target.GetPrimaryIndex().NumColumns()) for i := range referencedColNames { - referencedColNames[i] = tree.Name(target.PrimaryIndexInterface().GetColumnName(i)) + referencedColNames[i] = tree.Name(target.GetPrimaryIndex().GetColumnName(i)) } } @@ -1025,7 +1025,7 @@ func addInterleave( if err != nil { return err } - parentIndex := parentTable.PrimaryIndexInterface() + parentIndex := parentTable.GetPrimaryIndex() // typeOfIndex is used to give more informative error messages. var typeOfIndex string @@ -1647,7 +1647,7 @@ func NewTableDesc( } // If explicit primary keys are required, error out since a primary key was not supplied. - if desc.PrimaryIndexInterface().NumColumns() == 0 && desc.IsPhysicalTable() && evalCtx != nil && + if desc.GetPrimaryIndex().NumColumns() == 0 && desc.IsPhysicalTable() && evalCtx != nil && evalCtx.SessionData != nil && evalCtx.SessionData.RequireExplicitPrimaryKeys { return nil, errors.Errorf( "no primary key specified for table %s (require_explicit_primary_keys = true)", desc.Name) @@ -1707,19 +1707,19 @@ func NewTableDesc( } if n.Interleave != nil { - if err := addInterleave(ctx, txn, vt, &desc, desc.PrimaryIndexInterface().IndexDesc(), n.Interleave); err != nil { + if err := addInterleave(ctx, txn, vt, &desc, desc.GetPrimaryIndex().IndexDesc(), n.Interleave); err != nil { return nil, err } } if n.PartitionBy != nil { partitioning, err := CreatePartitioning( - ctx, st, evalCtx, &desc, desc.PrimaryIndexInterface().IndexDesc(), n.PartitionBy) + ctx, st, evalCtx, &desc, desc.GetPrimaryIndex().IndexDesc(), n.PartitionBy) if err != nil { return nil, err } { - newPrimaryIndex := *desc.PrimaryIndexInterface().IndexDesc() + newPrimaryIndex := *desc.GetPrimaryIndex().IndexDesc() newPrimaryIndex.Partitioning = partitioning desc.SetPrimaryIndex(newPrimaryIndex) } diff --git a/pkg/sql/delete_range.go b/pkg/sql/delete_range.go index 55608262e27b..2535d66f7e90 100644 --- a/pkg/sql/delete_range.go +++ b/pkg/sql/delete_range.go @@ -96,13 +96,13 @@ func (d *deleteRangeNode) startExec(params runParams) error { allTables := make([]row.FetcherTableArgs, len(d.interleavedDesc)+1) allTables[0] = row.FetcherTableArgs{ Desc: d.desc, - Index: d.desc.PrimaryIndexInterface().IndexDesc(), + Index: d.desc.GetPrimaryIndex().IndexDesc(), Spans: d.spans, } for i, interleaved := range d.interleavedDesc { allTables[i+1] = row.FetcherTableArgs{ Desc: interleaved, - Index: interleaved.PrimaryIndexInterface().IndexDesc(), + Index: interleaved.GetPrimaryIndex().IndexDesc(), Spans: d.spans, } } diff --git a/pkg/sql/drop_index.go b/pkg/sql/drop_index.go index 4d7a9f4f666e..5627069b7dbf 100644 --- a/pkg/sql/drop_index.go +++ b/pkg/sql/drop_index.go @@ -313,7 +313,7 @@ func (p *planner) dropIndexByName( // is another index that could replace the one we are deleting for a given // foreign key constraint. remainingIndexes := make([]*descpb.IndexDescriptor, 1, len(tableDesc.ActiveIndexes())) - remainingIndexes[0] = tableDesc.PrimaryIndexInterface().IndexDesc() + remainingIndexes[0] = tableDesc.GetPrimaryIndex().IndexDesc() for _, index := range tableDesc.PublicNonPrimaryIndexes() { if index.GetID() != idx.ID { remainingIndexes = append(remainingIndexes, index.IndexDesc()) diff --git a/pkg/sql/information_schema.go b/pkg/sql/information_schema.go index 9e803063f9ce..ccaaa4b6f489 100644 --- a/pkg/sql/information_schema.go +++ b/pkg/sql/information_schema.go @@ -1312,8 +1312,8 @@ CREATE TABLE information_schema.statistics ( } if hasImplicitCols { implicitCols = make(map[string]struct{}) - for i := 0; i < table.PrimaryIndexInterface().NumColumns(); i++ { - col := table.PrimaryIndexInterface().GetColumnName(i) + for i := 0; i < table.GetPrimaryIndex().NumColumns(); i++ { + col := table.GetPrimaryIndex().GetColumnName(i) implicitCols[col] = struct{}{} } } @@ -1346,8 +1346,8 @@ CREATE TABLE information_schema.statistics ( // // Note that simply iterating over implicitCols map // produces non-deterministic output. - for i := 0; i < table.PrimaryIndexInterface().NumColumns(); i++ { - col := table.PrimaryIndexInterface().GetColumnName(i) + for i := 0; i < table.GetPrimaryIndex().NumColumns(); i++ { + col := table.GetPrimaryIndex().GetColumnName(i) if _, isImplicit := implicitCols[col]; isImplicit { // We add a row for each implicit column of index. if err := appendRow(index.IndexDesc(), col, sequence, diff --git a/pkg/sql/opt_catalog.go b/pkg/sql/opt_catalog.go index 769251ea0031..e8f54e730a31 100644 --- a/pkg/sql/opt_catalog.go +++ b/pkg/sql/opt_catalog.go @@ -719,7 +719,7 @@ func newOptTable( for i := range ot.indexes { var idxDesc *descpb.IndexDescriptor if i == 0 { - idxDesc = desc.PrimaryIndexInterface().IndexDesc() + idxDesc = desc.GetPrimaryIndex().IndexDesc() } else { idxDesc = secondaryIndexes[i-1].IndexDesc() } @@ -1125,7 +1125,7 @@ func (oi *optIndex) init( oi.zone = zone oi.indexOrdinal = indexOrdinal oi.invertedVirtualColOrd = invertedVirtualColOrd - if desc == tab.desc.PrimaryIndexInterface().IndexDesc() { + if desc == tab.desc.GetPrimaryIndex().IndexDesc() { // Although the primary index contains all columns in the table, the index // descriptor does not contain columns that are not explicitly part of the // primary key. Retrieve those columns from the table descriptor. diff --git a/pkg/sql/opt_exec_factory.go b/pkg/sql/opt_exec_factory.go index 6e798e82b96d..b8c30794f99e 100644 --- a/pkg/sql/opt_exec_factory.go +++ b/pkg/sql/opt_exec_factory.go @@ -547,7 +547,7 @@ func (ef *execFactory) ConstructIndexJoin( return nil, err } - primaryIndex := tabDesc.PrimaryIndexInterface() + primaryIndex := tabDesc.GetPrimaryIndex() tableScan.index = primaryIndex.IndexDesc() tableScan.disableBatchLimit() @@ -1557,7 +1557,7 @@ func (ef *execFactory) ConstructDeleteRange( autoCommit bool, ) (exec.Node, error) { tabDesc := table.(*optTable).desc - indexDesc := tabDesc.PrimaryIndexInterface().IndexDesc() + indexDesc := tabDesc.GetPrimaryIndex().IndexDesc() sb := span.MakeBuilder(ef.planner.EvalContext(), ef.planner.ExecCfg().Codec, tabDesc, indexDesc) if err := ef.planner.maybeSetSystemConfig(tabDesc.GetID()); err != nil { diff --git a/pkg/sql/partition_test.go b/pkg/sql/partition_test.go index 0859d27ce203..da6a3d7b6a1d 100644 --- a/pkg/sql/partition_test.go +++ b/pkg/sql/partition_test.go @@ -50,7 +50,7 @@ func TestRemovePartitioningOSS(t *testing.T) { // Hack in partitions. Doing this properly requires a CCL binary. { - primaryIndex := *tableDesc.PrimaryIndexInterface().IndexDesc() + primaryIndex := *tableDesc.GetPrimaryIndex().IndexDesc() primaryIndex.Partitioning = descpb.PartitioningDescriptor{ NumColumns: 1, Range: []descpb.PartitioningDescriptor_Range{{ diff --git a/pkg/sql/physicalplan/fake_span_resolver_test.go b/pkg/sql/physicalplan/fake_span_resolver_test.go index a52c69688973..8f90519a72f5 100644 --- a/pkg/sql/physicalplan/fake_span_resolver_test.go +++ b/pkg/sql/physicalplan/fake_span_resolver_test.go @@ -57,7 +57,7 @@ func TestFakeSpanResolver(t *testing.T) { it := resolver.NewSpanResolverIterator(txn) tableDesc := catalogkv.TestingGetTableDescriptor(db, keys.SystemSQLCodec, "test", "t") - primIdxValDirs := catalogkeys.IndexKeyValDirs(tableDesc.PrimaryIndexInterface().IndexDesc()) + primIdxValDirs := catalogkeys.IndexKeyValDirs(tableDesc.GetPrimaryIndex().IndexDesc()) span := tableDesc.PrimaryIndexSpan(keys.SystemSQLCodec) diff --git a/pkg/sql/refresh_materialized_view.go b/pkg/sql/refresh_materialized_view.go index 1b8f52128abe..b0f3b55e459c 100644 --- a/pkg/sql/refresh_materialized_view.go +++ b/pkg/sql/refresh_materialized_view.go @@ -67,7 +67,7 @@ func (n *refreshMaterializedViewNode) startExec(params runParams) error { } // Prepare the new set of indexes by cloning all existing indexes on the view. - newPrimaryIndex := n.desc.PrimaryIndexInterface().IndexDescDeepCopy() + newPrimaryIndex := n.desc.GetPrimaryIndex().IndexDescDeepCopy() newIndexes := make([]descpb.IndexDescriptor, len(n.desc.PublicNonPrimaryIndexes())) for i, idx := range n.desc.PublicNonPrimaryIndexes() { newIndexes[i] = idx.IndexDescDeepCopy() diff --git a/pkg/sql/row/deleter.go b/pkg/sql/row/deleter.go index 8394957d3d1c..4ebe2e29232b 100644 --- a/pkg/sql/row/deleter.go +++ b/pkg/sql/row/deleter.go @@ -67,8 +67,8 @@ func MakeDeleter( } return nil } - for j := 0; j < tableDesc.PrimaryIndexInterface().NumColumns(); j++ { - colID := tableDesc.PrimaryIndexInterface().GetColumnID(j) + for j := 0; j < tableDesc.GetPrimaryIndex().NumColumns(); j++ { + colID := tableDesc.GetPrimaryIndex().GetColumnID(j) if err := maybeAddCol(colID); err != nil { return Deleter{} } diff --git a/pkg/sql/row/fetcher_mvcc_test.go b/pkg/sql/row/fetcher_mvcc_test.go index eedfccc22575..5ff5d707f7ec 100644 --- a/pkg/sql/row/fetcher_mvcc_test.go +++ b/pkg/sql/row/fetcher_mvcc_test.go @@ -101,7 +101,7 @@ func TestRowFetcherMVCCMetadata(t *testing.T) { args = append(args, row.FetcherTableArgs{ Spans: desc.AllIndexSpans(keys.SystemSQLCodec), Desc: desc, - Index: desc.PrimaryIndexInterface().IndexDesc(), + Index: desc.GetPrimaryIndex().IndexDesc(), ColIdxMap: colIdxMap, IsSecondaryIndex: false, Cols: desc.Columns, diff --git a/pkg/sql/row/fetcher_test.go b/pkg/sql/row/fetcher_test.go index 896ec181429d..86245eb097ae 100644 --- a/pkg/sql/row/fetcher_test.go +++ b/pkg/sql/row/fetcher_test.go @@ -194,7 +194,7 @@ func TestNextRowSingle(t *testing.T) { if desc.GetID() != tableDesc.ID || index.ID != tableDesc.GetPrimaryIndexID() { t.Fatalf( "unexpected row retrieved from fetcher.\nnexpected: table %s - index %s\nactual: table %s - index %s", - tableDesc.Name, tableDesc.PrimaryIndexInterface().GetName(), + tableDesc.Name, tableDesc.GetPrimaryIndex().GetName(), desc.GetName(), index.Name, ) } @@ -314,7 +314,7 @@ func TestNextRowBatchLimiting(t *testing.T) { if desc.GetID() != tableDesc.ID || index.ID != tableDesc.GetPrimaryIndexID() { t.Fatalf( "unexpected row retrieved from fetcher.\nnexpected: table %s - index %s\nactual: table %s - index %s", - tableDesc.Name, tableDesc.PrimaryIndexInterface().GetName(), + tableDesc.Name, tableDesc.GetPrimaryIndex().GetName(), desc.GetName(), index.Name, ) } diff --git a/pkg/sql/row/helper.go b/pkg/sql/row/helper.go index 95f431b97be9..8a83c055455b 100644 --- a/pkg/sql/row/helper.go +++ b/pkg/sql/row/helper.go @@ -49,7 +49,7 @@ func newRowHelper( // Pre-compute the encoding directions of the index key values for // pretty-printing in traces. - rh.primIndexValDirs = catalogkeys.IndexKeyValDirs(rh.TableDesc.PrimaryIndexInterface().IndexDesc()) + rh.primIndexValDirs = catalogkeys.IndexKeyValDirs(rh.TableDesc.GetPrimaryIndex().IndexDesc()) rh.secIndexValDirs = make([][]encoding.Direction, len(rh.Indexes)) for i := range rh.Indexes { @@ -89,7 +89,7 @@ func (rh *rowHelper) encodePrimaryIndex( rh.TableDesc.GetPrimaryIndexID()) } primaryIndexKey, _, err = rowenc.EncodeIndexKey( - rh.TableDesc, rh.TableDesc.PrimaryIndexInterface().IndexDesc(), colIDtoRowIndex, values, rh.primaryIndexKeyPrefix) + rh.TableDesc, rh.TableDesc.GetPrimaryIndex().IndexDesc(), colIDtoRowIndex, values, rh.primaryIndexKeyPrefix) return primaryIndexKey, err } @@ -137,8 +137,8 @@ func (rh *rowHelper) encodeSecondaryIndexes( // #6233). Once it is, use the shared one. func (rh *rowHelper) skipColumnInPK(colID descpb.ColumnID, value tree.Datum) (bool, error) { if rh.primaryIndexCols.Empty() { - for i := 0; i < rh.TableDesc.PrimaryIndexInterface().NumColumns(); i++ { - pkColID := rh.TableDesc.PrimaryIndexInterface().GetColumnID(i) + for i := 0; i < rh.TableDesc.GetPrimaryIndex().NumColumns(); i++ { + pkColID := rh.TableDesc.GetPrimaryIndex().GetColumnID(i) rh.primaryIndexCols.Add(pkColID) } } diff --git a/pkg/sql/row/inserter.go b/pkg/sql/row/inserter.go index acaf3b549518..00a9cf11e5f1 100644 --- a/pkg/sql/row/inserter.go +++ b/pkg/sql/row/inserter.go @@ -63,10 +63,10 @@ func MakeInserter( marshaled: make([]roachpb.Value, len(insertCols)), } - for i := 0; i < tableDesc.PrimaryIndexInterface().NumColumns(); i++ { - colID := tableDesc.PrimaryIndexInterface().GetColumnID(i) + for i := 0; i < tableDesc.GetPrimaryIndex().NumColumns(); i++ { + colID := tableDesc.GetPrimaryIndex().GetColumnID(i) if _, ok := ri.InsertColIDtoRowIndex.Get(colID); !ok { - return Inserter{}, fmt.Errorf("missing %q primary key column", tableDesc.PrimaryIndexInterface().GetColumnName(i)) + return Inserter{}, fmt.Errorf("missing %q primary key column", tableDesc.GetPrimaryIndex().GetColumnName(i)) } } diff --git a/pkg/sql/row/updater.go b/pkg/sql/row/updater.go index 4996cf31b0f5..95e08c18d66d 100644 --- a/pkg/sql/row/updater.go +++ b/pkg/sql/row/updater.go @@ -98,8 +98,8 @@ func MakeUpdater( updateColIDtoRowIndex := ColIDtoRowIndexFromCols(updateCols) var primaryIndexCols catalog.TableColSet - for i := 0; i < tableDesc.PrimaryIndexInterface().NumColumns(); i++ { - colID := tableDesc.PrimaryIndexInterface().GetColumnID(i) + for i := 0; i < tableDesc.GetPrimaryIndex().NumColumns(); i++ { + colID := tableDesc.GetPrimaryIndex().GetColumnID(i) primaryIndexCols.Add(colID) } @@ -212,8 +212,8 @@ func MakeUpdater( // Fetch all columns in the primary key so that we can construct the // keys when writing out the new kvs to the primary index. - for i := 0; i < tableDesc.PrimaryIndexInterface().NumColumns(); i++ { - colID := tableDesc.PrimaryIndexInterface().GetColumnID(i) + for i := 0; i < tableDesc.GetPrimaryIndex().NumColumns(); i++ { + colID := tableDesc.GetPrimaryIndex().GetColumnID(i) if err := maybeAddCol(colID); err != nil { return Updater{}, err } diff --git a/pkg/sql/rowenc/client_index_encoding_test.go b/pkg/sql/rowenc/client_index_encoding_test.go index 755878279ab1..81baed783953 100644 --- a/pkg/sql/rowenc/client_index_encoding_test.go +++ b/pkg/sql/rowenc/client_index_encoding_test.go @@ -74,29 +74,29 @@ func TestAdjustStartKeyForInterleave(t *testing.T) { // See encodeStartConstraintAscending. { - index: parent.PrimaryIndexInterface(), + index: parent.GetPrimaryIndex(), input: "/NOTNULLASC", expected: "/NOTNULLASC", }, { - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/1/#/2/NOTNULLASC", expected: "/1/#/2/NOTNULLASC", }, { - index: grandchild.PrimaryIndexInterface(), + index: grandchild.GetPrimaryIndex(), input: "/1/#/2/3/#/NOTNULLASC", expected: "/1/#/2/3/#/NOTNULLASC", }, { - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/1/#/NOTNULLASC", expected: "/1/#/NOTNULLASC", }, { - index: grandchild.PrimaryIndexInterface(), + index: grandchild.GetPrimaryIndex(), input: "/1/#/2/NOTNULLASC", expected: "/1/#/2/NOTNULLASC", }, @@ -131,19 +131,19 @@ func TestAdjustStartKeyForInterleave(t *testing.T) { // Keys that belong to the given index (neither parent nor // children keys) do not need to be tightened. { - index: parent.PrimaryIndexInterface(), + index: parent.GetPrimaryIndex(), input: "/1", expected: "/1", }, { - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/1/#/2/3", expected: "/1/#/2/3", }, // Parent keys wrt child index is not tightened. { - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/1", expected: "/1", }, @@ -151,24 +151,24 @@ func TestAdjustStartKeyForInterleave(t *testing.T) { // Children keys wrt to parent index is tightened (pushed // forwards) to the next parent key. { - index: parent.PrimaryIndexInterface(), + index: parent.GetPrimaryIndex(), input: "/1/#/2/3", expected: "/2", }, { - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/1/#/2/3/#/4", expected: "/1/#/2/4", }, // Key with len > 1 tokens. { - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/12345678901234/#/1234/1234567890/#/123/1234567", expected: "/12345678901234/#/1234/1234567891", }, { - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/12345678901234/#/d1403.2594/shelloworld/#/123/1234567", expected: "/12345678901234/#/d1403.2594/shelloworld/PrefixEnd", }, @@ -291,21 +291,21 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { { table: parent, - index: parent.PrimaryIndexInterface(), + index: parent.GetPrimaryIndex(), input: "/NOTNULLASC", expected: "/NULLASC/#", }, { table: child, - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/1/#/2/NOTNULLASC", expected: "/1/#/2/NULLASC/#", }, { table: grandchild, - index: grandchild.PrimaryIndexInterface(), + index: grandchild.GetPrimaryIndex(), input: "/1/#/2/3/#/NOTNULLASC", expected: "/1/#/2/3/#/NULLASC/#", }, @@ -314,7 +314,7 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { // partial primary key columns. { table: child, - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/1/#/NOTNULLASC", expected: "/1/#/NOTNULLASC", }, @@ -322,7 +322,7 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { // No change since key belongs to an ancestor. { table: grandchild, - index: grandchild.PrimaryIndexInterface(), + index: grandchild.GetPrimaryIndex(), input: "/1/#/2/NOTNULLASC", expected: "/1/#/2/NOTNULLASC", }, @@ -367,7 +367,7 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { { table: parent, - index: parent.PrimaryIndexInterface(), + index: parent.GetPrimaryIndex(), input: "/NULLASC", inclusive: true, expected: "/NULLASC/#", @@ -375,7 +375,7 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { { table: child, - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/1/#/2/NULLASC", inclusive: true, expected: "/1/#/2/NULLASC/#", @@ -387,13 +387,13 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { { table: parent, - index: parent.PrimaryIndexInterface(), + index: parent.GetPrimaryIndex(), input: "/1", expected: "/0/#", }, { table: parent, - index: parent.PrimaryIndexInterface(), + index: parent.GetPrimaryIndex(), input: "/1", inclusive: true, expected: "/1/#", @@ -401,13 +401,13 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { { table: child, - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/1/#/2/3", expected: "/1/#/2/2/#", }, { table: child, - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/1/#/2/3", inclusive: true, expected: "/1/#/2/3/#", @@ -417,19 +417,19 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { { table: parent, - index: parent.PrimaryIndexInterface(), + index: parent.GetPrimaryIndex(), input: "/1/#", expected: "/1/#", }, { table: child, - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/1/#", expected: "/1/#", }, { table: child, - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/1/#/2/2/#", expected: "/1/#/2/2/#", }, @@ -439,13 +439,13 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { { table: parent, - index: parent.PrimaryIndexInterface(), + index: parent.GetPrimaryIndex(), input: "/1/#/2/3", expected: "/1/#", }, { table: parent, - index: parent.PrimaryIndexInterface(), + index: parent.GetPrimaryIndex(), input: "/1/#/2/3", inclusive: true, expected: "/1/#", @@ -453,13 +453,13 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { { table: child, - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/1/#/2/3/#/4", expected: "/1/#/2/3/#", }, { table: child, - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/1/#/2/3/#/4", inclusive: true, expected: "/1/#/2/3/#", @@ -469,13 +469,13 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { { table: child, - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/1", expected: "/1", }, { table: child, - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/1", inclusive: true, expected: "/2", @@ -486,13 +486,13 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { { table: child, - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/1/#/2", expected: "/1/#/2", }, { table: child, - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/1/#/2", inclusive: true, expected: "/1/#/3", @@ -531,7 +531,7 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { // Key with len > 1 tokens. { table: child, - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/12345678901234/#/12345/12345678901234/#/123/1234567", expected: "/12345678901234/#/12345/12345678901234/#", }, @@ -606,20 +606,20 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { // decimal value. { table: child, - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/1/#/2/d3.4567", expected: "/1/#/2/d3.4567", }, { table: child, - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/1/#/2/d3.4567", inclusive: true, expected: "/1/#/2/d3.4567/#", }, { table: child, - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/1/#/2/d3.4567/#/8", expected: "/1/#/2/d3.4567/#", }, @@ -629,20 +629,20 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { // bytes value. { table: child, - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/1/#/2/shelloworld", expected: "/1/#/2/shelloworld", }, { table: child, - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/1/#/2/shelloworld", inclusive: true, expected: "/1/#/2/shelloworld/#", }, { table: child, - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/1/#/2/shelloworld/#/3", expected: "/1/#/2/shelloworld/#", }, diff --git a/pkg/sql/rowenc/index_encoding_test.go b/pkg/sql/rowenc/index_encoding_test.go index dc664f9f06ab..2fd77c186f69 100644 --- a/pkg/sql/rowenc/index_encoding_test.go +++ b/pkg/sql/rowenc/index_encoding_test.go @@ -225,7 +225,7 @@ func TestIndexKey(t *testing.T) { codec := keys.SystemSQLCodec primaryKeyPrefix := MakeIndexKeyPrefix(codec, tableDesc, tableDesc.GetPrimaryIndexID()) - primaryKey, _, err := EncodeIndexKey(tableDesc, tableDesc.PrimaryIndexInterface().IndexDesc(), colMap, testValues, primaryKeyPrefix) + primaryKey, _, err := EncodeIndexKey(tableDesc, tableDesc.GetPrimaryIndex().IndexDesc(), colMap, testValues, primaryKeyPrefix) if err != nil { t.Fatal(err) } @@ -275,7 +275,7 @@ func TestIndexKey(t *testing.T) { } } - checkEntry(tableDesc.PrimaryIndexInterface().IndexDesc(), primaryIndexKV) + checkEntry(tableDesc.GetPrimaryIndex().IndexDesc(), primaryIndexKV) checkEntry(tableDesc.PublicNonPrimaryIndexes()[0].IndexDesc(), secondaryIndexKV) } } @@ -941,7 +941,7 @@ func TestIndexKeyEquivSignature(t *testing.T) { desc, colMap := makeTableDescForTest(tc.table.indexKeyArgs) primaryKeyPrefix := MakeIndexKeyPrefix(keys.SystemSQLCodec, desc, desc.GetPrimaryIndexID()) primaryKey, _, err := EncodeIndexKey( - desc, desc.PrimaryIndexInterface().IndexDesc(), colMap, tc.table.values, primaryKeyPrefix) + desc, desc.GetPrimaryIndex().IndexDesc(), colMap, tc.table.values, primaryKeyPrefix) if err != nil { t.Fatal(err) } @@ -962,7 +962,7 @@ func TestIndexKeyEquivSignature(t *testing.T) { // Column values should be at the beginning of the // remaining bytes of the key. - pkIndexDesc := desc.PrimaryIndexInterface().IndexDesc() + pkIndexDesc := desc.GetPrimaryIndex().IndexDesc() colVals, null, err := EncodeColumns(pkIndexDesc.ColumnIDs, pkIndexDesc.ColumnDirections, colMap, tc.table.values, nil /*key*/) if err != nil { t.Fatal(err) @@ -998,7 +998,7 @@ func TestTableEquivSignatures(t *testing.T) { tc.table.indexKeyArgs.primaryValues = tc.table.values // Setup descriptors and form an index key. desc, _ := makeTableDescForTest(tc.table.indexKeyArgs) - equivSigs, err := TableEquivSignatures(&desc.TableDescriptor, desc.PrimaryIndexInterface().IndexDesc()) + equivSigs, err := TableEquivSignatures(&desc.TableDescriptor, desc.GetPrimaryIndex().IndexDesc()) if err != nil { t.Fatal(err) } @@ -1083,13 +1083,13 @@ func TestEquivSignature(t *testing.T) { desc, colMap := makeTableDescForTest(table.indexKeyArgs) primaryKeyPrefix := MakeIndexKeyPrefix(keys.SystemSQLCodec, desc, desc.GetPrimaryIndexID()) primaryKey, _, err := EncodeIndexKey( - desc, desc.PrimaryIndexInterface().IndexDesc(), colMap, table.values, primaryKeyPrefix) + desc, desc.GetPrimaryIndex().IndexDesc(), colMap, table.values, primaryKeyPrefix) if err != nil { t.Fatal(err) } // Extract out the table's equivalence signature. - tempEquivSigs, err := TableEquivSignatures(&desc.TableDescriptor, desc.PrimaryIndexInterface().IndexDesc()) + tempEquivSigs, err := TableEquivSignatures(&desc.TableDescriptor, desc.GetPrimaryIndex().IndexDesc()) if err != nil { t.Fatal(err) } @@ -1323,6 +1323,6 @@ func ExtractIndexKey( decodedValues[len(values)+i] = value.Datum } indexKey, _, err := EncodeIndexKey( - tableDesc, tableDesc.PrimaryIndexInterface().IndexDesc(), colMap, decodedValues, indexKeyPrefix) + tableDesc, tableDesc.GetPrimaryIndex().IndexDesc(), colMap, decodedValues, indexKeyPrefix) return indexKey, err } diff --git a/pkg/sql/rowenc/testutils.go b/pkg/sql/rowenc/testutils.go index c3f72944f33c..4b2a721b2666 100644 --- a/pkg/sql/rowenc/testutils.go +++ b/pkg/sql/rowenc/testutils.go @@ -999,7 +999,7 @@ func RandEncDatumRowsOfTypes(rng *rand.Rand, numRows int, types []*types.T) EncD func TestingMakePrimaryIndexKey( desc catalog.TableDescriptor, vals ...interface{}, ) (roachpb.Key, error) { - index := desc.PrimaryIndexInterface() + index := desc.GetPrimaryIndex() if len(vals) > index.NumColumns() { return nil, errors.Errorf("got %d values, PK has %d columns", len(vals), index.NumColumns()) } diff --git a/pkg/sql/rowexec/zigzagjoiner.go b/pkg/sql/rowexec/zigzagjoiner.go index 944f70e2edb8..f4e943ba47b1 100644 --- a/pkg/sql/rowexec/zigzagjoiner.go +++ b/pkg/sql/rowexec/zigzagjoiner.go @@ -646,8 +646,8 @@ func (zi *zigzagJoinerInfo) eqOrdering() (colinfo.ColumnOrdering, error) { if err != nil { return nil, err } - } else if idx := findColumnID(zi.table.PrimaryIndexInterface().IndexDesc().ColumnIDs, colID); idx != -1 { - direction, err = zi.table.PrimaryIndexInterface().GetColumnDirection(idx).ToEncodingDirection() + } else if idx := findColumnID(zi.table.GetPrimaryIndex().IndexDesc().ColumnIDs, colID); idx != -1 { + direction, err = zi.table.GetPrimaryIndex().GetColumnDirection(idx).ToEncodingDirection() if err != nil { return nil, err } diff --git a/pkg/sql/scan.go b/pkg/sql/scan.go index 3d453f0f5ef4..0c77a9469df1 100644 --- a/pkg/sql/scan.go +++ b/pkg/sql/scan.go @@ -298,7 +298,7 @@ func initColsForScan( // Initializes the column structures. func (n *scanNode) initDescDefaults(colCfg scanColumnsConfig) error { n.colCfg = colCfg - n.index = n.desc.PrimaryIndexInterface().IndexDesc() + n.index = n.desc.GetPrimaryIndex().IndexDesc() var err error n.cols, err = initColsForScan(n.desc, n.colCfg) diff --git a/pkg/sql/schema_changer.go b/pkg/sql/schema_changer.go index 5a8197674235..efb6083fa104 100644 --- a/pkg/sql/schema_changer.go +++ b/pkg/sql/schema_changer.go @@ -1176,7 +1176,7 @@ func (sc *SchemaChanger) done(ctx context.Context) error { // existing indexes on the table. if mutation.Direction == descpb.DescriptorMutation_ADD { desc := fmt.Sprintf("REFRESH MATERIALIZED VIEW %q cleanup", scTable.Name) - pkJob, err := sc.createIndexGCJob(ctx, scTable.PrimaryIndexInterface().IndexDesc(), txn, desc) + pkJob, err := sc.createIndexGCJob(ctx, scTable.GetPrimaryIndex().IndexDesc(), txn, desc) if err != nil { return err } diff --git a/pkg/sql/schema_changer_test.go b/pkg/sql/schema_changer_test.go index 9771eecc7a3b..e4b4cebbeaa8 100644 --- a/pkg/sql/schema_changer_test.go +++ b/pkg/sql/schema_changer_test.go @@ -3078,7 +3078,7 @@ CREATE TABLE t.test (k INT NOT NULL, v INT); if len(tableDesc.Mutations) != 0 { return errors.Errorf("expected 0 mutations after cancellation, found %d", len(tableDesc.Mutations)) } - if tableDesc.PrimaryIndexInterface().NumColumns() != 1 || tableDesc.PrimaryIndexInterface().GetColumnName(0) != "rowid" { + if tableDesc.GetPrimaryIndex().NumColumns() != 1 || tableDesc.GetPrimaryIndex().GetColumnName(0) != "rowid" { return errors.Errorf("expected primary key change to not succeed after cancellation") } return nil diff --git a/pkg/sql/scrub.go b/pkg/sql/scrub.go index 1b7fbb64fddb..e6e5f7416da9 100644 --- a/pkg/sql/scrub.go +++ b/pkg/sql/scrub.go @@ -286,8 +286,8 @@ func (n *scrubNode) startScrubTable( func getPrimaryColIdxs( tableDesc *tabledesc.Immutable, columns []*descpb.ColumnDescriptor, ) (primaryColIdxs []int, err error) { - for i := 0; i < tableDesc.PrimaryIndexInterface().NumColumns(); i++ { - colID := tableDesc.PrimaryIndexInterface().GetColumnID(i) + for i := 0; i < tableDesc.GetPrimaryIndex().NumColumns(); i++ { + colID := tableDesc.GetPrimaryIndex().GetColumnID(i) rowIdx := -1 for idx, col := range columns { if col.ID == colID { @@ -299,7 +299,7 @@ func getPrimaryColIdxs( return nil, errors.Errorf( "could not find primary index column in projection: columnID=%d columnName=%s", colID, - tableDesc.PrimaryIndexInterface().GetColumnName(i)) + tableDesc.GetPrimaryIndex().GetColumnName(i)) } primaryColIdxs = append(primaryColIdxs, rowIdx) } diff --git a/pkg/sql/scrub_fk.go b/pkg/sql/scrub_fk.go index b81fbd9c69af..ceba8cbff41e 100644 --- a/pkg/sql/scrub_fk.go +++ b/pkg/sql/scrub_fk.go @@ -114,8 +114,8 @@ func (o *sqlForeignKeyCheckOperation) Start(params runParams) error { // Get primary key columns not included in the FK. var colIDs []descpb.ColumnID colIDs = append(colIDs, o.constraint.FK.OriginColumnIDs...) - for i := 0; i < o.tableDesc.PrimaryIndexInterface().NumColumns(); i++ { - pkColID := o.tableDesc.PrimaryIndexInterface().GetColumnID(i) + for i := 0; i < o.tableDesc.GetPrimaryIndex().NumColumns(); i++ { + pkColID := o.tableDesc.GetPrimaryIndex().GetColumnID(i) found := false for _, id := range o.constraint.FK.OriginColumnIDs { if pkColID == id { @@ -148,9 +148,9 @@ func (o *sqlForeignKeyCheckOperation) Next(params runParams) (tree.Datums, error // Collect the primary index values for generating the primary key // pretty string. - primaryKeyDatums := make(tree.Datums, 0, o.tableDesc.PrimaryIndexInterface().NumColumns()) - for i := 0; i < o.tableDesc.PrimaryIndexInterface().NumColumns(); i++ { - id := o.tableDesc.PrimaryIndexInterface().GetColumnID(i) + primaryKeyDatums := make(tree.Datums, 0, o.tableDesc.GetPrimaryIndex().NumColumns()) + for i := 0; i < o.tableDesc.GetPrimaryIndex().NumColumns(); i++ { + id := o.tableDesc.GetPrimaryIndex().GetColumnID(i) idx := o.colIDToRowIdx.GetDefault(id) primaryKeyDatums = append(primaryKeyDatums, row[idx]) } @@ -165,8 +165,8 @@ func (o *sqlForeignKeyCheckOperation) Next(params runParams) (tree.Datums, error } rowDetails[col.Name] = row[idx].String() } - for i := 0; i < o.tableDesc.PrimaryIndexInterface().NumColumns(); i++ { - id := o.tableDesc.PrimaryIndexInterface().GetColumnID(i) + for i := 0; i < o.tableDesc.GetPrimaryIndex().NumColumns(); i++ { + id := o.tableDesc.GetPrimaryIndex().GetColumnID(i) found := false for _, fkID := range o.constraint.FK.OriginColumnIDs { if id == fkID { diff --git a/pkg/sql/scrub_index.go b/pkg/sql/scrub_index.go index ece75437b5ed..6a1d566e7c1b 100644 --- a/pkg/sql/scrub_index.go +++ b/pkg/sql/scrub_index.go @@ -83,8 +83,8 @@ func (o *indexCheckOperation) Start(params runParams) error { var pkColumns, otherColumns []*descpb.ColumnDescriptor - for i := 0; i < o.tableDesc.PrimaryIndexInterface().NumColumns(); i++ { - colID := o.tableDesc.PrimaryIndexInterface().GetColumnID(i) + for i := 0; i < o.tableDesc.GetPrimaryIndex().NumColumns(); i++ { + colID := o.tableDesc.GetPrimaryIndex().GetColumnID(i) col := &o.tableDesc.Columns[colToIdx.GetDefault(colID)] pkColumns = append(pkColumns, col) colToIdx.Set(colID, -1) diff --git a/pkg/sql/scrub_test.go b/pkg/sql/scrub_test.go index 4ff368838957..5cb5cb2da662 100644 --- a/pkg/sql/scrub_test.go +++ b/pkg/sql/scrub_test.go @@ -353,7 +353,7 @@ INSERT INTO t.test VALUES (10, 2); primaryIndexKeyPrefix := rowenc.MakeIndexKeyPrefix( keys.SystemSQLCodec, tableDesc, tableDesc.GetPrimaryIndexID()) primaryIndexKey, _, err := rowenc.EncodeIndexKey( - tableDesc, tableDesc.PrimaryIndexInterface().IndexDesc(), colIDtoRowIndex, values, primaryIndexKeyPrefix) + tableDesc, tableDesc.GetPrimaryIndex().IndexDesc(), colIDtoRowIndex, values, primaryIndexKeyPrefix) if err != nil { t.Fatalf("unexpected error: %s", err) } @@ -593,7 +593,7 @@ INSERT INTO t.test VALUES (217, 314); primaryIndexKeyPrefix := rowenc.MakeIndexKeyPrefix( keys.SystemSQLCodec, tableDesc, tableDesc.GetPrimaryIndexID()) primaryIndexKey, _, err := rowenc.EncodeIndexKey( - tableDesc, tableDesc.PrimaryIndexInterface().IndexDesc(), colIDtoRowIndex, values, primaryIndexKeyPrefix) + tableDesc, tableDesc.GetPrimaryIndex().IndexDesc(), colIDtoRowIndex, values, primaryIndexKeyPrefix) if err != nil { t.Fatalf("unexpected error: %s", err) } @@ -676,7 +676,7 @@ INSERT INTO t.test VALUES (217, 314, 1337); primaryIndexKeyPrefix := rowenc.MakeIndexKeyPrefix( keys.SystemSQLCodec, tableDesc, tableDesc.GetPrimaryIndexID()) primaryIndexKey, _, err := rowenc.EncodeIndexKey( - tableDesc, tableDesc.PrimaryIndexInterface().IndexDesc(), colIDtoRowIndex, values, primaryIndexKeyPrefix) + tableDesc, tableDesc.GetPrimaryIndex().IndexDesc(), colIDtoRowIndex, values, primaryIndexKeyPrefix) if err != nil { t.Fatalf("unexpected error: %s", err) } @@ -781,7 +781,7 @@ CREATE TABLE t.test ( primaryIndexKeyPrefix := rowenc.MakeIndexKeyPrefix( keys.SystemSQLCodec, tableDesc, tableDesc.GetPrimaryIndexID()) primaryIndexKey, _, err := rowenc.EncodeIndexKey( - tableDesc, tableDesc.PrimaryIndexInterface().IndexDesc(), colIDtoRowIndex, values, primaryIndexKeyPrefix) + tableDesc, tableDesc.GetPrimaryIndex().IndexDesc(), colIDtoRowIndex, values, primaryIndexKeyPrefix) if err != nil { t.Fatalf("unexpected error: %s", err) } @@ -886,7 +886,7 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v1 INT, v2 INT); primaryIndexKeyPrefix := rowenc.MakeIndexKeyPrefix( keys.SystemSQLCodec, tableDesc, tableDesc.GetPrimaryIndexID()) primaryIndexKey, _, err := rowenc.EncodeIndexKey( - tableDesc, tableDesc.PrimaryIndexInterface().IndexDesc(), colIDtoRowIndex, values, primaryIndexKeyPrefix) + tableDesc, tableDesc.GetPrimaryIndex().IndexDesc(), colIDtoRowIndex, values, primaryIndexKeyPrefix) if err != nil { t.Fatalf("unexpected error: %s", err) } diff --git a/pkg/sql/show_create.go b/pkg/sql/show_create.go index c5193f14e5d9..bf0bf3a09403 100644 --- a/pkg/sql/show_create.go +++ b/pkg/sql/show_create.go @@ -96,16 +96,16 @@ func ShowCreateTable( return "", err } f.WriteString(colstr) - if desc.IsPhysicalTable() && desc.PrimaryIndexInterface().GetColumnID(0) == col.ID { + if desc.IsPhysicalTable() && desc.GetPrimaryIndex().GetColumnID(0) == col.ID { // Only set primaryKeyIsOnVisibleColumn to true if the primary key // is on a visible column (not rowid). primaryKeyIsOnVisibleColumn = true } } if primaryKeyIsOnVisibleColumn || - (desc.IsPhysicalTable() && desc.PrimaryIndexInterface().IsSharded()) { + (desc.IsPhysicalTable() && desc.GetPrimaryIndex().IsSharded()) { f.WriteString(",\n\tCONSTRAINT ") - formatQuoteNames(&f.Buffer, desc.PrimaryIndexInterface().GetName()) + formatQuoteNames(&f.Buffer, desc.GetPrimaryIndex().GetName()) f.WriteString(" ") f.WriteString(desc.PrimaryKeyString()) } @@ -142,7 +142,7 @@ func ShowCreateTable( } allIdx := append( append([]catalog.Index{}, desc.PublicNonPrimaryIndexes()...), - desc.PrimaryIndexInterface()) + desc.GetPrimaryIndex()) for _, idx := range allIdx { // Only add indexes to the create_statement column, and not to the // create_nofks column if they are not associated with an INTERLEAVE @@ -189,11 +189,11 @@ func ShowCreateTable( return "", err } - if err := showCreateInterleave(desc.PrimaryIndexInterface().IndexDesc(), &f.Buffer, dbPrefix, lCtx); err != nil { + if err := showCreateInterleave(desc.GetPrimaryIndex().IndexDesc(), &f.Buffer, dbPrefix, lCtx); err != nil { return "", err } if err := ShowCreatePartitioning( - a, p.ExecCfg().Codec, desc, desc.PrimaryIndexInterface().IndexDesc(), &desc.PrimaryIndexInterface().IndexDesc().Partitioning, &f.Buffer, 0 /* indent */, 0, /* colOffset */ + a, p.ExecCfg().Codec, desc, desc.GetPrimaryIndex().IndexDesc(), &desc.GetPrimaryIndex().IndexDesc().Partitioning, &f.Buffer, 0 /* indent */, 0, /* colOffset */ ); err != nil { return "", err } diff --git a/pkg/sql/tablewriter_delete.go b/pkg/sql/tablewriter_delete.go index ac0a84c0d86b..fed9c125ca18 100644 --- a/pkg/sql/tablewriter_delete.go +++ b/pkg/sql/tablewriter_delete.go @@ -123,7 +123,7 @@ func (td *tableDeleter) deleteAllRowsScan( var rf row.Fetcher tableArgs := row.FetcherTableArgs{ Desc: td.tableDesc(), - Index: td.tableDesc().PrimaryIndexInterface().IndexDesc(), + Index: td.tableDesc().GetPrimaryIndex().IndexDesc(), ColIdxMap: td.rd.FetchColIDtoRowIndex, Cols: td.rd.FetchCols, ValNeededForCol: valNeededForCol, @@ -254,7 +254,7 @@ func (td *tableDeleter) deleteIndexScan( var rf row.Fetcher tableArgs := row.FetcherTableArgs{ Desc: td.tableDesc(), - Index: td.tableDesc().PrimaryIndexInterface().IndexDesc(), + Index: td.tableDesc().GetPrimaryIndex().IndexDesc(), ColIdxMap: td.rd.FetchColIDtoRowIndex, Cols: td.rd.FetchCols, ValNeededForCol: valNeededForCol, diff --git a/pkg/sql/zone_config.go b/pkg/sql/zone_config.go index dd6fe8345f1c..db8aa866cba5 100644 --- a/pkg/sql/zone_config.go +++ b/pkg/sql/zone_config.go @@ -317,7 +317,7 @@ func resolveSubzone( indexName := string(zs.TableOrIndex.Index) var index catalog.Index if indexName == "" { - index = table.PrimaryIndexInterface() + index = table.GetPrimaryIndex() indexName = index.GetName() } else { var err error