From 8f3e764c6a565c145f1c96040af1fa100a1d59a2 Mon Sep 17 00:00:00 2001 From: Marius Posta Date: Fri, 8 Jan 2021 10:25:47 -0500 Subject: [PATCH] sql: Revert "Merge #58273" This reverts commit c4947ade83681cc37266c5a4ead5aa1fb46cc337, reversing changes made to d8b5cb0fdd5f0b1b28919539b78c17aee68dc5f4. Release note: None --- pkg/ccl/backupccl/backup_planning.go | 10 +- pkg/ccl/backupccl/restore_job.go | 12 +- pkg/ccl/backupccl/restore_planning.go | 3 +- pkg/ccl/backupccl/targets.go | 10 +- pkg/ccl/changefeedccl/avro_test.go | 2 +- pkg/ccl/changefeedccl/encoder.go | 8 +- pkg/ccl/changefeedccl/rowfetcher_cache.go | 4 +- pkg/ccl/importccl/import_processor.go | 10 +- pkg/ccl/partitionccl/drop_test.go | 6 +- pkg/ccl/partitionccl/partition.go | 7 +- pkg/ccl/partitionccl/partition_test.go | 20 +- pkg/ccl/partitionccl/zone_test.go | 7 +- pkg/ccl/storageccl/key_rewriter.go | 18 +- pkg/server/settingsworker.go | 2 +- pkg/sql/alter_column_type.go | 11 +- pkg/sql/alter_index.go | 4 +- pkg/sql/alter_primary_key.go | 25 +- pkg/sql/alter_table.go | 54 +- pkg/sql/backfill.go | 13 +- pkg/sql/backfill/backfill.go | 4 +- pkg/sql/catalog/BUILD.bazel | 2 - pkg/sql/catalog/descriptor.go | 335 +---------- pkg/sql/catalog/tabledesc/BUILD.bazel | 4 - pkg/sql/catalog/tabledesc/index.go | 527 ------------------ pkg/sql/catalog/tabledesc/index_test.go | 319 ----------- pkg/sql/catalog/tabledesc/safe_format.go | 12 +- pkg/sql/catalog/tabledesc/structured.go | 464 ++++++++++----- pkg/sql/catalog/tabledesc/table.go | 20 +- pkg/sql/catalog/tabledesc/table_desc.go | 217 +++----- pkg/sql/check.go | 6 +- pkg/sql/crdb_internal.go | 91 +-- pkg/sql/create_index.go | 6 +- pkg/sql/create_stats.go | 28 +- pkg/sql/create_table.go | 111 ++-- pkg/sql/delete.go | 3 +- pkg/sql/delete_range.go | 4 +- pkg/sql/descriptor_mutation_test.go | 18 +- pkg/sql/distsql_physical_planner.go | 11 +- pkg/sql/distsql_plan_stats.go | 6 +- pkg/sql/drop_index.go | 123 ++-- pkg/sql/drop_table.go | 21 +- pkg/sql/drop_test.go | 20 +- pkg/sql/flowinfra/cluster_test.go | 2 +- pkg/sql/information_schema.go | 26 +- pkg/sql/insert.go | 3 +- pkg/sql/opt_catalog.go | 37 +- pkg/sql/opt_exec_factory.go | 4 +- pkg/sql/partition_test.go | 6 +- pkg/sql/partition_utils.go | 12 +- pkg/sql/pg_catalog.go | 119 ++-- pkg/sql/pgwire/testdata/pgtest/notice | 2 +- .../physicalplan/fake_span_resolver_test.go | 2 +- pkg/sql/physicalplan/span_resolver_test.go | 2 +- pkg/sql/refresh_materialized_view.go | 11 +- pkg/sql/rename_column.go | 14 +- pkg/sql/rename_index.go | 6 +- pkg/sql/resolver.go | 4 +- pkg/sql/revert.go | 8 +- pkg/sql/row/deleter.go | 17 +- pkg/sql/row/errors.go | 35 +- pkg/sql/row/fetcher.go | 7 +- pkg/sql/row/fetcher_mvcc_test.go | 2 +- pkg/sql/row/fetcher_test.go | 32 +- pkg/sql/row/helper.go | 9 +- pkg/sql/row/inserter.go | 15 +- pkg/sql/row/partial_index.go | 57 +- pkg/sql/row/row_converter.go | 2 +- pkg/sql/row/updater.go | 29 +- pkg/sql/rowenc/client_index_encoding_test.go | 89 +-- pkg/sql/rowenc/index_encoding.go | 14 +- pkg/sql/rowenc/index_encoding_test.go | 30 +- pkg/sql/rowenc/testutils.go | 12 +- pkg/sql/rowexec/BUILD.bazel | 1 - pkg/sql/rowexec/joinreader_test.go | 17 +- pkg/sql/rowexec/scrub_tablereader.go | 9 +- pkg/sql/rowexec/tablereader_test.go | 2 +- pkg/sql/rowexec/zigzagjoiner.go | 10 +- pkg/sql/scan.go | 31 +- pkg/sql/schema_changer.go | 25 +- pkg/sql/schema_changer_test.go | 43 +- pkg/sql/scrub.go | 22 +- pkg/sql/scrub_fk.go | 11 +- pkg/sql/scrub_index.go | 3 +- pkg/sql/scrub_test.go | 50 +- pkg/sql/sem/builtins/builtins.go | 15 +- pkg/sql/set_zone_config.go | 17 +- pkg/sql/show_create.go | 25 +- pkg/sql/show_create_clauses.go | 4 +- pkg/sql/show_fingerprints.go | 8 +- pkg/sql/span_builder_test.go | 4 +- pkg/sql/table_ref_test.go | 2 +- pkg/sql/table_test.go | 13 +- pkg/sql/tablewriter_delete.go | 4 +- pkg/sql/tests/hash_sharded_test.go | 12 +- pkg/sql/truncate.go | 44 +- pkg/sql/update.go | 7 +- pkg/sql/upsert.go | 3 +- pkg/sql/virtual_schema.go | 7 +- pkg/sql/zone_config.go | 10 +- 99 files changed, 1285 insertions(+), 2300 deletions(-) delete mode 100644 pkg/sql/catalog/tabledesc/index.go delete mode 100644 pkg/sql/catalog/tabledesc/index_test.go diff --git a/pkg/ccl/backupccl/backup_planning.go b/pkg/ccl/backupccl/backup_planning.go index 4276f65345d5..668bf5dbf0fd 100644 --- a/pkg/ccl/backupccl/backup_planning.go +++ b/pkg/ccl/backupccl/backup_planning.go @@ -182,13 +182,13 @@ func getLogicallyMergedTableSpans( checkForKVInBounds func(start, end roachpb.Key, endTime hlc.Timestamp) (bool, error), ) ([]roachpb.Span, error) { var nonDropIndexIDs []descpb.IndexID - if err := catalog.ForEachNonDropIndex(table, func(idx catalog.Index) error { - key := tableAndIndex{tableID: table.GetID(), indexID: idx.GetID()} + if err := table.ForeachNonDropIndex(func(idxDesc *descpb.IndexDescriptor) error { + key := tableAndIndex{tableID: table.GetID(), indexID: idxDesc.ID} if added[key] { return nil } added[key] = true - nonDropIndexIDs = append(nonDropIndexIDs, idx.GetID()) + nonDropIndexIDs = append(nonDropIndexIDs, idxDesc.ID) return nil }); err != nil { return nil, err @@ -225,11 +225,11 @@ func getLogicallyMergedTableSpans( lhsSpan := table.IndexSpan(codec, lhsIndexID) rhsSpan := table.IndexSpan(codec, rhsIndexID) - lhsIndex, err := table.FindIndexWithID(lhsIndexID) + lhsIndex, err := table.FindIndexByID(lhsIndexID) if err != nil { return nil, err } - rhsIndex, err := table.FindIndexWithID(rhsIndexID) + rhsIndex, err := table.FindIndexByID(rhsIndexID) if err != nil { return nil, err } diff --git a/pkg/ccl/backupccl/restore_job.go b/pkg/ccl/backupccl/restore_job.go index 1c805ce45207..105db55cbd7a 100644 --- a/pkg/ccl/backupccl/restore_job.go +++ b/pkg/ccl/backupccl/restore_job.go @@ -833,11 +833,11 @@ func spansForAllRestoreTableIndexes( added := make(map[tableAndIndex]bool, len(tables)) sstIntervalTree := interval.NewTree(interval.ExclusiveOverlapper) for _, table := range tables { - for _, index := range table.NonDropIndexes() { - if err := sstIntervalTree.Insert(intervalSpan(table.IndexSpan(codec, index.GetID())), false); err != nil { + for _, index := range table.AllNonDropIndexes() { + if err := sstIntervalTree.Insert(intervalSpan(table.IndexSpan(codec, index.ID)), false); err != nil { panic(errors.NewAssertionErrorWithWrappedErrf(err, "IndexSpan")) } - added[tableAndIndex{tableID: table.GetID(), indexID: index.GetID()}] = true + added[tableAndIndex{tableID: table.GetID(), indexID: index.ID}] = true } } // If there are desc revisions, ensure that we also add any index spans @@ -853,10 +853,10 @@ func spansForAllRestoreTableIndexes( rawTbl := descpb.TableFromDescriptor(rev.Desc, hlc.Timestamp{}) if rawTbl != nil && rawTbl.State != descpb.DescriptorState_DROP { tbl := tabledesc.NewImmutable(*rawTbl) - for _, idx := range tbl.NonDropIndexes() { - key := tableAndIndex{tableID: tbl.ID, indexID: idx.GetID()} + for _, idx := range tbl.AllNonDropIndexes() { + key := tableAndIndex{tableID: tbl.ID, indexID: idx.ID} if !added[key] { - if err := sstIntervalTree.Insert(intervalSpan(tbl.IndexSpan(codec, idx.GetID())), false); err != nil { + if err := sstIntervalTree.Insert(intervalSpan(tbl.IndexSpan(codec, idx.ID)), false); err != nil { panic(errors.NewAssertionErrorWithWrappedErrf(err, "IndexSpan")) } added[key] = true diff --git a/pkg/ccl/backupccl/restore_planning.go b/pkg/ccl/backupccl/restore_planning.go index faf3da356b85..11180465d8dd 100644 --- a/pkg/ccl/backupccl/restore_planning.go +++ b/pkg/ccl/backupccl/restore_planning.go @@ -1021,8 +1021,7 @@ func RewriteTableDescs( return err } - if err := catalog.ForEachNonDropIndex(table, func(indexI catalog.Index) error { - index := indexI.IndexDesc() + if err := table.ForeachNonDropIndex(func(index *descpb.IndexDescriptor) error { // Verify that for any interleaved index being restored, the interleave // parent is also being restored. Otherwise, the interleave entries in the // restored IndexDescriptors won't have anything to point to. diff --git a/pkg/ccl/backupccl/targets.go b/pkg/ccl/backupccl/targets.go index 15cd4b683fec..9ad845ec076c 100644 --- a/pkg/ccl/backupccl/targets.go +++ b/pkg/ccl/backupccl/targets.go @@ -668,19 +668,17 @@ func ensureInterleavesIncluded(tables []catalog.TableDescriptor) error { } for _, table := range tables { - if err := catalog.ForEachIndex(table, catalog.IndexOpts{ + if err := table.ForeachIndex(catalog.IndexOpts{ AddMutations: true, - }, func(index catalog.Index) error { - for i := 0; i < index.NumInterleaveAncestors(); i++ { - a := index.GetInterleaveAncestor(i) + }, func(index *descpb.IndexDescriptor, _ bool) error { + for _, a := range index.Interleave.Ancestors { if !inBackup[a.TableID] { return errors.Errorf( "cannot backup table %q without interleave parent (ID %d)", table.GetName(), a.TableID, ) } } - for i := 0; i < index.NumInterleavedBy(); i++ { - c := index.GetInterleavedBy(i) + for _, c := range index.InterleavedBy { if !inBackup[c.Table] { return errors.Errorf( "cannot backup table %q without interleave child table (ID %d)", table.GetName(), c.Table, diff --git a/pkg/ccl/changefeedccl/avro_test.go b/pkg/ccl/changefeedccl/avro_test.go index 612f2e6853aa..1fb6cdb3e690 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.GetPrimaryIndex().IndexDesc()) + indexSchema, err := indexToAvroSchema(tableDesc, tableDesc.GetPrimaryIndex()) 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 e49f13a3a8d5..d4028a232997 100644 --- a/pkg/ccl/changefeedccl/encoder.go +++ b/pkg/ccl/changefeedccl/encoder.go @@ -142,10 +142,8 @@ 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.GetPrimaryIndex() - jsonEntries := make([]interface{}, primaryIndex.NumColumns()) - for i := 0; i < primaryIndex.NumColumns(); i++ { - colID := primaryIndex.GetColumnID(i) + jsonEntries := make([]interface{}, len(row.tableDesc.GetPrimaryIndex().ColumnIDs)) + for i, colID := range row.tableDesc.GetPrimaryIndex().ColumnIDs { idx, ok := colIdxByID.Get(colID) if !ok { return nil, errors.Errorf(`unknown column id: %d`, colID) @@ -344,7 +342,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.GetPrimaryIndex().IndexDesc()) + registered.schema, err = indexToAvroSchema(row.tableDesc, row.tableDesc.GetPrimaryIndex()) if err != nil { return nil, err } diff --git a/pkg/ccl/changefeedccl/rowfetcher_cache.go b/pkg/ccl/changefeedccl/rowfetcher_cache.go index 15ec48f390e1..4d1146e2f388 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.GetPrimaryIndex().NumColumns(); skippedCols++ { + for ; skippedCols < len(tableDesc.GetPrimaryIndex().ColumnIDs); 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.GetPrimaryIndex().IndexDesc(), + Index: tableDesc.GetPrimaryIndex(), ColIdxMap: colIdxMap, IsSecondaryIndex: false, Cols: tableDesc.Columns, diff --git a/pkg/ccl/importccl/import_processor.go b/pkg/ccl/importccl/import_processor.go index 9da93ace868f..4ebe56aa50e3 100644 --- a/pkg/ccl/importccl/import_processor.go +++ b/pkg/ccl/importccl/import_processor.go @@ -17,7 +17,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/ccl/storageccl" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" @@ -191,10 +190,11 @@ func makeInputConverter( } if singleTable != nil { - if idx := catalog.FindDeletableNonPrimaryIndex(singleTable, func(idx catalog.Index) bool { - return idx.IsPartial() - }); idx != nil { - return nil, unimplemented.NewWithIssue(50225, "cannot import into table with partial indexes") + indexes := singleTable.DeletableIndexes() + for _, idx := range indexes { + if idx.IsPartial() { + return nil, unimplemented.NewWithIssue(50225, "cannot import into table with partial indexes") + } } // If we're using a format like CSV where data columns are not "named", and diff --git a/pkg/ccl/partitionccl/drop_test.go b/pkg/ccl/partitionccl/drop_test.go index 4fc714521c88..19fa3028bb72 100644 --- a/pkg/ccl/partitionccl/drop_test.go +++ b/pkg/ccl/partitionccl/drop_test.go @@ -70,11 +70,11 @@ func TestDropIndexWithZoneConfigCCL(t *testing.T) { PARTITION p2 VALUES IN (2) )`) tableDesc := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "kv") - index, err := tableDesc.FindIndexWithName("i") + indexDesc, _, err := tableDesc.FindIndexByName("i") if err != nil { t.Fatal(err) } - indexSpan := tableDesc.IndexSpan(keys.SystemSQLCodec, index.GetID()) + indexSpan := tableDesc.IndexSpan(keys.SystemSQLCodec, indexDesc.ID) tests.CheckKeyCount(t, kvDB, indexSpan, numRows) // Set zone configs on the primary index, secondary index, and one partition @@ -117,7 +117,7 @@ func TestDropIndexWithZoneConfigCCL(t *testing.T) { } } tableDesc = catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "kv") - if _, err := tableDesc.FindIndexWithName("i"); err == nil { + if _, _, err := tableDesc.FindIndexByName("i"); err == nil { t.Fatalf("table descriptor still contains index after index is dropped") } close(asyncNotification) diff --git a/pkg/ccl/partitionccl/partition.go b/pkg/ccl/partitionccl/partition.go index 5b3941281c89..a9119a9b7169 100644 --- a/pkg/ccl/partitionccl/partition.go +++ b/pkg/ccl/partitionccl/partition.go @@ -275,11 +275,12 @@ func selectPartitionExprs( a := &rowenc.DatumAlloc{} var prefixDatums []tree.Datum - if err := catalog.ForEachIndex(tableDesc, catalog.IndexOpts{ + if err := tableDesc.ForeachIndex(catalog.IndexOpts{ AddMutations: true, - }, func(idx catalog.Index) error { + }, func(idxDesc *descpb.IndexDescriptor, _ bool) error { + genExpr := true return selectPartitionExprsByName( - a, evalCtx, tableDesc, idx.IndexDesc(), &idx.IndexDesc().Partitioning, prefixDatums, exprsByPartName, true /* genExpr */) + a, evalCtx, tableDesc, idxDesc, &idxDesc.Partitioning, prefixDatums, exprsByPartName, genExpr) }); err != nil { return nil, err } diff --git a/pkg/ccl/partitionccl/partition_test.go b/pkg/ccl/partitionccl/partition_test.go index a48ccc3fb810..c63d395e1d97 100644 --- a/pkg/ccl/partitionccl/partition_test.go +++ b/pkg/ccl/partitionccl/partition_test.go @@ -177,21 +177,21 @@ func (pt *partitioningTest) parse() error { if !strings.HasPrefix(indexName, "@") { panic(errors.Errorf("unsupported config: %s", c)) } - idx, err := pt.parsed.tableDesc.FindIndexWithName(indexName[1:]) + idxDesc, _, err := pt.parsed.tableDesc.FindIndexByName(indexName[1:]) if err != nil { return errors.Wrapf(err, "could not find index %s", indexName) } - subzone.IndexID = uint32(idx.GetID()) + subzone.IndexID = uint32(idxDesc.ID) if len(constraints) > 0 { if subzone.PartitionName == "" { fmt.Fprintf(&zoneConfigStmts, `ALTER INDEX %s@%s CONFIGURE ZONE USING constraints = '[%s]';`, - pt.parsed.tableName, idx.GetName(), constraints, + pt.parsed.tableName, idxDesc.Name, constraints, ) } else { fmt.Fprintf(&zoneConfigStmts, `ALTER PARTITION %s OF INDEX %s@%s CONFIGURE ZONE USING constraints = '[%s]';`, - subzone.PartitionName, pt.parsed.tableName, idx.GetName(), constraints, + subzone.PartitionName, pt.parsed.tableName, idxDesc.Name, constraints, ) } } @@ -1326,23 +1326,23 @@ func TestRepartitioning(t *testing.T) { } sqlDB.Exec(t, fmt.Sprintf("ALTER TABLE %s RENAME TO %s", test.old.parsed.tableName, test.new.parsed.tableName)) - testIndex, err := test.new.parsed.tableDesc.FindIndexWithName(test.index) + testIndex, _, err := test.new.parsed.tableDesc.FindIndexByName(test.index) if err != nil { t.Fatalf("%+v", err) } var repartition bytes.Buffer - if testIndex.GetID() == test.new.parsed.tableDesc.GetPrimaryIndexID() { + if testIndex.ID == test.new.parsed.tableDesc.GetPrimaryIndexID() { fmt.Fprintf(&repartition, `ALTER TABLE %s `, test.new.parsed.tableName) } else { - fmt.Fprintf(&repartition, `ALTER INDEX %s@%s `, test.new.parsed.tableName, testIndex.GetName()) + fmt.Fprintf(&repartition, `ALTER INDEX %s@%s `, test.new.parsed.tableName, testIndex.Name) } - if testIndex.GetPartitioning().NumColumns == 0 { + if testIndex.Partitioning.NumColumns == 0 { repartition.WriteString(`PARTITION BY NOTHING`) } else { if err := sql.ShowCreatePartitioning( - &rowenc.DatumAlloc{}, keys.SystemSQLCodec, test.new.parsed.tableDesc, testIndex.IndexDesc(), - &testIndex.IndexDesc().Partitioning, &repartition, 0 /* indent */, 0, /* colOffset */ + &rowenc.DatumAlloc{}, keys.SystemSQLCodec, test.new.parsed.tableDesc, testIndex, + &testIndex.Partitioning, &repartition, 0 /* indent */, 0, /* colOffset */ ); err != nil { t.Fatalf("%+v", err) } diff --git a/pkg/ccl/partitionccl/zone_test.go b/pkg/ccl/partitionccl/zone_test.go index 86c9b1e19182..2e3190f3f8e1 100644 --- a/pkg/ccl/partitionccl/zone_test.go +++ b/pkg/ccl/partitionccl/zone_test.go @@ -301,14 +301,13 @@ func TestGenerateSubzoneSpans(t *testing.T) { var actual []string for _, span := range spans { subzone := test.parsed.subzones[span.SubzoneIndex] - idx, err := test.parsed.tableDesc.FindIndexWithID(descpb.IndexID(subzone.IndexID)) + idxDesc, err := test.parsed.tableDesc.FindIndexByID(descpb.IndexID(subzone.IndexID)) if err != nil { t.Fatalf("could not find index with ID %d: %+v", subzone.IndexID, err) } directions := []encoding.Direction{encoding.Ascending /* index ID */} - for i := 0; i < idx.NumColumns(); i++ { - cd := idx.GetColumnDirection(i) + for _, cd := range idxDesc.ColumnDirections { ed, err := cd.ToEncodingDirection() if err != nil { t.Fatal(err) @@ -320,7 +319,7 @@ func TestGenerateSubzoneSpans(t *testing.T) { if len(subzone.PartitionName) > 0 { subzoneShort = "." + subzone.PartitionName } else { - subzoneShort = "@" + idx.GetName() + subzoneShort = "@" + idxDesc.Name } // Verify that we're always doing the space savings when we can. diff --git a/pkg/ccl/storageccl/key_rewriter.go b/pkg/ccl/storageccl/key_rewriter.go index c12b906fbd9e..a44734e71763 100644 --- a/pkg/ccl/storageccl/key_rewriter.go +++ b/pkg/ccl/storageccl/key_rewriter.go @@ -105,9 +105,9 @@ func MakeKeyRewriter(descs map[descpb.ID]*tabledesc.Immutable) (*KeyRewriter, er // The PrefixEnd() of index 1 is the same as the prefix of index 2, so use a // map to avoid duplicating entries. - for _, index := range desc.NonDropIndexes() { - oldPrefix := roachpb.Key(makeKeyRewriterPrefixIgnoringInterleaved(oldID, index.GetID())) - newPrefix := roachpb.Key(makeKeyRewriterPrefixIgnoringInterleaved(desc.ID, index.GetID())) + for _, index := range desc.AllNonDropIndexes() { + oldPrefix := roachpb.Key(makeKeyRewriterPrefixIgnoringInterleaved(oldID, index.ID)) + newPrefix := roachpb.Key(makeKeyRewriterPrefixIgnoringInterleaved(desc.ID, index.ID)) if !seenPrefixes[string(oldPrefix)] { seenPrefixes[string(oldPrefix)] = true prefixes.rewrites = append(prefixes.rewrites, prefixRewrite{ @@ -190,22 +190,22 @@ func (kr *KeyRewriter) RewriteKey(key []byte, isFromSpan bool) ([]byte, bool, er // If there isn't any more data, we are at some split boundary. return key, true, nil } - idx, err := desc.FindIndexWithID(descpb.IndexID(indexID)) + idx, err := desc.FindIndexByID(descpb.IndexID(indexID)) if err != nil { return nil, false, err } - if idx.NumInterleavedBy() == 0 { + if len(idx.InterleavedBy) == 0 { // Not interleaved. return key, true, nil } // We do not support interleaved secondary indexes. - if !idx.Primary() { + if idx.ID != desc.GetPrimaryIndexID() { return nil, false, errors.New("restoring interleaved secondary indexes not supported") } - colIDs, _ := idx.IndexDesc().FullColumnIDs() + colIDs, _ := idx.FullColumnIDs() var skipCols int - for i := 0; i < idx.NumInterleaveAncestors(); i++ { - skipCols += int(idx.GetInterleaveAncestor(i).SharedPrefixLen) + for _, ancestor := range idx.Interleave.Ancestors { + skipCols += int(ancestor.SharedPrefixLen) } for i := 0; i < len(colIDs)-skipCols; i++ { n, err := encoding.PeekLength(k) diff --git a/pkg/server/settingsworker.go b/pkg/server/settingsworker.go index 9d099ee31457..17ed42db5dac 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.GetPrimaryIndex().IndexDesc(), types, nameRow, nil, kv.Key) + _, matches, _, err := rowenc.DecodeIndexKey(codec, tbl, tbl.GetPrimaryIndex(), types, nameRow, nil, kv.Key) if err != nil { return errors.Wrap(err, "failed to decode key") } diff --git a/pkg/sql/alter_column_type.go b/pkg/sql/alter_column_type.go index 5e9d1e84b69b..b134a4e0d80c 100644 --- a/pkg/sql/alter_column_type.go +++ b/pkg/sql/alter_column_type.go @@ -220,14 +220,9 @@ func alterColumnTypeGeneral( // Disallow ALTER COLUMN TYPE general for columns that are // part of indexes. - for _, idx := range tableDesc.NonDropIndexes() { - for i := 0; i < idx.NumColumns(); i++ { - if idx.GetColumnID(i) == col.ID { - return colInIndexNotSupportedErr - } - } - for i := 0; i < idx.NumExtraColumns(); i++ { - if idx.GetExtraColumnID(i) == col.ID { + for _, idx := range tableDesc.AllNonDropIndexes() { + for _, id := range append(idx.ColumnIDs, idx.ExtraColumnIDs...) { + if col.ID == id { return colInIndexNotSupportedErr } } diff --git a/pkg/sql/alter_index.go b/pkg/sql/alter_index.go index e6e04c7a0d1c..73db4001448a 100644 --- a/pkg/sql/alter_index.go +++ b/pkg/sql/alter_index.go @@ -48,11 +48,11 @@ func (p *planner) AlterIndex(ctx context.Context, n *tree.AlterIndex) (planNode, // different copy than the one in the tableDesc. To make it easier for the // code below, get a pointer to the index descriptor that's actually in // tableDesc. - index, err := tableDesc.FindIndexWithID(indexDesc.ID) + indexDesc, err = tableDesc.FindIndexByID(indexDesc.ID) if err != nil { return nil, err } - return &alterIndexNode{n: n, tableDesc: tableDesc, indexDesc: index.IndexDesc()}, nil + return &alterIndexNode{n: n, tableDesc: tableDesc, indexDesc: indexDesc}, nil } // ReadingOwnWrites implements the planNodeReadingOwnWrites interface. diff --git a/pkg/sql/alter_primary_key.go b/pkg/sql/alter_primary_key.go index 9f4a2777418d..70fd900f7f55 100644 --- a/pkg/sql/alter_primary_key.go +++ b/pkg/sql/alter_primary_key.go @@ -91,19 +91,19 @@ func (p *planner) AlterPrimaryKey( } // Disable primary key changes on tables that are interleaved parents. - if tableDesc.GetPrimaryIndex().NumInterleavedBy() != 0 { + if len(tableDesc.GetPrimaryIndex().InterleavedBy) != 0 { var sb strings.Builder sb.WriteString("[") comma := ", " - for i := 0; i < tableDesc.GetPrimaryIndex().NumInterleavedBy(); i++ { - interleaveTableID := tableDesc.GetPrimaryIndex().GetInterleavedBy(i).Table + for i := range tableDesc.GetPrimaryIndex().InterleavedBy { + interleave := &tableDesc.GetPrimaryIndex().InterleavedBy[i] if i != 0 { sb.WriteString(comma) } childTable, err := p.Descriptors().GetTableVersionByID( ctx, p.Txn(), - interleaveTableID, + interleave.Table, tree.ObjectLookupFlags{}, ) if err != nil { @@ -120,7 +120,7 @@ func (p *planner) AlterPrimaryKey( } nameExists := func(name string) bool { - _, err := tableDesc.FindIndexWithName(name) + _, _, err := tableDesc.FindIndexByName(name) return err == nil } @@ -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.GetPrimaryIndex().IndexDescDeepCopy() + oldPrimaryIndexCopy := protoutil.Clone(tableDesc.GetPrimaryIndex()).(*descpb.IndexDescriptor) // Clear the name of the index so that it gets generated by AllocateIDs. oldPrimaryIndexCopy.Name = "" oldPrimaryIndexCopy.StoreColumnIDs = nil @@ -230,7 +230,7 @@ func (p *planner) AlterPrimaryKey( // Make the copy of the old primary index not-interleaved. This decision // can be revisited based on user experience. oldPrimaryIndexCopy.Interleave = descpb.InterleaveDescriptor{} - if err := addIndexMutationWithSpecificPrimaryKey(ctx, tableDesc, &oldPrimaryIndexCopy, newPrimaryIndexDesc); err != nil { + if err := addIndexMutationWithSpecificPrimaryKey(ctx, tableDesc, oldPrimaryIndexCopy, newPrimaryIndexDesc); err != nil { return err } } @@ -261,9 +261,10 @@ func (p *planner) AlterPrimaryKey( return shouldRewrite || !idx.Unique || idx.Type == descpb.IndexDescriptor_INVERTED } var indexesToRewrite []*descpb.IndexDescriptor - for _, idx := range tableDesc.PublicNonPrimaryIndexes() { - if idx.GetID() != newPrimaryIndexDesc.ID && shouldRewriteIndex(idx.IndexDesc()) { - indexesToRewrite = append(indexesToRewrite, idx.IndexDesc()) + for i := range tableDesc.GetPublicNonPrimaryIndexes() { + idx := &tableDesc.GetPublicNonPrimaryIndexes()[i] + if idx.ID != newPrimaryIndexDesc.ID && shouldRewriteIndex(idx) { + indexesToRewrite = append(indexesToRewrite, idx) } } @@ -314,7 +315,7 @@ func (p *planner) AlterPrimaryKey( // Mark the primary key of the table as valid. { - primaryIndex := *tableDesc.GetPrimaryIndex().IndexDesc() + primaryIndex := *tableDesc.GetPrimaryIndex() primaryIndex.Disabled = false tableDesc.SetPrimaryIndex(primaryIndex) } @@ -354,7 +355,7 @@ func shouldCopyPrimaryKey(desc *tabledesc.Mutable, newPK *descpb.IndexDescriptor // The first column in the columnIDs is the shard column, which will be different. // Slice it out to see what the actual index columns are. if oldPK.IsSharded() && newPK.IsSharded() && - descpb.ColumnIDs(oldPK.IndexDesc().ColumnIDs[1:]).Equals(newPK.ColumnIDs[1:]) { + descpb.ColumnIDs(oldPK.ColumnIDs[1:]).Equals(newPK.ColumnIDs[1:]) { return false } diff --git a/pkg/sql/alter_table.go b/pkg/sql/alter_table.go index 6d4e170bdb40..fd2f2cc7a672 100644 --- a/pkg/sql/alter_table.go +++ b/pkg/sql/alter_table.go @@ -228,9 +228,9 @@ func (n *alterTableNode) startExec(params runParams) error { } idx.Partitioning = partitioning } - foundIndex, err := n.tableDesc.FindIndexWithName(string(d.Name)) + _, dropped, err := n.tableDesc.FindIndexByName(string(d.Name)) if err == nil { - if foundIndex.Dropped() { + if dropped { return pgerror.Newf(pgcode.ObjectNotInPrerequisiteState, "index %q being dropped, try again later", d.Name) } @@ -441,7 +441,7 @@ func (n *alterTableNode) startExec(params runParams) error { "column %q is referenced by the primary key", colToDrop.Name) } var idxNamesToDelete []string - for _, idx := range n.tableDesc.NonDropIndexes() { + for _, idx := range n.tableDesc.AllNonDropIndexes() { // We automatically drop indexes that reference the column // being dropped. @@ -450,15 +450,14 @@ func (n *alterTableNode) startExec(params runParams) error { containsThisColumn := false // Analyze the index. - for j := 0; j < idx.NumColumns(); j++ { - if idx.GetColumnID(j) == colToDrop.ID { + for _, id := range idx.ColumnIDs { + if id == colToDrop.ID { containsThisColumn = true break } } if !containsThisColumn { - for j := 0; j < idx.NumExtraColumns(); j++ { - id := idx.GetExtraColumnID(j) + for _, id := range idx.ExtraColumnIDs { if n.tableDesc.GetPrimaryIndex().ContainsColumnID(id) { // All secondary indices necessary contain the PK // columns, too. (See the comments on the definition of @@ -477,8 +476,8 @@ func (n *alterTableNode) startExec(params runParams) error { // The loop above this comment is for the old STORING encoding. The // loop below is for the new encoding (where the STORING columns are // always in the value part of a KV). - for j := 0; j < idx.NumStoredColumns(); j++ { - if idx.GetStoredColumnID(j) == colToDrop.ID { + for _, id := range idx.StoreColumnIDs { + if id == colToDrop.ID { containsThisColumn = true break } @@ -488,7 +487,7 @@ func (n *alterTableNode) startExec(params runParams) error { // If the column being dropped is referenced in the partial // index predicate, then the index should be dropped. if !containsThisColumn && idx.IsPartial() { - expr, err := parser.ParseExpr(idx.GetPredicate()) + expr, err := parser.ParseExpr(idx.Predicate) if err != nil { return err } @@ -505,7 +504,7 @@ func (n *alterTableNode) startExec(params runParams) error { // Perform the DROP. if containsThisColumn { - idxNamesToDelete = append(idxNamesToDelete, idx.GetName()) + idxNamesToDelete = append(idxNamesToDelete, idx.Name) } } @@ -740,28 +739,26 @@ func (n *alterTableNode) startExec(params runParams) error { descriptorChanged = true case *tree.AlterTablePartitionBy: - primaryIndex := n.tableDesc.GetPrimaryIndex() - newPartitioning, err := CreatePartitioning( + partitioning, err := CreatePartitioning( params.ctx, params.p.ExecCfg().Settings, params.EvalContext(), - n.tableDesc, primaryIndex.IndexDesc(), t.PartitionBy) + n.tableDesc, n.tableDesc.GetPrimaryIndex(), t.PartitionBy) if err != nil { return err } - oldPartitioning := primaryIndex.GetPartitioning() - descriptorChanged = descriptorChanged || !oldPartitioning.Equal(&newPartitioning) + descriptorChanged = descriptorChanged || !n.tableDesc.GetPrimaryIndex().Partitioning.Equal(&partitioning) err = deleteRemovedPartitionZoneConfigs( params.ctx, params.p.txn, - n.tableDesc, primaryIndex.IndexDesc(), &oldPartitioning, - &newPartitioning, params.extendedEvalCtx.ExecCfg, + n.tableDesc, n.tableDesc.GetPrimaryIndex(), &n.tableDesc.GetPrimaryIndex().Partitioning, + &partitioning, params.extendedEvalCtx.ExecCfg, ) if err != nil { return err } { - newPrimaryIndex := *primaryIndex.IndexDesc() - newPrimaryIndex.Partitioning = newPartitioning - n.tableDesc.SetPrimaryIndex(newPrimaryIndex) + primaryIndex := *n.tableDesc.GetPrimaryIndex() + primaryIndex.Partitioning = partitioning + n.tableDesc.SetPrimaryIndex(primaryIndex) } case *tree.AlterTableSetAudit: @@ -816,11 +813,16 @@ func (n *alterTableNode) startExec(params runParams) error { // new name exists. This is what postgres does. switch details.Kind { case descpb.ConstraintTypeUnique, descpb.ConstraintTypePK: - if catalog.FindNonDropIndex(n.tableDesc, func(idx catalog.Index) bool { - return idx.GetName() == string(t.NewName) - }) != nil { - return pgerror.Newf(pgcode.DuplicateRelation, - "relation %v already exists", t.NewName) + if err := n.tableDesc.ForeachNonDropIndex(func( + descriptor *descpb.IndexDescriptor, + ) error { + if descriptor.Name == string(t.NewName) { + return pgerror.Newf(pgcode.DuplicateRelation, + "relation %v already exists", t.NewName) + } + return nil + }); err != nil { + return err } } diff --git a/pkg/sql/backfill.go b/pkg/sql/backfill.go index 08401ad1eedf..a15711fd4004 100644 --- a/pkg/sql/backfill.go +++ b/pkg/sql/backfill.go @@ -1648,28 +1648,27 @@ func runSchemaChangesInTxn( // write the modified table descriptors explicitly. for _, idxID := range append( []descpb.IndexID{pkSwap.OldPrimaryIndexId}, pkSwap.OldIndexes...) { - oldIndex, err := tableDesc.FindIndexWithID(idxID) + oldIndex, err := tableDesc.FindIndexByID(idxID) if err != nil { return err } - if oldIndex.NumInterleaveAncestors() != 0 { - ancestorInfo := oldIndex.GetInterleaveAncestor(oldIndex.NumInterleaveAncestors() - 1) + if len(oldIndex.Interleave.Ancestors) != 0 { + ancestorInfo := oldIndex.Interleave.Ancestors[len(oldIndex.Interleave.Ancestors)-1] ancestor, err := planner.Descriptors().GetMutableTableVersionByID(ctx, ancestorInfo.TableID, planner.txn) if err != nil { return err } - ancestorIdxI, err := ancestor.FindIndexWithID(ancestorInfo.IndexID) + ancestorIdx, err := ancestor.FindIndexByID(ancestorInfo.IndexID) if err != nil { return err } - ancestorIdx := ancestorIdxI.IndexDesc() foundAncestor := false for k, ref := range ancestorIdx.InterleavedBy { - if ref.Table == tableDesc.ID && ref.Index == oldIndex.GetID() { + if ref.Table == tableDesc.ID && ref.Index == oldIndex.ID { if foundAncestor { return errors.AssertionFailedf( "ancestor entry in %s for %s@%s found more than once", - ancestor.Name, tableDesc.Name, oldIndex.GetName()) + ancestor.Name, tableDesc.Name, oldIndex.Name) } ancestorIdx.InterleavedBy = append( ancestorIdx.InterleavedBy[:k], ancestorIdx.InterleavedBy[k+1:]...) diff --git a/pkg/sql/backfill/backfill.go b/pkg/sql/backfill/backfill.go index 4812f6a2b016..030cab6a678a 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.GetPrimaryIndex().IndexDesc(), + Index: desc.GetPrimaryIndex(), ColIdxMap: desc.ColumnIdxMap(), Cols: desc.Columns, ValNeededForCol: valNeededForCol, @@ -602,7 +602,7 @@ func (ib *IndexBackfiller) init( tableArgs := row.FetcherTableArgs{ Desc: desc, - Index: desc.GetPrimaryIndex().IndexDesc(), + Index: desc.GetPrimaryIndex(), ColIdxMap: ib.colIdxMap, Cols: ib.cols, ValNeededForCol: valNeededForCol, diff --git a/pkg/sql/catalog/BUILD.bazel b/pkg/sql/catalog/BUILD.bazel index 36575a9cefe6..7f26b7ae03db 100644 --- a/pkg/sql/catalog/BUILD.bazel +++ b/pkg/sql/catalog/BUILD.bazel @@ -14,7 +14,6 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/sql/catalog", visibility = ["//visibility:public"], deps = [ - "//pkg/geo/geoindex", "//pkg/keys", "//pkg/kv", "//pkg/roachpb", @@ -26,7 +25,6 @@ go_library( "//pkg/sql/types", "//pkg/util", "//pkg/util/hlc", - "//pkg/util/iterutil", "@com_github_cockroachdb_errors//:errors", "@com_github_cockroachdb_redact//:redact", ], diff --git a/pkg/sql/catalog/descriptor.go b/pkg/sql/catalog/descriptor.go index cc5001dfe33c..e21607bf954f 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" @@ -21,14 +20,13 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/hlc" - "github.com/cockroachdb/cockroach/pkg/util/iterutil" "github.com/cockroachdb/errors" "github.com/cockroachdb/redact" ) -// IndexOpts configures the behavior of catalog.ForEachIndex and -// catalog.FindIndex. +// IndexOpts configures the behavior of TableDescriptor.ForeachIndex. type IndexOpts struct { // NonPhysicalPrimaryIndex should be included. NonPhysicalPrimaryIndex bool @@ -114,79 +112,22 @@ type TableDescriptor interface { GetFormatVersion() descpb.FormatVersion GetPrimaryIndexID() descpb.IndexID - GetPrimaryIndex() Index + GetPrimaryIndex() *descpb.IndexDescriptor PrimaryIndexSpan(codec keys.SQLCodec) roachpb.Span + GetPublicNonPrimaryIndexes() []descpb.IndexDescriptor + ForeachIndex(opts IndexOpts, f func(idxDesc *descpb.IndexDescriptor, isPrimary bool) error) error + AllNonDropIndexes() []*descpb.IndexDescriptor + ForeachNonDropIndex(f func(idxDesc *descpb.IndexDescriptor) error) error IndexSpan(codec keys.SQLCodec, id descpb.IndexID) roachpb.Span + FindIndexByID(id descpb.IndexID) (*descpb.IndexDescriptor, error) + FindIndexByName(name string) (_ *descpb.IndexDescriptor, dropped bool, _ error) + FindIndexesWithPartition(name string) []*descpb.IndexDescriptor GetIndexMutationCapabilities(id descpb.IndexID) (isMutation, isWriteOnly bool) KeysPerRow(id descpb.IndexID) (int, error) - - // AllIndexes returns a slice with all indexes, public and non-public, - // in the underlying proto, in their canonical order: - // - the primary index, - // - the public non-primary indexes in the Indexes array, in order, - // - the non-public indexes present in the Mutations array, in order. - // - // See also Index.Ordinal(). - AllIndexes() []Index - - // ActiveIndexes returns a slice with all public indexes in the underlying - // proto, in their canonical order: - // - the primary index, - // - the public non-primary indexes in the Indexes array, in order. - // - // See also Index.Ordinal(). - ActiveIndexes() []Index - - // NonDropIndexes returns a slice of all non-drop indexes in the underlying - // proto, in their canonical order. This means: - // - the primary index, if the table is a physical table, - // - the public non-primary indexes in the Indexes array, in order, - // - the non-public indexes present in the Mutations array, in order, - // if the mutation is not a drop. - // - // See also Index.Ordinal(). - NonDropIndexes() []Index - - // NonDropIndexes returns a slice of all partial indexes in the underlying - // proto, in their canonical order. This is equivalent to taking the slice - // produced by AllIndexes and removing indexes with empty expressions. - PartialIndexes() []Index - - // PublicNonPrimaryIndexes returns a slice of all active secondary indexes, - // in their canonical order. This is equivalent to the Indexes array in the - // proto. - PublicNonPrimaryIndexes() []Index - - // WritableNonPrimaryIndexes returns a slice of all non-primary indexes which - // allow being written to: public + delete-and-write-only, in their canonical - // order. This is equivalent to taking the slice produced by - // DeletableNonPrimaryIndexes and removing the indexes which are in mutations - // in the delete-only state. - WritableNonPrimaryIndexes() []Index - - // DeletableNonPrimaryIndexes returns a slice of all non-primary indexes - // which allow being deleted from: public + delete-and-write-only + - // delete-only, in their canonical order. This is equivalent to taking - // the slice produced by AllIndexes and removing the primary index. - DeletableNonPrimaryIndexes() []Index - - // DeleteOnlyNonPrimaryIndexes returns a slice of all non-primary indexes - // which allow only being deleted from, in their canonical order. This is - // equivalent to taking the slice produced by DeletableNonPrimaryIndexes and - // removing the indexes which are not in mutations or not in the delete-only - // state. - DeleteOnlyNonPrimaryIndexes() []Index - - // FindIndexWithID returns the first catalog.Index that matches the id - // in the set of all indexes, or an error if none was found. The order of - // traversal is the canonical order, see Index.Ordinal(). - FindIndexWithID(id descpb.IndexID) (Index, error) - - // FindIndexWithName returns the first catalog.Index that matches the name in - // the set of all indexes, excluding the primary index of non-physical - // tables, or an error if none was found. The order of traversal is the - // canonical order, see Index.Ordinal(). - FindIndexWithName(name string) (Index, error) + PartialIndexOrds() util.FastIntSet + WritableIndexes() []descpb.IndexDescriptor + DeletableIndexes() []descpb.IndexDescriptor + DeleteOnlyIndexes() []descpb.IndexDescriptor HasPrimaryKey() bool PrimaryKeyString() string @@ -246,106 +187,6 @@ type TableDescriptor interface { WritableColumns() []descpb.ColumnDescriptor } -// 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 - IsValidReferencedIndex(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 -} - // TypeDescriptor will eventually be called typedesc.Descriptor. // It is implemented by (Imm|M)utableTypeDescriptor. type TypeDescriptor interface { @@ -441,151 +282,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) -} diff --git a/pkg/sql/catalog/tabledesc/BUILD.bazel b/pkg/sql/catalog/tabledesc/BUILD.bazel index 69ed02683e20..a107a91d4188 100644 --- a/pkg/sql/catalog/tabledesc/BUILD.bazel +++ b/pkg/sql/catalog/tabledesc/BUILD.bazel @@ -3,7 +3,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "tabledesc", srcs = [ - "index.go", "safe_format.go", "structured.go", "table.go", @@ -12,7 +11,6 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc", visibility = ["//visibility:public"], deps = [ - "//pkg/geo/geoindex", "//pkg/keys", "//pkg/roachpb", "//pkg/settings/cluster", @@ -45,7 +43,6 @@ go_test( name = "tabledesc_test", srcs = [ "helpers_test.go", - "index_test.go", "main_test.go", "safe_format_test.go", "structured_test.go", @@ -55,7 +52,6 @@ go_test( embed = [":tabledesc"], deps = [ "//pkg/base", - "//pkg/geo/geoindex", "//pkg/keys", "//pkg/security", "//pkg/security/securitytest", diff --git a/pkg/sql/catalog/tabledesc/index.go b/pkg/sql/catalog/tabledesc/index.go deleted file mode 100644 index e13cc7fcfe5a..000000000000 --- a/pkg/sql/catalog/tabledesc/index.go +++ /dev/null @@ -1,527 +0,0 @@ -// Copyright 2020 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/geo/geoindex" - "github.com/cockroachdb/cockroach/pkg/sql/catalog" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/util/protoutil" -) - -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 -} - -// IndexDesc returns the underlying protobuf descriptor. -// Ideally, this method should be called as rarely as possible. -func (w index) IndexDesc() *descpb.IndexDescriptor { - return w.desc -} - -// IndexDescDeepCopy returns a deep copy of the underlying protobuf descriptor. -func (w index) IndexDescDeepCopy() descpb.IndexDescriptor { - return *protoutil.Clone(w.desc).(*descpb.IndexDescriptor) -} - -// Ordinal returns the ordinal of the index in its parent TableDescriptor. -// The ordinal is defined as follows: -// - 0 is the ordinal of the primary index, -// - [1:1+len(desc.Indexes)] is the range of public non-primary indexes, -// - [1+len(desc.Indexes):] is the range of non-public indexes. -func (w index) Ordinal() int { - return w.ordinal -} - -// Primary returns true iff the index is the primary index for the table -// descriptor. -func (w index) Primary() bool { - return w.ordinal == 0 -} - -// 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 -} - -// GetID returns the index ID. -func (w index) GetID() descpb.IndexID { - return w.desc.ID -} - -// GetName returns the index name. -func (w index) GetName() string { - return w.desc.Name -} - -// IsInterleaved returns true iff the index is interleaved. -func (w index) IsInterleaved() bool { - return w.desc.IsInterleaved() -} - -// IsPartial returns true iff the index is a partial index. -func (w index) IsPartial() bool { - return w.desc.IsPartial() -} - -// IsUnique returns true iff the index is a unique index. -func (w index) IsUnique() bool { - return w.desc.Unique -} - -// IsDisabled returns true iff the index is disabled. -func (w index) IsDisabled() bool { - return w.desc.Disabled -} - -// IsSharded returns true iff the index is hash sharded. -func (w index) IsSharded() bool { - return w.desc.IsSharded() -} - -// IsCreatedExplicitly returns true iff this index was created explicitly, i.e. -// via 'CREATE INDEX' statement. -func (w index) IsCreatedExplicitly() bool { - return w.desc.CreatedExplicitly -} - -// GetPredicate returns the empty string when the index is not partial, -// otherwise it returns the corresponding expression of the partial index. -// Columns are referred to in the expression by their name. -func (w index) GetPredicate() string { - return w.desc.Predicate -} - -// GetType returns the type of index, inverted or forward. -func (w index) GetType() descpb.IndexDescriptor_Type { - return w.desc.Type -} - -// GetPartitioning returns the partitioning descriptor of the index. -func (w index) GetPartitioning() descpb.PartitioningDescriptor { - return w.desc.Partitioning -} - -// FindPartitionByName searches the index's partitioning descriptor for a -// partition whose name is the input and returns it, or nil if no match is found. -func (w index) FindPartitionByName(name string) descpb.PartitioningDescriptor { - return *w.desc.Partitioning.FindPartitionByName(name) -} - -// PartitionNames returns a slice containing the name of every partition and -// subpartition in an arbitrary order. -func (w index) PartitionNames() []string { - return w.desc.Partitioning.PartitionNames() -} - -// IsValidOriginIndex returns whether the index can serve as an origin index for -// a foreign key constraint with the provided set of originColIDs. -func (w index) IsValidOriginIndex(originColIDs descpb.ColumnIDs) bool { - return w.desc.IsValidOriginIndex(originColIDs) -} - -// IsValidReferencedIndex returns whether the index can serve as a referenced -// index for a foreign key constraint with the provided set of -// referencedColumnIDs. -func (w index) IsValidReferencedIndex(referencedColIDs descpb.ColumnIDs) bool { - return w.desc.IsValidReferencedIndex(referencedColIDs) -} - -// HasOldStoredColumns returns whether the index has stored columns in the old -// format (data encoded the same way as if they were in an implicit column). -func (w index) HasOldStoredColumns() bool { - return w.desc.HasOldStoredColumns() -} - -// InvertedColumnID returns the ColumnID of the inverted column of the inverted -// index. This is always the last column in ColumnIDs. Panics if the index is -// not inverted. -func (w index) InvertedColumnID() descpb.ColumnID { - return w.desc.InvertedColumnID() -} - -// InvertedColumnName returns the name of the inverted column of the inverted -// index. This is always the last column in ColumnNames. Panics if the index is -// not inverted. -func (w index) InvertedColumnName() string { - return w.desc.InvertedColumnName() -} - -// ContainsColumnID returns true if the index descriptor contains the specified -// column ID either in its explicit column IDs, the extra column IDs, or the -// stored column IDs. -func (w index) ContainsColumnID(colID descpb.ColumnID) bool { - return w.desc.ContainsColumnID(colID) -} - -// GetGeoConfig returns the geo config in the index descriptor. -func (w index) GetGeoConfig() geoindex.Config { - return w.desc.GeoConfig -} - -// GetSharded returns the ShardedDescriptor in the index descriptor -func (w index) GetSharded() descpb.ShardedDescriptor { - return w.desc.Sharded -} - -// GetShardColumnName returns the name of the shard column if the index is hash -// sharded, empty string otherwise. -func (w index) GetShardColumnName() string { - return w.desc.Sharded.Name -} - -// GetVersion returns the version of the index descriptor. -func (w index) GetVersion() descpb.IndexDescriptorVersion { - return w.desc.Version -} - -// GetEncodingType returns the encoding type of this index. For backward -// compatibility reasons, this might not match what is stored in -// w.desc.EncodingType. -func (w index) GetEncodingType() descpb.IndexDescriptorEncodingType { - if w.Primary() { - // Primary indexes always use the PrimaryIndexEncoding, regardless of what - // desc.EncodingType indicates. - return descpb.PrimaryIndexEncoding - } - return w.desc.EncodingType -} - -// NumInterleaveAncestors returns the number of interleave ancestors as per the -// index descriptor. -func (w index) NumInterleaveAncestors() int { - return len(w.desc.Interleave.Ancestors) -} - -// GetInterleaveAncestor returns the ancestorOrdinal-th interleave ancestor. -func (w index) GetInterleaveAncestor(ancestorOrdinal int) descpb.InterleaveDescriptor_Ancestor { - return w.desc.Interleave.Ancestors[ancestorOrdinal] -} - -// NumInterleavedBy returns the number of tables/indexes that are interleaved -// into this index. -func (w index) NumInterleavedBy() int { - return len(w.desc.InterleavedBy) -} - -// GetInterleavedBy returns the interleavedByOrdinal-th table/index that is -// interleaved into this index. -func (w index) GetInterleavedBy(interleavedByOrdinal int) descpb.ForeignKeyReference { - return w.desc.InterleavedBy[interleavedByOrdinal] -} - -// NumColumns returns the number of columns as per the index descriptor. -func (w index) NumColumns() int { - return len(w.desc.ColumnIDs) -} - -// GetColumnID returns the ID of the columnOrdinal-th column. -func (w index) GetColumnID(columnOrdinal int) descpb.ColumnID { - return w.desc.ColumnIDs[columnOrdinal] -} - -// GetColumnName returns the name of the columnOrdinal-th column. -func (w index) GetColumnName(columnOrdinal int) string { - return w.desc.ColumnNames[columnOrdinal] -} - -// GetColumnDirection returns the direction of the columnOrdinal-th column. -func (w index) GetColumnDirection(columnOrdinal int) descpb.IndexDescriptor_Direction { - return w.desc.ColumnDirections[columnOrdinal] -} - -// ForEachColumnID applies its argument fn to each of the column IDs in the -// index descriptor. If there is an error, that error is returned immediately. -func (w index) ForEachColumnID(fn func(colID descpb.ColumnID) error) error { - return w.desc.RunOverAllColumns(fn) -} - -// NumStoredColumns returns the number of columns which the index stores in -// addition to the columns which are explicitly part of the index (STORING -// clause). Only used for secondary indexes. -func (w index) NumStoredColumns() int { - return len(w.desc.StoreColumnIDs) -} - -// GetStoredColumnID returns the ID of the storeColumnOrdinal-th store column. -func (w index) GetStoredColumnID(storedColumnOrdinal int) descpb.ColumnID { - return w.desc.StoreColumnIDs[storedColumnOrdinal] -} - -// GetStoredColumnName returns the name of the storeColumnOrdinal-th store column. -func (w index) GetStoredColumnName(storedColumnOrdinal int) string { - return w.desc.StoreColumnNames[storedColumnOrdinal] -} - -// NumExtraColumns returns the number of additional columns referenced by the -// index descriptor. -func (w index) NumExtraColumns() int { - return len(w.desc.ExtraColumnIDs) -} - -// GetExtraColumnID returns the ID of the extraColumnOrdinal-th extra column. -func (w index) GetExtraColumnID(extraColumnOrdinal int) descpb.ColumnID { - return w.desc.ExtraColumnIDs[extraColumnOrdinal] -} - -// NumCompositeColumns returns the number of composite columns referenced by the -// index descriptor. -func (w index) NumCompositeColumns() int { - return len(w.desc.CompositeColumnIDs) -} - -// GetCompositeColumnID returns the ID of the compositeColumnOrdinal-th -// composite column. -func (w index) GetCompositeColumnID(compositeColumnOrdinal int) descpb.ColumnID { - return w.desc.CompositeColumnIDs[compositeColumnOrdinal] -} - -type backingSlicesState int - -const ( - uninitialized backingSlicesState = iota - primary - active - all -) - -// indexCache contains lazily precomputed slices of catalog.Index interfaces. -// The valid initial state is indexCache{}. -type indexCache struct { - // These first two slices grow as needed to derive all other slices below. - // Each interface in backingInterfaces points to the corresponding entry - // in backingStructs, this is done to minimize allocations. - backingStructs []index - backingInterfaces []catalog.Index - // backingSlicesState specifies how far the above slices have grown. - backingSlicesState backingSlicesState - - // The remaining fields store the precomputed slices of interest. - // A value of nil signifies it hasn't been precomputed yet, - // therefore empty precomputations need to be stored as []catalog.Index{}. - all []catalog.Index - active []catalog.Index - nonDrop []catalog.Index - publicNonPrimary []catalog.Index - writableNonPrimary []catalog.Index - deletableNonPrimary []catalog.Index - deleteOnlyNonPrimary []catalog.Index - partial []catalog.Index -} - -// primaryIndex returns the catalog.Index interface for the primary index. -func (c *indexCache) primaryIndex(desc *wrapper) catalog.Index { - c.ensureBackingSlices(desc, primary) - return c.backingInterfaces[0] -} - -// ensureBackingSlices ensures that backingStructs and backingInterfaces are -// populated with enough entries, as specified by requiredState. -func (c *indexCache) ensureBackingSlices(desc *wrapper, requiredState backingSlicesState) { - switch c.backingSlicesState { - case uninitialized: - // We need to transition the state to `primary` and perhaps beyond. - c.backingStructs = make([]index, 0, 1+len(desc.Indexes)+len(desc.Mutations)) - c.backingInterfaces = make([]catalog.Index, 0, cap(c.backingStructs)) - c.addToBackingSlices(index{desc: &desc.PrimaryIndex}) - c.backingSlicesState = primary - if requiredState <= primary { - return - } - fallthrough - case primary: - // We need to transition the state to `active` and perhaps beyond. - for i := range desc.Indexes { - c.addToBackingSlices(index{desc: &desc.Indexes[i]}) - } - c.backingSlicesState = active - if requiredState <= active { - return - } - fallthrough - case active: - // We need to transition the state to `all`. - for _, m := range desc.Mutations { - if idxDesc := m.GetIndex(); idxDesc != nil { - c.addToBackingSlices(index{ - desc: idxDesc, - mutationID: m.MutationID, - mutationState: m.State, - mutationDirection: m.Direction, - }) - } - } - c.backingSlicesState = all - fallthrough - case all: - // We're good, the `all` state is terminal. - return - } -} - -// addToBackingSlices adds a new index to the backing slices and sets its -// ordinal number in order of addition. -// This is a convenience method for ensureBackingSlices. -func (c *indexCache) addToBackingSlices(newIndex index) { - newIndex.ordinal = len(c.backingStructs) - c.backingStructs = append(c.backingStructs, newIndex) - c.backingInterfaces = append(c.backingInterfaces, &c.backingStructs[newIndex.ordinal]) -} - -// cachedIndexes returns an already-build slice of catalog.Index interfaces if -// it exists, if not it builds it using the provided factory function and args, -// after populating the backing slices as much as required. -// Notice that, as a whole, empty slices need to be handled carefully. -func (c *indexCache) cachedIndexes( - cached *[]catalog.Index, - factory func(c *indexCache, desc *wrapper) []catalog.Index, - requiredState backingSlicesState, - desc *wrapper, -) []catalog.Index { - if *cached == nil { - c.ensureBackingSlices(desc, requiredState) - s := factory(c, desc) - *cached = append(make([]catalog.Index, 0, len(s)), s...) - } - if len(*cached) == 0 { - return nil - } - return *cached -} - -// buildPublicNonPrimary builds a fresh return value for -// desc.PublicNonPrimaryIndexes(). -func buildPublicNonPrimary(c *indexCache, desc *wrapper) []catalog.Index { - return c.backingInterfaces[1 : 1+len(desc.Indexes)] -} - -func (c *indexCache) publicNonPrimaryIndexes(desc *wrapper) []catalog.Index { - return c.cachedIndexes(&c.publicNonPrimary, buildPublicNonPrimary, active, desc) -} - -// buildActive builds a fresh return value for desc.ActiveIndexes(). -func buildActive(c *indexCache, desc *wrapper) []catalog.Index { - return c.backingInterfaces[:1+len(desc.Indexes)] -} - -func (c *indexCache) activeIndexes(desc *wrapper) []catalog.Index { - return c.cachedIndexes(&c.active, buildActive, active, desc) -} - -// buildAll builds a fresh return value for desc.AllIndexes(). -func buildAll(c *indexCache, _ *wrapper) []catalog.Index { - return c.backingInterfaces -} - -func (c *indexCache) allIndexes(desc *wrapper) []catalog.Index { - return c.cachedIndexes(&c.all, buildAll, all, desc) -} - -// buildDeletableNonPrimary builds a fresh return value for -// desc.DeletableNonPrimaryIndexes(). -func buildDeletableNonPrimary(c *indexCache, _ *wrapper) []catalog.Index { - return c.backingInterfaces[1:] -} - -func (c *indexCache) deletableNonPrimaryIndexes(desc *wrapper) []catalog.Index { - return c.cachedIndexes(&c.deletableNonPrimary, buildDeletableNonPrimary, all, desc) -} - -// buildWritableNonPrimary builds a fresh return value for -// desc.WritableNonPrimaryIndexes(). -func buildWritableNonPrimary(c *indexCache, _ *wrapper) []catalog.Index { - s := make([]catalog.Index, 0, len(c.backingInterfaces[1:])) - for _, idx := range c.backingInterfaces[1:] { - if idx.Public() || idx.WriteAndDeleteOnly() { - s = append(s, idx) - } - } - return s -} - -func (c *indexCache) writableNonPrimaryIndexes(desc *wrapper) []catalog.Index { - return c.cachedIndexes(&c.writableNonPrimary, buildWritableNonPrimary, all, desc) -} - -// buildDeleteOnlyNonPrimary builds a fresh return value for -// desc.DeleteOnlyNonPrimaryIndexes(). -func buildDeleteOnlyNonPrimary(c *indexCache, desc *wrapper) []catalog.Index { - s := make([]catalog.Index, 0, len(desc.Mutations)) - for _, idx := range c.backingInterfaces[1+len(desc.Indexes):] { - if idx.DeleteOnly() { - s = append(s, idx) - } - } - return s -} - -func (c *indexCache) deleteOnlyNonPrimaryIndexes(desc *wrapper) []catalog.Index { - return c.cachedIndexes(&c.deleteOnlyNonPrimary, buildDeleteOnlyNonPrimary, all, desc) -} - -// buildNonDrop builds a fresh return value for desc.NonDropIndexes(). -func buildNonDrop(c *indexCache, desc *wrapper) []catalog.Index { - s := make([]catalog.Index, 0, len(c.backingInterfaces)) - for _, idx := range c.backingInterfaces { - if !idx.Dropped() && (!idx.Primary() || desc.IsPhysicalTable()) { - s = append(s, idx) - } - } - return s -} - -func (c *indexCache) nonDropIndexes(desc *wrapper) []catalog.Index { - return c.cachedIndexes(&c.nonDrop, buildNonDrop, all, desc) -} - -// buildPartial builds a fresh return value for desc.PartialIndexes(). -func buildPartial(c *indexCache, _ *wrapper) []catalog.Index { - s := make([]catalog.Index, 0, len(c.backingInterfaces[1:])) - for _, idx := range c.backingInterfaces[1:] { - if idx.IsPartial() { - s = append(s, idx) - } - } - return s -} - -func (c *indexCache) partialIndexes(desc *wrapper) []catalog.Index { - return c.cachedIndexes(&c.partial, buildPartial, all, desc) -} diff --git a/pkg/sql/catalog/tabledesc/index_test.go b/pkg/sql/catalog/tabledesc/index_test.go deleted file mode 100644 index 810d5902a49a..000000000000 --- a/pkg/sql/catalog/tabledesc/index_test.go +++ /dev/null @@ -1,319 +0,0 @@ -// 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_test - -import ( - "context" - "fmt" - "testing" - - "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/geo/geoindex" - "github.com/cockroachdb/cockroach/pkg/keys" - "github.com/cockroachdb/cockroach/pkg/sql/catalog" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" - "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" - "github.com/cockroachdb/cockroach/pkg/util/leaktest" - "github.com/stretchr/testify/require" -) - -func TestIndexInterface(t *testing.T) { - defer leaktest.AfterTest(t)() - - // This server is only used to turn a CREATE TABLE statement into a - // catalog.TableDescriptor. - s, conn, db := serverutils.StartServer(t, base.TestServerArgs{}) - defer s.Stopper().Stop(context.Background()) - if _, err := conn.Exec(`CREATE DATABASE d`); err != nil { - t.Fatalf("%+v", err) - } - sqlutils.MakeSQLRunner(conn).Exec(t, ` - SET experimental_enable_hash_sharded_indexes = on; - CREATE TABLE d.t ( - c1 INT, - c2 INT, - c3 INT, - c4 VARCHAR, - c5 VARCHAR, - c6 JSONB, - c7 GEOGRAPHY(GEOMETRY,4326) NULL, - CONSTRAINT pk PRIMARY KEY (c1 ASC, c2 ASC, c3 ASC), - INDEX s1 (c4 DESC, c5 DESC), - INVERTED INDEX s2 (c6), - INDEX s3 (c2, c3) STORING (c5, c6), - INDEX s4 (c5) USING HASH WITH BUCKET_COUNT=8, - UNIQUE INDEX s5 (c1, c4) WHERE c4 = 'x', - INVERTED INDEX s6 (c7) WITH (s2_level_mod=2) - ); - `) - - indexNames := []string{"pk", "s1", "s2", "s3", "s4", "s5", "s6"} - indexColumns := [][]string{ - {"c1", "c2", "c3"}, - {"c4", "c5"}, - {"c6"}, - {"c2", "c3"}, - {"crdb_internal_c5_shard_8", "c5"}, - {"c1", "c4"}, - {"c7"}, - } - extraColumnsAsPkColOrdinals := [][]int{ - {}, - {0, 1, 2}, - {0, 1, 2}, - {0}, - {0, 1, 2}, - {1, 2}, - {0, 1, 2}, - } - - immutable := catalogkv.TestingGetImmutableTableDescriptor(db, keys.SystemSQLCodec, "d", "t") - require.NotNil(t, immutable) - var tableI catalog.TableDescriptor = immutable - require.NotNil(t, tableI) - - // Find indexes by name, check that names are correct and that indexes - // are in the correct order. - indexes := make([]catalog.Index, len(indexNames)) - for i, name := range indexNames { - idx, err := tableI.FindIndexWithName(name) - require.NoError(t, err) - require.Equal(t, name, idx.GetName()) - require.Equal(t, i, idx.Ordinal()) - indexes[i] = idx - } - - pk := indexes[0] - s1 := indexes[1] - s2 := indexes[2] - s3 := indexes[3] - s4 := indexes[4] - s5 := indexes[5] - s6 := indexes[6] - - // Check that GetPrimaryIndex returns the primary index. - require.Equal(t, pk, tableI.GetPrimaryIndex()) - require.Equal(t, pk.GetID(), tableI.GetPrimaryIndexID()) - require.True(t, pk.Primary()) - require.True(t, pk.Public()) - require.Equal(t, descpb.PrimaryIndexEncoding, pk.GetEncodingType()) - - // Check that ActiveIndexes returns the same indexes in the same order. - require.Equal(t, indexes, tableI.ActiveIndexes()) - - // Check that PublicNonPrimaryIndexes returns the same indexes sans primary. - require.Equal(t, indexes[1:], tableI.PublicNonPrimaryIndexes()) - for _, idx := range tableI.PublicNonPrimaryIndexes() { - require.False(t, idx.Primary()) - require.True(t, idx.Public()) - require.Equal(t, descpb.SecondaryIndexEncoding, idx.GetEncodingType()) - } - - // Check that ForEachActiveIndex visits indexes in the same order as well. - { - expectedOrdinal := 0 - err := catalog.ForEachActiveIndex(tableI, func(idx catalog.Index) error { - if idx.Ordinal() != expectedOrdinal { - return fmt.Errorf("expected ordinal %d for index %s, instead got %d", - expectedOrdinal, idx.GetName(), idx.Ordinal()) - } - expectedOrdinal++ - return nil - }) - require.NoError(t, err) - } - - // Check various catalog.ForEach* and catalog.Find* functions. - for _, s := range []struct { - name string - expectedIndexNames []string - sliceFunc func(catalog.TableDescriptor) []catalog.Index - forEachFunc func(catalog.TableDescriptor, func(catalog.Index) error) error - findFunc func(catalog.TableDescriptor, func(catalog.Index) bool) catalog.Index - }{ - { - "ActiveIndex", - indexNames, - catalog.TableDescriptor.ActiveIndexes, - catalog.ForEachActiveIndex, - catalog.FindActiveIndex, - }, - { - "NonDropIndex", - indexNames, - catalog.TableDescriptor.NonDropIndexes, - catalog.ForEachNonDropIndex, - catalog.FindNonDropIndex, - }, - { - "PartialIndex", - []string{"s5"}, - catalog.TableDescriptor.PartialIndexes, - catalog.ForEachPartialIndex, - catalog.FindPartialIndex, - }, - { - "PublicNonPrimaryIndex", - indexNames[1:], - catalog.TableDescriptor.PublicNonPrimaryIndexes, - catalog.ForEachPublicNonPrimaryIndex, - catalog.FindPublicNonPrimaryIndex, - }, - { - "WritableNonPrimaryIndex", - indexNames[1:], - catalog.TableDescriptor.WritableNonPrimaryIndexes, - catalog.ForEachWritableNonPrimaryIndex, - catalog.FindWritableNonPrimaryIndex, - }, - { - "DeletableNonPrimaryIndex", - indexNames[1:], - catalog.TableDescriptor.DeletableNonPrimaryIndexes, - catalog.ForEachDeletableNonPrimaryIndex, - catalog.FindDeletableNonPrimaryIndex, - }, - { - "DeleteOnlyNonPrimaryIndex", - []string{}, - catalog.TableDescriptor.DeleteOnlyNonPrimaryIndexes, - catalog.ForEachDeleteOnlyNonPrimaryIndex, - catalog.FindDeleteOnlyNonPrimaryIndex, - }, - } { - expected := s.sliceFunc(tableI) - var actual []catalog.Index - err := s.forEachFunc(tableI, func(index catalog.Index) error { - actual = append(actual, index) - return nil - }) - require.NoErrorf(t, err, "Unexpected error from ForEach%s.", s.name) - require.Equalf(t, expected, actual, "Unexpected results from ForEach%s.", s.name) - actualNames := make([]string, len(actual)) - for i, idx := range actual { - actualNames[i] = idx.GetName() - } - require.Equalf(t, s.expectedIndexNames, actualNames, "Unexpected results from %ses.", s.name) - for _, expectedIndex := range expected { - foundIndex := s.findFunc(tableI, func(index catalog.Index) bool { - return index == expectedIndex - }) - require.Equalf(t, foundIndex, expectedIndex, "Unexpected results from Find%s.", s.name) - } - } - - // Check that finding indexes by ID is correct. - for _, idx := range indexes { - found, err := tableI.FindIndexWithID(idx.GetID()) - require.NoError(t, err) - require.Equalf(t, idx.GetID(), found.GetID(), - "mismatched IDs for index '%s'", idx.GetName()) - } - - // Check index metadata. - for _, idx := range indexes { - require.False(t, idx.WriteAndDeleteOnly()) - require.False(t, idx.DeleteOnly()) - require.False(t, idx.Adding()) - require.False(t, idx.Dropped()) - } - - errMsgFmt := "Unexpected %s result for index '%s'." - - // Check index methods on features not tested here. - for _, idx := range indexes { - require.False(t, idx.IsInterleaved(), - errMsgFmt, "IsInterleaved", idx.GetName()) - require.False(t, idx.IsDisabled(), - errMsgFmt, "IsDisabled", idx.GetName()) - require.False(t, idx.IsCreatedExplicitly(), - errMsgFmt, "IsCreatedExplicitly", idx.GetName()) - require.Equal(t, descpb.IndexDescriptorVersion(0x2), idx.GetVersion(), - errMsgFmt, "GetVersion", idx.GetName()) - require.Equal(t, descpb.PartitioningDescriptor{}, idx.GetPartitioning(), - errMsgFmt, "GetPartitioning", idx.GetName()) - require.Equal(t, []string(nil), idx.PartitionNames(), - errMsgFmt, "PartitionNames", idx.GetName()) - require.Equal(t, 0, idx.NumInterleaveAncestors(), - errMsgFmt, "NumInterleaveAncestors", idx.GetName()) - require.Equal(t, 0, idx.NumInterleavedBy(), - errMsgFmt, "NumInterleavedBy", idx.GetName()) - require.False(t, idx.HasOldStoredColumns(), - errMsgFmt, "HasOldStoredColumns", idx.GetName()) - require.Equalf(t, 0, idx.NumCompositeColumns(), - errMsgFmt, "NumCompositeColumns", idx.GetName()) - } - - // Check particular index features. - require.Equal(t, "c4 = 'x':::STRING", s5.GetPredicate()) - require.Equal(t, "crdb_internal_c5_shard_8", s4.GetShardColumnName()) - require.Equal(t, int32(2), s6.GetGeoConfig().S2Geography.S2Config.LevelMod) - for _, idx := range indexes { - require.Equalf(t, idx == s5, idx.IsPartial(), - errMsgFmt, "IsPartial", idx.GetName()) - require.Equal(t, idx == s5, idx.GetPredicate() != "", - errMsgFmt, "GetPredicate", idx.GetName()) - require.Equal(t, idx == s5 || idx == pk, idx.IsUnique(), - errMsgFmt, "IsUnique", idx.GetName()) - require.Equal(t, idx == s2 || idx == s6, idx.GetType() == descpb.IndexDescriptor_INVERTED, - errMsgFmt, "GetType", idx.GetName()) - require.Equal(t, idx == s4, idx.IsSharded(), - errMsgFmt, "IsSharded", idx.GetName()) - require.Equal(t, idx == s6, !(&geoindex.Config{}).Equal(idx.GetGeoConfig()), - errMsgFmt, "GetGeoConfig", idx.GetName()) - require.Equal(t, idx == s4, idx.GetShardColumnName() != "", - errMsgFmt, "GetShardColumnName", idx.GetName()) - require.Equal(t, idx == s4, !(&descpb.ShardedDescriptor{}).Equal(idx.GetSharded()), - errMsgFmt, "GetSharded", idx.GetName()) - require.Equalf(t, idx != s3, idx.NumStoredColumns() == 0, - errMsgFmt, "NumStoredColumns", idx.GetName()) - } - - // Check index columns. - for i, idx := range indexes { - expectedColNames := indexColumns[i] - actualColNames := make([]string, idx.NumColumns()) - for j := range actualColNames { - actualColNames[j] = idx.GetColumnName(j) - require.Equalf(t, idx == s1, idx.GetColumnDirection(j) == descpb.IndexDescriptor_DESC, - "mismatched column directions for index '%s'", idx.GetName()) - require.True(t, idx.ContainsColumnID(idx.GetColumnID(j)), - "column ID resolution failure for column '%s' in index '%s'", idx.GetColumnName(j), idx.GetName()) - } - require.Equalf(t, expectedColNames, actualColNames, - "mismatched columns for index '%s'", idx.GetName()) - } - - // Check index extra columns. - for i, idx := range indexes { - expectedExtraColIDs := make([]descpb.ColumnID, len(extraColumnsAsPkColOrdinals[i])) - for j, pkColOrdinal := range extraColumnsAsPkColOrdinals[i] { - expectedExtraColIDs[j] = pk.GetColumnID(pkColOrdinal) - } - actualExtraColIDs := make([]descpb.ColumnID, idx.NumExtraColumns()) - for j := range actualExtraColIDs { - actualExtraColIDs[j] = idx.GetExtraColumnID(j) - } - require.Equalf(t, expectedExtraColIDs, actualExtraColIDs, - "mismatched extra columns for index '%s'", idx.GetName()) - } - - // Check particular index column features. - require.Equal(t, "c6", s2.InvertedColumnName()) - require.Equal(t, s2.GetColumnID(0), s2.InvertedColumnID()) - require.Equal(t, "c7", s6.InvertedColumnName()) - require.Equal(t, s6.GetColumnID(0), s6.InvertedColumnID()) - require.Equal(t, 2, s3.NumStoredColumns()) - require.Equal(t, "c5", s3.GetStoredColumnName(0)) - require.Equal(t, "c6", s3.GetStoredColumnName(1)) -} diff --git a/pkg/sql/catalog/tabledesc/safe_format.go b/pkg/sql/catalog/tabledesc/safe_format.go index db975755e374..ee54864a27a6 100644 --- a/pkg/sql/catalog/tabledesc/safe_format.go +++ b/pkg/sql/catalog/tabledesc/safe_format.go @@ -110,13 +110,11 @@ func formatSafeTableIndexes(w *redact.StringBuilder, desc catalog.TableDescripto w.Printf(", PrimaryIndex: %d", desc.GetPrimaryIndexID()) w.Printf(", NextIndexID: %d", desc.TableDesc().NextIndexID) w.Printf(", Indexes: [") - _ = catalog.ForEachActiveIndex(desc, func(idx catalog.Index) error { - if !idx.Primary() { - w.Printf(", ") - } - formatSafeIndex(w, idx.IndexDesc(), nil) - return nil - }) + formatSafeIndex(w, desc.GetPrimaryIndex(), nil) + for i := range desc.GetPublicNonPrimaryIndexes() { + w.Printf(", ") + formatSafeIndex(w, &desc.GetPublicNonPrimaryIndexes()[i], nil) + } w.Printf("]") } diff --git a/pkg/sql/catalog/tabledesc/structured.go b/pkg/sql/catalog/tabledesc/structured.go index 1ad6d78a04be..0bf4321506dc 100644 --- a/pkg/sql/catalog/tabledesc/structured.go +++ b/pkg/sql/catalog/tabledesc/structured.go @@ -98,27 +98,31 @@ func NewFilledInExistingMutable( skipFKsWithMissingTable bool, tbl *descpb.TableDescriptor, ) (*Mutable, error) { - changes, err := maybeFillInDescriptor(ctx, dg, tbl, skipFKsWithMissingTable) + desc, err := makeFilledInImmutable(ctx, dg, tbl, skipFKsWithMissingTable) if err != nil { return nil, err } - w := wrapper{TableDescriptor: *tbl, postDeserializationChanges: changes} - return &Mutable{wrapper: w, ClusterVersion: *tbl}, nil + m := &Mutable{wrapper: desc.wrapper} + m.ClusterVersion = *tbl + return m, nil } // MakeImmutable returns an Immutable from the given TableDescriptor. func MakeImmutable(tbl descpb.TableDescriptor) Immutable { publicAndNonPublicCols := tbl.Columns + publicAndNonPublicIndexes := tbl.Indexes readableCols := tbl.Columns - desc := Immutable{wrapper: wrapper{TableDescriptor: tbl, indexCache: &indexCache{}}} + desc := Immutable{wrapper: wrapper{TableDescriptor: tbl}} if len(tbl.Mutations) > 0 { publicAndNonPublicCols = make([]descpb.ColumnDescriptor, 0, len(tbl.Columns)+len(tbl.Mutations)) + publicAndNonPublicIndexes = make([]descpb.IndexDescriptor, 0, len(tbl.Indexes)+len(tbl.Mutations)) readableCols = make([]descpb.ColumnDescriptor, 0, len(tbl.Columns)+len(tbl.Mutations)) publicAndNonPublicCols = append(publicAndNonPublicCols, tbl.Columns...) + publicAndNonPublicIndexes = append(publicAndNonPublicIndexes, tbl.Indexes...) readableCols = append(readableCols, tbl.Columns...) // Fill up mutations into the column/index lists by placing the writable columns/indexes @@ -126,7 +130,10 @@ func MakeImmutable(tbl descpb.TableDescriptor) Immutable { for _, m := range tbl.Mutations { switch m.State { case descpb.DescriptorMutation_DELETE_AND_WRITE_ONLY: - if col := m.GetColumn(); col != nil { + if idx := m.GetIndex(); idx != nil { + publicAndNonPublicIndexes = append(publicAndNonPublicIndexes, *idx) + desc.writeOnlyIndexCount++ + } else if col := m.GetColumn(); col != nil { publicAndNonPublicCols = append(publicAndNonPublicCols, *col) desc.writeOnlyColCount++ } @@ -136,7 +143,9 @@ func MakeImmutable(tbl descpb.TableDescriptor) Immutable { for _, m := range tbl.Mutations { switch m.State { case descpb.DescriptorMutation_DELETE_ONLY: - if col := m.GetColumn(); col != nil { + if idx := m.GetIndex(); idx != nil { + publicAndNonPublicIndexes = append(publicAndNonPublicIndexes, *idx) + } else if col := m.GetColumn(); col != nil { publicAndNonPublicCols = append(publicAndNonPublicCols, *col) } } @@ -153,12 +162,20 @@ func MakeImmutable(tbl descpb.TableDescriptor) Immutable { desc.readableColumns = readableCols desc.publicAndNonPublicCols = publicAndNonPublicCols + desc.publicAndNonPublicIndexes = publicAndNonPublicIndexes desc.allChecks = make([]descpb.TableDescriptor_CheckConstraint, len(tbl.Checks)) for i, c := range tbl.Checks { desc.allChecks[i] = *c } + // Track partial index ordinals. + for i := range publicAndNonPublicIndexes { + if publicAndNonPublicIndexes[i].IsPartial() { + desc.partialIndexOrds.Add(i) + } + } + // Remember what columns have user defined types. for i := range desc.publicAndNonPublicCols { typ := desc.publicAndNonPublicCols[i].Type @@ -194,12 +211,10 @@ func NewImmutableWithIsUncommittedVersion( func NewFilledInImmutable( ctx context.Context, dg catalog.DescGetter, tbl *descpb.TableDescriptor, ) (*Immutable, error) { - changes, err := maybeFillInDescriptor(ctx, dg, tbl, false /* skipFKsWithNoMatchingTable */) + desc, err := makeFilledInImmutable(ctx, dg, tbl, false /* skipFKsWithMissingTable */) if err != nil { return nil, err } - desc := MakeImmutable(*tbl) - desc.postDeserializationChanges = changes return &desc, nil } @@ -221,6 +236,21 @@ type PostDeserializationTableDescriptorChanges struct { UpgradedForeignKeyRepresentation bool } +func makeFilledInImmutable( + ctx context.Context, + dg catalog.DescGetter, + tbl *descpb.TableDescriptor, + skipFKsWithMissingTable bool, +) (Immutable, error) { + changes, err := maybeFillInDescriptor(ctx, dg, tbl, skipFKsWithMissingTable) + if err != nil { + return Immutable{}, err + } + desc := MakeImmutable(*tbl) + desc.postDeserializationChanges = changes + return desc, nil +} + // FindIndexPartitionByName searches this index descriptor for a partition whose name // is the input and returns it, or nil if no match is found. func FindIndexPartitionByName( @@ -239,6 +269,12 @@ func (desc *Mutable) SetName(name string) { desc.Name = name } +// GetPrimaryIndex returns a pointer to the primary index of the table +// descriptor. +func (desc *wrapper) GetPrimaryIndex() *descpb.IndexDescriptor { + return &desc.PrimaryIndex +} + // GetParentSchemaID returns the ParentSchemaID if the descriptor has // one. If the descriptor was created before the field was added, then the // descriptor belongs to a table under the `public` physical schema. The static @@ -261,6 +297,32 @@ func (desc *wrapper) IsPhysicalTable() bool { return desc.IsSequence() || (desc.IsTable() && !desc.IsVirtualTable()) || desc.MaterializedView() } +// FindIndexByID finds an index (active or inactive) with the specified ID. +// Must return a pointer to the IndexDescriptor in the TableDescriptor, so that +// callers can use returned values to modify the TableDesc. +func (desc *wrapper) FindIndexByID(id descpb.IndexID) (*descpb.IndexDescriptor, error) { + if desc.PrimaryIndex.ID == id { + return &desc.PrimaryIndex, nil + } + for i := range desc.Indexes { + idx := &desc.Indexes[i] + if idx.ID == id { + return idx, nil + } + } + for _, m := range desc.Mutations { + if idx := m.GetIndex(); idx != nil && idx.ID == id { + return idx, nil + } + } + for _, m := range desc.GCMutations { + if m.IndexID == id { + return nil, ErrIndexGCMutationsList + } + } + return nil, fmt.Errorf("index-id \"%d\" does not exist", id) +} + // KeysPerRow returns the maximum number of keys used to encode a row for the // given index. If a secondary index doesn't store any columns, then it only // has one k/v pair, but if it stores some columns, it can return up to one @@ -269,11 +331,11 @@ func (desc *wrapper) KeysPerRow(indexID descpb.IndexID) (int, error) { if desc.PrimaryIndex.ID == indexID { return len(desc.Families), nil } - idx, err := desc.FindIndexWithID(indexID) + idx, err := desc.FindIndexByID(indexID) if err != nil { return 0, err } - if idx.NumStoredColumns() == 0 { + if len(idx.StoreColumnIDs) == 0 { return 1, nil } return len(desc.Families), nil @@ -294,11 +356,10 @@ func (desc *wrapper) AllNonDropColumns() []descpb.ColumnDescriptor { return cols } -// buildIndexName sets desc.Name to a value that is not EqualName to any +// allocateName sets desc.Name to a value that is not EqualName to any // of tableDesc's indexes. allocateName roughly follows PostgreSQL's // convention for automatically-named indexes. -func buildIndexName(tableDesc *Mutable, index catalog.Index) string { - idx := index.IndexDesc() +func allocateIndexName(tableDesc *Mutable, idx *descpb.IndexDescriptor) { segments := make([]string, 0, len(idx.ColumnNames)+2) segments = append(segments, tableDesc.Name) segments = append(segments, idx.ColumnNames...) @@ -310,15 +371,36 @@ func buildIndexName(tableDesc *Mutable, index catalog.Index) string { baseName := strings.Join(segments, "_") name := baseName - for i := 1; ; i++ { - foundIndex, _ := tableDesc.FindIndexWithName(name) - if foundIndex == nil { - break - } + + exists := func(name string) bool { + _, _, err := tableDesc.FindIndexByName(name) + return err == nil + } + for i := 1; exists(name); i++ { name = fmt.Sprintf("%s%d", baseName, i) } - return name + idx.Name = name +} + +// AllNonDropIndexes returns all the indexes, including those being added +// in the mutations. +func (desc *wrapper) AllNonDropIndexes() []*descpb.IndexDescriptor { + indexes := make([]*descpb.IndexDescriptor, 0, 1+len(desc.Indexes)+len(desc.Mutations)) + if desc.IsPhysicalTable() { + indexes = append(indexes, &desc.PrimaryIndex) + } + for i := range desc.Indexes { + indexes = append(indexes, &desc.Indexes[i]) + } + for _, m := range desc.Mutations { + if idx := m.GetIndex(); idx != nil { + if m.Direction == descpb.DescriptorMutation_ADD { + indexes = append(indexes, idx) + } + } + } + return indexes } // AllActiveAndInactiveChecks returns all check constraints, including both @@ -445,6 +527,48 @@ func (desc *wrapper) ForeachNonDropColumn(f func(column *descpb.ColumnDescriptor return nil } +// ForeachNonDropIndex runs a function on all indexes, including those being +// added in the mutations. +func (desc *wrapper) ForeachNonDropIndex(f func(*descpb.IndexDescriptor) error) error { + if err := desc.ForeachIndex(catalog.IndexOpts{AddMutations: true}, func(idxDesc *descpb.IndexDescriptor, isPrimary bool) error { + return f(idxDesc) + }); err != nil { + return err + } + return nil +} + +// ForeachIndex runs a function on the set of indexes as specified by opts. +func (desc *wrapper) ForeachIndex( + opts catalog.IndexOpts, f func(idxDesc *descpb.IndexDescriptor, isPrimary bool) error, +) error { + if desc.IsPhysicalTable() || opts.NonPhysicalPrimaryIndex { + if err := f(&desc.PrimaryIndex, true /* isPrimary */); err != nil { + return err + } + } + for i := range desc.Indexes { + if err := f(&desc.Indexes[i], false /* isPrimary */); err != nil { + return err + } + } + if !opts.AddMutations && !opts.DropMutations { + return nil + } + for _, m := range desc.Mutations { + idx := m.GetIndex() + if idx == nil || + (m.Direction == descpb.DescriptorMutation_ADD && !opts.AddMutations) || + (m.Direction == descpb.DescriptorMutation_DROP && !opts.DropMutations) { + continue + } + if err := f(idx, false /* isPrimary */); err != nil { + return err + } + } + return nil +} + // ForeachDependedOnBy runs a function on all indexes, including those being // added in the mutations. func (desc *wrapper) ForeachDependedOnBy( @@ -630,7 +754,7 @@ func maybeUpgradeForeignKeyRepOnIndex( } } if tbl, ok := otherUnupgradedTables[ref.Table]; ok { - referencedIndex, err := tbl.FindIndexWithID(ref.Index) + referencedIndex, err := tbl.FindIndexByID(ref.Index) if err != nil { return false, err } @@ -639,7 +763,7 @@ func maybeUpgradeForeignKeyRepOnIndex( OriginTableID: desc.ID, OriginColumnIDs: idx.ColumnIDs[:numCols], ReferencedTableID: ref.Table, - ReferencedColumnIDs: referencedIndex.IndexDesc().ColumnIDs[:numCols], + ReferencedColumnIDs: referencedIndex.ColumnIDs[:numCols], Name: ref.Name, Validity: ref.Validity, OnDelete: ref.OnDelete, @@ -668,11 +792,10 @@ func maybeUpgradeForeignKeyRepOnIndex( } if otherTable, ok := otherUnupgradedTables[ref.Table]; ok { - originIndexI, err := otherTable.FindIndexWithID(ref.Index) + originIndex, err := otherTable.FindIndexByID(ref.Index) if err != nil { return false, err } - originIndex := originIndexI.IndexDesc() // There are two cases. Either the other table is old (not upgraded yet), // or it's new (already upgraded). var inFK descpb.ForeignKeyConstraint @@ -829,9 +952,9 @@ func ForEachExprStringInTableDesc(descI catalog.TableDescriptor, f func(expr *st } return nil } - doIndex := func(i catalog.Index) error { + doIndex := func(i *descpb.IndexDescriptor) error { if i.IsPartial() { - return f(&i.IndexDesc().Predicate) + return f(&i.Predicate) } return nil } @@ -846,14 +969,15 @@ func ForEachExprStringInTableDesc(descI catalog.TableDescriptor, f func(expr *st } } - // Process all indexes. - if err := catalog.ForEachIndex(descI, catalog.IndexOpts{ - NonPhysicalPrimaryIndex: true, - DropMutations: true, - AddMutations: true, - }, doIndex); err != nil { + // Process indexes. + if err := doIndex(&desc.PrimaryIndex); err != nil { return err } + for i := range desc.Indexes { + if err := doIndex(&desc.Indexes[i]); err != nil { + return err + } + } // Process checks. for i := range desc.Checks { @@ -862,13 +986,18 @@ func ForEachExprStringInTableDesc(descI catalog.TableDescriptor, f func(expr *st } } - // Process all non-index mutations. + // Process all mutations. for _, mut := range desc.GetMutations() { if c := mut.GetColumn(); c != nil { if err := doCol(c); err != nil { return err } } + if i := mut.GetIndex(); i != nil { + if err := doIndex(i); err != nil { + return err + } + } if c := mut.GetConstraint(); c != nil && c.ConstraintType == descpb.ConstraintToUpdate_CHECK { if err := doCheck(&c.Check); err != nil { @@ -1047,13 +1176,30 @@ func (desc *Mutable) allocateIndexIDs(columnNames map[string]descpb.ColumnID) er desc.NextIndexID = 1 } - // Assign names to unnamed indexes. - _ = catalog.ForEachDeletableNonPrimaryIndex(desc, func(idx catalog.Index) error { - if len(idx.GetName()) == 0 { - idx.IndexDesc().Name = buildIndexName(desc, idx) + // Keep track of unnamed indexes. + anonymousIndexes := make([]*descpb.IndexDescriptor, 0, len(desc.Indexes)+len(desc.Mutations)) + + // Create a slice of modifiable index descriptors. + indexes := make([]*descpb.IndexDescriptor, 0, 1+len(desc.Indexes)+len(desc.Mutations)) + indexes = append(indexes, &desc.PrimaryIndex) + collectIndexes := func(index *descpb.IndexDescriptor) { + if len(index.Name) == 0 { + anonymousIndexes = append(anonymousIndexes, index) } - return nil - }) + indexes = append(indexes, index) + } + for i := range desc.Indexes { + collectIndexes(&desc.Indexes[i]) + } + for _, m := range desc.Mutations { + if index := m.GetIndex(); index != nil { + collectIndexes(index) + } + } + + for _, index := range anonymousIndexes { + allocateIndexName(desc, index) + } var compositeColIDs catalog.TableColSet for i := range desc.Columns { @@ -1064,12 +1210,11 @@ func (desc *Mutable) allocateIndexIDs(columnNames map[string]descpb.ColumnID) er } // Populate IDs. - for _, idx := range desc.AllIndexes() { - if idx.GetID() != 0 { + for _, index := range indexes { + if index.ID != 0 { // This index has already been populated. Nothing to do. continue } - index := idx.IndexDesc() index.ID = desc.NextIndexID desc.NextIndexID++ @@ -1082,7 +1227,7 @@ func (desc *Mutable) allocateIndexIDs(columnNames map[string]descpb.ColumnID) er } } - if !idx.Primary() && index.EncodingType == descpb.SecondaryIndexEncoding { + if index != &desc.PrimaryIndex && index.EncodingType == descpb.SecondaryIndexEncoding { indexHasOldStoredColumns := index.HasOldStoredColumns() // Need to clear ExtraColumnIDs and StoreColumnIDs because they are used // by ContainsColumnID. @@ -1405,12 +1550,12 @@ func (desc *wrapper) validateCrossReferences(ctx context.Context, dg catalog.Des return nil, nil, errors.Wrapf(err, "missing table=%d index=%d", errors.Safe(tableID), errors.Safe(indexID)) } - targetIndex, err := targetTable.FindIndexWithID(indexID) + targetIndex, err := targetTable.FindIndexByID(indexID) if err != nil { return nil, nil, errors.Wrapf(err, "missing table=%s index=%d", targetTable.GetName(), errors.Safe(indexID)) } - return targetTable, targetIndex.IndexDesc(), nil + return targetTable, targetIndex, nil } // Check foreign keys. @@ -1437,13 +1582,15 @@ func (desc *wrapper) validateCrossReferences(ctx context.Context, dg catalog.Des // un-upgraded foreign key references on the other table. This logic // somewhat parallels the logic in maybeUpgradeForeignKeyRepOnIndex. unupgradedFKsPresent := false - if err := catalog.ForEachIndex(referencedTable, catalog.IndexOpts{}, func(referencedIdx catalog.Index) error { + if err := referencedTable.ForeachIndex(catalog.IndexOpts{}, func( + referencedIdx *descpb.IndexDescriptor, isPrimary bool, + ) error { if found { // TODO (lucy): If we ever revisit the tabledesc.Immutable methods, add // a way to break out of the index loop. return nil } - if len(referencedIdx.IndexDesc().ReferencedBy) > 0 { + if len(referencedIdx.ReferencedBy) > 0 { unupgradedFKsPresent = true } else { return nil @@ -1455,8 +1602,8 @@ func (desc *wrapper) validateCrossReferences(ctx context.Context, dg catalog.Des } // Now check the backreferences. Backreferences in ReferencedBy only had // Index and Table populated. - for i := range referencedIdx.IndexDesc().ReferencedBy { - backref := &referencedIdx.IndexDesc().ReferencedBy[i] + for i := range referencedIdx.ReferencedBy { + backref := &referencedIdx.ReferencedBy[i] if backref.Table != desc.ID { continue } @@ -1465,7 +1612,7 @@ func (desc *wrapper) validateCrossReferences(ctx context.Context, dg catalog.Des // that it shouldn't be possible for this index to not exist. See // planner.MaybeUpgradeDependentOldForeignKeyVersionTables, which is // called from the drop index implementation.) - originalOriginIndex, err := desc.FindIndexWithID(backref.Index) + originalOriginIndex, err := desc.FindIndexByID(backref.Index) if err != nil { return errors.AssertionFailedf( "missing index %d on %q from pre-19.2 foreign key "+ @@ -1516,13 +1663,15 @@ func (desc *wrapper) validateCrossReferences(ctx context.Context, dg catalog.Des // un-upgraded foreign key references on the other table. This logic // somewhat parallels the logic in maybeUpgradeForeignKeyRepOnIndex. unupgradedFKsPresent := false - if err := catalog.ForEachIndex(originTable, catalog.IndexOpts{}, func(originIdx catalog.Index) error { + if err := originTable.ForeachIndex(catalog.IndexOpts{}, func( + originIdx *descpb.IndexDescriptor, isPrimary bool, + ) error { if found { // TODO (lucy): If we ever revisit the tabledesc.Immutable methods, add // a way to break out of the index loop. return nil } - fk := originIdx.IndexDesc().ForeignKey + fk := originIdx.ForeignKey if fk.IsSet() { unupgradedFKsPresent = true } else { @@ -1542,7 +1691,7 @@ func (desc *wrapper) validateCrossReferences(ctx context.Context, dg catalog.Des // that it shouldn't be possible for this index to not exist. See // planner.MaybeUpgradeDependentOldForeignKeyVersionTables, which is // called from the drop index implementation.) - originalReferencedIndex, err := desc.FindIndexWithID(fk.Index) + originalReferencedIndex, err := desc.FindIndexByID(fk.Index) if err != nil { return errors.AssertionFailedf( "missing index %d on %q from pre-19.2 foreign key forward reference %q on %q", @@ -1568,8 +1717,7 @@ func (desc *wrapper) validateCrossReferences(ctx context.Context, dg catalog.Des backref.Name, desc.Name, originTable.GetName()) } - for _, indexI := range desc.NonDropIndexes() { - index := indexI.IndexDesc() + for _, index := range desc.AllNonDropIndexes() { // Check interleaves. if len(index.Interleave.Ancestors) > 0 { // Only check the most recent ancestor, the rest of them don't point @@ -1604,23 +1752,23 @@ func (desc *wrapper) validateCrossReferences(ctx context.Context, dg catalog.Des "invalid interleave backreference table=%d index=%d", backref.Table, backref.Index) } - targetIndex, err := targetTable.FindIndexWithID(backref.Index) + targetIndex, err := targetTable.FindIndexByID(backref.Index) if err != nil { return errors.Wrapf(err, "invalid interleave backreference table=%s index=%d", targetTable.GetName(), backref.Index) } - if targetIndex.NumInterleaveAncestors() == 0 { + if len(targetIndex.Interleave.Ancestors) == 0 { return errors.AssertionFailedf( "broken interleave backward reference from %q@%q to %q@%q", - desc.Name, index.Name, targetTable.GetName(), targetIndex.GetName()) + desc.Name, index.Name, targetTable.GetName(), targetIndex.Name) } // The last ancestor is required to be a backreference. - ancestor := targetIndex.GetInterleaveAncestor(targetIndex.NumInterleaveAncestors() - 1) + ancestor := targetIndex.Interleave.Ancestors[len(targetIndex.Interleave.Ancestors)-1] if ancestor.TableID != desc.ID || ancestor.IndexID != index.ID { return errors.AssertionFailedf( "broken interleave backward reference from %q@%q to %q@%q", - desc.Name, index.Name, targetTable.GetName(), targetIndex.GetName()) + desc.Name, index.Name, targetTable.GetName(), targetIndex.Name) } } } @@ -1740,10 +1888,10 @@ func ValidateTableLocalityConfig( // ValidateIndexNameIsUnique validates that the index name does not exist. func (desc *wrapper) ValidateIndexNameIsUnique(indexName string) error { - if catalog.FindNonDropIndex(desc, func(idx catalog.Index) bool { - return idx.GetName() == indexName - }) != nil { - return sqlerrors.NewRelationAlreadyExistsError(indexName) + for _, index := range desc.AllNonDropIndexes() { + if indexName == index.Name { + return sqlerrors.NewRelationAlreadyExistsError(indexName) + } } return nil } @@ -2178,8 +2326,7 @@ func (desc *wrapper) validateTableIndexes(columnNames map[string]descpb.ColumnID indexNames := map[string]struct{}{} indexIDs := map[descpb.IndexID]string{} - for _, indexI := range desc.NonDropIndexes() { - index := indexI.IndexDesc() + for _, index := range desc.AllNonDropIndexes() { if err := catalog.ValidateName(index.Name, "index"); err != nil { return err } @@ -2488,14 +2635,25 @@ func (ps partitionInterval) Range() interval.Range { return interval.Range{Start: []byte(ps.start), End: []byte(ps.end)} } +// FindIndexesWithPartition returns all IndexDescriptors (potentially including +// the primary index) which have a partition with the given name. +func (desc *wrapper) FindIndexesWithPartition(name string) []*descpb.IndexDescriptor { + var indexes []*descpb.IndexDescriptor + for _, idx := range desc.AllNonDropIndexes() { + if FindIndexPartitionByName(idx, name) != nil { + indexes = append(indexes, idx) + } + } + return indexes +} + // validatePartitioning validates that any PartitioningDescriptors contained in // table indexes are well-formed. See validatePartitioningDesc for details. func (desc *wrapper) validatePartitioning() error { partitionNames := make(map[string]string) a := &rowenc.DatumAlloc{} - return catalog.ForEachNonDropIndex(desc, func(idx catalog.Index) error { - idxDesc := idx.IndexDesc() + return desc.ForeachNonDropIndex(func(idxDesc *descpb.IndexDescriptor) error { return desc.validatePartitioningDescriptor( a, idxDesc, &idxDesc.Partitioning, 0 /* colOffset */, partitionNames, ) @@ -2684,19 +2842,27 @@ func (desc *Mutable) RenameColumnDescriptor(column *descpb.ColumnDescriptor, new } } - for _, idx := range desc.AllIndexes() { - idxDesc := idx.IndexDesc() - for i, id := range idxDesc.ColumnIDs { + renameColumnInIndex := func(idx *descpb.IndexDescriptor) { + for i, id := range idx.ColumnIDs { if id == colID { - idxDesc.ColumnNames[i] = newColName + idx.ColumnNames[i] = newColName } } - for i, id := range idxDesc.StoreColumnIDs { + for i, id := range idx.StoreColumnIDs { if id == colID { - idxDesc.StoreColumnNames[i] = newColName + idx.StoreColumnNames[i] = newColName } } } + renameColumnInIndex(&desc.PrimaryIndex) + for i := range desc.Indexes { + renameColumnInIndex(&desc.Indexes[i]) + } + for _, m := range desc.Mutations { + if idx := m.GetIndex(); idx != nil { + renameColumnInIndex(idx) + } + } } // FindActiveColumnsByNames finds all requested columns (in the requested order) @@ -2855,14 +3021,6 @@ func (desc *wrapper) ContainsUserDefinedTypes() bool { return len(desc.GetColumnOrdinalsWithUserDefinedTypes()) > 0 } -// ContainsUserDefinedTypes returns whether or not this table descriptor has -// any columns of user defined types. -// This method is re-implemented for Immutable only for the purpose of calling -// the correct GetColumnOrdinalsWithUserDefinedTypes() method on desc. -func (desc *Immutable) ContainsUserDefinedTypes() bool { - return len(desc.GetColumnOrdinalsWithUserDefinedTypes()) > 0 -} - // GetColumnOrdinalsWithUserDefinedTypes returns a slice of column ordinals // of columns that contain user defined types. func (desc *Immutable) GetColumnOrdinalsWithUserDefinedTypes() []int { @@ -2885,25 +3043,6 @@ func (desc *wrapper) UserDefinedTypeColsHaveSameVersion(otherDesc catalog.TableD return true } -// UserDefinedTypeColsHaveSameVersion returns whether this 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. -// This method is re-implemented for Immutable only for the purpose of calling -// the correct DeletableColumns() and GetColumnOrdinalsWithUserDefinedTypes() -// methods on desc. -func (desc *Immutable) UserDefinedTypeColsHaveSameVersion(otherDesc catalog.TableDescriptor) bool { - thisCols := desc.DeletableColumns() - otherCols := otherDesc.DeletableColumns() - for _, idx := range desc.GetColumnOrdinalsWithUserDefinedTypes() { - this, other := thisCols[idx].Type, otherCols[idx].Type - if this.TypeMeta.Version != other.TypeMeta.Version { - return false - } - } - return true -} - // FindReadableColumnByID finds the readable column with specified ID. The // column may be undergoing a schema change and is marked nullable regardless // of its configuration. It returns true if the column is undergoing a @@ -2930,6 +3069,30 @@ func (desc *wrapper) FindFamilyByID(id descpb.FamilyID) (*descpb.ColumnFamilyDes return nil, fmt.Errorf("family-id \"%d\" does not exist", id) } +// FindIndexByName finds the index with the specified name in the active +// list or the mutations list. It returns true if the index is being dropped. +func (desc *wrapper) FindIndexByName( + name string, +) (_ *descpb.IndexDescriptor, dropped bool, _ error) { + if desc.IsPhysicalTable() && desc.PrimaryIndex.Name == name { + return &desc.PrimaryIndex, false, nil + } + for i := range desc.Indexes { + idx := &desc.Indexes[i] + if idx.Name == name { + return idx, false, nil + } + } + for _, m := range desc.Mutations { + if idx := m.GetIndex(); idx != nil { + if idx.Name == name { + return idx, m.Direction == descpb.DescriptorMutation_DROP, nil + } + } + } + return nil, false, fmt.Errorf("index %q does not exist", name) +} + // NamesForColumnIDs returns the names for the given column ids, or an error // if one or more column ids was missing. Note - this allocates! It's not for // hot path code. @@ -3176,6 +3339,21 @@ func (desc *Mutable) RenameConstraint( } } +// FindActiveIndexByID returns the index with the specified ID, or nil if it +// does not exist. It only searches active indexes. +func (desc *wrapper) FindActiveIndexByID(id descpb.IndexID) *descpb.IndexDescriptor { + if desc.PrimaryIndex.ID == id { + return &desc.PrimaryIndex + } + for i := range desc.Indexes { + idx := &desc.Indexes[i] + if idx.ID == id { + return idx + } + } + return nil +} + // FindIndexByIndexIdx returns an active index with the specified // index's index which has a domain of [0, # of secondary indexes] and whether // the index is a secondary index. @@ -3228,9 +3406,12 @@ func (desc *wrapper) FindFKByName(name string) (*descpb.ForeignKeyConstraint, er // IsInterleaved returns true if any part of this this table is interleaved with // another table's data. func (desc *wrapper) IsInterleaved() bool { - return nil != catalog.FindNonDropIndex(desc, func(idx catalog.Index) bool { - return idx.IsInterleaved() - }) + for _, index := range desc.AllNonDropIndexes() { + if index.IsInterleaved() { + return true + } + } + return false } // IsPrimaryIndexDefaultRowID returns whether or not the table's primary @@ -3358,25 +3539,25 @@ func (desc *Mutable) MakeMutationComplete(m descpb.DescriptorMutation) error { // Promote the new primary index into the primary index position on the descriptor, // and remove it from the secondary indexes list. - newIndex, err := desc.FindIndexWithID(args.NewPrimaryIndexId) + newIndex, err := desc.FindIndexByID(args.NewPrimaryIndexId) if err != nil { return err } + if args.NewPrimaryIndexName == "" { + newIndex.Name = PrimaryKeyIndexName + } else { + newIndex.Name = args.NewPrimaryIndexName + } { - primaryIndex := newIndex.IndexDescDeepCopy() - if args.NewPrimaryIndexName == "" { - primaryIndex.Name = PrimaryKeyIndexName - } else { - primaryIndex.Name = args.NewPrimaryIndexName - } + primaryIndex := *protoutil.Clone(newIndex).(*descpb.IndexDescriptor) // The primary index "implicitly" stores all columns in the table. // Explicitly including them in the stored columns list is incorrect. primaryIndex.StoreColumnNames, primaryIndex.StoreColumnIDs = nil, nil desc.SetPrimaryIndex(primaryIndex) } - idx, err := getIndexIdxByID(newIndex.GetID()) + idx, err := getIndexIdxByID(newIndex.ID) if err != nil { return err } @@ -3388,7 +3569,7 @@ func (desc *Mutable) MakeMutationComplete(m descpb.DescriptorMutation) error { newID := args.NewIndexes[j] // All our new indexes have been inserted into the table descriptor by now, since the primary key swap // is the last mutation processed in a group of mutations under the same mutation ID. - newIndex, err := desc.FindIndexWithID(newID) + newIndex, err := desc.FindIndexByID(newID) if err != nil { return err } @@ -3401,7 +3582,7 @@ func (desc *Mutable) MakeMutationComplete(m descpb.DescriptorMutation) error { return err } oldIndexCopy := protoutil.Clone(oldIndex).(*descpb.IndexDescriptor) - newIndex.IndexDesc().Name = oldIndexCopy.Name + newIndex.Name = oldIndexCopy.Name // Splice out old index from the indexes list. desc.RemovePublicNonPrimaryIndex(oldIndexIdx) // Add a drop mutation for the old index. The code that calls this function will schedule @@ -3874,8 +4055,12 @@ func (desc *wrapper) InvalidateFKConstraints() { // being added in the mutations. func (desc *wrapper) AllIndexSpans(codec keys.SQLCodec) roachpb.Spans { var spans roachpb.Spans - for _, index := range desc.NonDropIndexes() { - spans = append(spans, desc.IndexSpan(codec, index.GetID())) + err := desc.ForeachNonDropIndex(func(index *descpb.IndexDescriptor) error { + spans = append(spans, desc.IndexSpan(codec, index.ID)) + return nil + }) + if err != nil { + panic(err) } return spans } @@ -3989,8 +4174,8 @@ func (desc *wrapper) GetFamilyOfColumn( // subpartition in an arbitrary order. func (desc *wrapper) PartitionNames() []string { var names []string - for _, index := range desc.NonDropIndexes() { - names = append(names, index.PartitionNames()...) + for _, index := range desc.AllNonDropIndexes() { + names = append(names, index.Partitioning.PartitionNames()...) } return names } @@ -4021,13 +4206,16 @@ func (desc *wrapper) FindAllReferences() (map[descpb.ID]struct{}, error) { fk := &desc.InboundFKs[i] refs[fk.OriginTableID] = struct{}{} } - for _, index := range desc.NonDropIndexes() { - for i := 0; i < index.NumInterleaveAncestors(); i++ { - refs[index.GetInterleaveAncestor(i).TableID] = struct{}{} + if err := desc.ForeachNonDropIndex(func(index *descpb.IndexDescriptor) error { + for _, a := range index.Interleave.Ancestors { + refs[a.TableID] = struct{}{} } - for i := 0; i < index.NumInterleavedBy(); i++ { - refs[index.GetInterleavedBy(i).Table] = struct{}{} + for _, c := range index.InterleavedBy { + refs[c.Table] = struct{}{} } + return nil + }); err != nil { + return nil, err } for _, c := range desc.AllNonDropColumns() { @@ -4068,12 +4256,36 @@ func (desc *Immutable) MutationColumns() []descpb.ColumnDescriptor { return desc.publicAndNonPublicCols[len(desc.Columns):] } +// WritableIndexes returns a list of public and write-only mutation indexes. +func (desc *Immutable) WritableIndexes() []descpb.IndexDescriptor { + return desc.publicAndNonPublicIndexes[:len(desc.Indexes)+desc.writeOnlyIndexCount] +} + +// DeletableIndexes returns a list of public and non-public indexes. +func (desc *Immutable) DeletableIndexes() []descpb.IndexDescriptor { + return desc.publicAndNonPublicIndexes +} + +// DeleteOnlyIndexes returns a list of delete-only mutation indexes. +func (desc *Immutable) DeleteOnlyIndexes() []descpb.IndexDescriptor { + return desc.publicAndNonPublicIndexes[len(desc.Indexes)+desc.writeOnlyIndexCount:] +} + +// PartialIndexOrds returns a set containing the ordinal of each partial index +// defined on the table. +func (desc *Immutable) PartialIndexOrds() util.FastIntSet { + return desc.partialIndexOrds +} + // IsShardColumn returns true if col corresponds to a non-dropped hash sharded // index. This method assumes that col is currently a member of desc. func (desc *Mutable) IsShardColumn(col *descpb.ColumnDescriptor) bool { - return nil != catalog.FindNonDropIndex(desc, func(idx catalog.Index) bool { - return idx.IsSharded() && idx.GetShardColumnName() == col.Name - }) + for _, idx := range desc.AllNonDropIndexes() { + if idx.Sharded.IsSharded && idx.Sharded.Name == col.Name { + return true + } + } + return false } // TableDesc implements the TableDescriptor interface. diff --git a/pkg/sql/catalog/tabledesc/table.go b/pkg/sql/catalog/tabledesc/table.go index 2deaa3bdf681..ee4498e38e1a 100644 --- a/pkg/sql/catalog/tabledesc/table.go +++ b/pkg/sql/catalog/tabledesc/table.go @@ -211,8 +211,8 @@ func (desc *wrapper) collectConstraintInfo( info := make(map[string]descpb.ConstraintDetail) // Indexes provide PK and Unique constraints that are enforced by an index. - for _, indexI := range desc.NonDropIndexes() { - index := indexI.IndexDesc() + indexes := desc.AllNonDropIndexes() + for _, index := range indexes { if index.ID == desc.PrimaryIndex.ID { if _, ok := info[index.Name]; ok { return nil, pgerror.Newf(pgcode.DuplicateObject, @@ -341,12 +341,14 @@ func FindFKReferencedIndex( // key columns. primaryIndex := referencedTable.GetPrimaryIndex() if primaryIndex.IsValidReferencedIndex(referencedColIDs) { - return primaryIndex.IndexDesc(), nil + return primaryIndex, nil } // If the PK doesn't match, find the index corresponding to the referenced column. - for _, idx := range referencedTable.PublicNonPrimaryIndexes() { + indexes := referencedTable.GetPublicNonPrimaryIndexes() + for i := range indexes { + idx := &indexes[i] if idx.IsValidReferencedIndex(referencedColIDs) { - return idx.IndexDesc(), nil + return idx, nil } } return nil, pgerror.Newf( @@ -364,12 +366,14 @@ func FindFKOriginIndex( // Search for an index on the origin table that matches our foreign // key columns. if primaryIndex := originTable.GetPrimaryIndex(); primaryIndex.IsValidOriginIndex(originColIDs) { - return primaryIndex.IndexDesc(), nil + return primaryIndex, nil } // If the PK doesn't match, find the index corresponding to the origin column. - for _, idx := range originTable.PublicNonPrimaryIndexes() { + indexes := originTable.GetPublicNonPrimaryIndexes() + for i := range indexes { + idx := &indexes[i] if idx.IsValidOriginIndex(originColIDs) { - return idx.IndexDesc(), nil + return idx, nil } } return nil, pgerror.Newf( diff --git a/pkg/sql/catalog/tabledesc/table_desc.go b/pkg/sql/catalog/tabledesc/table_desc.go index fa83f8907817..24a774723813 100644 --- a/pkg/sql/catalog/tabledesc/table_desc.go +++ b/pkg/sql/catalog/tabledesc/table_desc.go @@ -14,8 +14,8 @@ package tabledesc import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/protoutil" - "github.com/cockroachdb/errors" ) var _ catalog.TableDescriptor = (*Immutable)(nil) @@ -27,12 +27,6 @@ var _ catalog.TableDescriptor = (*wrapper)(nil) // interface, which is overloaded by Immutable and Mutable. type wrapper struct { descpb.TableDescriptor - - // indexCache, when not nil, points to a struct containing precomputed - // catalog.Index slices. This can therefore only be set when creating an - // Immutable. - indexCache *indexCache - postDeserializationChanges PostDeserializationTableDescriptorChanges } @@ -50,8 +44,69 @@ func (desc *wrapper) GetPostDeserializationChanges() PostDeserializationTableDes return desc.postDeserializationChanges } -// mutationColumns returns all non-public writable columns in the specified -// state. +// PartialIndexOrds returns a set containing the ordinal of each partial index +// defined on the table. +func (desc *wrapper) PartialIndexOrds() util.FastIntSet { + var partialIndexOrds util.FastIntSet + for i, idx := range desc.DeletableIndexes() { + if idx.IsPartial() { + partialIndexOrds.Add(i) + } + } + return partialIndexOrds +} + +// mutationIndexes returns all non-public indexes in the specified state. +func (desc *wrapper) mutationIndexes( + mutationState descpb.DescriptorMutation_State, +) []descpb.IndexDescriptor { + if len(desc.Mutations) == 0 { + return nil + } + indexes := make([]descpb.IndexDescriptor, 0, len(desc.Mutations)) + for _, m := range desc.Mutations { + if m.State != mutationState { + continue + } + if idx := m.GetIndex(); idx != nil { + indexes = append(indexes, *idx) + } + } + return indexes +} + +// DeleteOnlyIndexes returns a list of delete-only mutation indexes. +func (desc *wrapper) DeleteOnlyIndexes() []descpb.IndexDescriptor { + return desc.mutationIndexes(descpb.DescriptorMutation_DELETE_ONLY) +} + +// WritableIndexes returns a list of public and write-only mutation indexes. +func (desc *wrapper) WritableIndexes() []descpb.IndexDescriptor { + if len(desc.Mutations) == 0 { + return desc.Indexes + } + indexes := make([]descpb.IndexDescriptor, 0, len(desc.Indexes)+len(desc.Mutations)) + // Add all public indexes. + indexes = append(indexes, desc.Indexes...) + // Add all non-public writable indexes. + indexes = append(indexes, desc.mutationIndexes(descpb.DescriptorMutation_DELETE_AND_WRITE_ONLY)...) + return indexes +} + +// DeletableIndexes implements the catalog.Descriptor interface. +func (desc *wrapper) DeletableIndexes() []descpb.IndexDescriptor { + if len(desc.Mutations) == 0 { + return desc.Indexes + } + indexes := make([]descpb.IndexDescriptor, 0, len(desc.Indexes)+len(desc.Mutations)) + // Add all writable indexes. + indexes = append(indexes, desc.WritableIndexes()...) + // Add all delete-only indexes. + indexes = append(indexes, desc.DeleteOnlyIndexes()...) + return indexes +} + +// mutationColumns returns all non-public writable columns in the specified state. func (desc *wrapper) mutationColumns( mutationState descpb.DescriptorMutation_State, ) []descpb.ColumnDescriptor { @@ -142,10 +197,18 @@ type Immutable struct { // It is partitioned by the state of the column: public, write-only, delete-only publicAndNonPublicCols []descpb.ColumnDescriptor - writeOnlyColCount int + // publicAndNonPublicCols is a list of public and non-public indexes. + // It is partitioned by the state of the index: public, write-only, delete-only + publicAndNonPublicIndexes []descpb.IndexDescriptor + + writeOnlyColCount int + writeOnlyIndexCount int allChecks []descpb.TableDescriptor_CheckConstraint + // partialIndexOrds contains the ordinal of each partial index. + partialIndexOrds util.FastIntSet + // readableColumns is a list of columns (including those undergoing a schema change) // which can be scanned. Columns in the process of a schema change // are all set to nullable while column backfilling is still in @@ -182,6 +245,11 @@ func (desc *wrapper) GetPrimaryIndexID() descpb.IndexID { return desc.PrimaryIndex.ID } +// GetPublicNonPrimaryIndexes returns the public non-primary indexes of the descriptor. +func (desc *wrapper) GetPublicNonPrimaryIndexes() []descpb.IndexDescriptor { + return desc.Indexes +} + // IsTemporary returns true if this is a temporary table. func (desc *wrapper) IsTemporary() bool { return desc.GetTemporary() @@ -197,8 +265,7 @@ func (desc *wrapper) GetColumnAtIdx(idx int) *descpb.ColumnDescriptor { return &desc.Columns[idx] } -// ReadableColumns returns a list of columns (including those undergoing a -// schema change) which can be scanned. +// ReadableColumns implements the catalog.TableDescriptor interface func (desc *Immutable) ReadableColumns() []descpb.ColumnDescriptor { return desc.readableColumns } @@ -251,129 +318,3 @@ func (desc *Mutable) SetPrimaryIndex(index descpb.IndexDescriptor) { func (desc *Mutable) SetPublicNonPrimaryIndex(indexOrdinal int, index descpb.IndexDescriptor) { desc.Indexes[indexOrdinal-1] = index } - -// GetPrimaryIndex returns the primary index in the form of a catalog.Index -// interface. -func (desc *wrapper) GetPrimaryIndex() catalog.Index { - return desc.getExistingOrNewIndexCache().primaryIndex(desc) -} - -// getExistingOrNewIndexCache should be the only place where the indexCache -// field in wrapper is ever read. -func (desc *wrapper) getExistingOrNewIndexCache() *indexCache { - if desc.indexCache == nil { - return &indexCache{} - } - return desc.indexCache -} - -// AllIndexes returns a slice with all indexes, public and non-public, -// in the underlying proto, in their canonical order: -// - the primary index, -// - the public non-primary indexes in the Indexes array, in order, -// - the non-public indexes present in the Mutations array, in order. -// -// See also catalog.Index.Ordinal(). -func (desc *wrapper) AllIndexes() []catalog.Index { - return desc.getExistingOrNewIndexCache().allIndexes(desc) -} - -// ActiveIndexes returns a slice with all public indexes in the underlying -// proto, in their canonical order: -// - the primary index, -// - the public non-primary indexes in the Indexes array, in order. -// -// See also catalog.Index.Ordinal(). -func (desc *wrapper) ActiveIndexes() []catalog.Index { - return desc.getExistingOrNewIndexCache().activeIndexes(desc) -} - -// NonDropIndexes returns a slice of all non-drop indexes in the underlying -// proto, in their canonical order. This means: -// - the primary index, if the table is a physical table, -// - the public non-primary indexes in the Indexes array, in order, -// - the non-public indexes present in the Mutations array, in order, -// if the mutation is not a drop. -// -// See also catalog.Index.Ordinal(). -func (desc *wrapper) NonDropIndexes() []catalog.Index { - return desc.getExistingOrNewIndexCache().nonDropIndexes(desc) -} - -// NonDropIndexes returns a slice of all partial indexes in the underlying -// proto, in their canonical order. This is equivalent to taking the slice -// produced by AllIndexes and filtering indexes with non-empty expressions. -func (desc *wrapper) PartialIndexes() []catalog.Index { - return desc.getExistingOrNewIndexCache().partialIndexes(desc) -} - -// PublicNonPrimaryIndexes returns a slice of all active secondary indexes, -// in their canonical order. This is equivalent to the Indexes array in the -// proto. -func (desc *wrapper) PublicNonPrimaryIndexes() []catalog.Index { - return desc.getExistingOrNewIndexCache().publicNonPrimaryIndexes(desc) -} - -// WritableNonPrimaryIndexes returns a slice of all non-primary indexes which -// allow being written to: public + delete-and-write-only, in their canonical -// order. This is equivalent to taking the slice produced by -// DeletableNonPrimaryIndexes and removing the indexes which are in mutations -// in the delete-only state. -func (desc *wrapper) WritableNonPrimaryIndexes() []catalog.Index { - return desc.getExistingOrNewIndexCache().writableNonPrimaryIndexes(desc) -} - -// DeletableNonPrimaryIndexes returns a slice of all non-primary indexes -// which allow being deleted from: public + delete-and-write-only + -// delete-only, in their canonical order. This is equivalent to taking -// the slice produced by AllIndexes and removing the primary index. -func (desc *wrapper) DeletableNonPrimaryIndexes() []catalog.Index { - return desc.getExistingOrNewIndexCache().deletableNonPrimaryIndexes(desc) -} - -// DeleteOnlyNonPrimaryIndexes returns a slice of all non-primary indexes -// which allow only being deleted from, in their canonical order. This is -// equivalent to taking the slice produced by DeletableNonPrimaryIndexes and -// removing the indexes which are not in mutations or not in the delete-only -// state. -func (desc *wrapper) DeleteOnlyNonPrimaryIndexes() []catalog.Index { - return desc.getExistingOrNewIndexCache().deleteOnlyNonPrimaryIndexes(desc) -} - -// FindIndexWithID returns the first catalog.Index that matches the id -// in the set of all indexes, or an error if none was found. The order of -// traversal is the canonical order, see catalog.Index.Ordinal(). -func (desc *wrapper) FindIndexWithID(id descpb.IndexID) (catalog.Index, error) { - if idx := catalog.FindIndex(desc, catalog.IndexOpts{ - NonPhysicalPrimaryIndex: true, - DropMutations: true, - AddMutations: true, - }, func(idx catalog.Index) bool { - return idx.GetID() == id - }); idx != nil { - return idx, nil - } - for _, m := range desc.GCMutations { - if m.IndexID == id { - return nil, ErrIndexGCMutationsList - } - } - return nil, errors.Errorf("index-id \"%d\" does not exist", id) -} - -// FindIndexWithName returns the first catalog.Index that matches the name in -// the set of all indexes, excluding the primary index of non-physical -// tables, or an error if none was found. The order of traversal is the -// canonical order, see catalog.Index.Ordinal(). -func (desc *wrapper) FindIndexWithName(name string) (catalog.Index, error) { - if idx := catalog.FindIndex(desc, catalog.IndexOpts{ - NonPhysicalPrimaryIndex: false, - DropMutations: true, - AddMutations: true, - }, func(idx catalog.Index) bool { - return idx.GetName() == name - }); idx != nil { - return idx, nil - } - return nil, errors.Errorf("index %q does not exist", name) -} diff --git a/pkg/sql/check.go b/pkg/sql/check.go index a2541655aa78..bfed97e5bac2 100644 --- a/pkg/sql/check.go +++ b/pkg/sql/check.go @@ -96,8 +96,7 @@ func matchFullUnacceptableKeyQuery( srcNotNullExistsClause[i] = fmt.Sprintf("%s IS NOT NULL", srcCols[i]) } - for i := 0; i < srcTbl.GetPrimaryIndex().NumColumns(); i++ { - id := srcTbl.GetPrimaryIndex().GetColumnID(i) + for _, id := range srcTbl.GetPrimaryIndex().ColumnIDs { alreadyPresent := false for _, otherID := range fk.OriginColumnIDs { if id == otherID { @@ -160,8 +159,7 @@ func nonMatchingRowQuery( return "", nil, err } // Get primary key columns not included in the FK - for i := 0; i < srcTbl.GetPrimaryIndex().NumColumns(); i++ { - pkColID := srcTbl.GetPrimaryIndex().GetColumnID(i) + for _, pkColID := range srcTbl.GetPrimaryIndex().ColumnIDs { found := false for _, id := range fk.OriginColumnIDs { if pkColID == id { diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index b5f1e76064e7..b803e6cb2f4b 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -1807,7 +1807,7 @@ CREATE TABLE crdb_internal.create_statements ( if err != nil { return err } - if err := showAlterStatementWithInterleave(ctx, &name, contextName, lookup, table.PublicNonPrimaryIndexes(), table, alterStmts, + if err := showAlterStatementWithInterleave(ctx, &name, contextName, lookup, table.GetPublicNonPrimaryIndexes(), table, alterStmts, validateStmts, &p.semaCtx); err != nil { return err } @@ -1823,8 +1823,12 @@ CREATE TABLE crdb_internal.create_statements ( if createNofk == "" { createNofk = stmt } - hasPartitions := nil != catalog.FindIndex(table, catalog.IndexOpts{}, func(idx catalog.Index) bool { - return idx.GetPartitioning().NumColumns != 0 + hasPartitions := false + _ = table.ForeachIndex(catalog.IndexOpts{}, func(idxDesc *descpb.IndexDescriptor, isPrimary bool) error { + if idxDesc.Partitioning.NumColumns != 0 { + hasPartitions = true + } + return nil }) return addRow( dbDescID, @@ -1847,7 +1851,7 @@ func showAlterStatementWithInterleave( tn *tree.TableName, contextName string, lCtx simpleSchemaResolver, - allIdx []catalog.Index, + allIdx []descpb.IndexDescriptor, table catalog.TableDescriptor, alterStmts *tree.DArray, validateStmts *tree.DArray, @@ -1888,12 +1892,14 @@ func showAlterStatementWithInterleave( return err } - for _, idx := range allIdx { + for i := range allIdx { + idx := &allIdx[i] // Create CREATE INDEX commands for INTERLEAVE tables. These commands // are included in the ALTER TABLE statements. - if idx.NumInterleaveAncestors() > 0 { + if len(idx.Interleave.Ancestors) > 0 { f := tree.NewFmtCtx(tree.FmtSimple) - parentTableID := idx.GetInterleaveAncestor(idx.NumInterleaveAncestors() - 1).TableID + intl := idx.Interleave + parentTableID := intl.Ancestors[len(intl.Ancestors)-1].TableID var err error var parentName tree.TableName if lCtx != nil { @@ -1919,11 +1925,11 @@ func showAlterStatementWithInterleave( tableName.ExplicitSchema = false } var sharedPrefixLen int - for i := 0; i < idx.NumInterleaveAncestors(); i++ { - sharedPrefixLen += int(idx.GetInterleaveAncestor(i).SharedPrefixLen) + for _, ancestor := range intl.Ancestors { + sharedPrefixLen += int(ancestor.SharedPrefixLen) } // Write the CREATE INDEX statements. - if err := showCreateIndexWithInterleave(ctx, f, table, idx.IndexDesc(), tableName, parentName, sharedPrefixLen, semaCtx); err != nil { + if err := showCreateIndexWithInterleave(ctx, f, table, idx, tableName, parentName, sharedPrefixLen, semaCtx); err != nil { return err } if err := alterStmts.Append(tree.NewDString(f.CloseAndGetString())); err != nil { @@ -2050,22 +2056,22 @@ CREATE TABLE crdb_internal.table_indexes ( tableName := tree.NewDString(table.GetName()) // We report the primary index of non-physical tables here. These // indexes are not reported as a part of ForeachIndex. - return catalog.ForEachIndex(table, catalog.IndexOpts{ + return table.ForeachIndex(catalog.IndexOpts{ NonPhysicalPrimaryIndex: true, - }, func(idx catalog.Index) error { + }, func(idx *descpb.IndexDescriptor, isPrimary bool) error { row = row[:0] idxType := secondary - if idx.Primary() { + if isPrimary { idxType = primary } row = append(row, tableID, tableName, - tree.NewDInt(tree.DInt(idx.GetID())), - tree.NewDString(idx.GetName()), + tree.NewDInt(tree.DInt(idx.ID)), + tree.NewDString(idx.Name), idxType, - tree.MakeDBool(tree.DBool(idx.IsUnique())), - tree.MakeDBool(idx.GetType() == descpb.IndexDescriptor_INVERTED), + tree.MakeDBool(tree.DBool(idx.Unique)), + tree.MakeDBool(idx.Type == descpb.IndexDescriptor_INVERTED), ) return pusher.pushRow(row...) }) @@ -2110,8 +2116,7 @@ CREATE TABLE crdb_internal.index_columns ( parentName := parent.GetName() tableName := tree.NewDString(table.GetName()) - reportIndex := func(idxI catalog.Index) error { - idx := idxI.IndexDesc() + reportIndex := func(idx *descpb.IndexDescriptor) error { idxID := tree.NewDInt(tree.DInt(idx.ID)) idxName := tree.NewDString(idx.Name) @@ -2179,7 +2184,11 @@ CREATE TABLE crdb_internal.index_columns ( return nil } - return catalog.ForEachIndex(table, catalog.IndexOpts{NonPhysicalPrimaryIndex: true}, reportIndex) + return table.ForeachIndex(catalog.IndexOpts{ + NonPhysicalPrimaryIndex: true, + }, func(idxDesc *descpb.IndexDescriptor, _ bool) error { + return reportIndex(idxDesc) + }) }) }, } @@ -2211,19 +2220,17 @@ CREATE TABLE crdb_internal.backward_dependencies ( viewDep := tree.NewDString("view") sequenceDep := tree.NewDString("sequence") interleaveDep := tree.NewDString("interleave") - return forEachTableDescAllWithTableLookup(ctx, p, dbContext, hideVirtual, func( db *dbdesc.Immutable, _ string, table catalog.TableDescriptor, tableLookup tableLookupFn, ) error { tableID := tree.NewDInt(tree.DInt(table.GetID())) tableName := tree.NewDString(table.GetName()) - reportIdxDeps := func(idx catalog.Index) error { - for i := 0; i < idx.NumInterleaveAncestors(); i++ { - interleaveParent := idx.GetInterleaveAncestor(i) + reportIdxDeps := func(idx *descpb.IndexDescriptor) error { + for _, interleaveParent := range idx.Interleave.Ancestors { if err := addRow( tableID, tableName, - tree.NewDInt(tree.DInt(idx.GetID())), + tree.NewDInt(tree.DInt(idx.ID)), tree.DNull, tree.NewDInt(tree.DInt(interleaveParent.TableID)), interleaveDep, @@ -2261,7 +2268,10 @@ CREATE TABLE crdb_internal.backward_dependencies ( } // Record the backward references of the primary index. - if err := catalog.ForEachIndex(table, catalog.IndexOpts{}, reportIdxDeps); err != nil { + if err := table.ForeachIndex(catalog.IndexOpts{}, + func(idxDesc *descpb.IndexDescriptor, _ bool) error { + return reportIdxDeps(idxDesc) + }); err != nil { return err } @@ -2357,12 +2367,11 @@ CREATE TABLE crdb_internal.forward_dependencies ( tableID := tree.NewDInt(tree.DInt(table.GetID())) tableName := tree.NewDString(table.GetName()) - reportIdxDeps := func(idx catalog.Index) error { - for i := 0; i < idx.NumInterleavedBy(); i++ { - interleaveRef := idx.GetInterleavedBy(i) + reportIdxDeps := func(idx *descpb.IndexDescriptor) error { + for _, interleaveRef := range idx.InterleavedBy { if err := addRow( tableID, tableName, - tree.NewDInt(tree.DInt(idx.GetID())), + tree.NewDInt(tree.DInt(idx.ID)), tree.NewDInt(tree.DInt(interleaveRef.Table)), interleaveDep, tree.NewDInt(tree.DInt(interleaveRef.Index)), @@ -2390,7 +2399,9 @@ CREATE TABLE crdb_internal.forward_dependencies ( } // Record the backward references of the primary index. - if err := catalog.ForEachIndex(table, catalog.IndexOpts{}, reportIdxDeps); err != nil { + if err := table.ForeachIndex(catalog.IndexOpts{}, func(idxDesc *descpb.IndexDescriptor, isPrimary bool) error { + return reportIdxDeps(idxDesc) + }); err != nil { return err } reportDependedOnBy := func( @@ -2502,8 +2513,8 @@ CREATE TABLE crdb_internal.ranges_no_leases ( parents[id] = uint32(desc.ParentID) tableNames[id] = desc.GetName() indexNames[id] = make(map[uint32]string) - for _, idx := range desc.PublicNonPrimaryIndexes() { - indexNames[id][uint32(idx.GetID())] = idx.GetName() + for _, idx := range desc.GetPublicNonPrimaryIndexes() { + indexNames[id][uint32(idx.ID)] = idx.Name } case *dbdesc.Immutable: dbNames[id] = desc.GetName() @@ -2820,9 +2831,7 @@ CREATE TABLE crdb_internal.zones ( } for i, s := range subzones { - index := catalog.FindActiveIndex(table, func(idx catalog.Index) bool { - return idx.GetID() == descpb.IndexID(s.IndexID) - }) + index := table.FindActiveIndexByID(descpb.IndexID(s.IndexID)) if index == nil { // If we can't find an active index that corresponds to this index // ID then continue, as the index is being dropped, or is already @@ -2831,7 +2840,7 @@ CREATE TABLE crdb_internal.zones ( } if zoneSpecifier != nil { zs := zs - zs.TableOrIndex.Index = tree.UnrestrictedName(index.GetName()) + zs.TableOrIndex.Index = tree.UnrestrictedName(index.Name) zs.Partition = tree.Name(s.PartitionName) zoneSpecifier = &zs } @@ -2847,7 +2856,7 @@ CREATE TABLE crdb_internal.zones ( } else { // We have a partition. Get the parent index partition from the zone and // have it inherit constraints. - if indexSubzone := fullZone.GetSubzone(uint32(index.GetID()), ""); indexSubzone != nil { + if indexSubzone := fullZone.GetSubzone(uint32(index.ID), ""); indexSubzone != nil { subZoneConfig.InheritFromParent(&indexSubzone.Config) } // Inherit remaining fields from the full parent zone. @@ -3404,10 +3413,10 @@ CREATE TABLE crdb_internal.partitions ( worker := func(pusher rowPusher) error { return forEachTableDescAll(ctx, p, dbContext, hideVirtual, /* virtual tables have no partitions*/ func(db *dbdesc.Immutable, _ string, table catalog.TableDescriptor) error { - return catalog.ForEachIndex(table, catalog.IndexOpts{ + return table.ForeachIndex(catalog.IndexOpts{ AddMutations: true, - }, func(index catalog.Index) error { - return addPartitioningRows(ctx, p, dbName, table, index.IndexDesc(), &index.IndexDesc().Partitioning, + }, func(index *descpb.IndexDescriptor, _ bool) error { + return addPartitioningRows(ctx, p, dbName, table, index, &index.Partitioning, tree.DNull /* parentName */, 0 /* colOffset */, pusher.pushRow) }) }) diff --git a/pkg/sql/create_index.go b/pkg/sql/create_index.go index 1a649bea41c1..f57500b1e8a2 100644 --- a/pkg/sql/create_index.go +++ b/pkg/sql/create_index.go @@ -371,9 +371,9 @@ func maybeCreateAndAddShardCol( func (n *createIndexNode) startExec(params runParams) error { telemetry.Inc(sqltelemetry.SchemaChangeCreateCounter("index")) - foundIndex, err := n.tableDesc.FindIndexWithName(string(n.n.Name)) + _, dropped, err := n.tableDesc.FindIndexByName(string(n.n.Name)) if err == nil { - if foundIndex.Dropped() { + if dropped { return pgerror.Newf(pgcode.ObjectNotInPrerequisiteState, "index %q being dropped, try again later", string(n.n.Name)) } @@ -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.GetPrimaryIndex().GetPartitioning().NumColumns > 0 { + if n.n.PartitionBy == nil && n.tableDesc.GetPrimaryIndex().Partitioning.NumColumns > 0 { params.p.BufferClientNotice( params.ctx, errors.WithHint( diff --git a/pkg/sql/create_stats.go b/pkg/sql/create_stats.go index 4f94b3e6fe8d..f727c46f2cdf 100644 --- a/pkg/sql/create_stats.go +++ b/pkg/sql/create_stats.go @@ -303,7 +303,7 @@ const maxNonIndexCols = 100 func createStatsDefaultColumns( desc *tabledesc.Immutable, multiColEnabled bool, ) ([]jobspb.CreateStatsDetails_ColStat, error) { - colStats := make([]jobspb.CreateStatsDetails_ColStat, 0, len(desc.ActiveIndexes())) + colStats := make([]jobspb.CreateStatsDetails_ColStat, 0, len(desc.GetPublicNonPrimaryIndexes())+1) requestedStats := make(map[string]struct{}) @@ -350,19 +350,16 @@ func createStatsDefaultColumns( } // Add column stats for the primary key. - for i := 0; i < desc.GetPrimaryIndex().NumColumns(); i++ { + for i := range desc.GetPrimaryIndex().ColumnIDs { // Generate stats for each column in the primary key. - addIndexColumnStatsIfNotExists(desc.GetPrimaryIndex().GetColumnID(i), false /* isInverted */) + addIndexColumnStatsIfNotExists(desc.GetPrimaryIndex().ColumnIDs[i], false /* isInverted */) // Only collect multi-column stats if enabled. if i == 0 || !multiColEnabled { continue } - colIDs := make([]descpb.ColumnID, i+1) - for j := 0; j <= i; j++ { - colIDs[j] = desc.GetPrimaryIndex().GetColumnID(j) - } + colIDs := desc.GetPrimaryIndex().ColumnIDs[: i+1 : i+1] // Remember the requested stats so we don't request duplicates. trackStatsIfNotExists(colIDs) @@ -375,22 +372,19 @@ func createStatsDefaultColumns( } // Add column stats for each secondary index. - for _, idx := range desc.PublicNonPrimaryIndexes() { - isInverted := idx.GetType() == descpb.IndexDescriptor_INVERTED + for i := range desc.GetPublicNonPrimaryIndexes() { + isInverted := desc.GetPublicNonPrimaryIndexes()[i].Type == descpb.IndexDescriptor_INVERTED - for j := 0; j < idx.NumColumns(); j++ { + for j := range desc.GetPublicNonPrimaryIndexes()[i].ColumnIDs { // Generate stats for each indexed column. - addIndexColumnStatsIfNotExists(idx.GetColumnID(j), isInverted) + addIndexColumnStatsIfNotExists(desc.GetPublicNonPrimaryIndexes()[i].ColumnIDs[j], isInverted) // Only collect multi-column stats if enabled. if j == 0 || !multiColEnabled { continue } - colIDs := make([]descpb.ColumnID, j+1) - for k := 0; k <= j; k++ { - colIDs[k] = idx.GetColumnID(k) - } + colIDs := desc.GetPublicNonPrimaryIndexes()[i].ColumnIDs[: j+1 : j+1] // Check for existing stats and remember the requested stats. if !trackStatsIfNotExists(colIDs) { @@ -405,8 +399,8 @@ func createStatsDefaultColumns( } // Add columns referenced in partial index predicate expressions. - if idx.IsPartial() { - expr, err := parser.ParseExpr(idx.GetPredicate()) + if desc.GetPublicNonPrimaryIndexes()[i].IsPartial() { + expr, err := parser.ParseExpr(desc.GetPublicNonPrimaryIndexes()[i].Predicate) if err != nil { return nil, err } diff --git a/pkg/sql/create_table.go b/pkg/sql/create_table.go index 57df9d1981d7..23f5dcd21ac9 100644 --- a/pkg/sql/create_table.go +++ b/pkg/sql/create_table.go @@ -350,9 +350,9 @@ func (n *createTableNode) startExec(params runParams) error { } } - for _, index := range desc.NonDropIndexes() { - if index.NumInterleaveAncestors() > 0 { - if err := params.p.finalizeInterleave(params.ctx, desc, index.IndexDesc()); err != nil { + for _, index := range desc.AllNonDropIndexes() { + if len(index.Interleave.Ancestors) > 0 { + if err := params.p.finalizeInterleave(params.ctx, desc, index); err != nil { return err } } @@ -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.GetPrimaryIndex().NumColumns()) - for i := range referencedColNames { - referencedColNames[i] = tree.Name(target.GetPrimaryIndex().GetColumnName(i)) + referencedColNames = make(tree.NameList, len(target.GetPrimaryIndex().ColumnNames)) + for i, n := range target.GetPrimaryIndex().ColumnNames { + referencedColNames[i] = tree.Name(n) } } @@ -977,8 +977,8 @@ func addIndexForFK( if err := tbl.AllocateIDs(ctx); err != nil { return 0, err } - added := tbl.PublicNonPrimaryIndexes()[len(tbl.PublicNonPrimaryIndexes())-1] - return added.GetID(), nil + added := tbl.GetPublicNonPrimaryIndexes()[len(tbl.GetPublicNonPrimaryIndexes())-1] + return added.ID, nil } // TODO (lucy): In the EmptyTable case, we add an index mutation, making this @@ -1035,12 +1035,12 @@ func addInterleave( typeOfIndex = "index" } - if len(interleave.Fields) != parentIndex.NumColumns() { + if len(interleave.Fields) != len(parentIndex.ColumnIDs) { return pgerror.Newf( pgcode.InvalidSchemaDefinition, "declared interleaved columns (%s) must match the parent's primary index (%s)", &interleave.Fields, - strings.Join(parentIndex.IndexDesc().ColumnNames, ", "), + strings.Join(parentIndex.ColumnNames, ", "), ) } if len(interleave.Fields) > len(index.ColumnIDs) { @@ -1053,8 +1053,7 @@ func addInterleave( ) } - for i := 0; i < parentIndex.NumColumns(); i++ { - targetColID := parentIndex.GetColumnID(i) + for i, targetColID := range parentIndex.ColumnIDs { targetCol, err := parentTable.FindColumnByID(targetColID) if err != nil { return err @@ -1072,25 +1071,22 @@ func addInterleave( strings.Join(index.ColumnNames, ", "), ) } - if !col.Type.Identical(targetCol.Type) || index.ColumnDirections[i] != parentIndex.GetColumnDirection(i) { + if !col.Type.Identical(targetCol.Type) || index.ColumnDirections[i] != parentIndex.ColumnDirections[i] { return pgerror.Newf( pgcode.InvalidSchemaDefinition, "declared interleaved columns (%s) must match type and sort direction of the parent's primary index (%s)", &interleave.Fields, - strings.Join(parentIndex.IndexDesc().ColumnNames, ", "), + strings.Join(parentIndex.ColumnNames, ", "), ) } } - ancestorPrefix := make([]descpb.InterleaveDescriptor_Ancestor, parentIndex.NumInterleaveAncestors()) - for i := range ancestorPrefix { - ancestorPrefix[i] = parentIndex.GetInterleaveAncestor(i) - } - + ancestorPrefix := append( + []descpb.InterleaveDescriptor_Ancestor(nil), parentIndex.Interleave.Ancestors...) intl := descpb.InterleaveDescriptor_Ancestor{ TableID: parentTable.ID, - IndexID: parentIndex.GetID(), - SharedPrefixLen: uint32(parentIndex.NumColumns()), + IndexID: parentIndex.ID, + SharedPrefixLen: uint32(len(parentIndex.ColumnIDs)), } for _, ancestor := range ancestorPrefix { intl.SharedPrefixLen -= ancestor.SharedPrefixLen @@ -1122,11 +1118,11 @@ func (p *planner) finalizeInterleave( return err } } - ancestorIndex, err := ancestorTable.FindIndexWithID(ancestor.IndexID) + ancestorIndex, err := ancestorTable.FindIndexByID(ancestor.IndexID) if err != nil { return err } - ancestorIndex.IndexDesc().InterleavedBy = append(ancestorIndex.IndexDesc().InterleavedBy, + ancestorIndex.InterleavedBy = append(ancestorIndex.InterleavedBy, descpb.ForeignKeyReference{Table: desc.ID, Index: index.ID}) if err := p.writeSchemaChange( @@ -1652,7 +1648,7 @@ func NewTableDesc( } // If explicit primary keys are required, error out since a primary key was not supplied. - if desc.GetPrimaryIndex().NumColumns() == 0 && desc.IsPhysicalTable() && evalCtx != nil && + if len(desc.GetPrimaryIndex().ColumnNames) == 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) @@ -1686,14 +1682,14 @@ func NewTableDesc( // Assign any implicitly added shard columns to the column family of the first column // in their corresponding set of index columns. - for _, index := range desc.NonDropIndexes() { - if index.IsSharded() && !columnsInExplicitFamilies[index.GetShardColumnName()] { + for _, index := range desc.AllNonDropIndexes() { + if index.IsSharded() && !columnsInExplicitFamilies[index.Sharded.Name] { // Ensure that the shard column wasn't explicitly assigned a column family // during table creation (this will happen when a create statement is // "roundtripped", for example). - family := tabledesc.GetColumnFamilyForShard(&desc, index.GetSharded().ColumnNames) + family := tabledesc.GetColumnFamilyForShard(&desc, index.Sharded.ColumnNames) if family != "" { - if err := desc.AddColumnToFamilyMaybeCreate(index.GetShardColumnName(), family, false, false); err != nil { + if err := desc.AddColumnToFamilyMaybeCreate(index.Sharded.Name, family, false, false); err != nil { return nil, err } } @@ -1704,27 +1700,28 @@ func NewTableDesc( return nil, err } - for _, idx := range desc.PublicNonPrimaryIndexes() { + for i := range desc.GetPublicNonPrimaryIndexes() { + idx := &desc.GetPublicNonPrimaryIndexes()[i] // Increment the counter if this index could be storing data across multiple column families. - if idx.NumStoredColumns() > 1 && len(desc.Families) > 1 { + if len(idx.StoreColumnNames) > 1 && len(desc.Families) > 1 { telemetry.Inc(sqltelemetry.SecondaryIndexColumnFamiliesCounter) } } if n.Interleave != nil { - if err := addInterleave(ctx, txn, vt, &desc, desc.GetPrimaryIndex().IndexDesc(), n.Interleave); err != nil { + if err := addInterleave(ctx, txn, vt, &desc, desc.GetPrimaryIndex(), n.Interleave); err != nil { return nil, err } } if n.PartitionBy != nil { partitioning, err := CreatePartitioning( - ctx, st, evalCtx, &desc, desc.GetPrimaryIndex().IndexDesc(), n.PartitionBy) + ctx, st, evalCtx, &desc, desc.GetPrimaryIndex(), n.PartitionBy) if err != nil { return nil, err } { - newPrimaryIndex := *desc.GetPrimaryIndex().IndexDesc() + newPrimaryIndex := *desc.GetPrimaryIndex() newPrimaryIndex.Partitioning = partitioning desc.SetPrimaryIndex(newPrimaryIndex) } @@ -1886,17 +1883,16 @@ func NewTableDesc( } // Record the types of indexes that the table has. - if err := catalog.ForEachNonDropIndex(&desc, func(idx catalog.Index) error { + if err := desc.ForeachNonDropIndex(func(idx *descpb.IndexDescriptor) error { if idx.IsSharded() { telemetry.Inc(sqltelemetry.HashShardedIndexCounter) } - if idx.GetType() == descpb.IndexDescriptor_INVERTED { + if idx.Type == descpb.IndexDescriptor_INVERTED { telemetry.Inc(sqltelemetry.InvertedIndexCounter) - geoConfig := idx.GetGeoConfig() - if !geoindex.IsEmptyConfig(&geoConfig) { - if geoindex.IsGeographyConfig(&geoConfig) { + if !geoindex.IsEmptyConfig(&idx.GeoConfig) { + if geoindex.IsGeographyConfig(&idx.GeoConfig) { telemetry.Inc(sqltelemetry.GeographyInvertedIndexCounter) - } else if geoindex.IsGeometryConfig(&geoConfig) { + } else if geoindex.IsGeometryConfig(&idx.GeoConfig) { telemetry.Inc(sqltelemetry.GeometryInvertedIndexCounter) } } @@ -2138,50 +2134,47 @@ func replaceLikeTableOpts(n *tree.CreateTable, params runParams) (tree.TableDefs } } if opts.Has(tree.LikeTableOptIndexes) { - for _, idx := range td.NonDropIndexes() { + for _, idx := range td.AllNonDropIndexes() { indexDef := tree.IndexTableDef{ - Name: tree.Name(idx.GetName()), - Inverted: idx.GetType() == descpb.IndexDescriptor_INVERTED, - Storing: make(tree.NameList, 0, idx.NumStoredColumns()), - Columns: make(tree.IndexElemList, 0, idx.NumColumns()), + Name: tree.Name(idx.Name), + Inverted: idx.Type == descpb.IndexDescriptor_INVERTED, + Storing: make(tree.NameList, 0, len(idx.StoreColumnNames)), + Columns: make(tree.IndexElemList, 0, len(idx.ColumnNames)), } - numColumns := idx.NumColumns() + columnNames := idx.ColumnNames if idx.IsSharded() { indexDef.Sharded = &tree.ShardedIndexDef{ - ShardBuckets: tree.NewDInt(tree.DInt(idx.GetSharded().ShardBuckets)), + ShardBuckets: tree.NewDInt(tree.DInt(idx.Sharded.ShardBuckets)), } - numColumns = len(idx.GetSharded().ColumnNames) + columnNames = idx.Sharded.ColumnNames } - for j := 0; j < numColumns; j++ { - name := idx.GetColumnName(j) - if idx.IsSharded() { - name = idx.GetSharded().ColumnNames[j] - } + for i, name := range columnNames { elem := tree.IndexElem{ Column: tree.Name(name), Direction: tree.Ascending, } - if idx.GetColumnDirection(j) == descpb.IndexDescriptor_DESC { + if idx.ColumnDirections[i] == descpb.IndexDescriptor_DESC { elem.Direction = tree.Descending } indexDef.Columns = append(indexDef.Columns, elem) } - for j := 0; j < idx.NumStoredColumns(); j++ { - indexDef.Storing = append(indexDef.Storing, tree.Name(idx.GetStoredColumnName(j))) + for _, name := range idx.StoreColumnNames { + indexDef.Storing = append(indexDef.Storing, tree.Name(name)) } var def tree.TableDef = &indexDef - if idx.IsUnique() { - if idx.Primary() && td.IsPrimaryIndexDefaultRowID() { + if idx.Unique { + isPK := idx.ID == td.GetPrimaryIndexID() + if isPK && td.IsPrimaryIndexDefaultRowID() { continue } def = &tree.UniqueConstraintTableDef{ IndexTableDef: indexDef, - PrimaryKey: idx.Primary(), + PrimaryKey: isPK, } } if idx.IsPartial() { - indexDef.Predicate, err = parser.ParseExpr(idx.GetPredicate()) + indexDef.Predicate, err = parser.ParseExpr(idx.Predicate) if err != nil { return nil, err } diff --git a/pkg/sql/delete.go b/pkg/sql/delete.go index e51d96794d83..26a73ba2537d 100644 --- a/pkg/sql/delete.go +++ b/pkg/sql/delete.go @@ -155,7 +155,8 @@ func (d *deleteNode) processSourceRow(params runParams, sourceVals tree.Datums) // satisfy the predicate and therefore do not exist in the partial index. // This set is passed as a argument to tableDeleter.row below. var pm row.PartialIndexUpdateHelper - if len(d.run.td.tableDesc().PartialIndexes()) > 0 { + partialIndexOrds := d.run.td.tableDesc().PartialIndexOrds() + if !partialIndexOrds.Empty() { partialIndexDelVals := sourceVals[d.run.partialIndexDelValsOffset:] err := pm.Init(tree.Datums{}, partialIndexDelVals, d.run.td.tableDesc()) diff --git a/pkg/sql/delete_range.go b/pkg/sql/delete_range.go index 2535d66f7e90..6885ec9de161 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.GetPrimaryIndex().IndexDesc(), + Index: d.desc.GetPrimaryIndex(), Spans: d.spans, } for i, interleaved := range d.interleavedDesc { allTables[i+1] = row.FetcherTableArgs{ Desc: interleaved, - Index: interleaved.GetPrimaryIndex().IndexDesc(), + Index: interleaved.GetPrimaryIndex(), Spans: d.spans, } } diff --git a/pkg/sql/descriptor_mutation_test.go b/pkg/sql/descriptor_mutation_test.go index d90650f22a2f..45585f5acbf2 100644 --- a/pkg/sql/descriptor_mutation_test.go +++ b/pkg/sql/descriptor_mutation_test.go @@ -480,14 +480,20 @@ func (mt mutationTest) writeIndexMutation( ctx context.Context, index string, m descpb.DescriptorMutation, ) { tableDesc := mt.tableDesc - idx, err := tableDesc.FindIndexWithName(index) + idx, _, err := tableDesc.FindIndexByName(index) if err != nil { mt.Fatal(err) } // The rewrite below potentially invalidates the original object with an overwrite. // Clarify what's going on. - idxCopy := *idx.IndexDesc() - tableDesc.RemovePublicNonPrimaryIndex(idx.Ordinal()) + idxCopy := *idx + for i, index := range tableDesc.GetPublicNonPrimaryIndexes() { + if idxCopy.ID == index.ID { + tableDesc.RemovePublicNonPrimaryIndex(i + 1) + break + } + } + m.Descriptor_ = &descpb.DescriptorMutation_Index{Index: &idxCopy} mt.writeMutation(ctx, m) } @@ -642,9 +648,9 @@ CREATE TABLE t.test (k CHAR PRIMARY KEY, v CHAR, INDEX foo (v)); // Check that a mutation can only be inserted with an explicit mutation state. tableDesc = mTest.tableDesc - index := tableDesc.PublicNonPrimaryIndexes()[len(tableDesc.PublicNonPrimaryIndexes())-1] - tableDesc.Mutations = []descpb.DescriptorMutation{{Descriptor_: &descpb.DescriptorMutation_Index{Index: index.IndexDesc()}}} - tableDesc.RemovePublicNonPrimaryIndex(index.Ordinal()) + indexIdx := len(tableDesc.GetPublicNonPrimaryIndexes()) + tableDesc.Mutations = []descpb.DescriptorMutation{{Descriptor_: &descpb.DescriptorMutation_Index{Index: &tableDesc.GetPublicNonPrimaryIndexes()[indexIdx-1]}}} + tableDesc.RemovePublicNonPrimaryIndex(indexIdx) if err := tableDesc.ValidateTable(ctx); !testutils.IsError(err, "mutation in state UNKNOWN, direction NONE, index foo, id 2") { t.Fatal(err) } diff --git a/pkg/sql/distsql_physical_planner.go b/pkg/sql/distsql_physical_planner.go index e9ecc462845a..603f80d8d44b 100644 --- a/pkg/sql/distsql_physical_planner.go +++ b/pkg/sql/distsql_physical_planner.go @@ -960,9 +960,14 @@ func (dsp *DistSQLPlanner) nodeVersionIsCompatible(nodeID roachpb.NodeID) bool { } func getIndexIdx(index *descpb.IndexDescriptor, desc *tabledesc.Immutable) (uint32, error) { - foundIndex, _ := desc.FindIndexWithID(index.ID) - if foundIndex != nil && foundIndex.Public() { - return uint32(foundIndex.Ordinal()), nil + if index.ID == desc.GetPrimaryIndexID() { + return 0, nil + } + for i := range desc.GetPublicNonPrimaryIndexes() { + if index.ID == desc.GetPublicNonPrimaryIndexes()[i].ID { + // IndexIdx is 1 based (0 means primary index). + return uint32(i + 1), nil + } } return 0, errors.Errorf("invalid index %v (table %s)", index, desc.Name) } diff --git a/pkg/sql/distsql_plan_stats.go b/pkg/sql/distsql_plan_stats.go index 2420beb0808b..dbbdce5166fe 100644 --- a/pkg/sql/distsql_plan_stats.go +++ b/pkg/sql/distsql_plan_stats.go @@ -130,9 +130,9 @@ func (dsp *DistSQLPlanner) createStatsPlan( // TODO(mjibson): allow multiple inverted indexes on the same column (i.e., // with different configurations). See #50655. col := s.columns[0] - for _, index := range desc.PublicNonPrimaryIndexes() { - if index.GetType() == descpb.IndexDescriptor_INVERTED && index.GetColumnID(0) == col { - spec.Index = index.IndexDesc() + for _, indexDesc := range desc.GetPublicNonPrimaryIndexes() { + if indexDesc.Type == descpb.IndexDescriptor_INVERTED && indexDesc.ColumnIDs[0] == col { + spec.Index = &indexDesc break } } diff --git a/pkg/sql/drop_index.go b/pkg/sql/drop_index.go index 2beb76a77bb6..b3e93f340706 100644 --- a/pkg/sql/drop_index.go +++ b/pkg/sql/drop_index.go @@ -20,7 +20,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvclient" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server/telemetry" - "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" @@ -118,12 +117,12 @@ func (n *dropIndexNode) startExec(params runParams) error { // If we couldn't find the index by name, this is either a legitimate error or // this statement contains an 'IF EXISTS' qualifier. Both of these cases are // handled by `dropIndexByName()` below so we just ignore the error here. - idx, _ := tableDesc.FindIndexWithName(string(index.idxName)) + idxDesc, dropped, _ := tableDesc.FindIndexByName(string(index.idxName)) var shardColName string // If we're dropping a sharded index, record the name of its shard column to // potentially drop it if no other index refers to it. - if idx != nil && idx.IsSharded() && !idx.Dropped() { - shardColName = idx.GetShardColumnName() + if idxDesc != nil && idxDesc.IsSharded() && !dropped { + shardColName = idxDesc.Sharded.Name } if err := params.p.dropIndexByName( @@ -202,9 +201,14 @@ func (n *dropIndexNode) maybeDropShardColumn( if dropped { return nil } - if catalog.FindNonDropIndex(tableDesc, func(otherIdx catalog.Index) bool { - return otherIdx.ContainsColumnID(shardColDesc.ID) - }) != nil { + shouldDropShardColumn := true + for _, otherIdx := range tableDesc.AllNonDropIndexes() { + if otherIdx.ContainsColumnID(shardColDesc.ID) { + shouldDropShardColumn = false + break + } + } + if !shouldDropShardColumn { return nil } return n.dropShardColumnAndConstraint(params, tableDesc, shardColDesc) @@ -241,7 +245,7 @@ func (p *planner) dropIndexByName( constraintBehavior dropIndexConstraintBehavior, jobDesc string, ) error { - idxI, err := tableDesc.FindIndexWithName(string(idxName)) + idx, dropped, err := tableDesc.FindIndexByName(string(idxName)) if err != nil { // Only index names of the form "table@idx" throw an error here if they // don't exist. @@ -252,11 +256,10 @@ func (p *planner) dropIndexByName( // Index does not exist, but we want it to: error out. return pgerror.WithCandidateCode(err, pgcode.UndefinedObject) } - if idxI.Dropped() { + if dropped { return nil } - idx := idxI.IndexDesc() if idx.Unique && behavior != tree.DropCascade && constraintBehavior != ignoreIdxConstraint && !idx.CreatedExplicitly { return errors.WithHint( pgerror.Newf(pgcode.DependentObjectsStillExist, @@ -312,11 +315,12 @@ func (p *planner) dropIndexByName( // Construct a list of all the remaining indexes, so that we can see if there // 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.GetPrimaryIndex().IndexDesc() - for _, index := range tableDesc.PublicNonPrimaryIndexes() { - if index.GetID() != idx.ID { - remainingIndexes = append(remainingIndexes, index.IndexDesc()) + remainingIndexes := make([]*descpb.IndexDescriptor, 0, len(tableDesc.GetPublicNonPrimaryIndexes())+1) + remainingIndexes = append(remainingIndexes, tableDesc.GetPrimaryIndex()) + for i := range tableDesc.GetPublicNonPrimaryIndexes() { + index := &tableDesc.GetPublicNonPrimaryIndexes()[i] + if index.ID != idx.ID { + remainingIndexes = append(remainingIndexes, index) } } @@ -456,11 +460,51 @@ func (p *planner) dropIndexByName( ) } - foundIndex := catalog.FindPublicNonPrimaryIndex(tableDesc, func(idxEntry catalog.Index) bool { - return idxEntry.GetID() == idx.ID - }) + found := false + for i, idxEntry := range tableDesc.GetPublicNonPrimaryIndexes() { + if idxEntry.ID == idx.ID { + // Unsplit all manually split ranges in the index so they can be + // automatically merged by the merge queue. Gate this on being the + // system tenant because secondary tenants aren't allowed to scan + // the meta ranges directly. + if p.ExecCfg().Codec.ForSystemTenant() { + span := tableDesc.IndexSpan(p.ExecCfg().Codec, idxEntry.ID) + ranges, err := kvclient.ScanMetaKVs(ctx, p.txn, span) + if err != nil { + return err + } + for _, r := range ranges { + var desc roachpb.RangeDescriptor + if err := r.ValueProto(&desc); err != nil { + return err + } + // We have to explicitly check that the range descriptor's start key + // lies within the span of the index since ScanMetaKVs returns all + // intersecting spans. + if !desc.GetStickyBit().IsEmpty() && span.Key.Compare(desc.StartKey.AsRawKey()) <= 0 { + // Swallow "key is not the start of a range" errors because it would + // mean that the sticky bit was removed and merged concurrently. DROP + // INDEX should not fail because of this. + if err := p.ExecCfg().DB.AdminUnsplit(ctx, desc.StartKey); err != nil && !strings.Contains(err.Error(), "is not the start of a range") { + return err + } + } + } + } - if foundIndex == nil { + // the idx we picked up with FindIndexByID at the top may not + // contain the same field any more due to other schema changes + // intervening since the initial lookup. So we send the recent + // copy idxEntry for drop instead. + if err := tableDesc.AddIndexMutation(&idxEntry, descpb.DescriptorMutation_DROP); err != nil { + return err + } + tableDesc.RemovePublicNonPrimaryIndex(i + 1) + found = true + break + } + } + if !found { return pgerror.Newf( pgcode.ObjectNotInPrerequisiteState, "index %q in the middle of being added, try again later", @@ -468,47 +512,6 @@ func (p *planner) dropIndexByName( ) } - idxEntry := *foundIndex.IndexDesc() - idxOrdinal := foundIndex.Ordinal() - - // Unsplit all manually split ranges in the index so they can be - // automatically merged by the merge queue. Gate this on being the - // system tenant because secondary tenants aren't allowed to scan - // the meta ranges directly. - if p.ExecCfg().Codec.ForSystemTenant() { - span := tableDesc.IndexSpan(p.ExecCfg().Codec, idxEntry.ID) - ranges, err := kvclient.ScanMetaKVs(ctx, p.txn, span) - if err != nil { - return err - } - for _, r := range ranges { - var desc roachpb.RangeDescriptor - if err := r.ValueProto(&desc); err != nil { - return err - } - // We have to explicitly check that the range descriptor's start key - // lies within the span of the index since ScanMetaKVs returns all - // intersecting spans. - if !desc.GetStickyBit().IsEmpty() && span.Key.Compare(desc.StartKey.AsRawKey()) <= 0 { - // Swallow "key is not the start of a range" errors because it would - // mean that the sticky bit was removed and merged concurrently. DROP - // INDEX should not fail because of this. - if err := p.ExecCfg().DB.AdminUnsplit(ctx, desc.StartKey); err != nil && !strings.Contains(err.Error(), "is not the start of a range") { - return err - } - } - } - } - - // the idx we picked up with FindIndexByID at the top may not - // contain the same field any more due to other schema changes - // intervening since the initial lookup. So we send the recent - // copy idxEntry for drop instead. - if err := tableDesc.AddIndexMutation(&idxEntry, descpb.DescriptorMutation_DROP); err != nil { - return err - } - tableDesc.RemovePublicNonPrimaryIndex(idxOrdinal) - if err := p.removeIndexComment(ctx, tableDesc.ID, idx.ID); err != nil { return err } diff --git a/pkg/sql/drop_table.go b/pkg/sql/drop_table.go index 609884ae1a85..2aafe6d4c145 100644 --- a/pkg/sql/drop_table.go +++ b/pkg/sql/drop_table.go @@ -84,9 +84,8 @@ func (p *planner) DropTable(ctx context.Context, n *tree.DropTable) (planNode, e } } } - for _, idx := range droppedDesc.NonDropIndexes() { - for i := 0; i < idx.NumInterleavedBy(); i++ { - ref := idx.GetInterleavedBy(i) + for _, idx := range droppedDesc.AllNonDropIndexes() { + for _, ref := range idx.InterleavedBy { if _, ok := td[ref.Table]; !ok { if err := p.canRemoveInterleave(ctx, droppedDesc.Name, ref, n.DropBehavior); err != nil { return nil, err @@ -258,11 +257,11 @@ func (p *planner) removeInterleave(ctx context.Context, ref descpb.ForeignKeyRef // The referenced table is being dropped. No need to modify it further. return nil } - idx, err := table.FindIndexWithID(ref.Index) + idx, err := table.FindIndexByID(ref.Index) if err != nil { return err } - idx.IndexDesc().Interleave.Ancestors = nil + idx.Interleave.Ancestors = nil // No job description, since this is presumably part of some larger schema change. return p.writeSchemaChange(ctx, table, descpb.InvalidMutationID, "") } @@ -301,14 +300,13 @@ func (p *planner) dropTableImpl( tableDesc.InboundFKs = nil // Remove interleave relationships. - for _, idx := range tableDesc.NonDropIndexes() { - if idx.NumInterleaveAncestors() > 0 { - if err := p.removeInterleaveBackReference(ctx, tableDesc, idx.IndexDesc()); err != nil { + for _, idx := range tableDesc.AllNonDropIndexes() { + if len(idx.Interleave.Ancestors) > 0 { + if err := p.removeInterleaveBackReference(ctx, tableDesc, idx); err != nil { return droppedViews, err } } - for i := 0; i < idx.NumInterleavedBy(); i++ { - ref := idx.GetInterleavedBy(i) + for _, ref := range idx.InterleavedBy { if err := p.removeInterleave(ctx, ref); err != nil { return droppedViews, err } @@ -644,11 +642,10 @@ func (p *planner) removeInterleaveBackReference( // The referenced table is being dropped. No need to modify it further. return nil } - targetIdxI, err := t.FindIndexWithID(ancestor.IndexID) + targetIdx, err := t.FindIndexByID(ancestor.IndexID) if err != nil { return err } - targetIdx := targetIdxI.IndexDesc() foundAncestor := false for k, ref := range targetIdx.InterleavedBy { if ref.Table == tableDesc.ID && ref.Index == idx.ID { diff --git a/pkg/sql/drop_test.go b/pkg/sql/drop_test.go index 7ee23c0358da..7b7249be34a4 100644 --- a/pkg/sql/drop_test.go +++ b/pkg/sql/drop_test.go @@ -425,18 +425,18 @@ func TestDropIndex(t *testing.T) { } tableDesc := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "kv") tests.CheckKeyCount(t, kvDB, tableDesc.TableSpan(keys.SystemSQLCodec), 3*numRows) - idx, err := tableDesc.FindIndexWithName("foo") + idx, _, err := tableDesc.FindIndexByName("foo") if err != nil { t.Fatal(err) } - indexSpan := tableDesc.IndexSpan(keys.SystemSQLCodec, idx.GetID()) + indexSpan := tableDesc.IndexSpan(keys.SystemSQLCodec, idx.ID) tests.CheckKeyCount(t, kvDB, indexSpan, numRows) if _, err := sqlDB.Exec(`DROP INDEX t.kv@foo`); err != nil { t.Fatal(err) } tableDesc = catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "kv") - if _, err := tableDesc.FindIndexWithName("foo"); err == nil { + if _, _, err := tableDesc.FindIndexByName("foo"); err == nil { t.Fatalf("table descriptor still contains index after index is dropped") } // Index data hasn't been deleted. @@ -462,11 +462,11 @@ func TestDropIndex(t *testing.T) { } tableDesc = catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "kv") - newIdx, err := tableDesc.FindIndexWithName("foo") + newIdx, _, err := tableDesc.FindIndexByName("foo") if err != nil { t.Fatal(err) } - newIdxSpan := tableDesc.IndexSpan(keys.SystemSQLCodec, newIdx.GetID()) + newIdxSpan := tableDesc.IndexSpan(keys.SystemSQLCodec, newIdx.ID) tests.CheckKeyCount(t, kvDB, newIdxSpan, numRows) tests.CheckKeyCount(t, kvDB, tableDesc.TableSpan(keys.SystemSQLCodec), 4*numRows) @@ -525,11 +525,11 @@ func TestDropIndexWithZoneConfigOSS(t *testing.T) { t.Fatal(err) } tableDesc := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "kv") - index, err := tableDesc.FindIndexWithName("foo") + indexDesc, _, err := tableDesc.FindIndexByName("foo") if err != nil { t.Fatal(err) } - indexSpan := tableDesc.IndexSpan(keys.SystemSQLCodec, index.GetID()) + indexSpan := tableDesc.IndexSpan(keys.SystemSQLCodec, indexDesc.ID) tests.CheckKeyCount(t, kvDB, indexSpan, numRows) // Hack in zone configs for the primary and secondary indexes. (You need a CCL @@ -539,7 +539,7 @@ func TestDropIndexWithZoneConfigOSS(t *testing.T) { zoneConfig := zonepb.ZoneConfig{ Subzones: []zonepb.Subzone{ {IndexID: uint32(tableDesc.GetPrimaryIndexID()), Config: s.(*server.TestServer).Cfg.DefaultZoneConfig}, - {IndexID: uint32(index.GetID()), Config: s.(*server.TestServer).Cfg.DefaultZoneConfig}, + {IndexID: uint32(indexDesc.ID), Config: s.(*server.TestServer).Cfg.DefaultZoneConfig}, }, } zoneConfigBytes, err := protoutil.Marshal(&zoneConfig) @@ -564,7 +564,7 @@ func TestDropIndexWithZoneConfigOSS(t *testing.T) { // declares column families. tableDesc = catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "kv") - if _, err := tableDesc.FindIndexWithName("foo"); err == nil { + if _, _, err := tableDesc.FindIndexByName("foo"); err == nil { t.Fatalf("table descriptor still contains index after index is dropped") } } @@ -597,7 +597,7 @@ func TestDropIndexInterleaved(t *testing.T) { // Ensure that index is not active. tableDesc = catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "intlv") - if _, err := tableDesc.FindIndexWithName("intlv_idx"); err == nil { + if _, _, err := tableDesc.FindIndexByName("intlv_idx"); err == nil { t.Fatalf("table descriptor still contains index after index is dropped") } } diff --git a/pkg/sql/flowinfra/cluster_test.go b/pkg/sql/flowinfra/cluster_test.go index 99587f2fd52c..f001207c7b7d 100644 --- a/pkg/sql/flowinfra/cluster_test.go +++ b/pkg/sql/flowinfra/cluster_test.go @@ -71,7 +71,7 @@ func TestClusterFlow(t *testing.T) { desc := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "test", "t") makeIndexSpan := func(start, end int) execinfrapb.TableReaderSpan { var span roachpb.Span - prefix := roachpb.Key(rowenc.MakeIndexKeyPrefix(keys.SystemSQLCodec, desc, desc.PublicNonPrimaryIndexes()[0].GetID())) + prefix := roachpb.Key(rowenc.MakeIndexKeyPrefix(keys.SystemSQLCodec, desc, desc.GetPublicNonPrimaryIndexes()[0].ID)) span.Key = append(prefix, encoding.EncodeVarintAscending(nil, int64(start))...) span.EndKey = append(span.EndKey, prefix...) span.EndKey = append(span.EndKey, encoding.EncodeVarintAscending(nil, int64(end))...) diff --git a/pkg/sql/information_schema.go b/pkg/sql/information_schema.go index d4dfae34c806..1aabaf87d425 100644 --- a/pkg/sql/information_schema.go +++ b/pkg/sql/information_schema.go @@ -1297,7 +1297,7 @@ CREATE TABLE information_schema.statistics ( ) } - return catalog.ForEachIndex(table, catalog.IndexOpts{}, func(index catalog.Index) error { + return table.ForeachIndex(catalog.IndexOpts{}, func(index *descpb.IndexDescriptor, _ bool) error { // Columns in the primary key that aren't in index.ColumnNames or // index.StoreColumnNames are implicit columns in the index. var implicitCols map[string]struct{} @@ -1305,34 +1305,31 @@ CREATE TABLE information_schema.statistics ( if index.HasOldStoredColumns() { // Old STORING format: implicit columns are extra columns minus stored // columns. - hasImplicitCols = index.NumExtraColumns() > index.NumStoredColumns() + hasImplicitCols = len(index.ExtraColumnIDs) > len(index.StoreColumnNames) } else { // New STORING format: implicit columns are extra columns. - hasImplicitCols = index.NumExtraColumns() > 0 + hasImplicitCols = len(index.ExtraColumnIDs) > 0 } if hasImplicitCols { implicitCols = make(map[string]struct{}) - for i := 0; i < table.GetPrimaryIndex().NumColumns(); i++ { - col := table.GetPrimaryIndex().GetColumnName(i) + for _, col := range table.GetPrimaryIndex().ColumnNames { implicitCols[col] = struct{}{} } } sequence := 1 - for i := 0; i < index.NumColumns(); i++ { - col := index.GetColumnName(i) + for i, col := range index.ColumnNames { // We add a row for each column of index. - dir := dStringForIndexDirection(index.GetColumnDirection(i)) - if err := appendRow(index.IndexDesc(), col, sequence, dir, false, false); err != nil { + dir := dStringForIndexDirection(index.ColumnDirections[i]) + if err := appendRow(index, col, sequence, dir, false, false); err != nil { return err } sequence++ delete(implicitCols, col) } - for i := 0; i < index.NumStoredColumns(); i++ { - col := index.GetStoredColumnName(i) + for _, col := range index.StoreColumnNames { // We add a row for each stored column of index. - if err := appendRow(index.IndexDesc(), col, sequence, + if err := appendRow(index, col, sequence, indexDirectionNA, true, false); err != nil { return err } @@ -1346,11 +1343,10 @@ CREATE TABLE information_schema.statistics ( // // Note that simply iterating over implicitCols map // produces non-deterministic output. - for i := 0; i < table.GetPrimaryIndex().NumColumns(); i++ { - col := table.GetPrimaryIndex().GetColumnName(i) + for _, col := range table.GetPrimaryIndex().ColumnNames { if _, isImplicit := implicitCols[col]; isImplicit { // We add a row for each implicit column of index. - if err := appendRow(index.IndexDesc(), col, sequence, + if err := appendRow(index, col, sequence, indexDirectionAsc, false, true); err != nil { return err } diff --git a/pkg/sql/insert.go b/pkg/sql/insert.go index b4b7dc30e654..47b6521502b0 100644 --- a/pkg/sql/insert.go +++ b/pkg/sql/insert.go @@ -129,7 +129,8 @@ func (r *insertRun) processSourceRow(params runParams, rowVals tree.Datums) erro // written to when they are partial indexes and the row does not satisfy the // predicate. This set is passed as a parameter to tableInserter.row below. var pm row.PartialIndexUpdateHelper - if len(r.ti.tableDesc().PartialIndexes()) > 0 { + partialIndexOrds := r.ti.tableDesc().PartialIndexOrds() + if !partialIndexOrds.Empty() { partialIndexPutVals := rowVals[len(r.insertCols)+r.checkOrds.Len():] err := pm.Init(partialIndexPutVals, tree.Datums{}, r.ti.tableDesc()) diff --git a/pkg/sql/opt_catalog.go b/pkg/sql/opt_catalog.go index 2536937821ce..823bc26db9ad 100644 --- a/pkg/sql/opt_catalog.go +++ b/pkg/sql/opt_catalog.go @@ -629,9 +629,9 @@ func newOptTable( colDescs := ot.desc.DeletableColumns() numCols := len(colDescs) + len(colinfo.AllSystemColumnDescs) // One for each inverted index virtual column. - secondaryIndexes := ot.desc.DeletableNonPrimaryIndexes() - for _, index := range secondaryIndexes { - if index.GetType() == descpb.IndexDescriptor_INVERTED { + secondaryIndexes := ot.desc.DeletableIndexes() + for i := range secondaryIndexes { + if secondaryIndexes[i].Type == descpb.IndexDescriptor_INVERTED { numCols++ } } @@ -719,9 +719,9 @@ func newOptTable( for i := range ot.indexes { var idxDesc *descpb.IndexDescriptor if i == 0 { - idxDesc = desc.GetPrimaryIndex().IndexDesc() + idxDesc = desc.GetPrimaryIndex() } else { - idxDesc = secondaryIndexes[i-1].IndexDesc() + idxDesc = &secondaryIndexes[i-1] } // If there is a subzone that applies to the entire index, use that, @@ -989,19 +989,19 @@ func (ot *optTable) getColDesc(i int) *descpb.ColumnDescriptor { // IndexCount is part of the cat.Table interface. func (ot *optTable) IndexCount() int { // Primary index is always present, so count is always >= 1. - return len(ot.desc.ActiveIndexes()) + return 1 + len(ot.desc.GetPublicNonPrimaryIndexes()) } // WritableIndexCount is part of the cat.Table interface. func (ot *optTable) WritableIndexCount() int { // Primary index is always present, so count is always >= 1. - return 1 + len(ot.desc.WritableNonPrimaryIndexes()) + return 1 + len(ot.desc.WritableIndexes()) } // DeletableIndexCount is part of the cat.Table interface. func (ot *optTable) DeletableIndexCount() int { // Primary index is always present, so count is always >= 1. - return len(ot.desc.AllIndexes()) + return 1 + len(ot.desc.DeletableIndexes()) } // Index is part of the cat.Table interface. @@ -1121,7 +1121,7 @@ func (oi *optIndex) init( oi.zone = zone oi.indexOrdinal = indexOrdinal oi.invertedVirtualColOrd = invertedVirtualColOrd - if desc == tab.desc.GetPrimaryIndex().IndexDesc() { + if desc == tab.desc.GetPrimaryIndex() { // 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. @@ -1717,7 +1717,7 @@ func newOptVirtualTable( // Build the indexes (add 1 to account for lack of primary index in // indexes slice). - ot.indexes = make([]optVirtualIndex, len(ot.desc.ActiveIndexes())) + ot.indexes = make([]optVirtualIndex, 1+len(ot.desc.GetPublicNonPrimaryIndexes())) // Set up the primary index. ot.indexes[0] = optVirtualIndex{ tab: ot, @@ -1730,16 +1730,17 @@ func newOptVirtualTable( }, } - for _, idx := range ot.desc.PublicNonPrimaryIndexes() { - if idx.NumColumns() > 1 { + for i := range ot.desc.GetPublicNonPrimaryIndexes() { + idxDesc := &ot.desc.GetPublicNonPrimaryIndexes()[i] + if len(idxDesc.ColumnIDs) > 1 { panic(errors.AssertionFailedf("virtual indexes with more than 1 col not supported")) } // Add 1, since the 0th index will the primary that we added above. - ot.indexes[idx.Ordinal()] = optVirtualIndex{ + ot.indexes[i+1] = optVirtualIndex{ tab: ot, - desc: idx.IndexDesc(), - indexOrdinal: idx.Ordinal(), + desc: idxDesc, + indexOrdinal: i + 1, // The virtual indexes don't return the bogus PK key? numCols: ot.ColumnCount(), } @@ -1811,19 +1812,19 @@ func (ot *optVirtualTable) getColDesc(i int) *descpb.ColumnDescriptor { // IndexCount is part of the cat.Table interface. func (ot *optVirtualTable) IndexCount() int { // Primary index is always present, so count is always >= 1. - return len(ot.desc.ActiveIndexes()) + return 1 + len(ot.desc.GetPublicNonPrimaryIndexes()) } // WritableIndexCount is part of the cat.Table interface. func (ot *optVirtualTable) WritableIndexCount() int { // Primary index is always present, so count is always >= 1. - return 1 + len(ot.desc.WritableNonPrimaryIndexes()) + return 1 + len(ot.desc.WritableIndexes()) } // DeletableIndexCount is part of the cat.Table interface. func (ot *optVirtualTable) DeletableIndexCount() int { // Primary index is always present, so count is always >= 1. - return len(ot.desc.AllIndexes()) + return 1 + len(ot.desc.DeletableIndexes()) } // Index is part of the cat.Table interface. diff --git a/pkg/sql/opt_exec_factory.go b/pkg/sql/opt_exec_factory.go index e8d50074d62d..a3ce1f1e12f0 100644 --- a/pkg/sql/opt_exec_factory.go +++ b/pkg/sql/opt_exec_factory.go @@ -548,7 +548,7 @@ func (ef *execFactory) ConstructIndexJoin( } primaryIndex := tabDesc.GetPrimaryIndex() - tableScan.index = primaryIndex.IndexDesc() + tableScan.index = primaryIndex tableScan.disableBatchLimit() n := &indexJoinNode{ @@ -1562,7 +1562,7 @@ func (ef *execFactory) ConstructDeleteRange( autoCommit bool, ) (exec.Node, error) { tabDesc := table.(*optTable).desc - indexDesc := tabDesc.GetPrimaryIndex().IndexDesc() + indexDesc := tabDesc.GetPrimaryIndex() 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 da6a3d7b6a1d..f00219c80e4d 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.GetPrimaryIndex().IndexDesc() + primaryIndex := *tableDesc.GetPrimaryIndex() primaryIndex.Partitioning = descpb.PartitioningDescriptor{ NumColumns: 1, Range: []descpb.PartitioningDescriptor_Range{{ @@ -63,7 +63,7 @@ func TestRemovePartitioningOSS(t *testing.T) { } { - secondaryIndex := *tableDesc.PublicNonPrimaryIndexes()[0].IndexDesc() + secondaryIndex := tableDesc.GetPublicNonPrimaryIndexes()[0] secondaryIndex.Partitioning = descpb.PartitioningDescriptor{ NumColumns: 1, Range: []descpb.PartitioningDescriptor_Range{{ @@ -109,7 +109,7 @@ func TestRemovePartitioningOSS(t *testing.T) { Config: s.(*server.TestServer).Cfg.DefaultZoneConfig, }, { - IndexID: uint32(tableDesc.PublicNonPrimaryIndexes()[0].GetID()), + IndexID: uint32(tableDesc.GetPublicNonPrimaryIndexes()[0].ID), PartitionName: "p2", Config: s.(*server.TestServer).Cfg.DefaultZoneConfig, }, diff --git a/pkg/sql/partition_utils.go b/pkg/sql/partition_utils.go index f354a28c7e2b..87f21cb282cb 100644 --- a/pkg/sql/partition_utils.go +++ b/pkg/sql/partition_utils.go @@ -100,23 +100,23 @@ func GenerateSubzoneSpans( var indexCovering covering.Covering var partitionCoverings []covering.Covering - if err := catalog.ForEachIndex(tableDesc, catalog.IndexOpts{ + if err := tableDesc.ForeachIndex(catalog.IndexOpts{ AddMutations: true, - }, func(idx catalog.Index) error { - _, indexSubzoneExists := subzoneIndexByIndexID[idx.GetID()] + }, func(idxDesc *descpb.IndexDescriptor, _ bool) error { + _, indexSubzoneExists := subzoneIndexByIndexID[idxDesc.ID] if indexSubzoneExists { - idxSpan := tableDesc.IndexSpan(codec, idx.GetID()) + idxSpan := tableDesc.IndexSpan(codec, idxDesc.ID) // Each index starts with a unique prefix, so (from a precedence // perspective) it's safe to append them all together. indexCovering = append(indexCovering, covering.Range{ Start: idxSpan.Key, End: idxSpan.EndKey, - Payload: zonepb.Subzone{IndexID: uint32(idx.GetID())}, + Payload: zonepb.Subzone{IndexID: uint32(idxDesc.ID)}, }) } var emptyPrefix []tree.Datum indexPartitionCoverings, err := indexCoveringsForPartitioning( - a, codec, tableDesc, idx.IndexDesc(), &idx.IndexDesc().Partitioning, subzoneIndexByPartition, emptyPrefix) + a, codec, tableDesc, idxDesc, &idxDesc.Partitioning, subzoneIndexByPartition, emptyPrefix) if err != nil { return err } diff --git a/pkg/sql/pg_catalog.go b/pkg/sql/pg_catalog.go index 0e0e18dce763..d2a098285593 100644 --- a/pkg/sql/pg_catalog.go +++ b/pkg/sql/pg_catalog.go @@ -445,10 +445,9 @@ https://www.postgresql.org/docs/12/catalog-pg-attribute.html`, // Columns for each index. columnIdxMap := table.ColumnIdxMap() - return catalog.ForEachIndex(table, catalog.IndexOpts{}, func(index catalog.Index) error { - for i := 0; i < index.NumColumns(); i++ { - colID := index.GetColumnID(i) - idxID := h.IndexOid(table.GetID(), index.GetID()) + return table.ForeachIndex(catalog.IndexOpts{}, func(index *descpb.IndexDescriptor, _ bool) error { + for _, colID := range index.ColumnIDs { + idxID := h.IndexOid(table.GetID(), index.ID) column := table.GetColumnAtIdx(columnIdxMap.GetDefault(colID)) if err := addColumn(column, idxID, column.GetPGAttributeNum()); err != nil { return err @@ -625,31 +624,31 @@ https://www.postgresql.org/docs/9.5/catalog-pg-class.html`, } // Indexes. - return catalog.ForEachIndex(table, catalog.IndexOpts{}, func(index catalog.Index) error { + return table.ForeachIndex(catalog.IndexOpts{}, func(index *descpb.IndexDescriptor, _ bool) error { indexType := forwardIndexOid - if index.GetType() == descpb.IndexDescriptor_INVERTED { + if index.Type == descpb.IndexDescriptor_INVERTED { indexType = invertedIndexOid } return addRow( - h.IndexOid(table.GetID(), index.GetID()), // oid - tree.NewDName(index.GetName()), // relname - namespaceOid, // relnamespace - oidZero, // reltype - oidZero, // reloftype - getOwnerOID(table), // relowner - indexType, // relam - oidZero, // relfilenode - oidZero, // reltablespace - tree.DNull, // relpages - tree.DNull, // reltuples - zeroVal, // relallvisible - oidZero, // reltoastrelid - tree.DBoolFalse, // relhasindex - tree.DBoolFalse, // relisshared - relPersistencePermanent, // relPersistence - tree.DBoolFalse, // relistemp - relKindIndex, // relkind - tree.NewDInt(tree.DInt(index.NumColumns())), // relnatts + h.IndexOid(table.GetID(), index.ID), // oid + tree.NewDName(index.Name), // relname + namespaceOid, // relnamespace + oidZero, // reltype + oidZero, // reloftype + getOwnerOID(table), // relowner + indexType, // relam + oidZero, // relfilenode + oidZero, // reltablespace + tree.DNull, // relpages + tree.DNull, // reltuples + zeroVal, // relallvisible + oidZero, // reltoastrelid + tree.DBoolFalse, // relhasindex + tree.DBoolFalse, // relisshared + relPersistencePermanent, // relPersistence + tree.DBoolFalse, // relistemp + relKindIndex, // relkind + tree.NewDInt(tree.DInt(len(index.ColumnNames))), // relnatts zeroVal, // relchecks tree.DBoolFalse, // relhasoids tree.DBoolFalse, // relhaspkey @@ -1442,15 +1441,11 @@ https://www.postgresql.org/docs/9.5/catalog-pg-index.html`, return forEachTableDesc(ctx, p, dbContext, hideVirtual, /* virtual tables do not have indexes */ func(db *dbdesc.Immutable, scName string, table catalog.TableDescriptor) error { tableOid := tableOid(table.GetID()) - return catalog.ForEachIndex(table, catalog.IndexOpts{}, func(index catalog.Index) error { + return table.ForeachIndex(catalog.IndexOpts{}, func(index *descpb.IndexDescriptor, isPrimary bool) error { isMutation, isWriteOnly := - table.GetIndexMutationCapabilities(index.GetID()) + table.GetIndexMutationCapabilities(index.ID) isReady := isMutation && isWriteOnly - columnIDs := make([]descpb.ColumnID, index.NumColumns()) - for i := range columnIDs { - columnIDs[i] = index.GetColumnID(i) - } - indkey, err := colIDArrayToVector(columnIDs) + indkey, err := colIDArrayToVector(index.ColumnIDs) if err != nil { return err } @@ -1460,7 +1455,7 @@ https://www.postgresql.org/docs/9.5/catalog-pg-index.html`, // is ASC/DESC and if nulls appear first/last. collationOids := tree.NewDArray(types.Oid) indoption := tree.NewDArray(types.Int) - for i, columnID := range columnIDs { + for i, columnID := range index.ColumnIDs { col, err := table.FindColumnByID(columnID) if err != nil { return err @@ -1471,7 +1466,7 @@ https://www.postgresql.org/docs/9.5/catalog-pg-index.html`, // Currently, nulls always appear first if the order is ascending, // and always appear last if the order is descending. var thisIndOption tree.DInt - if index.GetColumnDirection(i) == descpb.IndexDescriptor_ASC { + if index.ColumnDirections[i] == descpb.IndexDescriptor_ASC { thisIndOption = indoptionNullsFirst } else { thisIndOption = indoptionDesc @@ -1484,30 +1479,30 @@ https://www.postgresql.org/docs/9.5/catalog-pg-index.html`, indoptionIntVector := tree.NewDIntVectorFromDArray(indoption) // TODO(bram): #27763 indclass still needs to be populated but it // requires pg_catalog.pg_opclass first. - indclass, err := makeZeroedOidVector(index.NumColumns()) + indclass, err := makeZeroedOidVector(len(index.ColumnIDs)) if err != nil { return err } return addRow( - h.IndexOid(table.GetID(), index.GetID()), // indexrelid - tableOid, // indrelid - tree.NewDInt(tree.DInt(index.NumColumns())), // indnatts - tree.MakeDBool(tree.DBool(index.IsUnique())), // indisunique - tree.MakeDBool(tree.DBool(index.Primary())), // indisprimary - tree.DBoolFalse, // indisexclusion - tree.MakeDBool(tree.DBool(index.IsUnique())), // indimmediate - tree.DBoolFalse, // indisclustered - tree.MakeDBool(tree.DBool(!isMutation)), // indisvalid - tree.DBoolFalse, // indcheckxmin - tree.MakeDBool(tree.DBool(isReady)), // indisready - tree.DBoolTrue, // indislive - tree.DBoolFalse, // indisreplident - indkey, // indkey - collationOidVector, // indcollation - indclass, // indclass - indoptionIntVector, // indoption - tree.DNull, // indexprs - tree.DNull, // indpred + h.IndexOid(table.GetID(), index.ID), // indexrelid + tableOid, // indrelid + tree.NewDInt(tree.DInt(len(index.ColumnNames))), // indnatts + tree.MakeDBool(tree.DBool(index.Unique)), // indisunique + tree.MakeDBool(tree.DBool(isPrimary)), // indisprimary + tree.DBoolFalse, // indisexclusion + tree.MakeDBool(tree.DBool(index.Unique)), // indimmediate + tree.DBoolFalse, // indisclustered + tree.MakeDBool(tree.DBool(!isMutation)), // indisvalid + tree.DBoolFalse, // indcheckxmin + tree.MakeDBool(tree.DBool(isReady)), // indisready + tree.DBoolTrue, // indislive + tree.DBoolFalse, // indisreplident + indkey, // indkey + collationOidVector, // indcollation + indclass, // indclass + indoptionIntVector, // indoption + tree.DNull, // indexprs + tree.DNull, // indpred ) }) }) @@ -1524,18 +1519,18 @@ https://www.postgresql.org/docs/9.5/view-pg-indexes.html`, func(db *dbdesc.Immutable, scName string, table catalog.TableDescriptor, tableLookup tableLookupFn) error { scNameName := tree.NewDName(scName) tblName := tree.NewDName(table.GetName()) - return catalog.ForEachIndex(table, catalog.IndexOpts{}, func(index catalog.Index) error { - def, err := indexDefFromDescriptor(ctx, p, db, table, index.IndexDesc(), tableLookup) + return table.ForeachIndex(catalog.IndexOpts{}, func(index *descpb.IndexDescriptor, _ bool) error { + def, err := indexDefFromDescriptor(ctx, p, db, table, index, tableLookup) if err != nil { return err } return addRow( - h.IndexOid(table.GetID(), index.GetID()), // oid - scNameName, // schemaname - tblName, // tablename - tree.NewDName(index.GetName()), // indexname - tree.DNull, // tablespace - tree.NewDString(def), // indexdef + h.IndexOid(table.GetID(), index.ID), // oid + scNameName, // schemaname + tblName, // tablename + tree.NewDName(index.Name), // indexname + tree.DNull, // tablespace + tree.NewDString(def), // indexdef ) }) }) diff --git a/pkg/sql/pgwire/testdata/pgtest/notice b/pkg/sql/pgwire/testdata/pgtest/notice index 648ad7f6a688..342927ff44b1 100644 --- a/pkg/sql/pgwire/testdata/pgtest/notice +++ b/pkg/sql/pgwire/testdata/pgtest/notice @@ -55,7 +55,7 @@ Query {"String": "DROP INDEX t_x_idx"} until crdb_only CommandComplete ---- -{"Severity":"NOTICE","Code":"00000","Message":"the data for dropped indexes is reclaimed asynchronously","Detail":"","Hint":"The reclamation delay can be customized in the zone configuration for the table.","Position":0,"InternalPosition":0,"InternalQuery":"","Where":"","SchemaName":"","TableName":"","ColumnName":"","DataTypeName":"","ConstraintName":"","File":"drop_index.go","Line":528,"Routine":"dropIndexByName","UnknownFields":null} +{"Severity":"NOTICE","Code":"00000","Message":"the data for dropped indexes is reclaimed asynchronously","Detail":"","Hint":"The reclamation delay can be customized in the zone configuration for the table.","Position":0,"InternalPosition":0,"InternalQuery":"","Where":"","SchemaName":"","TableName":"","ColumnName":"","DataTypeName":"","ConstraintName":"","File":"drop_index.go","Line":531,"Routine":"dropIndexByName","UnknownFields":null} {"Type":"CommandComplete","CommandTag":"DROP INDEX"} until noncrdb_only diff --git a/pkg/sql/physicalplan/fake_span_resolver_test.go b/pkg/sql/physicalplan/fake_span_resolver_test.go index 8f90519a72f5..016902f142eb 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.GetPrimaryIndex().IndexDesc()) + primIdxValDirs := catalogkeys.IndexKeyValDirs(tableDesc.GetPrimaryIndex()) span := tableDesc.PrimaryIndexSpan(keys.SystemSQLCodec) diff --git a/pkg/sql/physicalplan/span_resolver_test.go b/pkg/sql/physicalplan/span_resolver_test.go index 90ff68eb6204..69d38ad76a2e 100644 --- a/pkg/sql/physicalplan/span_resolver_test.go +++ b/pkg/sql/physicalplan/span_resolver_test.go @@ -169,7 +169,7 @@ func populateCache(db *gosql.DB, expectedNumRows int) error { func splitRangeAtVal( ts *server.TestServer, tableDesc *tabledesc.Immutable, pk int, ) (roachpb.RangeDescriptor, roachpb.RangeDescriptor, error) { - if len(tableDesc.PublicNonPrimaryIndexes()) != 0 { + if len(tableDesc.GetPublicNonPrimaryIndexes()) != 0 { return roachpb.RangeDescriptor{}, roachpb.RangeDescriptor{}, errors.AssertionFailedf("expected table with just a PK, got: %+v", tableDesc) } diff --git a/pkg/sql/refresh_materialized_view.go b/pkg/sql/refresh_materialized_view.go index b0f3b55e459c..692ddde82121 100644 --- a/pkg/sql/refresh_materialized_view.go +++ b/pkg/sql/refresh_materialized_view.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" ) type refreshMaterializedViewNode struct { @@ -67,10 +68,10 @@ func (n *refreshMaterializedViewNode) startExec(params runParams) error { } // Prepare the new set of indexes by cloning all existing indexes on the view. - newPrimaryIndex := n.desc.GetPrimaryIndex().IndexDescDeepCopy() - newIndexes := make([]descpb.IndexDescriptor, len(n.desc.PublicNonPrimaryIndexes())) - for i, idx := range n.desc.PublicNonPrimaryIndexes() { - newIndexes[i] = idx.IndexDescDeepCopy() + newPrimaryIndex := protoutil.Clone(n.desc.GetPrimaryIndex()).(*descpb.IndexDescriptor) + newIndexes := make([]descpb.IndexDescriptor, len(n.desc.GetPublicNonPrimaryIndexes())) + for i := range n.desc.GetPublicNonPrimaryIndexes() { + newIndexes[i] = *protoutil.Clone(&n.desc.GetPublicNonPrimaryIndexes()[i]).(*descpb.IndexDescriptor) } // Reset and allocate new IDs for the new indexes. @@ -86,7 +87,7 @@ func (n *refreshMaterializedViewNode) startExec(params runParams) error { // Queue the refresh mutation. n.desc.AddMaterializedViewRefreshMutation(&descpb.MaterializedViewRefresh{ - NewPrimaryIndex: newPrimaryIndex, + NewPrimaryIndex: *newPrimaryIndex, NewIndexes: newIndexes, AsOf: params.p.Txn().ReadTimestamp(), ShouldBackfill: n.n.RefreshDataOption != tree.RefreshDataClear, diff --git a/pkg/sql/rename_column.go b/pkg/sql/rename_column.go index 0d39b3a240ba..f50bd0b9605e 100644 --- a/pkg/sql/rename_column.go +++ b/pkg/sql/rename_column.go @@ -178,15 +178,13 @@ func (p *planner) renameColumn( } // Rename the column in partial index predicates. - for _, index := range tableDesc.PublicNonPrimaryIndexes() { - if index.IsPartial() { - newExpr, err := schemaexpr.RenameColumn(index.GetPredicate(), *oldName, *newName) + for i := range tableDesc.GetPublicNonPrimaryIndexes() { + if index := &tableDesc.GetPublicNonPrimaryIndexes()[i]; index.IsPartial() { + newExpr, err := schemaexpr.RenameColumn(index.Predicate, *oldName, *newName) if err != nil { return false, err } - indexDesc := *index.IndexDesc() - indexDesc.Predicate = newExpr - tableDesc.SetPublicNonPrimaryIndex(index.Ordinal(), indexDesc) + index.Predicate = newExpr } } @@ -218,8 +216,8 @@ func (p *planner) renameColumn( // Keep the shardedDesc name in sync with the column name. shardedDesc.Name = string(newName) } - for _, idx := range tableDesc.NonDropIndexes() { - maybeUpdateShardedDesc(&idx.IndexDesc().Sharded) + for _, idx := range tableDesc.AllNonDropIndexes() { + maybeUpdateShardedDesc(&idx.Sharded) } // Rename the column in the indexes. diff --git a/pkg/sql/rename_index.go b/pkg/sql/rename_index.go index e4d1de5993cd..baff4d82f67d 100644 --- a/pkg/sql/rename_index.go +++ b/pkg/sql/rename_index.go @@ -52,7 +52,7 @@ func (p *planner) RenameIndex(ctx context.Context, n *tree.RenameIndex) (planNod return newZeroNode(nil /* columns */), nil } - idx, err := tableDesc.FindIndexWithName(string(n.Index.Index)) + idx, _, err := tableDesc.FindIndexByName(string(n.Index.Index)) if err != nil { if n.IfExists { // Noop. @@ -66,7 +66,7 @@ func (p *planner) RenameIndex(ctx context.Context, n *tree.RenameIndex) (planNod return nil, err } - return &renameIndexNode{n: n, idx: idx.IndexDesc(), tableDesc: tableDesc}, nil + return &renameIndexNode{n: n, idx: idx, tableDesc: tableDesc}, nil } // ReadingOwnWrites implements the planNodeReadingOwnWrites interface. @@ -98,7 +98,7 @@ func (n *renameIndexNode) startExec(params runParams) error { return nil } - if _, err := tableDesc.FindIndexWithName(string(n.n.NewName)); err == nil { + if _, _, err := tableDesc.FindIndexByName(string(n.n.NewName)); err == nil { return pgerror.Newf(pgcode.DuplicateRelation, "index name %q already exists", string(n.n.NewName)) } diff --git a/pkg/sql/resolver.go b/pkg/sql/resolver.go index c8a2238c5d31..164d0ce09bb7 100644 --- a/pkg/sql/resolver.go +++ b/pkg/sql/resolver.go @@ -461,8 +461,8 @@ func findTableContainingIndex( continue } - idx, err := tableDesc.FindIndexWithName(string(idxName)) - if err != nil || idx.Dropped() { + _, dropped, err := tableDesc.FindIndexByName(string(idxName)) + if err != nil || dropped { // err is nil if the index does not exist on the table. continue } diff --git a/pkg/sql/revert.go b/pkg/sql/revert.go index 8c11a550c58b..714c1b10ddbe 100644 --- a/pkg/sql/revert.go +++ b/pkg/sql/revert.go @@ -54,15 +54,13 @@ func RevertTables( if !tables[i].IsPhysicalTable() { return errors.Errorf("cannot revert virtual table %s", tables[i].Name) } - for _, idx := range tables[i].NonDropIndexes() { - for j := 0; j < idx.NumInterleaveAncestors(); j++ { - parent := idx.GetInterleaveAncestor(j) + for _, idx := range tables[i].AllNonDropIndexes() { + for _, parent := range idx.Interleave.Ancestors { if !reverting[parent.TableID] { return errors.New("cannot revert table without reverting all interleaved tables and indexes") } } - for j := 0; j < idx.NumInterleavedBy(); j++ { - child := idx.GetInterleavedBy(j) + for _, child := range idx.InterleavedBy { if !reverting[child.Table] { return errors.New("cannot revert table without reverting all interleaved tables and indexes") } diff --git a/pkg/sql/row/deleter.go b/pkg/sql/row/deleter.go index 4ebe2e29232b..039872d08202 100644 --- a/pkg/sql/row/deleter.go +++ b/pkg/sql/row/deleter.go @@ -44,11 +44,7 @@ type Deleter struct { func MakeDeleter( codec keys.SQLCodec, tableDesc *tabledesc.Immutable, requestedCols []descpb.ColumnDescriptor, ) Deleter { - indexes := tableDesc.DeletableNonPrimaryIndexes() - indexDescs := make([]descpb.IndexDescriptor, len(indexes)) - for i, index := range indexes { - indexDescs[i] = *index.IndexDesc() - } + indexes := tableDesc.DeletableIndexes() var fetchCols []descpb.ColumnDescriptor var fetchColIDtoRowIndex catalog.TableColMap @@ -67,22 +63,19 @@ func MakeDeleter( } return nil } - for j := 0; j < tableDesc.GetPrimaryIndex().NumColumns(); j++ { - colID := tableDesc.GetPrimaryIndex().GetColumnID(j) + for _, colID := range tableDesc.GetPrimaryIndex().ColumnIDs { if err := maybeAddCol(colID); err != nil { return Deleter{} } } for _, index := range indexes { - for j := 0; j < index.NumColumns(); j++ { - colID := index.GetColumnID(j) + for _, colID := range index.ColumnIDs { if err := maybeAddCol(colID); err != nil { return Deleter{} } } // The extra columns are needed to fix #14601. - for j := 0; j < index.NumExtraColumns(); j++ { - colID := index.GetExtraColumnID(j) + for _, colID := range index.ExtraColumnIDs { if err := maybeAddCol(colID); err != nil { return Deleter{} } @@ -91,7 +84,7 @@ func MakeDeleter( } rd := Deleter{ - Helper: newRowHelper(codec, tableDesc, indexDescs), + Helper: newRowHelper(codec, tableDesc, indexes), FetchCols: fetchCols, FetchColIDtoRowIndex: fetchColIDtoRowIndex, } diff --git a/pkg/sql/row/errors.go b/pkg/sql/row/errors.go index fbe3ff3a2f22..4303f1caebc3 100644 --- a/pkg/sql/row/errors.go +++ b/pkg/sql/row/errors.go @@ -154,32 +154,25 @@ func DecodeRowInfo( if err != nil { return nil, nil, nil, err } - index, err := tableDesc.FindIndexWithID(indexID) + index, err := tableDesc.FindIndexByID(indexID) if err != nil { return nil, nil, nil, err } var rf Fetcher - var colIDs []descpb.ColumnID - if !allColumns { - colIDs = make([]descpb.ColumnID, index.NumColumns()) - for i := range colIDs { - colIDs[i] = index.GetColumnID(i) + colIDs := index.ColumnIDs + if allColumns { + if index.ID == tableDesc.GetPrimaryIndexID() { + publicColumns := tableDesc.GetPublicColumns() + colIDs = make([]descpb.ColumnID, len(publicColumns)) + for i := range publicColumns { + colIDs[i] = publicColumns[i].ID + } + } else { + colIDs, _ = index.FullColumnIDs() + colIDs = append(colIDs, index.StoreColumnIDs...) } - } else if index.Primary() { - publicColumns := tableDesc.GetPublicColumns() - colIDs = make([]descpb.ColumnID, len(publicColumns)) - for i := range publicColumns { - colIDs[i] = publicColumns[i].ID - } - } else { - colIDs = make([]descpb.ColumnID, 0, index.NumColumns()+index.NumExtraColumns()+index.NumStoredColumns()) - _ = index.ForEachColumnID(func(id descpb.ColumnID) error { - colIDs = append(colIDs, id) - return nil - }) } - var valNeededForCol util.FastIntSet valNeededForCol.AddRange(0, len(colIDs)-1) @@ -196,7 +189,7 @@ func DecodeRowInfo( tableArgs := FetcherTableArgs{ Desc: tableDesc, - Index: index.IndexDesc(), + Index: index, ColIdxMap: colIdxMap, IsSecondaryIndex: indexID != tableDesc.GetPrimaryIndexID(), Cols: cols, @@ -239,7 +232,7 @@ func DecodeRowInfo( } values[i] = datums[i].String() } - return index.IndexDesc(), names, values, nil + return index, names, values, nil } func (f *singleKVFetcher) close(context.Context) {} diff --git a/pkg/sql/row/fetcher.go b/pkg/sql/row/fetcher.go index 57cea21ff34d..f196d5fd99cd 100644 --- a/pkg/sql/row/fetcher.go +++ b/pkg/sql/row/fetcher.go @@ -1418,12 +1418,7 @@ func (rf *Fetcher) checkPrimaryIndexDatumEncodings(ctx context.Context) error { return nil }) - indexes := make([]descpb.IndexDescriptor, len(table.desc.PublicNonPrimaryIndexes())) - for i, idx := range table.desc.PublicNonPrimaryIndexes() { - indexes[i] = *idx.IndexDesc() - } - - rh := rowHelper{TableDesc: table.desc, Indexes: indexes} + rh := rowHelper{TableDesc: table.desc, Indexes: table.desc.GetPublicNonPrimaryIndexes()} return table.desc.ForeachFamily(func(family *descpb.ColumnFamilyDescriptor) error { var lastColID descpb.ColumnID diff --git a/pkg/sql/row/fetcher_mvcc_test.go b/pkg/sql/row/fetcher_mvcc_test.go index 5ff5d707f7ec..cd932a7eb0a9 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.GetPrimaryIndex().IndexDesc(), + Index: desc.GetPrimaryIndex(), ColIdxMap: colIdxMap, IsSecondaryIndex: false, Cols: desc.Columns, diff --git a/pkg/sql/row/fetcher_test.go b/pkg/sql/row/fetcher_test.go index 86245eb097ae..a911085d33e8 100644 --- a/pkg/sql/row/fetcher_test.go +++ b/pkg/sql/row/fetcher_test.go @@ -49,13 +49,22 @@ func makeFetcherArgs(entries []initFetcherArgs) []FetcherTableArgs { fetcherArgs := make([]FetcherTableArgs, len(entries)) for i, entry := range entries { - index := entry.tableDesc.ActiveIndexes()[entry.indexIdx] + var index *descpb.IndexDescriptor + var isSecondaryIndex bool + + if entry.indexIdx > 0 { + index = &entry.tableDesc.GetPublicNonPrimaryIndexes()[entry.indexIdx-1] + isSecondaryIndex = true + } else { + index = entry.tableDesc.GetPrimaryIndex() + } + fetcherArgs[i] = FetcherTableArgs{ Spans: entry.spans, Desc: entry.tableDesc, - Index: index.IndexDesc(), + Index: index, ColIdxMap: entry.tableDesc.ColumnIdxMap(), - IsSecondaryIndex: !index.Primary(), + IsSecondaryIndex: isSecondaryIndex, Cols: entry.tableDesc.Columns, ValNeededForCol: entry.valNeededForCol, } @@ -194,7 +203,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.GetPrimaryIndex().GetName(), + tableDesc.Name, tableDesc.GetPrimaryIndex().Name, desc.GetName(), index.Name, ) } @@ -314,7 +323,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.GetPrimaryIndex().GetName(), + tableDesc.Name, tableDesc.GetPrimaryIndex().Name, desc.GetName(), index.Name, ) } @@ -646,7 +655,7 @@ func TestNextRowSecondaryIndex(t *testing.T) { if err := rf.StartScan( context.Background(), kv.NewTxn(ctx, kvDB, 0), - roachpb.Spans{tableDesc.IndexSpan(keys.SystemSQLCodec, tableDesc.PublicNonPrimaryIndexes()[0].GetID())}, + roachpb.Spans{tableDesc.IndexSpan(keys.SystemSQLCodec, tableDesc.GetPublicNonPrimaryIndexes()[0].ID)}, false, /*limitBatches*/ 0, /*limitHint*/ false, /*traceKV*/ @@ -668,10 +677,10 @@ func TestNextRowSecondaryIndex(t *testing.T) { count++ - if desc.GetID() != tableDesc.ID || index.ID != tableDesc.PublicNonPrimaryIndexes()[0].GetID() { + if desc.GetID() != tableDesc.ID || index.ID != tableDesc.GetPublicNonPrimaryIndexes()[0].ID { t.Fatalf( "unexpected row retrieved from fetcher.\nnexpected: table %s - index %s\nactual: table %s - index %s", - tableDesc.Name, tableDesc.PublicNonPrimaryIndexes()[0].GetName(), + tableDesc.Name, tableDesc.GetPublicNonPrimaryIndexes()[0].Name, desc.GetName(), index.Name, ) } @@ -978,7 +987,12 @@ func TestNextRowInterleaved(t *testing.T) { idLookups := make(map[uint64]*fetcherEntryArgs, len(entries)) for i, entry := range entries { tableDesc := catalogkv.TestingGetImmutableTableDescriptor(kvDB, keys.SystemSQLCodec, sqlutils.TestDB, entry.tableName) - indexID := tableDesc.ActiveIndexes()[entry.indexIdx].GetID() + var indexID descpb.IndexID + if entry.indexIdx == 0 { + indexID = tableDesc.GetPrimaryIndexID() + } else { + indexID = tableDesc.GetPublicNonPrimaryIndexes()[entry.indexIdx-1].ID + } idLookups[idLookupKey(tableDesc.ID, indexID)] = entry // We take every entry's index span (primary or diff --git a/pkg/sql/row/helper.go b/pkg/sql/row/helper.go index 8a83c055455b..b9a40fc5dc9a 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.GetPrimaryIndex().IndexDesc()) + rh.primIndexValDirs = catalogkeys.IndexKeyValDirs(rh.TableDesc.GetPrimaryIndex()) 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.GetPrimaryIndex().IndexDesc(), colIDtoRowIndex, values, rh.primaryIndexKeyPrefix) + rh.TableDesc, rh.TableDesc.GetPrimaryIndex(), colIDtoRowIndex, values, rh.primaryIndexKeyPrefix) return primaryIndexKey, err } @@ -137,9 +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.GetPrimaryIndex().NumColumns(); i++ { - pkColID := rh.TableDesc.GetPrimaryIndex().GetColumnID(i) - rh.primaryIndexCols.Add(pkColID) + for _, colID := range rh.TableDesc.GetPrimaryIndex().ColumnIDs { + rh.primaryIndexCols.Add(colID) } } if !rh.primaryIndexCols.Contains(colID) { diff --git a/pkg/sql/row/inserter.go b/pkg/sql/row/inserter.go index 00a9cf11e5f1..695d49a3cefa 100644 --- a/pkg/sql/row/inserter.go +++ b/pkg/sql/row/inserter.go @@ -50,23 +50,16 @@ func MakeInserter( insertCols []descpb.ColumnDescriptor, alloc *rowenc.DatumAlloc, ) (Inserter, error) { - writableIndexes := tableDesc.WritableNonPrimaryIndexes() - writableIndexDescs := make([]descpb.IndexDescriptor, len(writableIndexes)) - for i, index := range writableIndexes { - writableIndexDescs[i] = *index.IndexDesc() - } - ri := Inserter{ - Helper: newRowHelper(codec, tableDesc, writableIndexDescs), + Helper: newRowHelper(codec, tableDesc, tableDesc.WritableIndexes()), InsertCols: insertCols, InsertColIDtoRowIndex: ColIDtoRowIndexFromCols(insertCols), marshaled: make([]roachpb.Value, len(insertCols)), } - 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.GetPrimaryIndex().GetColumnName(i)) + for i, col := range tableDesc.GetPrimaryIndex().ColumnIDs { + if _, ok := ri.InsertColIDtoRowIndex.Get(col); !ok { + return Inserter{}, fmt.Errorf("missing %q primary key column", tableDesc.GetPrimaryIndex().ColumnNames[i]) } } diff --git a/pkg/sql/row/partial_index.go b/pkg/sql/row/partial_index.go index 597046e7e6d1..8b57e193b7de 100644 --- a/pkg/sql/row/partial_index.go +++ b/pkg/sql/row/partial_index.go @@ -44,38 +44,43 @@ func (pm *PartialIndexUpdateHelper) Init( partialIndexPutVals tree.Datums, partialIndexDelVals tree.Datums, tabDesc catalog.TableDescriptor, ) error { colIdx := 0 + partialIndexOrds := tabDesc.PartialIndexOrds() + indexes := tabDesc.DeletableIndexes() - for _, idx := range tabDesc.PartialIndexes() { + for i, ok := partialIndexOrds.Next(0); ok; i, ok = partialIndexOrds.Next(i + 1) { + index := &indexes[i] + if index.IsPartial() { - // Check the boolean partial index put column, if it exists. - if colIdx < len(partialIndexPutVals) { - val, err := tree.GetBool(partialIndexPutVals[colIdx]) - if err != nil { - return err + // Check the boolean partial index put column, if it exists. + if colIdx < len(partialIndexPutVals) { + val, err := tree.GetBool(partialIndexPutVals[colIdx]) + if err != nil { + return err + } + if !val { + // If the value of the column for the index predicate + // expression is false, the row should not be added to the + // partial index. + pm.IgnoreForPut.Add(int(index.ID)) + } } - if !val { - // If the value of the column for the index predicate - // expression is false, the row should not be added to the - // partial index. - pm.IgnoreForPut.Add(int(idx.GetID())) - } - } - // Check the boolean partial index del column, if it exists. - if colIdx < len(partialIndexDelVals) { - val, err := tree.GetBool(partialIndexDelVals[colIdx]) - if err != nil { - return err - } - if !val { - // If the value of the column for the index predicate - // expression is false, the row should not be removed from - // the partial index. - pm.IgnoreForDel.Add(int(idx.GetID())) + // Check the boolean partial index del column, if it exists. + if colIdx < len(partialIndexDelVals) { + val, err := tree.GetBool(partialIndexDelVals[colIdx]) + if err != nil { + return err + } + if !val { + // If the value of the column for the index predicate + // expression is false, the row should not be removed from + // the partial index. + pm.IgnoreForDel.Add(int(index.ID)) + } } - } - colIdx++ + colIdx++ + } } return nil diff --git a/pkg/sql/row/row_converter.go b/pkg/sql/row/row_converter.go index f23d2b7729b2..1096a1cf3e1b 100644 --- a/pkg/sql/row/row_converter.go +++ b/pkg/sql/row/row_converter.go @@ -414,7 +414,7 @@ func NewDatumRowConverter( return nil, errors.New("unexpected hidden column") } - padding := 2 * (len(tableDesc.PublicNonPrimaryIndexes()) + len(tableDesc.Families)) + padding := 2 * (len(tableDesc.GetPublicNonPrimaryIndexes()) + len(tableDesc.Families)) c.BatchCap = kvDatumRowConverterBatchSize + padding c.KvBatch.KVs = make([]roachpb.KeyValue, 0, c.BatchCap) diff --git a/pkg/sql/row/updater.go b/pkg/sql/row/updater.go index f37b59ffd6c3..ebb451dce3d9 100644 --- a/pkg/sql/row/updater.go +++ b/pkg/sql/row/updater.go @@ -98,8 +98,7 @@ func MakeUpdater( updateColIDtoRowIndex := ColIDtoRowIndexFromCols(updateCols) var primaryIndexCols catalog.TableColSet - for i := 0; i < tableDesc.GetPrimaryIndex().NumColumns(); i++ { - colID := tableDesc.GetPrimaryIndex().GetColumnID(i) + for _, colID := range tableDesc.GetPrimaryIndex().ColumnIDs { primaryIndexCols.Add(colID) } @@ -113,7 +112,7 @@ func MakeUpdater( // needsUpdate returns true if the given index may need to be updated for // the current UPDATE mutation. - needsUpdate := func(index catalog.Index) bool { + needsUpdate := func(index descpb.IndexDescriptor) bool { // If the UPDATE is set to only update columns and not secondary // indexes, return false. if updateType == UpdaterOnlyColumns { @@ -134,7 +133,7 @@ func MakeUpdater( if index.IsPartial() { return true } - return index.ForEachColumnID(func(id descpb.ColumnID) error { + return index.RunOverAllColumns(func(id descpb.ColumnID) error { if _, ok := updateColIDtoRowIndex.Get(id); ok { return returnTruePseudoError } @@ -142,20 +141,22 @@ func MakeUpdater( }) != nil } - includeIndexes := make([]descpb.IndexDescriptor, 0, len(tableDesc.WritableNonPrimaryIndexes())) - var deleteOnlyIndexes []descpb.IndexDescriptor - for _, index := range tableDesc.DeletableNonPrimaryIndexes() { - if !needsUpdate(index) { - continue + writableIndexes := tableDesc.WritableIndexes() + includeIndexes := make([]descpb.IndexDescriptor, 0, len(writableIndexes)) + for _, index := range writableIndexes { + if needsUpdate(index) { + includeIndexes = append(includeIndexes, index) } - if !index.DeleteOnly() { - includeIndexes = append(includeIndexes, *index.IndexDesc()) - } else { + } + + var deleteOnlyIndexes []descpb.IndexDescriptor + for _, idx := range tableDesc.DeleteOnlyIndexes() { + if needsUpdate(idx) { if deleteOnlyIndexes == nil { // Allocate at most once. - deleteOnlyIndexes = make([]descpb.IndexDescriptor, 0, len(tableDesc.DeleteOnlyNonPrimaryIndexes())) + deleteOnlyIndexes = make([]descpb.IndexDescriptor, 0, len(tableDesc.DeleteOnlyIndexes())) } - deleteOnlyIndexes = append(deleteOnlyIndexes, *index.IndexDesc()) + deleteOnlyIndexes = append(deleteOnlyIndexes, idx) } } diff --git a/pkg/sql/rowenc/client_index_encoding_test.go b/pkg/sql/rowenc/client_index_encoding_test.go index 81baed783953..7a1e207f6408 100644 --- a/pkg/sql/rowenc/client_index_encoding_test.go +++ b/pkg/sql/rowenc/client_index_encoding_test.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" @@ -56,14 +57,14 @@ func TestAdjustStartKeyForInterleave(t *testing.T) { child := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, sqlutils.TestDB, "child1") grandchild := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, sqlutils.TestDB, "grandchild1") - parentDescIdx := parent.PublicNonPrimaryIndexes()[0] - childDescIdx := child.PublicNonPrimaryIndexes()[0] - childNonUniqueIdx := child.PublicNonPrimaryIndexes()[1] - childUniqueIdx := child.PublicNonPrimaryIndexes()[2] - grandchildDescIdx := grandchild.PublicNonPrimaryIndexes()[0] + parentDescIdx := parent.GetPublicNonPrimaryIndexes()[0] + childDescIdx := child.GetPublicNonPrimaryIndexes()[0] + childNonUniqueIdx := child.GetPublicNonPrimaryIndexes()[1] + childUniqueIdx := child.GetPublicNonPrimaryIndexes()[2] + grandchildDescIdx := grandchild.GetPublicNonPrimaryIndexes()[0] testCases := []struct { - index catalog.Index + index *descpb.IndexDescriptor // See ShortToLongKeyFmt for how to represent a key. input string expected string @@ -107,23 +108,23 @@ func TestAdjustStartKeyForInterleave(t *testing.T) { // See encodeStartConstraintDescending. { - index: parentDescIdx, + index: &parentDescIdx, input: "/NULLDESC", expected: "/NULLDESC", }, { - index: childDescIdx, + index: &childDescIdx, input: "/1/#/2/NULLDESC", expected: "/1/#/2/NULLDESC", }, { - index: grandchildDescIdx, + index: &grandchildDescIdx, input: "/1/#/2/3/#/NULLDESC", expected: "/1/#/2/3/#/NULLDESC", }, { - index: childDescIdx, + index: &childDescIdx, input: "/1/#/NULLDESC", expected: "/1/#/NULLDESC", }, @@ -177,22 +178,22 @@ func TestAdjustStartKeyForInterleave(t *testing.T) { // We should expect two extra columns (in addition to the // two index columns). { - index: childNonUniqueIdx, + index: &childNonUniqueIdx, input: "/2/3", expected: "/2/3", }, { - index: childNonUniqueIdx, + index: &childNonUniqueIdx, input: "/2/3/4", expected: "/2/3/4", }, { - index: childNonUniqueIdx, + index: &childNonUniqueIdx, input: "/2/3/4/5", expected: "/2/3/4/5", }, { - index: childNonUniqueIdx, + index: &childNonUniqueIdx, input: "/2/3/4/5/#/10", expected: "/2/3/4/6", }, @@ -200,27 +201,27 @@ func TestAdjustStartKeyForInterleave(t *testing.T) { // Unique indexes only include implicit columns if they have // a NULL value. { - index: childUniqueIdx, + index: &childUniqueIdx, input: "/2/3", expected: "/2/3", }, { - index: childUniqueIdx, + index: &childUniqueIdx, input: "/2/3/4", expected: "/2/4", }, { - index: childUniqueIdx, + index: &childUniqueIdx, input: "/2/NULLASC/4", expected: "/2/NULLASC/4", }, { - index: childUniqueIdx, + index: &childUniqueIdx, input: "/2/NULLASC/4/5", expected: "/2/NULLASC/4/5", }, { - index: childUniqueIdx, + index: &childUniqueIdx, input: "/2/NULLASC/4/5/#/6", expected: "/2/NULLASC/4/6", }, @@ -230,7 +231,7 @@ func TestAdjustStartKeyForInterleave(t *testing.T) { t.Run(strconv.Itoa(i), func(t *testing.T) { codec := keys.SystemSQLCodec actual := EncodeTestKey(t, kvDB, codec, ShortToLongKeyFmt(tc.input)) - actual, err := rowenc.AdjustStartKeyForInterleave(codec, tc.index.IndexDesc(), actual) + actual, err := rowenc.AdjustStartKeyForInterleave(codec, tc.index, actual) if err != nil { t.Fatal(err) } @@ -268,15 +269,15 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { child := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, sqlutils.TestDB, "child1") grandchild := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, sqlutils.TestDB, "grandchild1") - parentDescIdx := parent.PublicNonPrimaryIndexes()[0] - childDescIdx := child.PublicNonPrimaryIndexes()[0] - childNonUniqueIdx := child.PublicNonPrimaryIndexes()[1] - childUniqueIdx := child.PublicNonPrimaryIndexes()[2] - grandchildDescIdx := grandchild.PublicNonPrimaryIndexes()[0] + parentDescIdx := parent.GetPublicNonPrimaryIndexes()[0] + childDescIdx := child.GetPublicNonPrimaryIndexes()[0] + childNonUniqueIdx := child.GetPublicNonPrimaryIndexes()[1] + childUniqueIdx := child.GetPublicNonPrimaryIndexes()[2] + grandchildDescIdx := grandchild.GetPublicNonPrimaryIndexes()[0] testCases := []struct { table catalog.TableDescriptor - index catalog.Index + index *descpb.IndexDescriptor // See ShortToLongKeyFmt for how to represent a key. input string // If the end key is assumed to be inclusive when passed to @@ -338,25 +339,25 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { { table: parent, - index: parentDescIdx, + index: &parentDescIdx, input: "/NOTNULLDESC", expected: "/NOTNULLDESC", }, { table: child, - index: childDescIdx, + index: &childDescIdx, input: "/1/#/2/NOTNULLDESC", expected: "/1/#/2/NOTNULLDESC", }, { table: grandchild, - index: grandchildDescIdx, + index: &grandchildDescIdx, input: "/1/#/2/3/#/NOTNULLDESC", expected: "/1/#/2/3/#/NOTNULLDESC", }, { table: grandchild, - index: grandchildDescIdx, + index: &grandchildDescIdx, input: "/1/#/2/NOTNULLDESC", expected: "/1/#/2/NOTNULLDESC", }, @@ -503,26 +504,26 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { { table: child, - index: childDescIdx, + index: &childDescIdx, input: "/1/#/2/3", expected: "/1/#/2/3", }, { table: child, - index: childDescIdx, + index: &childDescIdx, input: "/1/#/2/3", inclusive: true, expected: "/1/#/2/4", }, { table: child, - index: childDescIdx, + index: &childDescIdx, input: "/1/#/2", expected: "/1/#/2", }, { table: child, - index: childDescIdx, + index: &childDescIdx, input: "/1/#/2", inclusive: true, expected: "/1/#/3", @@ -541,19 +542,19 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { // two index columns). { table: child, - index: childNonUniqueIdx, + index: &childNonUniqueIdx, input: "/2/3", expected: "/2/3", }, { table: child, - index: childNonUniqueIdx, + index: &childNonUniqueIdx, input: "/2/3/4", expected: "/2/3/4", }, { table: child, - index: childNonUniqueIdx, + index: &childNonUniqueIdx, input: "/2/3/4/5", expected: "/2/3/4/5", }, @@ -561,14 +562,14 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { // interleaved rows. { table: child, - index: childNonUniqueIdx, + index: &childNonUniqueIdx, input: "/2/3/4/5/#/10", expected: "/2/3/4/5/#/10", }, { table: child, - index: childUniqueIdx, + index: &childUniqueIdx, input: "/2/3", expected: "/2/3", }, @@ -576,19 +577,19 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { // interleaved rows. { table: child, - index: childUniqueIdx, + index: &childUniqueIdx, input: "/2/3/4", expected: "/2/3/4", }, { table: child, - index: childUniqueIdx, + index: &childUniqueIdx, input: "/2/NULLASC/4", expected: "/2/NULLASC/4", }, { table: child, - index: childUniqueIdx, + index: &childUniqueIdx, input: "/2/NULLASC/4/5", expected: "/2/NULLASC/4/5", }, @@ -596,7 +597,7 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { // interleaved rows. { table: child, - index: childUniqueIdx, + index: &childUniqueIdx, input: "/2/NULLASC/4/5/#/6", expected: "/2/NULLASC/4/5/#/6", }, @@ -652,7 +653,7 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { t.Run(strconv.Itoa(i), func(t *testing.T) { codec := keys.SystemSQLCodec actual := EncodeTestKey(t, kvDB, codec, ShortToLongKeyFmt(tc.input)) - actual, err := rowenc.AdjustEndKeyForInterleave(codec, tc.table, tc.index.IndexDesc(), actual, tc.inclusive) + actual, err := rowenc.AdjustEndKeyForInterleave(codec, tc.table, tc.index, actual, tc.inclusive) if err != nil { t.Fatal(err) } diff --git a/pkg/sql/rowenc/index_encoding.go b/pkg/sql/rowenc/index_encoding.go index a04bda3da543..60a394389c1a 100644 --- a/pkg/sql/rowenc/index_encoding.go +++ b/pkg/sql/rowenc/index_encoding.go @@ -41,8 +41,8 @@ import ( func MakeIndexKeyPrefix( codec keys.SQLCodec, desc catalog.TableDescriptor, indexID descpb.IndexID, ) []byte { - if i, err := desc.FindIndexWithID(indexID); err == nil && i.NumInterleaveAncestors() > 0 { - ancestor := i.GetInterleaveAncestor(0) + if i, err := desc.FindIndexByID(indexID); err == nil && len(i.Interleave.Ancestors) > 0 { + ancestor := &i.Interleave.Ancestors[0] return codec.IndexPrefix(uint32(ancestor.TableID), uint32(ancestor.IndexID)) } return codec.IndexPrefix(uint32(desc.GetID()), uint32(indexID)) @@ -546,7 +546,7 @@ func DecodeIndexKeyPrefix( // TODO(dan): This whole operation is n^2 because of the interleaves // bookkeeping. We could improve it to n with a prefix tree of components. - interleaves := append(make([]catalog.Index, 0, len(desc.ActiveIndexes())), desc.ActiveIndexes()...) + interleaves := append([]descpb.IndexDescriptor{*desc.GetPrimaryIndex()}, desc.GetPublicNonPrimaryIndexes()...) for component := 0; ; component++ { var tableID descpb.ID @@ -561,9 +561,9 @@ func DecodeIndexKeyPrefix( } for i := len(interleaves) - 1; i >= 0; i-- { - if interleaves[i].NumInterleaveAncestors() <= component || - interleaves[i].GetInterleaveAncestor(component).TableID != tableID || - interleaves[i].GetInterleaveAncestor(component).IndexID != indexID { + if len(interleaves[i].Interleave.Ancestors) <= component || + interleaves[i].Interleave.Ancestors[component].TableID != tableID || + interleaves[i].Interleave.Ancestors[component].IndexID != indexID { // This component, and thus this interleave, doesn't match what was // decoded, remove it. @@ -578,7 +578,7 @@ func DecodeIndexKeyPrefix( // Anything left has the same SharedPrefixLen at index `component`, so just // use the first one. - for i := uint32(0); i < interleaves[0].GetInterleaveAncestor(component).SharedPrefixLen; i++ { + for i := uint32(0); i < interleaves[0].Interleave.Ancestors[component].SharedPrefixLen; i++ { l, err := encoding.PeekLength(key) if err != nil { return 0, nil, err diff --git a/pkg/sql/rowenc/index_encoding_test.go b/pkg/sql/rowenc/index_encoding_test.go index 2fd77c186f69..8189a66e7e18 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.GetPrimaryIndex().IndexDesc(), colMap, testValues, primaryKeyPrefix) + primaryKey, _, err := EncodeIndexKey(tableDesc, tableDesc.GetPrimaryIndex(), colMap, testValues, primaryKeyPrefix) if err != nil { t.Fatal(err) } @@ -233,7 +233,7 @@ func TestIndexKey(t *testing.T) { primaryIndexKV := kv.KeyValue{Key: primaryKey, Value: &primaryValue} secondaryIndexEntry, err := EncodeSecondaryIndex( - codec, tableDesc, tableDesc.PublicNonPrimaryIndexes()[0].IndexDesc(), colMap, testValues, true /* includeEmpty */) + codec, tableDesc, &tableDesc.GetPublicNonPrimaryIndexes()[0], colMap, testValues, true /* includeEmpty */) if len(secondaryIndexEntry) != 1 { t.Fatalf("expected 1 index entry, got %d. got %#v", len(secondaryIndexEntry), secondaryIndexEntry) } @@ -275,8 +275,8 @@ func TestIndexKey(t *testing.T) { } } - checkEntry(tableDesc.GetPrimaryIndex().IndexDesc(), primaryIndexKV) - checkEntry(tableDesc.PublicNonPrimaryIndexes()[0].IndexDesc(), secondaryIndexKV) + checkEntry(tableDesc.GetPrimaryIndex(), primaryIndexKV) + checkEntry(&tableDesc.GetPublicNonPrimaryIndexes()[0], secondaryIndexKV) } } @@ -377,8 +377,8 @@ func TestInvertedIndexKey(t *testing.T) { indexKeyTest{50, nil, nil, primaryValues, secondaryValues, }) - for _, idx := range tableDesc.PublicNonPrimaryIndexes() { - idx.IndexDesc().Version = version + for i := range tableDesc.GetPublicNonPrimaryIndexes() { + tableDesc.GetPublicNonPrimaryIndexes()[i].Version = version } testValues := append(primaryValues, secondaryValues...) @@ -386,7 +386,7 @@ func TestInvertedIndexKey(t *testing.T) { codec := keys.SystemSQLCodec secondaryIndexEntries, err := EncodeSecondaryIndex( - codec, tableDesc, tableDesc.PublicNonPrimaryIndexes()[0].IndexDesc(), colMap, testValues, true /* includeEmpty */) + codec, tableDesc, &tableDesc.GetPublicNonPrimaryIndexes()[0], colMap, testValues, true /* includeEmpty */) if err != nil { t.Fatal(err) } @@ -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.GetPrimaryIndex().IndexDesc(), colMap, tc.table.values, primaryKeyPrefix) + desc, desc.GetPrimaryIndex(), colMap, tc.table.values, primaryKeyPrefix) if err != nil { t.Fatal(err) } @@ -962,8 +962,7 @@ func TestIndexKeyEquivSignature(t *testing.T) { // Column values should be at the beginning of the // remaining bytes of the key. - pkIndexDesc := desc.GetPrimaryIndex().IndexDesc() - colVals, null, err := EncodeColumns(pkIndexDesc.ColumnIDs, pkIndexDesc.ColumnDirections, colMap, tc.table.values, nil /*key*/) + colVals, null, err := EncodeColumns(desc.GetPrimaryIndex().ColumnIDs, desc.GetPrimaryIndex().ColumnDirections, colMap, tc.table.values, nil /*key*/) if err != nil { t.Fatal(err) } @@ -998,7 +997,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.GetPrimaryIndex().IndexDesc()) + equivSigs, err := TableEquivSignatures(&desc.TableDescriptor, desc.GetPrimaryIndex()) if err != nil { t.Fatal(err) } @@ -1083,13 +1082,13 @@ func TestEquivSignature(t *testing.T) { desc, colMap := makeTableDescForTest(table.indexKeyArgs) primaryKeyPrefix := MakeIndexKeyPrefix(keys.SystemSQLCodec, desc, desc.GetPrimaryIndexID()) primaryKey, _, err := EncodeIndexKey( - desc, desc.GetPrimaryIndex().IndexDesc(), colMap, table.values, primaryKeyPrefix) + desc, desc.GetPrimaryIndex(), colMap, table.values, primaryKeyPrefix) if err != nil { t.Fatal(err) } // Extract out the table's equivalence signature. - tempEquivSigs, err := TableEquivSignatures(&desc.TableDescriptor, desc.GetPrimaryIndex().IndexDesc()) + tempEquivSigs, err := TableEquivSignatures(&desc.TableDescriptor, desc.GetPrimaryIndex()) if err != nil { t.Fatal(err) } @@ -1242,11 +1241,10 @@ func ExtractIndexKey( return entry.Key, nil } - indexI, err := tableDesc.FindIndexWithID(indexID) + index, err := tableDesc.FindIndexByID(indexID) if err != nil { return nil, err } - index := indexI.IndexDesc() // Extract the values for index.ColumnIDs. indexTypes, err := colinfo.GetColumnTypes(tableDesc, index.ColumnIDs, nil) @@ -1323,6 +1321,6 @@ func ExtractIndexKey( decodedValues[len(values)+i] = value.Datum } indexKey, _, err := EncodeIndexKey( - tableDesc, tableDesc.GetPrimaryIndex().IndexDesc(), colMap, decodedValues, indexKeyPrefix) + tableDesc, tableDesc.GetPrimaryIndex(), colMap, decodedValues, indexKeyPrefix) return indexKey, err } diff --git a/pkg/sql/rowenc/testutils.go b/pkg/sql/rowenc/testutils.go index 4b2a721b2666..46e49737aaa9 100644 --- a/pkg/sql/rowenc/testutils.go +++ b/pkg/sql/rowenc/testutils.go @@ -1000,8 +1000,8 @@ func TestingMakePrimaryIndexKey( desc catalog.TableDescriptor, vals ...interface{}, ) (roachpb.Key, error) { index := desc.GetPrimaryIndex() - if len(vals) > index.NumColumns() { - return nil, errors.Errorf("got %d values, PK has %d columns", len(vals), index.NumColumns()) + if len(vals) > len(index.ColumnIDs) { + return nil, errors.Errorf("got %d values, PK has %d columns", len(vals), len(index.ColumnIDs)) } datums := make([]tree.Datum, len(vals)) for i, v := range vals { @@ -1018,7 +1018,7 @@ func TestingMakePrimaryIndexKey( return nil, errors.Errorf("unexpected value type %T", v) } // Check that the value type matches. - colID := index.GetColumnID(i) + colID := index.ColumnIDs[i] var done bool if err := desc.ForeachPublicColumn(func(c *descpb.ColumnDescriptor) error { if !done && c.ID == colID { @@ -1037,11 +1037,11 @@ func TestingMakePrimaryIndexKey( // MakeIndexKeyPrefix. var colIDToRowIndex catalog.TableColMap for i := range vals { - colIDToRowIndex.Set(index.GetColumnID(i), i) + colIDToRowIndex.Set(index.ColumnIDs[i], i) } - keyPrefix := MakeIndexKeyPrefix(keys.SystemSQLCodec, desc, index.GetID()) - key, _, err := EncodeIndexKey(desc, index.IndexDesc(), colIDToRowIndex, datums, keyPrefix) + keyPrefix := MakeIndexKeyPrefix(keys.SystemSQLCodec, desc, index.ID) + key, _, err := EncodeIndexKey(desc, index, colIDToRowIndex, datums, keyPrefix) if err != nil { return nil, err } diff --git a/pkg/sql/rowexec/BUILD.bazel b/pkg/sql/rowexec/BUILD.bazel index 8f0d1cdef2ca..d82f7fa2e548 100644 --- a/pkg/sql/rowexec/BUILD.bazel +++ b/pkg/sql/rowexec/BUILD.bazel @@ -140,7 +140,6 @@ go_test( "//pkg/settings/cluster", "//pkg/sql", "//pkg/sql/backfill", - "//pkg/sql/catalog", "//pkg/sql/catalog/catalogkeys", "//pkg/sql/catalog/catalogkv", "//pkg/sql/catalog/colinfo", diff --git a/pkg/sql/rowexec/joinreader_test.go b/pkg/sql/rowexec/joinreader_test.go index 83f738d3730d..d295cb3185d9 100644 --- a/pkg/sql/rowexec/joinreader_test.go +++ b/pkg/sql/rowexec/joinreader_test.go @@ -25,7 +25,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" @@ -1274,14 +1273,18 @@ func BenchmarkJoinReader(b *testing.B) { // Get the table descriptor and find the index that will provide us with // the expected match ratio. tableDesc := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "test", tableName) - foundIndex := catalog.FindPublicNonPrimaryIndex(tableDesc, func(idx catalog.Index) bool { - require.Equal(b, 1, idx.NumColumns(), "all indexes created in this benchmark should only contain one column") - return idx.GetColumnName(0) == columnDef.name - }) - if foundIndex == nil { + indexIdx := uint32(0) + for i := range tableDesc.GetPublicNonPrimaryIndexes() { + require.Equal(b, 1, len(tableDesc.GetPublicNonPrimaryIndexes()[i].ColumnNames), "all indexes created in this benchmark should only contain one column") + if tableDesc.GetPublicNonPrimaryIndexes()[i].ColumnNames[0] == columnDef.name { + // Found indexIdx. + indexIdx = uint32(i + 1) + break + } + } + if indexIdx == 0 { b.Fatalf("failed to find secondary index for column %s", columnDef.name) } - indexIdx := uint32(foundIndex.Ordinal()) input := newRowGeneratingSource(rowenc.OneIntCol, sqlutils.ToRowFn(func(rowIdx int) tree.Datum { // Convert to 0-based. return tree.NewDInt(tree.DInt(rowIdx - 1)) diff --git a/pkg/sql/rowexec/scrub_tablereader.go b/pkg/sql/rowexec/scrub_tablereader.go index f5a3fd3be4f7..c0cd6011f7db 100644 --- a/pkg/sql/rowexec/scrub_tablereader.go +++ b/pkg/sql/rowexec/scrub_tablereader.go @@ -149,7 +149,12 @@ func (tr *scrubTableReader) generateScrubErrorRow( row rowenc.EncDatumRow, scrubErr *scrub.Error, ) (rowenc.EncDatumRow, error) { details := make(map[string]interface{}) - index := tr.tableDesc.ActiveIndexes()[tr.indexIdx] + var index *descpb.IndexDescriptor + if tr.indexIdx == 0 { + index = tr.tableDesc.GetPrimaryIndex() + } else { + index = &tr.tableDesc.GetPublicNonPrimaryIndexes()[tr.indexIdx-1] + } // Collect all the row values into JSON rowDetails := make(map[string]interface{}) for i, colIdx := range tr.fetcherResultToColIdx { @@ -158,7 +163,7 @@ func (tr *scrubTableReader) generateScrubErrorRow( rowDetails[col.Name] = row[i].String(col.Type) } details["row_data"] = rowDetails - details["index_name"] = index.GetName() + details["index_name"] = index.Name details["error_message"] = scrub.UnwrapScrubError(error(scrubErr)).Error() detailsJSON, err := tree.MakeDJSON(details) diff --git a/pkg/sql/rowexec/tablereader_test.go b/pkg/sql/rowexec/tablereader_test.go index af472b0dfaa1..d7fa4ae09c08 100644 --- a/pkg/sql/rowexec/tablereader_test.go +++ b/pkg/sql/rowexec/tablereader_test.go @@ -72,7 +72,7 @@ func TestTableReader(t *testing.T) { makeIndexSpan := func(start, end int) execinfrapb.TableReaderSpan { var span roachpb.Span - prefix := roachpb.Key(rowenc.MakeIndexKeyPrefix(keys.SystemSQLCodec, td, td.PublicNonPrimaryIndexes()[0].GetID())) + prefix := roachpb.Key(rowenc.MakeIndexKeyPrefix(keys.SystemSQLCodec, td, td.GetPublicNonPrimaryIndexes()[0].ID)) span.Key = append(prefix, encoding.EncodeVarintAscending(nil, int64(start))...) span.EndKey = append(span.EndKey, prefix...) span.EndKey = append(span.EndKey, encoding.EncodeVarintAscending(nil, int64(end))...) diff --git a/pkg/sql/rowexec/zigzagjoiner.go b/pkg/sql/rowexec/zigzagjoiner.go index c723dbe664c9..2eee1170d716 100644 --- a/pkg/sql/rowexec/zigzagjoiner.go +++ b/pkg/sql/rowexec/zigzagjoiner.go @@ -419,7 +419,11 @@ func (z *zigzagJoiner) setupInfo( info.table = &tables[side] info.eqColumns = spec.EqColumns[side].Columns indexOrdinal := spec.IndexOrdinals[side] - info.index = info.table.ActiveIndexes()[indexOrdinal].IndexDesc() + if indexOrdinal == 0 { + info.index = info.table.GetPrimaryIndex() + } else { + info.index = &info.table.GetPublicNonPrimaryIndexes()[indexOrdinal-1] + } var columnIDs []descpb.ColumnID columnIDs, info.indexDirs = info.index.FullColumnIDs() @@ -647,8 +651,8 @@ func (zi *zigzagJoinerInfo) eqOrdering() (colinfo.ColumnOrdering, error) { if err != nil { return nil, err } - } else if idx := findColumnID(zi.table.GetPrimaryIndex().IndexDesc().ColumnIDs, colID); idx != -1 { - direction, err = zi.table.GetPrimaryIndex().GetColumnDirection(idx).ToEncodingDirection() + } else if idx := findColumnID(zi.table.GetPrimaryIndex().ColumnIDs, colID); idx != -1 { + direction, err = zi.table.GetPrimaryIndex().ColumnDirections[idx].ToEncodingDirection() if err != nil { return nil, err } diff --git a/pkg/sql/scan.go b/pkg/sql/scan.go index 435c318eb1db..0d66d4d1b34e 100644 --- a/pkg/sql/scan.go +++ b/pkg/sql/scan.go @@ -230,18 +230,35 @@ func (n *scanNode) initTable( func (n *scanNode) lookupSpecifiedIndex(indexFlags *tree.IndexFlags) error { if indexFlags.Index != "" { // Search index by name. - foundIndex, _ := n.desc.FindIndexWithName(string(indexFlags.Index)) - if foundIndex == nil || !foundIndex.Public() { + indexName := string(indexFlags.Index) + if indexName == n.desc.GetPrimaryIndex().Name { + n.specifiedIndex = n.desc.GetPrimaryIndex() + } else { + for i := range n.desc.GetPublicNonPrimaryIndexes() { + if indexName == n.desc.GetPublicNonPrimaryIndexes()[i].Name { + n.specifiedIndex = &n.desc.GetPublicNonPrimaryIndexes()[i] + break + } + } + } + if n.specifiedIndex == nil { return errors.Errorf("index %q not found", tree.ErrString(&indexFlags.Index)) } - n.specifiedIndex = foundIndex.IndexDesc() } else if indexFlags.IndexID != 0 { // Search index by ID. - foundIndex, _ := n.desc.FindIndexWithID(descpb.IndexID(indexFlags.IndexID)) - if foundIndex == nil || !foundIndex.Public() { + if n.desc.GetPrimaryIndexID() == descpb.IndexID(indexFlags.IndexID) { + n.specifiedIndex = n.desc.GetPrimaryIndex() + } else { + for i := range n.desc.GetPublicNonPrimaryIndexes() { + if n.desc.GetPublicNonPrimaryIndexes()[i].ID == descpb.IndexID(indexFlags.IndexID) { + n.specifiedIndex = &n.desc.GetPublicNonPrimaryIndexes()[i] + break + } + } + } + if n.specifiedIndex == nil { return errors.Errorf("index [%d] not found", indexFlags.IndexID) } - n.specifiedIndex = foundIndex.IndexDesc() } return nil } @@ -315,7 +332,7 @@ func initColsForScan( // Initializes the column structures. func (n *scanNode) initDescDefaults(colCfg scanColumnsConfig) error { n.colCfg = colCfg - n.index = n.desc.GetPrimaryIndex().IndexDesc() + n.index = n.desc.GetPrimaryIndex() 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 efb6083fa104..49463d69be91 100644 --- a/pkg/sql/schema_changer.go +++ b/pkg/sql/schema_changer.go @@ -1070,12 +1070,12 @@ func (sc *SchemaChanger) done(ctx context.Context) error { // If any old indexes (including the old primary index) being rewritten are interleaved // children, we will have to update their parents as well. for _, idxID := range append([]descpb.IndexID{swap.OldPrimaryIndexId}, swap.OldIndexes...) { - oldIndex, err := scTable.FindIndexWithID(idxID) + oldIndex, err := scTable.FindIndexByID(idxID) if err != nil { return err } - if oldIndex.NumInterleaveAncestors() != 0 { - ancestor := oldIndex.GetInterleaveAncestor(oldIndex.NumInterleaveAncestors() - 1) + if len(oldIndex.Interleave.Ancestors) != 0 { + ancestor := oldIndex.Interleave.Ancestors[len(oldIndex.Interleave.Ancestors)-1] if ancestor.TableID != scTable.ID { interleaveParents[ancestor.TableID] = struct{}{} } @@ -1176,13 +1176,13 @@ 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.GetPrimaryIndex().IndexDesc(), txn, desc) + pkJob, err := sc.createIndexGCJob(ctx, scTable.GetPrimaryIndex(), txn, desc) if err != nil { return err } childJobs = append(childJobs, pkJob) - for _, idx := range scTable.PublicNonPrimaryIndexes() { - idxJob, err := sc.createIndexGCJob(ctx, idx.IndexDesc(), txn, desc) + for i := range scTable.GetPublicNonPrimaryIndexes() { + idxJob, err := sc.createIndexGCJob(ctx, &scTable.GetPublicNonPrimaryIndexes()[i], txn, desc) if err != nil { return err } @@ -1222,28 +1222,27 @@ func (sc *SchemaChanger) done(ctx context.Context) error { // corresponding piece in runSchemaChangesInTxn. for _, idxID := range append( []descpb.IndexID{pkSwap.OldPrimaryIndexId}, pkSwap.OldIndexes...) { - oldIndex, err := scTable.FindIndexWithID(idxID) + oldIndex, err := scTable.FindIndexByID(idxID) if err != nil { return err } - if oldIndex.NumInterleaveAncestors() != 0 { - ancestorInfo := oldIndex.GetInterleaveAncestor(oldIndex.NumInterleaveAncestors() - 1) + if len(oldIndex.Interleave.Ancestors) != 0 { + ancestorInfo := oldIndex.Interleave.Ancestors[len(oldIndex.Interleave.Ancestors)-1] ancestor, err := descsCol.GetMutableTableVersionByID(ctx, ancestorInfo.TableID, txn) if err != nil { return err } - ancestorIdxI, err := ancestor.FindIndexWithID(ancestorInfo.IndexID) + ancestorIdx, err := ancestor.FindIndexByID(ancestorInfo.IndexID) if err != nil { return err } - ancestorIdx := ancestorIdxI.IndexDesc() foundAncestor := false for k, ref := range ancestorIdx.InterleavedBy { - if ref.Table == scTable.ID && ref.Index == oldIndex.GetID() { + if ref.Table == scTable.ID && ref.Index == oldIndex.ID { if foundAncestor { return errors.AssertionFailedf( "ancestor entry in %s for %s@%s found more than once", - ancestor.Name, scTable.Name, oldIndex.GetName()) + ancestor.Name, scTable.Name, oldIndex.Name) } ancestorIdx.InterleavedBy = append( ancestorIdx.InterleavedBy[:k], ancestorIdx.InterleavedBy[k+1:]...) diff --git a/pkg/sql/schema_changer_test.go b/pkg/sql/schema_changer_test.go index e4b4cebbeaa8..affeb9f7960f 100644 --- a/pkg/sql/schema_changer_test.go +++ b/pkg/sql/schema_changer_test.go @@ -143,7 +143,7 @@ INSERT INTO t.test VALUES ('a', 'b'), ('c', 'd'); // Check that RunStateMachineBeforeBackfill functions properly. expectedVersion = tableDesc.Version // Make a copy of the index for use in a mutation. - index := tableDesc.PublicNonPrimaryIndexes()[0].IndexDescDeepCopy() + index := protoutil.Clone(&tableDesc.GetPublicNonPrimaryIndexes()[0]).(*descpb.IndexDescriptor) index.Name = "bar" index.ID = tableDesc.NextIndexID tableDesc.NextIndexID++ @@ -152,7 +152,7 @@ INSERT INTO t.test VALUES ('a', 'b'), ('c', 'd'); &execCfg, cluster.MakeTestingClusterSettings(), ) tableDesc.Mutations = append(tableDesc.Mutations, descpb.DescriptorMutation{ - Descriptor_: &descpb.DescriptorMutation_Index{Index: &index}, + Descriptor_: &descpb.DescriptorMutation_Index{Index: index}, Direction: descpb.DescriptorMutation_ADD, State: descpb.DescriptorMutation_DELETE_ONLY, MutationID: tableDesc.NextMutationID, @@ -251,7 +251,7 @@ CREATE INDEX foo ON t.test (v) for r := retry.Start(retryOpts); r.Next(); { tableDesc = catalogkv.TestingGetMutableExistingTableDescriptor( kvDB, keys.SystemSQLCodec, "t", "test") - if len(tableDesc.PublicNonPrimaryIndexes()) == 1 { + if len(tableDesc.GetPublicNonPrimaryIndexes()) == 1 { break } } @@ -278,7 +278,7 @@ CREATE INDEX foo ON t.test (v) // Ensure that the version gets incremented. tableDesc = catalogkv.TestingGetMutableExistingTableDescriptor( kvDB, keys.SystemSQLCodec, "t", "test") - name := tableDesc.PublicNonPrimaryIndexes()[0].GetName() + name := tableDesc.GetPublicNonPrimaryIndexes()[0].Name if name != "ufo" { t.Fatalf("bad index name %s", name) } @@ -298,7 +298,7 @@ CREATE INDEX foo ON t.test (v) for r := retry.Start(retryOpts); r.Next(); { tableDesc = catalogkv.TestingGetMutableExistingTableDescriptor( kvDB, keys.SystemSQLCodec, "t", "test") - if len(tableDesc.PublicNonPrimaryIndexes()) == count+1 { + if len(tableDesc.GetPublicNonPrimaryIndexes()) == count+1 { break } } @@ -2218,8 +2218,8 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v INT UNIQUE DEFAULT 23 CREATE FAMILY F3 // The index is not regenerated. tableDesc := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "test") - if len(tableDesc.PublicNonPrimaryIndexes()) > 0 { - t.Fatalf("indexes %+v", tableDesc.PublicNonPrimaryIndexes()) + if len(tableDesc.GetPublicNonPrimaryIndexes()) > 0 { + t.Fatalf("indexes %+v", tableDesc.GetPublicNonPrimaryIndexes()) } // Unfortunately this is the same failure present when an index drop @@ -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.GetPrimaryIndex().NumColumns() != 1 || tableDesc.GetPrimaryIndex().GetColumnName(0) != "rowid" { + if len(tableDesc.GetPrimaryIndex().ColumnNames) != 1 || tableDesc.GetPrimaryIndex().ColumnNames[0] != "rowid" { return errors.Errorf("expected primary key change to not succeed after cancellation") } return nil @@ -3663,17 +3663,16 @@ CREATE TABLE d.t ( kvDB, keys.SystemSQLCodec, "d", "t") // Verify that this descriptor uses the new STORING encoding. Overwrite it // with one that uses the old encoding. - for _, index := range tableDesc.PublicNonPrimaryIndexes() { - if index.NumExtraColumns() != 1 { + for i, index := range tableDesc.GetPublicNonPrimaryIndexes() { + if len(index.ExtraColumnIDs) != 1 { t.Fatalf("ExtraColumnIDs not set properly: %s", tableDesc) } - if index.NumStoredColumns() != 1 { + if len(index.StoreColumnIDs) != 1 { t.Fatalf("StoreColumnIDs not set properly: %s", tableDesc) } - newIndexDesc := *index.IndexDesc() - newIndexDesc.ExtraColumnIDs = append(newIndexDesc.ExtraColumnIDs, newIndexDesc.StoreColumnIDs...) - newIndexDesc.StoreColumnIDs = nil - tableDesc.SetPublicNonPrimaryIndex(index.Ordinal(), newIndexDesc) + index.ExtraColumnIDs = append(index.ExtraColumnIDs, index.StoreColumnIDs...) + index.StoreColumnIDs = nil + tableDesc.SetPublicNonPrimaryIndex(i+1, index) } if err := kvDB.Put( context.Background(), @@ -5109,8 +5108,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.Mutations) > 0 { - t.Fatalf("descriptor broken %d, %d", len(tableDesc.PublicNonPrimaryIndexes()), len(tableDesc.Mutations)) + if len(tableDesc.GetPublicNonPrimaryIndexes()) > 0 || len(tableDesc.Mutations) > 0 { + t.Fatalf("descriptor broken %d, %d", len(tableDesc.GetPublicNonPrimaryIndexes()), len(tableDesc.Mutations)) } } @@ -5182,8 +5181,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.Mutations) > 0 { - t.Fatalf("descriptor broken %d, %d", len(tableDesc.PublicNonPrimaryIndexes()), len(tableDesc.Mutations)) + if len(tableDesc.GetPublicNonPrimaryIndexes()) > 0 || len(tableDesc.Mutations) > 0 { + t.Fatalf("descriptor broken %d, %d", len(tableDesc.GetPublicNonPrimaryIndexes()), len(tableDesc.Mutations)) } } @@ -5889,7 +5888,7 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v INT8); // Wait until indexes are created. for r := retry.Start(retryOpts); r.Next(); { tableDesc = catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "test") - if len(tableDesc.PublicNonPrimaryIndexes()) == 1 { + if len(tableDesc.GetPublicNonPrimaryIndexes()) == 1 { break } } @@ -6294,13 +6293,13 @@ CREATE INDEX i ON t.test (a) WHERE b > 2 } tableDesc := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "test") - index, err := tableDesc.FindIndexWithName("i") + indexDesc, _, err := tableDesc.FindIndexByName("i") if err != nil { t.Fatalf("unexpected error: %s", err) } // Collect all the keys in the partial index. - span := tableDesc.IndexSpan(keys.SystemSQLCodec, index.GetID()) + span := tableDesc.IndexSpan(keys.SystemSQLCodec, indexDesc.ID) keys, err := kvDB.Scan(ctx, span.Key, span.EndKey, 0) if err != nil { t.Fatalf("unexpected error: %s", err) diff --git a/pkg/sql/scrub.go b/pkg/sql/scrub.go index e6e5f7416da9..ea7688858a58 100644 --- a/pkg/sql/scrub.go +++ b/pkg/sql/scrub.go @@ -286,8 +286,7 @@ func (n *scrubNode) startScrubTable( func getPrimaryColIdxs( tableDesc *tabledesc.Immutable, columns []*descpb.ColumnDescriptor, ) (primaryColIdxs []int, err error) { - for i := 0; i < tableDesc.GetPrimaryIndex().NumColumns(); i++ { - colID := tableDesc.GetPrimaryIndex().GetColumnID(i) + for i, colID := range tableDesc.GetPrimaryIndex().ColumnIDs { rowIdx := -1 for idx, col := range columns { if col.ID == colID { @@ -299,7 +298,7 @@ func getPrimaryColIdxs( return nil, errors.Errorf( "could not find primary index column in projection: columnID=%d columnName=%s", colID, - tableDesc.GetPrimaryIndex().GetColumnName(i)) + tableDesc.GetPrimaryIndex().ColumnNames[i]) } primaryColIdxs = append(primaryColIdxs, rowIdx) } @@ -346,8 +345,9 @@ func pairwiseOp(left []string, right []string, op string) []string { func createPhysicalCheckOperations( tableDesc *tabledesc.Immutable, tableName *tree.TableName, ) (checks []checkOperation) { - for _, idx := range tableDesc.ActiveIndexes() { - checks = append(checks, newPhysicalCheckOperation(tableName, tableDesc, idx.IndexDesc())) + checks = append(checks, newPhysicalCheckOperation(tableName, tableDesc, tableDesc.GetPrimaryIndex())) + for i := range tableDesc.GetPublicNonPrimaryIndexes() { + checks = append(checks, newPhysicalCheckOperation(tableName, tableDesc, &tableDesc.GetPublicNonPrimaryIndexes()[i])) } return checks } @@ -367,11 +367,11 @@ func createIndexCheckOperations( if indexNames == nil { // Populate results with all secondary indexes of the // table. - for _, idx := range tableDesc.PublicNonPrimaryIndexes() { + for i := range tableDesc.GetPublicNonPrimaryIndexes() { results = append(results, newIndexCheckOperation( tableName, tableDesc, - idx.IndexDesc(), + &tableDesc.GetPublicNonPrimaryIndexes()[i], asOf, )) } @@ -383,15 +383,15 @@ func createIndexCheckOperations( for _, idxName := range indexNames { names[idxName.String()] = struct{}{} } - for _, idx := range tableDesc.PublicNonPrimaryIndexes() { - if _, ok := names[idx.GetName()]; ok { + for i := range tableDesc.GetPublicNonPrimaryIndexes() { + if _, ok := names[tableDesc.GetPublicNonPrimaryIndexes()[i].Name]; ok { results = append(results, newIndexCheckOperation( tableName, tableDesc, - idx.IndexDesc(), + &tableDesc.GetPublicNonPrimaryIndexes()[i], asOf, )) - delete(names, idx.GetName()) + delete(names, tableDesc.GetPublicNonPrimaryIndexes()[i].Name) } } if len(names) > 0 { diff --git a/pkg/sql/scrub_fk.go b/pkg/sql/scrub_fk.go index ceba8cbff41e..d60d25e5b578 100644 --- a/pkg/sql/scrub_fk.go +++ b/pkg/sql/scrub_fk.go @@ -114,8 +114,7 @@ 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.GetPrimaryIndex().NumColumns(); i++ { - pkColID := o.tableDesc.GetPrimaryIndex().GetColumnID(i) + for _, pkColID := range o.tableDesc.GetPrimaryIndex().ColumnIDs { found := false for _, id := range o.constraint.FK.OriginColumnIDs { if pkColID == id { @@ -148,9 +147,8 @@ 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.GetPrimaryIndex().NumColumns()) - for i := 0; i < o.tableDesc.GetPrimaryIndex().NumColumns(); i++ { - id := o.tableDesc.GetPrimaryIndex().GetColumnID(i) + primaryKeyDatums := make(tree.Datums, 0, len(o.tableDesc.GetPrimaryIndex().ColumnIDs)) + for _, id := range o.tableDesc.GetPrimaryIndex().ColumnIDs { idx := o.colIDToRowIdx.GetDefault(id) primaryKeyDatums = append(primaryKeyDatums, row[idx]) } @@ -165,8 +163,7 @@ func (o *sqlForeignKeyCheckOperation) Next(params runParams) (tree.Datums, error } rowDetails[col.Name] = row[idx].String() } - for i := 0; i < o.tableDesc.GetPrimaryIndex().NumColumns(); i++ { - id := o.tableDesc.GetPrimaryIndex().GetColumnID(i) + for _, id := range o.tableDesc.GetPrimaryIndex().ColumnIDs { 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 6a1d566e7c1b..90bf988216c7 100644 --- a/pkg/sql/scrub_index.go +++ b/pkg/sql/scrub_index.go @@ -83,8 +83,7 @@ func (o *indexCheckOperation) Start(params runParams) error { var pkColumns, otherColumns []*descpb.ColumnDescriptor - for i := 0; i < o.tableDesc.GetPrimaryIndex().NumColumns(); i++ { - colID := o.tableDesc.GetPrimaryIndex().GetColumnID(i) + for _, colID := range o.tableDesc.GetPrimaryIndex().ColumnIDs { 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 5cb5cb2da662..cc1d74200b01 100644 --- a/pkg/sql/scrub_test.go +++ b/pkg/sql/scrub_test.go @@ -60,7 +60,7 @@ INSERT INTO t."tEst" VALUES (10, 20); // Construct datums for our row values (k, v). values := []tree.Datum{tree.NewDInt(10), tree.NewDInt(20)} tableDesc := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "tEst") - secondaryIndex := tableDesc.PublicNonPrimaryIndexes()[0] + secondaryIndex := &tableDesc.GetPublicNonPrimaryIndexes()[0] var colIDtoRowIndex catalog.TableColMap colIDtoRowIndex.Set(tableDesc.Columns[0].ID, 0) @@ -69,7 +69,7 @@ INSERT INTO t."tEst" VALUES (10, 20); // Construct the secondary index key that is currently in the // database. secondaryIndexKey, err := rowenc.EncodeSecondaryIndex( - keys.SystemSQLCodec, tableDesc, secondaryIndex.IndexDesc(), colIDtoRowIndex, values, true /* includeEmpty */) + keys.SystemSQLCodec, tableDesc, secondaryIndex, colIDtoRowIndex, values, true /* includeEmpty */) if err != nil { t.Fatalf("unexpected error: %s", err) } @@ -129,7 +129,7 @@ CREATE INDEX secondary ON t.test (v); } tableDesc := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "test") - secondaryIndex := tableDesc.PublicNonPrimaryIndexes()[0] + secondaryIndexDesc := &tableDesc.GetPublicNonPrimaryIndexes()[0] var colIDtoRowIndex catalog.TableColMap colIDtoRowIndex.Set(tableDesc.Columns[0].ID, 0) @@ -137,18 +137,18 @@ CREATE INDEX secondary ON t.test (v); // Construct datums and secondary k/v for our row values (k, v). values := []tree.Datum{tree.NewDInt(10), tree.NewDInt(314)} - secondaryIndexKey, err := rowenc.EncodeSecondaryIndex( - keys.SystemSQLCodec, tableDesc, secondaryIndex.IndexDesc(), colIDtoRowIndex, values, true /* includeEmpty */) + secondaryIndex, err := rowenc.EncodeSecondaryIndex( + keys.SystemSQLCodec, tableDesc, secondaryIndexDesc, colIDtoRowIndex, values, true /* includeEmpty */) if err != nil { t.Fatalf("unexpected error: %s", err) } - if len(secondaryIndexKey) != 1 { - t.Fatalf("expected 1 index entry, got %d. got %#v", len(secondaryIndexKey), secondaryIndexKey) + if len(secondaryIndex) != 1 { + t.Fatalf("expected 1 index entry, got %d. got %#v", len(secondaryIndex), secondaryIndex) } // Put the new secondary k/v into the database. - if err := kvDB.Put(context.Background(), secondaryIndexKey[0].Key, &secondaryIndexKey[0].Value); err != nil { + if err := kvDB.Put(context.Background(), secondaryIndex[0].Key, &secondaryIndex[0].Value); err != nil { t.Fatalf("unexpected error: %s", err) } @@ -223,7 +223,7 @@ INSERT INTO t.test VALUES (10, 20, 1337); } tableDesc := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "test") - secondaryIndex := tableDesc.PublicNonPrimaryIndexes()[0] + secondaryIndexDesc := &tableDesc.GetPublicNonPrimaryIndexes()[0] var colIDtoRowIndex catalog.TableColMap colIDtoRowIndex.Set(tableDesc.Columns[0].ID, 0) @@ -232,30 +232,30 @@ INSERT INTO t.test VALUES (10, 20, 1337); // Generate the existing secondary index key. values := []tree.Datum{tree.NewDInt(10), tree.NewDInt(20), tree.NewDInt(1337)} - secondaryIndexKey, err := rowenc.EncodeSecondaryIndex( - keys.SystemSQLCodec, tableDesc, secondaryIndex.IndexDesc(), colIDtoRowIndex, values, true /* includeEmpty */) + secondaryIndex, err := rowenc.EncodeSecondaryIndex( + keys.SystemSQLCodec, tableDesc, secondaryIndexDesc, colIDtoRowIndex, values, true /* includeEmpty */) - if len(secondaryIndexKey) != 1 { - t.Fatalf("expected 1 index entry, got %d. got %#v", len(secondaryIndexKey), secondaryIndexKey) + if len(secondaryIndex) != 1 { + t.Fatalf("expected 1 index entry, got %d. got %#v", len(secondaryIndex), secondaryIndex) } if err != nil { t.Fatalf("unexpected error: %s", err) } // Delete the existing secondary k/v. - if err := kvDB.Del(context.Background(), secondaryIndexKey[0].Key); err != nil { + if err := kvDB.Del(context.Background(), secondaryIndex[0].Key); err != nil { t.Fatalf("unexpected error: %s", err) } // Generate a secondary index k/v that has a different value. values = []tree.Datum{tree.NewDInt(10), tree.NewDInt(20), tree.NewDInt(314)} - secondaryIndexKey, err = rowenc.EncodeSecondaryIndex( - keys.SystemSQLCodec, tableDesc, secondaryIndex.IndexDesc(), colIDtoRowIndex, values, true /* includeEmpty */) + secondaryIndex, err = rowenc.EncodeSecondaryIndex( + keys.SystemSQLCodec, tableDesc, secondaryIndexDesc, colIDtoRowIndex, values, true /* includeEmpty */) if err != nil { t.Fatalf("unexpected error: %s", err) } // Put the incorrect secondary k/v. - if err := kvDB.Put(context.Background(), secondaryIndexKey[0].Key, &secondaryIndexKey[0].Value); err != nil { + if err := kvDB.Put(context.Background(), secondaryIndex[0].Key, &secondaryIndex[0].Value); err != nil { t.Fatalf("unexpected error: %s", err) } @@ -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.GetPrimaryIndex().IndexDesc(), colIDtoRowIndex, values, primaryIndexKeyPrefix) + tableDesc, tableDesc.GetPrimaryIndex(), colIDtoRowIndex, values, primaryIndexKeyPrefix) if err != nil { t.Fatalf("unexpected error: %s", err) } @@ -445,7 +445,7 @@ func TestScrubFKConstraintFKMissing(t *testing.T) { // Construct datums for the child row values (child_id, parent_id). values := []tree.Datum{tree.NewDInt(10), tree.NewDInt(314)} - secondaryIndex := tableDesc.PublicNonPrimaryIndexes()[0] + secondaryIndex := &tableDesc.GetPublicNonPrimaryIndexes()[0] var colIDtoRowIndex catalog.TableColMap colIDtoRowIndex.Set(tableDesc.Columns[0].ID, 0) @@ -454,7 +454,7 @@ func TestScrubFKConstraintFKMissing(t *testing.T) { // Construct the secondary index key entry as it exists in the // database. secondaryIndexKey, err := rowenc.EncodeSecondaryIndex( - keys.SystemSQLCodec, tableDesc, secondaryIndex.IndexDesc(), colIDtoRowIndex, values, true /* includeEmpty */) + keys.SystemSQLCodec, tableDesc, secondaryIndex, colIDtoRowIndex, values, true /* includeEmpty */) if err != nil { t.Fatalf("unexpected error: %s", err) } @@ -474,7 +474,7 @@ func TestScrubFKConstraintFKMissing(t *testing.T) { // Construct the new secondary index key that will be inserted. secondaryIndexKey, err = rowenc.EncodeSecondaryIndex( - keys.SystemSQLCodec, tableDesc, secondaryIndex.IndexDesc(), colIDtoRowIndex, values, true /* includeEmpty */) + keys.SystemSQLCodec, tableDesc, secondaryIndex, colIDtoRowIndex, values, true /* includeEmpty */) 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.GetPrimaryIndex().IndexDesc(), colIDtoRowIndex, values, primaryIndexKeyPrefix) + tableDesc, tableDesc.GetPrimaryIndex(), 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.GetPrimaryIndex().IndexDesc(), colIDtoRowIndex, values, primaryIndexKeyPrefix) + tableDesc, tableDesc.GetPrimaryIndex(), 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.GetPrimaryIndex().IndexDesc(), colIDtoRowIndex, values, primaryIndexKeyPrefix) + tableDesc, tableDesc.GetPrimaryIndex(), 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.GetPrimaryIndex().IndexDesc(), colIDtoRowIndex, values, primaryIndexKeyPrefix) + tableDesc, tableDesc.GetPrimaryIndex(), colIDtoRowIndex, values, primaryIndexKeyPrefix) if err != nil { t.Fatalf("unexpected error: %s", err) } diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go index 54d02c7a444e..2395dc5e9629 100644 --- a/pkg/sql/sem/builtins/builtins.go +++ b/pkg/sql/sem/builtins/builtins.go @@ -3673,11 +3673,10 @@ may increase either contention or retry errors, or both.`, if err != nil { return nil, err } - index, err := tableDesc.FindIndexWithID(descpb.IndexID(indexID)) + indexDesc, err := tableDesc.FindIndexByID(descpb.IndexID(indexID)) if err != nil { return nil, err } - indexDesc := index.IndexDesc() // Collect the index columns. If the index is a non-unique secondary // index, it might have some extra key columns. indexColIDs := indexDesc.ColumnIDs @@ -4137,14 +4136,14 @@ may increase either contention or retry errors, or both.`, if err != nil { return nil, err } - index, err := tableDesc.FindIndexWithID(descpb.IndexID(indexID)) + indexDesc, err := tableDesc.FindIndexByID(descpb.IndexID(indexID)) if err != nil { return nil, err } - if index.GetGeoConfig().S2Geography == nil { + if indexDesc.GeoConfig.S2Geography == nil { return nil, errors.Errorf("index_id %d is not a geography inverted index", indexID) } - keys, err := rowenc.EncodeGeoInvertedIndexTableKeys(g, nil, index.IndexDesc()) + keys, err := rowenc.EncodeGeoInvertedIndexTableKeys(g, nil, indexDesc) if err != nil { return nil, err } @@ -4171,14 +4170,14 @@ may increase either contention or retry errors, or both.`, if err != nil { return nil, err } - index, err := tableDesc.FindIndexWithID(descpb.IndexID(indexID)) + indexDesc, err := tableDesc.FindIndexByID(descpb.IndexID(indexID)) if err != nil { return nil, err } - if index.GetGeoConfig().S2Geometry == nil { + if indexDesc.GeoConfig.S2Geometry == nil { return nil, errors.Errorf("index_id %d is not a geometry inverted index", indexID) } - keys, err := rowenc.EncodeGeoInvertedIndexTableKeys(g, nil, index.IndexDesc()) + keys, err := rowenc.EncodeGeoInvertedIndexTableKeys(g, nil, indexDesc) if err != nil { return nil, err } diff --git a/pkg/sql/set_zone_config.go b/pkg/sql/set_zone_config.go index bf8fe6322feb..e2998034aeae 100644 --- a/pkg/sql/set_zone_config.go +++ b/pkg/sql/set_zone_config.go @@ -332,19 +332,12 @@ func (n *setZoneConfigNode) startExec(params runParams) error { // Backward compatibility for ALTER PARTITION ... OF TABLE. Determine which // index has the specified partition. partitionName := string(n.zoneSpecifier.Partition) - - var indexes []catalog.Index - for _, idx := range table.NonDropIndexes() { - if tabledesc.FindIndexPartitionByName(idx.IndexDesc(), partitionName) != nil { - indexes = append(indexes, idx) - } - } - + indexes := table.FindIndexesWithPartition(partitionName) switch len(indexes) { case 0: return fmt.Errorf("partition %q does not exist on table %q", partitionName, table.GetName()) case 1: - n.zoneSpecifier.TableOrIndex.Index = tree.UnrestrictedName(indexes[0].GetName()) + n.zoneSpecifier.TableOrIndex.Index = tree.UnrestrictedName(indexes[0].Name) default: err := fmt.Errorf( "partition %q exists on multiple indexes of table %q", partitionName, table.GetName()) @@ -360,10 +353,10 @@ func (n *setZoneConfigNode) startExec(params runParams) error { var specifiers []tree.ZoneSpecifier if n.zoneSpecifier.TargetsPartition() && n.allIndexes { sqltelemetry.IncrementPartitioningCounter(sqltelemetry.AlterAllPartitions) - for _, idx := range table.NonDropIndexes() { - if p := tabledesc.FindIndexPartitionByName(idx.IndexDesc(), string(n.zoneSpecifier.Partition)); p != nil { + for _, idx := range table.AllNonDropIndexes() { + if p := tabledesc.FindIndexPartitionByName(idx, string(n.zoneSpecifier.Partition)); p != nil { zs := n.zoneSpecifier - zs.TableOrIndex.Index = tree.UnrestrictedName(idx.GetName()) + zs.TableOrIndex.Index = tree.UnrestrictedName(idx.Name) specifiers = append(specifiers, zs) } } diff --git a/pkg/sql/show_create.go b/pkg/sql/show_create.go index bf0bf3a09403..58b6794b2b42 100644 --- a/pkg/sql/show_create.go +++ b/pkg/sql/show_create.go @@ -96,7 +96,7 @@ func ShowCreateTable( return "", err } f.WriteString(colstr) - if desc.IsPhysicalTable() && desc.GetPrimaryIndex().GetColumnID(0) == col.ID { + if desc.IsPhysicalTable() && desc.GetPrimaryIndex().ColumnIDs[0] == col.ID { // Only set primaryKeyIsOnVisibleColumn to true if the primary key // is on a visible column (not rowid). primaryKeyIsOnVisibleColumn = true @@ -105,7 +105,7 @@ func ShowCreateTable( if primaryKeyIsOnVisibleColumn || (desc.IsPhysicalTable() && desc.GetPrimaryIndex().IsSharded()) { f.WriteString(",\n\tCONSTRAINT ") - formatQuoteNames(&f.Buffer, desc.GetPrimaryIndex().GetName()) + formatQuoteNames(&f.Buffer, desc.GetPrimaryIndex().Name) f.WriteString(" ") f.WriteString(desc.PrimaryKeyString()) } @@ -141,15 +141,16 @@ func ShowCreateTable( } } allIdx := append( - append([]catalog.Index{}, desc.PublicNonPrimaryIndexes()...), - desc.GetPrimaryIndex()) - for _, idx := range allIdx { + append([]descpb.IndexDescriptor{}, desc.GetPublicNonPrimaryIndexes()...), + *desc.GetPrimaryIndex()) + for i := range allIdx { + idx := &allIdx[i] // Only add indexes to the create_statement column, and not to the // create_nofks column if they are not associated with an INTERLEAVE // statement. // Initialize to false if Interleave has no ancestors, indicating that the // index is not interleaved at all. - includeInterleaveClause := idx.NumInterleaveAncestors() == 0 + includeInterleaveClause := len(idx.Interleave.Ancestors) == 0 if displayOptions.FKDisplayMode != OmitFKClausesFromCreate { // The caller is instructing us to not omit FK clauses from inside the CREATE. // (i.e. the caller does not want them as separate DDL.) @@ -157,10 +158,10 @@ func ShowCreateTable( // clauses as well. includeInterleaveClause = true } - if !idx.Primary() && includeInterleaveClause { + if idx.ID != desc.GetPrimaryIndex().ID && includeInterleaveClause { // Showing the primary index is handled above. f.WriteString(",\n\t") - idxStr, err := catformat.IndexForDisplay(ctx, desc, &descpb.AnonymousTable, idx.IndexDesc(), &p.RunParams(ctx).p.semaCtx) + idxStr, err := catformat.IndexForDisplay(ctx, desc, &descpb.AnonymousTable, idx, &p.RunParams(ctx).p.semaCtx) if err != nil { return "", err } @@ -171,12 +172,12 @@ func ShowCreateTable( // Add interleave or Foreign Key indexes only to the create_table columns, // and not the create_nofks column. if includeInterleaveClause { - if err := showCreateInterleave(idx.IndexDesc(), &f.Buffer, dbPrefix, lCtx); err != nil { + if err := showCreateInterleave(idx, &f.Buffer, dbPrefix, lCtx); err != nil { return "", err } } if err := ShowCreatePartitioning( - a, p.ExecCfg().Codec, desc, idx.IndexDesc(), &idx.IndexDesc().Partitioning, &f.Buffer, 1 /* indent */, 0, /* colOffset */ + a, p.ExecCfg().Codec, desc, idx, &idx.Partitioning, &f.Buffer, 1 /* indent */, 0, /* colOffset */ ); err != nil { return "", err } @@ -189,11 +190,11 @@ func ShowCreateTable( return "", err } - if err := showCreateInterleave(desc.GetPrimaryIndex().IndexDesc(), &f.Buffer, dbPrefix, lCtx); err != nil { + if err := showCreateInterleave(desc.GetPrimaryIndex(), &f.Buffer, dbPrefix, lCtx); err != nil { return "", err } if err := ShowCreatePartitioning( - a, p.ExecCfg().Codec, desc, desc.GetPrimaryIndex().IndexDesc(), &desc.GetPrimaryIndex().IndexDesc().Partitioning, &f.Buffer, 0 /* indent */, 0, /* colOffset */ + a, p.ExecCfg().Codec, desc, desc.GetPrimaryIndex(), &desc.GetPrimaryIndex().Partitioning, &f.Buffer, 0 /* indent */, 0, /* colOffset */ ); err != nil { return "", err } diff --git a/pkg/sql/show_create_clauses.go b/pkg/sql/show_create_clauses.go index 340e5a86709e..4044d4bed0e5 100644 --- a/pkg/sql/show_create_clauses.go +++ b/pkg/sql/show_create_clauses.go @@ -136,7 +136,7 @@ func showComments( } for _, indexComment := range tc.indexes { - idx, err := table.FindIndexWithID(descpb.IndexID(indexComment.subID)) + idx, err := table.FindIndexByID(descpb.IndexID(indexComment.subID)) if err != nil { return err } @@ -145,7 +145,7 @@ func showComments( f.FormatNode(&tree.CommentOnIndex{ Index: tree.TableIndexName{ Table: *tn, - Index: tree.UnrestrictedName(idx.GetName()), + Index: tree.UnrestrictedName(idx.Name), }, Comment: &indexComment.comment, }) diff --git a/pkg/sql/show_fingerprints.go b/pkg/sql/show_fingerprints.go index 3472fe97afc3..6e6084769e93 100644 --- a/pkg/sql/show_fingerprints.go +++ b/pkg/sql/show_fingerprints.go @@ -65,15 +65,9 @@ func (p *planner) ShowFingerprints( return nil, err } - indexes := tableDesc.NonDropIndexes() - indexDescs := make([]*descpb.IndexDescriptor, len(indexes)) - for i, index := range indexes { - indexDescs[i] = index.IndexDesc() - } - return &showFingerprintsNode{ tableDesc: tableDesc, - indexes: indexDescs, + indexes: tableDesc.AllNonDropIndexes(), }, nil } diff --git a/pkg/sql/span_builder_test.go b/pkg/sql/span_builder_test.go index cddec253abe1..cb68ac354417 100644 --- a/pkg/sql/span_builder_test.go +++ b/pkg/sql/span_builder_test.go @@ -103,11 +103,11 @@ func TestSpanBuilderCanSplitSpan(t *testing.T) { t.Fatal(err) } desc := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "t") - idx, err := desc.FindIndexWithName(tc.index) + idx, _, err := desc.FindIndexByName(tc.index) if err != nil { t.Fatal(err) } - builder := span.MakeBuilder(evalCtx, execCfg.Codec, desc, idx.IndexDesc()) + builder := span.MakeBuilder(evalCtx, execCfg.Codec, desc, idx) if res := builder.CanSplitSpanIntoSeparateFamilies( tc.numNeededFamilies, tc.prefixLen, tc.containsNull); res != tc.canSplit { t.Errorf("expected result to be %v, but found %v", tc.canSplit, res) diff --git a/pkg/sql/table_ref_test.go b/pkg/sql/table_ref_test.go index def879d5350d..a3dd0c2ee320 100644 --- a/pkg/sql/table_ref_test.go +++ b/pkg/sql/table_ref_test.go @@ -61,7 +61,7 @@ CREATE INDEX bc ON test.t(b, c); } } pkID := tableDesc.GetPrimaryIndexID() - secID := tableDesc.PublicNonPrimaryIndexes()[0].GetID() + secID := tableDesc.GetPublicNonPrimaryIndexes()[0].ID // Retrieve the numeric descriptors. tableDesc = catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "test", "hidden") diff --git a/pkg/sql/table_test.go b/pkg/sql/table_test.go index c504131e2cb3..af8e676e69fc 100644 --- a/pkg/sql/table_test.go +++ b/pkg/sql/table_test.go @@ -302,16 +302,11 @@ func TestMakeTableDescIndexes(t *testing.T) { if err != nil { t.Fatalf("%d (%s): %v", i, d.sql, err) } - activeIndexDescs := make([]descpb.IndexDescriptor, len(schema.ActiveIndexes())) - for i, index := range schema.ActiveIndexes() { - activeIndexDescs[i] = *index.IndexDesc() + if !reflect.DeepEqual(d.primary, *schema.GetPrimaryIndex()) { + t.Fatalf("%d (%s): primary mismatch: expected %+v, but got %+v", i, d.sql, d.primary, schema.GetPrimaryIndex()) } - - if !reflect.DeepEqual(d.primary, activeIndexDescs[0]) { - t.Fatalf("%d (%s): primary mismatch: expected %+v, but got %+v", i, d.sql, d.primary, activeIndexDescs[0]) - } - if !reflect.DeepEqual(d.indexes, activeIndexDescs[1:]) { - t.Fatalf("%d (%s): index mismatch: expected %+v, but got %+v", i, d.sql, d.indexes, activeIndexDescs[1:]) + if !reflect.DeepEqual(d.indexes, append([]descpb.IndexDescriptor{}, schema.GetPublicNonPrimaryIndexes()...)) { + t.Fatalf("%d (%s): index mismatch: expected %+v, but got %+v", i, d.sql, d.indexes, schema.GetPublicNonPrimaryIndexes()) } } diff --git a/pkg/sql/tablewriter_delete.go b/pkg/sql/tablewriter_delete.go index fed9c125ca18..e081bbf6dace 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().GetPrimaryIndex().IndexDesc(), + Index: td.tableDesc().GetPrimaryIndex(), 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().GetPrimaryIndex().IndexDesc(), + Index: td.tableDesc().GetPrimaryIndex(), ColIdxMap: td.rd.FetchColIDtoRowIndex, Cols: td.rd.FetchCols, ValNeededForCol: valNeededForCol, diff --git a/pkg/sql/tests/hash_sharded_test.go b/pkg/sql/tests/hash_sharded_test.go index 588edd51347d..aee5ac1fcbe1 100644 --- a/pkg/sql/tests/hash_sharded_test.go +++ b/pkg/sql/tests/hash_sharded_test.go @@ -30,11 +30,11 @@ import ( func getShardColumnID( t *testing.T, tableDesc *tabledesc.Immutable, shardedIndexName string, ) descpb.ColumnID { - idx, err := tableDesc.FindIndexWithName(shardedIndexName) + idx, _, err := tableDesc.FindIndexByName(shardedIndexName) if err != nil { t.Fatal(err) } - shardCol, _, err := tableDesc.FindColumnByName(tree.Name(idx.GetShardColumnName())) + shardCol, _, err := tableDesc.FindColumnByName(tree.Name(idx.Sharded.Name)) if err != nil { t.Fatal(err) } @@ -49,7 +49,7 @@ func getShardColumnID( func verifyTableDescriptorState( t *testing.T, tableDesc *tabledesc.Immutable, shardedIndexName string, ) { - idx, err := tableDesc.FindIndexWithName(shardedIndexName) + idx, _, err := tableDesc.FindIndexByName(shardedIndexName) if err != nil { t.Fatal(err) } @@ -73,7 +73,7 @@ func verifyTableDescriptorState( if !foundCheckConstraint { t.Fatalf(`Could not find hidden check constraint for shard column`) } - if idx.GetColumnID(0) != shardColID { + if idx.ColumnIDs[0] != shardColID { t.Fatalf(`Expected shard column to be the first column in the set of index columns`) } } @@ -113,12 +113,12 @@ func TestBasicHashShardedIndexes(t *testing.T) { // Ensure that secondary indexes on table `kv` have the shard column in their // `ExtraColumnIDs` field so they can reconstruct the sharded primary key. - foo, err := tableDesc.FindIndexWithName("foo") + fooDesc, _, err := tableDesc.FindIndexByName("foo") if err != nil { t.Fatal(err) } foundShardColumn := false - for _, colID := range foo.IndexDesc().ExtraColumnIDs { + for _, colID := range fooDesc.ExtraColumnIDs { if colID == shardColID { foundShardColumn = true break diff --git a/pkg/sql/truncate.go b/pkg/sql/truncate.go index d1b54b0b40f6..43a711530bd1 100644 --- a/pkg/sql/truncate.go +++ b/pkg/sql/truncate.go @@ -132,9 +132,8 @@ func (t *truncateNode) startExec(params runParams) error { return err } } - for _, idx := range tableDesc.NonDropIndexes() { - for i := 0; i < idx.NumInterleavedBy(); i++ { - ref := idx.GetInterleavedBy(i) + for _, idx := range tableDesc.AllNonDropIndexes() { + for _, ref := range idx.InterleavedBy { if err := maybeEnqueue(ref.Table, "interleaved by"); err != nil { return err } @@ -194,19 +193,24 @@ func (p *planner) truncateTable( return err } - // Collect all of the old indexes and reset all of the index IDs. - oldIndexes := make([]descpb.IndexDescriptor, len(tableDesc.ActiveIndexes())) - for _, idx := range tableDesc.ActiveIndexes() { - oldIndexes[idx.Ordinal()] = idx.IndexDescDeepCopy() - newIndex := *idx.IndexDesc() - newIndex.ID = descpb.IndexID(0) - if idx.Primary() { - tableDesc.SetPrimaryIndex(newIndex) - } else { - tableDesc.SetPublicNonPrimaryIndex(idx.Ordinal(), newIndex) - } + // Collect all of the old indexes. + oldIndexes := make([]descpb.IndexDescriptor, len(tableDesc.GetPublicNonPrimaryIndexes())+1) + oldIndexes[0] = *protoutil.Clone(tableDesc.GetPrimaryIndex()).(*descpb.IndexDescriptor) + for i := range tableDesc.GetPublicNonPrimaryIndexes() { + oldIndexes[i+1] = *protoutil.Clone(&tableDesc.GetPublicNonPrimaryIndexes()[i]).(*descpb.IndexDescriptor) } + // Reset all of the index IDs. + { + primaryIndex := *tableDesc.GetPrimaryIndex() + primaryIndex.ID = descpb.IndexID(0) + tableDesc.SetPrimaryIndex(primaryIndex) + } + + for i, index := range tableDesc.GetPublicNonPrimaryIndexes() { + index.ID = descpb.IndexID(0) + tableDesc.SetPublicNonPrimaryIndex(i+1, index) + } // Create new ID's for all of the indexes in the table. if err := tableDesc.AllocateIDs(ctx); err != nil { return err @@ -214,8 +218,9 @@ func (p *planner) truncateTable( // Construct a mapping from old index ID's to new index ID's. indexIDMapping := make(map[descpb.IndexID]descpb.IndexID, len(oldIndexes)) - for _, idx := range tableDesc.ActiveIndexes() { - indexIDMapping[oldIndexes[idx.Ordinal()].ID] = idx.GetID() + indexIDMapping[oldIndexes[0].ID] = tableDesc.GetPrimaryIndexID() + for i := range tableDesc.GetPublicNonPrimaryIndexes() { + indexIDMapping[oldIndexes[i+1].ID] = tableDesc.GetPublicNonPrimaryIndexes()[i].ID } // Resolve all outstanding mutations. Make all new schema elements @@ -283,9 +288,9 @@ func (p *planner) truncateTable( for i := range oldIndexIDs { oldIndexIDs[i] = oldIndexes[i+1].ID } - newIndexIDs := make([]descpb.IndexID, len(tableDesc.PublicNonPrimaryIndexes())) + newIndexIDs := make([]descpb.IndexID, len(tableDesc.GetPublicNonPrimaryIndexes())) for i := range newIndexIDs { - newIndexIDs[i] = tableDesc.PublicNonPrimaryIndexes()[i].GetID() + newIndexIDs[i] = tableDesc.GetPublicNonPrimaryIndexes()[i].ID } swapInfo := &descpb.PrimaryKeySwap{ OldPrimaryIndexId: oldIndexes[0].ID, @@ -376,8 +381,7 @@ func (p *planner) reassignInterleaveIndexReferences( ) error { for _, table := range tables { changed := false - if err := catalog.ForEachNonDropIndex(table, func(indexI catalog.Index) error { - index := indexI.IndexDesc() + if err := table.ForeachNonDropIndex(func(index *descpb.IndexDescriptor) error { for j, a := range index.Interleave.Ancestors { if a.TableID == truncatedID { index.Interleave.Ancestors[j].IndexID = indexIDMapping[index.Interleave.Ancestors[j].IndexID] diff --git a/pkg/sql/update.go b/pkg/sql/update.go index 48ea5cf4a3ae..2a6097e253a6 100644 --- a/pkg/sql/update.go +++ b/pkg/sql/update.go @@ -296,11 +296,12 @@ func (u *updateNode) processSourceRow(params runParams, sourceVals tree.Datums) // Create a set of partial index IDs to not add entries or remove entries // from. var pm row.PartialIndexUpdateHelper - if n := len(u.run.tu.tableDesc().PartialIndexes()); n > 0 { + partialIndexOrds := u.run.tu.tableDesc().PartialIndexOrds() + if !partialIndexOrds.Empty() { partialIndexValOffset := len(u.run.tu.ru.FetchCols) + len(u.run.tu.ru.UpdateCols) + u.run.checkOrds.Len() + u.run.numPassthrough partialIndexVals := sourceVals[partialIndexValOffset:] - partialIndexPutVals := partialIndexVals[:n] - partialIndexDelVals := partialIndexVals[n : n*2] + partialIndexPutVals := partialIndexVals[:partialIndexOrds.Len()] + partialIndexDelVals := partialIndexVals[partialIndexOrds.Len() : partialIndexOrds.Len()*2] err := pm.Init(partialIndexPutVals, partialIndexDelVals, u.run.tu.tableDesc()) if err != nil { diff --git a/pkg/sql/upsert.go b/pkg/sql/upsert.go index 7ebef9e6b3f3..cc2741e8ec23 100644 --- a/pkg/sql/upsert.go +++ b/pkg/sql/upsert.go @@ -143,7 +143,8 @@ func (n *upsertNode) processSourceRow(params runParams, rowVals tree.Datums) err // Create a set of partial index IDs to not add or remove entries from. var pm row.PartialIndexUpdateHelper - if len(n.run.tw.tableDesc().PartialIndexes()) > 0 { + partialIndexOrds := n.run.tw.tableDesc().PartialIndexOrds() + if !partialIndexOrds.Empty() { partialIndexValOffset := len(n.run.insertCols) + len(n.run.tw.fetchCols) + len(n.run.tw.updateCols) + n.run.checkOrds.Len() if n.run.tw.canaryOrdinal != -1 { partialIndexValOffset++ diff --git a/pkg/sql/virtual_schema.go b/pkg/sql/virtual_schema.go index 1ea8ff00f0d0..7f1614c738f5 100644 --- a/pkg/sql/virtual_schema.go +++ b/pkg/sql/virtual_schema.go @@ -176,11 +176,10 @@ func (t virtualSchemaTable) initVirtualTableDesc( if err != nil { return mutDesc.TableDescriptor, err } - for _, index := range mutDesc.PublicNonPrimaryIndexes() { - if index.NumColumns() > 1 { + for i, idx := range mutDesc.GetPublicNonPrimaryIndexes() { + if len(idx.ColumnIDs) > 1 { panic("we don't know how to deal with virtual composite indexes yet") } - idx := *index.IndexDesc() // All indexes of virtual tables automatically STORE all other columns in // the table. idx.StoreColumnIDs = make([]descpb.ColumnID, len(mutDesc.Columns)-len(idx.ColumnIDs)) @@ -199,7 +198,7 @@ func (t virtualSchemaTable) initVirtualTableDesc( idx.StoreColumnNames[outputIdx] = mutDesc.Columns[j].Name outputIdx++ } - mutDesc.SetPublicNonPrimaryIndex(index.Ordinal(), idx) + mutDesc.SetPublicNonPrimaryIndex(i+1, idx) } return mutDesc.TableDescriptor, nil } diff --git a/pkg/sql/zone_config.go b/pkg/sql/zone_config.go index db8aa866cba5..014e4faeec69 100644 --- a/pkg/sql/zone_config.go +++ b/pkg/sql/zone_config.go @@ -315,13 +315,13 @@ func resolveSubzone( } indexName := string(zs.TableOrIndex.Index) - var index catalog.Index + var index *descpb.IndexDescriptor if indexName == "" { index = table.GetPrimaryIndex() - indexName = index.GetName() + indexName = index.Name } else { var err error - index, err = table.FindIndexWithName(indexName) + index, _, err = table.FindIndexByName(indexName) if err != nil { return nil, "", err } @@ -329,12 +329,12 @@ func resolveSubzone( partitionName := string(zs.Partition) if partitionName != "" { - if partitioning := tabledesc.FindIndexPartitionByName(index.IndexDesc(), partitionName); partitioning == nil { + if partitioning := tabledesc.FindIndexPartitionByName(index, partitionName); partitioning == nil { return nil, "", fmt.Errorf("partition %q does not exist on index %q", partitionName, indexName) } } - return index.IndexDesc(), partitionName, nil + return index, partitionName, nil } func deleteRemovedPartitionZoneConfigs(