Skip to content

Commit

Permalink
sql: remove deprecated GetPrimaryIndex method
Browse files Browse the repository at this point in the history
Previously, this method would be called on a table descriptor interface
(or backing struct) to obtain the primary index descriptor in the form
of a *descpb.IndexDescriptor.
This patch removes these calls, along with the method definition, in
favour of new methods which use the catalog.Index interface type instead.

Partially addresses #57465.

Release note: None
  • Loading branch information
Marius Posta committed Jan 4, 2021
1 parent 064b855 commit 5148f62
Show file tree
Hide file tree
Showing 39 changed files with 147 additions and 131 deletions.
2 changes: 1 addition & 1 deletion pkg/ccl/changefeedccl/avro_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.PrimaryIndexInterface().IndexDesc())
require.NoError(t, err)
require.Equal(t,
`{"type":"record","name":"_u2603_","fields":[`+
Expand Down
8 changes: 5 additions & 3 deletions pkg/ccl/changefeedccl/encoder.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.PrimaryIndexInterface()
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)
Expand Down Expand Up @@ -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.PrimaryIndexInterface().IndexDesc())
if err != nil {
return nil, err
}
Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/changefeedccl/rowfetcher_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -121,7 +121,7 @@ func (c *rowFetcherCache) TableDescForKey(
}

