-
Notifications
You must be signed in to change notification settings - Fork 3.8k
/
plan_opt.go
637 lines (587 loc) · 21.8 KB
/
plan_opt.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
// Copyright 2018 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"
"strings"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/opt"
"github.com/cockroachdb/cockroach/pkg/sql/opt/exec"
"github.com/cockroachdb/cockroach/pkg/sql/opt/exec/execbuilder"
"github.com/cockroachdb/cockroach/pkg/sql/opt/exec/explain"
"github.com/cockroachdb/cockroach/pkg/sql/opt/memo"
"github.com/cockroachdb/cockroach/pkg/sql/opt/optbuilder"
"github.com/cockroachdb/cockroach/pkg/sql/opt/xform"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/physicalplan"
"github.com/cockroachdb/cockroach/pkg/sql/querycache"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/errors"
)
var queryCacheEnabled = settings.RegisterBoolSetting(
"sql.query_cache.enabled", "enable the query cache", true,
)
// prepareUsingOptimizer builds a memo for a prepared statement and populates
// the following stmt.Prepared fields:
// - Columns
// - Types
// - AnonymizedStr
// - Memo (for reuse during exec, if appropriate).
func (p *planner) prepareUsingOptimizer(ctx context.Context) (planFlags, error) {
stmt := &p.stmt
opc := &p.optPlanningCtx
opc.reset()
switch stmt.AST.(type) {
case *tree.AlterIndex, *tree.AlterTable, *tree.AlterSequence,
*tree.Analyze,
*tree.BeginTransaction,
*tree.CommentOnColumn, *tree.CommentOnDatabase, *tree.CommentOnIndex, *tree.CommentOnTable, *tree.CommentOnSchema,
*tree.CommitTransaction,
*tree.CopyFrom, *tree.CreateDatabase, *tree.CreateIndex, *tree.CreateView,
*tree.CreateSequence,
*tree.CreateStats,
*tree.Deallocate, *tree.Discard, *tree.DropDatabase, *tree.DropIndex,
*tree.DropTable, *tree.DropView, *tree.DropSequence, *tree.DropType,
*tree.Execute,
*tree.Grant, *tree.GrantRole,
*tree.Prepare,
*tree.ReleaseSavepoint, *tree.RenameColumn, *tree.RenameDatabase,
*tree.RenameIndex, *tree.RenameTable, *tree.Revoke, *tree.RevokeRole,
*tree.RollbackToSavepoint, *tree.RollbackTransaction,
*tree.Savepoint, *tree.SetTransaction, *tree.SetTracing, *tree.SetSessionAuthorizationDefault,
*tree.SetSessionCharacteristics:
// These statements do not have result columns and do not support placeholders
// so there is no need to do anything during prepare.
//
// Some of these statements (like BeginTransaction) aren't supported by the
// optbuilder so they would error out. Others (like CreateIndex) have planning
// code that can introduce unnecessary txn retries (because of looking up
// descriptors and such).
return opc.flags, nil
case *tree.ExplainAnalyze:
// This statement returns result columns but does not support placeholders,
// and we don't want to do anything during prepare.
if len(p.semaCtx.Placeholders.Types) != 0 {
return 0, errors.Errorf("%s does not support placeholders", stmt.AST.StatementTag())
}
stmt.Prepared.Columns = colinfo.ExplainPlanColumns
return opc.flags, nil
}
if opc.useCache {
cachedData, ok := p.execCfg.QueryCache.Find(&p.queryCacheSession, stmt.SQL)
if ok && cachedData.PrepareMetadata != nil {
pm := cachedData.PrepareMetadata
// Check that the type hints match (the type hints affect type checking).
if !pm.TypeHints.Identical(p.semaCtx.Placeholders.TypeHints) {
opc.log(ctx, "query cache hit but type hints don't match")
} else {
isStale, err := cachedData.Memo.IsStale(ctx, p.EvalContext(), &opc.catalog)
if err != nil {
return 0, err
}
if !isStale {
opc.log(ctx, "query cache hit (prepare)")
opc.flags.Set(planFlagOptCacheHit)
stmt.Prepared.StatementNoConstants = pm.StatementNoConstants
stmt.Prepared.Columns = pm.Columns
stmt.Prepared.Types = pm.Types
stmt.Prepared.Memo = cachedData.Memo
return opc.flags, nil
}
opc.log(ctx, "query cache hit but memo is stale (prepare)")
}
} else if ok {
opc.log(ctx, "query cache hit but there is no prepare metadata")
} else {
opc.log(ctx, "query cache miss")
}
opc.flags.Set(planFlagOptCacheMiss)
}
memo, err := opc.buildReusableMemo(ctx)
if err != nil {
return 0, err
}
md := memo.Metadata()
physical := memo.RootProps()
resultCols := make(colinfo.ResultColumns, len(physical.Presentation))
for i, col := range physical.Presentation {
colMeta := md.ColumnMeta(col.ID)
resultCols[i].Name = col.Alias
resultCols[i].Typ = colMeta.Type
if err := checkResultType(resultCols[i].Typ); err != nil {
return 0, err
}
// If the column came from a table, set up the relevant metadata.
if colMeta.Table != opt.TableID(0) {
// Get the cat.Table that this column references.
tab := md.Table(colMeta.Table)
resultCols[i].TableID = descpb.ID(tab.ID())
// Convert the metadata opt.ColumnID to its ordinal position in the table.
colOrdinal := colMeta.Table.ColumnOrdinal(col.ID)
// Use that ordinal position to retrieve the column's stable ID.
var column catalog.Column
if catTable, ok := tab.(optCatalogTableInterface); ok {
column = catTable.getCol(colOrdinal)
}
if column != nil {
resultCols[i].PGAttributeNum = column.GetPGAttributeNum()
} else {
resultCols[i].PGAttributeNum = uint32(tab.Column(colOrdinal).ColID())
}
}
}
// Verify that all placeholder types have been set.
if err := p.semaCtx.Placeholders.Types.AssertAllSet(); err != nil {
return 0, err
}
stmt.Prepared.Columns = resultCols
stmt.Prepared.Types = p.semaCtx.Placeholders.Types
if opc.allowMemoReuse {
stmt.Prepared.Memo = memo
if opc.useCache {
// execPrepare sets the PrepareMetadata.InferredTypes field after this
// point. However, once the PrepareMetadata goes into the cache, it
// can't be modified without causing race conditions. So make a copy of
// it now.
// TODO(radu): Determine if the extra object allocation is really
// necessary.
pm := stmt.Prepared.PrepareMetadata
cachedData := querycache.CachedData{
SQL: stmt.SQL,
Memo: memo,
PrepareMetadata: &pm,
}
p.execCfg.QueryCache.Add(&p.queryCacheSession, &cachedData)
}
}
return opc.flags, nil
}
// makeOptimizerPlan generates a plan using the cost-based optimizer.
// On success, it populates p.curPlan.
func (p *planner) makeOptimizerPlan(ctx context.Context) error {
p.curPlan.init(&p.stmt, &p.instrumentation)
opc := &p.optPlanningCtx
opc.reset()
execMemo, err := opc.buildExecMemo(ctx)
if err != nil {
return err
}
// Build the plan tree.
if mode := p.SessionData().ExperimentalDistSQLPlanningMode; mode != sessiondatapb.ExperimentalDistSQLPlanningOff {
planningMode := distSQLDefaultPlanning
// If this transaction has modified or created any types, it is not safe to
// distribute due to limitations around leasing descriptors modified in the
// current transaction.
if p.Descriptors().HasUncommittedTypes() {
planningMode = distSQLLocalOnlyPlanning
}
err := opc.runExecBuilder(
&p.curPlan,
&p.stmt,
newDistSQLSpecExecFactory(p, planningMode),
execMemo,
p.EvalContext(),
p.autoCommit,
)
if err != nil {
if mode == sessiondatapb.ExperimentalDistSQLPlanningAlways &&
!strings.Contains(p.stmt.AST.StatementTag(), "SET") {
// We do not fallback to the old path because experimental
// planning is set to 'always' and we don't have a SET
// statement, so we return an error. SET statements are
// exceptions because we want to be able to execute them
// regardless of whether they are supported by the new factory.
// TODO(yuzefovich): update this once SET statements are
// supported (see #47473).
return err
}
// We will fallback to the old path.
} else {
// TODO(yuzefovich): think through whether subqueries or
// postqueries can be distributed. If that's the case, we might
// need to also look at the plan distribution of those.
m := p.curPlan.main
isPartiallyDistributed := m.physPlan.Distribution == physicalplan.PartiallyDistributedPlan
if isPartiallyDistributed && p.SessionData().PartiallyDistributedPlansDisabled {
// The planning has succeeded, but we've created a partially
// distributed plan yet the session variable prohibits such
// plan distribution - we need to replan with a new factory
// that forces local planning.
// TODO(yuzefovich): remove this logic when deleting old
// execFactory.
err = opc.runExecBuilder(
&p.curPlan,
&p.stmt,
newDistSQLSpecExecFactory(p, distSQLLocalOnlyPlanning),
execMemo,
p.EvalContext(),
p.autoCommit,
)
}
if err == nil {
return nil
}
}
// TODO(yuzefovich): make the logging conditional on the verbosity
// level once new DistSQL planning is no longer experimental.
log.Infof(
ctx, "distSQLSpecExecFactory failed planning with %v, falling back to the old path", err,
)
}
// If we got here, we did not create a plan above.
return opc.runExecBuilder(
&p.curPlan,
&p.stmt,
newExecFactory(p),
execMemo,
p.EvalContext(),
p.autoCommit,
)
}
type optPlanningCtx struct {
p *planner
// catalog is initialized once, and reset for each query. This allows the
// catalog objects to be reused across queries in the same session.
catalog optCatalog
// -- Fields below are reinitialized for each query ---
optimizer xform.Optimizer
// When set, we are allowed to reuse a memo, or store a memo for later reuse.
allowMemoReuse bool
// When set, we consult and update the query cache. Never set if
// allowMemoReuse is false.
useCache bool
flags planFlags
}
// init performs one-time initialization of the planning context; reset() must
// also be called before each use.
func (opc *optPlanningCtx) init(p *planner) {
opc.p = p
opc.catalog.init(p)
}
// reset initializes the planning context for the statement in the planner.
func (opc *optPlanningCtx) reset() {
p := opc.p
opc.catalog.reset()
opc.optimizer.Init(p.EvalContext(), &opc.catalog)
opc.flags = 0
// We only allow memo caching for SELECT/INSERT/UPDATE/DELETE. We could
// support it for all statements in principle, but it would increase the
// surface of potential issues (conditions we need to detect to invalidate a
// cached memo).
switch p.stmt.AST.(type) {
case *tree.ParenSelect, *tree.Select, *tree.SelectClause, *tree.UnionClause, *tree.ValuesClause,
*tree.Insert, *tree.Update, *tree.Delete, *tree.CannedOptPlan:
// If the current transaction has uncommitted DDL statements, we cannot rely
// on descriptor versions for detecting a "stale" memo. This is because
// descriptor versions are bumped at most once per transaction, even if there
// are multiple DDL operations; and transactions can be aborted leading to
// potential reuse of versions. To avoid these issues, we prevent saving a
// memo (for prepare) or reusing a saved memo (for execute).
opc.allowMemoReuse = !p.Descriptors().HasUncommittedTables()
opc.useCache = opc.allowMemoReuse && queryCacheEnabled.Get(&p.execCfg.Settings.SV)
if _, isCanned := p.stmt.AST.(*tree.CannedOptPlan); isCanned {
// It's unsafe to use the cache, since PREPARE AS OPT PLAN doesn't track
// dependencies and check permissions.
opc.useCache = false
}
default:
opc.allowMemoReuse = false
opc.useCache = false
}
}
func (opc *optPlanningCtx) log(ctx context.Context, msg string) {
if log.VDepth(1, 1) {
log.InfofDepth(ctx, 1, "%s: %s", log.Safe(msg), opc.p.stmt)
} else {
log.Event(ctx, msg)
}
}
// buildReusableMemo builds the statement into a memo that can be stored for
// prepared statements and can later be used as a starting point for
// optimization. The returned memo is fully detached from the planner and can be
// used with reuseMemo independently and concurrently by multiple threads.
func (opc *optPlanningCtx) buildReusableMemo(ctx context.Context) (_ *memo.Memo, _ error) {
p := opc.p
_, isCanned := opc.p.stmt.AST.(*tree.CannedOptPlan)
if isCanned {
if !p.EvalContext().SessionData().AllowPrepareAsOptPlan {
return nil, pgerror.New(pgcode.InsufficientPrivilege,
"PREPARE AS OPT PLAN is a testing facility that should not be used directly",
)
}
if !p.SessionData().User().IsRootUser() {
return nil, pgerror.New(pgcode.InsufficientPrivilege,
"PREPARE AS OPT PLAN may only be used by root",
)
}
}
if p.SessionData().SaveTablesPrefix != "" && !p.SessionData().User().IsRootUser() {
return nil, pgerror.New(pgcode.InsufficientPrivilege,
"sub-expression tables creation may only be used by root",
)
}
// Build the Memo (optbuild) and apply normalization rules to it. If the
// query contains placeholders, values are not assigned during this phase,
// as that only happens during the EXECUTE phase. If the query does not
// contain placeholders, then also apply exploration rules to the Memo so
// that there's even less to do during the EXECUTE phase.
//
f := opc.optimizer.Factory()
bld := optbuilder.New(ctx, &p.semaCtx, p.EvalContext(), &opc.catalog, f, opc.p.stmt.AST)
bld.KeepPlaceholders = true
if err := bld.Build(); err != nil {
return nil, err
}
if bld.DisableMemoReuse {
// The builder encountered a statement that prevents safe reuse of the memo.
opc.allowMemoReuse = false
opc.useCache = false
}
if isCanned {
if f.Memo().HasPlaceholders() {
// We don't support placeholders inside the canned plan. The main reason
// is that they would be invisible to the parser (which reports the number
// of placeholders, used to initialize the relevant structures).
return nil, pgerror.Newf(pgcode.Syntax,
"placeholders are not supported with PREPARE AS OPT PLAN")
}
// With a canned plan, we don't want to optimize the memo.
return opc.optimizer.DetachMemo(), nil
}
if f.Memo().HasPlaceholders() {
// Try the placeholder fast path.
_, ok, err := opc.optimizer.TryPlaceholderFastPath()
if err != nil {
return nil, err
}
if ok {
opc.log(ctx, "placeholder fast path")
}
} else {
// If the memo doesn't have placeholders and did not encounter any stable
// operators that can be constant folded, then fully optimize it now - it
// can be reused without further changes to build the execution tree.
if !f.FoldingControl().PreventedStableFold() {
opc.log(ctx, "optimizing (no placeholders)")
if _, err := opc.optimizer.Optimize(); err != nil {
return nil, err
}
}
}
// Detach the prepared memo from the factory and transfer its ownership
// to the prepared statement. DetachMemo will re-initialize the optimizer
// to an empty memo.
return opc.optimizer.DetachMemo(), nil
}
// reuseMemo returns an optimized memo using a cached memo as a starting point.
//
// The cached memo is not modified; it is safe to call reuseMemo on the same
// cachedMemo from multiple threads concurrently.
//
// The returned memo is only safe to use in one thread, during execution of the
// current statement.
func (opc *optPlanningCtx) reuseMemo(cachedMemo *memo.Memo) (*memo.Memo, error) {
if cachedMemo.IsOptimized() {
// The query could have been already fully optimized if there were no
// placeholders or the placeholder fast path succeeded (see
// buildReusableMemo).
return cachedMemo, nil
}
f := opc.optimizer.Factory()
// Finish optimization by assigning any remaining placeholders and
// applying exploration rules. Reinitialize the optimizer and construct a
// new memo that is copied from the prepared memo, but with placeholders
// assigned. Stable operators can be constant-folded at this time.
f.FoldingControl().AllowStableFolds()
if err := f.AssignPlaceholders(cachedMemo); err != nil {
return nil, err
}
if _, err := opc.optimizer.Optimize(); err != nil {
return nil, err
}
return f.Memo(), nil
}
// buildExecMemo creates a fully optimized memo, possibly reusing a previously
// cached memo as a starting point.
//
// The returned memo is only safe to use in one thread, during execution of the
// current statement.
func (opc *optPlanningCtx) buildExecMemo(ctx context.Context) (_ *memo.Memo, _ error) {
prepared := opc.p.stmt.Prepared
p := opc.p
if opc.allowMemoReuse && prepared != nil && prepared.Memo != nil {
// We are executing a previously prepared statement and a reusable memo is
// available.
// If the prepared memo has been invalidated by schema or other changes,
// re-prepare it.
if isStale, err := prepared.Memo.IsStale(ctx, p.EvalContext(), &opc.catalog); err != nil {
return nil, err
} else if isStale {
prepared.Memo, err = opc.buildReusableMemo(ctx)
opc.log(ctx, "rebuilding cached memo")
if err != nil {
return nil, err
}
}
opc.log(ctx, "reusing cached memo")
memo, err := opc.reuseMemo(prepared.Memo)
return memo, err
}
if opc.useCache {
// Consult the query cache.
cachedData, ok := p.execCfg.QueryCache.Find(&p.queryCacheSession, opc.p.stmt.SQL)
if ok {
if isStale, err := cachedData.Memo.IsStale(ctx, p.EvalContext(), &opc.catalog); err != nil {
return nil, err
} else if isStale {
cachedData.Memo, err = opc.buildReusableMemo(ctx)
if err != nil {
return nil, err
}
// Update the plan in the cache. If the cache entry had PrepareMetadata
// populated, it may no longer be valid.
cachedData.PrepareMetadata = nil
p.execCfg.QueryCache.Add(&p.queryCacheSession, &cachedData)
opc.log(ctx, "query cache hit but needed update")
opc.flags.Set(planFlagOptCacheMiss)
} else {
opc.log(ctx, "query cache hit")
opc.flags.Set(planFlagOptCacheHit)
}
memo, err := opc.reuseMemo(cachedData.Memo)
return memo, err
}
opc.flags.Set(planFlagOptCacheMiss)
opc.log(ctx, "query cache miss")
} else {
opc.log(ctx, "not using query cache")
}
// We are executing a statement for which there is no reusable memo
// available.
f := opc.optimizer.Factory()
f.FoldingControl().AllowStableFolds()
bld := optbuilder.New(ctx, &p.semaCtx, p.EvalContext(), &opc.catalog, f, opc.p.stmt.AST)
if err := bld.Build(); err != nil {
return nil, err
}
if _, isCanned := opc.p.stmt.AST.(*tree.CannedOptPlan); !isCanned {
if _, err := opc.optimizer.Optimize(); err != nil {
return nil, err
}
}
// If this statement doesn't have placeholders and we have not constant-folded
// any VolatilityStable operators, add it to the cache.
// Note that non-prepared statements from pgwire clients cannot have
// placeholders.
if opc.useCache && !bld.HadPlaceholders && !bld.DisableMemoReuse &&
!f.FoldingControl().PermittedStableFold() {
memo := opc.optimizer.DetachMemo()
cachedData := querycache.CachedData{
SQL: opc.p.stmt.SQL,
Memo: memo,
}
p.execCfg.QueryCache.Add(&p.queryCacheSession, &cachedData)
opc.log(ctx, "query cache add")
return memo, nil
}
return f.Memo(), nil
}
// runExecBuilder execbuilds a plan using the given factory and stores the
// result in planTop. If required, also captures explain data using the explain
// factory.
func (opc *optPlanningCtx) runExecBuilder(
planTop *planTop,
stmt *Statement,
f exec.Factory,
mem *memo.Memo,
evalCtx *tree.EvalContext,
allowAutoCommit bool,
) error {
var result *planComponents
var isDDL bool
var containsFullTableScan bool
var containsFullIndexScan bool
var containsLargeFullTableScan bool
var containsLargeFullIndexScan bool
var containsMutation bool
if !planTop.instrumentation.ShouldBuildExplainPlan() {
// No instrumentation.
bld := execbuilder.New(f, &opc.optimizer, mem, &opc.catalog, mem.RootExpr(), evalCtx, allowAutoCommit)
plan, err := bld.Build()
if err != nil {
return err
}
result = plan.(*planComponents)
isDDL = bld.IsDDL
containsFullTableScan = bld.ContainsFullTableScan
containsFullIndexScan = bld.ContainsFullIndexScan
containsLargeFullTableScan = bld.ContainsLargeFullTableScan
containsLargeFullIndexScan = bld.ContainsLargeFullIndexScan
containsMutation = bld.ContainsMutation
} else {
// Create an explain factory and record the explain.Plan.
explainFactory := explain.NewFactory(f)
bld := execbuilder.New(
explainFactory, &opc.optimizer, mem, &opc.catalog, mem.RootExpr(), evalCtx, allowAutoCommit,
)
plan, err := bld.Build()
if err != nil {
return err
}
explainPlan := plan.(*explain.Plan)
result = explainPlan.WrappedPlan.(*planComponents)
isDDL = bld.IsDDL
containsFullTableScan = bld.ContainsFullTableScan
containsFullIndexScan = bld.ContainsFullIndexScan
containsLargeFullTableScan = bld.ContainsLargeFullTableScan
containsLargeFullIndexScan = bld.ContainsLargeFullIndexScan
containsMutation = bld.ContainsMutation
planTop.instrumentation.RecordExplainPlan(explainPlan)
}
planTop.instrumentation.costEstimate = float64(mem.RootExpr().(memo.RelExpr).Cost())
if stmt.ExpectedTypes != nil {
cols := result.main.planColumns()
if !stmt.ExpectedTypes.TypesEqual(cols) {
return pgerror.New(pgcode.FeatureNotSupported, "cached plan must not change result type")
}
}
planTop.planComponents = *result
planTop.stmt = stmt
planTop.flags = opc.flags
if isDDL {
planTop.flags.Set(planFlagIsDDL)
}
if containsFullTableScan {
planTop.flags.Set(planFlagContainsFullTableScan)
}
if containsFullIndexScan {
planTop.flags.Set(planFlagContainsFullIndexScan)
}
if containsLargeFullTableScan {
planTop.flags.Set(planFlagContainsLargeFullTableScan)
}
if containsLargeFullIndexScan {
planTop.flags.Set(planFlagContainsLargeFullIndexScan)
}
if containsMutation {
planTop.flags.Set(planFlagContainsMutation)
}
if planTop.instrumentation.ShouldSaveMemo() {
planTop.mem = mem
planTop.catalog = &opc.catalog
}
return nil
}