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: #137798

Release note: None
  • Loading branch information
annrpom committed Jan 31, 2025
1 parent 7963889 commit e29b21b
Show file tree
Hide file tree
Showing 4 changed files with 116 additions and 36 deletions.
1 change: 1 addition & 0 deletions pkg/sql/backfill/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,7 @@ go_test(
],
embed = [":backfill"],
deps = [
"//pkg/keys",
"//pkg/kv",
"//pkg/roachpb",
"//pkg/sql/catalog",
Expand Down
34 changes: 27 additions & 7 deletions pkg/sql/backfill/backfill.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ import (
"time"
"unsafe"

"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings"
Expand Down Expand Up @@ -477,6 +478,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 @@ -506,7 +511,7 @@ func (ib *IndexBackfiller) InitForLocalUse(
) error {

// Initialize ib.added.
ib.initIndexes(desc, nil /* allowList */)
ib.initIndexes(evalCtx.Codec, desc, nil /* allowList */)

// Initialize ib.cols and ib.colIdxMap.
if err := ib.initCols(desc); err != nil {
Expand Down Expand Up @@ -644,18 +649,18 @@ func (ib *IndexBackfiller) InitForDistributedUse(
allowList []catid.IndexID,
mon *mon.BytesMonitor,
) error {
// We'll be modifying the eval.Context in BuildIndexEntriesChunk, so we need
// to make a copy.
evalCtx := flowCtx.NewEvalCtx()

// Initialize ib.added.
ib.initIndexes(desc, allowList)
ib.initIndexes(evalCtx.Codec, desc, allowList)

// Initialize ib.indexBackfillerCols.
if err := ib.initCols(desc); err != nil {
return err
}

// We'll be modifying the eval.Context in BuildIndexEntriesChunk, so we need
// to make a copy.
evalCtx := flowCtx.NewEvalCtx()
var predicates map[descpb.IndexID]tree.TypedExpr
var colExprs map[descpb.ColumnID]tree.TypedExpr
var referencedColumns catalog.TableColSet
Expand Down Expand Up @@ -733,17 +738,22 @@ func (ib *IndexBackfiller) initCols(desc catalog.TableDescriptor) (err error) {
}

// initIndexes is a helper to populate index metadata of an IndexBackfiller. It
// populates the added field to be all adding index mutations.
// populates the added field to be all adding index mutations, along with the
// keyPrefixes field to be the respective keyPrefixes (these slices should
// maintain the same ordering).
// If `allowList` is non-nil, we only add those in this list.
// If `allowList` is nil, we add all adding index mutations.
func (ib *IndexBackfiller) initIndexes(desc catalog.TableDescriptor, allowList []catid.IndexID) {
func (ib *IndexBackfiller) initIndexes(
codec keys.SQLCodec, desc catalog.TableDescriptor, allowList []catid.IndexID,
) {
var allowListAsSet catid.IndexSet
if len(allowList) > 0 {
allowListAsSet = catid.MakeIndexIDSet(allowList...)
}

mutations := desc.AllMutations()
mutationID := mutations[0].MutationID()
ib.keyPrefixes = make([][]byte, 0, len(ib.added))
// Mutations in the same transaction have the same ID. Loop through the
// mutations and collect all index mutations.
for _, m := range mutations {
Expand All @@ -754,6 +764,8 @@ func (ib *IndexBackfiller) initIndexes(desc catalog.TableDescriptor, allowList [
(allowListAsSet.Empty() || allowListAsSet.Contains(m.AsIndex().GetID())) {
idx := m.AsIndex()
ib.added = append(ib.added, idx)
keyPrefix := rowenc.MakeIndexKeyPrefix(codec, desc.GetID(), idx.GetID())
ib.keyPrefixes = append(ib.keyPrefixes, keyPrefix)
}
}
}
Expand All @@ -776,6 +788,7 @@ func (ib *IndexBackfiller) init(
ib.indexesToEncode = ib.added
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 @@ -918,6 +931,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 @@ -943,11 +957,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 @@ -962,6 +979,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 @@ -983,6 +1002,7 @@ func (ib *IndexBackfiller) BuildIndexEntriesChunk(
ib.evalCtx.Codec,
tableDesc,
ib.indexesToEncode,
ib.keyPrefixes,
ib.colIdxMap,
ib.rowVals,
buffer,
Expand Down
5 changes: 3 additions & 2 deletions pkg/sql/backfill/index_backfiller_cols_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,7 @@ package backfill
import (
"testing"

"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catenumpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
Expand Down Expand Up @@ -411,15 +412,15 @@ func TestInitIndexesAllowList(t *testing.T) {
t.Run("nil allowList", func(t *testing.T) {
// A nil allowList means no filtering.
ib := &IndexBackfiller{}
ib.initIndexes(desc, nil /* allowList */)
ib.initIndexes(keys.SystemSQLCodec, desc, nil /* allowList */)
require.Equal(t, 2, len(ib.added))
require.Equal(t, catid.IndexID(2), ib.added[0].GetID())
require.Equal(t, catid.IndexID(3), ib.added[1].GetID())
})

t.Run("non-nil allowList", func(t *testing.T) {
ib := &IndexBackfiller{}
ib.initIndexes(desc, []catid.IndexID{3} /* allowList */)
ib.initIndexes(keys.SystemSQLCodec, desc, []catid.IndexID{3} /* allowList */)
require.Equal(t, 1, len(ib.added))
require.Equal(t, catid.IndexID(3), ib.added[0].GetID())
})
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 @@ -1125,14 +1125,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 @@ -1142,6 +1136,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 @@ -1284,9 +1297,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 @@ -1295,29 +1307,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() == idxtype.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 @@ -1326,13 +1349,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 @@ -1593,14 +1639,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 @@ -1613,8 +1663,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]
// TODO(annie): For now, we recompute the key prefix of inverted indexes. This is because index
// keys with multiple associated values somehow get encoded into the same kv pair when using
// our precomputed key prefix. `inverted_index/arrays` (logictest) illustrates this issue.
if idx.GetType() == idxtype.INVERTED {
keyPrefix = MakeIndexKeyPrefix(codec, tableDesc.GetID(), idx.GetID())
}
entries, err := EncodeSecondaryIndexWithKeyPrefix(ctx, tableDesc, idx, keyPrefix, colMap, values,
includeEmpty)
if err != nil {
return secondaryIndexEntries, 0, err
}
Expand Down

0 comments on commit e29b21b

Please sign in to comment.