diff --git a/pkg/ccl/backupccl/backup_planning.go b/pkg/ccl/backupccl/backup_planning.go index 668bf5dbf0fd..4276f65345d5 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 := table.ForeachNonDropIndex(func(idxDesc *descpb.IndexDescriptor) error { - key := tableAndIndex{tableID: table.GetID(), indexID: idxDesc.ID} + if err := catalog.ForEachNonDropIndex(table, func(idx catalog.Index) error { + key := tableAndIndex{tableID: table.GetID(), indexID: idx.GetID()} if added[key] { return nil } added[key] = true - nonDropIndexIDs = append(nonDropIndexIDs, idxDesc.ID) + nonDropIndexIDs = append(nonDropIndexIDs, idx.GetID()) 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.FindIndexByID(lhsIndexID) + lhsIndex, err := table.FindIndexWithID(lhsIndexID) if err != nil { return nil, err } - rhsIndex, err := table.FindIndexByID(rhsIndexID) + rhsIndex, err := table.FindIndexWithID(rhsIndexID) if err != nil { return nil, err } diff --git a/pkg/ccl/backupccl/restore_job.go b/pkg/ccl/backupccl/restore_job.go index 105db55cbd7a..1c805ce45207 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.AllNonDropIndexes() { - if err := sstIntervalTree.Insert(intervalSpan(table.IndexSpan(codec, index.ID)), false); err != nil { + for _, index := range table.NonDropIndexes() { + if err := sstIntervalTree.Insert(intervalSpan(table.IndexSpan(codec, index.GetID())), false); err != nil { panic(errors.NewAssertionErrorWithWrappedErrf(err, "IndexSpan")) } - added[tableAndIndex{tableID: table.GetID(), indexID: index.ID}] = true + added[tableAndIndex{tableID: table.GetID(), indexID: index.GetID()}] = 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.AllNonDropIndexes() { - key := tableAndIndex{tableID: tbl.ID, indexID: idx.ID} + for _, idx := range tbl.NonDropIndexes() { + key := tableAndIndex{tableID: tbl.ID, indexID: idx.GetID()} if !added[key] { - if err := sstIntervalTree.Insert(intervalSpan(tbl.IndexSpan(codec, idx.ID)), false); err != nil { + if err := sstIntervalTree.Insert(intervalSpan(tbl.IndexSpan(codec, idx.GetID())), 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 11180465d8dd..faf3da356b85 100644 --- a/pkg/ccl/backupccl/restore_planning.go +++ b/pkg/ccl/backupccl/restore_planning.go @@ -1021,7 +1021,8 @@ func RewriteTableDescs( return err } - if err := table.ForeachNonDropIndex(func(index *descpb.IndexDescriptor) error { + if err := catalog.ForEachNonDropIndex(table, func(indexI catalog.Index) error { + index := indexI.IndexDesc() // 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 9ad845ec076c..15cd4b683fec 100644 --- a/pkg/ccl/backupccl/targets.go +++ b/pkg/ccl/backupccl/targets.go @@ -668,17 +668,19 @@ func ensureInterleavesIncluded(tables []catalog.TableDescriptor) error { } for _, table := range tables { - if err := table.ForeachIndex(catalog.IndexOpts{ + if err := catalog.ForEachIndex(table, catalog.IndexOpts{ AddMutations: true, - }, func(index *descpb.IndexDescriptor, _ bool) error { - for _, a := range index.Interleave.Ancestors { + }, func(index catalog.Index) error { + for i := 0; i < index.NumInterleaveAncestors(); i++ { + a := index.GetInterleaveAncestor(i) if !inBackup[a.TableID] { return errors.Errorf( "cannot backup table %q without interleave parent (ID %d)", table.GetName(), a.TableID, ) } } - for _, c := range index.InterleavedBy { + for i := 0; i < index.NumInterleavedBy(); i++ { + c := index.GetInterleavedBy(i) 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 1fb6cdb3e690..612f2e6853aa 100644 --- a/pkg/ccl/changefeedccl/avro_test.go +++ b/pkg/ccl/changefeedccl/avro_test.go @@ -285,7 +285,7 @@ func TestAvroSchema(t *testing.T) { `{"type":["null","long"],"name":"_u0001f366_","default":null,`+ `"__crdb__":"🍦 INT8 NOT NULL"}]}`, tableSchema.codec.Schema()) - indexSchema, err := indexToAvroSchema(tableDesc, tableDesc.GetPrimaryIndex()) + indexSchema, err := indexToAvroSchema(tableDesc, tableDesc.GetPrimaryIndex().IndexDesc()) require.NoError(t, err) require.Equal(t, `{"type":"record","name":"_u2603_","fields":[`+ diff --git a/pkg/ccl/changefeedccl/encoder.go b/pkg/ccl/changefeedccl/encoder.go index d4028a232997..e49f13a3a8d5 100644 --- a/pkg/ccl/changefeedccl/encoder.go +++ b/pkg/ccl/changefeedccl/encoder.go @@ -142,8 +142,10 @@ func (e *jsonEncoder) EncodeKey(_ context.Context, row encodeRow) ([]byte, error func (e *jsonEncoder) encodeKeyRaw(row encodeRow) ([]interface{}, error) { colIdxByID := row.tableDesc.ColumnIdxMap() - jsonEntries := make([]interface{}, len(row.tableDesc.GetPrimaryIndex().ColumnIDs)) - for i, colID := range row.tableDesc.GetPrimaryIndex().ColumnIDs { + primaryIndex := row.tableDesc.GetPrimaryIndex() + jsonEntries := make([]interface{}, primaryIndex.NumColumns()) + for i := 0; i < primaryIndex.NumColumns(); i++ { + colID := primaryIndex.GetColumnID(i) idx, ok := colIdxByID.Get(colID) if !ok { return nil, errors.Errorf(`unknown column id: %d`, colID) @@ -342,7 +344,7 @@ func (e *confluentAvroEncoder) EncodeKey(ctx context.Context, row encodeRow) ([] registered, ok := e.keyCache[cacheKey] if !ok { var err error - registered.schema, err = indexToAvroSchema(row.tableDesc, row.tableDesc.GetPrimaryIndex()) + registered.schema, err = indexToAvroSchema(row.tableDesc, row.tableDesc.GetPrimaryIndex().IndexDesc()) if err != nil { return nil, err } diff --git a/pkg/ccl/changefeedccl/rowfetcher_cache.go b/pkg/ccl/changefeedccl/rowfetcher_cache.go index 4d1146e2f388..15ec48f390e1 100644 --- a/pkg/ccl/changefeedccl/rowfetcher_cache.go +++ b/pkg/ccl/changefeedccl/rowfetcher_cache.go @@ -121,7 +121,7 @@ func (c *rowFetcherCache) TableDescForKey( } // Skip over the column data. - for ; skippedCols < len(tableDesc.GetPrimaryIndex().ColumnIDs); skippedCols++ { + for ; skippedCols < tableDesc.GetPrimaryIndex().NumColumns(); skippedCols++ { l, err := encoding.PeekLength(remaining) if err != nil { return nil, err @@ -173,7 +173,7 @@ func (c *rowFetcherCache) RowFetcherForTableDesc( row.FetcherTableArgs{ Spans: tableDesc.AllIndexSpans(c.codec), Desc: tableDesc, - Index: tableDesc.GetPrimaryIndex(), + Index: tableDesc.GetPrimaryIndex().IndexDesc(), ColIdxMap: colIdxMap, IsSecondaryIndex: false, Cols: tableDesc.Columns, diff --git a/pkg/ccl/importccl/import_processor.go b/pkg/ccl/importccl/import_processor.go index 4ebe56aa50e3..9da93ace868f 100644 --- a/pkg/ccl/importccl/import_processor.go +++ b/pkg/ccl/importccl/import_processor.go @@ -17,6 +17,7 @@ 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" @@ -190,11 +191,10 @@ func makeInputConverter( } if singleTable != nil { - indexes := singleTable.DeletableIndexes() - for _, idx := range indexes { - if idx.IsPartial() { - return nil, unimplemented.NewWithIssue(50225, "cannot import into table with partial indexes") - } + 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") } // 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 19fa3028bb72..4fc714521c88 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") - indexDesc, _, err := tableDesc.FindIndexByName("i") + index, err := tableDesc.FindIndexWithName("i") if err != nil { t.Fatal(err) } - indexSpan := tableDesc.IndexSpan(keys.SystemSQLCodec, indexDesc.ID) + indexSpan := tableDesc.IndexSpan(keys.SystemSQLCodec, index.GetID()) 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.FindIndexByName("i"); err == nil { + if _, err := tableDesc.FindIndexWithName("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 a9119a9b7169..5b3941281c89 100644 --- a/pkg/ccl/partitionccl/partition.go +++ b/pkg/ccl/partitionccl/partition.go @@ -275,12 +275,11 @@ func selectPartitionExprs( a := &rowenc.DatumAlloc{} var prefixDatums []tree.Datum - if err := tableDesc.ForeachIndex(catalog.IndexOpts{ + if err := catalog.ForEachIndex(tableDesc, catalog.IndexOpts{ AddMutations: true, - }, func(idxDesc *descpb.IndexDescriptor, _ bool) error { - genExpr := true + }, func(idx catalog.Index) error { return selectPartitionExprsByName( - a, evalCtx, tableDesc, idxDesc, &idxDesc.Partitioning, prefixDatums, exprsByPartName, genExpr) + a, evalCtx, tableDesc, idx.IndexDesc(), &idx.IndexDesc().Partitioning, prefixDatums, exprsByPartName, true /* genExpr */) }); err != nil { return nil, err } diff --git a/pkg/ccl/partitionccl/partition_test.go b/pkg/ccl/partitionccl/partition_test.go index c63d395e1d97..a48ccc3fb810 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)) } - idxDesc, _, err := pt.parsed.tableDesc.FindIndexByName(indexName[1:]) + idx, err := pt.parsed.tableDesc.FindIndexWithName(indexName[1:]) if err != nil { return errors.Wrapf(err, "could not find index %s", indexName) } - subzone.IndexID = uint32(idxDesc.ID) + subzone.IndexID = uint32(idx.GetID()) if len(constraints) > 0 { if subzone.PartitionName == "" { fmt.Fprintf(&zoneConfigStmts, `ALTER INDEX %s@%s CONFIGURE ZONE USING constraints = '[%s]';`, - pt.parsed.tableName, idxDesc.Name, constraints, + pt.parsed.tableName, idx.GetName(), constraints, ) } else { fmt.Fprintf(&zoneConfigStmts, `ALTER PARTITION %s OF INDEX %s@%s CONFIGURE ZONE USING constraints = '[%s]';`, - subzone.PartitionName, pt.parsed.tableName, idxDesc.Name, constraints, + subzone.PartitionName, pt.parsed.tableName, idx.GetName(), 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.FindIndexByName(test.index) + testIndex, err := test.new.parsed.tableDesc.FindIndexWithName(test.index) if err != nil { t.Fatalf("%+v", err) } var repartition bytes.Buffer - if testIndex.ID == test.new.parsed.tableDesc.GetPrimaryIndexID() { + if testIndex.GetID() == 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.Name) + fmt.Fprintf(&repartition, `ALTER INDEX %s@%s `, test.new.parsed.tableName, testIndex.GetName()) } - if testIndex.Partitioning.NumColumns == 0 { + if testIndex.GetPartitioning().NumColumns == 0 { repartition.WriteString(`PARTITION BY NOTHING`) } else { if err := sql.ShowCreatePartitioning( - &rowenc.DatumAlloc{}, keys.SystemSQLCodec, test.new.parsed.tableDesc, testIndex, - &testIndex.Partitioning, &repartition, 0 /* indent */, 0, /* colOffset */ + &rowenc.DatumAlloc{}, keys.SystemSQLCodec, test.new.parsed.tableDesc, testIndex.IndexDesc(), + &testIndex.IndexDesc().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 2e3190f3f8e1..86c9b1e19182 100644 --- a/pkg/ccl/partitionccl/zone_test.go +++ b/pkg/ccl/partitionccl/zone_test.go @@ -301,13 +301,14 @@ func TestGenerateSubzoneSpans(t *testing.T) { var actual []string for _, span := range spans { subzone := test.parsed.subzones[span.SubzoneIndex] - idxDesc, err := test.parsed.tableDesc.FindIndexByID(descpb.IndexID(subzone.IndexID)) + idx, err := test.parsed.tableDesc.FindIndexWithID(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 _, cd := range idxDesc.ColumnDirections { + for i := 0; i < idx.NumColumns(); i++ { + cd := idx.GetColumnDirection(i) ed, err := cd.ToEncodingDirection() if err != nil { t.Fatal(err) @@ -319,7 +320,7 @@ func TestGenerateSubzoneSpans(t *testing.T) { if len(subzone.PartitionName) > 0 { subzoneShort = "." + subzone.PartitionName } else { - subzoneShort = "@" + idxDesc.Name + subzoneShort = "@" + idx.GetName() } // 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 a44734e71763..c12b906fbd9e 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.AllNonDropIndexes() { - oldPrefix := roachpb.Key(makeKeyRewriterPrefixIgnoringInterleaved(oldID, index.ID)) - newPrefix := roachpb.Key(makeKeyRewriterPrefixIgnoringInterleaved(desc.ID, index.ID)) + for _, index := range desc.NonDropIndexes() { + oldPrefix := roachpb.Key(makeKeyRewriterPrefixIgnoringInterleaved(oldID, index.GetID())) + newPrefix := roachpb.Key(makeKeyRewriterPrefixIgnoringInterleaved(desc.ID, index.GetID())) 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.FindIndexByID(descpb.IndexID(indexID)) + idx, err := desc.FindIndexWithID(descpb.IndexID(indexID)) if err != nil { return nil, false, err } - if len(idx.InterleavedBy) == 0 { + if idx.NumInterleavedBy() == 0 { // Not interleaved. return key, true, nil } // We do not support interleaved secondary indexes. - if idx.ID != desc.GetPrimaryIndexID() { + if !idx.Primary() { return nil, false, errors.New("restoring interleaved secondary indexes not supported") } - colIDs, _ := idx.FullColumnIDs() + colIDs, _ := idx.IndexDesc().FullColumnIDs() var skipCols int - for _, ancestor := range idx.Interleave.Ancestors { - skipCols += int(ancestor.SharedPrefixLen) + for i := 0; i < idx.NumInterleaveAncestors(); i++ { + skipCols += int(idx.GetInterleaveAncestor(i).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 17ed42db5dac..9d099ee31457 100644 --- a/pkg/server/settingsworker.go +++ b/pkg/server/settingsworker.go @@ -50,7 +50,7 @@ func processSystemConfigKVs( { types := []*types.T{tbl.Columns[0].Type} nameRow := make([]rowenc.EncDatum, 1) - _, matches, _, err := rowenc.DecodeIndexKey(codec, tbl, tbl.GetPrimaryIndex(), types, nameRow, nil, kv.Key) + _, matches, _, err := rowenc.DecodeIndexKey(codec, tbl, tbl.GetPrimaryIndex().IndexDesc(), types, nameRow, nil, kv.Key) if err != nil { return errors.Wrap(err, "failed to decode key") } diff --git a/pkg/sql/alter_column_type.go b/pkg/sql/alter_column_type.go index b134a4e0d80c..5e9d1e84b69b 100644 --- a/pkg/sql/alter_column_type.go +++ b/pkg/sql/alter_column_type.go @@ -220,9 +220,14 @@ func alterColumnTypeGeneral( // Disallow ALTER COLUMN TYPE general for columns that are // part of indexes. - for _, idx := range tableDesc.AllNonDropIndexes() { - for _, id := range append(idx.ColumnIDs, idx.ExtraColumnIDs...) { - if col.ID == id { + 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 { return colInIndexNotSupportedErr } } diff --git a/pkg/sql/alter_index.go b/pkg/sql/alter_index.go index 73db4001448a..e6e04c7a0d1c 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. - indexDesc, err = tableDesc.FindIndexByID(indexDesc.ID) + index, err := tableDesc.FindIndexWithID(indexDesc.ID) if err != nil { return nil, err } - return &alterIndexNode{n: n, tableDesc: tableDesc, indexDesc: indexDesc}, nil + return &alterIndexNode{n: n, tableDesc: tableDesc, indexDesc: index.IndexDesc()}, nil } // ReadingOwnWrites implements the planNodeReadingOwnWrites interface. diff --git a/pkg/sql/alter_primary_key.go b/pkg/sql/alter_primary_key.go index 70fd900f7f55..9f4a2777418d 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 len(tableDesc.GetPrimaryIndex().InterleavedBy) != 0 { + if tableDesc.GetPrimaryIndex().NumInterleavedBy() != 0 { var sb strings.Builder sb.WriteString("[") comma := ", " - for i := range tableDesc.GetPrimaryIndex().InterleavedBy { - interleave := &tableDesc.GetPrimaryIndex().InterleavedBy[i] + for i := 0; i < tableDesc.GetPrimaryIndex().NumInterleavedBy(); i++ { + interleaveTableID := tableDesc.GetPrimaryIndex().GetInterleavedBy(i).Table if i != 0 { sb.WriteString(comma) } childTable, err := p.Descriptors().GetTableVersionByID( ctx, p.Txn(), - interleave.Table, + interleaveTableID, tree.ObjectLookupFlags{}, ) if err != nil { @@ -120,7 +120,7 @@ func (p *planner) AlterPrimaryKey( } nameExists := func(name string) bool { - _, _, err := tableDesc.FindIndexByName(name) + _, err := tableDesc.FindIndexWithName(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 := protoutil.Clone(tableDesc.GetPrimaryIndex()).(*descpb.IndexDescriptor) + oldPrimaryIndexCopy := tableDesc.GetPrimaryIndex().IndexDescDeepCopy() // 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,10 +261,9 @@ func (p *planner) AlterPrimaryKey( return shouldRewrite || !idx.Unique || idx.Type == descpb.IndexDescriptor_INVERTED } var indexesToRewrite []*descpb.IndexDescriptor - for i := range tableDesc.GetPublicNonPrimaryIndexes() { - idx := &tableDesc.GetPublicNonPrimaryIndexes()[i] - if idx.ID != newPrimaryIndexDesc.ID && shouldRewriteIndex(idx) { - indexesToRewrite = append(indexesToRewrite, idx) + for _, idx := range tableDesc.PublicNonPrimaryIndexes() { + if idx.GetID() != newPrimaryIndexDesc.ID && shouldRewriteIndex(idx.IndexDesc()) { + indexesToRewrite = append(indexesToRewrite, idx.IndexDesc()) } } @@ -315,7 +314,7 @@ func (p *planner) AlterPrimaryKey( // Mark the primary key of the table as valid. { - primaryIndex := *tableDesc.GetPrimaryIndex() + primaryIndex := *tableDesc.GetPrimaryIndex().IndexDesc() primaryIndex.Disabled = false tableDesc.SetPrimaryIndex(primaryIndex) } @@ -355,7 +354,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.ColumnIDs[1:]).Equals(newPK.ColumnIDs[1:]) { + descpb.ColumnIDs(oldPK.IndexDesc().ColumnIDs[1:]).Equals(newPK.ColumnIDs[1:]) { return false } diff --git a/pkg/sql/alter_table.go b/pkg/sql/alter_table.go index fd2f2cc7a672..6d4e170bdb40 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 } - _, dropped, err := n.tableDesc.FindIndexByName(string(d.Name)) + foundIndex, err := n.tableDesc.FindIndexWithName(string(d.Name)) if err == nil { - if dropped { + if foundIndex.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.AllNonDropIndexes() { + for _, idx := range n.tableDesc.NonDropIndexes() { // We automatically drop indexes that reference the column // being dropped. @@ -450,14 +450,15 @@ func (n *alterTableNode) startExec(params runParams) error { containsThisColumn := false // Analyze the index. - for _, id := range idx.ColumnIDs { - if id == colToDrop.ID { + for j := 0; j < idx.NumColumns(); j++ { + if idx.GetColumnID(j) == colToDrop.ID { containsThisColumn = true break } } if !containsThisColumn { - for _, id := range idx.ExtraColumnIDs { + for j := 0; j < idx.NumExtraColumns(); j++ { + id := idx.GetExtraColumnID(j) if n.tableDesc.GetPrimaryIndex().ContainsColumnID(id) { // All secondary indices necessary contain the PK // columns, too. (See the comments on the definition of @@ -476,8 +477,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 _, id := range idx.StoreColumnIDs { - if id == colToDrop.ID { + for j := 0; j < idx.NumStoredColumns(); j++ { + if idx.GetStoredColumnID(j) == colToDrop.ID { containsThisColumn = true break } @@ -487,7 +488,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.Predicate) + expr, err := parser.ParseExpr(idx.GetPredicate()) if err != nil { return err } @@ -504,7 +505,7 @@ func (n *alterTableNode) startExec(params runParams) error { // Perform the DROP. if containsThisColumn { - idxNamesToDelete = append(idxNamesToDelete, idx.Name) + idxNamesToDelete = append(idxNamesToDelete, idx.GetName()) } } @@ -739,26 +740,28 @@ func (n *alterTableNode) startExec(params runParams) error { descriptorChanged = true case *tree.AlterTablePartitionBy: - partitioning, err := CreatePartitioning( + primaryIndex := n.tableDesc.GetPrimaryIndex() + newPartitioning, err := CreatePartitioning( params.ctx, params.p.ExecCfg().Settings, params.EvalContext(), - n.tableDesc, n.tableDesc.GetPrimaryIndex(), t.PartitionBy) + n.tableDesc, primaryIndex.IndexDesc(), t.PartitionBy) if err != nil { return err } - descriptorChanged = descriptorChanged || !n.tableDesc.GetPrimaryIndex().Partitioning.Equal(&partitioning) + oldPartitioning := primaryIndex.GetPartitioning() + descriptorChanged = descriptorChanged || !oldPartitioning.Equal(&newPartitioning) err = deleteRemovedPartitionZoneConfigs( params.ctx, params.p.txn, - n.tableDesc, n.tableDesc.GetPrimaryIndex(), &n.tableDesc.GetPrimaryIndex().Partitioning, - &partitioning, params.extendedEvalCtx.ExecCfg, + n.tableDesc, primaryIndex.IndexDesc(), &oldPartitioning, + &newPartitioning, params.extendedEvalCtx.ExecCfg, ) if err != nil { return err } { - primaryIndex := *n.tableDesc.GetPrimaryIndex() - primaryIndex.Partitioning = partitioning - n.tableDesc.SetPrimaryIndex(primaryIndex) + newPrimaryIndex := *primaryIndex.IndexDesc() + newPrimaryIndex.Partitioning = newPartitioning + n.tableDesc.SetPrimaryIndex(newPrimaryIndex) } case *tree.AlterTableSetAudit: @@ -813,16 +816,11 @@ func (n *alterTableNode) startExec(params runParams) error { // new name exists. This is what postgres does. switch details.Kind { case descpb.ConstraintTypeUnique, descpb.ConstraintTypePK: - 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 + 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) } } diff --git a/pkg/sql/backfill.go b/pkg/sql/backfill.go index a15711fd4004..08401ad1eedf 100644 --- a/pkg/sql/backfill.go +++ b/pkg/sql/backfill.go @@ -1648,27 +1648,28 @@ func runSchemaChangesInTxn( // write the modified table descriptors explicitly. for _, idxID := range append( []descpb.IndexID{pkSwap.OldPrimaryIndexId}, pkSwap.OldIndexes...) { - oldIndex, err := tableDesc.FindIndexByID(idxID) + oldIndex, err := tableDesc.FindIndexWithID(idxID) if err != nil { return err } - if len(oldIndex.Interleave.Ancestors) != 0 { - ancestorInfo := oldIndex.Interleave.Ancestors[len(oldIndex.Interleave.Ancestors)-1] + if oldIndex.NumInterleaveAncestors() != 0 { + ancestorInfo := oldIndex.GetInterleaveAncestor(oldIndex.NumInterleaveAncestors() - 1) ancestor, err := planner.Descriptors().GetMutableTableVersionByID(ctx, ancestorInfo.TableID, planner.txn) if err != nil { return err } - ancestorIdx, err := ancestor.FindIndexByID(ancestorInfo.IndexID) + ancestorIdxI, err := ancestor.FindIndexWithID(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.ID { + if ref.Table == tableDesc.ID && ref.Index == oldIndex.GetID() { if foundAncestor { return errors.AssertionFailedf( "ancestor entry in %s for %s@%s found more than once", - ancestor.Name, tableDesc.Name, oldIndex.Name) + ancestor.Name, tableDesc.Name, oldIndex.GetName()) } 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 030cab6a678a..4812f6a2b016 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(), + Index: desc.GetPrimaryIndex().IndexDesc(), ColIdxMap: desc.ColumnIdxMap(), Cols: desc.Columns, ValNeededForCol: valNeededForCol, @@ -602,7 +602,7 @@ func (ib *IndexBackfiller) init( tableArgs := row.FetcherTableArgs{ Desc: desc, - Index: desc.GetPrimaryIndex(), + Index: desc.GetPrimaryIndex().IndexDesc(), ColIdxMap: ib.colIdxMap, Cols: ib.cols, ValNeededForCol: valNeededForCol, diff --git a/pkg/sql/catalog/BUILD.bazel b/pkg/sql/catalog/BUILD.bazel index 7f26b7ae03db..36575a9cefe6 100644 --- a/pkg/sql/catalog/BUILD.bazel +++ b/pkg/sql/catalog/BUILD.bazel @@ -14,6 +14,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/sql/catalog", visibility = ["//visibility:public"], deps = [ + "//pkg/geo/geoindex", "//pkg/keys", "//pkg/kv", "//pkg/roachpb", @@ -25,6 +26,7 @@ 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 e21607bf954f..cc5001dfe33c 100644 --- a/pkg/sql/catalog/descriptor.go +++ b/pkg/sql/catalog/descriptor.go @@ -13,6 +13,7 @@ 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" @@ -20,13 +21,14 @@ 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 TableDescriptor.ForeachIndex. +// IndexOpts configures the behavior of catalog.ForEachIndex and +// catalog.FindIndex. type IndexOpts struct { // NonPhysicalPrimaryIndex should be included. NonPhysicalPrimaryIndex bool @@ -112,22 +114,79 @@ type TableDescriptor interface { GetFormatVersion() descpb.FormatVersion GetPrimaryIndexID() descpb.IndexID - GetPrimaryIndex() *descpb.IndexDescriptor + GetPrimaryIndex() Index 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) - PartialIndexOrds() util.FastIntSet - WritableIndexes() []descpb.IndexDescriptor - DeletableIndexes() []descpb.IndexDescriptor - DeleteOnlyIndexes() []descpb.IndexDescriptor + + // 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) HasPrimaryKey() bool PrimaryKeyString() string @@ -187,6 +246,106 @@ 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 { @@ -282,3 +441,151 @@ 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 a107a91d4188..69ed02683e20 100644 --- a/pkg/sql/catalog/tabledesc/BUILD.bazel +++ b/pkg/sql/catalog/tabledesc/BUILD.bazel @@ -3,6 +3,7 @@ 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", @@ -11,6 +12,7 @@ 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", @@ -43,6 +45,7 @@ go_test( name = "tabledesc_test", srcs = [ "helpers_test.go", + "index_test.go", "main_test.go", "safe_format_test.go", "structured_test.go", @@ -52,6 +55,7 @@ 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 new file mode 100644 index 000000000000..e13cc7fcfe5a --- /dev/null +++ b/pkg/sql/catalog/tabledesc/index.go @@ -0,0 +1,527 @@ +// 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 new file mode 100644 index 000000000000..810d5902a49a --- /dev/null +++ b/pkg/sql/catalog/tabledesc/index_test.go @@ -0,0 +1,319 @@ +// 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 ee54864a27a6..db975755e374 100644 --- a/pkg/sql/catalog/tabledesc/safe_format.go +++ b/pkg/sql/catalog/tabledesc/safe_format.go @@ -110,11 +110,13 @@ func formatSafeTableIndexes(w *redact.StringBuilder, desc catalog.TableDescripto w.Printf(", PrimaryIndex: %d", desc.GetPrimaryIndexID()) w.Printf(", NextIndexID: %d", desc.TableDesc().NextIndexID) w.Printf(", Indexes: [") - formatSafeIndex(w, desc.GetPrimaryIndex(), nil) - for i := range desc.GetPublicNonPrimaryIndexes() { - w.Printf(", ") - formatSafeIndex(w, &desc.GetPublicNonPrimaryIndexes()[i], nil) - } + _ = catalog.ForEachActiveIndex(desc, func(idx catalog.Index) error { + if !idx.Primary() { + w.Printf(", ") + } + formatSafeIndex(w, idx.IndexDesc(), nil) + return nil + }) w.Printf("]") } diff --git a/pkg/sql/catalog/tabledesc/structured.go b/pkg/sql/catalog/tabledesc/structured.go index 0bf4321506dc..1ad6d78a04be 100644 --- a/pkg/sql/catalog/tabledesc/structured.go +++ b/pkg/sql/catalog/tabledesc/structured.go @@ -98,31 +98,27 @@ func NewFilledInExistingMutable( skipFKsWithMissingTable bool, tbl *descpb.TableDescriptor, ) (*Mutable, error) { - desc, err := makeFilledInImmutable(ctx, dg, tbl, skipFKsWithMissingTable) + changes, err := maybeFillInDescriptor(ctx, dg, tbl, skipFKsWithMissingTable) if err != nil { return nil, err } - m := &Mutable{wrapper: desc.wrapper} - m.ClusterVersion = *tbl - return m, nil + w := wrapper{TableDescriptor: *tbl, postDeserializationChanges: changes} + return &Mutable{wrapper: w, ClusterVersion: *tbl}, 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}} + desc := Immutable{wrapper: wrapper{TableDescriptor: tbl, indexCache: &indexCache{}}} 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 @@ -130,10 +126,7 @@ func MakeImmutable(tbl descpb.TableDescriptor) Immutable { for _, m := range tbl.Mutations { switch m.State { case descpb.DescriptorMutation_DELETE_AND_WRITE_ONLY: - if idx := m.GetIndex(); idx != nil { - publicAndNonPublicIndexes = append(publicAndNonPublicIndexes, *idx) - desc.writeOnlyIndexCount++ - } else if col := m.GetColumn(); col != nil { + if col := m.GetColumn(); col != nil { publicAndNonPublicCols = append(publicAndNonPublicCols, *col) desc.writeOnlyColCount++ } @@ -143,9 +136,7 @@ func MakeImmutable(tbl descpb.TableDescriptor) Immutable { for _, m := range tbl.Mutations { switch m.State { case descpb.DescriptorMutation_DELETE_ONLY: - if idx := m.GetIndex(); idx != nil { - publicAndNonPublicIndexes = append(publicAndNonPublicIndexes, *idx) - } else if col := m.GetColumn(); col != nil { + if col := m.GetColumn(); col != nil { publicAndNonPublicCols = append(publicAndNonPublicCols, *col) } } @@ -162,20 +153,12 @@ 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 @@ -211,10 +194,12 @@ func NewImmutableWithIsUncommittedVersion( func NewFilledInImmutable( ctx context.Context, dg catalog.DescGetter, tbl *descpb.TableDescriptor, ) (*Immutable, error) { - desc, err := makeFilledInImmutable(ctx, dg, tbl, false /* skipFKsWithMissingTable */) + changes, err := maybeFillInDescriptor(ctx, dg, tbl, false /* skipFKsWithNoMatchingTable */) if err != nil { return nil, err } + desc := MakeImmutable(*tbl) + desc.postDeserializationChanges = changes return &desc, nil } @@ -236,21 +221,6 @@ 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( @@ -269,12 +239,6 @@ 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 @@ -297,32 +261,6 @@ 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 @@ -331,11 +269,11 @@ func (desc *wrapper) KeysPerRow(indexID descpb.IndexID) (int, error) { if desc.PrimaryIndex.ID == indexID { return len(desc.Families), nil } - idx, err := desc.FindIndexByID(indexID) + idx, err := desc.FindIndexWithID(indexID) if err != nil { return 0, err } - if len(idx.StoreColumnIDs) == 0 { + if idx.NumStoredColumns() == 0 { return 1, nil } return len(desc.Families), nil @@ -356,10 +294,11 @@ func (desc *wrapper) AllNonDropColumns() []descpb.ColumnDescriptor { return cols } -// allocateName sets desc.Name to a value that is not EqualName to any +// buildIndexName 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 allocateIndexName(tableDesc *Mutable, idx *descpb.IndexDescriptor) { +func buildIndexName(tableDesc *Mutable, index catalog.Index) string { + idx := index.IndexDesc() segments := make([]string, 0, len(idx.ColumnNames)+2) segments = append(segments, tableDesc.Name) segments = append(segments, idx.ColumnNames...) @@ -371,36 +310,15 @@ func allocateIndexName(tableDesc *Mutable, idx *descpb.IndexDescriptor) { baseName := strings.Join(segments, "_") name := baseName - - exists := func(name string) bool { - _, _, err := tableDesc.FindIndexByName(name) - return err == nil - } - for i := 1; exists(name); i++ { + for i := 1; ; i++ { + foundIndex, _ := tableDesc.FindIndexWithName(name) + if foundIndex == nil { + break + } name = fmt.Sprintf("%s%d", baseName, i) } - 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 + return name } // AllActiveAndInactiveChecks returns all check constraints, including both @@ -527,48 +445,6 @@ 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( @@ -754,7 +630,7 @@ func maybeUpgradeForeignKeyRepOnIndex( } } if tbl, ok := otherUnupgradedTables[ref.Table]; ok { - referencedIndex, err := tbl.FindIndexByID(ref.Index) + referencedIndex, err := tbl.FindIndexWithID(ref.Index) if err != nil { return false, err } @@ -763,7 +639,7 @@ func maybeUpgradeForeignKeyRepOnIndex( OriginTableID: desc.ID, OriginColumnIDs: idx.ColumnIDs[:numCols], ReferencedTableID: ref.Table, - ReferencedColumnIDs: referencedIndex.ColumnIDs[:numCols], + ReferencedColumnIDs: referencedIndex.IndexDesc().ColumnIDs[:numCols], Name: ref.Name, Validity: ref.Validity, OnDelete: ref.OnDelete, @@ -792,10 +668,11 @@ func maybeUpgradeForeignKeyRepOnIndex( } if otherTable, ok := otherUnupgradedTables[ref.Table]; ok { - originIndex, err := otherTable.FindIndexByID(ref.Index) + originIndexI, err := otherTable.FindIndexWithID(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 @@ -952,9 +829,9 @@ func ForEachExprStringInTableDesc(descI catalog.TableDescriptor, f func(expr *st } return nil } - doIndex := func(i *descpb.IndexDescriptor) error { + doIndex := func(i catalog.Index) error { if i.IsPartial() { - return f(&i.Predicate) + return f(&i.IndexDesc().Predicate) } return nil } @@ -969,15 +846,14 @@ func ForEachExprStringInTableDesc(descI catalog.TableDescriptor, f func(expr *st } } - // Process indexes. - if err := doIndex(&desc.PrimaryIndex); err != nil { + // Process all indexes. + if err := catalog.ForEachIndex(descI, catalog.IndexOpts{ + NonPhysicalPrimaryIndex: true, + DropMutations: true, + AddMutations: true, + }, doIndex); 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 { @@ -986,18 +862,13 @@ func ForEachExprStringInTableDesc(descI catalog.TableDescriptor, f func(expr *st } } - // Process all mutations. + // Process all non-index 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 { @@ -1176,30 +1047,13 @@ func (desc *Mutable) allocateIndexIDs(columnNames map[string]descpb.ColumnID) er desc.NextIndexID = 1 } - // 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) + // Assign names to unnamed indexes. + _ = catalog.ForEachDeletableNonPrimaryIndex(desc, func(idx catalog.Index) error { + if len(idx.GetName()) == 0 { + idx.IndexDesc().Name = buildIndexName(desc, idx) } - 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) - } + return nil + }) var compositeColIDs catalog.TableColSet for i := range desc.Columns { @@ -1210,11 +1064,12 @@ func (desc *Mutable) allocateIndexIDs(columnNames map[string]descpb.ColumnID) er } // Populate IDs. - for _, index := range indexes { - if index.ID != 0 { + for _, idx := range desc.AllIndexes() { + if idx.GetID() != 0 { // This index has already been populated. Nothing to do. continue } + index := idx.IndexDesc() index.ID = desc.NextIndexID desc.NextIndexID++ @@ -1227,7 +1082,7 @@ func (desc *Mutable) allocateIndexIDs(columnNames map[string]descpb.ColumnID) er } } - if index != &desc.PrimaryIndex && index.EncodingType == descpb.SecondaryIndexEncoding { + if !idx.Primary() && index.EncodingType == descpb.SecondaryIndexEncoding { indexHasOldStoredColumns := index.HasOldStoredColumns() // Need to clear ExtraColumnIDs and StoreColumnIDs because they are used // by ContainsColumnID. @@ -1550,12 +1405,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.FindIndexByID(indexID) + targetIndex, err := targetTable.FindIndexWithID(indexID) if err != nil { return nil, nil, errors.Wrapf(err, "missing table=%s index=%d", targetTable.GetName(), errors.Safe(indexID)) } - return targetTable, targetIndex, nil + return targetTable, targetIndex.IndexDesc(), nil } // Check foreign keys. @@ -1582,15 +1437,13 @@ 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 := referencedTable.ForeachIndex(catalog.IndexOpts{}, func( - referencedIdx *descpb.IndexDescriptor, isPrimary bool, - ) error { + if err := catalog.ForEachIndex(referencedTable, catalog.IndexOpts{}, func(referencedIdx catalog.Index) 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.ReferencedBy) > 0 { + if len(referencedIdx.IndexDesc().ReferencedBy) > 0 { unupgradedFKsPresent = true } else { return nil @@ -1602,8 +1455,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.ReferencedBy { - backref := &referencedIdx.ReferencedBy[i] + for i := range referencedIdx.IndexDesc().ReferencedBy { + backref := &referencedIdx.IndexDesc().ReferencedBy[i] if backref.Table != desc.ID { continue } @@ -1612,7 +1465,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.FindIndexByID(backref.Index) + originalOriginIndex, err := desc.FindIndexWithID(backref.Index) if err != nil { return errors.AssertionFailedf( "missing index %d on %q from pre-19.2 foreign key "+ @@ -1663,15 +1516,13 @@ 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 := originTable.ForeachIndex(catalog.IndexOpts{}, func( - originIdx *descpb.IndexDescriptor, isPrimary bool, - ) error { + if err := catalog.ForEachIndex(originTable, catalog.IndexOpts{}, func(originIdx catalog.Index) 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.ForeignKey + fk := originIdx.IndexDesc().ForeignKey if fk.IsSet() { unupgradedFKsPresent = true } else { @@ -1691,7 +1542,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.FindIndexByID(fk.Index) + originalReferencedIndex, err := desc.FindIndexWithID(fk.Index) if err != nil { return errors.AssertionFailedf( "missing index %d on %q from pre-19.2 foreign key forward reference %q on %q", @@ -1717,7 +1568,8 @@ func (desc *wrapper) validateCrossReferences(ctx context.Context, dg catalog.Des backref.Name, desc.Name, originTable.GetName()) } - for _, index := range desc.AllNonDropIndexes() { + for _, indexI := range desc.NonDropIndexes() { + index := indexI.IndexDesc() // Check interleaves. if len(index.Interleave.Ancestors) > 0 { // Only check the most recent ancestor, the rest of them don't point @@ -1752,23 +1604,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.FindIndexByID(backref.Index) + targetIndex, err := targetTable.FindIndexWithID(backref.Index) if err != nil { return errors.Wrapf(err, "invalid interleave backreference table=%s index=%d", targetTable.GetName(), backref.Index) } - if len(targetIndex.Interleave.Ancestors) == 0 { + if targetIndex.NumInterleaveAncestors() == 0 { return errors.AssertionFailedf( "broken interleave backward reference from %q@%q to %q@%q", - desc.Name, index.Name, targetTable.GetName(), targetIndex.Name) + desc.Name, index.Name, targetTable.GetName(), targetIndex.GetName()) } // The last ancestor is required to be a backreference. - ancestor := targetIndex.Interleave.Ancestors[len(targetIndex.Interleave.Ancestors)-1] + ancestor := targetIndex.GetInterleaveAncestor(targetIndex.NumInterleaveAncestors() - 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.Name) + desc.Name, index.Name, targetTable.GetName(), targetIndex.GetName()) } } } @@ -1888,10 +1740,10 @@ func ValidateTableLocalityConfig( // ValidateIndexNameIsUnique validates that the index name does not exist. func (desc *wrapper) ValidateIndexNameIsUnique(indexName string) error { - for _, index := range desc.AllNonDropIndexes() { - if indexName == index.Name { - return sqlerrors.NewRelationAlreadyExistsError(indexName) - } + if catalog.FindNonDropIndex(desc, func(idx catalog.Index) bool { + return idx.GetName() == indexName + }) != nil { + return sqlerrors.NewRelationAlreadyExistsError(indexName) } return nil } @@ -2326,7 +2178,8 @@ func (desc *wrapper) validateTableIndexes(columnNames map[string]descpb.ColumnID indexNames := map[string]struct{}{} indexIDs := map[descpb.IndexID]string{} - for _, index := range desc.AllNonDropIndexes() { + for _, indexI := range desc.NonDropIndexes() { + index := indexI.IndexDesc() if err := catalog.ValidateName(index.Name, "index"); err != nil { return err } @@ -2635,25 +2488,14 @@ 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 desc.ForeachNonDropIndex(func(idxDesc *descpb.IndexDescriptor) error { + return catalog.ForEachNonDropIndex(desc, func(idx catalog.Index) error { + idxDesc := idx.IndexDesc() return desc.validatePartitioningDescriptor( a, idxDesc, &idxDesc.Partitioning, 0 /* colOffset */, partitionNames, ) @@ -2842,27 +2684,19 @@ func (desc *Mutable) RenameColumnDescriptor(column *descpb.ColumnDescriptor, new } } - renameColumnInIndex := func(idx *descpb.IndexDescriptor) { - for i, id := range idx.ColumnIDs { + for _, idx := range desc.AllIndexes() { + idxDesc := idx.IndexDesc() + for i, id := range idxDesc.ColumnIDs { if id == colID { - idx.ColumnNames[i] = newColName + idxDesc.ColumnNames[i] = newColName } } - for i, id := range idx.StoreColumnIDs { + for i, id := range idxDesc.StoreColumnIDs { if id == colID { - idx.StoreColumnNames[i] = newColName + idxDesc.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) @@ -3021,6 +2855,14 @@ 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 { @@ -3043,6 +2885,25 @@ 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 @@ -3069,30 +2930,6 @@ 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. @@ -3339,21 +3176,6 @@ 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. @@ -3406,12 +3228,9 @@ 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 { - for _, index := range desc.AllNonDropIndexes() { - if index.IsInterleaved() { - return true - } - } - return false + return nil != catalog.FindNonDropIndex(desc, func(idx catalog.Index) bool { + return idx.IsInterleaved() + }) } // IsPrimaryIndexDefaultRowID returns whether or not the table's primary @@ -3539,25 +3358,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.FindIndexByID(args.NewPrimaryIndexId) + newIndex, err := desc.FindIndexWithID(args.NewPrimaryIndexId) if err != nil { return err } - if args.NewPrimaryIndexName == "" { - newIndex.Name = PrimaryKeyIndexName - } else { - newIndex.Name = args.NewPrimaryIndexName - } { - primaryIndex := *protoutil.Clone(newIndex).(*descpb.IndexDescriptor) + primaryIndex := newIndex.IndexDescDeepCopy() + if args.NewPrimaryIndexName == "" { + primaryIndex.Name = PrimaryKeyIndexName + } else { + primaryIndex.Name = args.NewPrimaryIndexName + } // 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.ID) + idx, err := getIndexIdxByID(newIndex.GetID()) if err != nil { return err } @@ -3569,7 +3388,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.FindIndexByID(newID) + newIndex, err := desc.FindIndexWithID(newID) if err != nil { return err } @@ -3582,7 +3401,7 @@ func (desc *Mutable) MakeMutationComplete(m descpb.DescriptorMutation) error { return err } oldIndexCopy := protoutil.Clone(oldIndex).(*descpb.IndexDescriptor) - newIndex.Name = oldIndexCopy.Name + newIndex.IndexDesc().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 @@ -4055,12 +3874,8 @@ func (desc *wrapper) InvalidateFKConstraints() { // being added in the mutations. func (desc *wrapper) AllIndexSpans(codec keys.SQLCodec) roachpb.Spans { var spans roachpb.Spans - err := desc.ForeachNonDropIndex(func(index *descpb.IndexDescriptor) error { - spans = append(spans, desc.IndexSpan(codec, index.ID)) - return nil - }) - if err != nil { - panic(err) + for _, index := range desc.NonDropIndexes() { + spans = append(spans, desc.IndexSpan(codec, index.GetID())) } return spans } @@ -4174,8 +3989,8 @@ func (desc *wrapper) GetFamilyOfColumn( // subpartition in an arbitrary order. func (desc *wrapper) PartitionNames() []string { var names []string - for _, index := range desc.AllNonDropIndexes() { - names = append(names, index.Partitioning.PartitionNames()...) + for _, index := range desc.NonDropIndexes() { + names = append(names, index.PartitionNames()...) } return names } @@ -4206,16 +4021,13 @@ func (desc *wrapper) FindAllReferences() (map[descpb.ID]struct{}, error) { fk := &desc.InboundFKs[i] refs[fk.OriginTableID] = struct{}{} } - if err := desc.ForeachNonDropIndex(func(index *descpb.IndexDescriptor) error { - for _, a := range index.Interleave.Ancestors { - refs[a.TableID] = struct{}{} + for _, index := range desc.NonDropIndexes() { + for i := 0; i < index.NumInterleaveAncestors(); i++ { + refs[index.GetInterleaveAncestor(i).TableID] = struct{}{} } - for _, c := range index.InterleavedBy { - refs[c.Table] = struct{}{} + for i := 0; i < index.NumInterleavedBy(); i++ { + refs[index.GetInterleavedBy(i).Table] = struct{}{} } - return nil - }); err != nil { - return nil, err } for _, c := range desc.AllNonDropColumns() { @@ -4256,36 +4068,12 @@ 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 { - for _, idx := range desc.AllNonDropIndexes() { - if idx.Sharded.IsSharded && idx.Sharded.Name == col.Name { - return true - } - } - return false + return nil != catalog.FindNonDropIndex(desc, func(idx catalog.Index) bool { + return idx.IsSharded() && idx.GetShardColumnName() == col.Name + }) } // TableDesc implements the TableDescriptor interface. diff --git a/pkg/sql/catalog/tabledesc/table.go b/pkg/sql/catalog/tabledesc/table.go index ee4498e38e1a..2deaa3bdf681 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. - indexes := desc.AllNonDropIndexes() - for _, index := range indexes { + for _, indexI := range desc.NonDropIndexes() { + index := indexI.IndexDesc() if index.ID == desc.PrimaryIndex.ID { if _, ok := info[index.Name]; ok { return nil, pgerror.Newf(pgcode.DuplicateObject, @@ -341,14 +341,12 @@ func FindFKReferencedIndex( // key columns. primaryIndex := referencedTable.GetPrimaryIndex() if primaryIndex.IsValidReferencedIndex(referencedColIDs) { - return primaryIndex, nil + return primaryIndex.IndexDesc(), nil } // If the PK doesn't match, find the index corresponding to the referenced column. - indexes := referencedTable.GetPublicNonPrimaryIndexes() - for i := range indexes { - idx := &indexes[i] + for _, idx := range referencedTable.PublicNonPrimaryIndexes() { if idx.IsValidReferencedIndex(referencedColIDs) { - return idx, nil + return idx.IndexDesc(), nil } } return nil, pgerror.Newf( @@ -366,14 +364,12 @@ 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, nil + return primaryIndex.IndexDesc(), nil } // If the PK doesn't match, find the index corresponding to the origin column. - indexes := originTable.GetPublicNonPrimaryIndexes() - for i := range indexes { - idx := &indexes[i] + for _, idx := range originTable.PublicNonPrimaryIndexes() { if idx.IsValidOriginIndex(originColIDs) { - return idx, nil + return idx.IndexDesc(), nil } } return nil, pgerror.Newf( diff --git a/pkg/sql/catalog/tabledesc/table_desc.go b/pkg/sql/catalog/tabledesc/table_desc.go index 24a774723813..fa83f8907817 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,6 +27,12 @@ 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 } @@ -44,69 +50,8 @@ func (desc *wrapper) GetPostDeserializationChanges() PostDeserializationTableDes return desc.postDeserializationChanges } -// 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. +// mutationColumns returns all non-public writable columns in the specified +// state. func (desc *wrapper) mutationColumns( mutationState descpb.DescriptorMutation_State, ) []descpb.ColumnDescriptor { @@ -197,18 +142,10 @@ type Immutable struct { // It is partitioned by the state of the column: public, write-only, delete-only publicAndNonPublicCols []descpb.ColumnDescriptor - // 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 + writeOnlyColCount 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 @@ -245,11 +182,6 @@ 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() @@ -265,7 +197,8 @@ func (desc *wrapper) GetColumnAtIdx(idx int) *descpb.ColumnDescriptor { return &desc.Columns[idx] } -// ReadableColumns implements the catalog.TableDescriptor interface +// ReadableColumns returns a list of columns (including those undergoing a +// schema change) which can be scanned. func (desc *Immutable) ReadableColumns() []descpb.ColumnDescriptor { return desc.readableColumns } @@ -318,3 +251,129 @@ 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 bfed97e5bac2..a2541655aa78 100644 --- a/pkg/sql/check.go +++ b/pkg/sql/check.go @@ -96,7 +96,8 @@ func matchFullUnacceptableKeyQuery( srcNotNullExistsClause[i] = fmt.Sprintf("%s IS NOT NULL", srcCols[i]) } - for _, id := range srcTbl.GetPrimaryIndex().ColumnIDs { + for i := 0; i < srcTbl.GetPrimaryIndex().NumColumns(); i++ { + id := srcTbl.GetPrimaryIndex().GetColumnID(i) alreadyPresent := false for _, otherID := range fk.OriginColumnIDs { if id == otherID { @@ -159,7 +160,8 @@ func nonMatchingRowQuery( return "", nil, err } // Get primary key columns not included in the FK - for _, pkColID := range srcTbl.GetPrimaryIndex().ColumnIDs { + for i := 0; i < srcTbl.GetPrimaryIndex().NumColumns(); i++ { + pkColID := srcTbl.GetPrimaryIndex().GetColumnID(i) found := false for _, id := range fk.OriginColumnIDs { if pkColID == id { diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index b803e6cb2f4b..b5f1e76064e7 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.GetPublicNonPrimaryIndexes(), table, alterStmts, + if err := showAlterStatementWithInterleave(ctx, &name, contextName, lookup, table.PublicNonPrimaryIndexes(), table, alterStmts, validateStmts, &p.semaCtx); err != nil { return err } @@ -1823,12 +1823,8 @@ CREATE TABLE crdb_internal.create_statements ( if createNofk == "" { createNofk = stmt } - hasPartitions := false - _ = table.ForeachIndex(catalog.IndexOpts{}, func(idxDesc *descpb.IndexDescriptor, isPrimary bool) error { - if idxDesc.Partitioning.NumColumns != 0 { - hasPartitions = true - } - return nil + hasPartitions := nil != catalog.FindIndex(table, catalog.IndexOpts{}, func(idx catalog.Index) bool { + return idx.GetPartitioning().NumColumns != 0 }) return addRow( dbDescID, @@ -1851,7 +1847,7 @@ func showAlterStatementWithInterleave( tn *tree.TableName, contextName string, lCtx simpleSchemaResolver, - allIdx []descpb.IndexDescriptor, + allIdx []catalog.Index, table catalog.TableDescriptor, alterStmts *tree.DArray, validateStmts *tree.DArray, @@ -1892,14 +1888,12 @@ func showAlterStatementWithInterleave( return err } - for i := range allIdx { - idx := &allIdx[i] + for _, idx := range allIdx { // Create CREATE INDEX commands for INTERLEAVE tables. These commands // are included in the ALTER TABLE statements. - if len(idx.Interleave.Ancestors) > 0 { + if idx.NumInterleaveAncestors() > 0 { f := tree.NewFmtCtx(tree.FmtSimple) - intl := idx.Interleave - parentTableID := intl.Ancestors[len(intl.Ancestors)-1].TableID + parentTableID := idx.GetInterleaveAncestor(idx.NumInterleaveAncestors() - 1).TableID var err error var parentName tree.TableName if lCtx != nil { @@ -1925,11 +1919,11 @@ func showAlterStatementWithInterleave( tableName.ExplicitSchema = false } var sharedPrefixLen int - for _, ancestor := range intl.Ancestors { - sharedPrefixLen += int(ancestor.SharedPrefixLen) + for i := 0; i < idx.NumInterleaveAncestors(); i++ { + sharedPrefixLen += int(idx.GetInterleaveAncestor(i).SharedPrefixLen) } // Write the CREATE INDEX statements. - if err := showCreateIndexWithInterleave(ctx, f, table, idx, tableName, parentName, sharedPrefixLen, semaCtx); err != nil { + if err := showCreateIndexWithInterleave(ctx, f, table, idx.IndexDesc(), tableName, parentName, sharedPrefixLen, semaCtx); err != nil { return err } if err := alterStmts.Append(tree.NewDString(f.CloseAndGetString())); err != nil { @@ -2056,22 +2050,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 table.ForeachIndex(catalog.IndexOpts{ + return catalog.ForEachIndex(table, catalog.IndexOpts{ NonPhysicalPrimaryIndex: true, - }, func(idx *descpb.IndexDescriptor, isPrimary bool) error { + }, func(idx catalog.Index) error { row = row[:0] idxType := secondary - if isPrimary { + if idx.Primary() { idxType = primary } row = append(row, tableID, tableName, - tree.NewDInt(tree.DInt(idx.ID)), - tree.NewDString(idx.Name), + tree.NewDInt(tree.DInt(idx.GetID())), + tree.NewDString(idx.GetName()), idxType, - tree.MakeDBool(tree.DBool(idx.Unique)), - tree.MakeDBool(idx.Type == descpb.IndexDescriptor_INVERTED), + tree.MakeDBool(tree.DBool(idx.IsUnique())), + tree.MakeDBool(idx.GetType() == descpb.IndexDescriptor_INVERTED), ) return pusher.pushRow(row...) }) @@ -2116,7 +2110,8 @@ CREATE TABLE crdb_internal.index_columns ( parentName := parent.GetName() tableName := tree.NewDString(table.GetName()) - reportIndex := func(idx *descpb.IndexDescriptor) error { + reportIndex := func(idxI catalog.Index) error { + idx := idxI.IndexDesc() idxID := tree.NewDInt(tree.DInt(idx.ID)) idxName := tree.NewDString(idx.Name) @@ -2184,11 +2179,7 @@ CREATE TABLE crdb_internal.index_columns ( return nil } - return table.ForeachIndex(catalog.IndexOpts{ - NonPhysicalPrimaryIndex: true, - }, func(idxDesc *descpb.IndexDescriptor, _ bool) error { - return reportIndex(idxDesc) - }) + return catalog.ForEachIndex(table, catalog.IndexOpts{NonPhysicalPrimaryIndex: true}, reportIndex) }) }, } @@ -2220,17 +2211,19 @@ 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 *descpb.IndexDescriptor) error { - for _, interleaveParent := range idx.Interleave.Ancestors { + reportIdxDeps := func(idx catalog.Index) error { + for i := 0; i < idx.NumInterleaveAncestors(); i++ { + interleaveParent := idx.GetInterleaveAncestor(i) if err := addRow( tableID, tableName, - tree.NewDInt(tree.DInt(idx.ID)), + tree.NewDInt(tree.DInt(idx.GetID())), tree.DNull, tree.NewDInt(tree.DInt(interleaveParent.TableID)), interleaveDep, @@ -2268,10 +2261,7 @@ CREATE TABLE crdb_internal.backward_dependencies ( } // Record the backward references of the primary index. - if err := table.ForeachIndex(catalog.IndexOpts{}, - func(idxDesc *descpb.IndexDescriptor, _ bool) error { - return reportIdxDeps(idxDesc) - }); err != nil { + if err := catalog.ForEachIndex(table, catalog.IndexOpts{}, reportIdxDeps); err != nil { return err } @@ -2367,11 +2357,12 @@ CREATE TABLE crdb_internal.forward_dependencies ( tableID := tree.NewDInt(tree.DInt(table.GetID())) tableName := tree.NewDString(table.GetName()) - reportIdxDeps := func(idx *descpb.IndexDescriptor) error { - for _, interleaveRef := range idx.InterleavedBy { + reportIdxDeps := func(idx catalog.Index) error { + for i := 0; i < idx.NumInterleavedBy(); i++ { + interleaveRef := idx.GetInterleavedBy(i) if err := addRow( tableID, tableName, - tree.NewDInt(tree.DInt(idx.ID)), + tree.NewDInt(tree.DInt(idx.GetID())), tree.NewDInt(tree.DInt(interleaveRef.Table)), interleaveDep, tree.NewDInt(tree.DInt(interleaveRef.Index)), @@ -2399,9 +2390,7 @@ CREATE TABLE crdb_internal.forward_dependencies ( } // Record the backward references of the primary index. - if err := table.ForeachIndex(catalog.IndexOpts{}, func(idxDesc *descpb.IndexDescriptor, isPrimary bool) error { - return reportIdxDeps(idxDesc) - }); err != nil { + if err := catalog.ForEachIndex(table, catalog.IndexOpts{}, reportIdxDeps); err != nil { return err } reportDependedOnBy := func( @@ -2513,8 +2502,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.GetPublicNonPrimaryIndexes() { - indexNames[id][uint32(idx.ID)] = idx.Name + for _, idx := range desc.PublicNonPrimaryIndexes() { + indexNames[id][uint32(idx.GetID())] = idx.GetName() } case *dbdesc.Immutable: dbNames[id] = desc.GetName() @@ -2831,7 +2820,9 @@ CREATE TABLE crdb_internal.zones ( } for i, s := range subzones { - index := table.FindActiveIndexByID(descpb.IndexID(s.IndexID)) + index := catalog.FindActiveIndex(table, func(idx catalog.Index) bool { + return idx.GetID() == 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 @@ -2840,7 +2831,7 @@ CREATE TABLE crdb_internal.zones ( } if zoneSpecifier != nil { zs := zs - zs.TableOrIndex.Index = tree.UnrestrictedName(index.Name) + zs.TableOrIndex.Index = tree.UnrestrictedName(index.GetName()) zs.Partition = tree.Name(s.PartitionName) zoneSpecifier = &zs } @@ -2856,7 +2847,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.ID), ""); indexSubzone != nil { + if indexSubzone := fullZone.GetSubzone(uint32(index.GetID()), ""); indexSubzone != nil { subZoneConfig.InheritFromParent(&indexSubzone.Config) } // Inherit remaining fields from the full parent zone. @@ -3413,10 +3404,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 table.ForeachIndex(catalog.IndexOpts{ + return catalog.ForEachIndex(table, catalog.IndexOpts{ AddMutations: true, - }, func(index *descpb.IndexDescriptor, _ bool) error { - return addPartitioningRows(ctx, p, dbName, table, index, &index.Partitioning, + }, func(index catalog.Index) error { + return addPartitioningRows(ctx, p, dbName, table, index.IndexDesc(), &index.IndexDesc().Partitioning, tree.DNull /* parentName */, 0 /* colOffset */, pusher.pushRow) }) }) diff --git a/pkg/sql/create_index.go b/pkg/sql/create_index.go index f57500b1e8a2..1a649bea41c1 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")) - _, dropped, err := n.tableDesc.FindIndexByName(string(n.n.Name)) + foundIndex, err := n.tableDesc.FindIndexWithName(string(n.n.Name)) if err == nil { - if dropped { + if foundIndex.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().Partitioning.NumColumns > 0 { + if n.n.PartitionBy == nil && n.tableDesc.GetPrimaryIndex().GetPartitioning().NumColumns > 0 { params.p.BufferClientNotice( params.ctx, errors.WithHint( diff --git a/pkg/sql/create_stats.go b/pkg/sql/create_stats.go index f727c46f2cdf..4f94b3e6fe8d 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.GetPublicNonPrimaryIndexes())+1) + colStats := make([]jobspb.CreateStatsDetails_ColStat, 0, len(desc.ActiveIndexes())) requestedStats := make(map[string]struct{}) @@ -350,16 +350,19 @@ func createStatsDefaultColumns( } // Add column stats for the primary key. - for i := range desc.GetPrimaryIndex().ColumnIDs { + for i := 0; i < desc.GetPrimaryIndex().NumColumns(); i++ { // Generate stats for each column in the primary key. - addIndexColumnStatsIfNotExists(desc.GetPrimaryIndex().ColumnIDs[i], false /* isInverted */) + addIndexColumnStatsIfNotExists(desc.GetPrimaryIndex().GetColumnID(i), false /* isInverted */) // Only collect multi-column stats if enabled. if i == 0 || !multiColEnabled { continue } - colIDs := desc.GetPrimaryIndex().ColumnIDs[: i+1 : i+1] + colIDs := make([]descpb.ColumnID, i+1) + for j := 0; j <= i; j++ { + colIDs[j] = desc.GetPrimaryIndex().GetColumnID(j) + } // Remember the requested stats so we don't request duplicates. trackStatsIfNotExists(colIDs) @@ -372,19 +375,22 @@ func createStatsDefaultColumns( } // Add column stats for each secondary index. - for i := range desc.GetPublicNonPrimaryIndexes() { - isInverted := desc.GetPublicNonPrimaryIndexes()[i].Type == descpb.IndexDescriptor_INVERTED + for _, idx := range desc.PublicNonPrimaryIndexes() { + isInverted := idx.GetType() == descpb.IndexDescriptor_INVERTED - for j := range desc.GetPublicNonPrimaryIndexes()[i].ColumnIDs { + for j := 0; j < idx.NumColumns(); j++ { // Generate stats for each indexed column. - addIndexColumnStatsIfNotExists(desc.GetPublicNonPrimaryIndexes()[i].ColumnIDs[j], isInverted) + addIndexColumnStatsIfNotExists(idx.GetColumnID(j), isInverted) // Only collect multi-column stats if enabled. if j == 0 || !multiColEnabled { continue } - colIDs := desc.GetPublicNonPrimaryIndexes()[i].ColumnIDs[: j+1 : j+1] + colIDs := make([]descpb.ColumnID, j+1) + for k := 0; k <= j; k++ { + colIDs[k] = idx.GetColumnID(k) + } // Check for existing stats and remember the requested stats. if !trackStatsIfNotExists(colIDs) { @@ -399,8 +405,8 @@ func createStatsDefaultColumns( } // Add columns referenced in partial index predicate expressions. - if desc.GetPublicNonPrimaryIndexes()[i].IsPartial() { - expr, err := parser.ParseExpr(desc.GetPublicNonPrimaryIndexes()[i].Predicate) + if idx.IsPartial() { + expr, err := parser.ParseExpr(idx.GetPredicate()) if err != nil { return nil, err } diff --git a/pkg/sql/create_table.go b/pkg/sql/create_table.go index 23f5dcd21ac9..57df9d1981d7 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.AllNonDropIndexes() { - if len(index.Interleave.Ancestors) > 0 { - if err := params.p.finalizeInterleave(params.ctx, desc, index); err != nil { + for _, index := range desc.NonDropIndexes() { + if index.NumInterleaveAncestors() > 0 { + if err := params.p.finalizeInterleave(params.ctx, desc, index.IndexDesc()); 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, len(target.GetPrimaryIndex().ColumnNames)) - for i, n := range target.GetPrimaryIndex().ColumnNames { - referencedColNames[i] = tree.Name(n) + referencedColNames = make(tree.NameList, target.GetPrimaryIndex().NumColumns()) + for i := range referencedColNames { + referencedColNames[i] = tree.Name(target.GetPrimaryIndex().GetColumnName(i)) } } @@ -977,8 +977,8 @@ func addIndexForFK( if err := tbl.AllocateIDs(ctx); err != nil { return 0, err } - added := tbl.GetPublicNonPrimaryIndexes()[len(tbl.GetPublicNonPrimaryIndexes())-1] - return added.ID, nil + added := tbl.PublicNonPrimaryIndexes()[len(tbl.PublicNonPrimaryIndexes())-1] + return added.GetID(), 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) != len(parentIndex.ColumnIDs) { + if len(interleave.Fields) != parentIndex.NumColumns() { return pgerror.Newf( pgcode.InvalidSchemaDefinition, "declared interleaved columns (%s) must match the parent's primary index (%s)", &interleave.Fields, - strings.Join(parentIndex.ColumnNames, ", "), + strings.Join(parentIndex.IndexDesc().ColumnNames, ", "), ) } if len(interleave.Fields) > len(index.ColumnIDs) { @@ -1053,7 +1053,8 @@ func addInterleave( ) } - for i, targetColID := range parentIndex.ColumnIDs { + for i := 0; i < parentIndex.NumColumns(); i++ { + targetColID := parentIndex.GetColumnID(i) targetCol, err := parentTable.FindColumnByID(targetColID) if err != nil { return err @@ -1071,22 +1072,25 @@ func addInterleave( strings.Join(index.ColumnNames, ", "), ) } - if !col.Type.Identical(targetCol.Type) || index.ColumnDirections[i] != parentIndex.ColumnDirections[i] { + if !col.Type.Identical(targetCol.Type) || index.ColumnDirections[i] != parentIndex.GetColumnDirection(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.ColumnNames, ", "), + strings.Join(parentIndex.IndexDesc().ColumnNames, ", "), ) } } - ancestorPrefix := append( - []descpb.InterleaveDescriptor_Ancestor(nil), parentIndex.Interleave.Ancestors...) + ancestorPrefix := make([]descpb.InterleaveDescriptor_Ancestor, parentIndex.NumInterleaveAncestors()) + for i := range ancestorPrefix { + ancestorPrefix[i] = parentIndex.GetInterleaveAncestor(i) + } + intl := descpb.InterleaveDescriptor_Ancestor{ TableID: parentTable.ID, - IndexID: parentIndex.ID, - SharedPrefixLen: uint32(len(parentIndex.ColumnIDs)), + IndexID: parentIndex.GetID(), + SharedPrefixLen: uint32(parentIndex.NumColumns()), } for _, ancestor := range ancestorPrefix { intl.SharedPrefixLen -= ancestor.SharedPrefixLen @@ -1118,11 +1122,11 @@ func (p *planner) finalizeInterleave( return err } } - ancestorIndex, err := ancestorTable.FindIndexByID(ancestor.IndexID) + ancestorIndex, err := ancestorTable.FindIndexWithID(ancestor.IndexID) if err != nil { return err } - ancestorIndex.InterleavedBy = append(ancestorIndex.InterleavedBy, + ancestorIndex.IndexDesc().InterleavedBy = append(ancestorIndex.IndexDesc().InterleavedBy, descpb.ForeignKeyReference{Table: desc.ID, Index: index.ID}) if err := p.writeSchemaChange( @@ -1648,7 +1652,7 @@ func NewTableDesc( } // If explicit primary keys are required, error out since a primary key was not supplied. - if len(desc.GetPrimaryIndex().ColumnNames) == 0 && desc.IsPhysicalTable() && evalCtx != nil && + if desc.GetPrimaryIndex().NumColumns() == 0 && desc.IsPhysicalTable() && evalCtx != nil && evalCtx.SessionData != nil && evalCtx.SessionData.RequireExplicitPrimaryKeys { return nil, errors.Errorf( "no primary key specified for table %s (require_explicit_primary_keys = true)", desc.Name) @@ -1682,14 +1686,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.AllNonDropIndexes() { - if index.IsSharded() && !columnsInExplicitFamilies[index.Sharded.Name] { + for _, index := range desc.NonDropIndexes() { + if index.IsSharded() && !columnsInExplicitFamilies[index.GetShardColumnName()] { // 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.Sharded.ColumnNames) + family := tabledesc.GetColumnFamilyForShard(&desc, index.GetSharded().ColumnNames) if family != "" { - if err := desc.AddColumnToFamilyMaybeCreate(index.Sharded.Name, family, false, false); err != nil { + if err := desc.AddColumnToFamilyMaybeCreate(index.GetShardColumnName(), family, false, false); err != nil { return nil, err } } @@ -1700,28 +1704,27 @@ func NewTableDesc( return nil, err } - for i := range desc.GetPublicNonPrimaryIndexes() { - idx := &desc.GetPublicNonPrimaryIndexes()[i] + for _, idx := range desc.PublicNonPrimaryIndexes() { // Increment the counter if this index could be storing data across multiple column families. - if len(idx.StoreColumnNames) > 1 && len(desc.Families) > 1 { + if idx.NumStoredColumns() > 1 && len(desc.Families) > 1 { telemetry.Inc(sqltelemetry.SecondaryIndexColumnFamiliesCounter) } } if n.Interleave != nil { - if err := addInterleave(ctx, txn, vt, &desc, desc.GetPrimaryIndex(), n.Interleave); err != nil { + if err := addInterleave(ctx, txn, vt, &desc, desc.GetPrimaryIndex().IndexDesc(), n.Interleave); err != nil { return nil, err } } if n.PartitionBy != nil { partitioning, err := CreatePartitioning( - ctx, st, evalCtx, &desc, desc.GetPrimaryIndex(), n.PartitionBy) + ctx, st, evalCtx, &desc, desc.GetPrimaryIndex().IndexDesc(), n.PartitionBy) if err != nil { return nil, err } { - newPrimaryIndex := *desc.GetPrimaryIndex() + newPrimaryIndex := *desc.GetPrimaryIndex().IndexDesc() newPrimaryIndex.Partitioning = partitioning desc.SetPrimaryIndex(newPrimaryIndex) } @@ -1883,16 +1886,17 @@ func NewTableDesc( } // Record the types of indexes that the table has. - if err := desc.ForeachNonDropIndex(func(idx *descpb.IndexDescriptor) error { + if err := catalog.ForEachNonDropIndex(&desc, func(idx catalog.Index) error { if idx.IsSharded() { telemetry.Inc(sqltelemetry.HashShardedIndexCounter) } - if idx.Type == descpb.IndexDescriptor_INVERTED { + if idx.GetType() == descpb.IndexDescriptor_INVERTED { telemetry.Inc(sqltelemetry.InvertedIndexCounter) - if !geoindex.IsEmptyConfig(&idx.GeoConfig) { - if geoindex.IsGeographyConfig(&idx.GeoConfig) { + geoConfig := idx.GetGeoConfig() + if !geoindex.IsEmptyConfig(&geoConfig) { + if geoindex.IsGeographyConfig(&geoConfig) { telemetry.Inc(sqltelemetry.GeographyInvertedIndexCounter) - } else if geoindex.IsGeometryConfig(&idx.GeoConfig) { + } else if geoindex.IsGeometryConfig(&geoConfig) { telemetry.Inc(sqltelemetry.GeometryInvertedIndexCounter) } } @@ -2134,47 +2138,50 @@ func replaceLikeTableOpts(n *tree.CreateTable, params runParams) (tree.TableDefs } } if opts.Has(tree.LikeTableOptIndexes) { - for _, idx := range td.AllNonDropIndexes() { + for _, idx := range td.NonDropIndexes() { indexDef := tree.IndexTableDef{ - 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)), + 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()), } - columnNames := idx.ColumnNames + numColumns := idx.NumColumns() if idx.IsSharded() { indexDef.Sharded = &tree.ShardedIndexDef{ - ShardBuckets: tree.NewDInt(tree.DInt(idx.Sharded.ShardBuckets)), + ShardBuckets: tree.NewDInt(tree.DInt(idx.GetSharded().ShardBuckets)), } - columnNames = idx.Sharded.ColumnNames + numColumns = len(idx.GetSharded().ColumnNames) } - for i, name := range columnNames { + for j := 0; j < numColumns; j++ { + name := idx.GetColumnName(j) + if idx.IsSharded() { + name = idx.GetSharded().ColumnNames[j] + } elem := tree.IndexElem{ Column: tree.Name(name), Direction: tree.Ascending, } - if idx.ColumnDirections[i] == descpb.IndexDescriptor_DESC { + if idx.GetColumnDirection(j) == descpb.IndexDescriptor_DESC { elem.Direction = tree.Descending } indexDef.Columns = append(indexDef.Columns, elem) } - for _, name := range idx.StoreColumnNames { - indexDef.Storing = append(indexDef.Storing, tree.Name(name)) + for j := 0; j < idx.NumStoredColumns(); j++ { + indexDef.Storing = append(indexDef.Storing, tree.Name(idx.GetStoredColumnName(j))) } var def tree.TableDef = &indexDef - if idx.Unique { - isPK := idx.ID == td.GetPrimaryIndexID() - if isPK && td.IsPrimaryIndexDefaultRowID() { + if idx.IsUnique() { + if idx.Primary() && td.IsPrimaryIndexDefaultRowID() { continue } def = &tree.UniqueConstraintTableDef{ IndexTableDef: indexDef, - PrimaryKey: isPK, + PrimaryKey: idx.Primary(), } } if idx.IsPartial() { - indexDef.Predicate, err = parser.ParseExpr(idx.Predicate) + indexDef.Predicate, err = parser.ParseExpr(idx.GetPredicate()) if err != nil { return nil, err } diff --git a/pkg/sql/delete.go b/pkg/sql/delete.go index 26a73ba2537d..e51d96794d83 100644 --- a/pkg/sql/delete.go +++ b/pkg/sql/delete.go @@ -155,8 +155,7 @@ 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 - partialIndexOrds := d.run.td.tableDesc().PartialIndexOrds() - if !partialIndexOrds.Empty() { + if len(d.run.td.tableDesc().PartialIndexes()) > 0 { 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 6885ec9de161..2535d66f7e90 100644 --- a/pkg/sql/delete_range.go +++ b/pkg/sql/delete_range.go @@ -96,13 +96,13 @@ func (d *deleteRangeNode) startExec(params runParams) error { allTables := make([]row.FetcherTableArgs, len(d.interleavedDesc)+1) allTables[0] = row.FetcherTableArgs{ Desc: d.desc, - Index: d.desc.GetPrimaryIndex(), + Index: d.desc.GetPrimaryIndex().IndexDesc(), Spans: d.spans, } for i, interleaved := range d.interleavedDesc { allTables[i+1] = row.FetcherTableArgs{ Desc: interleaved, - Index: interleaved.GetPrimaryIndex(), + Index: interleaved.GetPrimaryIndex().IndexDesc(), Spans: d.spans, } } diff --git a/pkg/sql/descriptor_mutation_test.go b/pkg/sql/descriptor_mutation_test.go index 45585f5acbf2..d90650f22a2f 100644 --- a/pkg/sql/descriptor_mutation_test.go +++ b/pkg/sql/descriptor_mutation_test.go @@ -480,20 +480,14 @@ func (mt mutationTest) writeIndexMutation( ctx context.Context, index string, m descpb.DescriptorMutation, ) { tableDesc := mt.tableDesc - idx, _, err := tableDesc.FindIndexByName(index) + idx, err := tableDesc.FindIndexWithName(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 - for i, index := range tableDesc.GetPublicNonPrimaryIndexes() { - if idxCopy.ID == index.ID { - tableDesc.RemovePublicNonPrimaryIndex(i + 1) - break - } - } - + idxCopy := *idx.IndexDesc() + tableDesc.RemovePublicNonPrimaryIndex(idx.Ordinal()) m.Descriptor_ = &descpb.DescriptorMutation_Index{Index: &idxCopy} mt.writeMutation(ctx, m) } @@ -648,9 +642,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 - indexIdx := len(tableDesc.GetPublicNonPrimaryIndexes()) - tableDesc.Mutations = []descpb.DescriptorMutation{{Descriptor_: &descpb.DescriptorMutation_Index{Index: &tableDesc.GetPublicNonPrimaryIndexes()[indexIdx-1]}}} - tableDesc.RemovePublicNonPrimaryIndex(indexIdx) + index := tableDesc.PublicNonPrimaryIndexes()[len(tableDesc.PublicNonPrimaryIndexes())-1] + tableDesc.Mutations = []descpb.DescriptorMutation{{Descriptor_: &descpb.DescriptorMutation_Index{Index: index.IndexDesc()}}} + tableDesc.RemovePublicNonPrimaryIndex(index.Ordinal()) 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 603f80d8d44b..e9ecc462845a 100644 --- a/pkg/sql/distsql_physical_planner.go +++ b/pkg/sql/distsql_physical_planner.go @@ -960,14 +960,9 @@ func (dsp *DistSQLPlanner) nodeVersionIsCompatible(nodeID roachpb.NodeID) bool { } func getIndexIdx(index *descpb.IndexDescriptor, desc *tabledesc.Immutable) (uint32, error) { - 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 - } + foundIndex, _ := desc.FindIndexWithID(index.ID) + if foundIndex != nil && foundIndex.Public() { + return uint32(foundIndex.Ordinal()), 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 dbbdce5166fe..2420beb0808b 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 _, indexDesc := range desc.GetPublicNonPrimaryIndexes() { - if indexDesc.Type == descpb.IndexDescriptor_INVERTED && indexDesc.ColumnIDs[0] == col { - spec.Index = &indexDesc + for _, index := range desc.PublicNonPrimaryIndexes() { + if index.GetType() == descpb.IndexDescriptor_INVERTED && index.GetColumnID(0) == col { + spec.Index = index.IndexDesc() break } } diff --git a/pkg/sql/drop_index.go b/pkg/sql/drop_index.go index b3e93f340706..2beb76a77bb6 100644 --- a/pkg/sql/drop_index.go +++ b/pkg/sql/drop_index.go @@ -20,6 +20,7 @@ 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" @@ -117,12 +118,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. - idxDesc, dropped, _ := tableDesc.FindIndexByName(string(index.idxName)) + idx, _ := tableDesc.FindIndexWithName(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 idxDesc != nil && idxDesc.IsSharded() && !dropped { - shardColName = idxDesc.Sharded.Name + if idx != nil && idx.IsSharded() && !idx.Dropped() { + shardColName = idx.GetShardColumnName() } if err := params.p.dropIndexByName( @@ -201,14 +202,9 @@ func (n *dropIndexNode) maybeDropShardColumn( if dropped { return nil } - shouldDropShardColumn := true - for _, otherIdx := range tableDesc.AllNonDropIndexes() { - if otherIdx.ContainsColumnID(shardColDesc.ID) { - shouldDropShardColumn = false - break - } - } - if !shouldDropShardColumn { + if catalog.FindNonDropIndex(tableDesc, func(otherIdx catalog.Index) bool { + return otherIdx.ContainsColumnID(shardColDesc.ID) + }) != nil { return nil } return n.dropShardColumnAndConstraint(params, tableDesc, shardColDesc) @@ -245,7 +241,7 @@ func (p *planner) dropIndexByName( constraintBehavior dropIndexConstraintBehavior, jobDesc string, ) error { - idx, dropped, err := tableDesc.FindIndexByName(string(idxName)) + idxI, err := tableDesc.FindIndexWithName(string(idxName)) if err != nil { // Only index names of the form "table@idx" throw an error here if they // don't exist. @@ -256,10 +252,11 @@ func (p *planner) dropIndexByName( // Index does not exist, but we want it to: error out. return pgerror.WithCandidateCode(err, pgcode.UndefinedObject) } - if dropped { + if idxI.Dropped() { return nil } + idx := idxI.IndexDesc() if idx.Unique && behavior != tree.DropCascade && constraintBehavior != ignoreIdxConstraint && !idx.CreatedExplicitly { return errors.WithHint( pgerror.Newf(pgcode.DependentObjectsStillExist, @@ -315,12 +312,11 @@ 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, 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) + 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()) } } @@ -460,51 +456,11 @@ func (p *planner) dropIndexByName( ) } - 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 - } - } - } - } + foundIndex := catalog.FindPublicNonPrimaryIndex(tableDesc, func(idxEntry catalog.Index) bool { + return idxEntry.GetID() == idx.ID + }) - // 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 { + if foundIndex == nil { return pgerror.Newf( pgcode.ObjectNotInPrerequisiteState, "index %q in the middle of being added, try again later", @@ -512,6 +468,47 @@ 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 2aafe6d4c145..609884ae1a85 100644 --- a/pkg/sql/drop_table.go +++ b/pkg/sql/drop_table.go @@ -84,8 +84,9 @@ func (p *planner) DropTable(ctx context.Context, n *tree.DropTable) (planNode, e } } } - for _, idx := range droppedDesc.AllNonDropIndexes() { - for _, ref := range idx.InterleavedBy { + for _, idx := range droppedDesc.NonDropIndexes() { + for i := 0; i < idx.NumInterleavedBy(); i++ { + ref := idx.GetInterleavedBy(i) if _, ok := td[ref.Table]; !ok { if err := p.canRemoveInterleave(ctx, droppedDesc.Name, ref, n.DropBehavior); err != nil { return nil, err @@ -257,11 +258,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.FindIndexByID(ref.Index) + idx, err := table.FindIndexWithID(ref.Index) if err != nil { return err } - idx.Interleave.Ancestors = nil + idx.IndexDesc().Interleave.Ancestors = nil // No job description, since this is presumably part of some larger schema change. return p.writeSchemaChange(ctx, table, descpb.InvalidMutationID, "") } @@ -300,13 +301,14 @@ func (p *planner) dropTableImpl( tableDesc.InboundFKs = nil // Remove interleave relationships. - for _, idx := range tableDesc.AllNonDropIndexes() { - if len(idx.Interleave.Ancestors) > 0 { - if err := p.removeInterleaveBackReference(ctx, tableDesc, idx); err != nil { + for _, idx := range tableDesc.NonDropIndexes() { + if idx.NumInterleaveAncestors() > 0 { + if err := p.removeInterleaveBackReference(ctx, tableDesc, idx.IndexDesc()); err != nil { return droppedViews, err } } - for _, ref := range idx.InterleavedBy { + for i := 0; i < idx.NumInterleavedBy(); i++ { + ref := idx.GetInterleavedBy(i) if err := p.removeInterleave(ctx, ref); err != nil { return droppedViews, err } @@ -642,10 +644,11 @@ func (p *planner) removeInterleaveBackReference( // The referenced table is being dropped. No need to modify it further. return nil } - targetIdx, err := t.FindIndexByID(ancestor.IndexID) + targetIdxI, err := t.FindIndexWithID(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 7b7249be34a4..7ee23c0358da 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.FindIndexByName("foo") + idx, err := tableDesc.FindIndexWithName("foo") if err != nil { t.Fatal(err) } - indexSpan := tableDesc.IndexSpan(keys.SystemSQLCodec, idx.ID) + indexSpan := tableDesc.IndexSpan(keys.SystemSQLCodec, idx.GetID()) 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.FindIndexByName("foo"); err == nil { + if _, err := tableDesc.FindIndexWithName("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.FindIndexByName("foo") + newIdx, err := tableDesc.FindIndexWithName("foo") if err != nil { t.Fatal(err) } - newIdxSpan := tableDesc.IndexSpan(keys.SystemSQLCodec, newIdx.ID) + newIdxSpan := tableDesc.IndexSpan(keys.SystemSQLCodec, newIdx.GetID()) 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") - indexDesc, _, err := tableDesc.FindIndexByName("foo") + index, err := tableDesc.FindIndexWithName("foo") if err != nil { t.Fatal(err) } - indexSpan := tableDesc.IndexSpan(keys.SystemSQLCodec, indexDesc.ID) + indexSpan := tableDesc.IndexSpan(keys.SystemSQLCodec, index.GetID()) 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(indexDesc.ID), Config: s.(*server.TestServer).Cfg.DefaultZoneConfig}, + {IndexID: uint32(index.GetID()), 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.FindIndexByName("foo"); err == nil { + if _, err := tableDesc.FindIndexWithName("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.FindIndexByName("intlv_idx"); err == nil { + if _, err := tableDesc.FindIndexWithName("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 f001207c7b7d..99587f2fd52c 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.GetPublicNonPrimaryIndexes()[0].ID)) + prefix := roachpb.Key(rowenc.MakeIndexKeyPrefix(keys.SystemSQLCodec, desc, desc.PublicNonPrimaryIndexes()[0].GetID())) 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 1aabaf87d425..d4dfae34c806 100644 --- a/pkg/sql/information_schema.go +++ b/pkg/sql/information_schema.go @@ -1297,7 +1297,7 @@ CREATE TABLE information_schema.statistics ( ) } - return table.ForeachIndex(catalog.IndexOpts{}, func(index *descpb.IndexDescriptor, _ bool) error { + return catalog.ForEachIndex(table, catalog.IndexOpts{}, func(index catalog.Index) 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,31 +1305,34 @@ CREATE TABLE information_schema.statistics ( if index.HasOldStoredColumns() { // Old STORING format: implicit columns are extra columns minus stored // columns. - hasImplicitCols = len(index.ExtraColumnIDs) > len(index.StoreColumnNames) + hasImplicitCols = index.NumExtraColumns() > index.NumStoredColumns() } else { // New STORING format: implicit columns are extra columns. - hasImplicitCols = len(index.ExtraColumnIDs) > 0 + hasImplicitCols = index.NumExtraColumns() > 0 } if hasImplicitCols { implicitCols = make(map[string]struct{}) - for _, col := range table.GetPrimaryIndex().ColumnNames { + for i := 0; i < table.GetPrimaryIndex().NumColumns(); i++ { + col := table.GetPrimaryIndex().GetColumnName(i) implicitCols[col] = struct{}{} } } sequence := 1 - for i, col := range index.ColumnNames { + for i := 0; i < index.NumColumns(); i++ { + col := index.GetColumnName(i) // We add a row for each column of index. - dir := dStringForIndexDirection(index.ColumnDirections[i]) - if err := appendRow(index, col, sequence, dir, false, false); err != nil { + dir := dStringForIndexDirection(index.GetColumnDirection(i)) + if err := appendRow(index.IndexDesc(), col, sequence, dir, false, false); err != nil { return err } sequence++ delete(implicitCols, col) } - for _, col := range index.StoreColumnNames { + for i := 0; i < index.NumStoredColumns(); i++ { + col := index.GetStoredColumnName(i) // We add a row for each stored column of index. - if err := appendRow(index, col, sequence, + if err := appendRow(index.IndexDesc(), col, sequence, indexDirectionNA, true, false); err != nil { return err } @@ -1343,10 +1346,11 @@ CREATE TABLE information_schema.statistics ( // // Note that simply iterating over implicitCols map // produces non-deterministic output. - for _, col := range table.GetPrimaryIndex().ColumnNames { + for i := 0; i < table.GetPrimaryIndex().NumColumns(); i++ { + col := table.GetPrimaryIndex().GetColumnName(i) if _, isImplicit := implicitCols[col]; isImplicit { // We add a row for each implicit column of index. - if err := appendRow(index, col, sequence, + if err := appendRow(index.IndexDesc(), col, sequence, indexDirectionAsc, false, true); err != nil { return err } diff --git a/pkg/sql/insert.go b/pkg/sql/insert.go index 47b6521502b0..b4b7dc30e654 100644 --- a/pkg/sql/insert.go +++ b/pkg/sql/insert.go @@ -129,8 +129,7 @@ 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 - partialIndexOrds := r.ti.tableDesc().PartialIndexOrds() - if !partialIndexOrds.Empty() { + if len(r.ti.tableDesc().PartialIndexes()) > 0 { 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 823bc26db9ad..2536937821ce 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.DeletableIndexes() - for i := range secondaryIndexes { - if secondaryIndexes[i].Type == descpb.IndexDescriptor_INVERTED { + secondaryIndexes := ot.desc.DeletableNonPrimaryIndexes() + for _, index := range secondaryIndexes { + if index.GetType() == 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() + idxDesc = desc.GetPrimaryIndex().IndexDesc() } else { - idxDesc = &secondaryIndexes[i-1] + idxDesc = secondaryIndexes[i-1].IndexDesc() } // 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 1 + len(ot.desc.GetPublicNonPrimaryIndexes()) + return len(ot.desc.ActiveIndexes()) } // 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.WritableIndexes()) + return 1 + len(ot.desc.WritableNonPrimaryIndexes()) } // DeletableIndexCount is part of the cat.Table interface. func (ot *optTable) DeletableIndexCount() int { // Primary index is always present, so count is always >= 1. - return 1 + len(ot.desc.DeletableIndexes()) + return len(ot.desc.AllIndexes()) } // 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() { + if desc == tab.desc.GetPrimaryIndex().IndexDesc() { // Although the primary index contains all columns in the table, the index // descriptor does not contain columns that are not explicitly part of the // primary key. Retrieve those columns from the table descriptor. @@ -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, 1+len(ot.desc.GetPublicNonPrimaryIndexes())) + ot.indexes = make([]optVirtualIndex, len(ot.desc.ActiveIndexes())) // Set up the primary index. ot.indexes[0] = optVirtualIndex{ tab: ot, @@ -1730,17 +1730,16 @@ func newOptVirtualTable( }, } - for i := range ot.desc.GetPublicNonPrimaryIndexes() { - idxDesc := &ot.desc.GetPublicNonPrimaryIndexes()[i] - if len(idxDesc.ColumnIDs) > 1 { + for _, idx := range ot.desc.PublicNonPrimaryIndexes() { + if idx.NumColumns() > 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[i+1] = optVirtualIndex{ + ot.indexes[idx.Ordinal()] = optVirtualIndex{ tab: ot, - desc: idxDesc, - indexOrdinal: i + 1, + desc: idx.IndexDesc(), + indexOrdinal: idx.Ordinal(), // The virtual indexes don't return the bogus PK key? numCols: ot.ColumnCount(), } @@ -1812,19 +1811,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 1 + len(ot.desc.GetPublicNonPrimaryIndexes()) + return len(ot.desc.ActiveIndexes()) } // 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.WritableIndexes()) + return 1 + len(ot.desc.WritableNonPrimaryIndexes()) } // DeletableIndexCount is part of the cat.Table interface. func (ot *optVirtualTable) DeletableIndexCount() int { // Primary index is always present, so count is always >= 1. - return 1 + len(ot.desc.DeletableIndexes()) + return len(ot.desc.AllIndexes()) } // 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 a3ce1f1e12f0..e8d50074d62d 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 + tableScan.index = primaryIndex.IndexDesc() 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 := tabDesc.GetPrimaryIndex().IndexDesc() sb := span.MakeBuilder(ef.planner.EvalContext(), ef.planner.ExecCfg().Codec, tabDesc, indexDesc) if err := ef.planner.maybeSetSystemConfig(tabDesc.GetID()); err != nil { diff --git a/pkg/sql/partition_test.go b/pkg/sql/partition_test.go index f00219c80e4d..da6a3d7b6a1d 100644 --- a/pkg/sql/partition_test.go +++ b/pkg/sql/partition_test.go @@ -50,7 +50,7 @@ func TestRemovePartitioningOSS(t *testing.T) { // Hack in partitions. Doing this properly requires a CCL binary. { - primaryIndex := *tableDesc.GetPrimaryIndex() + primaryIndex := *tableDesc.GetPrimaryIndex().IndexDesc() primaryIndex.Partitioning = descpb.PartitioningDescriptor{ NumColumns: 1, Range: []descpb.PartitioningDescriptor_Range{{ @@ -63,7 +63,7 @@ func TestRemovePartitioningOSS(t *testing.T) { } { - secondaryIndex := tableDesc.GetPublicNonPrimaryIndexes()[0] + secondaryIndex := *tableDesc.PublicNonPrimaryIndexes()[0].IndexDesc() 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.GetPublicNonPrimaryIndexes()[0].ID), + IndexID: uint32(tableDesc.PublicNonPrimaryIndexes()[0].GetID()), PartitionName: "p2", Config: s.(*server.TestServer).Cfg.DefaultZoneConfig, }, diff --git a/pkg/sql/partition_utils.go b/pkg/sql/partition_utils.go index 87f21cb282cb..f354a28c7e2b 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 := tableDesc.ForeachIndex(catalog.IndexOpts{ + if err := catalog.ForEachIndex(tableDesc, catalog.IndexOpts{ AddMutations: true, - }, func(idxDesc *descpb.IndexDescriptor, _ bool) error { - _, indexSubzoneExists := subzoneIndexByIndexID[idxDesc.ID] + }, func(idx catalog.Index) error { + _, indexSubzoneExists := subzoneIndexByIndexID[idx.GetID()] if indexSubzoneExists { - idxSpan := tableDesc.IndexSpan(codec, idxDesc.ID) + idxSpan := tableDesc.IndexSpan(codec, idx.GetID()) // 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(idxDesc.ID)}, + Payload: zonepb.Subzone{IndexID: uint32(idx.GetID())}, }) } var emptyPrefix []tree.Datum indexPartitionCoverings, err := indexCoveringsForPartitioning( - a, codec, tableDesc, idxDesc, &idxDesc.Partitioning, subzoneIndexByPartition, emptyPrefix) + a, codec, tableDesc, idx.IndexDesc(), &idx.IndexDesc().Partitioning, subzoneIndexByPartition, emptyPrefix) if err != nil { return err } diff --git a/pkg/sql/pg_catalog.go b/pkg/sql/pg_catalog.go index d2a098285593..0e0e18dce763 100644 --- a/pkg/sql/pg_catalog.go +++ b/pkg/sql/pg_catalog.go @@ -445,9 +445,10 @@ https://www.postgresql.org/docs/12/catalog-pg-attribute.html`, // Columns for each index. columnIdxMap := table.ColumnIdxMap() - return table.ForeachIndex(catalog.IndexOpts{}, func(index *descpb.IndexDescriptor, _ bool) error { - for _, colID := range index.ColumnIDs { - idxID := h.IndexOid(table.GetID(), index.ID) + 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()) column := table.GetColumnAtIdx(columnIdxMap.GetDefault(colID)) if err := addColumn(column, idxID, column.GetPGAttributeNum()); err != nil { return err @@ -624,31 +625,31 @@ https://www.postgresql.org/docs/9.5/catalog-pg-class.html`, } // Indexes. - return table.ForeachIndex(catalog.IndexOpts{}, func(index *descpb.IndexDescriptor, _ bool) error { + return catalog.ForEachIndex(table, catalog.IndexOpts{}, func(index catalog.Index) error { indexType := forwardIndexOid - if index.Type == descpb.IndexDescriptor_INVERTED { + if index.GetType() == descpb.IndexDescriptor_INVERTED { indexType = invertedIndexOid } return addRow( - 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 + 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 zeroVal, // relchecks tree.DBoolFalse, // relhasoids tree.DBoolFalse, // relhaspkey @@ -1441,11 +1442,15 @@ 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 table.ForeachIndex(catalog.IndexOpts{}, func(index *descpb.IndexDescriptor, isPrimary bool) error { + return catalog.ForEachIndex(table, catalog.IndexOpts{}, func(index catalog.Index) error { isMutation, isWriteOnly := - table.GetIndexMutationCapabilities(index.ID) + table.GetIndexMutationCapabilities(index.GetID()) isReady := isMutation && isWriteOnly - indkey, err := colIDArrayToVector(index.ColumnIDs) + columnIDs := make([]descpb.ColumnID, index.NumColumns()) + for i := range columnIDs { + columnIDs[i] = index.GetColumnID(i) + } + indkey, err := colIDArrayToVector(columnIDs) if err != nil { return err } @@ -1455,7 +1460,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 index.ColumnIDs { + for i, columnID := range columnIDs { col, err := table.FindColumnByID(columnID) if err != nil { return err @@ -1466,7 +1471,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.ColumnDirections[i] == descpb.IndexDescriptor_ASC { + if index.GetColumnDirection(i) == descpb.IndexDescriptor_ASC { thisIndOption = indoptionNullsFirst } else { thisIndOption = indoptionDesc @@ -1479,30 +1484,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(len(index.ColumnIDs)) + indclass, err := makeZeroedOidVector(index.NumColumns()) if err != nil { return err } return addRow( - 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 + 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 ) }) }) @@ -1519,18 +1524,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 table.ForeachIndex(catalog.IndexOpts{}, func(index *descpb.IndexDescriptor, _ bool) error { - def, err := indexDefFromDescriptor(ctx, p, db, table, index, tableLookup) + return catalog.ForEachIndex(table, catalog.IndexOpts{}, func(index catalog.Index) error { + def, err := indexDefFromDescriptor(ctx, p, db, table, index.IndexDesc(), tableLookup) if err != nil { return err } return addRow( - h.IndexOid(table.GetID(), index.ID), // oid - scNameName, // schemaname - tblName, // tablename - tree.NewDName(index.Name), // indexname - tree.DNull, // tablespace - tree.NewDString(def), // indexdef + h.IndexOid(table.GetID(), index.GetID()), // oid + scNameName, // schemaname + tblName, // tablename + tree.NewDName(index.GetName()), // 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 342927ff44b1..648ad7f6a688 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":531,"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":528,"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 016902f142eb..8f90519a72f5 100644 --- a/pkg/sql/physicalplan/fake_span_resolver_test.go +++ b/pkg/sql/physicalplan/fake_span_resolver_test.go @@ -57,7 +57,7 @@ func TestFakeSpanResolver(t *testing.T) { it := resolver.NewSpanResolverIterator(txn) tableDesc := catalogkv.TestingGetTableDescriptor(db, keys.SystemSQLCodec, "test", "t") - primIdxValDirs := catalogkeys.IndexKeyValDirs(tableDesc.GetPrimaryIndex()) + primIdxValDirs := catalogkeys.IndexKeyValDirs(tableDesc.GetPrimaryIndex().IndexDesc()) span := tableDesc.PrimaryIndexSpan(keys.SystemSQLCodec) diff --git a/pkg/sql/physicalplan/span_resolver_test.go b/pkg/sql/physicalplan/span_resolver_test.go index 69d38ad76a2e..90ff68eb6204 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.GetPublicNonPrimaryIndexes()) != 0 { + if len(tableDesc.PublicNonPrimaryIndexes()) != 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 692ddde82121..b0f3b55e459c 100644 --- a/pkg/sql/refresh_materialized_view.go +++ b/pkg/sql/refresh_materialized_view.go @@ -19,7 +19,6 @@ 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 { @@ -68,10 +67,10 @@ func (n *refreshMaterializedViewNode) startExec(params runParams) error { } // Prepare the new set of indexes by cloning all existing indexes on the view. - 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) + newPrimaryIndex := n.desc.GetPrimaryIndex().IndexDescDeepCopy() + newIndexes := make([]descpb.IndexDescriptor, len(n.desc.PublicNonPrimaryIndexes())) + for i, idx := range n.desc.PublicNonPrimaryIndexes() { + newIndexes[i] = idx.IndexDescDeepCopy() } // Reset and allocate new IDs for the new indexes. @@ -87,7 +86,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 f50bd0b9605e..0d39b3a240ba 100644 --- a/pkg/sql/rename_column.go +++ b/pkg/sql/rename_column.go @@ -178,13 +178,15 @@ func (p *planner) renameColumn( } // Rename the column in partial index predicates. - for i := range tableDesc.GetPublicNonPrimaryIndexes() { - if index := &tableDesc.GetPublicNonPrimaryIndexes()[i]; index.IsPartial() { - newExpr, err := schemaexpr.RenameColumn(index.Predicate, *oldName, *newName) + for _, index := range tableDesc.PublicNonPrimaryIndexes() { + if index.IsPartial() { + newExpr, err := schemaexpr.RenameColumn(index.GetPredicate(), *oldName, *newName) if err != nil { return false, err } - index.Predicate = newExpr + indexDesc := *index.IndexDesc() + indexDesc.Predicate = newExpr + tableDesc.SetPublicNonPrimaryIndex(index.Ordinal(), indexDesc) } } @@ -216,8 +218,8 @@ func (p *planner) renameColumn( // Keep the shardedDesc name in sync with the column name. shardedDesc.Name = string(newName) } - for _, idx := range tableDesc.AllNonDropIndexes() { - maybeUpdateShardedDesc(&idx.Sharded) + for _, idx := range tableDesc.NonDropIndexes() { + maybeUpdateShardedDesc(&idx.IndexDesc().Sharded) } // Rename the column in the indexes. diff --git a/pkg/sql/rename_index.go b/pkg/sql/rename_index.go index baff4d82f67d..e4d1de5993cd 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.FindIndexByName(string(n.Index.Index)) + idx, err := tableDesc.FindIndexWithName(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, tableDesc: tableDesc}, nil + return &renameIndexNode{n: n, idx: idx.IndexDesc(), tableDesc: tableDesc}, nil } // ReadingOwnWrites implements the planNodeReadingOwnWrites interface. @@ -98,7 +98,7 @@ func (n *renameIndexNode) startExec(params runParams) error { return nil } - if _, _, err := tableDesc.FindIndexByName(string(n.n.NewName)); err == nil { + if _, err := tableDesc.FindIndexWithName(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 164d0ce09bb7..c8a2238c5d31 100644 --- a/pkg/sql/resolver.go +++ b/pkg/sql/resolver.go @@ -461,8 +461,8 @@ func findTableContainingIndex( continue } - _, dropped, err := tableDesc.FindIndexByName(string(idxName)) - if err != nil || dropped { + idx, err := tableDesc.FindIndexWithName(string(idxName)) + if err != nil || idx.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 714c1b10ddbe..8c11a550c58b 100644 --- a/pkg/sql/revert.go +++ b/pkg/sql/revert.go @@ -54,13 +54,15 @@ func RevertTables( if !tables[i].IsPhysicalTable() { return errors.Errorf("cannot revert virtual table %s", tables[i].Name) } - for _, idx := range tables[i].AllNonDropIndexes() { - for _, parent := range idx.Interleave.Ancestors { + for _, idx := range tables[i].NonDropIndexes() { + for j := 0; j < idx.NumInterleaveAncestors(); j++ { + parent := idx.GetInterleaveAncestor(j) if !reverting[parent.TableID] { return errors.New("cannot revert table without reverting all interleaved tables and indexes") } } - for _, child := range idx.InterleavedBy { + for j := 0; j < idx.NumInterleavedBy(); j++ { + child := idx.GetInterleavedBy(j) 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 039872d08202..4ebe2e29232b 100644 --- a/pkg/sql/row/deleter.go +++ b/pkg/sql/row/deleter.go @@ -44,7 +44,11 @@ type Deleter struct { func MakeDeleter( codec keys.SQLCodec, tableDesc *tabledesc.Immutable, requestedCols []descpb.ColumnDescriptor, ) Deleter { - indexes := tableDesc.DeletableIndexes() + indexes := tableDesc.DeletableNonPrimaryIndexes() + indexDescs := make([]descpb.IndexDescriptor, len(indexes)) + for i, index := range indexes { + indexDescs[i] = *index.IndexDesc() + } var fetchCols []descpb.ColumnDescriptor var fetchColIDtoRowIndex catalog.TableColMap @@ -63,19 +67,22 @@ func MakeDeleter( } return nil } - for _, colID := range tableDesc.GetPrimaryIndex().ColumnIDs { + for j := 0; j < tableDesc.GetPrimaryIndex().NumColumns(); j++ { + colID := tableDesc.GetPrimaryIndex().GetColumnID(j) if err := maybeAddCol(colID); err != nil { return Deleter{} } } for _, index := range indexes { - for _, colID := range index.ColumnIDs { + for j := 0; j < index.NumColumns(); j++ { + colID := index.GetColumnID(j) if err := maybeAddCol(colID); err != nil { return Deleter{} } } // The extra columns are needed to fix #14601. - for _, colID := range index.ExtraColumnIDs { + for j := 0; j < index.NumExtraColumns(); j++ { + colID := index.GetExtraColumnID(j) if err := maybeAddCol(colID); err != nil { return Deleter{} } @@ -84,7 +91,7 @@ func MakeDeleter( } rd := Deleter{ - Helper: newRowHelper(codec, tableDesc, indexes), + Helper: newRowHelper(codec, tableDesc, indexDescs), FetchCols: fetchCols, FetchColIDtoRowIndex: fetchColIDtoRowIndex, } diff --git a/pkg/sql/row/errors.go b/pkg/sql/row/errors.go index 4303f1caebc3..fbe3ff3a2f22 100644 --- a/pkg/sql/row/errors.go +++ b/pkg/sql/row/errors.go @@ -154,25 +154,32 @@ func DecodeRowInfo( if err != nil { return nil, nil, nil, err } - index, err := tableDesc.FindIndexByID(indexID) + index, err := tableDesc.FindIndexWithID(indexID) if err != nil { return nil, nil, nil, err } var rf Fetcher - 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...) + var colIDs []descpb.ColumnID + if !allColumns { + colIDs = make([]descpb.ColumnID, index.NumColumns()) + for i := range colIDs { + colIDs[i] = index.GetColumnID(i) } + } 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) @@ -189,7 +196,7 @@ func DecodeRowInfo( tableArgs := FetcherTableArgs{ Desc: tableDesc, - Index: index, + Index: index.IndexDesc(), ColIdxMap: colIdxMap, IsSecondaryIndex: indexID != tableDesc.GetPrimaryIndexID(), Cols: cols, @@ -232,7 +239,7 @@ func DecodeRowInfo( } values[i] = datums[i].String() } - return index, names, values, nil + return index.IndexDesc(), names, values, nil } func (f *singleKVFetcher) close(context.Context) {} diff --git a/pkg/sql/row/fetcher.go b/pkg/sql/row/fetcher.go index f196d5fd99cd..57cea21ff34d 100644 --- a/pkg/sql/row/fetcher.go +++ b/pkg/sql/row/fetcher.go @@ -1418,7 +1418,12 @@ func (rf *Fetcher) checkPrimaryIndexDatumEncodings(ctx context.Context) error { return nil }) - rh := rowHelper{TableDesc: table.desc, Indexes: table.desc.GetPublicNonPrimaryIndexes()} + 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} 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 cd932a7eb0a9..5ff5d707f7ec 100644 --- a/pkg/sql/row/fetcher_mvcc_test.go +++ b/pkg/sql/row/fetcher_mvcc_test.go @@ -101,7 +101,7 @@ func TestRowFetcherMVCCMetadata(t *testing.T) { args = append(args, row.FetcherTableArgs{ Spans: desc.AllIndexSpans(keys.SystemSQLCodec), Desc: desc, - Index: desc.GetPrimaryIndex(), + Index: desc.GetPrimaryIndex().IndexDesc(), ColIdxMap: colIdxMap, IsSecondaryIndex: false, Cols: desc.Columns, diff --git a/pkg/sql/row/fetcher_test.go b/pkg/sql/row/fetcher_test.go index a911085d33e8..86245eb097ae 100644 --- a/pkg/sql/row/fetcher_test.go +++ b/pkg/sql/row/fetcher_test.go @@ -49,22 +49,13 @@ func makeFetcherArgs(entries []initFetcherArgs) []FetcherTableArgs { fetcherArgs := make([]FetcherTableArgs, len(entries)) for i, entry := range entries { - 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() - } - + index := entry.tableDesc.ActiveIndexes()[entry.indexIdx] fetcherArgs[i] = FetcherTableArgs{ Spans: entry.spans, Desc: entry.tableDesc, - Index: index, + Index: index.IndexDesc(), ColIdxMap: entry.tableDesc.ColumnIdxMap(), - IsSecondaryIndex: isSecondaryIndex, + IsSecondaryIndex: !index.Primary(), Cols: entry.tableDesc.Columns, ValNeededForCol: entry.valNeededForCol, } @@ -203,7 +194,7 @@ func TestNextRowSingle(t *testing.T) { if desc.GetID() != tableDesc.ID || index.ID != tableDesc.GetPrimaryIndexID() { t.Fatalf( "unexpected row retrieved from fetcher.\nnexpected: table %s - index %s\nactual: table %s - index %s", - tableDesc.Name, tableDesc.GetPrimaryIndex().Name, + tableDesc.Name, tableDesc.GetPrimaryIndex().GetName(), desc.GetName(), index.Name, ) } @@ -323,7 +314,7 @@ func TestNextRowBatchLimiting(t *testing.T) { if desc.GetID() != tableDesc.ID || index.ID != tableDesc.GetPrimaryIndexID() { t.Fatalf( "unexpected row retrieved from fetcher.\nnexpected: table %s - index %s\nactual: table %s - index %s", - tableDesc.Name, tableDesc.GetPrimaryIndex().Name, + tableDesc.Name, tableDesc.GetPrimaryIndex().GetName(), desc.GetName(), index.Name, ) } @@ -655,7 +646,7 @@ func TestNextRowSecondaryIndex(t *testing.T) { if err := rf.StartScan( context.Background(), kv.NewTxn(ctx, kvDB, 0), - roachpb.Spans{tableDesc.IndexSpan(keys.SystemSQLCodec, tableDesc.GetPublicNonPrimaryIndexes()[0].ID)}, + roachpb.Spans{tableDesc.IndexSpan(keys.SystemSQLCodec, tableDesc.PublicNonPrimaryIndexes()[0].GetID())}, false, /*limitBatches*/ 0, /*limitHint*/ false, /*traceKV*/ @@ -677,10 +668,10 @@ func TestNextRowSecondaryIndex(t *testing.T) { count++ - if desc.GetID() != tableDesc.ID || index.ID != tableDesc.GetPublicNonPrimaryIndexes()[0].ID { + if desc.GetID() != tableDesc.ID || index.ID != tableDesc.PublicNonPrimaryIndexes()[0].GetID() { t.Fatalf( "unexpected row retrieved from fetcher.\nnexpected: table %s - index %s\nactual: table %s - index %s", - tableDesc.Name, tableDesc.GetPublicNonPrimaryIndexes()[0].Name, + tableDesc.Name, tableDesc.PublicNonPrimaryIndexes()[0].GetName(), desc.GetName(), index.Name, ) } @@ -987,12 +978,7 @@ 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) - var indexID descpb.IndexID - if entry.indexIdx == 0 { - indexID = tableDesc.GetPrimaryIndexID() - } else { - indexID = tableDesc.GetPublicNonPrimaryIndexes()[entry.indexIdx-1].ID - } + indexID := tableDesc.ActiveIndexes()[entry.indexIdx].GetID() 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 b9a40fc5dc9a..8a83c055455b 100644 --- a/pkg/sql/row/helper.go +++ b/pkg/sql/row/helper.go @@ -49,7 +49,7 @@ func newRowHelper( // Pre-compute the encoding directions of the index key values for // pretty-printing in traces. - rh.primIndexValDirs = catalogkeys.IndexKeyValDirs(rh.TableDesc.GetPrimaryIndex()) + rh.primIndexValDirs = catalogkeys.IndexKeyValDirs(rh.TableDesc.GetPrimaryIndex().IndexDesc()) rh.secIndexValDirs = make([][]encoding.Direction, len(rh.Indexes)) for i := range rh.Indexes { @@ -89,7 +89,7 @@ func (rh *rowHelper) encodePrimaryIndex( rh.TableDesc.GetPrimaryIndexID()) } primaryIndexKey, _, err = rowenc.EncodeIndexKey( - rh.TableDesc, rh.TableDesc.GetPrimaryIndex(), colIDtoRowIndex, values, rh.primaryIndexKeyPrefix) + rh.TableDesc, rh.TableDesc.GetPrimaryIndex().IndexDesc(), colIDtoRowIndex, values, rh.primaryIndexKeyPrefix) return primaryIndexKey, err } @@ -137,8 +137,9 @@ 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 _, colID := range rh.TableDesc.GetPrimaryIndex().ColumnIDs { - rh.primaryIndexCols.Add(colID) + for i := 0; i < rh.TableDesc.GetPrimaryIndex().NumColumns(); i++ { + pkColID := rh.TableDesc.GetPrimaryIndex().GetColumnID(i) + rh.primaryIndexCols.Add(pkColID) } } if !rh.primaryIndexCols.Contains(colID) { diff --git a/pkg/sql/row/inserter.go b/pkg/sql/row/inserter.go index 695d49a3cefa..00a9cf11e5f1 100644 --- a/pkg/sql/row/inserter.go +++ b/pkg/sql/row/inserter.go @@ -50,16 +50,23 @@ 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, tableDesc.WritableIndexes()), + Helper: newRowHelper(codec, tableDesc, writableIndexDescs), InsertCols: insertCols, InsertColIDtoRowIndex: ColIDtoRowIndexFromCols(insertCols), marshaled: make([]roachpb.Value, len(insertCols)), } - 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]) + 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)) } } diff --git a/pkg/sql/row/partial_index.go b/pkg/sql/row/partial_index.go index 8b57e193b7de..597046e7e6d1 100644 --- a/pkg/sql/row/partial_index.go +++ b/pkg/sql/row/partial_index.go @@ -44,43 +44,38 @@ func (pm *PartialIndexUpdateHelper) Init( partialIndexPutVals tree.Datums, partialIndexDelVals tree.Datums, tabDesc catalog.TableDescriptor, ) error { colIdx := 0 - partialIndexOrds := tabDesc.PartialIndexOrds() - indexes := tabDesc.DeletableIndexes() - for i, ok := partialIndexOrds.Next(0); ok; i, ok = partialIndexOrds.Next(i + 1) { - index := &indexes[i] - if index.IsPartial() { + for _, idx := range tabDesc.PartialIndexes() { - // 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)) - } + // 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 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)) - } + 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())) } + } - colIdx++ + // 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())) + } } + + colIdx++ } return nil diff --git a/pkg/sql/row/row_converter.go b/pkg/sql/row/row_converter.go index 1096a1cf3e1b..f23d2b7729b2 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.GetPublicNonPrimaryIndexes()) + len(tableDesc.Families)) + padding := 2 * (len(tableDesc.PublicNonPrimaryIndexes()) + 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 ebb451dce3d9..f37b59ffd6c3 100644 --- a/pkg/sql/row/updater.go +++ b/pkg/sql/row/updater.go @@ -98,7 +98,8 @@ func MakeUpdater( updateColIDtoRowIndex := ColIDtoRowIndexFromCols(updateCols) var primaryIndexCols catalog.TableColSet - for _, colID := range tableDesc.GetPrimaryIndex().ColumnIDs { + for i := 0; i < tableDesc.GetPrimaryIndex().NumColumns(); i++ { + colID := tableDesc.GetPrimaryIndex().GetColumnID(i) primaryIndexCols.Add(colID) } @@ -112,7 +113,7 @@ func MakeUpdater( // needsUpdate returns true if the given index may need to be updated for // the current UPDATE mutation. - needsUpdate := func(index descpb.IndexDescriptor) bool { + needsUpdate := func(index catalog.Index) bool { // If the UPDATE is set to only update columns and not secondary // indexes, return false. if updateType == UpdaterOnlyColumns { @@ -133,7 +134,7 @@ func MakeUpdater( if index.IsPartial() { return true } - return index.RunOverAllColumns(func(id descpb.ColumnID) error { + return index.ForEachColumnID(func(id descpb.ColumnID) error { if _, ok := updateColIDtoRowIndex.Get(id); ok { return returnTruePseudoError } @@ -141,22 +142,20 @@ func MakeUpdater( }) != nil } - writableIndexes := tableDesc.WritableIndexes() - includeIndexes := make([]descpb.IndexDescriptor, 0, len(writableIndexes)) - for _, index := range writableIndexes { - if needsUpdate(index) { - includeIndexes = append(includeIndexes, index) - } - } - + includeIndexes := make([]descpb.IndexDescriptor, 0, len(tableDesc.WritableNonPrimaryIndexes())) var deleteOnlyIndexes []descpb.IndexDescriptor - for _, idx := range tableDesc.DeleteOnlyIndexes() { - if needsUpdate(idx) { + for _, index := range tableDesc.DeletableNonPrimaryIndexes() { + if !needsUpdate(index) { + continue + } + if !index.DeleteOnly() { + includeIndexes = append(includeIndexes, *index.IndexDesc()) + } else { if deleteOnlyIndexes == nil { // Allocate at most once. - deleteOnlyIndexes = make([]descpb.IndexDescriptor, 0, len(tableDesc.DeleteOnlyIndexes())) + deleteOnlyIndexes = make([]descpb.IndexDescriptor, 0, len(tableDesc.DeleteOnlyNonPrimaryIndexes())) } - deleteOnlyIndexes = append(deleteOnlyIndexes, idx) + deleteOnlyIndexes = append(deleteOnlyIndexes, *index.IndexDesc()) } } diff --git a/pkg/sql/rowenc/client_index_encoding_test.go b/pkg/sql/rowenc/client_index_encoding_test.go index 7a1e207f6408..81baed783953 100644 --- a/pkg/sql/rowenc/client_index_encoding_test.go +++ b/pkg/sql/rowenc/client_index_encoding_test.go @@ -24,7 +24,6 @@ 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" @@ -57,14 +56,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.GetPublicNonPrimaryIndexes()[0] - childDescIdx := child.GetPublicNonPrimaryIndexes()[0] - childNonUniqueIdx := child.GetPublicNonPrimaryIndexes()[1] - childUniqueIdx := child.GetPublicNonPrimaryIndexes()[2] - grandchildDescIdx := grandchild.GetPublicNonPrimaryIndexes()[0] + parentDescIdx := parent.PublicNonPrimaryIndexes()[0] + childDescIdx := child.PublicNonPrimaryIndexes()[0] + childNonUniqueIdx := child.PublicNonPrimaryIndexes()[1] + childUniqueIdx := child.PublicNonPrimaryIndexes()[2] + grandchildDescIdx := grandchild.PublicNonPrimaryIndexes()[0] testCases := []struct { - index *descpb.IndexDescriptor + index catalog.Index // See ShortToLongKeyFmt for how to represent a key. input string expected string @@ -108,23 +107,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", }, @@ -178,22 +177,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", }, @@ -201,27 +200,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", }, @@ -231,7 +230,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, actual) + actual, err := rowenc.AdjustStartKeyForInterleave(codec, tc.index.IndexDesc(), actual) if err != nil { t.Fatal(err) } @@ -269,15 +268,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.GetPublicNonPrimaryIndexes()[0] - childDescIdx := child.GetPublicNonPrimaryIndexes()[0] - childNonUniqueIdx := child.GetPublicNonPrimaryIndexes()[1] - childUniqueIdx := child.GetPublicNonPrimaryIndexes()[2] - grandchildDescIdx := grandchild.GetPublicNonPrimaryIndexes()[0] + parentDescIdx := parent.PublicNonPrimaryIndexes()[0] + childDescIdx := child.PublicNonPrimaryIndexes()[0] + childNonUniqueIdx := child.PublicNonPrimaryIndexes()[1] + childUniqueIdx := child.PublicNonPrimaryIndexes()[2] + grandchildDescIdx := grandchild.PublicNonPrimaryIndexes()[0] testCases := []struct { table catalog.TableDescriptor - index *descpb.IndexDescriptor + index catalog.Index // See ShortToLongKeyFmt for how to represent a key. input string // If the end key is assumed to be inclusive when passed to @@ -339,25 +338,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", }, @@ -504,26 +503,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", @@ -542,19 +541,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", }, @@ -562,14 +561,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", }, @@ -577,19 +576,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", }, @@ -597,7 +596,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", }, @@ -653,7 +652,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, actual, tc.inclusive) + actual, err := rowenc.AdjustEndKeyForInterleave(codec, tc.table, tc.index.IndexDesc(), 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 60a394389c1a..a04bda3da543 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.FindIndexByID(indexID); err == nil && len(i.Interleave.Ancestors) > 0 { - ancestor := &i.Interleave.Ancestors[0] + if i, err := desc.FindIndexWithID(indexID); err == nil && i.NumInterleaveAncestors() > 0 { + ancestor := i.GetInterleaveAncestor(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([]descpb.IndexDescriptor{*desc.GetPrimaryIndex()}, desc.GetPublicNonPrimaryIndexes()...) + interleaves := append(make([]catalog.Index, 0, len(desc.ActiveIndexes())), desc.ActiveIndexes()...) for component := 0; ; component++ { var tableID descpb.ID @@ -561,9 +561,9 @@ func DecodeIndexKeyPrefix( } for i := len(interleaves) - 1; i >= 0; i-- { - if len(interleaves[i].Interleave.Ancestors) <= component || - interleaves[i].Interleave.Ancestors[component].TableID != tableID || - interleaves[i].Interleave.Ancestors[component].IndexID != indexID { + if interleaves[i].NumInterleaveAncestors() <= component || + interleaves[i].GetInterleaveAncestor(component).TableID != tableID || + interleaves[i].GetInterleaveAncestor(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].Interleave.Ancestors[component].SharedPrefixLen; i++ { + for i := uint32(0); i < interleaves[0].GetInterleaveAncestor(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 8189a66e7e18..2fd77c186f69 100644 --- a/pkg/sql/rowenc/index_encoding_test.go +++ b/pkg/sql/rowenc/index_encoding_test.go @@ -225,7 +225,7 @@ func TestIndexKey(t *testing.T) { codec := keys.SystemSQLCodec primaryKeyPrefix := MakeIndexKeyPrefix(codec, tableDesc, tableDesc.GetPrimaryIndexID()) - primaryKey, _, err := EncodeIndexKey(tableDesc, tableDesc.GetPrimaryIndex(), colMap, testValues, primaryKeyPrefix) + primaryKey, _, err := EncodeIndexKey(tableDesc, tableDesc.GetPrimaryIndex().IndexDesc(), 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.GetPublicNonPrimaryIndexes()[0], colMap, testValues, true /* includeEmpty */) + codec, tableDesc, tableDesc.PublicNonPrimaryIndexes()[0].IndexDesc(), 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(), primaryIndexKV) - checkEntry(&tableDesc.GetPublicNonPrimaryIndexes()[0], secondaryIndexKV) + checkEntry(tableDesc.GetPrimaryIndex().IndexDesc(), primaryIndexKV) + checkEntry(tableDesc.PublicNonPrimaryIndexes()[0].IndexDesc(), secondaryIndexKV) } } @@ -377,8 +377,8 @@ func TestInvertedIndexKey(t *testing.T) { indexKeyTest{50, nil, nil, primaryValues, secondaryValues, }) - for i := range tableDesc.GetPublicNonPrimaryIndexes() { - tableDesc.GetPublicNonPrimaryIndexes()[i].Version = version + for _, idx := range tableDesc.PublicNonPrimaryIndexes() { + idx.IndexDesc().Version = version } testValues := append(primaryValues, secondaryValues...) @@ -386,7 +386,7 @@ func TestInvertedIndexKey(t *testing.T) { codec := keys.SystemSQLCodec secondaryIndexEntries, err := EncodeSecondaryIndex( - codec, tableDesc, &tableDesc.GetPublicNonPrimaryIndexes()[0], colMap, testValues, true /* includeEmpty */) + codec, tableDesc, tableDesc.PublicNonPrimaryIndexes()[0].IndexDesc(), 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(), colMap, tc.table.values, primaryKeyPrefix) + desc, desc.GetPrimaryIndex().IndexDesc(), colMap, tc.table.values, primaryKeyPrefix) if err != nil { t.Fatal(err) } @@ -962,7 +962,8 @@ func TestIndexKeyEquivSignature(t *testing.T) { // Column values should be at the beginning of the // remaining bytes of the key. - colVals, null, err := EncodeColumns(desc.GetPrimaryIndex().ColumnIDs, desc.GetPrimaryIndex().ColumnDirections, colMap, tc.table.values, nil /*key*/) + pkIndexDesc := desc.GetPrimaryIndex().IndexDesc() + colVals, null, err := EncodeColumns(pkIndexDesc.ColumnIDs, pkIndexDesc.ColumnDirections, colMap, tc.table.values, nil /*key*/) if err != nil { t.Fatal(err) } @@ -997,7 +998,7 @@ func TestTableEquivSignatures(t *testing.T) { tc.table.indexKeyArgs.primaryValues = tc.table.values // Setup descriptors and form an index key. desc, _ := makeTableDescForTest(tc.table.indexKeyArgs) - equivSigs, err := TableEquivSignatures(&desc.TableDescriptor, desc.GetPrimaryIndex()) + equivSigs, err := TableEquivSignatures(&desc.TableDescriptor, desc.GetPrimaryIndex().IndexDesc()) if err != nil { t.Fatal(err) } @@ -1082,13 +1083,13 @@ func TestEquivSignature(t *testing.T) { desc, colMap := makeTableDescForTest(table.indexKeyArgs) primaryKeyPrefix := MakeIndexKeyPrefix(keys.SystemSQLCodec, desc, desc.GetPrimaryIndexID()) primaryKey, _, err := EncodeIndexKey( - desc, desc.GetPrimaryIndex(), colMap, table.values, primaryKeyPrefix) + desc, desc.GetPrimaryIndex().IndexDesc(), colMap, table.values, primaryKeyPrefix) if err != nil { t.Fatal(err) } // Extract out the table's equivalence signature. - tempEquivSigs, err := TableEquivSignatures(&desc.TableDescriptor, desc.GetPrimaryIndex()) + tempEquivSigs, err := TableEquivSignatures(&desc.TableDescriptor, desc.GetPrimaryIndex().IndexDesc()) if err != nil { t.Fatal(err) } @@ -1241,10 +1242,11 @@ func ExtractIndexKey( return entry.Key, nil } - index, err := tableDesc.FindIndexByID(indexID) + indexI, err := tableDesc.FindIndexWithID(indexID) if err != nil { return nil, err } + index := indexI.IndexDesc() // Extract the values for index.ColumnIDs. indexTypes, err := colinfo.GetColumnTypes(tableDesc, index.ColumnIDs, nil) @@ -1321,6 +1323,6 @@ func ExtractIndexKey( decodedValues[len(values)+i] = value.Datum } indexKey, _, err := EncodeIndexKey( - tableDesc, tableDesc.GetPrimaryIndex(), colMap, decodedValues, indexKeyPrefix) + tableDesc, tableDesc.GetPrimaryIndex().IndexDesc(), colMap, decodedValues, indexKeyPrefix) return indexKey, err } diff --git a/pkg/sql/rowenc/testutils.go b/pkg/sql/rowenc/testutils.go index 46e49737aaa9..4b2a721b2666 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) > len(index.ColumnIDs) { - return nil, errors.Errorf("got %d values, PK has %d columns", len(vals), len(index.ColumnIDs)) + if len(vals) > index.NumColumns() { + return nil, errors.Errorf("got %d values, PK has %d columns", len(vals), index.NumColumns()) } 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.ColumnIDs[i] + colID := index.GetColumnID(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.ColumnIDs[i], i) + colIDToRowIndex.Set(index.GetColumnID(i), i) } - keyPrefix := MakeIndexKeyPrefix(keys.SystemSQLCodec, desc, index.ID) - key, _, err := EncodeIndexKey(desc, index, colIDToRowIndex, datums, keyPrefix) + keyPrefix := MakeIndexKeyPrefix(keys.SystemSQLCodec, desc, index.GetID()) + key, _, err := EncodeIndexKey(desc, index.IndexDesc(), colIDToRowIndex, datums, keyPrefix) if err != nil { return nil, err } diff --git a/pkg/sql/rowexec/BUILD.bazel b/pkg/sql/rowexec/BUILD.bazel index d82f7fa2e548..8f0d1cdef2ca 100644 --- a/pkg/sql/rowexec/BUILD.bazel +++ b/pkg/sql/rowexec/BUILD.bazel @@ -140,6 +140,7 @@ 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 d295cb3185d9..83f738d3730d 100644 --- a/pkg/sql/rowexec/joinreader_test.go +++ b/pkg/sql/rowexec/joinreader_test.go @@ -25,6 +25,7 @@ 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" @@ -1273,18 +1274,14 @@ 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) - 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 { + 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 { 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 c0cd6011f7db..f5a3fd3be4f7 100644 --- a/pkg/sql/rowexec/scrub_tablereader.go +++ b/pkg/sql/rowexec/scrub_tablereader.go @@ -149,12 +149,7 @@ func (tr *scrubTableReader) generateScrubErrorRow( row rowenc.EncDatumRow, scrubErr *scrub.Error, ) (rowenc.EncDatumRow, error) { details := make(map[string]interface{}) - var index *descpb.IndexDescriptor - if tr.indexIdx == 0 { - index = tr.tableDesc.GetPrimaryIndex() - } else { - index = &tr.tableDesc.GetPublicNonPrimaryIndexes()[tr.indexIdx-1] - } + index := tr.tableDesc.ActiveIndexes()[tr.indexIdx] // Collect all the row values into JSON rowDetails := make(map[string]interface{}) for i, colIdx := range tr.fetcherResultToColIdx { @@ -163,7 +158,7 @@ func (tr *scrubTableReader) generateScrubErrorRow( rowDetails[col.Name] = row[i].String(col.Type) } details["row_data"] = rowDetails - details["index_name"] = index.Name + details["index_name"] = index.GetName() 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 d7fa4ae09c08..af472b0dfaa1 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.GetPublicNonPrimaryIndexes()[0].ID)) + prefix := roachpb.Key(rowenc.MakeIndexKeyPrefix(keys.SystemSQLCodec, td, td.PublicNonPrimaryIndexes()[0].GetID())) 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 2eee1170d716..c723dbe664c9 100644 --- a/pkg/sql/rowexec/zigzagjoiner.go +++ b/pkg/sql/rowexec/zigzagjoiner.go @@ -419,11 +419,7 @@ func (z *zigzagJoiner) setupInfo( info.table = &tables[side] info.eqColumns = spec.EqColumns[side].Columns indexOrdinal := spec.IndexOrdinals[side] - if indexOrdinal == 0 { - info.index = info.table.GetPrimaryIndex() - } else { - info.index = &info.table.GetPublicNonPrimaryIndexes()[indexOrdinal-1] - } + info.index = info.table.ActiveIndexes()[indexOrdinal].IndexDesc() var columnIDs []descpb.ColumnID columnIDs, info.indexDirs = info.index.FullColumnIDs() @@ -651,8 +647,8 @@ func (zi *zigzagJoinerInfo) eqOrdering() (colinfo.ColumnOrdering, error) { if err != nil { return nil, err } - } else if idx := findColumnID(zi.table.GetPrimaryIndex().ColumnIDs, colID); idx != -1 { - direction, err = zi.table.GetPrimaryIndex().ColumnDirections[idx].ToEncodingDirection() + } else if idx := findColumnID(zi.table.GetPrimaryIndex().IndexDesc().ColumnIDs, colID); idx != -1 { + direction, err = zi.table.GetPrimaryIndex().GetColumnDirection(idx).ToEncodingDirection() if err != nil { return nil, err } diff --git a/pkg/sql/scan.go b/pkg/sql/scan.go index 0d66d4d1b34e..435c318eb1db 100644 --- a/pkg/sql/scan.go +++ b/pkg/sql/scan.go @@ -230,35 +230,18 @@ func (n *scanNode) initTable( func (n *scanNode) lookupSpecifiedIndex(indexFlags *tree.IndexFlags) error { if indexFlags.Index != "" { // Search index by name. - 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 { + foundIndex, _ := n.desc.FindIndexWithName(string(indexFlags.Index)) + if foundIndex == nil || !foundIndex.Public() { return errors.Errorf("index %q not found", tree.ErrString(&indexFlags.Index)) } + n.specifiedIndex = foundIndex.IndexDesc() } else if indexFlags.IndexID != 0 { // Search index by ID. - 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 { + foundIndex, _ := n.desc.FindIndexWithID(descpb.IndexID(indexFlags.IndexID)) + if foundIndex == nil || !foundIndex.Public() { return errors.Errorf("index [%d] not found", indexFlags.IndexID) } + n.specifiedIndex = foundIndex.IndexDesc() } return nil } @@ -332,7 +315,7 @@ func initColsForScan( // Initializes the column structures. func (n *scanNode) initDescDefaults(colCfg scanColumnsConfig) error { n.colCfg = colCfg - n.index = n.desc.GetPrimaryIndex() + n.index = n.desc.GetPrimaryIndex().IndexDesc() var err error n.cols, err = initColsForScan(n.desc, n.colCfg) diff --git a/pkg/sql/schema_changer.go b/pkg/sql/schema_changer.go index 49463d69be91..efb6083fa104 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.FindIndexByID(idxID) + oldIndex, err := scTable.FindIndexWithID(idxID) if err != nil { return err } - if len(oldIndex.Interleave.Ancestors) != 0 { - ancestor := oldIndex.Interleave.Ancestors[len(oldIndex.Interleave.Ancestors)-1] + if oldIndex.NumInterleaveAncestors() != 0 { + ancestor := oldIndex.GetInterleaveAncestor(oldIndex.NumInterleaveAncestors() - 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(), txn, desc) + pkJob, err := sc.createIndexGCJob(ctx, scTable.GetPrimaryIndex().IndexDesc(), txn, desc) if err != nil { return err } childJobs = append(childJobs, pkJob) - for i := range scTable.GetPublicNonPrimaryIndexes() { - idxJob, err := sc.createIndexGCJob(ctx, &scTable.GetPublicNonPrimaryIndexes()[i], txn, desc) + for _, idx := range scTable.PublicNonPrimaryIndexes() { + idxJob, err := sc.createIndexGCJob(ctx, idx.IndexDesc(), txn, desc) if err != nil { return err } @@ -1222,27 +1222,28 @@ 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.FindIndexByID(idxID) + oldIndex, err := scTable.FindIndexWithID(idxID) if err != nil { return err } - if len(oldIndex.Interleave.Ancestors) != 0 { - ancestorInfo := oldIndex.Interleave.Ancestors[len(oldIndex.Interleave.Ancestors)-1] + if oldIndex.NumInterleaveAncestors() != 0 { + ancestorInfo := oldIndex.GetInterleaveAncestor(oldIndex.NumInterleaveAncestors() - 1) ancestor, err := descsCol.GetMutableTableVersionByID(ctx, ancestorInfo.TableID, txn) if err != nil { return err } - ancestorIdx, err := ancestor.FindIndexByID(ancestorInfo.IndexID) + ancestorIdxI, err := ancestor.FindIndexWithID(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.ID { + if ref.Table == scTable.ID && ref.Index == oldIndex.GetID() { if foundAncestor { return errors.AssertionFailedf( "ancestor entry in %s for %s@%s found more than once", - ancestor.Name, scTable.Name, oldIndex.Name) + ancestor.Name, scTable.Name, oldIndex.GetName()) } 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 affeb9f7960f..e4b4cebbeaa8 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 := protoutil.Clone(&tableDesc.GetPublicNonPrimaryIndexes()[0]).(*descpb.IndexDescriptor) + index := tableDesc.PublicNonPrimaryIndexes()[0].IndexDescDeepCopy() 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.GetPublicNonPrimaryIndexes()) == 1 { + if len(tableDesc.PublicNonPrimaryIndexes()) == 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.GetPublicNonPrimaryIndexes()[0].Name + name := tableDesc.PublicNonPrimaryIndexes()[0].GetName() 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.GetPublicNonPrimaryIndexes()) == count+1 { + if len(tableDesc.PublicNonPrimaryIndexes()) == 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.GetPublicNonPrimaryIndexes()) > 0 { - t.Fatalf("indexes %+v", tableDesc.GetPublicNonPrimaryIndexes()) + if len(tableDesc.PublicNonPrimaryIndexes()) > 0 { + t.Fatalf("indexes %+v", tableDesc.PublicNonPrimaryIndexes()) } // 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 len(tableDesc.GetPrimaryIndex().ColumnNames) != 1 || tableDesc.GetPrimaryIndex().ColumnNames[0] != "rowid" { + if tableDesc.GetPrimaryIndex().NumColumns() != 1 || tableDesc.GetPrimaryIndex().GetColumnName(0) != "rowid" { return errors.Errorf("expected primary key change to not succeed after cancellation") } return nil @@ -3663,16 +3663,17 @@ 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 i, index := range tableDesc.GetPublicNonPrimaryIndexes() { - if len(index.ExtraColumnIDs) != 1 { + for _, index := range tableDesc.PublicNonPrimaryIndexes() { + if index.NumExtraColumns() != 1 { t.Fatalf("ExtraColumnIDs not set properly: %s", tableDesc) } - if len(index.StoreColumnIDs) != 1 { + if index.NumStoredColumns() != 1 { t.Fatalf("StoreColumnIDs not set properly: %s", tableDesc) } - index.ExtraColumnIDs = append(index.ExtraColumnIDs, index.StoreColumnIDs...) - index.StoreColumnIDs = nil - tableDesc.SetPublicNonPrimaryIndex(i+1, index) + newIndexDesc := *index.IndexDesc() + newIndexDesc.ExtraColumnIDs = append(newIndexDesc.ExtraColumnIDs, newIndexDesc.StoreColumnIDs...) + newIndexDesc.StoreColumnIDs = nil + tableDesc.SetPublicNonPrimaryIndex(index.Ordinal(), newIndexDesc) } if err := kvDB.Put( context.Background(), @@ -5108,8 +5109,8 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v INT); } tableDesc = catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "test") - if len(tableDesc.GetPublicNonPrimaryIndexes()) > 0 || len(tableDesc.Mutations) > 0 { - t.Fatalf("descriptor broken %d, %d", len(tableDesc.GetPublicNonPrimaryIndexes()), len(tableDesc.Mutations)) + if len(tableDesc.PublicNonPrimaryIndexes()) > 0 || len(tableDesc.Mutations) > 0 { + t.Fatalf("descriptor broken %d, %d", len(tableDesc.PublicNonPrimaryIndexes()), len(tableDesc.Mutations)) } } @@ -5181,8 +5182,8 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v JSON); } tableDesc = catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "test") - if len(tableDesc.GetPublicNonPrimaryIndexes()) > 0 || len(tableDesc.Mutations) > 0 { - t.Fatalf("descriptor broken %d, %d", len(tableDesc.GetPublicNonPrimaryIndexes()), len(tableDesc.Mutations)) + if len(tableDesc.PublicNonPrimaryIndexes()) > 0 || len(tableDesc.Mutations) > 0 { + t.Fatalf("descriptor broken %d, %d", len(tableDesc.PublicNonPrimaryIndexes()), len(tableDesc.Mutations)) } } @@ -5888,7 +5889,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.GetPublicNonPrimaryIndexes()) == 1 { + if len(tableDesc.PublicNonPrimaryIndexes()) == 1 { break } } @@ -6293,13 +6294,13 @@ CREATE INDEX i ON t.test (a) WHERE b > 2 } tableDesc := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "test") - indexDesc, _, err := tableDesc.FindIndexByName("i") + index, err := tableDesc.FindIndexWithName("i") if err != nil { t.Fatalf("unexpected error: %s", err) } // Collect all the keys in the partial index. - span := tableDesc.IndexSpan(keys.SystemSQLCodec, indexDesc.ID) + span := tableDesc.IndexSpan(keys.SystemSQLCodec, index.GetID()) 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 ea7688858a58..e6e5f7416da9 100644 --- a/pkg/sql/scrub.go +++ b/pkg/sql/scrub.go @@ -286,7 +286,8 @@ func (n *scrubNode) startScrubTable( func getPrimaryColIdxs( tableDesc *tabledesc.Immutable, columns []*descpb.ColumnDescriptor, ) (primaryColIdxs []int, err error) { - for i, colID := range tableDesc.GetPrimaryIndex().ColumnIDs { + for i := 0; i < tableDesc.GetPrimaryIndex().NumColumns(); i++ { + colID := tableDesc.GetPrimaryIndex().GetColumnID(i) rowIdx := -1 for idx, col := range columns { if col.ID == colID { @@ -298,7 +299,7 @@ func getPrimaryColIdxs( return nil, errors.Errorf( "could not find primary index column in projection: columnID=%d columnName=%s", colID, - tableDesc.GetPrimaryIndex().ColumnNames[i]) + tableDesc.GetPrimaryIndex().GetColumnName(i)) } primaryColIdxs = append(primaryColIdxs, rowIdx) } @@ -345,9 +346,8 @@ func pairwiseOp(left []string, right []string, op string) []string { func createPhysicalCheckOperations( tableDesc *tabledesc.Immutable, tableName *tree.TableName, ) (checks []checkOperation) { - checks = append(checks, newPhysicalCheckOperation(tableName, tableDesc, tableDesc.GetPrimaryIndex())) - for i := range tableDesc.GetPublicNonPrimaryIndexes() { - checks = append(checks, newPhysicalCheckOperation(tableName, tableDesc, &tableDesc.GetPublicNonPrimaryIndexes()[i])) + for _, idx := range tableDesc.ActiveIndexes() { + checks = append(checks, newPhysicalCheckOperation(tableName, tableDesc, idx.IndexDesc())) } return checks } @@ -367,11 +367,11 @@ func createIndexCheckOperations( if indexNames == nil { // Populate results with all secondary indexes of the // table. - for i := range tableDesc.GetPublicNonPrimaryIndexes() { + for _, idx := range tableDesc.PublicNonPrimaryIndexes() { results = append(results, newIndexCheckOperation( tableName, tableDesc, - &tableDesc.GetPublicNonPrimaryIndexes()[i], + idx.IndexDesc(), asOf, )) } @@ -383,15 +383,15 @@ func createIndexCheckOperations( for _, idxName := range indexNames { names[idxName.String()] = struct{}{} } - for i := range tableDesc.GetPublicNonPrimaryIndexes() { - if _, ok := names[tableDesc.GetPublicNonPrimaryIndexes()[i].Name]; ok { + for _, idx := range tableDesc.PublicNonPrimaryIndexes() { + if _, ok := names[idx.GetName()]; ok { results = append(results, newIndexCheckOperation( tableName, tableDesc, - &tableDesc.GetPublicNonPrimaryIndexes()[i], + idx.IndexDesc(), asOf, )) - delete(names, tableDesc.GetPublicNonPrimaryIndexes()[i].Name) + delete(names, idx.GetName()) } } if len(names) > 0 { diff --git a/pkg/sql/scrub_fk.go b/pkg/sql/scrub_fk.go index d60d25e5b578..ceba8cbff41e 100644 --- a/pkg/sql/scrub_fk.go +++ b/pkg/sql/scrub_fk.go @@ -114,7 +114,8 @@ func (o *sqlForeignKeyCheckOperation) Start(params runParams) error { // Get primary key columns not included in the FK. var colIDs []descpb.ColumnID colIDs = append(colIDs, o.constraint.FK.OriginColumnIDs...) - for _, pkColID := range o.tableDesc.GetPrimaryIndex().ColumnIDs { + for i := 0; i < o.tableDesc.GetPrimaryIndex().NumColumns(); i++ { + pkColID := o.tableDesc.GetPrimaryIndex().GetColumnID(i) found := false for _, id := range o.constraint.FK.OriginColumnIDs { if pkColID == id { @@ -147,8 +148,9 @@ func (o *sqlForeignKeyCheckOperation) Next(params runParams) (tree.Datums, error // Collect the primary index values for generating the primary key // pretty string. - primaryKeyDatums := make(tree.Datums, 0, len(o.tableDesc.GetPrimaryIndex().ColumnIDs)) - for _, id := range o.tableDesc.GetPrimaryIndex().ColumnIDs { + primaryKeyDatums := make(tree.Datums, 0, o.tableDesc.GetPrimaryIndex().NumColumns()) + for i := 0; i < o.tableDesc.GetPrimaryIndex().NumColumns(); i++ { + id := o.tableDesc.GetPrimaryIndex().GetColumnID(i) idx := o.colIDToRowIdx.GetDefault(id) primaryKeyDatums = append(primaryKeyDatums, row[idx]) } @@ -163,7 +165,8 @@ func (o *sqlForeignKeyCheckOperation) Next(params runParams) (tree.Datums, error } rowDetails[col.Name] = row[idx].String() } - for _, id := range o.tableDesc.GetPrimaryIndex().ColumnIDs { + for i := 0; i < o.tableDesc.GetPrimaryIndex().NumColumns(); i++ { + id := o.tableDesc.GetPrimaryIndex().GetColumnID(i) found := false for _, fkID := range o.constraint.FK.OriginColumnIDs { if id == fkID { diff --git a/pkg/sql/scrub_index.go b/pkg/sql/scrub_index.go index 90bf988216c7..6a1d566e7c1b 100644 --- a/pkg/sql/scrub_index.go +++ b/pkg/sql/scrub_index.go @@ -83,7 +83,8 @@ func (o *indexCheckOperation) Start(params runParams) error { var pkColumns, otherColumns []*descpb.ColumnDescriptor - for _, colID := range o.tableDesc.GetPrimaryIndex().ColumnIDs { + for i := 0; i < o.tableDesc.GetPrimaryIndex().NumColumns(); i++ { + colID := o.tableDesc.GetPrimaryIndex().GetColumnID(i) col := &o.tableDesc.Columns[colToIdx.GetDefault(colID)] pkColumns = append(pkColumns, col) colToIdx.Set(colID, -1) diff --git a/pkg/sql/scrub_test.go b/pkg/sql/scrub_test.go index cc1d74200b01..5cb5cb2da662 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.GetPublicNonPrimaryIndexes()[0] + secondaryIndex := tableDesc.PublicNonPrimaryIndexes()[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, colIDtoRowIndex, values, true /* includeEmpty */) + keys.SystemSQLCodec, tableDesc, secondaryIndex.IndexDesc(), 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") - secondaryIndexDesc := &tableDesc.GetPublicNonPrimaryIndexes()[0] + secondaryIndex := tableDesc.PublicNonPrimaryIndexes()[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)} - secondaryIndex, err := rowenc.EncodeSecondaryIndex( - keys.SystemSQLCodec, tableDesc, secondaryIndexDesc, colIDtoRowIndex, values, true /* includeEmpty */) + secondaryIndexKey, err := rowenc.EncodeSecondaryIndex( + keys.SystemSQLCodec, tableDesc, secondaryIndex.IndexDesc(), colIDtoRowIndex, values, true /* includeEmpty */) if err != nil { t.Fatalf("unexpected error: %s", err) } - if len(secondaryIndex) != 1 { - t.Fatalf("expected 1 index entry, got %d. got %#v", len(secondaryIndex), secondaryIndex) + if len(secondaryIndexKey) != 1 { + t.Fatalf("expected 1 index entry, got %d. got %#v", len(secondaryIndexKey), secondaryIndexKey) } // Put the new secondary k/v into the database. - if err := kvDB.Put(context.Background(), secondaryIndex[0].Key, &secondaryIndex[0].Value); err != nil { + if err := kvDB.Put(context.Background(), secondaryIndexKey[0].Key, &secondaryIndexKey[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") - secondaryIndexDesc := &tableDesc.GetPublicNonPrimaryIndexes()[0] + secondaryIndex := tableDesc.PublicNonPrimaryIndexes()[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)} - secondaryIndex, err := rowenc.EncodeSecondaryIndex( - keys.SystemSQLCodec, tableDesc, secondaryIndexDesc, colIDtoRowIndex, values, true /* includeEmpty */) + secondaryIndexKey, err := rowenc.EncodeSecondaryIndex( + keys.SystemSQLCodec, tableDesc, secondaryIndex.IndexDesc(), colIDtoRowIndex, values, true /* includeEmpty */) - if len(secondaryIndex) != 1 { - t.Fatalf("expected 1 index entry, got %d. got %#v", len(secondaryIndex), secondaryIndex) + if len(secondaryIndexKey) != 1 { + t.Fatalf("expected 1 index entry, got %d. got %#v", len(secondaryIndexKey), secondaryIndexKey) } if err != nil { t.Fatalf("unexpected error: %s", err) } // Delete the existing secondary k/v. - if err := kvDB.Del(context.Background(), secondaryIndex[0].Key); err != nil { + if err := kvDB.Del(context.Background(), secondaryIndexKey[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)} - secondaryIndex, err = rowenc.EncodeSecondaryIndex( - keys.SystemSQLCodec, tableDesc, secondaryIndexDesc, colIDtoRowIndex, values, true /* includeEmpty */) + secondaryIndexKey, err = rowenc.EncodeSecondaryIndex( + keys.SystemSQLCodec, tableDesc, secondaryIndex.IndexDesc(), 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(), secondaryIndex[0].Key, &secondaryIndex[0].Value); err != nil { + if err := kvDB.Put(context.Background(), secondaryIndexKey[0].Key, &secondaryIndexKey[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(), colIDtoRowIndex, values, primaryIndexKeyPrefix) + tableDesc, tableDesc.GetPrimaryIndex().IndexDesc(), 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.GetPublicNonPrimaryIndexes()[0] + secondaryIndex := tableDesc.PublicNonPrimaryIndexes()[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, colIDtoRowIndex, values, true /* includeEmpty */) + keys.SystemSQLCodec, tableDesc, secondaryIndex.IndexDesc(), 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, colIDtoRowIndex, values, true /* includeEmpty */) + keys.SystemSQLCodec, tableDesc, secondaryIndex.IndexDesc(), 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(), colIDtoRowIndex, values, primaryIndexKeyPrefix) + tableDesc, tableDesc.GetPrimaryIndex().IndexDesc(), colIDtoRowIndex, values, primaryIndexKeyPrefix) if err != nil { t.Fatalf("unexpected error: %s", err) } @@ -676,7 +676,7 @@ INSERT INTO t.test VALUES (217, 314, 1337); primaryIndexKeyPrefix := rowenc.MakeIndexKeyPrefix( keys.SystemSQLCodec, tableDesc, tableDesc.GetPrimaryIndexID()) primaryIndexKey, _, err := rowenc.EncodeIndexKey( - tableDesc, tableDesc.GetPrimaryIndex(), colIDtoRowIndex, values, primaryIndexKeyPrefix) + tableDesc, tableDesc.GetPrimaryIndex().IndexDesc(), colIDtoRowIndex, values, primaryIndexKeyPrefix) if err != nil { t.Fatalf("unexpected error: %s", err) } @@ -781,7 +781,7 @@ CREATE TABLE t.test ( primaryIndexKeyPrefix := rowenc.MakeIndexKeyPrefix( keys.SystemSQLCodec, tableDesc, tableDesc.GetPrimaryIndexID()) primaryIndexKey, _, err := rowenc.EncodeIndexKey( - tableDesc, tableDesc.GetPrimaryIndex(), colIDtoRowIndex, values, primaryIndexKeyPrefix) + tableDesc, tableDesc.GetPrimaryIndex().IndexDesc(), colIDtoRowIndex, values, primaryIndexKeyPrefix) if err != nil { t.Fatalf("unexpected error: %s", err) } @@ -886,7 +886,7 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v1 INT, v2 INT); primaryIndexKeyPrefix := rowenc.MakeIndexKeyPrefix( keys.SystemSQLCodec, tableDesc, tableDesc.GetPrimaryIndexID()) primaryIndexKey, _, err := rowenc.EncodeIndexKey( - tableDesc, tableDesc.GetPrimaryIndex(), colIDtoRowIndex, values, primaryIndexKeyPrefix) + tableDesc, tableDesc.GetPrimaryIndex().IndexDesc(), 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 2395dc5e9629..54d02c7a444e 100644 --- a/pkg/sql/sem/builtins/builtins.go +++ b/pkg/sql/sem/builtins/builtins.go @@ -3673,10 +3673,11 @@ may increase either contention or retry errors, or both.`, if err != nil { return nil, err } - indexDesc, err := tableDesc.FindIndexByID(descpb.IndexID(indexID)) + index, err := tableDesc.FindIndexWithID(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 @@ -4136,14 +4137,14 @@ may increase either contention or retry errors, or both.`, if err != nil { return nil, err } - indexDesc, err := tableDesc.FindIndexByID(descpb.IndexID(indexID)) + index, err := tableDesc.FindIndexWithID(descpb.IndexID(indexID)) if err != nil { return nil, err } - if indexDesc.GeoConfig.S2Geography == nil { + if index.GetGeoConfig().S2Geography == nil { return nil, errors.Errorf("index_id %d is not a geography inverted index", indexID) } - keys, err := rowenc.EncodeGeoInvertedIndexTableKeys(g, nil, indexDesc) + keys, err := rowenc.EncodeGeoInvertedIndexTableKeys(g, nil, index.IndexDesc()) if err != nil { return nil, err } @@ -4170,14 +4171,14 @@ may increase either contention or retry errors, or both.`, if err != nil { return nil, err } - indexDesc, err := tableDesc.FindIndexByID(descpb.IndexID(indexID)) + index, err := tableDesc.FindIndexWithID(descpb.IndexID(indexID)) if err != nil { return nil, err } - if indexDesc.GeoConfig.S2Geometry == nil { + if index.GetGeoConfig().S2Geometry == nil { return nil, errors.Errorf("index_id %d is not a geometry inverted index", indexID) } - keys, err := rowenc.EncodeGeoInvertedIndexTableKeys(g, nil, indexDesc) + keys, err := rowenc.EncodeGeoInvertedIndexTableKeys(g, nil, index.IndexDesc()) if err != nil { return nil, err } diff --git a/pkg/sql/set_zone_config.go b/pkg/sql/set_zone_config.go index e2998034aeae..bf8fe6322feb 100644 --- a/pkg/sql/set_zone_config.go +++ b/pkg/sql/set_zone_config.go @@ -332,12 +332,19 @@ 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) - indexes := table.FindIndexesWithPartition(partitionName) + + var indexes []catalog.Index + for _, idx := range table.NonDropIndexes() { + if tabledesc.FindIndexPartitionByName(idx.IndexDesc(), partitionName) != nil { + indexes = append(indexes, idx) + } + } + 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].Name) + n.zoneSpecifier.TableOrIndex.Index = tree.UnrestrictedName(indexes[0].GetName()) default: err := fmt.Errorf( "partition %q exists on multiple indexes of table %q", partitionName, table.GetName()) @@ -353,10 +360,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.AllNonDropIndexes() { - if p := tabledesc.FindIndexPartitionByName(idx, string(n.zoneSpecifier.Partition)); p != nil { + for _, idx := range table.NonDropIndexes() { + if p := tabledesc.FindIndexPartitionByName(idx.IndexDesc(), string(n.zoneSpecifier.Partition)); p != nil { zs := n.zoneSpecifier - zs.TableOrIndex.Index = tree.UnrestrictedName(idx.Name) + zs.TableOrIndex.Index = tree.UnrestrictedName(idx.GetName()) specifiers = append(specifiers, zs) } } diff --git a/pkg/sql/show_create.go b/pkg/sql/show_create.go index 58b6794b2b42..bf0bf3a09403 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().ColumnIDs[0] == col.ID { + if desc.IsPhysicalTable() && desc.GetPrimaryIndex().GetColumnID(0) == col.ID { // Only set primaryKeyIsOnVisibleColumn to true if the primary key // is on a visible column (not rowid). primaryKeyIsOnVisibleColumn = true @@ -105,7 +105,7 @@ func ShowCreateTable( if primaryKeyIsOnVisibleColumn || (desc.IsPhysicalTable() && desc.GetPrimaryIndex().IsSharded()) { f.WriteString(",\n\tCONSTRAINT ") - formatQuoteNames(&f.Buffer, desc.GetPrimaryIndex().Name) + formatQuoteNames(&f.Buffer, desc.GetPrimaryIndex().GetName()) f.WriteString(" ") f.WriteString(desc.PrimaryKeyString()) } @@ -141,16 +141,15 @@ func ShowCreateTable( } } allIdx := append( - append([]descpb.IndexDescriptor{}, desc.GetPublicNonPrimaryIndexes()...), - *desc.GetPrimaryIndex()) - for i := range allIdx { - idx := &allIdx[i] + append([]catalog.Index{}, desc.PublicNonPrimaryIndexes()...), + desc.GetPrimaryIndex()) + for _, idx := range allIdx { // Only add indexes to the create_statement column, and not to the // create_nofks column if they are not associated with an INTERLEAVE // statement. // Initialize to false if Interleave has no ancestors, indicating that the // index is not interleaved at all. - includeInterleaveClause := len(idx.Interleave.Ancestors) == 0 + includeInterleaveClause := idx.NumInterleaveAncestors() == 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.) @@ -158,10 +157,10 @@ func ShowCreateTable( // clauses as well. includeInterleaveClause = true } - if idx.ID != desc.GetPrimaryIndex().ID && includeInterleaveClause { + if !idx.Primary() && includeInterleaveClause { // Showing the primary index is handled above. f.WriteString(",\n\t") - idxStr, err := catformat.IndexForDisplay(ctx, desc, &descpb.AnonymousTable, idx, &p.RunParams(ctx).p.semaCtx) + idxStr, err := catformat.IndexForDisplay(ctx, desc, &descpb.AnonymousTable, idx.IndexDesc(), &p.RunParams(ctx).p.semaCtx) if err != nil { return "", err } @@ -172,12 +171,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, &f.Buffer, dbPrefix, lCtx); err != nil { + if err := showCreateInterleave(idx.IndexDesc(), &f.Buffer, dbPrefix, lCtx); err != nil { return "", err } } if err := ShowCreatePartitioning( - a, p.ExecCfg().Codec, desc, idx, &idx.Partitioning, &f.Buffer, 1 /* indent */, 0, /* colOffset */ + a, p.ExecCfg().Codec, desc, idx.IndexDesc(), &idx.IndexDesc().Partitioning, &f.Buffer, 1 /* indent */, 0, /* colOffset */ ); err != nil { return "", err } @@ -190,11 +189,11 @@ func ShowCreateTable( return "", err } - if err := showCreateInterleave(desc.GetPrimaryIndex(), &f.Buffer, dbPrefix, lCtx); err != nil { + if err := showCreateInterleave(desc.GetPrimaryIndex().IndexDesc(), &f.Buffer, dbPrefix, lCtx); err != nil { return "", err } if err := ShowCreatePartitioning( - a, p.ExecCfg().Codec, desc, desc.GetPrimaryIndex(), &desc.GetPrimaryIndex().Partitioning, &f.Buffer, 0 /* indent */, 0, /* colOffset */ + a, p.ExecCfg().Codec, desc, desc.GetPrimaryIndex().IndexDesc(), &desc.GetPrimaryIndex().IndexDesc().Partitioning, &f.Buffer, 0 /* indent */, 0, /* colOffset */ ); err != nil { return "", err } diff --git a/pkg/sql/show_create_clauses.go b/pkg/sql/show_create_clauses.go index 4044d4bed0e5..340e5a86709e 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.FindIndexByID(descpb.IndexID(indexComment.subID)) + idx, err := table.FindIndexWithID(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.Name), + Index: tree.UnrestrictedName(idx.GetName()), }, Comment: &indexComment.comment, }) diff --git a/pkg/sql/show_fingerprints.go b/pkg/sql/show_fingerprints.go index 6e6084769e93..3472fe97afc3 100644 --- a/pkg/sql/show_fingerprints.go +++ b/pkg/sql/show_fingerprints.go @@ -65,9 +65,15 @@ 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: tableDesc.AllNonDropIndexes(), + indexes: indexDescs, }, nil } diff --git a/pkg/sql/span_builder_test.go b/pkg/sql/span_builder_test.go index cb68ac354417..cddec253abe1 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.FindIndexByName(tc.index) + idx, err := desc.FindIndexWithName(tc.index) if err != nil { t.Fatal(err) } - builder := span.MakeBuilder(evalCtx, execCfg.Codec, desc, idx) + builder := span.MakeBuilder(evalCtx, execCfg.Codec, desc, idx.IndexDesc()) 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 a3dd0c2ee320..def879d5350d 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.GetPublicNonPrimaryIndexes()[0].ID + secID := tableDesc.PublicNonPrimaryIndexes()[0].GetID() // 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 af8e676e69fc..c504131e2cb3 100644 --- a/pkg/sql/table_test.go +++ b/pkg/sql/table_test.go @@ -302,11 +302,16 @@ func TestMakeTableDescIndexes(t *testing.T) { if err != nil { t.Fatalf("%d (%s): %v", i, d.sql, err) } - 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()) + activeIndexDescs := make([]descpb.IndexDescriptor, len(schema.ActiveIndexes())) + for i, index := range schema.ActiveIndexes() { + activeIndexDescs[i] = *index.IndexDesc() } - 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()) + + 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:]) } } diff --git a/pkg/sql/tablewriter_delete.go b/pkg/sql/tablewriter_delete.go index e081bbf6dace..fed9c125ca18 100644 --- a/pkg/sql/tablewriter_delete.go +++ b/pkg/sql/tablewriter_delete.go @@ -123,7 +123,7 @@ func (td *tableDeleter) deleteAllRowsScan( var rf row.Fetcher tableArgs := row.FetcherTableArgs{ Desc: td.tableDesc(), - Index: td.tableDesc().GetPrimaryIndex(), + Index: td.tableDesc().GetPrimaryIndex().IndexDesc(), ColIdxMap: td.rd.FetchColIDtoRowIndex, Cols: td.rd.FetchCols, ValNeededForCol: valNeededForCol, @@ -254,7 +254,7 @@ func (td *tableDeleter) deleteIndexScan( var rf row.Fetcher tableArgs := row.FetcherTableArgs{ Desc: td.tableDesc(), - Index: td.tableDesc().GetPrimaryIndex(), + Index: td.tableDesc().GetPrimaryIndex().IndexDesc(), 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 aee5ac1fcbe1..588edd51347d 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.FindIndexByName(shardedIndexName) + idx, err := tableDesc.FindIndexWithName(shardedIndexName) if err != nil { t.Fatal(err) } - shardCol, _, err := tableDesc.FindColumnByName(tree.Name(idx.Sharded.Name)) + shardCol, _, err := tableDesc.FindColumnByName(tree.Name(idx.GetShardColumnName())) 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.FindIndexByName(shardedIndexName) + idx, err := tableDesc.FindIndexWithName(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.ColumnIDs[0] != shardColID { + if idx.GetColumnID(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. - fooDesc, _, err := tableDesc.FindIndexByName("foo") + foo, err := tableDesc.FindIndexWithName("foo") if err != nil { t.Fatal(err) } foundShardColumn := false - for _, colID := range fooDesc.ExtraColumnIDs { + for _, colID := range foo.IndexDesc().ExtraColumnIDs { if colID == shardColID { foundShardColumn = true break diff --git a/pkg/sql/truncate.go b/pkg/sql/truncate.go index 43a711530bd1..d1b54b0b40f6 100644 --- a/pkg/sql/truncate.go +++ b/pkg/sql/truncate.go @@ -132,8 +132,9 @@ func (t *truncateNode) startExec(params runParams) error { return err } } - for _, idx := range tableDesc.AllNonDropIndexes() { - for _, ref := range idx.InterleavedBy { + for _, idx := range tableDesc.NonDropIndexes() { + for i := 0; i < idx.NumInterleavedBy(); i++ { + ref := idx.GetInterleavedBy(i) if err := maybeEnqueue(ref.Table, "interleaved by"); err != nil { return err } @@ -193,24 +194,19 @@ func (p *planner) truncateTable( return err } - // 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) + // 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) + } } - 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 @@ -218,9 +214,8 @@ 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)) - indexIDMapping[oldIndexes[0].ID] = tableDesc.GetPrimaryIndexID() - for i := range tableDesc.GetPublicNonPrimaryIndexes() { - indexIDMapping[oldIndexes[i+1].ID] = tableDesc.GetPublicNonPrimaryIndexes()[i].ID + for _, idx := range tableDesc.ActiveIndexes() { + indexIDMapping[oldIndexes[idx.Ordinal()].ID] = idx.GetID() } // Resolve all outstanding mutations. Make all new schema elements @@ -288,9 +283,9 @@ func (p *planner) truncateTable( for i := range oldIndexIDs { oldIndexIDs[i] = oldIndexes[i+1].ID } - newIndexIDs := make([]descpb.IndexID, len(tableDesc.GetPublicNonPrimaryIndexes())) + newIndexIDs := make([]descpb.IndexID, len(tableDesc.PublicNonPrimaryIndexes())) for i := range newIndexIDs { - newIndexIDs[i] = tableDesc.GetPublicNonPrimaryIndexes()[i].ID + newIndexIDs[i] = tableDesc.PublicNonPrimaryIndexes()[i].GetID() } swapInfo := &descpb.PrimaryKeySwap{ OldPrimaryIndexId: oldIndexes[0].ID, @@ -381,7 +376,8 @@ func (p *planner) reassignInterleaveIndexReferences( ) error { for _, table := range tables { changed := false - if err := table.ForeachNonDropIndex(func(index *descpb.IndexDescriptor) error { + if err := catalog.ForEachNonDropIndex(table, func(indexI catalog.Index) error { + index := indexI.IndexDesc() 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 2a6097e253a6..48ea5cf4a3ae 100644 --- a/pkg/sql/update.go +++ b/pkg/sql/update.go @@ -296,12 +296,11 @@ 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 - partialIndexOrds := u.run.tu.tableDesc().PartialIndexOrds() - if !partialIndexOrds.Empty() { + if n := len(u.run.tu.tableDesc().PartialIndexes()); n > 0 { 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[:partialIndexOrds.Len()] - partialIndexDelVals := partialIndexVals[partialIndexOrds.Len() : partialIndexOrds.Len()*2] + partialIndexPutVals := partialIndexVals[:n] + partialIndexDelVals := partialIndexVals[n : n*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 cc2741e8ec23..7ebef9e6b3f3 100644 --- a/pkg/sql/upsert.go +++ b/pkg/sql/upsert.go @@ -143,8 +143,7 @@ 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 - partialIndexOrds := n.run.tw.tableDesc().PartialIndexOrds() - if !partialIndexOrds.Empty() { + if len(n.run.tw.tableDesc().PartialIndexes()) > 0 { 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 7f1614c738f5..1ea8ff00f0d0 100644 --- a/pkg/sql/virtual_schema.go +++ b/pkg/sql/virtual_schema.go @@ -176,10 +176,11 @@ func (t virtualSchemaTable) initVirtualTableDesc( if err != nil { return mutDesc.TableDescriptor, err } - for i, idx := range mutDesc.GetPublicNonPrimaryIndexes() { - if len(idx.ColumnIDs) > 1 { + for _, index := range mutDesc.PublicNonPrimaryIndexes() { + if index.NumColumns() > 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)) @@ -198,7 +199,7 @@ func (t virtualSchemaTable) initVirtualTableDesc( idx.StoreColumnNames[outputIdx] = mutDesc.Columns[j].Name outputIdx++ } - mutDesc.SetPublicNonPrimaryIndex(i+1, idx) + mutDesc.SetPublicNonPrimaryIndex(index.Ordinal(), idx) } return mutDesc.TableDescriptor, nil } diff --git a/pkg/sql/zone_config.go b/pkg/sql/zone_config.go index 014e4faeec69..db8aa866cba5 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 *descpb.IndexDescriptor + var index catalog.Index if indexName == "" { index = table.GetPrimaryIndex() - indexName = index.Name + indexName = index.GetName() } else { var err error - index, _, err = table.FindIndexByName(indexName) + index, err = table.FindIndexWithName(indexName) if err != nil { return nil, "", err } @@ -329,12 +329,12 @@ func resolveSubzone( partitionName := string(zs.Partition) if partitionName != "" { - if partitioning := tabledesc.FindIndexPartitionByName(index, partitionName); partitioning == nil { + if partitioning := tabledesc.FindIndexPartitionByName(index.IndexDesc(), partitionName); partitioning == nil { return nil, "", fmt.Errorf("partition %q does not exist on index %q", partitionName, indexName) } } - return index, partitionName, nil + return index.IndexDesc(), partitionName, nil } func deleteRemovedPartitionZoneConfigs(