Skip to content

Commit

Permalink
sql: rename PrimaryIndexInterface method to GetPrimaryIndex
Browse files Browse the repository at this point in the history
Previously, the name PrimaryIndexInterface had been chosen because
a deprecated GetPrimaryIndex() method already existed and because
PrimaryIndex is an existing field in descpb.IndexDescriptor which is
embedded in the structs implementing the catalog.TableDescriptor
interface.

This patch renames the method to GetPrimaryIndex: although its return
type differs it has the same meaning as the recently-removed method of
the same name.

Release note: None
  • Loading branch information
Marius Posta committed Jan 4, 2021
1 parent 5148f62 commit adf0ec1
Show file tree
Hide file tree
Showing 42 changed files with 150 additions and 150 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.PrimaryIndexInterface().IndexDesc())
indexSchema, err := indexToAvroSchema(tableDesc, tableDesc.GetPrimaryIndex().IndexDesc())
require.NoError(t, err)
require.Equal(t,
`{"type":"record","name":"_u2603_","fields":[`+
Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/changefeedccl/encoder.go
Original file line number Diff line number Diff line change
Expand Up @@ -142,7 +142,7 @@ func (e *jsonEncoder) EncodeKey(_ context.Context, row encodeRow) ([]byte, error

func (e *jsonEncoder) encodeKeyRaw(row encodeRow) ([]interface{}, error) {
colIdxByID := row.tableDesc.ColumnIdxMap()
primaryIndex := row.tableDesc.PrimaryIndexInterface()
primaryIndex := row.tableDesc.GetPrimaryIndex()
jsonEntries := make([]interface{}, primaryIndex.NumColumns())
for i := 0; i < primaryIndex.NumColumns(); i++ {
colID := primaryIndex.GetColumnID(i)
Expand Down Expand Up @@ -344,7 +344,7 @@ func (e *confluentAvroEncoder) EncodeKey(ctx context.Context, row encodeRow) ([]
registered, ok := e.keyCache[cacheKey]
if !ok {
var err error
registered.schema, err = indexToAvroSchema(row.tableDesc, row.tableDesc.PrimaryIndexInterface().IndexDesc())
registered.schema, err = indexToAvroSchema(row.tableDesc, row.tableDesc.GetPrimaryIndex().IndexDesc())
if err != nil {
return nil, err
}
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 < tableDesc.PrimaryIndexInterface().NumColumns(); skippedCols++ {
for ; skippedCols < tableDesc.GetPrimaryIndex().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.PrimaryIndexInterface().IndexDesc(),
Index: tableDesc.GetPrimaryIndex().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.PrimaryIndexInterface().IndexDesc(), types, nameRow, nil, kv.Key)
_, matches, _, err := rowenc.DecodeIndexKey(codec, tbl, tbl.GetPrimaryIndex().IndexDesc(), types, nameRow, nil, kv.Key)
if err != nil {
return errors.Wrap(err, "failed to decode key")
}
Expand Down
12 changes: 6 additions & 6 deletions pkg/sql/alter_primary_key.go
Original file line number Diff line number Diff line change
Expand Up @@ -91,12 +91,12 @@ func (p *planner) AlterPrimaryKey(
}

// Disable primary key changes on tables that are interleaved parents.
if tableDesc.PrimaryIndexInterface().NumInterleavedBy() != 0 {
if tableDesc.GetPrimaryIndex().NumInterleavedBy() != 0 {
var sb strings.Builder
sb.WriteString("[")
comma := ", "
for i := 0; i < tableDesc.PrimaryIndexInterface().NumInterleavedBy(); i++ {
interleaveTableID := tableDesc.PrimaryIndexInterface().GetInterleavedBy(i).Table
for i := 0; i < tableDesc.GetPrimaryIndex().NumInterleavedBy(); i++ {
interleaveTableID := tableDesc.GetPrimaryIndex().GetInterleavedBy(i).Table
if i != 0 {
sb.WriteString(comma)
}
Expand Down Expand Up @@ -222,7 +222,7 @@ func (p *planner) AlterPrimaryKey(
// Create a new index that indexes everything the old primary index
// does, but doesn't store anything.
if shouldCopyPrimaryKey(tableDesc, newPrimaryIndexDesc) {
oldPrimaryIndexCopy := tableDesc.PrimaryIndexInterface().IndexDescDeepCopy()
oldPrimaryIndexCopy := tableDesc.GetPrimaryIndex().IndexDescDeepCopy()
// Clear the name of the index so that it gets generated by AllocateIDs.
oldPrimaryIndexCopy.Name = ""
oldPrimaryIndexCopy.StoreColumnIDs = nil
Expand Down Expand Up @@ -314,7 +314,7 @@ func (p *planner) AlterPrimaryKey(

// Mark the primary key of the table as valid.
{
primaryIndex := *tableDesc.PrimaryIndexInterface().IndexDesc()
primaryIndex := *tableDesc.GetPrimaryIndex().IndexDesc()
primaryIndex.Disabled = false
tableDesc.SetPrimaryIndex(primaryIndex)
}
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.PrimaryIndexInterface()
oldPK := desc.GetPrimaryIndex()
if !desc.HasPrimaryKey() {
return false
}
Expand Down
8 changes: 4 additions & 4 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.PrimaryIndexInterface().ContainsColumnID(colToDrop.ID) {
if n.tableDesc.GetPrimaryIndex().ContainsColumnID(colToDrop.ID) {
return pgerror.Newf(pgcode.InvalidColumnReference,
"column %q is referenced by the primary key", colToDrop.Name)
}
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.PrimaryIndexInterface().ContainsColumnID(id) {
if n.tableDesc.GetPrimaryIndex().ContainsColumnID(id) {
// All secondary indices necessary contain the PK
// columns, too. (See the comments on the definition of
// IndexDescriptor). The presence of a PK column in the
Expand Down Expand Up @@ -740,7 +740,7 @@ func (n *alterTableNode) startExec(params runParams) error {
descriptorChanged = true

case *tree.AlterTablePartitionBy:
primaryIndex := n.tableDesc.PrimaryIndexInterface()
primaryIndex := n.tableDesc.GetPrimaryIndex()
newPartitioning, err := CreatePartitioning(
params.ctx, params.p.ExecCfg().Settings,
params.EvalContext(),
Expand Down Expand Up @@ -1035,7 +1035,7 @@ func applyColumnMutation(
}

// Prevent a column in a primary key from becoming non-null.
if tableDesc.PrimaryIndexInterface().ContainsColumnID(col.ID) {
if tableDesc.GetPrimaryIndex().ContainsColumnID(col.ID) {
return pgerror.Newf(pgcode.InvalidTableDefinition,
`column "%s" is in a primary index`, col.Name)
}
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.PrimaryIndexInterface().IndexDesc(),
Index: desc.GetPrimaryIndex().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.PrimaryIndexInterface().IndexDesc(),
Index: desc.GetPrimaryIndex().IndexDesc(),
ColIdxMap: ib.colIdxMap,
Cols: ib.cols,
ValNeededForCol: valNeededForCol,
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/catalog/descriptor.go
Original file line number Diff line number Diff line change
Expand Up @@ -117,7 +117,7 @@ type TableDescriptor interface {
GetIndexMutationCapabilities(id descpb.IndexID) (isMutation, isWriteOnly bool)
KeysPerRow(id descpb.IndexID) (int, error)

PrimaryIndexInterface() Index
GetPrimaryIndex() Index
AllIndexes() []Index
ActiveIndexes() []Index
NonDropIndexes() []Index
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/catalog/tabledesc/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -339,7 +339,7 @@ func FindFKReferencedIndex(
) (*descpb.IndexDescriptor, error) {
// Search for a unique index on the referenced table that matches our foreign
// key columns.
primaryIndex := referencedTable.PrimaryIndexInterface()
primaryIndex := referencedTable.GetPrimaryIndex()
if primaryIndex.IsValidReferencedIndex(referencedColIDs) {
return primaryIndex.IndexDesc(), nil
}
Expand All @@ -363,7 +363,7 @@ func FindFKOriginIndex(
) (*descpb.IndexDescriptor, error) {
// Search for an index on the origin table that matches our foreign
// key columns.
if primaryIndex := originTable.PrimaryIndexInterface(); primaryIndex.IsValidOriginIndex(originColIDs) {
if primaryIndex := originTable.GetPrimaryIndex(); primaryIndex.IsValidOriginIndex(originColIDs) {
return primaryIndex.IndexDesc(), nil
}
// If the PK doesn't match, find the index corresponding to the origin column.
Expand Down
6 changes: 3 additions & 3 deletions pkg/sql/catalog/tabledesc/table_desc.go
Original file line number Diff line number Diff line change
Expand Up @@ -252,9 +252,9 @@ func (desc *Mutable) SetPublicNonPrimaryIndex(indexOrdinal int, index descpb.Ind
desc.Indexes[indexOrdinal-1] = index
}

// PrimaryIndexInterface returns the primary index in the form of a
// catalog.Index interface.
func (desc *wrapper) PrimaryIndexInterface() catalog.Index {
// GetPrimaryIndex returns the primary index in the form of a catalog.Index
// interface.
func (desc *wrapper) GetPrimaryIndex() catalog.Index {
return index{desc: &desc.PrimaryIndex}
}

Expand Down
8 changes: 4 additions & 4 deletions pkg/sql/check.go
Original file line number Diff line number Diff line change
Expand Up @@ -96,8 +96,8 @@ func matchFullUnacceptableKeyQuery(
srcNotNullExistsClause[i] = fmt.Sprintf("%s IS NOT NULL", srcCols[i])
}

for i := 0; i < srcTbl.PrimaryIndexInterface().NumColumns(); i++ {
id := srcTbl.PrimaryIndexInterface().GetColumnID(i)
for i := 0; i < srcTbl.GetPrimaryIndex().NumColumns(); i++ {
id := srcTbl.GetPrimaryIndex().GetColumnID(i)
alreadyPresent := false
for _, otherID := range fk.OriginColumnIDs {
if id == otherID {
Expand Down Expand Up @@ -160,8 +160,8 @@ func nonMatchingRowQuery(
return "", nil, err
}
// Get primary key columns not included in the FK
for i := 0; i < srcTbl.PrimaryIndexInterface().NumColumns(); i++ {
pkColID := srcTbl.PrimaryIndexInterface().GetColumnID(i)
for i := 0; i < srcTbl.GetPrimaryIndex().NumColumns(); i++ {
pkColID := srcTbl.GetPrimaryIndex().GetColumnID(i)
found := false
for _, id := range fk.OriginColumnIDs {
if pkColID == id {
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.PrimaryIndexInterface().GetPartitioning().NumColumns > 0 {
if n.n.PartitionBy == nil && n.tableDesc.GetPrimaryIndex().GetPartitioning().NumColumns > 0 {
params.p.BufferClientNotice(
params.ctx,
errors.WithHint(
Expand Down
6 changes: 3 additions & 3 deletions pkg/sql/create_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -350,9 +350,9 @@ func createStatsDefaultColumns(
}

// Add column stats for the primary key.
for i := 0; i < desc.PrimaryIndexInterface().NumColumns(); i++ {
for i := 0; i < desc.GetPrimaryIndex().NumColumns(); i++ {
// Generate stats for each column in the primary key.
addIndexColumnStatsIfNotExists(desc.PrimaryIndexInterface().GetColumnID(i), false /* isInverted */)
addIndexColumnStatsIfNotExists(desc.GetPrimaryIndex().GetColumnID(i), false /* isInverted */)

// Only collect multi-column stats if enabled.
if i == 0 || !multiColEnabled {
Expand All @@ -361,7 +361,7 @@ func createStatsDefaultColumns(

colIDs := make([]descpb.ColumnID, i+1)
for j := 0; j <= i; j++ {
colIDs[j] = desc.PrimaryIndexInterface().GetColumnID(j)
colIDs[j] = desc.GetPrimaryIndex().GetColumnID(j)
}

// Remember the requested stats so we don't request duplicates.
Expand Down
14 changes: 7 additions & 7 deletions pkg/sql/create_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -785,9 +785,9 @@ func ResolveFK(
referencedColNames := d.ToCols
// If no columns are specified, attempt to default to PK.
if len(referencedColNames) == 0 {
referencedColNames = make(tree.NameList, target.PrimaryIndexInterface().NumColumns())
referencedColNames = make(tree.NameList, target.GetPrimaryIndex().NumColumns())
for i := range referencedColNames {
referencedColNames[i] = tree.Name(target.PrimaryIndexInterface().GetColumnName(i))
referencedColNames[i] = tree.Name(target.GetPrimaryIndex().GetColumnName(i))
}
}

Expand Down Expand Up @@ -1025,7 +1025,7 @@ func addInterleave(
if err != nil {
return err
}
parentIndex := parentTable.PrimaryIndexInterface()
parentIndex := parentTable.GetPrimaryIndex()

// typeOfIndex is used to give more informative error messages.
var typeOfIndex string
Expand Down Expand Up @@ -1647,7 +1647,7 @@ func NewTableDesc(
}

// If explicit primary keys are required, error out since a primary key was not supplied.
if desc.PrimaryIndexInterface().NumColumns() == 0 && desc.IsPhysicalTable() && evalCtx != nil &&
if desc.GetPrimaryIndex().NumColumns() == 0 && desc.IsPhysicalTable() && evalCtx != nil &&
evalCtx.SessionData != nil && evalCtx.SessionData.RequireExplicitPrimaryKeys {
return nil, errors.Errorf(
"no primary key specified for table %s (require_explicit_primary_keys = true)", desc.Name)
Expand Down Expand Up @@ -1707,19 +1707,19 @@ func NewTableDesc(
}

if n.Interleave != nil {
if err := addInterleave(ctx, txn, vt, &desc, desc.PrimaryIndexInterface().IndexDesc(), n.Interleave); err != nil {
if err := addInterleave(ctx, txn, vt, &desc, desc.GetPrimaryIndex().IndexDesc(), n.Interleave); err != nil {
return nil, err
}
}

if n.PartitionBy != nil {
partitioning, err := CreatePartitioning(
ctx, st, evalCtx, &desc, desc.PrimaryIndexInterface().IndexDesc(), n.PartitionBy)
ctx, st, evalCtx, &desc, desc.GetPrimaryIndex().IndexDesc(), n.PartitionBy)
if err != nil {
return nil, err
}
{
newPrimaryIndex := *desc.PrimaryIndexInterface().IndexDesc()
newPrimaryIndex := *desc.GetPrimaryIndex().IndexDesc()
newPrimaryIndex.Partitioning = partitioning
desc.SetPrimaryIndex(newPrimaryIndex)
}
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/delete_range.go
Original file line number Diff line number Diff line change
Expand Up @@ -96,13 +96,13 @@ func (d *deleteRangeNode) startExec(params runParams) error {
allTables := make([]row.FetcherTableArgs, len(d.interleavedDesc)+1)
allTables[0] = row.FetcherTableArgs{
Desc: d.desc,
Index: d.desc.PrimaryIndexInterface().IndexDesc(),
Index: d.desc.GetPrimaryIndex().IndexDesc(),
Spans: d.spans,
}
for i, interleaved := range d.interleavedDesc {
allTables[i+1] = row.FetcherTableArgs{
Desc: interleaved,
Index: interleaved.PrimaryIndexInterface().IndexDesc(),
Index: interleaved.GetPrimaryIndex().IndexDesc(),
Spans: d.spans,
}
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/drop_index.go
Original file line number Diff line number Diff line change
Expand Up @@ -313,7 +313,7 @@ func (p *planner) dropIndexByName(
// is another index that could replace the one we are deleting for a given
// foreign key constraint.
remainingIndexes := make([]*descpb.IndexDescriptor, 1, len(tableDesc.ActiveIndexes()))
remainingIndexes[0] = tableDesc.PrimaryIndexInterface().IndexDesc()
remainingIndexes[0] = tableDesc.GetPrimaryIndex().IndexDesc()
for _, index := range tableDesc.PublicNonPrimaryIndexes() {
if index.GetID() != idx.ID {
remainingIndexes = append(remainingIndexes, index.IndexDesc())
Expand Down
8 changes: 4 additions & 4 deletions pkg/sql/information_schema.go
Original file line number Diff line number Diff line change
Expand Up @@ -1312,8 +1312,8 @@ CREATE TABLE information_schema.statistics (
}
if hasImplicitCols {
implicitCols = make(map[string]struct{})
for i := 0; i < table.PrimaryIndexInterface().NumColumns(); i++ {
col := table.PrimaryIndexInterface().GetColumnName(i)
for i := 0; i < table.GetPrimaryIndex().NumColumns(); i++ {
col := table.GetPrimaryIndex().GetColumnName(i)
implicitCols[col] = struct{}{}
}
}
Expand Down Expand Up @@ -1346,8 +1346,8 @@ CREATE TABLE information_schema.statistics (
//
// Note that simply iterating over implicitCols map
// produces non-deterministic output.
for i := 0; i < table.PrimaryIndexInterface().NumColumns(); i++ {
col := table.PrimaryIndexInterface().GetColumnName(i)
for i := 0; i < table.GetPrimaryIndex().NumColumns(); i++ {
col := table.GetPrimaryIndex().GetColumnName(i)
if _, isImplicit := implicitCols[col]; isImplicit {
// We add a row for each implicit column of index.
if err := appendRow(index.IndexDesc(), col, sequence,
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/opt_catalog.go
Original file line number Diff line number Diff line change
Expand Up @@ -719,7 +719,7 @@ func newOptTable(
for i := range ot.indexes {
var idxDesc *descpb.IndexDescriptor
if i == 0 {
idxDesc = desc.PrimaryIndexInterface().IndexDesc()
idxDesc = desc.GetPrimaryIndex().IndexDesc()
} else {
idxDesc = secondaryIndexes[i-1].IndexDesc()
}
Expand Down Expand Up @@ -1125,7 +1125,7 @@ func (oi *optIndex) init(
oi.zone = zone
oi.indexOrdinal = indexOrdinal
oi.invertedVirtualColOrd = invertedVirtualColOrd
if desc == tab.desc.PrimaryIndexInterface().IndexDesc() {
if desc == tab.desc.GetPrimaryIndex().IndexDesc() {
// Although the primary index contains all columns in the table, the index
// descriptor does not contain columns that are not explicitly part of the
// primary key. Retrieve those columns from the table descriptor.
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/opt_exec_factory.go
Original file line number Diff line number Diff line change
Expand Up @@ -547,7 +547,7 @@ func (ef *execFactory) ConstructIndexJoin(
return nil, err
}

primaryIndex := tabDesc.PrimaryIndexInterface()
primaryIndex := tabDesc.GetPrimaryIndex()
tableScan.index = primaryIndex.IndexDesc()
tableScan.disableBatchLimit()

Expand Down Expand Up @@ -1557,7 +1557,7 @@ func (ef *execFactory) ConstructDeleteRange(
autoCommit bool,
) (exec.Node, error) {
tabDesc := table.(*optTable).desc
indexDesc := tabDesc.PrimaryIndexInterface().IndexDesc()
indexDesc := tabDesc.GetPrimaryIndex().IndexDesc()
sb := span.MakeBuilder(ef.planner.EvalContext(), ef.planner.ExecCfg().Codec, tabDesc, indexDesc)

if err := ef.planner.maybeSetSystemConfig(tabDesc.GetID()); err != nil {
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/partition_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,7 @@ func TestRemovePartitioningOSS(t *testing.T) {

// Hack in partitions. Doing this properly requires a CCL binary.
{
primaryIndex := *tableDesc.PrimaryIndexInterface().IndexDesc()
primaryIndex := *tableDesc.GetPrimaryIndex().IndexDesc()
primaryIndex.Partitioning = descpb.PartitioningDescriptor{
NumColumns: 1,
Range: []descpb.PartitioningDescriptor_Range{{
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/physicalplan/fake_span_resolver_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -57,7 +57,7 @@ func TestFakeSpanResolver(t *testing.T) {
it := resolver.NewSpanResolverIterator(txn)

tableDesc := catalogkv.TestingGetTableDescriptor(db, keys.SystemSQLCodec, "test", "t")
primIdxValDirs := catalogkeys.IndexKeyValDirs(tableDesc.PrimaryIndexInterface().IndexDesc())
primIdxValDirs := catalogkeys.IndexKeyValDirs(tableDesc.GetPrimaryIndex().IndexDesc())

span := tableDesc.PrimaryIndexSpan(keys.SystemSQLCodec)

Expand Down
Loading

0 comments on commit adf0ec1

Please sign in to comment.