Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

sql: implement filter propagation. #12617

Merged
merged 1 commit into from Jan 12, 2017
Merged

sql: implement filter propagation. #12617

merged 1 commit into from Jan 12, 2017

Conversation

knz
Copy link
Contributor

@knz knz commented Dec 31, 2016

tl;dr: this patch implements a primitive form of filter propagation:
queries of the form SELECT * FROM a, b WHERE a.x = 10 and b.y = 20
are transformed to SELECT * FROM (SELECT * FROM a WHERE x = 10), (SELECT * FROM b WHERE y = 20).

Long explanation: go take a book on relational algebra. Filtering is
commutative with a bunch of stuff.

Note: this PR is based off the commits from #12616. I will rebase when #12616 is merged. Only the last two commits are specific to this PR.

Fixes #8566.
Fixes #10632.
Fixes #10633.
Fixes #11192.
Fixes #11723.


This change is Reviewable

@knz
Copy link
Contributor Author

knz commented Dec 31, 2016

cc @nvanbenschoten @petermattis @cuongdo. Just in time for Q4 2016!

@petermattis
Copy link
Collaborator

Nice! A quick scan of this code reveals that I've almost completely forgotten the sql code base.

@knz knz force-pushed the opt-filters2 branch 4 times, most recently from 7e5ce13 to e9b61e3 Compare January 2, 2017 19:26
@jordanlewis
Copy link
Member

generally :lgtm: although I don't have a full understanding of everything that's going on here.


Reviewed 2 of 2 files at r1, 34 of 34 files at r2, 8 of 8 files at r3.
Review status: all files reviewed at latest revision, 8 unresolved discussions, some commit checks failed.


pkg/sql/filter_opt.go, line 68 at r3 (raw file):

		}
		// TODO(knz) We could evaluate the filter here and set/reset
		// n.results accordingly.

This seems like an easy win - make a todo issue?


pkg/sql/filter_opt.go, line 93 at r3 (raw file):

	case *scanNode:
		n.filter = n.filterVars.Rebind(
			conjToExpr(mergeConj(exprToConj(n.filter), extraFilter)))

This smells a bit inefficient - how bad would it be to implement a mergeConjExprs directly? If that's too annoying I think this is fine.


