Skip to content

Commit d164968

Browse files
craig[bot]mgartner
andcommitted
Merge #146655
146655: opt,sql: assorted optimizations r=mgartner a=mgartner #### opt: add benchmarks for sysbench UPDATE statements Release note: None #### execbuilder: add global empty buildScalarCtx Release note: None #### sql: refactor MakeUpdater Some conditionals in the `needsUpdate` have been moved out of the closure. Also, `needsUpdate` no longer allocates a slice of column IDs. Instead, it iterates directly over a set of column IDs. Release note: None #### opt: convert needsUpdate closure to function Release note: None #### sql: lazily allocate includeIndexes Release note: None #### sql: combine index entry allocations Release note: None #### sql: combine allocations of fetch and update col lists Release note: None #### sql: combine index slice allocations for updates and upserts Release note: None #### 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 Co-authored-by: Marcus Gartner <[email protected]>
2 parents 553aa73 + 24272e5 commit d164968

File tree

10 files changed

+219
-131
lines changed

10 files changed

+219
-131
lines changed

pkg/sql/opt/bench/bench_test.go

Lines changed: 21 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -122,6 +122,15 @@ type benchQuery struct {
122122
var schemas = []string{
123123
`CREATE TABLE kv (k BIGINT NOT NULL PRIMARY KEY, v BYTES NOT NULL)`,
124124
`
125+
CREATE TABLE sbtest (
126+
id INT8 PRIMARY KEY,
127+
k INT8 NOT NULL DEFAULT 0,
128+
c CHAR(120) NOT NULL DEFAULT '',
129+
pad CHAR(60) NOT NULL DEFAULT '',
130+
INDEX (k)
131+
)
132+
`,
133+
`
125134
CREATE TABLE customer
126135
(
127136
c_id integer not null,
@@ -389,6 +398,18 @@ var queries = [...]benchQuery{
389398
args: []interface{}{},
390399
},
391400

401+
{
402+
name: "sysbench-update-index",
403+
query: `UPDATE sbtest SET k=k+1 WHERE id=$1`,
404+
args: []interface{}{10},
405+
},
406+
407+
{
408+
name: "sysbench-update-non-index",
409+
query: `UPDATE sbtest SET c=$2 WHERE id=$1`,
410+
args: []interface{}{10, "foo"},
411+
},
412+
392413
// 1. Table with many columns.
393414
// 2. Multi-column primary key.
394415
// 3. Mutiple indexes to consider.

pkg/sql/opt/exec/execbuilder/relational.go

Lines changed: 3 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -541,15 +541,14 @@ func (b *Builder) buildValuesRows(values *memo.ValuesExpr) ([][]tree.TypedExpr,
541541
numCols := len(values.Cols)
542542

543543
rows := makeTypedExprMatrix(len(values.Rows), numCols)
544-
scalarCtx := buildScalarCtx{}
545544
for i := range rows {
546545
tup := values.Rows[i].(*memo.TupleExpr)
547546
if len(tup.Elems) != numCols {
548547
return nil, fmt.Errorf("inconsistent row length %d vs %d", len(tup.Elems), numCols)
549548
}
550549
var err error
551550
for j := 0; j < numCols; j++ {
552-
rows[i][j], err = b.buildScalar(&scalarCtx, tup.Elems[j])
551+
rows[i][j], err = b.buildScalar(&emptyBuildScalarCtx, tup.Elems[j])
553552
if err != nil {
554553
return nil, err
555554
}
@@ -3565,11 +3564,10 @@ func (b *Builder) buildCall(c *memo.CallExpr) (_ execPlan, outputCols colOrdMap,
35653564

35663565
// Build the argument expressions.
35673566
var args tree.TypedExprs
3568-
ctx := buildScalarCtx{}
35693567
if len(udf.Args) > 0 {
35703568
args = make(tree.TypedExprs, len(udf.Args))
35713569
for i := range udf.Args {
3572-
args[i], err = b.buildScalar(&ctx, udf.Args[i])
3570+
args[i], err = b.buildScalar(&emptyBuildScalarCtx, udf.Args[i])
35733571
if err != nil {
35743572
return execPlan{}, colOrdMap{}, err
35753573
}
@@ -3993,8 +3991,7 @@ func (b *Builder) buildVectorSearch(
39933991
}
39943992
}
39953993
outColOrds, outColMap := b.getColumns(search.Cols, search.Table)
3996-
ctx := buildScalarCtx{}
3997-
queryVector, err := b.buildScalar(&ctx, search.QueryVector)
3994+
queryVector, err := b.buildScalar(&emptyBuildScalarCtx, search.QueryVector)
39983995
if err != nil {
39993996
return execPlan{}, colOrdMap{}, err
40003997
}

pkg/sql/opt/exec/execbuilder/scalar.go

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -40,6 +40,8 @@ type buildScalarCtx struct {
4040
ivarMap colOrdMap
4141
}
4242

43+
var emptyBuildScalarCtx buildScalarCtx
44+
4345
type buildFunc func(b *Builder, ctx *buildScalarCtx, scalar opt.ScalarExpr) (tree.TypedExpr, error)
4446

4547
var scalarBuildFuncMap [opt.NumOperators]buildFunc

pkg/sql/opt/exec/execbuilder/statement.go

Lines changed: 6 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -219,8 +219,7 @@ func (b *Builder) buildAlterTableSplit(
219219
if err != nil {
220220
return execPlan{}, colOrdMap{}, err
221221
}
222-
scalarCtx := buildScalarCtx{}
223-
expiration, err := b.buildScalar(&scalarCtx, split.Expiration)
222+
expiration, err := b.buildScalar(&emptyBuildScalarCtx, split.Expiration)
224223
if err != nil {
225224
return execPlan{}, colOrdMap{}, err
226225
}
@@ -295,12 +294,11 @@ func (b *Builder) buildAlterRangeRelocate(
295294
if err != nil {
296295
return execPlan{}, colOrdMap{}, err
297296
}
298-
scalarCtx := buildScalarCtx{}
299-
toStoreID, err := b.buildScalar(&scalarCtx, relocate.ToStoreID)
297+
toStoreID, err := b.buildScalar(&emptyBuildScalarCtx, relocate.ToStoreID)
300298
if err != nil {
301299
return execPlan{}, colOrdMap{}, err
302300
}
303-
fromStoreID, err := b.buildScalar(&scalarCtx, relocate.FromStoreID)
301+
fromStoreID, err := b.buildScalar(&emptyBuildScalarCtx, relocate.FromStoreID)
304302
if err != nil {
305303
return execPlan{}, colOrdMap{}, err
306304
}
@@ -325,8 +323,7 @@ func (b *Builder) buildControlJobs(
325323
return execPlan{}, colOrdMap{}, err
326324
}
327325

328-
scalarCtx := buildScalarCtx{}
329-
reason, err := b.buildScalar(&scalarCtx, ctl.Reason)
326+
reason, err := b.buildScalar(&emptyBuildScalarCtx, ctl.Reason)
330327
if err != nil {
331328
return execPlan{}, colOrdMap{}, err
332329
}
@@ -432,8 +429,7 @@ func (b *Builder) buildExport(
432429
return execPlan{}, colOrdMap{}, err
433430
}
434431

435-
scalarCtx := buildScalarCtx{}
436-
fileName, err := b.buildScalar(&scalarCtx, export.FileName)
432+
fileName, err := b.buildScalar(&emptyBuildScalarCtx, export.FileName)
437433
if err != nil {
438434
return execPlan{}, colOrdMap{}, err
439435
}
@@ -442,7 +438,7 @@ func (b *Builder) buildExport(
442438
for i, o := range export.Options {
443439
opts[i].Key = o.Key
444440
var err error
445-
opts[i].Value, err = b.buildScalar(&scalarCtx, o.Value)
441+
opts[i].Value, err = b.buildScalar(&emptyBuildScalarCtx, o.Value)
446442
if err != nil {
447443
return execPlan{}, colOrdMap{}, err
448444
}

pkg/sql/opt_exec_factory.go

Lines changed: 49 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -1401,6 +1401,25 @@ func ordinalsToIndexes(table cat.Table, ords cat.IndexOrdinals) []catalog.Index
14011401
return retval
14021402
}
14031403

1404+
func ordinalsToIndexes2(
1405+
table cat.Table, a, b cat.IndexOrdinals,
1406+
) ([]catalog.Index, []catalog.Index) {
1407+
lenA, lenB := len(a), len(b)
1408+
if lenA+lenB == 0 {
1409+
return nil, nil
1410+
}
1411+
1412+
indexes := make([]catalog.Index, lenA+lenB)
1413+
indexesA, indexesB := indexes[:lenA:lenA], indexes[lenA:]
1414+
for i, idx := range a {
1415+
indexesA[i] = table.Index(idx).(*optIndex).idx
1416+
}
1417+
for i, idx := range b {
1418+
indexesB[i] = table.Index(idx).(*optIndex).idx
1419+
}
1420+
return indexesA, indexesB
1421+
}
1422+
14041423
func (ef *execFactory) ConstructInsert(
14051424
input exec.Node,
14061425
table cat.Table,
@@ -1596,6 +1615,8 @@ func (ef *execFactory) ConstructUpdate(
15961615
}
15971616

15981617
// If rows are not needed, no columns are returned.
1618+
// TODO(mgartner): Combine returnCols allocations with allocations for
1619+
// fetchCols and updateCols in constructUpdateRun.
15991620
var returnCols []catalog.Column
16001621
if rowsNeeded {
16011622
returnCols = makeColList(table, returnColOrdSet)
@@ -1719,15 +1740,15 @@ func (ef *execFactory) constructUpdateRun(
17191740
lockedIndexes cat.IndexOrdinals,
17201741
) error {
17211742
tabDesc := table.(*optTable).desc
1722-
fetchCols := makeColList(table, fetchColOrdSet)
1723-
updateCols := makeColList(table, updateColOrdSet)
1743+
fetchCols, updateCols := makeColList2(table, fetchColOrdSet, updateColOrdSet)
17241744

17251745
// Create the table updater.
1746+
tombstoneIdxs, lockIdxs := ordinalsToIndexes2(table, uniqueWithTombstoneIndexes, lockedIndexes)
17261747
ru, err := row.MakeUpdater(
17271748
ef.planner.ExecCfg().Codec,
17281749
tabDesc,
1729-
ordinalsToIndexes(table, uniqueWithTombstoneIndexes),
1730-
ordinalsToIndexes(table, lockedIndexes),
1750+
tombstoneIdxs,
1751+
lockIdxs,
17311752
updateCols,
17321753
fetchCols,
17331754
row.UpdaterDefault,
@@ -1798,11 +1819,12 @@ func (ef *execFactory) ConstructUpsert(
17981819
}
17991820

18001821
// Create the table updater, which does the bulk of the update-related work.
1822+
tombstoneIdxs, lockIdxs := ordinalsToIndexes2(table, uniqueWithTombstoneIndexes, lockedIndexes)
18011823
ru, err := row.MakeUpdater(
18021824
ef.planner.ExecCfg().Codec,
18031825
tabDesc,
1804-
ordinalsToIndexes(table, uniqueWithTombstoneIndexes),
1805-
ordinalsToIndexes(table, lockedIndexes),
1826+
tombstoneIdxs,
1827+
lockIdxs,
18061828
updateCols,
18071829
fetchCols,
18081830
row.UpdaterDefault,
@@ -2569,6 +2591,27 @@ func makeColList(table cat.Table, cols exec.TableColumnOrdinalSet) []catalog.Col
25692591
return ret
25702592
}
25712593

2594+
// makeColList2 is similar to makeColList, but it takes two sets of ordinals and
2595+
// allocates a single slice which is split into two.
2596+
func makeColList2(
2597+
table cat.Table, a, b exec.TableColumnOrdinalSet,
2598+
) ([]catalog.Column, []catalog.Column) {
2599+
tab := table.(optCatalogTableInterface)
2600+
lenA, lenB := a.Len(), b.Len()
2601+
cols := make([]catalog.Column, 0, lenA+lenB)
2602+
listA, listB := cols[:0:lenA], cols[lenA:lenA]
2603+
for i, n := 0, table.ColumnCount(); i < n; i++ {
2604+
col := tab.getCol(i)
2605+
if a.Contains(i) {
2606+
listA = append(listA, col)
2607+
}
2608+
if b.Contains(i) {
2609+
listB = append(listB, col)
2610+
}
2611+
}
2612+
return listA, listB
2613+
}
2614+
25722615
// makePublicToReturnColumnIndexMapping returns a map from the ordinals
25732616
// of the table's public columns to ordinals in the returnColDescs slice.
25742617
//

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
}

0 commit comments

Comments
 (0)