Skip to content

Commit 24272e5

Browse files
committed
sql/row: lazily compute index column direction slices
The encoding directions of the primary and secondary indexes are only used in mutations when pretty-printing keys when tracing is enabled and when `CheckRowSize` emits a log event. These encoding directions are now lazily computed to eliminate their overhead in the common case. Release note: None
1 parent 4df8533 commit 24272e5

File tree

5 files changed

+86
-58
lines changed

5 files changed

+86
-58
lines changed

pkg/sql/row/deleter.go

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -179,10 +179,10 @@ func (rd *Deleter) DeleteRow(
179179
if oth.IsSet() {
180180
oth.DelWithCPut(ctx, b, &rd.key, expValue, traceKV)
181181
} else {
182-
delWithCPutFn(ctx, b, &rd.key, expValue, traceKV, rd.Helper.primIndexValDirs)
182+
delWithCPutFn(ctx, b, &rd.key, expValue, traceKV, &rd.Helper, primaryIndexDirs)
183183
}
184184
} else {
185-
delFn(ctx, b, &rd.key, !rd.primaryLocked /* needsLock */, traceKV, rd.Helper.primIndexValDirs)
185+
delFn(ctx, b, &rd.key, !rd.primaryLocked /* needsLock */, traceKV, &rd.Helper, primaryIndexDirs)
186186
}
187187