// Skip over the column data.
for ; skippedCols < len(tableDesc.GetPrimaryIndex().ColumnIDs); skippedCols++ {
for ; skippedCols < tableDesc.PrimaryIndexInterface().NumColumns(); skippedCols++ {
l, err := encoding.PeekLength(remaining)
if err != nil {
return nil, err
Expand Down Expand Up @@ -173,7 +173,7 @@ func (c *rowFetcherCache) RowFetcherForTableDesc(
row.FetcherTableArgs{
Spans: tableDesc.AllIndexSpans(c.codec),
Desc: tableDesc,
Index: tableDesc.GetPrimaryIndex(),
Index: tableDesc.PrimaryIndexInterface().IndexDesc(),
ColIdxMap: colIdxMap,
IsSecondaryIndex: false,
Cols: tableDesc.Columns,
Expand Down
2 changes: 1 addition & 1 deletion pkg/server/settingsworker.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.PrimaryIndexInterface().IndexDesc(), types, nameRow, nil, kv.Key)
if err != nil {
return errors.Wrap(err, "failed to decode key")
}
Expand Down
18 changes: 9 additions & 9 deletions pkg/sql/alter_primary_key.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.PrimaryIndexInterface().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.PrimaryIndexInterface().NumInterleavedBy(); i++ {
interleaveTableID := tableDesc.PrimaryIndexInterface().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 {
Expand Down Expand Up @@ -222,15 +222,15 @@ 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.PrimaryIndexInterface().IndexDescDeepCopy()
// Clear the name of the index so that it gets generated by AllocateIDs.
oldPrimaryIndexCopy.Name = ""
oldPrimaryIndexCopy.StoreColumnIDs = nil
oldPrimaryIndexCopy.StoreColumnNames = nil
// 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
}
}
Expand Down Expand Up @@ -314,7 +314,7 @@ func (p *planner) AlterPrimaryKey(

// Mark the primary key of the table as valid.
{
primaryIndex := *tableDesc.GetPrimaryIndex()
primaryIndex := *tableDesc.PrimaryIndexInterface().IndexDesc()
primaryIndex.Disabled = false
tableDesc.SetPrimaryIndex(primaryIndex)
}
Expand Down Expand Up @@ -344,7 +344,7 @@ func (p *planner) AlterPrimaryKey(
// * The new primary key isn't the same hash sharded old primary key with a
// different bucket count.
func shouldCopyPrimaryKey(desc *tabledesc.Mutable, newPK *descpb.IndexDescriptor) bool {
oldPK := desc.GetPrimaryIndex()
oldPK := desc.PrimaryIndexInterface()
if !desc.HasPrimaryKey() {
return false
}
Expand All @@ -354,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
}

Expand Down
24 changes: 13 additions & 11 deletions pkg/sql/alter_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -436,7 +436,7 @@ func (n *alterTableNode) startExec(params runParams) error {
return err
}

if n.tableDesc.GetPrimaryIndex().ContainsColumnID(colToDrop.ID) {
if n.tableDesc.PrimaryIndexInterface().ContainsColumnID(colToDrop.ID) {
return pgerror.Newf(pgcode.InvalidColumnReference,
"column %q is referenced by the primary key", colToDrop.Name)
}
Expand All @@ -459,7 +459,7 @@ func (n *alterTableNode) startExec(params runParams) error {
if !containsThisColumn {
for j := 0; j < idx.NumExtraColumns(); j++ {
id := idx.GetExtraColumnID(j)
if n.tableDesc.GetPrimaryIndex().ContainsColumnID(id) {
if n.tableDesc.PrimaryIndexInterface().ContainsColumnID(id) {
// All secondary indices necessary contain the PK
// columns, too. (See the comments on the definition of
// IndexDescriptor). The presence of a PK column in the
Expand Down Expand Up @@ -740,26 +740,28 @@ func (n *alterTableNode) startExec(params runParams) error {
descriptorChanged = true

case *tree.AlterTablePartitionBy:
partitioning, err := CreatePartitioning(
primaryIndex := n.tableDesc.PrimaryIndexInterface()
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:
Expand Down Expand Up @@ -1033,7 +1035,7 @@ func applyColumnMutation(
}

// Prevent a column in a primary key from becoming non-null.
if tableDesc.GetPrimaryIndex().ContainsColumnID(col.ID) {
if tableDesc.PrimaryIndexInterface().ContainsColumnID(col.ID) {
return pgerror.Newf(pgcode.InvalidTableDefinition,
`column "%s" is in a primary index`, col.Name)
}
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/backfill/backfill.go
Original file line number Diff line number Diff line change
Expand Up @@ -124,7 +124,7 @@ func (cb *ColumnBackfiller) init(

tableArgs := row.FetcherTableArgs{
Desc: desc,
Index: desc.GetPrimaryIndex(),
Index: desc.PrimaryIndexInterface().IndexDesc(),
ColIdxMap: desc.ColumnIdxMap(),
Cols: desc.Columns,
ValNeededForCol: valNeededForCol,
Expand Down Expand Up @@ -601,7 +601,7 @@ func (ib *IndexBackfiller) init(

tableArgs := row.FetcherTableArgs{
Desc: desc,
Index: desc.GetPrimaryIndex(),
Index: desc.PrimaryIndexInterface().IndexDesc(),
ColIdxMap: ib.colIdxMap,
Cols: ib.cols,
ValNeededForCol: valNeededForCol,
Expand Down
1 change: 0 additions & 1 deletion pkg/sql/catalog/descriptor.go
Original file line number Diff line number Diff line change
Expand Up @@ -112,7 +112,6 @@ type TableDescriptor interface {
GetFormatVersion() descpb.FormatVersion

GetPrimaryIndexID() descpb.IndexID
GetPrimaryIndex() *descpb.IndexDescriptor // deprecated
PrimaryIndexSpan(codec keys.SQLCodec) roachpb.Span
IndexSpan(codec keys.SQLCodec, id descpb.IndexID) roachpb.Span
GetIndexMutationCapabilities(id descpb.IndexID) (isMutation, isWriteOnly bool)
Expand Down
7 changes: 0 additions & 7 deletions pkg/sql/catalog/tabledesc/structured.go
Original file line number Diff line number Diff line change
Expand Up @@ -239,13 +239,6 @@ func (desc *Mutable) SetName(name string) {
desc.Name = name
}

// GetPrimaryIndex returns a pointer to the primary index of the table
// descriptor.
// This method is deprecated, use PrimaryIndexInterface instead.
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
Expand Down
8 changes: 4 additions & 4 deletions pkg/sql/catalog/tabledesc/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -339,9 +339,9 @@ func FindFKReferencedIndex(
) (*descpb.IndexDescriptor, error) {
// Search for a unique index on the referenced table that matches our foreign
// key columns.
primaryIndex := referencedTable.GetPrimaryIndex()
primaryIndex := referencedTable.PrimaryIndexInterface()
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.
for _, idx := range referencedTable.PublicNonPrimaryIndexes() {
Expand All @@ -363,8 +363,8 @@ func FindFKOriginIndex(
) (*descpb.IndexDescriptor, error) {
// Search for an index on the origin table that matches our foreign
// key columns.
if primaryIndex := originTable.GetPrimaryIndex(); primaryIndex.IsValidOriginIndex(originColIDs) {
return primaryIndex, nil
if primaryIndex := originTable.PrimaryIndexInterface(); primaryIndex.IsValidOriginIndex(originColIDs) {
return primaryIndex.IndexDesc(), nil
}
// If the PK doesn't match, find the index corresponding to the origin column.
for _, idx := range originTable.PublicNonPrimaryIndexes() {
Expand Down
6 changes: 4 additions & 2 deletions pkg/sql/check.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.PrimaryIndexInterface().NumColumns(); i++ {
id := srcTbl.PrimaryIndexInterface().GetColumnID(i)
alreadyPresent := false
for _, otherID := range fk.OriginColumnIDs {
if id == otherID {
Expand Down Expand Up @@ -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.PrimaryIndexInterface().NumColumns(); i++ {
pkColID := srcTbl.PrimaryIndexInterface().GetColumnID(i)
found := false
for _, id := range fk.OriginColumnIDs {
if pkColID == id {
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/create_index.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.PrimaryIndexInterface().GetPartitioning().NumColumns > 0 {
params.p.BufferClientNotice(
params.ctx,
errors.WithHint(
Expand Down
9 changes: 6 additions & 3 deletions pkg/sql/create_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -350,16 +350,19 @@ func createStatsDefaultColumns(
}

// Add column stats for the primary key.
for i := range desc.GetPrimaryIndex().ColumnIDs {
for i := 0; i < desc.PrimaryIndexInterface().NumColumns(); i++ {
// Generate stats for each column in the primary key.
addIndexColumnStatsIfNotExists(desc.GetPrimaryIndex().ColumnIDs[i], false /* isInverted */)
addIndexColumnStatsIfNotExists(desc.PrimaryIndexInterface().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.PrimaryIndexInterface().GetColumnID(j)
}

// Remember the requested stats so we don't request duplicates.
trackStatsIfNotExists(colIDs)
Expand Down
Loading

0 comments on commit 5148f62

Please sign in to comment.