Skip to content

Commit

Permalink
sql/rowenc: reduce index key prefix calls
Browse files Browse the repository at this point in the history
This patch removes redundant calls to `MakeIndexKeyPrefix` during
the construction of `IndexEntry`s by saving each first-time call in a
map that we can later lookup. Previously, we would make this call
for each row; however, as the prefix (table id + index id) for a
particular index remains the same, we do not need to do any
recomputation.

Epic: CRDB-42901
Fixes: cockroachdb#137798

Release note: None
  • Loading branch information
annrpom committed Jan 29, 2025
1 parent e508cf6 commit 2dec4f9
Show file tree
Hide file tree
Showing 2 changed files with 106 additions and 30 deletions.
24 changes: 21 additions & 3 deletions pkg/sql/backfill/backfill.go
Original file line number Diff line number Diff line change
Expand Up @@ -476,6 +476,10 @@ type IndexBackfiller struct {
// backfilled.
indexesToEncode []catalog.Index

// keyPrefixes is a slice of key prefixes for each index in indexesToEncode.
// indexesToEncode and keyPrefixes should both have the same ordering.
keyPrefixes [][]byte

alloc tree.DatumAlloc

// mon is a memory monitor linked with the IndexBackfiller on creation.
Expand Down Expand Up @@ -525,7 +529,7 @@ func (ib *IndexBackfiller) InitForLocalUse(
ib.valNeededForCol.Add(ib.colIdxMap.GetDefault(col))
})

return ib.init(evalCtx, predicates, colExprs, mon)
return ib.init(evalCtx, predicates, colExprs, mon, desc.GetID())
}

// constructExprs is a helper to construct the index and column expressions
Expand Down Expand Up @@ -689,7 +693,7 @@ func (ib *IndexBackfiller) InitForDistributedUse(
ib.valNeededForCol.Add(ib.colIdxMap.GetDefault(col))
})

return ib.init(evalCtx, predicates, colExprs, mon)
return ib.init(evalCtx, predicates, colExprs, mon, desc.GetID())
}

// Close releases the resources used by the IndexBackfiller. It can be called
Expand Down Expand Up @@ -763,18 +767,25 @@ func (ib *IndexBackfiller) init(
predicateExprs map[descpb.IndexID]tree.TypedExpr,
colExprs map[descpb.ColumnID]tree.TypedExpr,
mon *mon.BytesMonitor,
descID descpb.ID,
) error {
ib.evalCtx = evalCtx
ib.predicates = predicateExprs
ib.colExprs = colExprs

// Initialize a list of index descriptors to encode entries for. If there
// Initialize a list of index descriptors to encode entries for, along with a
// slice of key prefixes for each index (with the same ordering). If there
// are no partial indexes, the list is equivalent to the list of indexes
// being added. If there are partial indexes, allocate a new list that is
// reset in BuildIndexEntriesChunk for every row added.
ib.indexesToEncode = ib.added
for _, idx := range ib.added {
keyPrefix := rowenc.MakeIndexKeyPrefix(evalCtx.Codec, descID, idx.GetID())
ib.keyPrefixes = append(ib.keyPrefixes, keyPrefix)
}
if len(ib.predicates) > 0 {
ib.indexesToEncode = make([]catalog.Index, 0, len(ib.added))
ib.keyPrefixes = make([][]byte, 0, len(ib.added))
}

ib.types = make([]*types.T, len(ib.cols))
Expand Down Expand Up @@ -917,6 +928,7 @@ func (ib *IndexBackfiller) BuildIndexEntriesChunk(
}
return nil
}

