Skip to content

Commit

Permalink
add formula.
Browse files Browse the repository at this point in the history
  • Loading branch information
winoros committed Sep 14, 2018
1 parent 8e012e6 commit bdd26fa
Show file tree
Hide file tree
Showing 9 changed files with 452 additions and 18 deletions.
49 changes: 31 additions & 18 deletions docs/design/2018-09-04-histograms-in-plan.md
Original file line number Diff line number Diff line change
Expand Up @@ -40,57 +40,70 @@ This struct will be maintained when we call `deriveStats`.

Currently we don't change the histogram itself during planning. Because it will consume a lot of time and memory space. I’ll try to maintain ranges slice or the max/min value to improve the accuracy of row count estimation instead.

###Before we maintain range information
### Before we maintain range information

We maintain the histogram in `Projection`, `Selection`, `Join`, `Aggregation`, `Sort`, `Limit` and `DataSource` operators.

####`Sort`
#### `Sort`
We can just copy children's `statsInfo` without doing any change.

####`Limit`
#### `Limit`
we can just copy children's `statsInfo` or ignore the histogram information. As you know, its execution logic is based on randomization. It's hard to maintain the statistics information after `Limit`. But we may use the information before it to do some estimation in some scenarios.

####`Projection`
#### `Projection`
Change the reflection of the map we maintained.

####`Aggregation`
#### `Aggregation`
Use the histogram to estimate the NDV of group-by items. If one index cannot cover the group-by item, we’ll multiply the NDV of each group-by column. If the output of `Aggregation` includes group-by columns, we’ll maintain the histogram of them for future use.

####`Join`
#### `Join`
There're several kinds of joins.

#####Inner join
Use histograms to do the row count estimation with the join key condition. Since it won’t have one side filter, we only need to consider the composite filters after considering the join key. We can simply multiply `selectionFactor` if there are other composite filters in our first version of implementation. Since `Selectivity` cannot calculate selectivity of an expression containing multiple columns.
##### Inner join
Use histograms to do the row count estimation with the join key condition.

#####One side outer join
It can be calculated as this formula <img alt="$selecivity=joinKeySelectivity*leftRowCount*rightRowCount$" src="svgs/caab8c4da85732f108d5e1b1cfe5e285.png?invert_in_darkmode" align="middle" width="488.3218329pt" height="22.8310566pt"/>

Where <img alt="$selectivity = \frac{rowCount(t1) - nulls(t1.key)}{rowCount(t1)*NDV(t1.key)}*\frac{rowCount(t2) - nulls(t2.key)}{rowCount(t2)*NDV(t2.key)}*ndvAfterJoin$" src="svgs/305e3aca4fcdd416e505e59b337ec67f.png?invert_in_darkmode" align="middle" width="579.89943495pt" height="33.2053986pt"/>.

The `ndvAfterJoin` can be <img alt="$min(NDV(t1.key), NDV(t2.key))$" src="svgs/30df1c648fa9fe43985776847c8dbe60.png?invert_in_darkmode" align="middle" width="248.4423216pt" height="24.657534pt"/> or a more detailed one if we can caculate it.


Since it won’t have one side filter, we only need to consider the composite filters after considering the join key. We can simply multiply `selectionFactor` if there are other composite filters in our first version of implementation. Since `Selectivity` cannot calculate selectivity of an expression containing multiple columns.

##### One side outer join
It depends on the join keys’ NDV. And we can just use histograms to estimate it if there’re non-join-key filters.

#####Semi join
##### Semi join
It’s something similar to inner join. But no data expanding occurs. When we maintain the range information, we can get a nearly accurate answer of its row count.

#####Anti semi join
##### Anti semi join
Same with semi join.

####`Selection`
#### `Selection`
Just use it to calculate the selectivity.

####`DataSource`
And to maintain `NDV`, we can use <img alt="$NDV_{a} = NDV_{a} * (1 - (1 - selectivity_a)^{cntPerVal})*(1 - (1 - selectivity_{others})^{cntPerVal})$" src="svgs/9b88d7d105ad2b263532733169dd94fe.png?invert_in_darkmode" align="middle" width="636.8712141pt" height="27.9124395pt"/> to estimate it.

If we have NDV of every bucket of histogram, we can make this formula more accurate.

#### `DataSource`
If it’s a non-partitioned table, we just maintain the map. If it’s a partitioned table, we now only store the statistics of each partition. So we need to merge them. We need a cache or something else to ensure that we won’t merge them each time we need it, which will consume tooooo much time and memory space.

####Others
#### Others
For other plan operators or `DataSource` which do not contain any histogram, we just use the original way to maintain `statsInfo`. We won’t maintain histograms for them. Taking `Union` for an example, if we maintain histograms for it, we need to merge the histograms, which is really an expensive operation. But we can consider maintaining this if a hint is given.

###After we maintain range information
### After we maintain range information

Most things are the same with above. But there will be something more to do.

####`Selection` and `Join`
#### `Selection` and `Join`
We need to cut off the things which are not in ranges when doing estimation, and update the ranges information after the estimation. Also the NDV of column can be estimated more accurately.

####`Aggregation`
#### `Aggregation`
We only need to cut off the things which are not in ranges when doing estimation. There is no need to update the ranges information.

####`TopN`
#### `TopN`
We now have the ability to maintain histograms of the order-by items.


Expand Down
Loading
Sorry, something went wrong. Reload?
Sorry, we cannot display this file.
Sorry, this file is invalid so it cannot be displayed.
Loading

0 comments on commit bdd26fa

Please sign in to comment.