188188
rd.key = nil
@@ -218,7 +218,7 @@ func (rd *Deleter) DeleteRow(
218218
for _, e := range entries {
219219
if err = rd.Helper.deleteIndexEntry(
220220
ctx, b, index, &e.Key, alreadyLocked, rd.Helper.sd.BufferedWritesUseLockingOnNonUniqueIndexes,
221-
traceKV, rd.Helper.secIndexValDirs[i],
221+
traceKV, secondaryIndexDirs(i),
222222
); err != nil {
223223
return err
224224
}

pkg/sql/row/helper.go

Lines changed: 48 additions & 22 deletions
Original file line numberDiff line numberDiff line change
@@ -91,9 +91,11 @@ type RowHelper struct {
9191
UniqueWithTombstoneIndexes intsets.Fast
9292
indexEntries map[catalog.Index][]rowenc.IndexEntry
9393

94-
// Computed during initialization for pretty-printing.
95-
primIndexValDirs []encoding.Direction
96-
secIndexValDirs [][]encoding.Direction
94+
// Lazily computed for pretty-printing and CheckRowSize.
95+
dirs struct {
96+
primary []encoding.Direction
97+
secondary [][]encoding.Direction
98+
}
9799

98100
// Computed and cached.
99101
PrimaryIndexKeyPrefix []byte
@@ -126,28 +128,50 @@ func NewRowHelper(
126128
for _, index := range uniqueWithTombstoneIndexes {
127129
uniqueWithTombstoneIndexesSet.Add(index.Ordinal())
128130
}
129-
rh := RowHelper{
131+
return RowHelper{
130132
Codec: codec,
131133
TableDesc: desc,
132134
Indexes: indexes,
133135
UniqueWithTombstoneIndexes: uniqueWithTombstoneIndexesSet,
134136
sd: sd,
135137
metrics: metrics,
138+
maxRowSizeLog: uint32(maxRowSizeLog.Get(sv)),
139+
maxRowSizeErr: uint32(maxRowSizeErr.Get(sv)),
136140
}
141+
}
137142

138-
// Pre-compute the encoding directions of the index key values for
139-
// pretty-printing in traces.
140-
rh.primIndexValDirs = catalogkeys.IndexKeyValDirs(rh.TableDesc.GetPrimaryIndex())
143+
// lazyIndexDirs represents encoding directions of an index. Those directions
144+
// may not have been, and may never be computed. The value of -2 represents
145+
// empty encoding directions. The value of -1 represents the encoding directions
146+
// of the primary index, otherwise a value i represents the encoding directions
147+
// of the i-th secondary index.
148+
type lazyIndexDirs int
141149

142-
rh.secIndexValDirs = make([][]encoding.Direction, len(rh.Indexes))
143-
for i := range rh.Indexes {
144-
rh.secIndexValDirs[i] = catalogkeys.IndexKeyValDirs(rh.Indexes[i])
145-
}
150+
const (
151+
emptyIndexDirs lazyIndexDirs = -2
152+
primaryIndexDirs lazyIndexDirs = -1
153+
)
146154

147-
rh.maxRowSizeLog = uint32(maxRowSizeLog.Get(sv))
148-
rh.maxRowSizeErr = uint32(maxRowSizeErr.Get(sv))
155+
func secondaryIndexDirs(i int) lazyIndexDirs { return lazyIndexDirs(i) }
149156

150-
return rh
157+
func (d lazyIndexDirs) compute(rh *RowHelper) []encoding.Direction {
158+
switch d {
159+
case emptyIndexDirs:
160+
return nil
161+
case primaryIndexDirs:
162+
if rh.dirs.primary == nil {
163+
rh.dirs.primary = catalogkeys.IndexKeyValDirs(rh.TableDesc.GetPrimaryIndex())
164+
}
165+
return rh.dirs.primary
166+
default:
167+
if rh.dirs.secondary == nil {
168+
rh.dirs.secondary = make([][]encoding.Direction, len(rh.Indexes))
169+
for i := range rh.Indexes {
170+
rh.dirs.secondary[i] = catalogkeys.IndexKeyValDirs(rh.Indexes[i])
171+
}
172+
}
173+
return rh.dirs.secondary[d]
174+
}
151175
}
152176

153177
// encodeIndexes encodes the primary and secondary index keys. The
@@ -432,7 +456,7 @@ func (rh *RowHelper) CheckRowSize(
432456
RowSize: size,
433457
TableID: uint32(rh.TableDesc.GetID()),
434458
FamilyID: uint32(family),
435-
PrimaryKey: keys.PrettyPrint(rh.primIndexValDirs, *key),
459+
PrimaryKey: keys.PrettyPrint(primaryIndexDirs.compute(rh), *key),
436460
}
437461
if rh.sd.Internal && shouldErr {
438462
// Internal work should never err and always log if violating either limit.
@@ -471,11 +495,12 @@ func delFn(
471495
key *roachpb.Key,
472496
needsLock bool,
473497
traceKV bool,
474-
keyEncodingDirs []encoding.Direction,
498+
rh *RowHelper,
499+
dirs lazyIndexDirs,
475500
) {
476501
if needsLock {
477502
if traceKV {
478-
if keyEncodingDirs != nil {
503+
if keyEncodingDirs := dirs.compute(rh); keyEncodingDirs != nil {
479504
log.VEventf(ctx, 2, "Del (locking) %s", keys.PrettyPrint(keyEncodingDirs, *key))
480505
} else {
481506
log.VEventf(ctx, 2, "Del (locking) %s", *key)
@@ -484,7 +509,7 @@ func delFn(
484509
b.DelMustAcquireExclusiveLock(key)
485510
} else {
486511
if traceKV {
487-
if keyEncodingDirs != nil {
512+
if keyEncodingDirs := dirs.compute(rh); keyEncodingDirs != nil {
488513
log.VEventf(ctx, 2, "Del %s", keys.PrettyPrint(keyEncodingDirs, *key))
489514
} else {
490515
log.VEventf(ctx, 2, "Del %s", *key)
@@ -500,10 +525,11 @@ func delWithCPutFn(
500525
key *roachpb.Key,
501526
expVal []byte,
502527
traceKV bool,
503-
keyEncodingDirs []encoding.Direction,
528+
rh *RowHelper,
529+
dirs lazyIndexDirs,
504530
) {
505531
if traceKV {
506-
if keyEncodingDirs != nil {
532+
if keyEncodingDirs := dirs.compute(rh); keyEncodingDirs != nil {
507533
log.VEventf(ctx, 2, "CPut %s -> nil (delete)", keys.PrettyPrint(keyEncodingDirs, *key))
508534
} else {
509535
log.VEventf(ctx, 2, "CPut %s -> nil (delete)", *key)
@@ -520,7 +546,7 @@ func (rh *RowHelper) deleteIndexEntry(
520546
alreadyLocked bool,
521547
lockNonUnique bool,
522548
traceKV bool,
523-
valDirs []encoding.Direction,
549+
dirs lazyIndexDirs,
524550
) error {
525551
needsLock := !alreadyLocked && (index.IsUnique() || lockNonUnique)
526552
if index.UseDeletePreservingEncoding() {
@@ -537,7 +563,7 @@ func (rh *RowHelper) deleteIndexEntry(
537563
b.Put(key, deleteEncoding)
538564
}
539565
} else {
540-
delFn(ctx, b, key, needsLock, traceKV, valDirs)
566+
delFn(ctx, b, key, needsLock, traceKV, rh, dirs)
541567
}
542568
return nil
543569
}

pkg/sql/row/inserter.go

Lines changed: 11 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -16,7 +16,6 @@ import (
1616
"github.com/cockroachdb/cockroach/pkg/sql/rowinfra"
1717
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
1818
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
19-
"github.com/cockroachdb/cockroach/pkg/util/encoding"
2019
"github.com/cockroachdb/cockroach/pkg/util/log"
2120
"github.com/cockroachdb/errors"
2221
)
@@ -80,10 +79,11 @@ func insertCPutFn(
8079
key *roachpb.Key,
8180
value *roachpb.Value,
8281
traceKV bool,
83-
keyEncodingDirs []encoding.Direction,
82+
rh *RowHelper,
83+
dirs lazyIndexDirs,
8484
) {
8585
if traceKV {
86-
log.VEventfDepth(ctx, 1, 2, "CPut %s -> %s", keys.PrettyPrint(keyEncodingDirs, *key), value.PrettyPrint())
86+
log.VEventfDepth(ctx, 1, 2, "CPut %s -> %s", keys.PrettyPrint(dirs.compute(rh), *key), value.PrettyPrint())
8787
}
8888
b.CPut(key, value, nil /* expValue */)
8989
}
@@ -95,10 +95,11 @@ func insertPutFn(
9595
key *roachpb.Key,
9696
value *roachpb.Value,
9797
traceKV bool,
98-
keyEncodingDirs []encoding.Direction,
98+
rh *RowHelper,
99+
dirs lazyIndexDirs,
99100
) {
100101
if traceKV {
101-
log.VEventfDepth(ctx, 1, 2, "Put %s -> %s", keys.PrettyPrint(keyEncodingDirs, *key), value.PrettyPrint())
102+
log.VEventfDepth(ctx, 1, 2, "Put %s -> %s", keys.PrettyPrint(dirs.compute(rh), *key), value.PrettyPrint())
102103
}
103104
b.Put(key, value)
104105
}
@@ -112,10 +113,11 @@ func insertPutMustAcquireExclusiveLockFn(
112113
key *roachpb.Key,
113114
value *roachpb.Value,
114115
traceKV bool,
115-
keyEncodingDirs []encoding.Direction,
116+
rh *RowHelper,
117+
dirs lazyIndexDirs,
116118
) {
117119
if traceKV {
118-
log.VEventfDepth(ctx, 1, 2, "Put (locking) %s -> %s", keys.PrettyPrint(keyEncodingDirs, *key), value.PrettyPrint())
120+
log.VEventfDepth(ctx, 1, 2, "Put (locking) %s -> %s", keys.PrettyPrint(dirs.compute(rh), *key), value.PrettyPrint())
119121
}
120122
b.PutMustAcquireExclusiveLock(key, value)
121123
}
@@ -214,7 +216,7 @@ func (ri *Inserter) InsertRow(
214216
for idx, index := range ri.Helper.Indexes {
215217
entries, ok := secondaryIndexEntries[index]
216218
if ok {
217-
var putFn func(context.Context, Putter, *roachpb.Key, *roachpb.Value, bool, []encoding.Direction)
219+
var putFn func(context.Context, Putter, *roachpb.Key, *roachpb.Value, bool, *RowHelper, lazyIndexDirs)
218220
if index.ForcePut() {
219221
// See the comment on (catalog.Index).ForcePut() for more
220222
// details.
@@ -245,7 +247,7 @@ func (ri *Inserter) InsertRow(
245247
}
246248
for i := range entries {
247249
e := &entries[i]
248-
putFn(ctx, b, &e.Key, &e.Value, traceKV, ri.Helper.secIndexValDirs[idx])
250+
putFn(ctx, b, &e.Key, &e.Value, traceKV, &ri.Helper, secondaryIndexDirs(idx))
249251
}
250252

251253
// If a row does not satisfy a partial index predicate, it will have no

pkg/sql/row/updater.go

Lines changed: 15 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -23,7 +23,6 @@ import (
2323
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
2424
"github.com/cockroachdb/cockroach/pkg/util/buildutil"
2525
"github.com/cockroachdb/cockroach/pkg/util/deduplicate"
26-
"github.com/cockroachdb/cockroach/pkg/util/encoding"
2726
"github.com/cockroachdb/cockroach/pkg/util/intsets"
2827
"github.com/cockroachdb/cockroach/pkg/util/log"
2928
"github.com/cockroachdb/errors"
@@ -438,7 +437,7 @@ func (ru *Updater) UpdateRow(
438437
// putFn and sameKeyPutFn are the functions that should be invoked in
439438
// order to write the new k/v entry. If the key doesn't change,
440439
// sameKeyPutFn will be used, otherwise putFn will be used.
441-
var putFn, sameKeyPutFn func(context.Context, Putter, *roachpb.Key, *roachpb.Value, bool, []encoding.Direction)
440+
var putFn, sameKeyPutFn func(context.Context, Putter, *roachpb.Key, *roachpb.Value, bool, *RowHelper, lazyIndexDirs)
442441
if index.ForcePut() {
443442
// See the comment on (catalog.Index).ForcePut() for more details.
444443
// TODO(#140695): re-evaluate the lock need when we enable buffered
@@ -508,7 +507,7 @@ func (ru *Updater) UpdateRow(
508507
if err = ru.Helper.deleteIndexEntry(
509508
ctx, b, index, &oldEntry.Key, alreadyLocked,
510509
ru.Helper.sd.BufferedWritesUseLockingOnNonUniqueIndexes,
511-
traceKV, ru.Helper.secIndexValDirs[i],
510+
traceKV, secondaryIndexDirs(i),
512511
); err != nil {
513512
return nil, err
514513
}
@@ -528,9 +527,9 @@ func (ru *Updater) UpdateRow(
528527
}
529528

530529
if sameKey {
531-
sameKeyPutFn(ctx, b, &newEntry.Key, &newEntry.Value, traceKV, ru.Helper.secIndexValDirs[i])
530+
sameKeyPutFn(ctx, b, &newEntry.Key, &newEntry.Value, traceKV, &ru.Helper, secondaryIndexDirs(i))
532531
} else {
533-
putFn(ctx, b, &newEntry.Key, &newEntry.Value, traceKV, ru.Helper.secIndexValDirs[i])
532+
putFn(ctx, b, &newEntry.Key, &newEntry.Value, traceKV, &ru.Helper, secondaryIndexDirs(i))
534533
}
535534
writtenIndexes.Add(i)
536535
} else if oldEntry.Family < newEntry.Family {
@@ -545,7 +544,7 @@ func (ru *Updater) UpdateRow(
545544
if err = ru.Helper.deleteIndexEntry(
546545
ctx, b, index, &oldEntry.Key, alreadyLocked,
547546
ru.Helper.sd.BufferedWritesUseLockingOnNonUniqueIndexes,
548-
traceKV, ru.Helper.secIndexValDirs[i],
547+
traceKV, secondaryIndexDirs(i),
549548
); err != nil {
550549
return nil, err
551550
}
@@ -560,7 +559,7 @@ func (ru *Updater) UpdateRow(
560559

561560
// In this case, the index now has a k/v that did not exist
562561
// in the old row, so we put the new key in place.
563-
putFn(ctx, b, &newEntry.Key, &newEntry.Value, traceKV, ru.Helper.secIndexValDirs[i])
562+
putFn(ctx, b, &newEntry.Key, &newEntry.Value, traceKV, &ru.Helper, secondaryIndexDirs(i))
564563
writtenIndexes.Add(i)
565564
newIdx++
566565
}
@@ -574,7 +573,7 @@ func (ru *Updater) UpdateRow(
574573
if err = ru.Helper.deleteIndexEntry(
575574
ctx, b, index, &oldEntry.Key, alreadyLocked,
576575
ru.Helper.sd.BufferedWritesUseLockingOnNonUniqueIndexes,
577-
traceKV, ru.Helper.secIndexValDirs[i],
576+
traceKV, secondaryIndexDirs(i),
578577
); err != nil {
579578
return nil, err
580579
}
@@ -588,7 +587,7 @@ func (ru *Updater) UpdateRow(
588587
// and the old row values do not match the partial index
589588
// predicate.
590589
newEntry := &newEntries[newIdx]
591-
putFn(ctx, b, &newEntry.Key, &newEntry.Value, traceKV, ru.Helper.secIndexValDirs[i])
590+
putFn(ctx, b, &newEntry.Key, &newEntry.Value, traceKV, &ru.Helper, secondaryIndexDirs(i))
592591
writtenIndexes.Add(i)
593592
newIdx++
594593
}
@@ -597,14 +596,15 @@ func (ru *Updater) UpdateRow(
597596
for j := range ru.oldIndexEntries[i] {
598597
if err = ru.Helper.deleteIndexEntry(
599598
ctx, b, index, &ru.oldIndexEntries[i][j].Key, alreadyLocked,
600-
ru.Helper.sd.BufferedWritesUseLockingOnNonUniqueIndexes, traceKV, nil, /* valDirs */
599+
ru.Helper.sd.BufferedWritesUseLockingOnNonUniqueIndexes, traceKV, emptyIndexDirs,
601600
); err != nil {
602601
return nil, err
603602
}
604603
}
605604
// We're adding all of the inverted index entries from the row being updated.
606605
for j := range ru.newIndexEntries[i] {
607-
putFn(ctx, b, &ru.newIndexEntries[i][j].Key, &ru.newIndexEntries[i][j].Value, traceKV, ru.Helper.secIndexValDirs[i])
606+
putFn(ctx, b, &ru.newIndexEntries[i][j].Key, &ru.newIndexEntries[i][j].Value, traceKV,
607+
&ru.Helper, secondaryIndexDirs(i))
608608
}
609609
}
610610
}
@@ -631,7 +631,7 @@ func (ru *Updater) UpdateRow(
631631
for _, deletedSecondaryIndexEntry := range deletedSecondaryIndexEntries {
632632
if err = ru.DeleteHelper.deleteIndexEntry(
633633
ctx, b, index, &deletedSecondaryIndexEntry.Key, alreadyLocked,
634-
ru.Helper.sd.BufferedWritesUseLockingOnNonUniqueIndexes, traceKV, nil, /* valDirs */
634+
ru.Helper.sd.BufferedWritesUseLockingOnNonUniqueIndexes, traceKV, emptyIndexDirs,
635635
); err != nil {
636636
return nil, err
637637
}
@@ -671,11 +671,12 @@ func updateCPutFn(
671671
value *roachpb.Value,
672672
expVal []byte,
673673
traceKV bool,
674-
keyEncodingDirs []encoding.Direction,
674+
rh *RowHelper,
675+
dirs lazyIndexDirs,
675676
) {
676677
if traceKV {
677678
log.VEventfDepth(
678-
ctx, 1, 2, "CPut %s -> %s (swap)", keys.PrettyPrint(keyEncodingDirs, *key),
679+
ctx, 1, 2, "CPut %s -> %s (swap)", keys.PrettyPrint(dirs.compute(rh), *key),
679680
value.PrettyPrint(),
680681
)
681682
}

0 commit comments

Comments
 (0)