diff --git a/pkg/sql/app_stats.go b/pkg/sql/app_stats.go index b7f754f86898..3edc8cf5433e 100644 --- a/pkg/sql/app_stats.go +++ b/pkg/sql/app_stats.go @@ -37,8 +37,6 @@ import ( type stmtKey struct { stmt string failed bool - distSQLUsed bool - vectorized bool implicitTxn bool } @@ -56,6 +54,14 @@ type stmtStats struct { syncutil.Mutex data roachpb.StatementStatistics + + // distSQLUsed records whether the last instance of this statement used + // distribution. + distSQLUsed bool + + // vectorized records whether the last instance of this statement used + // vectorization. + vectorized bool } // transactionStats holds per-application transaction statistics. @@ -105,18 +111,10 @@ var logicalPlanCollectionPeriod = settings.RegisterPublicNonNegativeDurationSett ) func (s stmtKey) String() string { - return s.flags() + s.stmt -} - -func (s stmtKey) flags() string { - var b bytes.Buffer if s.failed { - b.WriteByte('!') - } - if s.distSQLUsed { - b.WriteByte('+') + return "!" + s.stmt } - return b.String() + return s.stmt } // recordStatement saves per-statement statistics. @@ -144,7 +142,7 @@ func (a *appStats) recordStatement( // Get the statistics object. s := a.getStatsForStmt( - stmt, distSQLUsed, vectorized, implicitTxn, + stmt, implicitTxn, err, true, /* createIfNonexistent */ ) @@ -172,24 +170,19 @@ func (a *appStats) recordStatement( s.data.OverheadLat.Record(s.data.Count, ovhLat) s.data.BytesRead.Record(s.data.Count, float64(stats.bytesRead)) s.data.RowsRead.Record(s.data.Count, float64(stats.rowsRead)) + s.distSQLUsed = distSQLUsed + s.vectorized = vectorized s.Unlock() } // getStatsForStmt retrieves the per-stmt stat object. func (a *appStats) getStatsForStmt( - stmt *Statement, - distSQLUsed bool, - vectorized bool, - implicitTxn bool, - err error, - createIfNonexistent bool, + stmt *Statement, implicitTxn bool, err error, createIfNonexistent bool, ) *stmtStats { // Extend the statement key with various characteristics, so // that we use separate buckets for the different situations. key := stmtKey{ failed: err != nil, - distSQLUsed: distSQLUsed, - vectorized: vectorized, implicitTxn: implicitTxn, } if stmt.AnonymizedStr != "" { @@ -299,16 +292,14 @@ func (a *appStats) recordTransaction(txnTimeSec float64, ev txnEvent, implicit b // sample logical plan for its corresponding fingerprint. We use // `logicalPlanCollectionPeriod` to assess how frequently to sample logical // plans. -func (a *appStats) shouldSaveLogicalPlanDescription( - stmt *Statement, useDistSQL bool, vectorized bool, implicitTxn bool, err error, -) bool { +func (a *appStats) shouldSaveLogicalPlanDescription(stmt *Statement, implicitTxn bool) bool { if !sampleLogicalPlans.Get(&a.st.SV) { return false } - stats := a.getStatsForStmt( - stmt, useDistSQL, vectorized, implicitTxn, - err, false, /* createIfNonexistent */ - ) + // We don't know yet if we will hit an error, so we assume we don't. The worst + // that can happen is that for statements that always error out, we will + // always save the tree plan. + stats := a.getStatsForStmt(stmt, implicitTxn, nil /* error */, false /* createIfNonexistent */) if stats == nil { // Save logical plan the first time we see new statement fingerprint. return true @@ -506,9 +497,9 @@ func (s *sqlStats) getStmtStats( if ok { k := roachpb.StatementStatisticsKey{ Query: maybeScrubbed, - DistSQL: q.distSQLUsed, + DistSQL: stats.distSQLUsed, Opt: true, - Vec: q.vectorized, + Vec: stats.vectorized, ImplicitTxn: q.implicitTxn, Failed: q.failed, App: maybeHashedAppName, diff --git a/pkg/sql/apply_join.go b/pkg/sql/apply_join.go index 470c5ac62a30..a6bc1e3e1a53 100644 --- a/pkg/sql/apply_join.go +++ b/pkg/sql/apply_join.go @@ -200,7 +200,7 @@ func (a *applyJoinNode) Next(params runParams) (bool, error) { // the right side of the join using the optimizer, with all outer columns // in the right side replaced by the bindings that were defined by the most // recently read left row. - p, err := a.planRightSideFn(leftRow) + p, err := a.planRightSideFn(newExecFactory(params.p), leftRow) if err != nil { return false, err } diff --git a/pkg/sql/conn_executor_exec.go b/pkg/sql/conn_executor_exec.go index ed5feaafdf76..30a405eb7972 100644 --- a/pkg/sql/conn_executor_exec.go +++ b/pkg/sql/conn_executor_exec.go @@ -913,10 +913,12 @@ func (ex *connExecutor) dispatchToExecutionEngine( // makeExecPlan creates an execution plan and populates planner.curPlan using // the cost-based optimizer. func (ex *connExecutor) makeExecPlan(ctx context.Context, planner *planner) error { - planner.curPlan.init(planner.stmt, ex.appStats) - if planner.collectBundle { - planner.curPlan.savePlanString = true - } + savePlanString := planner.collectBundle + planner.curPlan.init( + planner.stmt, + ex.appStats, + savePlanString, + ) if err := planner.makeOptimizerPlan(ctx); err != nil { log.VEventf(ctx, 1, "optimizer plan failed: %v", err) diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index abded746784f..7bae76f68aa5 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -778,10 +778,17 @@ CREATE TABLE crdb_internal.node_statement_statistics ( if s.data.SensitiveInfo.LastErr != "" { errString = tree.NewDString(s.data.SensitiveInfo.LastErr) } + var flags string + if s.distSQLUsed { + flags = "+" + } + if stmtKey.failed { + flags = "!" + flags + } err := addRow( tree.NewDInt(tree.DInt(nodeID)), tree.NewDString(appName), - tree.NewDString(stmtKey.flags()), + tree.NewDString(flags), tree.NewDString(stmtKey.stmt), anonymized, tree.NewDInt(tree.DInt(s.data.Count)), diff --git a/pkg/sql/executor_statement_metrics.go b/pkg/sql/executor_statement_metrics.go index a12176876d62..2d8aa97a3ccf 100644 --- a/pkg/sql/executor_statement_metrics.go +++ b/pkg/sql/executor_statement_metrics.go @@ -159,7 +159,7 @@ func (ex *connExecutor) recordStatementSummary( } ex.statsCollector.recordStatement( - stmt, planner.curPlan.savedPlanForStats, + stmt, planner.curPlan.planForStats, flags.IsDistributed(), flags.IsSet(planFlagVectorized), flags.IsSet(planFlagImplicitTxn), automaticRetryCount, rowsAffected, err, parseLat, planLat, runLat, svcLat, execOverhead, stats, diff --git a/pkg/sql/explain_plan.go b/pkg/sql/explain_plan.go index bd866099f3c9..55429405e5de 100644 --- a/pkg/sql/explain_plan.go +++ b/pkg/sql/explain_plan.go @@ -30,10 +30,6 @@ const ( // explainSubqueryFmtFlags is the format for subqueries within `EXPLAIN SQL` statements. // Since these are individually run, we don't need to scrub any data from subqueries. explainSubqueryFmtFlags = tree.FmtSimple - - // sampledLogicalPlanFmtFlags is the format for sampled logical plans. Because these exposed - // in the Admin UI, sampled plans should be scrubbed of sensitive information. - sampledLogicalPlanFmtFlags = tree.FmtHideConstants ) // explainPlanNode wraps the logic for EXPLAIN as a planNode. diff --git a/pkg/sql/explain_tree.go b/pkg/sql/explain_tree.go deleted file mode 100644 index 69b0482b37b7..000000000000 --- a/pkg/sql/explain_tree.go +++ /dev/null @@ -1,86 +0,0 @@ -// 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" - "fmt" - - "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/sql/opt/exec/explain" - "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" - "github.com/cockroachdb/cockroach/pkg/util" - "github.com/cockroachdb/errors" -) - -// planToTree returns a representation of the plan as a -// roachpb.ExplainTreePlanNode tree. -func planToTree(ctx context.Context, top *planTop) *roachpb.ExplainTreePlanNode { - var ob explain.OutputBuilder - observer := planObserver{ - // We set followRowSourceToPlanNode to true, to instruct the plan observer - // to follow the edges from rowSourceToPlanNodes (indicating that the prior - // node was not plannable by DistSQL) to the original planNodes that were - // replaced by DistSQL nodes. This prevents the walk from ending at these - // special replacement nodes. - // TODO(jordan): this is pretty hacky. We should modify DistSQL physical - // planning to avoid mutating its input planNode tree instead. - followRowSourceToPlanNode: true, - enterNode: func(ctx context.Context, nodeName string, plan planNode) (bool, error) { - if plan == nil { - ob.EnterMetaNode(nodeName) - } else { - ob.EnterNode(nodeName, planColumns(plan), planReqOrdering(plan)) - } - return true, nil - }, - expr: func(_ observeVerbosity, nodeName, fieldName string, n int, expr tree.Expr) { - if expr == nil { - return - } - ob.AddField(fieldName, tree.AsStringWithFlags(expr, sampledLogicalPlanFmtFlags)) - }, - spans: func(nodeName, fieldName string, index *descpb.IndexDescriptor, spans []roachpb.Span, hardLimitSet bool) { - // TODO(jordan): it's expensive to serialize long span - // strings. It's unfortunate that we're still calling - // PrettySpans, just to check to see whether the output is - or - // not. Unfortunately it's not so clear yet how to write a - // shorter function. Suggestions welcome. - spanss := sqlbase.PrettySpans(index, spans, 2) - if spanss != "" { - if spanss == "-" { - spanss = getAttrForSpansAll(hardLimitSet) - } else { - // Spans contain literal values from the query and thus - // cannot be spelled out in the collected plan. - spanss = fmt.Sprintf("%d span%s", len(spans), util.Pluralize(int64(len(spans)))) - } - ob.AddField(fieldName, spanss) - } - }, - attr: func(nodeName, fieldName, attr string) { - ob.AddField(fieldName, attr) - }, - leaveNode: func(nodeName string, plan planNode) error { - ob.LeaveNode() - return nil - }, - } - - if err := observePlan( - ctx, &top.planComponents, observer, true /* returnError */, sampledLogicalPlanFmtFlags, - ); err != nil { - panic(errors.AssertionFailedf("error while walking plan to save it to statement stats: %s", err.Error())) - } - return ob.BuildProtoTree() -} diff --git a/pkg/sql/explain_tree_test.go b/pkg/sql/explain_tree_test.go index 580e30e0f9c7..25d0650b99fe 100644 --- a/pkg/sql/explain_tree_test.go +++ b/pkg/sql/explain_tree_test.go @@ -66,16 +66,16 @@ func TestPlanToTreeAndPlanToString(t *testing.T) { p.stmt = &Statement{Statement: stmt} p.curPlan.savePlanString = true + p.curPlan.savePlanForStats = true if err := p.makeOptimizerPlan(ctx); err != nil { t.Fatal(err) } + p.curPlan.flags.Set(planFlagExecDone) + p.curPlan.close(ctx) if d.Cmd == "plan-string" { - p.curPlan.flags.Set(planFlagExecDone) - p.curPlan.close(ctx) return p.curPlan.planString } - tree := planToTree(ctx, &p.curPlan) - treeYaml, err := yaml.Marshal(tree) + treeYaml, err := yaml.Marshal(p.curPlan.planForStats) if err != nil { t.Fatal(err) } diff --git a/pkg/sql/logictest/testdata/logic_test/statement_statistics b/pkg/sql/logictest/testdata/logic_test/statement_statistics index fa85e91efe0f..faa70234b6eb 100644 --- a/pkg/sql/logictest/testdata/logic_test/statement_statistics +++ b/pkg/sql/logictest/testdata/logic_test/statement_statistics @@ -120,7 +120,6 @@ SELECT sqrt(_) ! SELECT x FROM (VALUES (_, _, __more1__), (__more1__)) AS t (x) · SELECT x FROM test WHERE y = (_ / z) !+ SELECT x FROM test WHERE y IN (_, _, _ + x, _, _) · -SELECT x FROM test WHERE y IN (_, _, __more3__) · SELECT x FROM test WHERE y IN (_, _, __more3__) + SELECT x FROM test WHERE y NOT IN (_, _, __more3__) · SET CLUSTER SETTING "debug.panic_on_failed_assertions" = DEFAULT · @@ -147,7 +146,6 @@ SELECT _ FROM (VALUES (_, _, __more1__), (__more1__)) AS _ (_) SELECT _ FROM _ WHERE _ = (_ / _) SELECT _ FROM _ WHERE _ IN (_, _, _ + _, _, _) SELECT _ FROM _ WHERE _ IN (_, _, __more3__) -SELECT _ FROM _ WHERE _ IN (_, _, __more3__) SELECT _ FROM _ WHERE _ NOT IN (_, _, __more3__) SET CLUSTER SETTING "debug.panic_on_failed_assertions" = DEFAULT SET CLUSTER SETTING "debug.panic_on_failed_assertions" = _ diff --git a/pkg/sql/opt/exec/execbuilder/relational.go b/pkg/sql/opt/exec/execbuilder/relational.go index d1ef86242d6b..4ab93d64cb80 100644 --- a/pkg/sql/opt/exec/execbuilder/relational.go +++ b/pkg/sql/opt/exec/execbuilder/relational.go @@ -723,7 +723,7 @@ func (b *Builder) buildApplyJoin(join memo.RelExpr) (execPlan, error) { // // Note: we put o outside of the function so we allocate it only once. var o xform.Optimizer - planRightSideFn := func(leftRow tree.Datums) (exec.Plan, error) { + planRightSideFn := func(ef exec.Factory, leftRow tree.Datums) (exec.Plan, error) { o.Init(b.evalCtx, b.catalog) f := o.Factory() @@ -746,7 +746,7 @@ func (b *Builder) buildApplyJoin(join memo.RelExpr) (execPlan, error) { return nil, err } - eb := New(b.factory, f.Memo(), b.catalog, newRightSide, b.evalCtx, false /* allowAutoCommit */) + eb := New(ef, f.Memo(), b.catalog, newRightSide, b.evalCtx, false /* allowAutoCommit */) eb.disableTelemetry = true plan, err := eb.Build() if err != nil { @@ -1795,7 +1795,6 @@ func (b *Builder) buildRecursiveCTE(rec *memo.RecursiveCTEExpr) (execPlan, error // To implement exec.RecursiveCTEIterationFn, we create a special Builder. innerBldTemplate := &Builder{ - factory: b.factory, mem: b.mem, catalog: b.catalog, evalCtx: b.evalCtx, @@ -1806,9 +1805,10 @@ func (b *Builder) buildRecursiveCTE(rec *memo.RecursiveCTEExpr) (execPlan, error withExprs: b.withExprs[:len(b.withExprs):len(b.withExprs)], } - fn := func(bufferRef exec.Node) (exec.Plan, error) { + fn := func(ef exec.Factory, bufferRef exec.Node) (exec.Plan, error) { // Use a separate builder each time. innerBld := *innerBldTemplate + innerBld.factory = ef innerBld.addBuiltWithExpr(rec.WithID, initial.outputCols, bufferRef) plan, err := innerBld.build(rec.Recursive) if err != nil { diff --git a/pkg/sql/opt/exec/explain/emit.go b/pkg/sql/opt/exec/explain/emit.go index ae3f9c4bc6a7..9344a6697a72 100644 --- a/pkg/sql/opt/exec/explain/emit.go +++ b/pkg/sql/opt/exec/explain/emit.go @@ -641,7 +641,15 @@ func (e *emitter) emitSpans( e.ob.Attr(field, "FULL SCAN") } } else { - e.ob.Attr(field, e.spanFormatFn(table, index, scanParams)) + if e.ob.flags.HideValues { + n := len(scanParams.InvertedConstraint) + if scanParams.IndexConstraint != nil { + n = scanParams.IndexConstraint.Spans.Count() + } + e.ob.Attrf(field, "%d span%s", n, util.Pluralize(int64(n))) + } else { + e.ob.Attr(field, e.spanFormatFn(table, index, scanParams)) + } } } diff --git a/pkg/sql/opt/exec/explain/flags.go b/pkg/sql/opt/exec/explain/flags.go index 89ee432e37b9..094c0da428e8 100644 --- a/pkg/sql/opt/exec/explain/flags.go +++ b/pkg/sql/opt/exec/explain/flags.go @@ -20,6 +20,10 @@ type Flags struct { // ShowTypes indicates that the types of columns are shown. // If ShowTypes is true, then Verbose is also true. ShowTypes bool + // If HideValues is true, we hide fields that may contain values from the + // query (e.g. spans). Used internally for the plan visible in the UI. + // If HideValues is true, then Verbose must be false. + HideValues bool } // MakeFlags crates Flags from ExplainOptions. diff --git a/pkg/sql/opt/exec/explain/output.go b/pkg/sql/opt/exec/explain/output.go index 397a1f83fe0f..f9978072cb44 100644 --- a/pkg/sql/opt/exec/explain/output.go +++ b/pkg/sql/opt/exec/explain/output.go @@ -116,6 +116,9 @@ func (ob *OutputBuilder) Expr(key string, expr tree.TypedExpr, varColumns sqlbas if ob.flags.ShowTypes { flags |= tree.FmtShowTypes } + if ob.flags.HideValues { + flags |= tree.FmtHideConstants + } f := tree.NewFmtCtx(flags) f.SetIndexedVarFormat(func(ctx *tree.FmtCtx, idx int) { // Ensure proper quoting. diff --git a/pkg/sql/opt/exec/explain/result_columns.go b/pkg/sql/opt/exec/explain/result_columns.go index 54f383bce26a..1aa7615b8892 100644 --- a/pkg/sql/opt/exec/explain/result_columns.go +++ b/pkg/sql/opt/exec/explain/result_columns.go @@ -196,7 +196,7 @@ func getResultColumns( } return sqlbase.ShowTraceColumns, nil - case createTableOp, createTableAsOp, createViewOp, controlJobsOp, + case createTableOp, createTableAsOp, createViewOp, controlJobsOp, controlSchedulesOp, cancelQueriesOp, cancelSessionsOp, errorIfRowsOp, deleteRangeOp: // These operations produce no columns. return nil, nil diff --git a/pkg/sql/opt/exec/factory.go b/pkg/sql/opt/exec/factory.go index 65fb883d794c..0461e788212a 100644 --- a/pkg/sql/opt/exec/factory.go +++ b/pkg/sql/opt/exec/factory.go @@ -196,12 +196,12 @@ type KVOption struct { // RecursiveCTEIterationFn creates a plan for an iteration of WITH RECURSIVE, // given the result of the last iteration (as a node created by // ConstructBuffer). -type RecursiveCTEIterationFn func(bufferRef Node) (Plan, error) +type RecursiveCTEIterationFn func(ef Factory, bufferRef Node) (Plan, error) // ApplyJoinPlanRightSideFn creates a plan for an iteration of ApplyJoin, given // a row produced from the left side. The plan is guaranteed to produce the // rightColumns passed to ConstructApplyJoin (in order). -type ApplyJoinPlanRightSideFn func(leftRow tree.Datums) (Plan, error) +type ApplyJoinPlanRightSideFn func(ef Factory, leftRow tree.Datums) (Plan, error) // Cascade describes a cascading query. The query uses a node created by // ConstructBuffer as an input; it should only be triggered if this buffer is diff --git a/pkg/sql/plan.go b/pkg/sql/plan.go index f1df28b7905f..c811778ce47e 100644 --- a/pkg/sql/plan.go +++ b/pkg/sql/plan.go @@ -12,6 +12,7 @@ package sql import ( "context" + "fmt" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" @@ -25,7 +26,7 @@ import ( "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/errors" + "github.com/cockroachdb/cockroach/pkg/util/log" ) // runParams is a struct containing all parameters passed to planNode.Next() and @@ -301,14 +302,18 @@ type planTop struct { // If we are collecting query diagnostics, flow diagrams are saved here. distSQLDiagrams []execinfrapb.FlowDiagram - appStats *appStats - savedPlanForStats *roachpb.ExplainTreePlanNode + // If savePlanForStats is true, an ExplainTreePlanNode tree will be saved in + // planForStats when the plan is closed. + savePlanForStats bool + // appStats is used to populate savePlanForStats. + appStats *appStats + planForStats *roachpb.ExplainTreePlanNode // If savePlanString is set to true, an EXPLAIN (VERBOSE)-style plan string - // will be saved in planString. + // will be saved in planString when the plan is closed. savePlanString bool - explainPlan *explain.Plan planString string + explainPlan *explain.Plan } // physicalPlanTop is a utility wrapper around PhysicalPlan that allows for @@ -419,40 +424,24 @@ func (p *planComponents) close(ctx context.Context) { // init resets planTop to point to a given statement; used at the start of the // planning process. -func (p *planTop) init(stmt *Statement, appStats *appStats) { - *p = planTop{stmt: stmt, appStats: appStats} +func (p *planTop) init(stmt *Statement, appStats *appStats, savePlanString bool) { + *p = planTop{ + stmt: stmt, + appStats: appStats, + savePlanString: savePlanString, + } } // close ensures that the plan's resources have been deallocated. func (p *planTop) close(ctx context.Context) { - if p.main.planNode != nil && p.flags.IsSet(planFlagExecDone) { - // TODO(yuzefovich): update this once we support creating table reader - // specs directly in the optimizer (see #47474). - if p.appStats != nil && p.appStats.shouldSaveLogicalPlanDescription( - p.stmt, - p.flags.IsDistributed(), - p.flags.IsSet(planFlagVectorized), - p.flags.IsSet(planFlagImplicitTxn), - p.execErr, - ) { - p.savedPlanForStats = planToTree(ctx, p) - } - - if p.savePlanString && p.explainPlan != nil { - var err error - p.planString, err = p.formatExplain() - if err != nil { - p.planString = err.Error() - } - } + if p.explainPlan != nil && p.flags.IsSet(planFlagExecDone) { + p.savePlanInfo(ctx) } p.planComponents.close(ctx) } -func (p *planTop) formatExplain() (string, error) { - if p.explainPlan == nil { - return "", errors.AssertionFailedf("no plan") - } +// savePlanInfo uses p.explainPlan to populate the plan string and/or tree. +func (p *planTop) savePlanInfo(ctx context.Context) { vectorized := p.flags.IsSet(planFlagVectorized) distribution := physicalplan.LocalPlan if p.flags.IsSet(planFlagFullyDistributed) { @@ -461,14 +450,28 @@ func (p *planTop) formatExplain() (string, error) { distribution = physicalplan.PartiallyDistributedPlan } - ob := explain.NewOutputBuilder(explain.Flags{ - Verbose: true, - ShowTypes: true, - }) - if err := emitExplain(ob, p.codec, p.explainPlan, distribution, vectorized); err != nil { - return "", err + if p.savePlanForStats { + ob := explain.NewOutputBuilder(explain.Flags{ + HideValues: true, + }) + if err := emitExplain(ob, p.codec, p.explainPlan, distribution, vectorized); err != nil { + log.Warningf(ctx, "unable to emit explain plan tree: %v", err) + } else { + p.planForStats = ob.BuildProtoTree() + } + } + + if p.savePlanString { + ob := explain.NewOutputBuilder(explain.Flags{ + Verbose: true, + ShowTypes: true, + }) + if err := emitExplain(ob, p.codec, p.explainPlan, distribution, vectorized); err != nil { + p.planString = fmt.Sprintf("error emitting plan: %v", err) + } else { + p.planString = ob.BuildString() + } } - return ob.BuildString(), nil } // formatOptPlan returns a visual representation of the optimizer plan that was diff --git a/pkg/sql/plan_opt.go b/pkg/sql/plan_opt.go index 543c556cc3db..f6d78364613e 100644 --- a/pkg/sql/plan_opt.go +++ b/pkg/sql/plan_opt.go @@ -552,7 +552,16 @@ func (opc *optPlanningCtx) runExecBuilder( var result *planComponents var explainPlan *explain.Plan var isDDL bool - if !planTop.savePlanString { + if planTop.appStats != nil { + // We do not set this flag upfront when initializing planTop because the + // planning process could in principle modify the AST, resulting in a + // different statement signature. + planTop.savePlanForStats = planTop.appStats.shouldSaveLogicalPlanDescription( + planTop.stmt, + allowAutoCommit, + ) + } + if !planTop.savePlanString && !planTop.savePlanForStats { // No instrumentation. bld := execbuilder.New(f, mem, &opc.catalog, mem.RootExpr(), evalCtx, allowAutoCommit) plan, err := bld.Build() diff --git a/pkg/sql/recursive_cte.go b/pkg/sql/recursive_cte.go index 568bdef376c9..c6c355aa076a 100644 --- a/pkg/sql/recursive_cte.go +++ b/pkg/sql/recursive_cte.go @@ -116,7 +116,7 @@ func (n *recursiveCTENode) Next(params runParams) (bool, error) { bufferedRows: lastWorkingRows, label: n.label, } - newPlan, err := n.genIterationFn(buf) + newPlan, err := n.genIterationFn(newExecFactory(params.p), buf) if err != nil { return false, err } diff --git a/pkg/sql/sem/tree/hide_constants.go b/pkg/sql/sem/tree/hide_constants.go index bf133f845dcb..40bf1302e81a 100644 --- a/pkg/sql/sem/tree/hide_constants.go +++ b/pkg/sql/sem/tree/hide_constants.go @@ -118,9 +118,9 @@ func (node *Tuple) formatHideConstants(ctx *FmtCtx) { v2.Exprs = append(make(Exprs, 0, 3), v2.Exprs[:2]...) if len(node.Exprs) > 2 { v2.Exprs = append(v2.Exprs, arityIndicator(len(node.Exprs)-2)) - } - if node.Labels != nil { - v2.Labels = node.Labels[:2] + if node.Labels != nil { + v2.Labels = node.Labels[:2] + } } v2.Format(ctx) return diff --git a/pkg/sql/testdata/explain_tree b/pkg/sql/testdata/explain_tree index 10e8b8741848..58cf5849d7be 100644 --- a/pkg/sql/testdata/explain_tree +++ b/pkg/sql/testdata/explain_tree @@ -18,6 +18,8 @@ SELECT oid FROM t.orders WHERE oid = 123 ---- name: scan attrs: +- key: missing stats + value: "" - key: table value: orders@primary - key: spans @@ -38,17 +40,13 @@ project (cid int, date date, value plan-tree SELECT cid, date, value FROM t.orders ---- -name: render -attrs: -- key: render - value: cid -- key: render - value: date -- key: render - value: value +name: project +attrs: [] children: - name: scan attrs: + - key: missing stats + value: "" - key: table value: orders@primary - key: spans @@ -85,12 +83,8 @@ project plan-tree SELECT cid, sum(value) FROM t.orders WHERE date > '2015-01-01' GROUP BY cid ORDER BY 1 - sum(value) ---- -name: render -attrs: -- key: render - value: cid -- key: render - value: sum +name: project +attrs: [] children: - name: sort attrs: @@ -98,13 +92,7 @@ children: value: +column7 children: - name: render - attrs: - - key: render - value: _ - sum - - key: render - value: cid - - key: render - value: sum + attrs: [] children: - name: group attrs: @@ -113,12 +101,8 @@ children: - key: group by value: cid children: - - name: render - attrs: - - key: render - value: cid - - key: render - value: value + - name: project + attrs: [] children: - name: filter attrs: @@ -127,6 +111,8 @@ children: children: - name: scan attrs: + - key: missing stats + value: "" - key: table value: orders@primary - key: spans @@ -148,6 +134,8 @@ SELECT value FROM (SELECT cid, date, value FROM t.orders) ---- name: scan attrs: +- key: missing stats + value: "" - key: table value: orders@primary - key: spans @@ -182,43 +170,44 @@ project (cid int, d plan-tree SELECT cid, date, value FROM t.orders WHERE date IN (SELECT date FROM t.orders) ---- -name: render -attrs: -- key: render - value: cid -- key: render - value: date -- key: render - value: value +name: project +attrs: [] children: -- name: hash join - attrs: - - key: type - value: inner - - key: equality - value: (date) = (date) - - key: right cols are key - value: "" +- name: project + attrs: [] children: - - name: scan + - name: hash join attrs: - - key: table - value: orders@primary - - key: spans - value: FULL SCAN - children: [] - - name: distinct - attrs: - - key: distinct on - value: date + - key: type + value: inner + - key: equality + value: (date) = (date) + - key: right cols are key + value: "" children: - name: scan attrs: + - key: missing stats + value: "" - key: table value: orders@primary - key: spans value: FULL SCAN children: [] + - name: distinct + attrs: + - key: distinct on + value: date + children: + - name: scan + attrs: + - key: missing stats + value: "" + - key: table + value: orders@primary + - key: spans + value: FULL SCAN + children: [] exec CREATE TABLE t.movies ( @@ -272,28 +261,27 @@ SELECT id AS movie_id, title, (SELECT name FROM t.actors WHERE name = 'Foo') FRO name: root attrs: [] children: -- name: render - attrs: - - key: render - value: id - - key: render - value: title - - key: render - value: (SELECT name FROM t.actors WHERE name = _) +- name: project + attrs: [] children: - - name: scan - attrs: - - key: table - value: movies@primary - - key: spans - value: FULL SCAN - children: [] + - name: render + attrs: [] + children: + - name: scan + attrs: + - key: missing stats + value: "" + - key: table + value: movies@primary + - key: spans + value: FULL SCAN + children: [] - name: subquery attrs: - key: id value: '@S1' - key: original sql - value: (SELECT name FROM t.actors WHERE name = _) + value: (SELECT name FROM t.actors WHERE name = 'Foo') - key: exec mode value: one row children: @@ -307,6 +295,8 @@ children: children: - name: scan attrs: + - key: missing stats + value: "" - key: table value: actors@primary - key: spans