Skip to content

Commit

Permalink
sql: cleanup handling of EXPLAIN in the new factory
Browse files Browse the repository at this point in the history
Previously, several EXPLAIN variants were mishandled in the new factory:
instead of being wrapped as other planNodes are they were kept separate
which resulted in a "mixed"-representation plan. This is now fixed by
properly wrapping them.

Additionally, this commit removes "partial support" that we had for
EXPLAIN (PLAN) - the new factory cannot currently implement it, so in
such cases we will be falling back to the old factory.

Release note: None
  • Loading branch information
yuzefovich committed Aug 3, 2020
1 parent f6d753a commit 08090b6
Show file tree
Hide file tree
Showing 11 changed files with 105 additions and 160 deletions.
45 changes: 26 additions & 19 deletions pkg/sql/distsql_spec_exec_factory.go
Original file line number Diff line number Diff line change
Expand Up @@ -349,12 +349,6 @@ func (e *distSQLSpecExecFactory) ConstructInvertedFilter(
func (e *distSQLSpecExecFactory) ConstructSimpleProject(
n exec.Node, cols []exec.NodeColumnOrdinal, colNames []string, reqOrdering exec.OutputOrdering,
) (exec.Node, error) {
// distSQLSpecExecFactory still constructs some of the planNodes (for
// example, some variants of EXPLAIN), and we need to be able to add a
// simple projection on top of them.
if p, ok := n.(planNode); ok {
return constructSimpleProjectForPlanNode(p, cols, colNames, reqOrdering)
}
physPlan, plan := getPhysPlan(n)
projection := make([]uint32, len(cols))
for i := range cols {
Expand Down Expand Up @@ -710,18 +704,7 @@ func (e *distSQLSpecExecFactory) ConstructWindow(
func (e *distSQLSpecExecFactory) ConstructRenameColumns(
input exec.Node, colNames []string,
) (exec.Node, error) {
var inputCols sqlbase.ResultColumns
// distSQLSpecExecFactory still constructs some of the planNodes (for
// example, some variants of EXPLAIN), and we need to be able to rename
// the columns on them.
switch plan := input.(type) {
case planMaybePhysical:
inputCols = plan.physPlan.ResultColumns
case planNode:
inputCols = planMutableColumns(plan)
default:
panic("unexpected node")
}
inputCols := input.(planMaybePhysical).physPlan.ResultColumns
for i := range inputCols {
inputCols[i].Name = colNames[i]
}
Expand All @@ -747,7 +730,31 @@ func (e *distSQLSpecExecFactory) ConstructExplain(
// variants of EXPLAIN when subqueries are present as we do in the old path.
// TODO(yuzefovich): make sure that local plan nodes that create
// distributed jobs are shown as "distributed". See distSQLExplainable.
return constructExplainPlanNode(options, stmtType, plan.(*planTop), e.planner)
p := plan.(*planTop)
explain, err := constructExplainPlanNode(options, stmtType, p, e.planner)
if err != nil {
return nil, err
}
explainNode := explain.(planNode)
if _, isExplainPlan := explainNode.(*explainPlanNode); isExplainPlan {
return nil, unimplemented.NewWithIssue(47473, "experimental opt-driven distsql planning: explain (plan)")
}
physPlan, err := e.dsp.wrapPlan(e.getPlanCtx(cannotDistribute), explainNode)
if err != nil {
return nil, err
}
physPlan.ResultColumns = planColumns(explainNode)
// Plan distribution of an explain node is considered to be the same as of
// the query being explained.
// TODO(yuzefovich): we might also need to look at the distribution of
// subqueries and postqueries.
physPlan.Distribution = p.main.physPlan.Distribution
return planMaybePhysical{
physPlan: &physicalPlanTop{
PhysicalPlan: physPlan,
planNodesToClose: []planNode{explainNode},
},
}, nil
}

func (e *distSQLSpecExecFactory) ConstructShowTrace(
Expand Down
66 changes: 3 additions & 63 deletions pkg/sql/exec_factory_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/errors"
)

Expand Down Expand Up @@ -128,9 +127,8 @@ func constructExplainPlanNode(

case tree.ExplainVec:
return &explainVecNode{
options: options,
plan: p.main,
subqueryPlans: p.subqueryPlans,
options: options,
plan: p.planComponents,
}, nil

case tree.ExplainPlan:
Expand All @@ -140,7 +138,7 @@ func constructExplainPlanNode(
return planner.makeExplainPlanNodeWithPlan(
context.TODO(),
options,
&p.planComponents,
p.planComponents,
)

default:
Expand Down Expand Up @@ -235,64 +233,6 @@ func convertOrdinalsToInts(ordinals []exec.NodeColumnOrdinal) []int {
return ints
}

func constructSimpleProjectForPlanNode(
n planNode, cols []exec.NodeColumnOrdinal, colNames []string, reqOrdering exec.OutputOrdering,
) (exec.Node, error) {
// If the top node is already a renderNode, just rearrange the columns. But
// we don't want to duplicate a rendering expression (in case it is expensive
// to compute or has side-effects); so if we have duplicates we avoid this
// optimization (and add a new renderNode).
if r, ok := n.(*renderNode); ok && !hasDuplicates(cols) {
oldCols, oldRenders := r.columns, r.render
r.columns = make(sqlbase.ResultColumns, len(cols))
r.render = make([]tree.TypedExpr, len(cols))
for i, ord := range cols {
r.columns[i] = oldCols[ord]
if colNames != nil {
r.columns[i].Name = colNames[i]
}
r.render[i] = oldRenders[ord]
}
r.reqOrdering = ReqOrdering(reqOrdering)
return r, nil
}
var inputCols sqlbase.ResultColumns
if colNames == nil {
// We will need the names of the input columns.
inputCols = planColumns(n.(planNode))
}

var rb renderBuilder
rb.init(n, reqOrdering)

exprs := make(tree.TypedExprs, len(cols))
for i, col := range cols {
exprs[i] = rb.r.ivarHelper.IndexedVar(int(col))
}
var resultTypes []*types.T
if colNames != nil {
// We will need updated result types.
resultTypes = make([]*types.T, len(cols))
for i := range exprs {
resultTypes[i] = exprs[i].ResolvedType()
}
}
resultCols := getResultColumnsForSimpleProject(cols, colNames, resultTypes, inputCols)
rb.setOutput(exprs, resultCols)
return rb.res, nil
}

func hasDuplicates(cols []exec.NodeColumnOrdinal) bool {
var set util.FastIntSet
for _, c := range cols {
if set.Contains(int(c)) {
return true
}
set.Add(int(c))
}
return false
}

func constructVirtualScan(
ef exec.Factory,
p *planner,
Expand Down
14 changes: 0 additions & 14 deletions pkg/sql/explain_distsql.go
Original file line number Diff line number Diff line change
Expand Up @@ -79,20 +79,6 @@ func getPlanDistributionForExplainPurposes(
if plan.isPhysicalPlan() {
return plan.physPlan.Distribution
}
switch p := plan.planNode.(type) {
case *explainDistSQLNode:
if p.plan.main.isPhysicalPlan() {
return p.plan.main.physPlan.Distribution
}
case *explainVecNode:
if p.plan.isPhysicalPlan() {
return p.plan.physPlan.Distribution
}
case *explainPlanNode:
if p.plan.main.isPhysicalPlan() {
return p.plan.main.physPlan.Distribution
}
}
if _, ok := plan.planNode.(distSQLExplainable); ok {
// This is a special case for plans that will be actually distributed
// but are represented using local plan nodes (for example, "create
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/explain_plan.go
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,7 @@ type explainPlanNode struct {
// makeExplainPlanNodeWithPlan instantiates a planNode that EXPLAINs an
// underlying plan.
func (p *planner) makeExplainPlanNodeWithPlan(
ctx context.Context, opts *tree.ExplainOptions, plan *planComponents,
ctx context.Context, opts *tree.ExplainOptions, plan planComponents,
) (planNode, error) {
flags := explain.MakeFlags(opts)

Expand All @@ -59,7 +59,7 @@ func (p *planner) makeExplainPlanNodeWithPlan(
columns = append(sqlbase.ResultColumns(nil), columns...)

node := &explainPlanNode{
plan: *plan,
plan: plan,
run: explainPlanRun{
results: p.newContainerValuesNode(columns, 0),
},
Expand Down
13 changes: 6 additions & 7 deletions pkg/sql/explain_vec.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,14 +35,13 @@ type explainVecNode struct {
optColumnsSlot

options *tree.ExplainOptions
plan planMaybePhysical
plan planComponents

run struct {
lines []string
// The current row returned by the node.
values tree.Datums
}
subqueryPlans []subquery
}

type flowWithNode struct {
Expand All @@ -55,17 +54,17 @@ func (n *explainVecNode) startExec(params runParams) error {
distSQLPlanner := params.extendedEvalCtx.DistSQLPlanner
distribution := getPlanDistributionForExplainPurposes(
params.ctx, params.p, params.extendedEvalCtx.ExecCfg.NodeID,
params.extendedEvalCtx.SessionData.DistSQLMode, n.plan,
params.extendedEvalCtx.SessionData.DistSQLMode, n.plan.main,
)
willDistribute := distribution.WillDistribute()
outerSubqueries := params.p.curPlan.subqueryPlans
planCtx := newPlanningCtxForExplainPurposes(distSQLPlanner, params, n.subqueryPlans, distribution)
planCtx := newPlanningCtxForExplainPurposes(distSQLPlanner, params, n.plan.subqueryPlans, distribution)
defer func() {
planCtx.planner.curPlan.subqueryPlans = outerSubqueries
}()
physPlan, err := newPhysPlanForExplainPurposes(planCtx, distSQLPlanner, n.plan)
physPlan, err := newPhysPlanForExplainPurposes(planCtx, distSQLPlanner, n.plan.main)
if err != nil {
if len(n.subqueryPlans) > 0 {
if len(n.plan.subqueryPlans) > 0 {
return errors.New("running EXPLAIN (VEC) on this query is " +
"unsupported because of the presence of subqueries")
}
Expand Down Expand Up @@ -202,5 +201,5 @@ func (n *explainVecNode) Next(runParams) (bool, error) {

func (n *explainVecNode) Values() tree.Datums { return n.run.values }
func (n *explainVecNode) Close(ctx context.Context) {
n.plan.Close(ctx)
n.plan.close(ctx)
}
Original file line number Diff line number Diff line change
Expand Up @@ -44,17 +44,10 @@ NULL /1 {5} 5
statement ok
SET experimental_distsql_planning = always

query TTT
EXPLAIN SELECT * FROM kv
----
· distribution full
· vectorized true


query BT
EXPLAIN (DISTSQL) SELECT * FROM kv
query T
SELECT url FROM [EXPLAIN (DISTSQL) SELECT * FROM kv]
----
true https://cockroachdb.github.io/distsqlplan/decode.html#eJyk0s1K80AYBeD9dxXhrD5lQv5dZKVohUBta9OFIFnEzEsJTTNxJilKyb1LJ4vaUnW0y_k588wLZwv1WiHG6Gk2vkkm1v-7JF2kj-MLKx2NR7cL69K6n08frNUGDLXgNMnXpBA_wwODD4YADCEYImQMjRQFKSXk7spWBxL-hthlKOuma3fbGUMhJCHeoi3bihBjkb9UNKeck3RcMHBq87LSzGpz3chynct3MKRNXqvYsh0fWc8gunb_pmrzJSH2evaFu-e6WkhOkvgBlfUnfjYRtmic6Ojiado_oD3zkb2fR3Z82wkMh_bNZd9ADmwnNJQDczkwkEPbiQzl0FwODeTI_kPBTrhzUo2oFRn1x90VkPiShrYq0cmCZlIUmhmWU53TG5xUO5x6wyKp9ZH-4Oew92346iDsHof9c-TgnHB4Tjj6VTjr_30EAAD__2R3o50=
https://cockroachdb.github.io/distsqlplan/decode.html#eJyk0s9q4zAQBvD7PoX5TruLjP_vwactbQqGNEnjHArBB9cagoljuZIdWoLfvUQupAlpqzZHSfPpNwOzg3qqECMdjUfXC6uTlXU7n95Zy9HDbHyVTKzfN0m6SO_Hf6y3kr9DwXqbgaEWnCb5hhTiJTww-GAIwBCCIULG0EhRkFJC7kt2OpDwZ8QuQ1k3Xbu_zhgKIQnxDm3ZVoQYi_yxojnlnKTjgoFTm5eVZtbb_40sN7l8AUPa5LWKLdvxkfUMomsPf6o2XxFir2cfuAeuq4XkJIkfUVl_prOJsEXjRCeF52n_iPbMR_a-HtnxbScwHNo3l30DObCd0FAOzOXAQA5tJzKUQ3M5NJAj-wcLdsadk2pErchof9z9AhJf0bCtSnSyoJkUhWaG41Tn9AUn1Q6v3nBIav2kG3wf9j4N_zsKu6dh_xI5uCQcXhKOvhXO-l-vAQAA__-hwaj8

# Note that we want to test DistSQL physical planning and the obvious choice
# would be to use EXPLAIN (DISTSQL). However, this explain variant doesn't have
Expand Down Expand Up @@ -136,12 +129,6 @@ EXPLAIN (VEC) SELECT k::REGCLASS FROM kv
└ *colrpc.Outbox
└ *colfetcher.ColBatchScan

query TTT
EXPLAIN SELECT k::REGCLASS FROM kv
----
· distribution partial
· vectorized true

# Check that hash join is supported by the new factory.
query II rowsort
SELECT kv.k, v FROM kv, kw WHERE v = w
Expand Down
6 changes: 3 additions & 3 deletions pkg/sql/logictest/testdata/logic_test/explain_analyze_plans
Original file line number Diff line number Diff line change
Expand Up @@ -82,10 +82,10 @@ statement ok
EXPLAIN ANALYZE (DISTSQL) EXPLAIN SELECT 1

# This query verifies support for zeroNode in DistSQL.
query B
SELECT automatic FROM [EXPLAIN (DISTSQL) SELECT sum(k) FROM kv WHERE FALSE]
query T
SELECT url FROM [EXPLAIN (DISTSQL) SELECT sum(k) FROM kv WHERE FALSE]
----
true
https://cockroachdb.github.io/distsqlplan/decode.html#eJyUkF9LwzAUxd_9FOE-dRBY42OenFqxsH-u8w-MPIT2WseyZuYmIpR-d2kj6ISJPt5zzzm_5LZArwYkFNk0u1qz4Ay7WS1mbJM9LaeTfM6S67xYF3fTEfu0UNgnu1F07d7Y4222ytizNoQKODS2wrneI4HcgADF4eBsiUTW9VI7GPLqHWTKYdscgu9lxaG0DkG24LfeIEh40CYgjVPgUKHXWzM0puySJSkrX0KzoxGojoMN_quFvK4RZNrxv5Mmde2w1t66sTimFfez5EKcxoj_YFZIB9sQHiFOf0BxwKrGeDSywZW4dLYcMHFcDLlBqJB83Io45E1c9Q_8Hha_hs9_hFV39hEAAP__TsavHw==

# This query verifies stat collection for the tableReader and windower.
query T
Expand Down
55 changes: 54 additions & 1 deletion pkg/sql/opt_exec_factory.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/span"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/encoding"
"github.com/cockroachdb/errors"
)
Expand Down Expand Up @@ -194,7 +195,59 @@ func (ef *execFactory) ConstructInvertedFilter(
func (ef *execFactory) ConstructSimpleProject(
n exec.Node, cols []exec.NodeColumnOrdinal, colNames []string, reqOrdering exec.OutputOrdering,
) (exec.Node, error) {
return constructSimpleProjectForPlanNode(n.(planNode), cols, colNames, reqOrdering)
// If the top node is already a renderNode, just rearrange the columns. But
// we don't want to duplicate a rendering expression (in case it is expensive
// to compute or has side-effects); so if we have duplicates we avoid this
// optimization (and add a new renderNode).
if r, ok := n.(*renderNode); ok && !hasDuplicates(cols) {
oldCols, oldRenders := r.columns, r.render
r.columns = make(sqlbase.ResultColumns, len(cols))
r.render = make([]tree.TypedExpr, len(cols))
for i, ord := range cols {
r.columns[i] = oldCols[ord]
if colNames != nil {
r.columns[i].Name = colNames[i]
}
r.render[i] = oldRenders[ord]
}
r.reqOrdering = ReqOrdering(reqOrdering)
return r, nil
}
var inputCols sqlbase.ResultColumns
if colNames == nil {
// We will need the names of the input columns.
inputCols = planColumns(n.(planNode))
}

var rb renderBuilder
rb.init(n, reqOrdering)

exprs := make(tree.TypedExprs, len(cols))
for i, col := range cols {
exprs[i] = rb.r.ivarHelper.IndexedVar(int(col))
}
var resultTypes []*types.T
if colNames != nil {
// We will need updated result types.
resultTypes = make([]*types.T, len(cols))
for i := range exprs {
resultTypes[i] = exprs[i].ResolvedType()
}
}
resultCols := getResultColumnsForSimpleProject(cols, colNames, resultTypes, inputCols)
rb.setOutput(exprs, resultCols)
return rb.res, nil
}

func hasDuplicates(cols []exec.NodeColumnOrdinal) bool {
var set util.FastIntSet
for _, c := range cols {
if set.Contains(int(c)) {
return true
}
set.Add(int(c))
}
return false
}

// ConstructRender is part of the exec.Factory interface.
Expand Down
Loading

0 comments on commit 08090b6

Please sign in to comment.