Skip to content

Commit 6b3b4fb

Browse files
authored
Merge pull request #140275 from annrpom/backport25.1-139256
release-25.1: sql/rowenc: reduce index key prefix calls
2 parents 1c87a84 + b34aef1 commit 6b3b4fb

File tree

5 files changed

+122
-36
lines changed

5 files changed

+122
-36
lines changed

pkg/sql/backfill/BUILD.bazel

+1
Original file line numberDiff line numberDiff line change
@@ -57,6 +57,7 @@ go_test(
5757
],
5858
embed = [":backfill"],
5959
deps = [
60+
"//pkg/keys",
6061
"//pkg/kv",
6162
"//pkg/roachpb",
6263
"//pkg/sql/catalog",

pkg/sql/backfill/backfill.go

+27-7
Original file line numberDiff line numberDiff line change
@@ -12,6 +12,7 @@ import (
1212
"time"
1313
"unsafe"
1414

15+
"github.com/cockroachdb/cockroach/pkg/keys"
1516
"github.com/cockroachdb/cockroach/pkg/kv"
1617
"github.com/cockroachdb/cockroach/pkg/roachpb"
1718
"github.com/cockroachdb/cockroach/pkg/settings"
@@ -476,6 +477,10 @@ type IndexBackfiller struct {
476477
// backfilled.
477478
indexesToEncode []catalog.Index
478479

480+
// keyPrefixes is a slice of key prefixes for each index in indexesToEncode.
481+
// indexesToEncode and keyPrefixes should both have the same ordering.
482+
keyPrefixes [][]byte
483+
479484
alloc tree.DatumAlloc
480485

481486
// mon is a memory monitor linked with the IndexBackfiller on creation.
@@ -505,7 +510,7 @@ func (ib *IndexBackfiller) InitForLocalUse(
505510
) error {
506511

507512
// Initialize ib.added.
508-
ib.initIndexes(desc, nil /* allowList */)
513+
ib.initIndexes(evalCtx.Codec, desc, nil /* allowList */)
509514

510515
// Initialize ib.cols and ib.colIdxMap.
511516
if err := ib.initCols(desc); err != nil {
@@ -643,18 +648,18 @@ func (ib *IndexBackfiller) InitForDistributedUse(
643648
allowList []catid.IndexID,
644649
mon *mon.BytesMonitor,
645650
) error {
651+
// We'll be modifying the eval.Context in BuildIndexEntriesChunk, so we need
652+
// to make a copy.
653+
evalCtx := flowCtx.NewEvalCtx()
646654

647655
// Initialize ib.added.
648-
ib.initIndexes(desc, allowList)
656+
ib.initIndexes(evalCtx.Codec, desc, allowList)
649657

650658
// Initialize ib.indexBackfillerCols.
651659
if err := ib.initCols(desc); err != nil {
652660
return err
653661
}
654662

655-
// We'll be modifying the eval.Context in BuildIndexEntriesChunk, so we need
656-
// to make a copy.
657-
evalCtx := flowCtx.NewEvalCtx()
658663
var predicates map[descpb.IndexID]tree.TypedExpr
659664
var colExprs map[descpb.ColumnID]tree.TypedExpr
660665
var referencedColumns catalog.TableColSet
@@ -732,17 +737,22 @@ func (ib *IndexBackfiller) initCols(desc catalog.TableDescriptor) (err error) {
732737
}
733738

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

744753
mutations := desc.AllMutations()
745754
mutationID := mutations[0].MutationID()
755+
ib.keyPrefixes = make([][]byte, 0, len(ib.added))
746756
// Mutations in the same transaction have the same ID. Loop through the
747757
// mutations and collect all index mutations.
748758
for _, m := range mutations {
@@ -753,6 +763,8 @@ func (ib *IndexBackfiller) initIndexes(desc catalog.TableDescriptor, allowList [
753763
(allowListAsSet.Empty() || allowListAsSet.Contains(m.AsIndex().GetID())) {
754764
idx := m.AsIndex()
755765
ib.added = append(ib.added, idx)
766+
keyPrefix := rowenc.MakeIndexKeyPrefix(codec, desc.GetID(), idx.GetID())
767+
ib.keyPrefixes = append(ib.keyPrefixes, keyPrefix)
756768
}
757769
}
758770
}
@@ -775,6 +787,7 @@ func (ib *IndexBackfiller) init(
775787
ib.indexesToEncode = ib.added
776788
if len(ib.predicates) > 0 {
777789
ib.indexesToEncode = make([]catalog.Index, 0, len(ib.added))
790+
ib.keyPrefixes = make([][]byte, 0, len(ib.added))
778791
}
779792

780793
ib.types = make([]*types.T, len(ib.cols))
@@ -917,6 +930,7 @@ func (ib *IndexBackfiller) BuildIndexEntriesChunk(
917930
}
918931
return nil
919932
}
933+
920934
for i := int64(0); i < chunkSize; i++ {
921935
ok, err := fetcher.NextRowDecodedInto(ctx, ib.rowVals, ib.colIdxMap)
922936
if err != nil {
@@ -942,11 +956,14 @@ func (ib *IndexBackfiller) BuildIndexEntriesChunk(
942956
// indexes that the current row should be added to.
943957
if len(ib.predicates) > 0 {
944958
ib.indexesToEncode = ib.indexesToEncode[:0]
959+
ib.keyPrefixes = ib.keyPrefixes[:0]
945960
for _, idx := range ib.added {
946961
if !idx.IsPartial() {
947962
// If the index is not a partial index, all rows should have
948963
// an entry.
949964
ib.indexesToEncode = append(ib.indexesToEncode, idx)
965+
keyPrefix := rowenc.MakeIndexKeyPrefix(ib.evalCtx.Codec, tableDesc.GetID(), idx.GetID())
966+
ib.keyPrefixes = append(ib.keyPrefixes, keyPrefix)
950967
continue
951968
}
952969

@@ -961,6 +978,8 @@ func (ib *IndexBackfiller) BuildIndexEntriesChunk(
961978

962979
if val == tree.DBoolTrue {
963980
ib.indexesToEncode = append(ib.indexesToEncode, idx)
981+
keyPrefix := rowenc.MakeIndexKeyPrefix(ib.evalCtx.Codec, tableDesc.GetID(), idx.GetID())
982+
ib.keyPrefixes = append(ib.keyPrefixes, keyPrefix)
964983
}
965984
}
966985
}
@@ -982,6 +1001,7 @@ func (ib *IndexBackfiller) BuildIndexEntriesChunk(
9821001
ib.evalCtx.Codec,
9831002
tableDesc,
9841003
ib.indexesToEncode,
1004+
ib.keyPrefixes,
9851005
ib.colIdxMap,
9861006
ib.rowVals,
9871007
buffer,

pkg/sql/backfill/index_backfiller_cols_test.go

+3-2
Original file line numberDiff line numberDiff line change
@@ -8,6 +8,7 @@ package backfill
88
import (
99
"testing"
1010

11+
"github.com/cockroachdb/cockroach/pkg/keys"
1112
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
1213
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catenumpb"
1314
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
@@ -411,15 +412,15 @@ func TestInitIndexesAllowList(t *testing.T) {
411412
t.Run("nil allowList", func(t *testing.T) {
412413
// A nil allowList means no filtering.
413414
ib := &IndexBackfiller{}
414-
ib.initIndexes(desc, nil /* allowList */)
415+
ib.initIndexes(keys.SystemSQLCodec, desc, nil /* allowList */)
415416
require.Equal(t, 2, len(ib.added))
416417
require.Equal(t, catid.IndexID(2), ib.added[0].GetID())
417418
require.Equal(t, catid.IndexID(3), ib.added[1].GetID())
418419
})
419420

420421
t.Run("non-nil allowList", func(t *testing.T) {
421422
ib := &IndexBackfiller{}
422-
ib.initIndexes(desc, []catid.IndexID{3} /* allowList */)
423+
ib.initIndexes(keys.SystemSQLCodec, desc, []catid.IndexID{3} /* allowList */)
423424
require.Equal(t, 1, len(ib.added))
424425
require.Equal(t, catid.IndexID(3), ib.added[0].GetID())
425426
})

pkg/sql/rowenc/index_encoding.go

+85-27
Original file line numberDiff line numberDiff line change
@@ -1124,14 +1124,8 @@ func encodeTrigramInvertedIndexTableKeys(
11241124
return outKeys, nil
11251125
}
11261126

1127-
// EncodePrimaryIndex constructs a list of k/v pairs for a
1128-
// row encoded as a primary index. This function mirrors the encoding
1129-
// logic in prepareInsertOrUpdateBatch in pkg/sql/row/writer.go.
1130-
// It is somewhat duplicated here due to the different arguments
1131-
// that prepareOrInsertUpdateBatch needs and uses to generate
1132-
// the k/v's for the row it inserts. includeEmpty controls
1133-
// whether or not k/v's with empty values should be returned.
1134-
// It returns indexEntries in family sorted order.
1127+
// EncodePrimaryIndex constructs the key prefix for the primary index and
1128+
// delegates the rest of the encoding to EncodePrimaryIndexWithKeyPrefix.
11351129
func EncodePrimaryIndex(
11361130
codec keys.SQLCodec,
11371131
tableDesc catalog.TableDescriptor,
@@ -1141,6 +1135,25 @@ func EncodePrimaryIndex(
11411135
includeEmpty bool,
11421136
) ([]IndexEntry, error) {
11431137
keyPrefix := MakeIndexKeyPrefix(codec, tableDesc.GetID(), index.GetID())
1138+
return EncodePrimaryIndexWithKeyPrefix(tableDesc, index, keyPrefix, colMap, values, includeEmpty)
1139+
}
1140+
1141+
// EncodePrimaryIndexWithKeyPrefix constructs a list of k/v pairs for a
1142+
// row encoded as a primary index, using the provided key prefix specific to
1143+
// that index. This function mirrors the encoding logic in
1144+
// prepareInsertOrUpdateBatch in pkg/sql/row/writer.go. It is somewhat
1145+
// duplicated here due to the different arguments that
1146+
// prepareOrInsertUpdateBatch needs and uses to generate the k/v's for the row
1147+
// it inserts. includeEmpty controls whether or not k/v's with empty values
1148+
// should be returned. It returns indexEntries in family sorted order.
1149+
func EncodePrimaryIndexWithKeyPrefix(
1150+
tableDesc catalog.TableDescriptor,
1151+
index catalog.Index,
1152+
keyPrefix []byte,
1153+
colMap catalog.TableColMap,
1154+
values []tree.Datum,
1155+
includeEmpty bool,
1156+
) ([]IndexEntry, error) {
11441157
indexKey, containsNull, err := EncodeIndexKey(tableDesc, index, colMap, values, keyPrefix)
11451158
if err != nil {
11461159
return nil, err
@@ -1283,9 +1296,8 @@ func MakeNullPKError(
12831296
return errors.AssertionFailedf("NULL value in unknown key column")
12841297
}
12851298

1286-
// EncodeSecondaryIndexKey encodes the key for a secondary index. The 'colMap'
1287-
// maps descpb.ColumnIDs to positions in 'values'. This function returns a slice
1288-
// of byte arrays representing the key values.
1299+
// EncodeSecondaryIndexKey constructs the key prefix for the secondary index and
1300+
// delegates the rest of the encoding to EncodeSecondaryIndexWithKeyPrefix.
12891301
func EncodeSecondaryIndexKey(
12901302
ctx context.Context,
12911303
codec keys.SQLCodec,
@@ -1294,29 +1306,40 @@ func EncodeSecondaryIndexKey(
12941306
colMap catalog.TableColMap,
12951307
values []tree.Datum,
12961308
) ([][]byte, bool, error) {
1297-
secondaryIndexKeyPrefix := MakeIndexKeyPrefix(codec, tableDesc.GetID(), secondaryIndex.GetID())
1309+
keyPrefix := MakeIndexKeyPrefix(codec, tableDesc.GetID(), secondaryIndex.GetID())
1310+
return EncodeSecondaryIndexKeyWithKeyPrefix(ctx, tableDesc, secondaryIndex, keyPrefix, colMap,
1311+
values)
1312+
}
12981313

1314+
// EncodeSecondaryIndexKeyWithKeyPrefix generates a slice of byte arrays
1315+
// representing encoded key values for the given secondary index, using the
1316+
// provided key prefix specific to that index. The colMap maps descpb.ColumnIDs
1317+
// to positions in the values slice.
1318+
func EncodeSecondaryIndexKeyWithKeyPrefix(
1319+
ctx context.Context,
1320+
tableDesc catalog.TableDescriptor,
1321+
secondaryIndex catalog.Index,
1322+
keyPrefix []byte,
1323+
colMap catalog.TableColMap,
1324+
values []tree.Datum,
1325+
) ([][]byte, bool, error) {
12991326
var containsNull = false
13001327
var secondaryKeys [][]byte
13011328
var err error
13021329
if secondaryIndex.GetType() == descpb.IndexDescriptor_INVERTED {
1303-
secondaryKeys, err = EncodeInvertedIndexKeys(ctx, secondaryIndex, colMap, values, secondaryIndexKeyPrefix)
1330+
secondaryKeys, err = EncodeInvertedIndexKeys(ctx, secondaryIndex, colMap, values, keyPrefix)
13041331
} else {
13051332
var secondaryIndexKey []byte
13061333
secondaryIndexKey, containsNull, err = EncodeIndexKey(
1307-
tableDesc, secondaryIndex, colMap, values, secondaryIndexKeyPrefix)
1334+
tableDesc, secondaryIndex, colMap, values, keyPrefix)
13081335

13091336
secondaryKeys = [][]byte{secondaryIndexKey}
13101337
}
13111338
return secondaryKeys, containsNull, err
13121339
}
13131340

1314-
// EncodeSecondaryIndex encodes key/values for a secondary
1315-
// index. colMap maps descpb.ColumnIDs to indices in `values`. This returns a
1316-
// slice of IndexEntry. includeEmpty controls whether or not
1317-
// EncodeSecondaryIndex should return k/v's that contain
1318-
// empty values. For forward indexes the returned list of
1319-
// index entries is in family sorted order.
1341+
// EncodeSecondaryIndex constructs the key prefix for the secondary index and
1342+
// delegates the rest of the encoding to EncodeSecondaryIndexWithKeyPrefix.
13201343
func EncodeSecondaryIndex(
13211344
ctx context.Context,
13221345
codec keys.SQLCodec,
@@ -1325,13 +1348,36 @@ func EncodeSecondaryIndex(
13251348
colMap catalog.TableColMap,
13261349
values []tree.Datum,
13271350
includeEmpty bool,
1351+
) ([]IndexEntry, error) {
1352+
keyPrefix := MakeIndexKeyPrefix(codec, tableDesc.GetID(), secondaryIndex.GetID())
1353+
return EncodeSecondaryIndexWithKeyPrefix(ctx, tableDesc, secondaryIndex, keyPrefix, colMap,
1354+
values, includeEmpty)
1355+
}
1356+
1357+
// EncodeSecondaryIndexWithKeyPrefix generates a slice of IndexEntry objects
1358+
// representing encoded key/value pairs for the given secondary index, using the
1359+
// provided key prefix specific to that index. This encoding is performed in
1360+
// EncodeSecondaryIndexKeyWithKeyPrefix for secondary indexes. The colMap maps
1361+
// descpb.ColumnIDs to positions in the values slice. The 'includeEmpty'
1362+
// parameter determines whether entries with empty values should be included.
1363+
// For forward indexes, the resulting entries are sorted by column family order.
1364+
func EncodeSecondaryIndexWithKeyPrefix(
1365+
ctx context.Context,
1366+
tableDesc catalog.TableDescriptor,
1367+
secondaryIndex catalog.Index,
1368+
keyPrefix []byte,
1369+
colMap catalog.TableColMap,
1370+
values []tree.Datum,
1371+
includeEmpty bool,
13281372
) ([]IndexEntry, error) {
13291373
// Use the primary key encoding for covering indexes.
13301374
if secondaryIndex.GetEncodingType() == catenumpb.PrimaryIndexEncoding {
1331-
return EncodePrimaryIndex(codec, tableDesc, secondaryIndex, colMap, values, includeEmpty)
1375+
return EncodePrimaryIndexWithKeyPrefix(tableDesc, secondaryIndex, keyPrefix, colMap, values,
1376+
includeEmpty)
13321377
}
13331378

1334-
secondaryKeys, containsNull, err := EncodeSecondaryIndexKey(ctx, codec, tableDesc, secondaryIndex, colMap, values)
1379+
secondaryKeys, containsNull, err := EncodeSecondaryIndexKeyWithKeyPrefix(ctx, tableDesc,
1380+
secondaryIndex, keyPrefix, colMap, values)
13351381
if err != nil {
13361382
return []IndexEntry{}, err
13371383
}
@@ -1592,14 +1638,18 @@ func writeColumnValues(
15921638
}
15931639

15941640
// EncodeSecondaryIndexes encodes key/values for the secondary indexes. colMap
1595-
// maps descpb.ColumnIDs to indices in `values`. secondaryIndexEntries is the return
1596-
// value (passed as a parameter so the caller can reuse between rows) and is
1597-
// expected to be the same length as indexes.
1641+
// maps descpb.ColumnIDs to indices in `values`. keyPrefixes is a slice that
1642+
// associates indexes to their key prefix; the caller can reuse this between
1643+
// rows to save work from creating key prefixes. the indexes and keyPrefixes
1644+
// slice should have the same ordering. secondaryIndexEntries is the return
1645+
// value (passed as a parameter so the caller can reuse between rows) and
1646+
// is expected to be the same length as indexes.
15981647
func EncodeSecondaryIndexes(
15991648
ctx context.Context,
16001649
codec keys.SQLCodec,
16011650
tableDesc catalog.TableDescriptor,
16021651
indexes []catalog.Index,
1652+
keyPrefixes [][]byte,
16031653
colMap catalog.TableColMap,
16041654
values []tree.Datum,
16051655
secondaryIndexEntries []IndexEntry,
@@ -1612,8 +1662,16 @@ func EncodeSecondaryIndexes(
16121662
}
16131663
const sizeOfIndexEntry = int64(unsafe.Sizeof(IndexEntry{}))
16141664

1615-
for i := range indexes {
1616-
entries, err := EncodeSecondaryIndex(ctx, codec, tableDesc, indexes[i], colMap, values, includeEmpty)
1665+
for i, idx := range indexes {
1666+
keyPrefix := keyPrefixes[i]
1667+
// TODO(annie): For now, we recompute the key prefix of inverted indexes. This is because index
1668+
// keys with multiple associated values somehow get encoded into the same kv pair when using
1669+
// our precomputed key prefix. `inverted_index/arrays` (logictest) illustrates this issue.
1670+
if idx.GetType() == descpb.IndexDescriptor_INVERTED {
1671+
keyPrefix = MakeIndexKeyPrefix(codec, tableDesc.GetID(), idx.GetID())
1672+
}
1673+
entries, err := EncodeSecondaryIndexWithKeyPrefix(ctx, tableDesc, idx, keyPrefix, colMap, values,
1674+
includeEmpty)
16171675
if err != nil {
16181676
return secondaryIndexEntries, 0, err
16191677
}

pkg/sql/rowexec/indexbackfiller_test.go

+6
Original file line numberDiff line numberDiff line change
@@ -7,6 +7,7 @@ package rowexec
77

88
import (
99
"context"
10+
"path/filepath"
1011
"testing"
1112
"time"
1213

@@ -18,6 +19,7 @@ import (
1819
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan"
1920
"github.com/cockroachdb/cockroach/pkg/sql/sem/eval"
2021
"github.com/cockroachdb/cockroach/pkg/sql/sqlerrors"
22+
"github.com/cockroachdb/cockroach/pkg/testutils"
2123
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
2224
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
2325
"github.com/cockroachdb/cockroach/pkg/util/log"
@@ -92,6 +94,9 @@ func BenchmarkIndexBackfill(b *testing.B) {
9294
stopTimer := func() {}
9395
startTimer := func() {}
9496

97+
dir, dirCleanupFn := testutils.TempDir(b)
98+
defer dirCleanupFn()
99+
95100
srv, sqlDB, _ := serverutils.StartServer(b, base.TestServerArgs{
96101
Knobs: base.TestingKnobs{
97102
SQLEvalContext: &eval.TestingKnobs{
@@ -120,6 +125,7 @@ func BenchmarkIndexBackfill(b *testing.B) {
120125
},
121126
},
122127
},
128+
StoreSpecs: []base.StoreSpec{{InMemory: false, Path: filepath.Join(dir, "testserver")}},
123129
})
124130
defer srv.Stopper().Stop(ctx)
125131

0 commit comments

Comments
 (0)