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

RFC: add logical property Cardinality on PlanNode #2279

Closed
st1page opened this issue May 3, 2022 · 10 comments
Closed

RFC: add logical property Cardinality on PlanNode #2279

st1page opened this issue May 3, 2022 · 10 comments
Labels
component/optimizer Query optimization. type/enhancement Improvements to existing implementation.

Comments

@st1page
Copy link
Contributor

st1page commented May 3, 2022

propose to add a logical property Cardinality on the PlanNode like https://github.com/cockroachdb/cockroach/blob/master/pkg/sql/opt/props/cardinality.go. It provides the bound of the rows number returned by the relational operator. It is just the logical bound of the result and not from statistics, which is always correct.

how can it be used?

now the scalar subquery needs that.
https://www.postgresql.org/docs/current/sql-expressions.html

It is an error to use a query that returns more than one row or more than one column as a scalar subquery. (But if, during a particular execution, the subquery returns no rows, there is no error; the scalar result is taken to be null.)
so the scalar subquery should be taken to be NULL when the subquery returns 0 rows, and give an error when it returns more than one row. our current implementation:

  1. now we can not check if the subquery returns more than one row.
  2. we use OuterJoin to handle returning 0 rows and might give a not good enough plan. for example here are some parts of the tpch q11 plan. and the q15 is very similar.
    LogicalProject { exprs: [$0, $1], expr_alias: [ps_partkey, value] }
      LogicalFilter { predicate: ($2 > $3) }
        LogicalJoin { type: LeftOuter, on: always }
          LogicalAgg { group_keys: [0], agg_calls: [sum($1), sum($1)] }
             (omitted)
          LogicalProject { exprs: [($0 * 0.0001000000:Decimal)], expr_alias: [ ] }
            LogicalAgg { group_keys: [], agg_calls: [sum($0)] }
             (omitted)

the filter-join is planned from the scalar subquery, and the outer join's right plan should return less than 1 row. the outer join is for the right input return 0 row(empty set). but in this plan, we know the right input plan, the proj-simpleAgg must returns 1 row. and we can use inner join but not outer join here, which can be more optimized by push down the predicate.

@st1page st1page added type/enhancement Improvements to existing implementation. component/optimizer Query optimization. labels May 3, 2022
@xiangjinwu
Copy link
Contributor

give an error when it returns more than one row

This has to be a runtime error. For example:

select (select v1 from t);

This runs happily when t has 0 or 1 rows, but reports an error when it has more.

A Cardinality on PlanNode is not enough for this check. Do we intend to pass Cardinality to backend executors?

Could you share more usage examples? I can only think of Cardinality == 1 for simple-agg or select-without-from, and all other cases will just be Cardinality in [0, +inf).
(Well... values can also have a fixed cardinality, and filter with an always false condition produces 0 rows. How useful are these?)

@st1page
Copy link
Contributor Author

st1page commented May 3, 2022

for 1, I had some misunderstanding about that and now notice the error was found in execution after planning in PG. but how can we give this error in execution with just a join planNode and it does not know the subquery? I have another proposal to give an error in frontend when the plan can give more than 1 row. It is a different behavior with PostgreSQL. in other word. we will give a error for a > (select b from t) and force user to write query a > (select b from t limit 1)

for 2, it is for simpleAgg, Limit and TopN. and the is it can derived from input. for example project will not change the Cardinality, filter and semi-join/anti-join will not change the Cardinality.max. filter can give more Cardinality with condition on special columns such as a > (select b from t where t.row_id = xxx) and row_id is our pk, which can ident only one row.

@likg227
Copy link
Contributor

likg227 commented May 3, 2022

I just tested it in RW, and it can't report a runtime error for this case, so we need a way to notice backend this limit. As for @st1page 's proposal, I'm neutral. Besides, we can try to derive Cardinality for different operators but I can't see any benefit of Cardinality in other places.

@st1page
Copy link
Contributor Author

st1page commented May 3, 2022

Not sure if it is useful in other places. so we can change it to an enum like

enum logicalCardinality {
   zero,
   one,
   max_one,
   multi,
}

@xiangjinwu
Copy link
Contributor

xiangjinwu commented May 4, 2022

1a] First we need to pick a behavior: runtime error or query rewrite. SQL Server introduced an Assert operator for runtime error. Not sure if we want to avoid runtime errors, especially in the following streaming case:

  • Original query is running fine because source table has 1 row;
  • A new row is inserted and all downstreams of subquery started returning errors;
  • Seeing the error, one row is removed and the user expects the whole stream to recover.

(But we still have runtime errors in backend, for example integer overflow.)

1b] In the query rewrite case, encouraging limit-without-order may not be a good idea. An aggregate ANY() is clearer about the semantic. If determinism is preferred over performance, the rewrite can use MIN() or limit-with-order / top-n.

2] You are right. I forget limit ... And project or filter can propagate the max. So we only care whether max > 1 for (plan time or runtime) error reporting, and only care whether min >= 1 for unconditional outer join simplification. The latter case seems less useful.

The plan above can actually avoid outer join by combining filter above:

LogicalJoin { type: Inner, on: ($2 > $3) }

Because nulls generated by outer join cannot eval to true for > $3.

@st1page
Copy link
Contributor Author

st1page commented May 4, 2022

The plan above can actually avoid outer join by combining filter above:

LogicalJoin { type: Inner, on: ($2 > $3) }

Because nulls generated by outer join cannot eval to true for > $3.

think about the optimization, I think it needs constant folding. we can try to improve our filter-join rule.

@st1page
Copy link
Contributor Author

st1page commented May 4, 2022

So we only care whether max > 1 for (plan time or runtime) error reporting, and only care whether min >= 1 for unconditional outer join simplification

so how about just adding the max1row and min1Row bool properties on the PlanNode?

@xiangjinwu
Copy link
Contributor

My point is:

  • Enhanced FilterJoin rule is better than min1Row in the cases above. But I did not get how it requires constant folding here.
  • The only thing left is max1row. No strong objection to it. Just unsure about the urgency to decide between runtime error vs query rewrite. Adding this basically allows us to show an error rather than returning wrong results, for an ill-formed query. There are a lot other ill-formed queries we are not rejecting properly now. It feels like the priority is to support more good queries.

btw still curious about its usage in cockroachdb

@st1page
Copy link
Contributor Author

st1page commented May 4, 2022

sure.

  • we can use a naive filter-outerJoin rule to try to simplify the outerJoin to inner join in order to optimize the tpch q11 and q15.
  • +1 for the opinion on scalar subquery error. It is a little complex, especially for the streaming operator.

As for "LogicalCardinality", "max1row", "min1row", "exact1row" or other similar properties, we can rethink them when we really need them to in optimizing.

@xiangjinwu
Copy link
Contributor

The outer join of q15 has been handled by #2349.
For other use cases, we will re-evaluate when they come.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
component/optimizer Query optimization. type/enhancement Improvements to existing implementation.
Projects
None yet
Development

No branches or pull requests

4 participants