pkg/sql/filter_opt.go, line 295 at r3 (raw file):

	if extraFilter != nil {
		// The filter that's being added refer to the rendered expressions,

grammar - you might mean refers?


pkg/sql/filter_opt.go, line 312 at r3 (raw file):

				renderExpr := s.render[iv.Idx]
				if d, ok := renderExpr.(parser.Datum); ok {
					// A simple datum is not complex, so it can be propagated further.

Maybe remove simple here? A Datum is not complex, so it can be propagated further.


pkg/sql/filter_opt.go, line 483 at r3 (raw file):

}

func shiftConj(ivarHelper parser.IndexedVarHelper, filter conjExpr, offset int) conjExpr {

I suggest adding a comment explaining the inputs and output of this function.


pkg/sql/filter_opt.go, line 555 at r3 (raw file):

// conjToExpr converts a conjunction of the form `[a, b, c]` to an
// expression of the form `a AND b AND c`.
func conjToExpr(filter conjExpr) parser.TypedExpr {

These following three functions seem like great candidates for some quick unit testing!


pkg/sql/filter_opt.go, line 570 at r3 (raw file):

// conjunction `[a, b, c]`.
func exprToConj(filter parser.TypedExpr) conjExpr {
	if filter == nil {

Tests?


pkg/sql/filter_opt.go, line 584 at r3 (raw file):

// that the union of [a, b] and [c, a] is [a, b, c]
// and not [a, b, c, a].
func mergeConj(left, right conjExpr) conjExpr {

Tests?


Comments from Reviewable

@knz
Copy link
Contributor Author

knz commented Jan 3, 2017

TFYR


Review status: 6 of 9 files reviewed at latest revision, 8 unresolved discussions.


pkg/sql/filter_opt.go, line 68 at r3 (raw file):

Previously, jordanlewis (Jordan Lewis) wrote…

This seems like an easy win - make a todo issue?

Done - added to #12618.


pkg/sql/filter_opt.go, line 93 at r3 (raw file):

Previously, jordanlewis (Jordan Lewis) wrote…

This smells a bit inefficient - how bad would it be to implement a mergeConjExprs directly? If that's too annoying I think this is fine.

It's bad -- exprToConj is a non-trivial recursion (it flattens an arbitrarily deep tree of ANDs into a single thing) so a mergeConjExprs would need do the same, there wouldn't be any win here.


pkg/sql/filter_opt.go, line 295 at r3 (raw file):

Previously, jordanlewis (Jordan Lewis) wrote…

grammar - you might mean refers?

Done.


pkg/sql/filter_opt.go, line 312 at r3 (raw file):

Previously, jordanlewis (Jordan Lewis) wrote…

Maybe remove simple here? A Datum is not complex, so it can be propagated further.

Done.


pkg/sql/filter_opt.go, line 483 at r3 (raw file):

Previously, jordanlewis (Jordan Lewis) wrote…

I suggest adding a comment explaining the inputs and output of this function.

Done.


pkg/sql/filter_opt.go, line 555 at r3 (raw file):

Previously, jordanlewis (Jordan Lewis) wrote…

These following three functions seem like great candidates for some quick unit testing!

Done.


pkg/sql/filter_opt.go, line 570 at r3 (raw file):

Previously, jordanlewis (Jordan Lewis) wrote…

Tests?

Done.


pkg/sql/filter_opt.go, line 584 at r3 (raw file):

Previously, jordanlewis (Jordan Lewis) wrote…

Tests?

Done.


Comments from Reviewable

@knz
Copy link
Contributor Author

knz commented Jan 4, 2017

@andreimatei any chance you could review this while radu is away? this way we could make this week's beta.

@jordanlewis
Copy link
Member

Reviewed 6 of 6 files at r4.
Review status: all files reviewed at latest revision, all discussions resolved, some commit checks pending.


Comments from Reviewable

@andreimatei
Copy link
Contributor

Review status: 7 of 32 files reviewed at latest revision, 16 unresolved discussions, all commit checks successful.


pkg/sql/filter_opt.go, line 27 at r4 (raw file):

// optimizeFilters performs filter propagation on the given plan.
func (p *planner) optimizeFilters(plan planNode) (planNode, error) {

how about we get rid of this function so that we don't have to explain the 2-way interaction between it and addFilter (which I found confusing)? Or rename it to fooWrapper, where foo is what we end up calling addFilter.


pkg/sql/filter_opt.go, line 43 at r4 (raw file):

// addFilter is the recursive algorithm that supports optimizeFilters().
//
// Its function is to try to add the supplied extra filter to the

how about we call this pushDownFilter and explain that it does two things:

  1. it asks each node to push down its own filter
  2. it can optionally receive an extraFilter from the parent, which is to be combined with the node's own filter. In case this is passed, part of it might be returned as a rejected expression.

Otherwise, the calls to optimizeFilter(), which have nothing to do with the filter being added, are confusing.


pkg/sql/filter_opt.go, line 51 at r4 (raw file):

// predicates that were not absorbed.
//
// If the autoWrap argument is set to true, a new filterNode is created

This sounds to me like:

function f(doSmthElse bool) {
  if !doSmthElse {
    doSmth();
  } else {
    doSmthElse();
  }
}

:P
How about we add another optional layer that does the wrapping as a separate function?


pkg/sql/filter_opt.go, line 54 at r4 (raw file):

// to wrap the plan and filter using the remaining predicates.
// Otherwise, the remaining filter is returned.
func (p *planner) addFilter(

should we start listing such methods near the planNode interface, as they'll need to be augmented for every new node type?


pkg/sql/filter_opt.go, line 67 at r4 (raw file):

			return plan, nil, nil
		}
		// TODO(knz) We could evaluate the filter here and set/reset

only if the filter is "pure", right?


pkg/sql/filter_opt.go, line 70 at r4 (raw file):

		// n.results accordingly.

		// Fallthrough: wrap or spill.

nit: "fallthrough" in the context of a switch means something else. Perhaps put a comment above the switch saying that sometimes the cases return, sometimes they don't.


pkg/sql/filter_opt.go, line 81 at r4 (raw file):

		}
		if len(remainingFilter) > 0 {
			n.ivarHelper.Reset()

mind documenting what this is about? As you've told me, it's about interacting with the "needed cols" infrastructure which needs to know that some columns might not be needed any more, if we got rid of parts of the filter. It might be worth mentioning that in the ivarHelper docs too. Or maybe it should be an argument to rebind() - whether or not to reset?


pkg/sql/filter_opt.go, line 92 at r4 (raw file):

	case *scanNode:
		n.filter = n.filterVars.Rebind(

explain that the filter is guaranteed to be... "simple" and fully acceptable by the scanNode (guaranteed by addRenderFilter(). Maybe worth a sanity check too.

So here we don't need n.filterVars.Reset() because we're only adding new indexVars, not removing any?


pkg/sql/filter_opt.go, line 122 at r4 (raw file):

		// late to be smart at this point. We probably shouldn't be here
		// anyway: addFilter() should be run before expandPlan() which
		// does index selection, not after. Before expandPlan(), there is

as we discussed, let's just panic in this unexpected case


pkg/sql/filter_opt.go, line 156 at r4 (raw file):

	case *selectTopNode:
		if n.limit == nil && n.window == nil && n.group == nil {
			// We only know how to propagate the filter if there is no

can we propagate the FILTER from an aggregation if it's common to all aggregations (or more realistically if there's a single aggregation)? :B Maybe leave a TODO.


pkg/sql/filter_opt.go, line 298 at r4 (raw file):

		// expressions, not the select's source node.

		// We want to propagate only those filters that use simple renders

why is that? What if we have a selectNode on top of another selectNode? Shouldn't we support propagating more complex filters? How about we always try to push down everything and we make nodes responsible of accepting only what they can support instead of deciding what their children can support? Does that make sense?


pkg/sql/filter_opt.go, line 310 at r4 (raw file):

		convFunc := func(v parser.VariableExpr) (bool, parser.Expr) {
			if iv, ok := v.(*parser.IndexedVar); ok {
				renderExpr := s.render[iv.Idx]

I'm confused :(
How do we know that the IndexedVars from extraFilter have s.render as a container? It would seem to make sense, but I'm looking at the IndexedVarContainer implementation in selectNode and I seem to see that they refer to s.sourceInfo[0]. I think I'm probably looking at the wrong level... The IndexedVars from extraFilter use another container, not s (I guess they use a parent of s?). Is there a way to assert that the container is the one we expect?

I guess your comment above has something to do with this:

// The filter that's being added refers to the 
// expressions, not the select's source node.

Maybe you could expand it and make it more technical.


pkg/sql/filter_opt.go, line 369 at r4 (raw file):

	}

	mergedBoundary := n.pred.numMergedEqualityColumns

can you explain what the assumed layout is here? The merged columns are the first in n.datasource? And what's sourceBoundary exactly?


pkg/sql/filter_opt.go, line 394 at r4 (raw file):

					// The expression is referring to one of the merged equality columns.
					// Substitute it with the appropriate COALESCE expression.
					return true, parser.NewTypedCoalesceExpr(

what's this COALESCE for? Why can't we use either of the sources directly?


pkg/sql/filter_opt.go, line 500 at r4 (raw file):

}

// conjExpr represents a conjunction of multiple sub-predicates.

perhaps you can hang more hints here: do we represent expressions as a series of conjunctions? (I think) it's because conjunctions can be independently "pushed down" on our trees


pkg/sql/filter_opt.go, line 505 at r4 (raw file):

// conjPredicate represents a single predicate in a conjunction.
type conjPredicate struct {
	// expr is the expression containing the predicate itself.

can this be, say, a disjunction? I think the name "predicate" might suggest very narrow expressions, maybe change it or expand more.


Comments from Reviewable

@knz
Copy link
Contributor Author

knz commented Jan 8, 2017

Review status: 5 of 14 files reviewed at latest revision, 16 unresolved discussions.


pkg/sql/filter_opt.go, line 27 at r4 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

how about we get rid of this function so that we don't have to explain the 2-way interaction between it and addFilter (which I found confusing)? Or rename it to fooWrapper, where foo is what we end up calling addFilter.

Done.


pkg/sql/filter_opt.go, line 43 at r4 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

how about we call this pushDownFilter and explain that it does two things:

  1. it asks each node to push down its own filter
  2. it can optionally receive an extraFilter from the parent, which is to be combined with the node's own filter. In case this is passed, part of it might be returned as a rejected expression.

Otherwise, the calls to optimizeFilter(), which have nothing to do with the filter being added, are confusing.

Done.


pkg/sql/filter_opt.go, line 51 at r4 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

This sounds to me like:

function f(doSmthElse bool) {
  if !doSmthElse {
    doSmth();
  } else {
    doSmthElse();
  }
}

:P
How about we add another optional layer that does the wrapping as a separate function?

Done.


pkg/sql/filter_opt.go, line 54 at r4 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

should we start listing such methods near the planNode interface, as they'll need to be augmented for every new node type?

Excellent idea. Done. (plan.go)


pkg/sql/filter_opt.go, line 67 at r4 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

only if the filter is "pure", right?

Right. Extended the comment accordingly.


pkg/sql/filter_opt.go, line 70 at r4 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

nit: "fallthrough" in the context of a switch means something else. Perhaps put a comment above the switch saying that sometimes the cases return, sometimes they don't.

Thanks for explaining. Done.


pkg/sql/filter_opt.go, line 81 at r4 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

mind documenting what this is about? As you've told me, it's about interacting with the "needed cols" infrastructure which needs to know that some columns might not be needed any more, if we got rid of parts of the filter. It might be worth mentioning that in the ivarHelper docs too. Or maybe it should be an argument to rebind() - whether or not to reset?

Good idea! I changed the interface of Rebind as suggested.


pkg/sql/filter_opt.go, line 92 at r4 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

explain that the filter is guaranteed to be... "simple" and fully acceptable by the scanNode (guaranteed by addRenderFilter(). Maybe worth a sanity check too.

So here we don't need n.filterVars.Reset() because we're only adding new indexVars, not removing any?

For consistency actually a Reset is needed here. Well spotted!

What do you mean "simple"?


pkg/sql/filter_opt.go, line 122 at r4 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

as we discussed, let's just panic in this unexpected case

Done.


pkg/sql/filter_opt.go, line 156 at r4 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

can we propagate the FILTER from an aggregation if it's common to all aggregations (or more realistically if there's a single aggregation)? :B Maybe leave a TODO.

I added the explanatory comment. Thanks for the suggestion. Also filed as #12762.


pkg/sql/filter_opt.go, line 298 at r4 (raw file):

why is that?

Added a paragraph with examples in the comment to clarify.

What if we have a selectNode on top of another selectNode?

That's a separate optimization. Filed as #12763.

Shouldn't we support propagating more complex filters?

No. There are no more complex filters. This code can propagate arbitrarily complex filters already.

How about we always try to push down everything and we make nodes responsible of accepting only what they can support instead of deciding what their children can support? Does that make sense?

This doesn't make sense (to me).
(Either you show me with code what you mean or you open a follow-up issue with your suggestion fleshed out with examples.)


pkg/sql/filter_opt.go, line 310 at r4 (raw file):

I'm confused :(

Happens to the best of us :)

How do we know that the IndexedVars from extraFilter have s.render as a container?

We don't! And the beauty of it: it doesn't matter!

The only thing that matters is 1) that the values of their Idx field is in the proper range; that has been taken care of by the parent node; and 2) that the node that eventually accepts the filter does a Rebind() to ensure that the container is attached properly.

The reason why the container doesn't matter during the transformation (i.e. until Rebind) is that the container is only used to retrieve the type of the node, in which case a "wrong" (previous) container will do just as well because the type of an indexedvar never changes across nodes, or to render it to a string, which never happens during this transform, or to evaluate it to a Datum, which also never happens here.

I guess your comment above has something to do with this:
// The filter that's being added refers to the
// expressions, not the select's source node.

Maybe you could expand it and make it more technical.

Done - added an explanatory comment at the beginning of the file.


pkg/sql/filter_opt.go, line 369 at r4 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

can you explain what the assumed layout is here? The merged columns are the first in n.datasource? And what's sourceBoundary exactly?

Done.


pkg/sql/filter_opt.go, line 394 at r4 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

what's this COALESCE for? Why can't we use either of the sources directly?

That's the definition of the results produced by USING or NATURAL. It's not equivalent to either operands because of NULLs. Blame the SQL standard.


pkg/sql/filter_opt.go, line 500 at r4 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

perhaps you can hang more hints here: do we represent expressions as a series of conjunctions? (I think) it's because conjunctions can be independently "pushed down" on our trees

Yes, see the new long comment at the beginning.


pkg/sql/filter_opt.go, line 505 at r4 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

can this be, say, a disjunction? I think the name "predicate" might suggest very narrow expressions, maybe change it or expand more.

No, the word "predicate" is general in logic theory, it really can be any boolean expression.


Comments from Reviewable

@RaduBerinde
Copy link
Member

pkg/sql/filter_opt.go, line 542 at r5 (raw file):

		} else {
			// This is a free expression.
			// Note: we arrive here if f.expr is either a constant

wouldn't a constant boolean expression, after normalization, be either true or false? or are there cases where we can't simplify even though it's constant?


Comments from Reviewable

@RaduBerinde
Copy link
Member

Review status: 5 of 14 files reviewed at latest revision, 30 unresolved discussions, all commit checks successful.


pkg/sql/filter_opt.go, line 394 at r4 (raw file):

Previously, knz (kena) wrote…

That's the definition of the results produced by USING or NATURAL. It's not equivalent to either operands because of NULLs. Blame the SQL standard.

Actually it is equivalent for inner joins (which is the case here)...


pkg/sql/filter_opt.go, line 40 at r5 (raw file):

// Definitions:
//
//     t( A )    -> f(A, T)

I had a lot of trouble understanding these definitions. -> typically means "implies", should we use = or :=? We are treating f, w, t as functions but it's not defined what they return. It looks like t and f return a data source and a predicate, and w just a predicate? In the definition of w we should first define R and B, e.g. "if f(A, P) = R,B then w(A, P) = .."

I think it would be very helpful to have a brief (if handwaivy) explanation of each function.


pkg/sql/filter_opt.go, line 64 at r5 (raw file):

//
//     f( [distinct FROM A], P ) -> R, [distinct FROM B]
//                                  where (R, B) := f(A, P)

Shouldn't this be T, [distinct FROM w(A, P)]? Why wouldn't we evaluate any "remaining" filter under the distinctNode/sortNode?


pkg/sql/filter_opt.go, line 68 at r5 (raw file):

//                                  where (R, B) := f(A, P)
//
// Some nodes "block" filtering entirely:

block filter propagation (not filtering)


pkg/sql/filter_opt.go, line 101 at r5 (raw file):

// General implementation principles:
//
// - predicates are encoded as conjunctions (list of predicates that

Is there really a big benefit to using lists of predicates vs just an expression? I understand the ease of merging but that isn't hard to do with expressions either (just expand any top-level AndExpr or add one).


pkg/sql/filter_opt.go, line 138 at r5 (raw file):

func (p *planner) propagateFilters(
	plan planNode, info *dataSourceInfo, extraFilter conjExpr,
) (newPlan planNode, remainingFilter conjExpr, err error) {

In most situations any remainingFilter would just end up in a filterNode right above the plan (i.e. what propagateOrWrapFilters does), so I'm wondering if we can't get away with just the propagateOrWrapFilters form.

It seems to me that the benefit of returning a remainingFilter would be to "bubble up" the remaining filter. Are there any situations where that is actually desirable?


pkg/sql/filter_opt.go, line 175 at r5 (raw file):

		// A distinct node can propagate a filter. Source filtering
		// reduces the amount of work.
		subPlan, remainingFilter, err = p.propagateFilters(n.plan, info, extraFilter)

As mentioned before, I think we should propagateOrWrap here (and below for sortNode).


pkg/sql/filter_opt.go, line 177 at r5 (raw file):

		subPlan, remainingFilter, err = p.propagateFilters(n.plan, info, extraFilter)
		if err != nil {
			return plan, extraFilter, err

Why not nil, nil, err in all the error cases? Typically the caller shouldn't rely on any results for errors


pkg/sql/filter_opt.go, line 182 at r5 (raw file):

	case *sortNode:
		// A sort node can propagate a filter, and source filter reduces

filtering


pkg/sql/filter_opt.go, line 321 at r5 (raw file):

// triggerFilterPropagation initiates filter propagation on the given plan.
func (p *planner) triggerFilterPropagation(plan planNode) (planNode, error) {
	newPlan, remainingFilter, err := p.propagateFilters(plan, nil, nil)

Should this be propagateOrWrapFilters? Otherwise AFAICT we would hit the panic below in many cases (e.g. if the plan is a filterNode over valuesNode).


pkg/sql/filter_opt.go, line 381 at r5 (raw file):

		// of columns from the source, that is, we do not replace:
		//
		//   SELECT a + 2 * b AS c FROM t WHERE c > 123

This query doesn't actually work (column "x" does not exist, checked in crdb and pg). What you are saying here still applies for subqueries though (SELECT ... FROM (SELECT ...) AS ... WHERE ...).

This would be worth mentioning in the big comment around "f propagates filters through render nodes".


pkg/sql/filter_opt.go, line 527 at r5 (raw file):

	}
	var combinedExpr, leftExpr, rightExpr conjExpr
	for _, f := range initialPred {

I am surprised we don't use splitFilter here, which does a somewhat better job than just classifying top-level conjunctions (it does work on things like (NOT left<1 OR right>2) among others). We should at least add a TODO.


pkg/sql/filter_opt.go, line 530 at r5 (raw file):

		hasLeft = false
		hasRight = false
		// We don't kneed to reset the helper here, as this will be done

need


pkg/sql/testdata/join, line 685 at r3 (raw file):

query TTTTTTTTIIITTI
SELECT     NULL::text  AS pktable_cat,
	   pkn.nspname AS pktable_schem,

Are these tabs? I think it's better to use spaces in these test files (plus it's inconsistent across this query)


Comments from Reviewable

@knz
Copy link
Contributor Author

knz commented Jan 10, 2017

Review status: 5 of 14 files reviewed at latest revision, 30 unresolved discussions, all commit checks successful.


pkg/sql/filter_opt.go, line 394 at r4 (raw file):

Previously, RaduBerinde wrote…

Actually it is equivalent for inner joins (which is the case here)...

Aw, I had forgotten that! Nice. Now we can propagate those filters too!


pkg/sql/filter_opt.go, line 40 at r5 (raw file):

Previously, RaduBerinde wrote…

I had a lot of trouble understanding these definitions. -> typically means "implies", should we use = or :=? We are treating f, w, t as functions but it's not defined what they return. It looks like t and f return a data source and a predicate, and w just a predicate? In the definition of w we should first define R and B, e.g. "if f(A, P) = R,B then w(A, P) = .."

I think it would be very helpful to have a brief (if handwaivy) explanation of each function.

Done.


pkg/sql/filter_opt.go, line 64 at r5 (raw file):

Previously, RaduBerinde wrote…

Shouldn't this be T, [distinct FROM w(A, P)]? Why wouldn't we evaluate any "remaining" filter under the distinctNode/sortNode?

Yeah, that's smarter! Thanks for finding this out. Done.


pkg/sql/filter_opt.go, line 68 at r5 (raw file):

Previously, RaduBerinde wrote…

block filter propagation (not filtering)

Done.


pkg/sql/filter_opt.go, line 101 at r5 (raw file):

Previously, RaduBerinde wrote…

Is there really a big benefit to using lists of predicates vs just an expression? I understand the ease of merging but that isn't hard to do with expressions either (just expand any top-level AndExpr or add one).

I found the code both easier to read and to test this way.
But after rewriting to use regular expressions I notice we get more possible optimizations. So I changed this throughout.
Thanks for suggesting!


pkg/sql/filter_opt.go, line 138 at r5 (raw file):

Previously, RaduBerinde wrote…

In most situations any remainingFilter would just end up in a filterNode right above the plan (i.e. what propagateOrWrapFilters does), so I'm wondering if we can't get away with just the propagateOrWrapFilters form.

It seems to me that the benefit of returning a remainingFilter would be to "bubble up" the remaining filter. Are there any situations where that is actually desirable?

  1. I had a single function before but that made Andrei miserable. I think it doesn't hurt this way.

  2. yes currently the remaining filter bubbles up out of renderNodes, and more will bubble up out of outer joins.


pkg/sql/filter_opt.go, line 175 at r5 (raw file):

Previously, RaduBerinde wrote…

As mentioned before, I think we should propagateOrWrap here (and below for sortNode).

Done.


pkg/sql/filter_opt.go, line 177 at r5 (raw file):

Previously, RaduBerinde wrote…

Why not nil, nil, err in all the error cases? Typically the caller shouldn't rely on any results for errors

It doesn't rely on results, but to avoid extra temp assignments I use e.g. n.plan, err = p.propagateOrWrapFilters(n.plan, nil, extraFilter); if err != nil ...


pkg/sql/filter_opt.go, line 182 at r5 (raw file):

Previously, RaduBerinde wrote…

filtering

Done.


pkg/sql/filter_opt.go, line 321 at r5 (raw file):

Previously, RaduBerinde wrote…

Should this be propagateOrWrapFilters? Otherwise AFAICT we would hit the panic below in many cases (e.g. if the plan is a filterNode over valuesNode).

No, it's never possible that we get more filters remaining after propagate than what we put in. (check the formulas)
Since we put nil in, we can never get more than nil out.

In your example the filterNode will absorb the remaining filter from the valuesNodes, if any.

(Although there was an error in the case for filterNode, which your example helped me recognize. It's now fixed.)


pkg/sql/filter_opt.go, line 381 at r5 (raw file):

Previously, RaduBerinde wrote…

This query doesn't actually work (column "x" does not exist, checked in crdb and pg). What you are saying here still applies for subqueries though (SELECT ... FROM (SELECT ...) AS ... WHERE ...).

This would be worth mentioning in the big comment around "f propagates filters through render nodes".

I had indeed made a mistake in the example, which is now fixed.

What do you mean with "this would be worth mentioning"? What "this" in this context?


pkg/sql/filter_opt.go, line 527 at r5 (raw file):

Previously, RaduBerinde wrote…

I am surprised we don't use splitFilter here, which does a somewhat better job than just classifying top-level conjunctions (it does work on things like (NOT left<1 OR right>2) among others). We should at least add a TODO.

Good idea. I changed to use splitFilter. It seems to work.


pkg/sql/filter_opt.go, line 530 at r5 (raw file):

Previously, RaduBerinde wrote…

need

Done.


pkg/sql/filter_opt.go, line 542 at r5 (raw file):

Previously, RaduBerinde wrote…

wouldn't a constant boolean expression, after normalization, be either true or false? or are there cases where we can't simplify even though it's constant?

now() = now() is constant (and true) but not evaluated during normalization.


pkg/sql/testdata/join, line 685 at r3 (raw file):

Previously, RaduBerinde wrote…

Are these tabs? I think it's better to use spaces in these test files (plus it's inconsistent across this query)

Done.


Comments from Reviewable

@knz
Copy link
Contributor Author

knz commented Jan 12, 2017

Review status: 2 of 12 files reviewed at latest revision, 14 unresolved discussions.


pkg/sql/filter_opt.go, line 92 at r4 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

I meant it's surprising that here the scanNode unconditionally accepts the filter. You explain why that's the case in addRenderFilter; I think a hint here would help.

The filter is only propagated to the sub-plan if it is expressed
// using renders that are either simple datums or simple column
// references to the source.

And I still think that a sanity check about the filter being acceptable would be worth it, except that I don't know how to do that. The IndexedVars only have indexes in them, so unless one of the indexes is too large for the n.filter's container, rebinding them will not detect screwy variables that don't belong. It'd be nice to have some notion of "provenance" for variables such that we can assert that the containers/indexes they can bound too still makes sense along transformations...

Wow, your comment tells me either you are envisioning several major potential problems here which I have not even thought about, or you are very confused.

Let me reiterate what I know to be true, and feel free to extend this with your analysis:

  1. a scanNode can evaluate an arbitrary filter that is a function of the column value it extracts from the underlying table.
  2. the current code can and will always propagate an arbitrarily complex filter through a renderNode as long as, mind these words, the individal vars in the complex filter refer to renderNode columns themselves produced by simple render expressions. This has nothing to do with the complexity of the filter itself.
    In other words we will simplify e.g.
    SELECT * FROM (SELECT z,a+2,y,a-3,x FROM (SELECT x*3 AS a, y, z, x FROM t)) WHERE complexexpr(x, y, z)
    to
    renderNode(@3,@1+2,@2,@1-3,@4) FROM renderNode(@1*3,@2,@3,@1) FROM scanNode(t WHERE complexexpr(@1, @2, @3))
    without even looking at the structure/complexity of complexexpr.
  3. because of point 1, there is absolutely no "sanity check" necessary on the incoming filter of a scanNode -- any expression is acceptable

pkg/sql/filter_opt.go, line 29 at r6 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

if it makes sense, I'd replace the tautology with either true or nil in this comment

T and F are the usual shorthand notations for true and false, do you foresee a readability issue here?


pkg/sql/filter_opt.go, line 35 at r6 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

I'd suggest renaming t,f,w to trigger, propagate, wrap.

I tried and it makes the long formulas below harder to read.


pkg/sql/filter_opt.go, line 46 at r6 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

see, it doesn't matter what language the project uses if you use your own functional one in the code that matters most :P

Don't laugh. This makes me unhappy: I spent half a day to troubleshoot an issue in my code that wasn't in the functional specification. If the former was automatically derived from the latter, this company would have saved several hundred dollars.


pkg/sql/parser/indexed_vars.go, line 188 at r6 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

s/initialized/initializes since you're here

Done.


Comments from Reviewable

@knz
Copy link
Contributor Author

knz commented Jan 12, 2017

Gtg?

@knz
Copy link
Contributor Author

knz commented Jan 12, 2017

@andreimatei I just extended the renderNode comment as suggested. PTAL

@RaduBerinde
Copy link
Member

Review status: 2 of 12 files reviewed at latest revision, 11 unresolved discussions.


pkg/sql/filter_opt.go, line 36 at r7 (raw file):

//
//     "t" is abbreviation for "triggerFilterPropagation".
//     Initiates filter propagation on the given data source.

Maybe mention that by definition of f, if T is passed to f, it will always return T for the predicate.


pkg/sql/filter_opt.go, line 51 at r7 (raw file):

//                  where (R, B) := f(A, P)
//
//     "f" is abbreviation for "propagateFilters".

I think this should be first (since the others reference it)


pkg/sql/filter_opt.go, line 149 at r7 (raw file):

// remaining filter.
//
// The reason why propagateFilter returns a remainingFilter instead of

👍


pkg/sql/filter_opt.go, line 506 at r7 (raw file):

	// sourceBoundary is the logical index of the first column in
	// the right data source.
	sourceBoundary := mergedBoundary + len(n.left.info.sourceColumns)

This name confused me for a bit, doesn't really suggest it's the boundary between left and right.. maybe leftBoundary or mergedEnd/leftEnd


pkg/sql/filter_opt.go, line 540 at r7 (raw file):

		// Then the rest of the optimization below can go forward, and
		// will take care of splitting the expressions among the left and
		// right operands.

I understand what we're doing (and it's pretty neat TBH), but I don't see why we can't simply add a case in the callbacks to the "main" splitFilter calls below to accept the merged columns, eg:

if iv.Idx < mergeBoundary {
  return IndexVar(n.pred.leftEquality[iv.Idx]
} else if iv.idx < sourceBoundary {
  return IndexVar(iv.Idx - mergeBoundary)
}

pkg/sql/filter_opt.go, line 579 at r7 (raw file):

		func(expr parser.VariableExpr) (bool, parser.Expr) {
			if iv, ok := expr.(*parser.IndexedVar); ok && iv.Idx < sourceBoundary {
				return true, expr

Aren't we letting through vars that refer to merged columns (leading to panics later in shiftConj)?

Also, it's a bit odd that instead of just reindexing the vars here, we do it after the fact.


pkg/sql/filter_opt.go, line 612 at r7 (raw file):

	}

	// Extract possibly new equality columns from the combined predicate, and

Nice!


Comments from Reviewable

@andreimatei
Copy link
Contributor

gtg gg thanks


Review status: 2 of 12 files reviewed at latest revision, 11 unresolved discussions.


pkg/sql/filter_opt.go, line 92 at r4 (raw file):

Previously, knz (kena) wrote…

Wow, your comment tells me either you are envisioning several major potential problems here which I have not even thought about, or you are very confused.

Let me reiterate what I know to be true, and feel free to extend this with your analysis:

  1. a scanNode can evaluate an arbitrary filter that is a function of the column value it extracts from the underlying table.
  2. the current code can and will always propagate an arbitrarily complex filter through a renderNode as long as, mind these words, the individal vars in the complex filter refer to renderNode columns themselves produced by simple render expressions. This has nothing to do with the complexity of the filter itself.
    In other words we will simplify e.g.
    SELECT * FROM (SELECT z,a+2,y,a-3,x FROM (SELECT x*3 AS a, y, z, x FROM t)) WHERE complexexpr(x, y, z)
    to
    renderNode(@3,@1+2,@2,@1-3,@4) FROM renderNode(@1*3,@2,@3,@1) FROM scanNode(t WHERE complexexpr(@1, @2, @3))
    without even looking at the structure/complexity of complexexpr.
  3. because of point 1, there is absolutely no "sanity check" necessary on the incoming filter of a scanNode -- any expression is acceptable

Ok now I understand what's going on, thank you.


Comments from Reviewable

@knz
Copy link
Contributor Author

knz commented Jan 12, 2017

Review status: 2 of 12 files reviewed at latest revision, 9 unresolved discussions.


pkg/sql/filter_opt.go, line 36 at r7 (raw file):

Previously, RaduBerinde wrote…

Maybe mention that by definition of f, if T is passed to f, it will always return T for the predicate.

Done.


pkg/sql/filter_opt.go, line 51 at r7 (raw file):

Previously, RaduBerinde wrote…

I think this should be first (since the others reference it)

Done.


pkg/sql/filter_opt.go, line 506 at r7 (raw file):

Previously, RaduBerinde wrote…

This name confused me for a bit, doesn't really suggest it's the boundary between left and right.. maybe leftBoundary or mergedEnd/leftEnd

Done.


pkg/sql/filter_opt.go, line 540 at r7 (raw file):

Previously, RaduBerinde wrote…

I understand what we're doing (and it's pretty neat TBH), but I don't see why we can't simply add a case in the callbacks to the "main" splitFilter calls below to accept the merged columns, eg:

if iv.Idx < mergeBoundary {
  return IndexVar(n.pred.leftEquality[iv.Idx]
} else if iv.idx < sourceBoundary {
  return IndexVar(iv.Idx - mergeBoundary)
}

oh, interesting. I'll try it out!


pkg/sql/filter_opt.go, line 579 at r7 (raw file):

Previously, RaduBerinde wrote…

Aren't we letting through vars that refer to merged columns (leading to panics later in shiftConj)?

Also, it's a bit odd that instead of just reindexing the vars here, we do it after the fact.

I was counting on the fact that the USING, NATURAL and ON syntaxes are mutually exclusive. So there cannot be an ON filter using merged columns in the input.

Although now you're saying it, if we propagate the filters two times, the 2nd time can run into this situation. I'll think about it more.


Comments from Reviewable

@knz
Copy link
Contributor Author

knz commented Jan 12, 2017

Review status: 2 of 12 files reviewed at latest revision, 9 unresolved discussions, some commit checks pending.


pkg/sql/filter_opt.go, line 540 at r7 (raw file):

Previously, knz (kena) wrote…

oh, interesting. I'll try it out!

So I tried and that doesn't work: if we do what you suggest, then the part of the predicate that refers to the merged columns will always migrate to the left join operand, and will not remain in the "remainder" expr (the one subsequently split into right/combined exprs). We really need to duplicate them ahead of the split.


pkg/sql/filter_opt.go, line 579 at r7 (raw file):

Previously, knz (kena) wrote…

I was counting on the fact that the USING, NATURAL and ON syntaxes are mutually exclusive. So there cannot be an ON filter using merged columns in the input.

Although now you're saying it, if we propagate the filters two times, the 2nd time can run into this situation. I'll think about it more.

I checked and by construction there can never be a filter that refers to the merged columns (because they are all rewritten by the code at the beginning). So this is fine as-is.


Comments from Reviewable

@RaduBerinde
Copy link
Member

pkg/sql/filter_opt.go, line 540 at r7 (raw file):

Previously, knz (kena) wrote…

So I tried and that doesn't work: if we do what you suggest, then the part of the predicate that refers to the merged columns will always migrate to the left join operand, and will not remain in the "remainder" expr (the one subsequently split into right/combined exprs). We really need to duplicate them ahead of the split.

The second call doesn't need to be run on the remaining expression, it can be run on the same expression as the first call..


Comments from Reviewable

@knz
Copy link
Contributor Author

knz commented Jan 12, 2017

Review status: 2 of 12 files reviewed at latest revision, 9 unresolved discussions, some commit checks pending.


pkg/sql/filter_opt.go, line 540 at r7 (raw file):

Previously, RaduBerinde wrote…

The second call doesn't need to be run on the remaining expression, it can be run on the same expression as the first call..

But how do you compute the combinedExpr then? We have to split both the left and right sub-exprs away from it don't we?


Comments from Reviewable

@RaduBerinde
Copy link
Member

pkg/sql/filter_opt.go, line 540 at r7 (raw file):

Previously, knz (kena) wrote…

But how do you compute the combinedExpr then? We have to split both the left and right sub-exprs away from it don't we?

Ah, but then we don't get the smallest "final" remaining expression (combinedExpr)..


Comments from Reviewable

@knz
Copy link
Contributor Author

knz commented Jan 12, 2017

Review status: 2 of 12 files reviewed at latest revision, 9 unresolved discussions, some commit checks pending.


pkg/sql/filter_opt.go, line 540 at r7 (raw file):

Previously, RaduBerinde wrote…

Ah, but then we don't get the smallest "final" remaining expression (combinedExpr)..

Yes that's the point. :-)
So, anything else?


Comments from Reviewable

@RaduBerinde
Copy link
Member

:lgtm:


Review status: 2 of 12 files reviewed at latest revision, 7 unresolved discussions, all commit checks successful.


pkg/sql/filter_opt.go, line 540 at r7 (raw file):

Previously, RaduBerinde wrote…

Ah, but then we don't get the smallest "final" remaining expression (combinedExpr)..

Sorry, typed at the same time. Yes, makes sense!


pkg/sql/filter_opt.go, line 579 at r7 (raw file):

Previously, knz (kena) wrote…

I checked and by construction there can never be a filter that refers to the merged columns (because they are all rewritten by the code at the beginning). So this is fine as-is.

You're right. Could you just add a comment here explaining that there should be no references to merged columns left?
Also consider remapping the index here instead of shiftConj, it would make things a bit more clear.


Comments from Reviewable

@knz
Copy link
Contributor Author

knz commented Jan 12, 2017

Review status: 2 of 12 files reviewed at latest revision, 5 unresolved discussions, all commit checks successful.


pkg/sql/filter_opt.go, line 579 at r7 (raw file):

Previously, RaduBerinde wrote…

You're right. Could you just add a comment here explaining that there should be no references to merged columns left?
Also consider remapping the index here instead of shiftConj, it would make things a bit more clear.

Done for the comment.

I tried to inline shiftConj but that really makes a big blob of code that is really better self-contained in a separate function.


Comments from Reviewable

@RaduBerinde
Copy link
Member

pkg/sql/filter_opt.go, line 579 at r7 (raw file):

Previously, knz (kena) wrote…

Done for the comment.

I tried to inline shiftConj but that really makes a big blob of code that is really better self-contained in a separate function.

I don't understand, isn't it just replacing expr in the first return with ivarHelper.IndexedVar(iv.Idx - leftBegin)? (and rightBegin for the other)


Comments from Reviewable

tl;dr: this patch implements a primitive form of filter propagation:
queries of the form `SELECT * FROM a, b WHERE a.x = 10 and b.y = 20`
are transformed to `SELECT * FROM (SELECT * FROM a WHERE x = 10),
(SELECT * FROM b WHERE y = 20)`.

Long explanation: go take a book on relational algebra. Filtering is
commutative with a bunch of stuff.
@knz
Copy link
Contributor Author

knz commented Jan 12, 2017

Review status: 2 of 12 files reviewed at latest revision, 5 unresolved discussions.


pkg/sql/filter_opt.go, line 579 at r7 (raw file):

Previously, RaduBerinde wrote…

I don't understand, isn't it just replacing expr in the first return with ivarHelper.IndexedVar(iv.Idx - leftBegin)? (and rightBegin for the other)

Oh now I understand. Thanks for the hint. Done.


Comments from Reviewable

@RaduBerinde
Copy link
Member

Review status: 2 of 12 files reviewed at latest revision, 5 unresolved discussions, all commit checks successful.


pkg/sql/filter_opt.go, line 573 at r9 (raw file):

		leftFilter := exprConvertVars(perhapsUsingMerged,
			func(expr parser.VariableExpr) (bool, parser.Expr) {
				if iv, ok := expr.(*parser.IndexedVar); ok && iv.Idx < leftBegin {

A suggestion (possibly for a future change) - it wouldn't be too hard to also duplicate any vars that refer to right columns which are constrained to be equal to a left column (and vice-versa). This can happen regardless if merged columns exist.

For example SELECT * FROM a JOIN b where a.x = b.y AND b.y > 5 AND b.y < 10. It would be nice for the 5-10 range to make it to the a source.


Comments from Reviewable

@knz
Copy link
Contributor Author

knz commented Jan 12, 2017

Review status: 2 of 12 files reviewed at latest revision, 5 unresolved discussions, all commit checks successful.


pkg/sql/filter_opt.go, line 573 at r9 (raw file):

Previously, RaduBerinde wrote…

A suggestion (possibly for a future change) - it wouldn't be too hard to also duplicate any vars that refer to right columns which are constrained to be equal to a left column (and vice-versa). This can happen regardless if merged columns exist.

For example SELECT * FROM a JOIN b where a.x = b.y AND b.y > 5 AND b.y < 10. It would be nice for the 5-10 range to make it to the a source.

Yes I initially thought about that and I was kinda hoping that analyzeExpr (simplifyExpr) was doing it, but it doesn't :)

Good idea nonetheless, filed as #12892.


Comments from Reviewable

@knz
Copy link
Contributor Author

knz commented Jan 12, 2017

Ok I'm going to merge this now. Makes me happy!

@knz knz merged commit b8da7e8 into cockroachdb:master Jan 12, 2017
@knz knz deleted the opt-filters2 branch January 12, 2017 20:46
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.

None yet

5 participants