Skip to content

distsql: partial planNode tree wrapping#28509

Merged
craig[bot] merged 3 commits intocockroachdb:masterfrom
jordanlewis:partial-wrap
Aug 16, 2018
Merged

distsql: partial planNode tree wrapping#28509
craig[bot] merged 3 commits intocockroachdb:masterfrom
jordanlewis:partial-wrap

Conversation

@jordanlewis
Copy link
Copy Markdown
Member

@jordanlewis jordanlewis commented Aug 12, 2018

Previously, when DistSQL didn't support a planNode, it would wrap the
entire tree starting from that planNode, leaving potentially
DistSQL-enabled planNodes in that tree to execute via the local
execution pathway.

This commit changes that behavior. When wrapping a planNode, the planner
now searches the planNode tree for all DistSQL-enabled children, resumes
DistSQL planning at those points, and hooks the result up to the wrapped
subtree.

In addition, processors that can be used as sources to mutations now need to optionally be able to read columns that have an active schema change. That information is now plumbed through the system.

Release note: None

@jordanlewis jordanlewis requested review from a team, asubiotto, rjnn and solongordon August 12, 2018 17:14
@cockroach-teamcity
Copy link
Copy Markdown
Member

This change is Reviewable

@jordanlewis jordanlewis force-pushed the partial-wrap branch 4 times, most recently from f74c354 to 6ac7d47 Compare August 13, 2018 05:55
Copy link
Copy Markdown
Contributor

@asubiotto asubiotto left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Great stuff. @knz, could you take a look at walk.go if you have a moment?

Reviewed 9 of 10 files at r2.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained


