-
Notifications
You must be signed in to change notification settings - Fork 3.8k
/
Copy pathplanner.go
651 lines (565 loc) · 21 KB
/
planner.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
// Copyright 2016 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
package sql
import (
"context"
"fmt"
"time"
"github.com/cockroachdb/cockroach/pkg/internal/client"
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/server/serverpb"
"github.com/cockroachdb/cockroach/pkg/sql/opt/exec"
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgwireconn"
"github.com/cockroachdb/cockroach/pkg/sql/querycache"
"github.com/cockroachdb/cockroach/pkg/sql/row"
"github.com/cockroachdb/cockroach/pkg/sql/sem/transform"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util/envutil"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/mon"
"github.com/cockroachdb/logtags"
"github.com/pkg/errors"
)
// extendedEvalContext extends tree.EvalContext with fields that are needed for
// distsql planning.
type extendedEvalContext struct {
tree.EvalContext
SessionMutator *sessionDataMutator
// SessionID for this connection.
SessionID ClusterWideID
// VirtualSchemas can be used to access virtual tables.
VirtualSchemas VirtualTabler
// ClientBuffer is used for communication between possible pgwire clients.
ClientBuffer pgwireconn.ClientBuffer
// Tracing provides access to the session's tracing interface. Changes to the
// tracing state should be done through the sessionDataMutator.
Tracing *SessionTracing
// StatusServer gives access to the Status service. Used to cancel queries.
StatusServer serverpb.StatusServer
// MemMetrics represent the group of metrics to which execution should
// contribute.
MemMetrics *MemoryMetrics
// Tables points to the Session's table collection (& cache).
Tables *TableCollection
ExecCfg *ExecutorConfig
DistSQLPlanner *DistSQLPlanner
TxnModesSetter txnModesSetter
SchemaChangers *schemaChangerCollection
Jobs *jobsCollection
schemaAccessors *schemaInterface
sqlStatsCollector *sqlStatsCollector
}
// copy returns a deep copy of ctx.
func (ctx *extendedEvalContext) copy() *extendedEvalContext {
cpy := *ctx
cpy.EvalContext = *ctx.EvalContext.Copy()
return &cpy
}
// QueueJob creates a new job from record and queues it for execution after
// the transaction commits.
func (ctx *extendedEvalContext) QueueJob(record jobs.Record) (*jobs.Job, error) {
job, err := ctx.ExecCfg.JobRegistry.CreateJobWithTxn(
ctx.Context,
record,
ctx.Txn,
)
if err != nil {
return nil, err
}
*ctx.Jobs = append(*ctx.Jobs, *job.ID())
return job, nil
}
// schemaInterface provides access to the database and table descriptors.
// See schema_accessors.go.
type schemaInterface struct {
physical SchemaAccessor
logical SchemaAccessor
}
// planner is the centerpiece of SQL statement execution combining session
// state and database state with the logic for SQL execution. It is logically
// scoped to the execution of a single statement, and should not be used to
// execute multiple statements. It is not safe to use the same planner from
// multiple goroutines concurrently.
//
// planners are usually created by using the newPlanner method on a Session.
// If one needs to be created outside of a Session, use makeInternalPlanner().
type planner struct {
txn *client.Txn
clientConnBuffer pgwireconn.ClientBuffer
// Reference to the corresponding sql Statement for this query.
stmt *Statement
// Contexts for different stages of planning and execution.
semaCtx tree.SemaContext
extendedEvalCtx extendedEvalContext
// sessionDataMutator is used to mutate the session variables. Read
// access to them is provided through evalCtx.
sessionDataMutator *sessionDataMutator
// execCfg is used to access the server configuration for the Executor.
execCfg *ExecutorConfig
preparedStatements preparedStatementsAccessor
// avoidCachedDescriptors, when true, instructs all code that
// accesses table/view descriptors to force reading the descriptors
// within the transaction. This is necessary to read descriptors
// from the store for:
// 1. Descriptors that are part of a schema change but are not
// modified by the schema change. (reading a table in CREATE VIEW)
// 2. Disable the use of the table cache in tests.
avoidCachedDescriptors bool
// If set, the planner should skip checking for the SELECT privilege when
// initializing plans to read from a table. This should be used with care.
skipSelectPrivilegeChecks bool
// autoCommit indicates whether we're planning for an implicit transaction.
// If autoCommit is true, the plan is allowed (but not required) to commit the
// transaction along with other KV operations. Committing the txn might be
// beneficial because it may enable the 1PC optimization.
//
// NOTE: plan node must be configured appropriately to actually perform an
// auto-commit. This is dependent on information from the optimizer.
autoCommit bool
// discardRows is set if we want to discard any results rather than sending
// them back to the client. Used for testing/benchmarking. Note that the
// resulting schema or the plan are not affected.
// See EXECUTE .. DISCARD ROWS.
discardRows bool
// cancelChecker is used by planNodes to check for cancellation of the associated
// query.
cancelChecker *sqlbase.CancelChecker
// isPreparing is true if this planner is currently preparing.
isPreparing bool
// curPlan collects the properties of the current plan being prepared. This state
// is undefined at the beginning of the planning of each new statement, and cannot
// be reused for an old prepared statement after a new statement has been prepared.
curPlan planTop
// Avoid allocations by embedding commonly used objects and visitors.
txCtx transform.ExprTransformContext
nameResolutionVisitor sqlbase.NameResolutionVisitor
tableName tree.TableName
// Use a common datum allocator across all the plan nodes. This separates the
// plan lifetime from the lifetime of returned results allowing plan nodes to
// be pool allocated.
alloc sqlbase.DatumAlloc
// optPlanningCtx stores the optimizer planning context, which contains
// data structures that can be reused between queries (for efficiency).
optPlanningCtx optPlanningCtx
queryCacheSession querycache.Session
}
func (ctx *extendedEvalContext) setSessionID(sessionID ClusterWideID) {
ctx.SessionID = sessionID
}
// noteworthyInternalMemoryUsageBytes is the minimum size tracked by each
// internal SQL pool before the pool starts explicitly logging overall usage
// growth in the log.
var noteworthyInternalMemoryUsageBytes = envutil.EnvOrDefaultInt64("COCKROACH_NOTEWORTHY_INTERNAL_MEMORY_USAGE", 1<<20 /* 1 MB */)
// NewInternalPlanner is an exported version of newInternalPlanner. It
// returns an interface{} so it can be used outside of the sql package.
func NewInternalPlanner(
opName string, txn *client.Txn, user string, memMetrics *MemoryMetrics, execCfg *ExecutorConfig,
) (interface{}, func()) {
return newInternalPlanner(opName, txn, user, memMetrics, execCfg)
}
// newInternalPlanner creates a new planner instance for internal usage. This
// planner is not associated with a sql session.
//
// Since it can't be reset, the planner can be used only for planning a single
// statement.
//
// Returns a cleanup function that must be called once the caller is done with
// the planner.
func newInternalPlanner(
opName string, txn *client.Txn, user string, memMetrics *MemoryMetrics, execCfg *ExecutorConfig,
) (*planner, func()) {
// We need a context that outlives all the uses of the planner (since the
// planner captures it in the EvalCtx, and so does the cleanup function that
// we're going to return. We just create one here instead of asking the caller
// for a ctx with this property. This is really ugly, but the alternative of
// asking the caller for one is hard to explain. What we need is better and
// separate interfaces for planning and running plans, which could take
// suitable contexts.
ctx := logtags.AddTag(context.Background(), opName, "")
clientConnBuffer := &silentClientBuffer{}
sd := &sessiondata.SessionData{
SearchPath: sqlbase.DefaultSearchPath,
User: user,
Database: "system",
SequenceState: sessiondata.NewSequenceState(),
DataConversion: sessiondata.DataConversionConfig{
Location: time.UTC,
},
}
// The table collection used by the internal planner does not rely on the
// databaseCache and there are no subscribers to the databaseCache, so we can
// leave it uninitialized.
tables := &TableCollection{
leaseMgr: execCfg.LeaseManager,
settings: execCfg.Settings,
}
dataMutator := &sessionDataMutator{
data: sd,
defaults: SessionDefaults(map[string]string{
"application_name": "crdb-internal",
"database": "system",
}),
settings: execCfg.Settings,
setCurTxnReadOnly: func(bool) {},
clientConnBuffer: clientConnBuffer,
}
var ts time.Time
if txn != nil {
readTimestamp := txn.ReadTimestamp()
if readTimestamp == (hlc.Timestamp{}) {
panic("makeInternalPlanner called with a transaction without timestamps")
}
ts = readTimestamp.GoTime()
}
p := &planner{execCfg: execCfg, clientConnBuffer: clientConnBuffer}
p.txn = txn
p.stmt = nil
p.cancelChecker = sqlbase.NewCancelChecker(ctx)
p.semaCtx = tree.MakeSemaContext()
p.semaCtx.Location = &sd.DataConversion.Location
p.semaCtx.SearchPath = sd.SearchPath
plannerMon := mon.MakeUnlimitedMonitor(ctx,
fmt.Sprintf("internal-planner.%s.%s", user, opName),
mon.MemoryResource,
memMetrics.CurBytesCount, memMetrics.MaxBytesHist,
noteworthyInternalMemoryUsageBytes, execCfg.Settings)
p.extendedEvalCtx = internalExtendedEvalCtx(
ctx, sd, dataMutator, tables, txn, ts, ts, execCfg, &plannerMon,
)
p.extendedEvalCtx.Planner = p
p.extendedEvalCtx.PrivilegedAccessor = p
p.extendedEvalCtx.SessionAccessor = p
p.extendedEvalCtx.Sequence = p
p.extendedEvalCtx.ClusterID = execCfg.ClusterID()
p.extendedEvalCtx.ClusterName = execCfg.RPCContext.ClusterName()
p.extendedEvalCtx.NodeID = execCfg.NodeID.Get()
p.extendedEvalCtx.Locality = execCfg.Locality
p.sessionDataMutator = dataMutator
p.autoCommit = false
p.extendedEvalCtx.MemMetrics = memMetrics
p.extendedEvalCtx.ExecCfg = execCfg
p.extendedEvalCtx.Placeholders = &p.semaCtx.Placeholders
p.extendedEvalCtx.Annotations = &p.semaCtx.Annotations
p.extendedEvalCtx.Tables = tables
p.queryCacheSession.Init()
p.optPlanningCtx.init(p)
return p, func() {
// Note that we capture ctx here. This is only valid as long as we create
// the context as explained at the top of the method.
plannerMon.Stop(ctx)
}
}
// internalExtendedEvalCtx creates an evaluation context for an "internal
// planner". Since the eval context is supposed to be tied to a session and
// there's no session to speak of here, different fields are filled in here to
// keep the tests using the internal planner passing.
func internalExtendedEvalCtx(
ctx context.Context,
sd *sessiondata.SessionData,
dataMutator *sessionDataMutator,
tables *TableCollection,
txn *client.Txn,
txnTimestamp time.Time,
stmtTimestamp time.Time,
execCfg *ExecutorConfig,
plannerMon *mon.BytesMonitor,
) extendedEvalContext {
var evalContextTestingKnobs tree.EvalContextTestingKnobs
var statusServer serverpb.StatusServer
evalContextTestingKnobs = execCfg.EvalContextTestingKnobs
statusServer = execCfg.StatusServer
return extendedEvalContext{
EvalContext: tree.EvalContext{
Txn: txn,
SessionData: sd,
TxnReadOnly: false,
TxnImplicit: true,
Settings: execCfg.Settings,
Context: ctx,
Mon: plannerMon,
TestingKnobs: evalContextTestingKnobs,
StmtTimestamp: stmtTimestamp,
TxnTimestamp: txnTimestamp,
},
SessionMutator: dataMutator,
VirtualSchemas: execCfg.VirtualSchemas,
Tracing: &SessionTracing{},
StatusServer: statusServer,
Tables: tables,
ExecCfg: execCfg,
schemaAccessors: newSchemaInterface(tables, execCfg.VirtualSchemas),
SchemaChangers: &schemaChangerCollection{},
DistSQLPlanner: execCfg.DistSQLPlanner,
}
}
func (p *planner) PhysicalSchemaAccessor() SchemaAccessor {
return p.extendedEvalCtx.schemaAccessors.physical
}
func (p *planner) LogicalSchemaAccessor() SchemaAccessor {
return p.extendedEvalCtx.schemaAccessors.logical
}
// Note: if the context will be modified, use ExtendedEvalContextCopy instead.
func (p *planner) ExtendedEvalContext() *extendedEvalContext {
return &p.extendedEvalCtx
}
func (p *planner) ExtendedEvalContextCopy() *extendedEvalContext {
return p.extendedEvalCtx.copy()
}
func (p *planner) CurrentDatabase() string {
return p.SessionData().Database
}
func (p *planner) CurrentSearchPath() sessiondata.SearchPath {
return p.SessionData().SearchPath
}
// EvalContext() provides convenient access to the planner's EvalContext().
func (p *planner) EvalContext() *tree.EvalContext {
return &p.extendedEvalCtx.EvalContext
}
func (p *planner) Tables() *TableCollection {
return p.extendedEvalCtx.Tables
}
// ExecCfg implements the PlanHookState interface.
func (p *planner) ExecCfg() *ExecutorConfig {
return p.extendedEvalCtx.ExecCfg
}
func (p *planner) LeaseMgr() *LeaseManager {
return p.Tables().leaseMgr
}
func (p *planner) Txn() *client.Txn {
return p.txn
}
func (p *planner) User() string {
return p.SessionData().User
}
func (p *planner) TemporarySchemaName() string {
return temporarySchemaName(p.ExtendedEvalContext().SessionID)
}
// DistSQLPlanner returns the DistSQLPlanner
func (p *planner) DistSQLPlanner() *DistSQLPlanner {
return p.extendedEvalCtx.DistSQLPlanner
}
// ParseType implements the tree.EvalPlanner interface.
// We define this here to break the dependency from eval.go to the parser.
func (p *planner) ParseType(sql string) (*types.T, error) {
return parser.ParseType(sql)
}
// ParseQualifiedTableName implements the tree.EvalDatabase interface.
// This exists to get around a circular dependency between sql/sem/tree and
// sql/parser. sql/parser depends on tree to make objects, so tree cannot import
// ParseQualifiedTableName even though some builtins need that function.
// TODO(jordan): remove this once builtins can be moved outside of sql/sem/tree.
func (p *planner) ParseQualifiedTableName(sql string) (*tree.TableName, error) {
return parser.ParseQualifiedTableName(sql)
}
// ResolveTableName implements the tree.EvalDatabase interface.
func (p *planner) ResolveTableName(ctx context.Context, tn *tree.TableName) (tree.ID, error) {
desc, err := ResolveExistingObject(ctx, p, tn, tree.ObjectLookupFlagsWithRequired(), ResolveAnyDescType)
if err != nil {
return 0, err
}
return tree.ID(desc.ID), nil
}
// LookupTableByID looks up a table, by the given descriptor ID. Based on the
// CommonLookupFlags, it could use or skip the TableCollection cache. See
// TableCollection.getTableVersionByID for how it's used.
func (p *planner) LookupTableByID(ctx context.Context, tableID sqlbase.ID) (row.TableEntry, error) {
flags := tree.ObjectLookupFlags{CommonLookupFlags: tree.CommonLookupFlags{AvoidCached: p.avoidCachedDescriptors}}
table, err := p.Tables().getTableVersionByID(ctx, p.txn, tableID, flags)
if err != nil {
if err == errTableAdding {
return row.TableEntry{IsAdding: true}, nil
}
return row.TableEntry{}, err
}
return row.TableEntry{Desc: table}, nil
}
// TypeAsString enforces (not hints) that the given expression typechecks as a
// string and returns a function that can be called to get the string value
// during (planNode).Start.
// To also allow NULLs to be returned, use typeAsStringOrNull() instead.
func (p *planner) TypeAsString(e tree.Expr, op string) (func() (string, error), error) {
typedE, err := tree.TypeCheckAndRequire(e, &p.semaCtx, types.String, op)
if err != nil {
return nil, err
}
evalFn := p.makeStringEvalFn(typedE)
return func() (string, error) {
isNull, str, err := evalFn()
if err != nil {
return "", err
}
if isNull {
return "", errors.Errorf("expected string, got NULL")
}
return str, nil
}, nil
}
// typeAsStringOrNull is like TypeAsString but allows NULLs.
func (p *planner) typeAsStringOrNull(e tree.Expr, op string) (func() (bool, string, error), error) {
typedE, err := tree.TypeCheckAndRequire(e, &p.semaCtx, types.String, op)
if err != nil {
return nil, err
}
return p.makeStringEvalFn(typedE), nil
}
func (p *planner) makeStringEvalFn(typedE tree.TypedExpr) func() (bool, string, error) {
return func() (bool, string, error) {
d, err := typedE.Eval(p.EvalContext())
if err != nil {
return false, "", err
}
if d == tree.DNull {
return true, "", nil
}
str, ok := d.(*tree.DString)
if !ok {
return false, "", errors.Errorf("failed to cast %T to string", d)
}
return false, string(*str), nil
}
}
// KVStringOptValidate indicates the requested validation of a TypeAsStringOpts
// option.
type KVStringOptValidate string
// KVStringOptValidate values
const (
KVStringOptAny KVStringOptValidate = `any`
KVStringOptRequireNoValue KVStringOptValidate = `no-value`
KVStringOptRequireValue KVStringOptValidate = `value`
)
// evalStringOptions evaluates the KVOption values as strings and returns them
// in a map. Options with no value have an empty string.
func evalStringOptions(
evalCtx *tree.EvalContext, opts []exec.KVOption, optValidate map[string]KVStringOptValidate,
) (map[string]string, error) {
res := make(map[string]string, len(opts))
for _, opt := range opts {
k := opt.Key
validate, ok := optValidate[k]
if !ok {
return nil, errors.Errorf("invalid option %q", k)
}
val, err := opt.Value.Eval(evalCtx)
if err != nil {
return nil, err
}
if val == tree.DNull {
if validate == KVStringOptRequireValue {
return nil, errors.Errorf("option %q requires a value", k)
}
res[k] = ""
} else {
if validate == KVStringOptRequireNoValue {
return nil, errors.Errorf("option %q does not take a value", k)
}
str, ok := val.(*tree.DString)
if !ok {
return nil, errors.Errorf("expected string value, got %T", val)
}
res[k] = string(*str)
}
}
return res, nil
}
// TypeAsStringOpts enforces (not hints) that the given expressions
// typecheck as strings, and returns a function that can be called to
// get the string value during (planNode).Start.
func (p *planner) TypeAsStringOpts(
opts tree.KVOptions, optValidate map[string]KVStringOptValidate,
) (func() (map[string]string, error), error) {
typed := make(map[string]tree.TypedExpr, len(opts))
for _, opt := range opts {
k := string(opt.Key)
validate, ok := optValidate[k]
if !ok {
return nil, errors.Errorf("invalid option %q", k)
}
if opt.Value == nil {
if validate == KVStringOptRequireValue {
return nil, errors.Errorf("option %q requires a value", k)
}
typed[k] = nil
continue
}
if validate == KVStringOptRequireNoValue {
return nil, errors.Errorf("option %q does not take a value", k)
}
r, err := tree.TypeCheckAndRequire(opt.Value, &p.semaCtx, types.String, k)
if err != nil {
return nil, err
}
typed[k] = r
}
fn := func() (map[string]string, error) {
res := make(map[string]string, len(typed))
for name, e := range typed {
if e == nil {
res[name] = ""
continue
}
d, err := e.Eval(p.EvalContext())
if err != nil {
return nil, err
}
str, ok := d.(*tree.DString)
if !ok {
return res, errors.Errorf("failed to cast %T to string", d)
}
res[name] = string(*str)
}
return res, nil
}
return fn, nil
}
// TypeAsStringArray enforces (not hints) that the given expressions all typecheck as
// strings and returns a function that can be called to get the string values
// during (planNode).Start.
func (p *planner) TypeAsStringArray(exprs tree.Exprs, op string) (func() ([]string, error), error) {
typedExprs := make([]tree.TypedExpr, len(exprs))
for i := range exprs {
typedE, err := tree.TypeCheckAndRequire(exprs[i], &p.semaCtx, types.String, op)
if err != nil {
return nil, err
}
typedExprs[i] = typedE
}
fn := func() ([]string, error) {
strs := make([]string, len(exprs))
for i := range exprs {
d, err := typedExprs[i].Eval(p.EvalContext())
if err != nil {
return nil, err
}
str, ok := d.(*tree.DString)
if !ok {
return strs, errors.Errorf("failed to cast %T to string", d)
}
strs[i] = string(*str)
}
return strs, nil
}
return fn, nil
}
// SessionData is part of the PlanHookState interface.
func (p *planner) SessionData() *sessiondata.SessionData {
return p.EvalContext().SessionData
}
// txnModesSetter is an interface used by SQL execution to influence the current
// transaction.
type txnModesSetter interface {
// setTransactionModes updates some characteristics of the current
// transaction.
// asOfTs, if not empty, is the evaluation of modes.AsOf.
setTransactionModes(modes tree.TransactionModes, asOfTs hlc.Timestamp) error
}