for i := int64(0); i < chunkSize; i++ {
ok, _, err := fetcher.NextRowDecodedInto(ctx, ib.rowVals, ib.colIdxMap)
if err != nil {
Expand All @@ -942,11 +954,14 @@ func (ib *IndexBackfiller) BuildIndexEntriesChunk(
// indexes that the current row should be added to.
if len(ib.predicates) > 0 {
ib.indexesToEncode = ib.indexesToEncode[:0]
ib.keyPrefixes = ib.keyPrefixes[:0]
for _, idx := range ib.added {
if !idx.IsPartial() {
// If the index is not a partial index, all rows should have
// an entry.
ib.indexesToEncode = append(ib.indexesToEncode, idx)
keyPrefix := rowenc.MakeIndexKeyPrefix(ib.evalCtx.Codec, tableDesc.GetID(), idx.GetID())
ib.keyPrefixes = append(ib.keyPrefixes, keyPrefix)
continue
}

Expand All @@ -961,6 +976,8 @@ func (ib *IndexBackfiller) BuildIndexEntriesChunk(

if val == tree.DBoolTrue {
ib.indexesToEncode = append(ib.indexesToEncode, idx)
keyPrefix := rowenc.MakeIndexKeyPrefix(ib.evalCtx.Codec, tableDesc.GetID(), idx.GetID())
ib.keyPrefixes = append(ib.keyPrefixes, keyPrefix)
}
}
}
Expand All @@ -982,6 +999,7 @@ func (ib *IndexBackfiller) BuildIndexEntriesChunk(
ib.evalCtx.Codec,
tableDesc,
ib.indexesToEncode,
ib.keyPrefixes,
ib.colIdxMap,
ib.rowVals,
buffer,
Expand Down
112 changes: 85 additions & 27 deletions pkg/sql/rowenc/index_encoding.go
Original file line number Diff line number Diff line change
Expand Up @@ -1124,14 +1124,8 @@ func encodeTrigramInvertedIndexTableKeys(
return outKeys, nil
}

// EncodePrimaryIndex constructs a list of k/v pairs for a
// row encoded as a primary index. This function mirrors the encoding
// logic in prepareInsertOrUpdateBatch in pkg/sql/row/writer.go.
// It is somewhat duplicated here due to the different arguments
// that prepareOrInsertUpdateBatch needs and uses to generate
// the k/v's for the row it inserts. includeEmpty controls
// whether or not k/v's with empty values should be returned.
// It returns indexEntries in family sorted order.
// EncodePrimaryIndex constructs the key prefix for the primary index and
// delegates the rest of the encoding to EncodePrimaryIndexWithKeyPrefix.
func EncodePrimaryIndex(
codec keys.SQLCodec,
tableDesc catalog.TableDescriptor,
Expand All @@ -1141,6 +1135,25 @@ func EncodePrimaryIndex(
includeEmpty bool,
) ([]IndexEntry, error) {
keyPrefix := MakeIndexKeyPrefix(codec, tableDesc.GetID(), index.GetID())
return EncodePrimaryIndexWithKeyPrefix(tableDesc, index, keyPrefix, colMap, values, includeEmpty)
}

// EncodePrimaryIndexWithKeyPrefix constructs a list of k/v pairs for a
// row encoded as a primary index, using the provided key prefix specific to
// that index. This function mirrors the encoding logic in
// prepareInsertOrUpdateBatch in pkg/sql/row/writer.go. It is somewhat
// duplicated here due to the different arguments that
// prepareOrInsertUpdateBatch needs and uses to generate the k/v's for the row
// it inserts. includeEmpty controls whether or not k/v's with empty values
// should be returned. It returns indexEntries in family sorted order.
func EncodePrimaryIndexWithKeyPrefix(
tableDesc catalog.TableDescriptor,
index catalog.Index,
keyPrefix []byte,
colMap catalog.TableColMap,
values []tree.Datum,
includeEmpty bool,
) ([]IndexEntry, error) {
indexKey, containsNull, err := EncodeIndexKey(tableDesc, index, colMap, values, keyPrefix)
if err != nil {
return nil, err
Expand Down Expand Up @@ -1283,9 +1296,8 @@ func MakeNullPKError(
return errors.AssertionFailedf("NULL value in unknown key column")
}

// EncodeSecondaryIndexKey encodes the key for a secondary index. The 'colMap'
// maps descpb.ColumnIDs to positions in 'values'. This function returns a slice
// of byte arrays representing the key values.
// EncodeSecondaryIndexKey constructs the key prefix for the secondary index and
// delegates the rest of the encoding to EncodeSecondaryIndexWithKeyPrefix.
func EncodeSecondaryIndexKey(
ctx context.Context,
codec keys.SQLCodec,
Expand All @@ -1294,29 +1306,40 @@ func EncodeSecondaryIndexKey(
colMap catalog.TableColMap,
values []tree.Datum,
) ([][]byte, bool, error) {
secondaryIndexKeyPrefix := MakeIndexKeyPrefix(codec, tableDesc.GetID(), secondaryIndex.GetID())
keyPrefix := MakeIndexKeyPrefix(codec, tableDesc.GetID(), secondaryIndex.GetID())
return EncodeSecondaryIndexKeyWithKeyPrefix(ctx, tableDesc, secondaryIndex, keyPrefix, colMap,
values)
}

// EncodeSecondaryIndexKeyWithKeyPrefix generates a slice of byte arrays
// representing encoded key values for the given secondary index, using the
// provided key prefix specific to that index. The colMap maps descpb.ColumnIDs
// to positions in the values slice.
func EncodeSecondaryIndexKeyWithKeyPrefix(
ctx context.Context,
tableDesc catalog.TableDescriptor,
secondaryIndex catalog.Index,
keyPrefix []byte,
colMap catalog.TableColMap,
values []tree.Datum,
) ([][]byte, bool, error) {
var containsNull = false
var secondaryKeys [][]byte
var err error
if secondaryIndex.GetType() == descpb.IndexDescriptor_INVERTED {
secondaryKeys, err = EncodeInvertedIndexKeys(ctx, secondaryIndex, colMap, values, secondaryIndexKeyPrefix)
secondaryKeys, err = EncodeInvertedIndexKeys(ctx, secondaryIndex, colMap, values, keyPrefix)
} else {
var secondaryIndexKey []byte
secondaryIndexKey, containsNull, err = EncodeIndexKey(
tableDesc, secondaryIndex, colMap, values, secondaryIndexKeyPrefix)
tableDesc, secondaryIndex, colMap, values, keyPrefix)

secondaryKeys = [][]byte{secondaryIndexKey}
}
return secondaryKeys, containsNull, err
}

// EncodeSecondaryIndex encodes key/values for a secondary
// index. colMap maps descpb.ColumnIDs to indices in `values`. This returns a
// slice of IndexEntry. includeEmpty controls whether or not
// EncodeSecondaryIndex should return k/v's that contain
// empty values. For forward indexes the returned list of
// index entries is in family sorted order.
// EncodeSecondaryIndex constructs the key prefix for the secondary index and
// delegates the rest of the encoding to EncodeSecondaryIndexWithKeyPrefix.
func EncodeSecondaryIndex(
ctx context.Context,
codec keys.SQLCodec,
Expand All @@ -1325,13 +1348,36 @@ func EncodeSecondaryIndex(
colMap catalog.TableColMap,
values []tree.Datum,
includeEmpty bool,
) ([]IndexEntry, error) {
keyPrefix := MakeIndexKeyPrefix(codec, tableDesc.GetID(), secondaryIndex.GetID())
return EncodeSecondaryIndexWithKeyPrefix(ctx, tableDesc, secondaryIndex, keyPrefix, colMap,
values, includeEmpty)
}

// EncodeSecondaryIndexWithKeyPrefix generates a slice of IndexEntry objects
// representing encoded key/value pairs for the given secondary index, using the
// provided key prefix specific to that index. This encoding is performed in
// EncodeSecondaryIndexKeyWithKeyPrefix for secondary indexes. The colMap maps
// descpb.ColumnIDs to positions in the values slice. The 'includeEmpty'
// parameter determines whether entries with empty values should be included.
// For forward indexes, the resulting entries are sorted by column family order.
func EncodeSecondaryIndexWithKeyPrefix(
ctx context.Context,
tableDesc catalog.TableDescriptor,
secondaryIndex catalog.Index,
keyPrefix []byte,
colMap catalog.TableColMap,
values []tree.Datum,
includeEmpty bool,
) ([]IndexEntry, error) {
// Use the primary key encoding for covering indexes.
if secondaryIndex.GetEncodingType() == catenumpb.PrimaryIndexEncoding {
return EncodePrimaryIndex(codec, tableDesc, secondaryIndex, colMap, values, includeEmpty)
return EncodePrimaryIndexWithKeyPrefix(tableDesc, secondaryIndex, keyPrefix, colMap, values,
includeEmpty)
}

secondaryKeys, containsNull, err := EncodeSecondaryIndexKey(ctx, codec, tableDesc, secondaryIndex, colMap, values)
secondaryKeys, containsNull, err := EncodeSecondaryIndexKeyWithKeyPrefix(ctx, tableDesc,
secondaryIndex, keyPrefix, colMap, values)
if err != nil {
return []IndexEntry{}, err
}
Expand Down Expand Up @@ -1592,14 +1638,18 @@ func writeColumnValues(
}

// EncodeSecondaryIndexes encodes key/values for the secondary indexes. colMap
// maps descpb.ColumnIDs to indices in `values`. secondaryIndexEntries is the return
// value (passed as a parameter so the caller can reuse between rows) and is
// expected to be the same length as indexes.
// maps descpb.ColumnIDs to indices in `values`. keyPrefixes is a slice that
// associates indexes to their key prefix; the caller can reuse this between
// rows to save work from creating key prefixes. the indexes and keyPrefixes
// slice should have the same ordering. secondaryIndexEntries is the return
// value (passed as a parameter so the caller can reuse between rows) and
// is expected to be the same length as indexes.
func EncodeSecondaryIndexes(
ctx context.Context,
codec keys.SQLCodec,
tableDesc catalog.TableDescriptor,
indexes []catalog.Index,
keyPrefixes [][]byte,
colMap catalog.TableColMap,
values []tree.Datum,
secondaryIndexEntries []IndexEntry,
Expand All @@ -1612,8 +1662,16 @@ func EncodeSecondaryIndexes(
}
const sizeOfIndexEntry = int64(unsafe.Sizeof(IndexEntry{}))

for i := range indexes {
entries, err := EncodeSecondaryIndex(ctx, codec, tableDesc, indexes[i], colMap, values, includeEmpty)
for i, idx := range indexes {
keyPrefix := keyPrefixes[i]
if keyPrefix == nil {
keyPrefix = MakeIndexKeyPrefix(codec, tableDesc.GetID(), idx.GetID())
if idx.GetType() != descpb.IndexDescriptor_INVERTED {
keyPrefixes[i] = keyPrefix
}
}
entries, err := EncodeSecondaryIndexWithKeyPrefix(ctx, tableDesc, idx, keyPrefix, colMap, values,
includeEmpty)
if err != nil {
return secondaryIndexEntries, 0, err
}
Expand Down

0 comments on commit 2dec4f9

Please sign in to comment.