pkg/sql/distsql_physical_planner.go, line 2404 at r2 (raw file):

		},
	})
	if nParents > 1 {

Is this ever > 1? It seems to me that you return false after incrementing the first time which would result in the planObserver stopping, right?


pkg/sql/distsql_physical_planner.go, line 2405 at r2 (raw file):

	})
	if nParents > 1 {
		return PhysicalPlan{}, errors.Errorf("Can't wrap plan %v %T with more than one input", n, n)

s/Can't/can't


pkg/sql/distsql_physical_planner.go, line 2457 at r2 (raw file):

	pIdx := p.AddProcessor(proc)
	p.ResultTypes = wrapper.outputTypes
	p.PlanToStreamColMap = identityMapInPlace(make([]int, len(p.ResultTypes)))

Please add a comment here, I'm not quite sure what's going on.


pkg/sql/plan_node_to_row_source.go, line 131 at r2 (raw file):

		p.node.Close(p.params.ctx)
		p.running = false
		p.started = true

Why is p.started being set here?


pkg/sql/row_source_to_plan_node.go, line 78 at r2 (raw file):

				return false, p.Err
			}
			if p.TraceData != nil {

Should we get rid of this if statement and hit the error below? I think I prefer that to silently skipping over trace data.


pkg/sql/distsqlrun/flow_diagram.go, line 337 at r2 (raw file):

// summary implements the diagramCellType interface.
func (r *LocalPlanNodeSpec) summary() (string, []string) {
	return fmt.Sprintf("local %s %d", *r.Name, *r.RowSourceIdx), []string{}

😄

Copy link
Copy Markdown
Member Author

@jordanlewis jordanlewis left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained


pkg/sql/distsql_physical_planner.go, line 2404 at r2 (raw file):

Previously, asubiotto (Alfonso Subiotto Marqués) wrote…

Is this ever > 1? It seems to me that you return false after incrementing the first time which would result in the planObserver stopping, right?

Yeah, it can be greater than one. Returning false will prevent further recursion into the current node you're looking at - but the parent of that node could be a node with multiple children.

This error shouldn't ever happen in practice, as all nodes with multiple chilrden (joins) are plannable via distsql.


pkg/sql/plan_node_to_row_source.go, line 131 at r2 (raw file):

Previously, asubiotto (Alfonso Subiotto Marqués) wrote…

Why is p.started being set here?

If you close this node before it started, it needs to be marked as started so that it doesn't do its startExec initialization, which will read values from its source.


pkg/sql/row_source_to_plan_node.go, line 78 at r2 (raw file):

Previously, asubiotto (Alfonso Subiotto Marqués) wrote…

Should we get rid of this if statement and hit the error below? I think I prefer that to silently skipping over trace data.

Oh yeah we can fix this now that we have the metadata sink. Will do.


pkg/sql/distsqlrun/flow_diagram.go, line 337 at r2 (raw file):

Previously, asubiotto (Alfonso Subiotto Marqués) wrote…

😄

Yeah! It's still not perfect though - if we wrap more than one node at time, you only get the name of the top node... which leaves out a lot of info.

@jordanlewis jordanlewis requested a review from a team August 13, 2018 23:50
Copy link
Copy Markdown
Member Author

@jordanlewis jordanlewis left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Had to rebase this on top of #28551, since nodes potentially expect that the TableReader returns non-public columns now.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained


pkg/sql/distsql_physical_planner.go, line 2405 at r2 (raw file):

Previously, asubiotto (Alfonso Subiotto Marqués) wrote…

s/Can't/can't

Done.


pkg/sql/distsql_physical_planner.go, line 2457 at r2 (raw file):

Previously, asubiotto (Alfonso Subiotto Marqués) wrote…

Please add a comment here, I'm not quite sure what's going on.

Done.


pkg/sql/row_source_to_plan_node.go, line 78 at r2 (raw file):

Previously, jordanlewis (Jordan Lewis) wrote…

Oh yeah we can fix this now that we have the metadata sink. Will do.

Done.

Copy link
Copy Markdown
Contributor

@knz knz left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

so yeah the walk thing doesn't work well for me. I have made a suggestion for an alternative see below.

Reviewed 2 of 6 files at r1, 12 of 13 files at r3, 10 of 10 files at r4, 10 of 10 files at r5.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained


pkg/sql/distsql_physical_planner.go, line 807 at r4 (raw file):

	n *scanNode, evalCtx *tree.EvalContext, indexVarMap []int,
) (distsqlrun.TableReaderSpec, distsqlrun.PostProcessSpec, error) {
	var visibility distsqlrun.ScanVisibility

This function would benefit from being either split up in sub-function or more extensively commented in-line to provide some human insight about the different things happening and why they are important.

Especially this here.


pkg/sql/walk.go, line 39 at r5 (raw file):

	// stop the recursion, and isn't invoked on planNodes that are stored in
	// their parents as concrete types.
	replaceNode func(ctx context.Context, nodeName string, plan *planNode) (bool, error)
replaceNode func(ctx context.Context, nodeName string, plan planNode) (recurse bool, newPlan planNode, err  error)

Also if I'm reading the rest of this patch right, the recurse bool is never used so perhaps you can remove it.


pkg/sql/walk.go, line 79 at r5 (raw file):

// visit is the recursive function that supports walkPlan().
func (v *planVisitor) visit(planPtr *planNode) {

visit(plan planNode)


pkg/sql/walk.go, line 88 at r5 (raw file):

	recurse := true
	if v.observer.replaceNode != nil {
		recurse, v.err = v.observer.replaceNode(v.ctx, name, planPtr)
recurse, newPlan, err := v.observer.replaceNode(plan)
if err != nil {
    v.err = err
    return plan
}
v.visitInternal(newPlan, ...)
if v.err != nil {
   return plan
}
return newPlan

pkg/sql/walk.go, line 191 at r5 (raw file):

			v.expr(name, "filter", -1, n.filter)
		}
		v.visit(&n.source.plan)

n.source.plan = v.visit(n.source.plan)

here and below


pkg/sql/distsqlrun/tablereader.go, line 69 at r4 (raw file):

	tr.limitHint = limitHint(spec.LimitHint, post)

	numCols := len(spec.Table.Columns)

ditto comments

Copy link
Copy Markdown
Member Author

@jordanlewis jordanlewis left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Great suggestion. I took it.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained


pkg/sql/distsql_physical_planner.go, line 807 at r4 (raw file):

Previously, knz (kena) wrote…

This function would benefit from being either split up in sub-function or more extensively commented in-line to provide some human insight about the different things happening and why they are important.

Especially this here.

Moved this elsewhere.


pkg/sql/row_source_to_plan_node.go, line 78 at r2 (raw file):

Previously, jordanlewis (Jordan Lewis) wrote…

Done.

Er, actually, we need this because of the Distinct local implementation. Once that's erased, we can erase this too. This doesn't swallow tracing data in normal wrapping operaiton.


pkg/sql/walk.go, line 39 at r5 (raw file):

Previously, knz (kena) wrote…
replaceNode func(ctx context.Context, nodeName string, plan planNode) (recurse bool, newPlan planNode, err  error)

Also if I'm reading the rest of this patch right, the recurse bool is never used so perhaps you can remove it.

Done.


pkg/sql/walk.go, line 79 at r5 (raw file):

Previously, knz (kena) wrote…

visit(plan planNode)

Done.


pkg/sql/walk.go, line 88 at r5 (raw file):

Previously, knz (kena) wrote…
recurse, newPlan, err := v.observer.replaceNode(plan)
if err != nil {
    v.err = err
    return plan
}
v.visitInternal(newPlan, ...)
if v.err != nil {
   return plan
}
return newPlan

Done.


pkg/sql/walk.go, line 191 at r5 (raw file):

Previously, knz (kena) wrote…

n.source.plan = v.visit(n.source.plan)

here and below

Done.

@jordanlewis jordanlewis force-pushed the partial-wrap branch 4 times, most recently from c3165ad to d21dc6b Compare August 15, 2018 05:10
@jordanlewis
Copy link
Copy Markdown
Member Author

PTAL.
@solongordon, turns out index_join needed to understand about mutation columns too, so I plumbed that through.

Copy link
Copy Markdown
Contributor

@knz knz left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

:lgtm: for what I reviewed earlier. thanks

Reviewed 5 of 17 files at r6, 1 of 2 files at r7, 10 of 10 files at r8.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (and 1 stale)


pkg/sql/walk.go, line 92 at r8 (raw file):

		if newNode != nil {
			return newNode
		}

In your first version you were recursing into the new node. Now you aren't. Is that intentional?

Copy link
Copy Markdown
Contributor

@solongordon solongordon left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained


pkg/sql/distsqlrun/processors.proto, line 279 at r8 (raw file):

  optional sqlbase.JoinType type = 6 [(gogoproto.nullable) = false];

  optional ScanVisibility visibility = 7 [(gogoproto.nullable) = false];

Feels a little funny that this new field is ignored by joinReader. Would it be worth adding an assertion in newJoinReader that this is PUBLIC?

Copy link
Copy Markdown
Contributor

@solongordon solongordon left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained


pkg/sql/sqlbase/structured.go, line 1818 at r8 (raw file):

// ColumnIdxMap returns a map from Column ID to the ordinal position of that
// column.
func (desc *TableDescriptor) ColumnIdxMap() map[ColumnID]int {

Minor: Could just call ColumnIdxMapWithMutations(false) to avoid redundant code. Or just make this function take the mutations argument and ditch the other.


pkg/sql/sqlbase/structured.go, line 2126 at r8 (raw file):

// ColumnTypes returns the types of all columns.
func (desc *TableDescriptor) ColumnTypes() []ColumnType {

Ditto above.

Copy link
Copy Markdown
Member Author

@jordanlewis jordanlewis left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (and 1 stale)


pkg/sql/walk.go, line 92 at r8 (raw file):

Previously, knz (kena) wrote…

In your first version you were recursing into the new node. Now you aren't. Is that intentional?

No, i wasn't recursing into the new node - at least not intentionally. We returned false when changing the node.


pkg/sql/distsqlrun/processors.proto, line 279 at r8 (raw file):

Previously, solongordon (Solon) wrote…

Feels a little funny that this new field is ignored by joinReader. Would it be worth adding an assertion in newJoinReader that this is PUBLIC?

Done.


pkg/sql/sqlbase/structured.go, line 2126 at r8 (raw file):

Previously, solongordon (Solon) wrote…

Ditto above.

Done.

@jordanlewis
Copy link
Copy Markdown
Member Author

TFTRs!

bors r+

@craig
Copy link
Copy Markdown
Contributor

craig bot commented Aug 15, 2018

Build failed

@jordanlewis
Copy link
Copy Markdown
Member Author

Flake

bors r+

@craig
Copy link
Copy Markdown
Contributor

craig bot commented Aug 15, 2018

Merge conflict (retrying...)

jordanlewis and others added 3 commits August 15, 2018 21:52
Previously, TableReaders could never access mutation columns - columns
that aren't yet marked public as part of the schema change process. This
is problematic for a future where UPDATE gets its input from a
TableReader and not a scanNode - the TableReader must be taught about
non-public columns.

This commit adds a new boolean field to the TableReaderSpec that should
be set to true to return non-public columns. When set to true, the
TableReader returns all columns, including the mutations.

Release note: None
Previously, when DistSQL didn't support a planNode, it would wrap the
entire tree starting from that planNode, leaving potentially
DistSQL-enabled planNodes in that tree to execute via the local
execution pathway.

This commit changes that behavior. When wrapping a planNode, the planner
now searches the planNode tree for all DistSQL-enabled children, resumes
DistSQL planning at those points, and hooks the result up to the wrapped
subtree.

Release note: None
@jordanlewis
Copy link
Copy Markdown
Member Author

bors r+

craig bot pushed a commit that referenced this pull request Aug 16, 2018
28509: distsql: partial planNode tree wrapping r=jordanlewis a=jordanlewis

Previously, when DistSQL didn't support a planNode, it would wrap the
entire tree starting from that planNode, leaving potentially
DistSQL-enabled planNodes in that tree to execute via the local
execution pathway.

This commit changes that behavior. When wrapping a planNode, the planner
now searches the planNode tree for all DistSQL-enabled children, resumes
DistSQL planning at those points, and hooks the result up to the wrapped
subtree.

In addition, processors that can be used as sources to mutations now need to optionally be able to read columns that have an active schema change. That information is now plumbed through the system.

Release note: None

Co-authored-by: Jordan Lewis <jordanthelewis@gmail.com>
Co-authored-by: Solon Gordon <solon@cockroachlabs.com>
@craig
Copy link
Copy Markdown
Contributor

craig bot commented Aug 16, 2018

Build succeeded

@craig craig bot merged commit 87d65e1 into cockroachdb:master Aug 16, 2018
@jordanlewis jordanlewis deleted the partial-wrap branch August 17, 2018 16:01
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants