Re: A problem about partitionwise join

2024-05-03 Thread Robert Haas
On Fri, May 3, 2024 at 7:47 AM Richard Guo  wrote:
>> Does the additional logic added by this patch have a noticeable
>> performance cost?
>
> I think one concern regarding performance cost is that the function
> exprs_known_equal() would be called O(N^2) times, where N is the number
> of partition key expressions.  But I think this might not be a problem.
> The number of a joinrel's partition key expressions would only be equal
> to the join degree, since each base relation within the join contributes
> only one partition key expression, according to
> set_joinrel_partition_key_exprs().  This number would not scale with the
> number of partitions.  But I have not measured the performance in
> practice by running benchmarks.  Maybe I'm just wrong.

I don't know, but I do think you should do some benchmarking and see
if you can find cases where this regresses performance. In my opinion,
this feature is worth having only if it's basically free. There's lots
of things we could do in the planner that would give better (perhaps
much better) plans in certain cases, but which we don't do because in
all other cases we'd pay a certain number of CPU cycles to have them
and it just doesn't make sense given how often we'd actually get a
benefit. This might be another such case.

I agree with you that the number of partition key expressions is
likely to be small, but that doesn't mean there's no problem. A big
part of the value of equivalence classes is that they let us make
deductions cheaply. Replacing that with some more complex matching
mechanism probably has a cost, and maybe that cost is material. If
it's not quite material with one partition key expression, going up to
2 or 3 of them might make it matter.

-- 
Robert Haas
EDB: http://www.enterprisedb.com




Re: A problem about partitionwise join

2024-05-03 Thread Richard Guo
On Wed, May 1, 2024 at 1:31 AM Robert Haas  wrote:

> I think it's slightly questionable whether this patch is worthwhile.
> The case memorialized in the regression tests, t1.a = t2.a AND t1.a =
> t2.b, is a very weird thing to do. The case mentioned in the original
> email, foo.k1 = bar.k1 and foo.k2 = bar.k2 and foo.k2 = 16, seems like
> something that could realistically happen, especially when there are
> views in use (e.g. the view joins foo and bar, and then someone
> queries the view for one of the join columns). In such a case, it's
> possible that foo.k2 = 16 is selective enough that we really don't
> care about partition-wise join any more, but it's also possible that
> it's not too selective and we do care about partition-wise join. So I
> don't think that the case that the patch fixes is something that can
> ever happen, but I do think it's probably fairly rare that brings any
> benefit, which is why I thought that EC-based matching was an OK
> approach to this problem initially. Perhaps that was the wrong idea,
> though.


Thank you for taking the time to review this patch!  I think Ashutosh
also mentioned that the new added test case looks artificial.  I must
admit that I'm not too sure how common we encounter queries with
partially-redundant join clauses in real-life scenarios.  It is possible
that such cases are quite rare, and this patch will then not be of much
use.

I initially brought up this issue because I noticed an inconsistency
regarding the generation of a partition-wise join: with 't1.k = t2.k and
t1.k = t2.val' we are able to generate a partition-wise join, while its
equivalent form 't1.k = t2.val and t1.k = t2.k' does not result in a
partition-wise join.  I think this inconsistency could be confusing.

The reason behind this is that with 't1.k = t2.val and t1.k = t2.k' it
happens to constrain other members (t1.k and t2.val) of the EC than the
ones we are looking for (t1.k and t2.k).  Our current code looks through
the join's restriction clauses for matched keys.  In addition to that,
this patch checks to see if any unmatched keys are known equal by ECs,
leveraging function exprs_known_equal().


> Does the additional logic added by this patch have a noticeable
> performance cost?


I think one concern regarding performance cost is that the function
exprs_known_equal() would be called O(N^2) times, where N is the number
of partition key expressions.  But I think this might not be a problem.
The number of a joinrel's partition key expressions would only be equal
to the join degree, since each base relation within the join contributes
only one partition key expression, according to
set_joinrel_partition_key_exprs().  This number would not scale with the
number of partitions.  But I have not measured the performance in
practice by running benchmarks.  Maybe I'm just wrong.

Thanks
Richard


Re: A problem about partitionwise join

2024-04-30 Thread Robert Haas
On Wed, Feb 21, 2024 at 6:25 AM Richard Guo  wrote:
> This patch was returned due to 'lack of interest'.  However, upon
> verification, it appears that the reported issue still exists, and the
> proposed fix in the thread remains valid.  Hence, resurrect this patch
> after rebasing it on master.  I've also written a detailed commit
> message which hopefully can help people review the changes more
> effectively.

I think it's slightly questionable whether this patch is worthwhile.
The case memorialized in the regression tests, t1.a = t2.a AND t1.a =
t2.b, is a very weird thing to do. The case mentioned in the original
email, foo.k1 = bar.k1 and foo.k2 = bar.k2 and foo.k2 = 16, seems like
something that could realistically happen, especially when there are
views in use (e.g. the view joins foo and bar, and then someone
queries the view for one of the join columns). In such a case, it's
possible that foo.k2 = 16 is selective enough that we really don't
care about partition-wise join any more, but it's also possible that
it's not too selective and we do care about partition-wise join. So I
don't think that the case that the patch fixes is something that can
ever happen, but I do think it's probably fairly rare that brings any
benefit, which is why I thought that EC-based matching was an OK
approach to this problem initially. Perhaps that was the wrong idea,
though.

Does the additional logic added by this patch have a noticeable
performance cost?

-- 
Robert Haas
EDB: http://www.enterprisedb.com




Re: A problem about partitionwise join

2024-03-25 Thread Ashutosh Bapat
On Mon, Mar 25, 2024 at 9:01 AM Richard Guo  wrote:

>
> create table p (k1 int, k2 int, val int) partition by range(k1, k2);
> create table p_1 partition of p for values from (1,1) to (10,100);
> create table p_2 partition of p for values from (10,100) to (20,200);
>
> set enable_partitionwise_join to on;
>
> explain (costs off)
> select * from p as foo join p as bar on foo.k1 = bar.k1 and foo.k2 =
> bar.k2 and foo.k2 = 5;
>QUERY PLAN
> -
>  Hash Join
>Hash Cond: (foo.k1 = bar.k1)
>->  Append
>  ->  Seq Scan on p_1 foo_1
>Filter: (k2 = 5)
>  ->  Seq Scan on p_2 foo_2
>Filter: (k2 = 5)
>->  Hash
>  ->  Append
>->  Seq Scan on p_1 bar_1
>  Filter: (k2 = 5)
>->  Seq Scan on p_2 bar_2
>  Filter: (k2 = 5)
> (13 rows)
>

Thanks for the example. You are right.

I think we need some way to avoid two different ways of looking up
partition keys - if we can't teach the EC machinery to produce clauses with
partition keys (always), we need to teach EC to contain partition keys in
case of outer joins. Tom alluded to this but I haven't seen any proposal.
The potential danger with the current patch is that it will continue to
have two loops even if we fix one of the above cases in future.

-- 
Best Wishes,
Ashutosh Bapat


Re: A problem about partitionwise join

2024-03-24 Thread Richard Guo
On Tue, Mar 19, 2024 at 3:40 PM Ashutosh Bapat 
wrote:

> On Tue, Mar 19, 2024 at 8:18 AM Richard Guo 
> wrote:
>
>> On Thu, Mar 7, 2024 at 7:13 PM Ashutosh Bapat <
>> ashutosh.bapat@gmail.com> wrote:
>>
>>> Approach
>>> 
>>> The equijoin condition between partition keys doesn't appear in the
>>> join's restrictilist because of 'best_score' strategy as you explained well
>>> in [2]. What if we add an extra score for clauses between partition keys
>>> and give preference to equijoin between partition keys? Have you given it a
>>> thought? I feel that having an equijoin clause involving partition keys has
>>> more usages compared to a clause with any random column. E.g. nextloop may
>>> be able to prune partitions from inner relation if the clause contains a
>>> partition key.
>>>
>>
>> Hmm, I think this approach won't work in cases where one certain pair of
>> partition keys has formed an EC that contains pseudoconstants.  In such
>> cases, the EC machinery will generate restriction clauses like 'pk =
>> const' rather than any join clauses.
>>
>
> That should be ok and more desirable. Clauses like pk = const will leave
> only one partition around in each of the joining relations thus PWJ won't
> be required OR it will be automatic - whichever way you see it.
>

No, that's not true.  There could be multiple partition keys, and the
particular key involved in the pushed-down restriction 'pk = const' may
not be able to prune away any partitions.  To be concrete, consider the
query:

create table p (k1 int, k2 int, val int) partition by range(k1, k2);
create table p_1 partition of p for values from (1,1) to (10,100);
create table p_2 partition of p for values from (10,100) to (20,200);

set enable_partitionwise_join to on;

explain (costs off)
select * from p as foo join p as bar on foo.k1 = bar.k1 and foo.k2 = bar.k2
and foo.k2 = 5;
   QUERY PLAN
-
 Hash Join
   Hash Cond: (foo.k1 = bar.k1)
   ->  Append
 ->  Seq Scan on p_1 foo_1
   Filter: (k2 = 5)
 ->  Seq Scan on p_2 foo_2
   Filter: (k2 = 5)
   ->  Hash
 ->  Append
   ->  Seq Scan on p_1 bar_1
 Filter: (k2 = 5)
   ->  Seq Scan on p_2 bar_2
 Filter: (k2 = 5)
(13 rows)

Thanks
Richard


Re: A problem about partitionwise join

2024-03-19 Thread Ashutosh Bapat
On Tue, Mar 19, 2024 at 8:18 AM Richard Guo  wrote:

> (Sorry it takes me some time to get back to this thread.)
>
> On Thu, Mar 7, 2024 at 7:13 PM Ashutosh Bapat <
> ashutosh.bapat@gmail.com> wrote:
>
>> I did a deeper review of the patch. Here are some comments
>>
>
> Thank you for the review!
>
>
>> Approach
>> 
>> The equijoin condition between partition keys doesn't appear in the
>> join's restrictilist because of 'best_score' strategy as you explained well
>> in [2]. What if we add an extra score for clauses between partition keys
>> and give preference to equijoin between partition keys? Have you given it a
>> thought? I feel that having an equijoin clause involving partition keys has
>> more usages compared to a clause with any random column. E.g. nextloop may
>> be able to prune partitions from inner relation if the clause contains a
>> partition key.
>>
>
> Hmm, I think this approach won't work in cases where one certain pair of
> partition keys has formed an EC that contains pseudoconstants.  In such
> cases, the EC machinery will generate restriction clauses like 'pk =
> const' rather than any join clauses.
>

That should be ok and more desirable. Clauses like pk = const will leave
only one partition around in each of the joining relations thus PWJ won't
be required OR it will be automatic - whichever way you see it.


>
> Besides, it seems to me that it's not a cheap operation to check whether
> a join clause is between partition keys when we generate join clauses
> from ECs in generate_join_implied_equalities().
>

Why? The code would be the same as what we have
in have_partkey_equi_join().

>
>


> Documentation
>> -
>> The patch does not modify any documentation. The only documentation I
>> could find about partitionwise join is the one for GUC
>> 'enable_partitionwise_join'. It says
>> --- quote
>> "Partitionwise join currently applies only when the join conditions
>> include all the partition keys, which must be of the same data type and
>> have one-to-one matching sets of child partitions.".
>> --- unquote
>> This sentence is general and IMO covers the case this patch considers.
>> But in general I feel that partitionwise join and aggregation deserve
>> separate sections next to "partition pruning" in [1]; It should mention
>> advanced partition matching algorithm as well. Would you be willing to
>> write one and then expand it for the case in the patch?
>>
>
> I don't think it should be part of this patch to add a new section in
> the docs to explain partitionwise join and aggregation.  Maybe that
> deserves a separate patch?
>

Yes.

3. The testcase added looks artificial. it outputs data that has same value
>> for two columns which is equal to the primary key of the other table - when
>> would somebody do that?. Is there some real life example where this change
>> will be useful?
>>
>
> Hmm, I think the test case is good as long as it reveals the issue that
> this patch fixes.  It follows the same format as the existing test case
> just above it.  I'm not sure if there are real life examples, but I
> think it may not always be necessary to derive test cases from them.
>

Let's defer this to the committer.


>
>
>> Code
>> 
>> Minor comment for now. It will be better to increment num_equal_pks
>> immediately after setting pk_known_equal[ipk] = true. Otherwise the code
>> gets confusing around line 2269. I will spend more time reviewing the code
>> next week.
>>
>
> Hmm, the increment of num_equal_pks on line 2272 is parallel to the one
> in the first loop (around line 2200).  Maybe it's better to keep them
> consistent as the current patch does?
>
>
In the first loop, setting pk_known_equal[ipk1] = true and ++num_equal_pks
happens on consecutive lines. That's not true in the second loop, where
there are at least some code line where num_equal_pks is inconsistent with
the number of "true" entries in pk_known_equal. We should avoid that.

-- 
Best Wishes,
Ashutosh Bapat


Re: A problem about partitionwise join

2024-03-18 Thread Richard Guo
(Sorry it takes me some time to get back to this thread.)

On Thu, Mar 7, 2024 at 7:13 PM Ashutosh Bapat 
wrote:

> I did a deeper review of the patch. Here are some comments
>

Thank you for the review!


> Approach
> 
> The equijoin condition between partition keys doesn't appear in the join's
> restrictilist because of 'best_score' strategy as you explained well in
> [2]. What if we add an extra score for clauses between partition keys and
> give preference to equijoin between partition keys? Have you given it a
> thought? I feel that having an equijoin clause involving partition keys has
> more usages compared to a clause with any random column. E.g. nextloop may
> be able to prune partitions from inner relation if the clause contains a
> partition key.
>

Hmm, I think this approach won't work in cases where one certain pair of
partition keys has formed an EC that contains pseudoconstants.  In such
cases, the EC machinery will generate restriction clauses like 'pk =
const' rather than any join clauses.

Besides, it seems to me that it's not a cheap operation to check whether
a join clause is between partition keys when we generate join clauses
from ECs in generate_join_implied_equalities().


> Documentation
> -
> The patch does not modify any documentation. The only documentation I
> could find about partitionwise join is the one for GUC
> 'enable_partitionwise_join'. It says
> --- quote
> "Partitionwise join currently applies only when the join conditions
> include all the partition keys, which must be of the same data type and
> have one-to-one matching sets of child partitions.".
> --- unquote
> This sentence is general and IMO covers the case this patch considers. But
> in general I feel that partitionwise join and aggregation deserve separate
> sections next to "partition pruning" in [1]; It should mention advanced
> partition matching algorithm as well. Would you be willing to write one and
> then expand it for the case in the patch?
>

I don't think it should be part of this patch to add a new section in
the docs to explain partitionwise join and aggregation.  Maybe that
deserves a separate patch?


> Tests
> -
> The patch adds a testcase for single column partitioning. I think we need
> to do better like
> 1. Test for partitioning on expression, multilevel partitioning, advanced
> partition matching. Those all might just work. Having tests helps us to
> notice any future breakage.
> 2. Some negative test case e.g. equijoin clauses with disjunction, with
> inequality operator, equality operators with operators from different
> families etc.
>

Thanks for the suggestions.  We can do that.


> 3. The testcase added looks artificial. it outputs data that has same
> value for two columns which is equal to the primary key of the other table
> - when would somebody do that?. Is there some real life example where this
> change will be useful?
>

Hmm, I think the test case is good as long as it reveals the issue that
this patch fixes.  It follows the same format as the existing test case
just above it.  I'm not sure if there are real life examples, but I
think it may not always be necessary to derive test cases from them.


> Code
> 
> Minor comment for now. It will be better to increment num_equal_pks
> immediately after setting pk_known_equal[ipk] = true. Otherwise the code
> gets confusing around line 2269. I will spend more time reviewing the code
> next week.
>

Hmm, the increment of num_equal_pks on line 2272 is parallel to the one
in the first loop (around line 2200).  Maybe it's better to keep them
consistent as the current patch does?

Thanks
Richard


Re: A problem about partitionwise join

2024-03-07 Thread Ashutosh Bapat
On Thu, Feb 22, 2024 at 2:56 PM Ashutosh Bapat 
wrote:

> On Wed, Feb 21, 2024 at 4:55 PM Richard Guo 
> wrote:
> >
> >
> > On Tue, Aug 2, 2022 at 4:24 AM Jacob Champion 
> wrote:
> >>
> >> Once you think you've built up some community support and the patchset
> >> is ready for review, you (or any interested party) can resurrect the
> >> patch entry by visiting
> >>
> >> https://commitfest.postgresql.org/38/2266/
> >>
> >> and changing the status to "Needs Review", and then changing the
> >> status again to "Move to next CF". (Don't forget the second step;
> >> hopefully we will have streamlined this in the near future!)
> >
> >
> > This patch was returned due to 'lack of interest'.  However, upon
> > verification, it appears that the reported issue still exists, and the
> > proposed fix in the thread remains valid.  Hence, resurrect this patch
> > after rebasing it on master.  I've also written a detailed commit
> > message which hopefully can help people review the changes more
> > effectively.
>
>
I did a deeper review of the patch. Here are some comments

Approach

The equijoin condition between partition keys doesn't appear in the join's
restrictilist because of 'best_score' strategy as you explained well in
[2]. What if we add an extra score for clauses between partition keys and
give preference to equijoin between partition keys? Have you given it a
thought? I feel that having an equijoin clause involving partition keys has
more usages compared to a clause with any random column. E.g. nextloop may
be able to prune partitions from inner relation if the clause contains a
partition key.

Partition pruning requires equality clauses on partition keys as well.
create_append_plan() fetches those from best_path->param_info. If we
created and saved the clauses involving partition keys somewhere
separately, similar to the clauses involving index keys, it might help this
case as well as the partition pruning code. Have you considered this idea?

There was a proposal to use ECs for outer joins as well and then use only
ECs to decide whether equijoins between partition keys exist. I don't think
the proposal has materialized. So we have to continue looking at
restrictlist as well. I don't see a point waiting for it, but others might
feel differently.

I am just trying to find ways to avoid two loops in
have_partkey_equi_join(). If the alternatives are worse, I think the
current approach is fine.

Documentation
-
The patch does not modify any documentation. The only documentation I could
find about partitionwise join is the one for GUC
'enable_partitionwise_join'. It says
--- quote
"Partitionwise join currently applies only when the join conditions include
all the partition keys, which must be of the same data type and have
one-to-one matching sets of child partitions.".
--- unquote
This sentence is general and IMO covers the case this patch considers. But
in general I feel that partitionwise join and aggregation deserve separate
sections next to "partition pruning" in [1]; It should mention advanced
partition matching algorithm as well. Would you be willing to write one and
then expand it for the case in the patch?

Tests
-
The patch adds a testcase for single column partitioning. I think we need
to do better like
1. Test for partitioning on expression, multilevel partitioning, advanced
partition matching. Those all might just work. Having tests helps us to
notice any future breakage.
2. Some negative test case e.g. equijoin clauses with disjunction, with
inequality operator, equality operators with operators from different
families etc.
3. The testcase added looks artificial. it outputs data that has same value
for two columns which is equal to the primary key of the other table - when
would somebody do that?. Is there some real life example where this change
will be useful?

Code

Minor comment for now. It will be better to increment num_equal_pks
immediately after setting pk_known_equal[ipk] = true. Otherwise the code
gets confusing around line 2269. I will spend more time reviewing the code
next week.

[1] https://www.postgresql.org/docs/current/ddl-partitioning.html
[2]
https://www.postgresql.org/message-id/can_9jtxucgdvy9tv6uxq0cdhrw98bztxpkfbf_75qdpi5wb...@mail.gmail.com

-- 
Best Wishes,
Ashutosh Bapat


Re: A problem about partitionwise join

2024-02-22 Thread Ashutosh Bapat
On Wed, Feb 21, 2024 at 4:55 PM Richard Guo  wrote:
>
>
> On Tue, Aug 2, 2022 at 4:24 AM Jacob Champion  wrote:
>>
>> Once you think you've built up some community support and the patchset
>> is ready for review, you (or any interested party) can resurrect the
>> patch entry by visiting
>>
>> https://commitfest.postgresql.org/38/2266/
>>
>> and changing the status to "Needs Review", and then changing the
>> status again to "Move to next CF". (Don't forget the second step;
>> hopefully we will have streamlined this in the near future!)
>
>
> This patch was returned due to 'lack of interest'.  However, upon
> verification, it appears that the reported issue still exists, and the
> proposed fix in the thread remains valid.  Hence, resurrect this patch
> after rebasing it on master.  I've also written a detailed commit
> message which hopefully can help people review the changes more
> effectively.

The concept looks useful. The SQL statement added in the test looks
cooked though (it outputs data that has same value for two columns
which is equal to primary key of other table - when would somebody do
that?). Is there some real life example of this?

The patch uses restrictclauses as well as EC's. Tom has proposed to
make EC work with outer joins sensibly. Has that happened? Can this
patch leverage it rather than having two loops?


-- 
Best Wishes,
Ashutosh Bapat




Re: A problem about partitionwise join

2024-02-21 Thread Richard Guo
On Tue, Aug 2, 2022 at 4:24 AM Jacob Champion 
wrote:

> Once you think you've built up some community support and the patchset
> is ready for review, you (or any interested party) can resurrect the
> patch entry by visiting
>
> https://commitfest.postgresql.org/38/2266/
>
> and changing the status to "Needs Review", and then changing the
> status again to "Move to next CF". (Don't forget the second step;
> hopefully we will have streamlined this in the near future!)


This patch was returned due to 'lack of interest'.  However, upon
verification, it appears that the reported issue still exists, and the
proposed fix in the thread remains valid.  Hence, resurrect this patch
after rebasing it on master.  I've also written a detailed commit
message which hopefully can help people review the changes more
effectively.

Thanks
Richard


v7-0001-Fix-partitionwise-join-with-partially-redundant-join-clauses.patch
Description: Binary data


Re: A problem about partitionwise join

2022-08-01 Thread Jacob Champion
As discussed in [1], we're taking this opportunity to return some
patchsets that don't appear to be getting enough reviewer interest.

This is not a rejection, since we don't necessarily think there's
anything unacceptable about the entry, but it differs from a standard
"Returned with Feedback" in that there's probably not much actionable
feedback at all. Rather than code changes, what this patch needs is more
community interest. You might

- ask people for help with your approach,
- see if there are similar patches that your code could supplement,
- get interested parties to agree to review your patch in a CF, or
- possibly present the functionality in a way that's easier to review
  overall.

(Doing these things is no guarantee that there will be interest, but
it's hopefully better than endlessly rebasing a patchset that is not
receiving any feedback from the community.)

Once you think you've built up some community support and the patchset
is ready for review, you (or any interested party) can resurrect the
patch entry by visiting

https://commitfest.postgresql.org/38/2266/

and changing the status to "Needs Review", and then changing the
status again to "Move to next CF". (Don't forget the second step;
hopefully we will have streamlined this in the near future!)

Thanks,
--Jacob

[1]
https://postgr.es/m/flat/0ab66589-2f71-69b3-2002-49e821740b0d%40timescale.com




Re: A problem about partitionwise join

2022-04-25 Thread Richard Guo
On Mon, Nov 22, 2021 at 3:04 PM Richard Guo  wrote:

>
> The suggested changes have already been included in v5 patch. Sorry for
> the confusion.
>
> Verified that the patch still applies and works on latest master. So I'm
> moving it to the next CF (which is Commitfest 2022-01). Please correct
> me if this is not the right thing to do.
>

Rebased the patch with latest master. Appreciate any comments.

Thanks
Richard


v6-0001-Fix-up-partitionwise-join.patch
Description: Binary data


Re: A problem about partitionwise join

2021-11-21 Thread Richard Guo
On Wed, Oct 6, 2021 at 1:19 AM Jaime Casanova 
wrote:

> On Wed, Jul 21, 2021 at 04:44:53PM +0800, Richard Guo wrote:
> > On Fri, Nov 27, 2020 at 8:05 PM Ashutosh Bapat <
> ashutosh.bapat@gmail.com>
> > wrote:
> >
> > >
> > > In the example you gave earlier, the equi join on partition key was
> > > there but it was replaced by individual constant assignment clauses.
> > > So if we keep the original restrictclause in there with a new flag
> > > indicating that it's redundant, have_partkey_equi_join will still be
> > > able to use it without much change. Depending upon where all we need
> > > to use avoid restrictclauses with the redundant flag, this might be an
> > > easier approach. However, with Tom's idea partition-wise join may be
> > > used even when there is no equi-join between partition keys but there
> > > are clauses like pk = const for all tables involved and const is the
> > > same for all such tables.
> > >
> >
> > Correct. So with Tom's idea partition-wise join can cope with clauses
> > such as 'foo.k1 = bar.k1 and foo.k2 = 16 and bar.k2 = 16'.
> >
> >
> > >
> > > In the spirit of small improvement made to the performance of
> > > have_partkey_equi_join(), pk_has_clause should be renamed as
> > > pk_known_equal and pks_known_equal as num_equal_pks.
> > >
> >
> > Thanks for the suggestion. Will do that in the new version of patch.
> >
>
> Hi Richard,
>
> We are marking this CF entry as "Returned with Feedback", which means
> you are encouraged to send a new patch (and create a new entry for a
> future CF for it) with the suggested changes.
>

Hi,

The suggested changes have already been included in v5 patch. Sorry for
the confusion.

Verified that the patch still applies and works on latest master. So I'm
moving it to the next CF (which is Commitfest 2022-01). Please correct
me if this is not the right thing to do.

Thanks
Richard


Re: A problem about partitionwise join

2021-10-05 Thread Jaime Casanova
On Wed, Jul 21, 2021 at 04:44:53PM +0800, Richard Guo wrote:
> On Fri, Nov 27, 2020 at 8:05 PM Ashutosh Bapat 
> wrote:
> 
> >
> > In the example you gave earlier, the equi join on partition key was
> > there but it was replaced by individual constant assignment clauses.
> > So if we keep the original restrictclause in there with a new flag
> > indicating that it's redundant, have_partkey_equi_join will still be
> > able to use it without much change. Depending upon where all we need
> > to use avoid restrictclauses with the redundant flag, this might be an
> > easier approach. However, with Tom's idea partition-wise join may be
> > used even when there is no equi-join between partition keys but there
> > are clauses like pk = const for all tables involved and const is the
> > same for all such tables.
> >
> 
> Correct. So with Tom's idea partition-wise join can cope with clauses
> such as 'foo.k1 = bar.k1 and foo.k2 = 16 and bar.k2 = 16'.
> 
> 
> >
> > In the spirit of small improvement made to the performance of
> > have_partkey_equi_join(), pk_has_clause should be renamed as
> > pk_known_equal and pks_known_equal as num_equal_pks.
> >
> 
> Thanks for the suggestion. Will do that in the new version of patch.
> 

Hi Richard,

We are marking this CF entry as "Returned with Feedback", which means
you are encouraged to send a new patch (and create a new entry for a
future CF for it) with the suggested changes.

-- 
Jaime Casanova
Director de Servicios Profesionales
SystemGuards - Consultores de PostgreSQL




Re: A problem about partitionwise join

2021-07-21 Thread Richard Guo
On Fri, Nov 27, 2020 at 8:05 PM Ashutosh Bapat 
wrote:

> On Tue, Nov 10, 2020 at 2:43 PM Richard Guo 
> wrote:
> > Thanks Anastasia. I've rebased the patch with latest master.
> >
> > To recap, the problem we are fixing here is when generating join clauses
> > from equivalence classes, we only select the joinclause with the 'best
> > score', or the first joinclause with a score of 3. This may cause us to
> > miss some joinclause on partition keys and thus fail to generate
> > partitionwise join.
> >
> > The initial idea for the fix is to create all the RestrictInfos from ECs
> > in order to check whether there exist equi-join conditions involving
> > pairs of matching partition keys of the relations being joined for all
> > partition keys. And then Tom proposed a much better idea which leverages
> > function exprs_known_equal() to tell whether the partkeys can be found
> > in the same eclass, which is the current implementation in the latest
> > patch.
> >
>
> In the example you gave earlier, the equi join on partition key was
> there but it was replaced by individual constant assignment clauses.
> So if we keep the original restrictclause in there with a new flag
> indicating that it's redundant, have_partkey_equi_join will still be
> able to use it without much change. Depending upon where all we need
> to use avoid restrictclauses with the redundant flag, this might be an
> easier approach. However, with Tom's idea partition-wise join may be
> used even when there is no equi-join between partition keys but there
> are clauses like pk = const for all tables involved and const is the
> same for all such tables.
>

Correct. So with Tom's idea partition-wise join can cope with clauses
such as 'foo.k1 = bar.k1 and foo.k2 = 16 and bar.k2 = 16'.


>
> In the spirit of small improvement made to the performance of
> have_partkey_equi_join(), pk_has_clause should be renamed as
> pk_known_equal and pks_known_equal as num_equal_pks.
>

Thanks for the suggestion. Will do that in the new version of patch.


>
> The loop traversing the partition keys at a given position, may be
> optimized further if we pass lists to exprs_known_equal() which in
> turns checks whether one expression from each list is member of a
> given EC. This will avoid traversing all equivalence classes for each
> partition key expression, which can be a huge improvement when there
> are many ECs. But I think if one of the partition key expression at a
> given position is member of an equivalence class all the other
> partition key expressions at that position should be part of that
> equivalence class since there should be an equi-join between those. So
> the loop in loop may not be required to start with.
>

Good point. Quote from one of Tom's earlier emails,
 "It seems at least plausible that in the cases we care about, all the
 partkeys on each side would be in the same eclasses anyway, so that
 comparing the first members of each list would be sufficient."

But I'm not sure if this holds true in all cases. However, since each
base relation within the join contributes only one partexpr, the number
of partexprs would only be equal to the join degree. Thus the loop in
loop may not be a big problem?

PS. Sorry for delaying so long time!

Thanks
Richard


v5-0001-Fix-up-partitionwise-join.patch
Description: Binary data


Re: A problem about partitionwise join

2021-03-09 Thread David Steele

On 11/27/20 7:05 AM, Ashutosh Bapat wrote:

On Tue, Nov 10, 2020 at 2:43 PM Richard Guo  wrote:


To recap, the problem we are fixing here is when generating join clauses
from equivalence classes, we only select the joinclause with the 'best
score', or the first joinclause with a score of 3. This may cause us to
miss some joinclause on partition keys and thus fail to generate
partitionwise join.

The initial idea for the fix is to create all the RestrictInfos from ECs
in order to check whether there exist equi-join conditions involving
pairs of matching partition keys of the relations being joined for all
partition keys. And then Tom proposed a much better idea which leverages
function exprs_known_equal() to tell whether the partkeys can be found
in the same eclass, which is the current implementation in the latest
patch.


In the example you gave earlier, the equi join on partition key was
there but it was replaced by individual constant assignment clauses.
So if we keep the original restrictclause in there with a new flag
indicating that it's redundant, have_partkey_equi_join will still be
able to use it without much change. Depending upon where all we need
to use avoid restrictclauses with the redundant flag, this might be an
easier approach. However, with Tom's idea partition-wise join may be
used even when there is no equi-join between partition keys but there
are clauses like pk = const for all tables involved and const is the
same for all such tables.

In the spirit of small improvement made to the performance of
have_partkey_equi_join(), pk_has_clause should be renamed as
pk_known_equal and pks_known_equal as num_equal_pks.

The loop traversing the partition keys at a given position, may be
optimized further if we pass lists to exprs_known_equal() which in
turns checks whether one expression from each list is member of a
given EC. This will avoid traversing all equivalence classes for each
partition key expression, which can be a huge improvement when there
are many ECs. But I think if one of the partition key expression at a
given position is member of an equivalence class all the other
partition key expressions at that position should be part of that
equivalence class since there should be an equi-join between those. So
the loop in loop may not be required to start with.


Richard, any thoughts on Ashutosh's comments?

Regards,
--
-David
da...@pgmasters.net




Re: A problem about partitionwise join

2020-11-27 Thread Ashutosh Bapat
On Tue, Nov 10, 2020 at 2:43 PM Richard Guo  wrote:
>
>
> On Fri, Nov 6, 2020 at 11:26 PM Anastasia Lubennikova 
>  wrote:
>>
>> Status update for a commitfest entry.
>>
>> According to CFbot this patch fails to apply. Richard, can you send an 
>> update, please?
>>
>> Also, I see that the thread was inactive for a while.
>> Are you going to continue this work? I think it would be helpful, if you 
>> could write a short recap about current state of the patch and list open 
>> questions for reviewers.
>>
>> The new status of this patch is: Waiting on Author
>
>
> Thanks Anastasia. I've rebased the patch with latest master.
>
> To recap, the problem we are fixing here is when generating join clauses
> from equivalence classes, we only select the joinclause with the 'best
> score', or the first joinclause with a score of 3. This may cause us to
> miss some joinclause on partition keys and thus fail to generate
> partitionwise join.
>
> The initial idea for the fix is to create all the RestrictInfos from ECs
> in order to check whether there exist equi-join conditions involving
> pairs of matching partition keys of the relations being joined for all
> partition keys. And then Tom proposed a much better idea which leverages
> function exprs_known_equal() to tell whether the partkeys can be found
> in the same eclass, which is the current implementation in the latest
> patch.
>

In the example you gave earlier, the equi join on partition key was
there but it was replaced by individual constant assignment clauses.
So if we keep the original restrictclause in there with a new flag
indicating that it's redundant, have_partkey_equi_join will still be
able to use it without much change. Depending upon where all we need
to use avoid restrictclauses with the redundant flag, this might be an
easier approach. However, with Tom's idea partition-wise join may be
used even when there is no equi-join between partition keys but there
are clauses like pk = const for all tables involved and const is the
same for all such tables.

In the spirit of small improvement made to the performance of
have_partkey_equi_join(), pk_has_clause should be renamed as
pk_known_equal and pks_known_equal as num_equal_pks.

The loop traversing the partition keys at a given position, may be
optimized further if we pass lists to exprs_known_equal() which in
turns checks whether one expression from each list is member of a
given EC. This will avoid traversing all equivalence classes for each
partition key expression, which can be a huge improvement when there
are many ECs. But I think if one of the partition key expression at a
given position is member of an equivalence class all the other
partition key expressions at that position should be part of that
equivalence class since there should be an equi-join between those. So
the loop in loop may not be required to start with.

-- 
Best Wishes,
Ashutosh Bapat




Re: A problem about partitionwise join

2020-11-10 Thread Richard Guo
On Fri, Nov 6, 2020 at 11:26 PM Anastasia Lubennikova <
a.lubennik...@postgrespro.ru> wrote:

> Status update for a commitfest entry.
>
> According to CFbot this patch fails to apply. Richard, can you send an
> update, please?
>
> Also, I see that the thread was inactive for a while.
> Are you going to continue this work? I think it would be helpful, if you
> could write a short recap about current state of the patch and list open
> questions for reviewers.
>
> The new status of this patch is: Waiting on Author
>

Thanks Anastasia. I've rebased the patch with latest master.

To recap, the problem we are fixing here is when generating join clauses
from equivalence classes, we only select the joinclause with the 'best
score', or the first joinclause with a score of 3. This may cause us to
miss some joinclause on partition keys and thus fail to generate
partitionwise join.

The initial idea for the fix is to create all the RestrictInfos from ECs
in order to check whether there exist equi-join conditions involving
pairs of matching partition keys of the relations being joined for all
partition keys. And then Tom proposed a much better idea which leverages
function exprs_known_equal() to tell whether the partkeys can be found
in the same eclass, which is the current implementation in the latest
patch.

Thanks
Richard


v4-0001-Fix-up-partitionwise-join.patch
Description: Binary data


Re: A problem about partitionwise join

2020-11-06 Thread Anastasia Lubennikova
Status update for a commitfest entry.

According to CFbot this patch fails to apply. Richard, can you send an update, 
please?

Also, I see that the thread was inactive for a while.
Are you going to continue this work? I think it would be helpful, if you could 
write a short recap about current state of the patch and list open questions 
for reviewers.

The new status of this patch is: Waiting on Author


Re: A problem about partitionwise join

2020-04-09 Thread Ashutosh Bapat
>
> I think it would not work for outer joins if we only check
> exprs_known_equal() for equivalences. If the equi-join conditions
> involving pairs of matching partition keys are outer join quals
> mentioning nonnullable side rels, they would not exist in any EC
> according to the current EC infrastructure. So we still have to look
> through restrictlist.
>

When I wrote that function and even today, EC didn't accommodate outer
join equality conditions. If we can somehow do that,
have_partkey_equi_join() can be completely eliminated.

-- 
Best Wishes,
Ashutosh Bapat




Re: A problem about partitionwise join

2020-04-08 Thread Tom Lane
Richard Guo  writes:
> On Thu, Apr 9, 2020 at 1:07 AM Tom Lane  wrote:
>> I have hopes of being able to incorporate outer
>> joins into the EC logic in a less squishy way in the future, by making
>> the representation of Vars distinguish explicitly between
>> value-before-outer-join and value-after-outer-join, after which we could
>> make bulletproof assertions about what is equal to what, even with outer
>> joins in the mix.  If that works out it might produce a cleaner answer
>> in this area too.

> This is very appealing. Do we have ongoing discussions/threads about
> this idea?

There's some preliminary noodling in this thread:

https://www.postgresql.org/message-id/flat/15848.1576515643%40sss.pgh.pa.us

I've pushed the earlier work discussed there, but stalled out due to
the call of other responsibilities after posting the currently-last
message in the thread.  Hoping to get back into that over the summer.

regards, tom lane




Re: A problem about partitionwise join

2020-04-08 Thread Richard Guo
On Thu, Apr 9, 2020 at 1:07 AM Tom Lane  wrote:

> Richard Guo  writes:
> > On Sun, Apr 5, 2020 at 4:38 AM Tom Lane  wrote:
> >> There is already something in equivclass.c that would almost do what
> >> we want here: exprs_known_equal() would tell us whether the partkeys
> >> can be found in the same eclass, without having to generate data
> >> structures along the way.  The current implementation is not watertight
> >> because it doesn't check opclass semantics, but that consideration
> >> can be bolted on readily enough.  So that leads me to something like
> >> the attached.
>
> > I have some concern about we only check non-nullable partexprs. Is it
> > possible that two nullable partexprs come from the same EC? I tried to
> > give an example but failed.
>
> Currently the EC infrastructure doesn't really cope with outer join
> equijoins.  They are not treated as producing true equivalences,
> so I think that the case you're worried about can't occur (which is why
> I didn't code for it).  I have hopes of being able to incorporate outer
> joins into the EC logic in a less squishy way in the future, by making
> the representation of Vars distinguish explicitly between
> value-before-outer-join and value-after-outer-join, after which we could
> make bulletproof assertions about what is equal to what, even with outer
> joins in the mix.  If that works out it might produce a cleaner answer
> in this area too.
>

This is very appealing. Do we have ongoing discussions/threads about
this idea?


>  (I did experiment with
> removing all the rest of have_partkey_equi_join() and having it
> *only* ask exprs_known_equal() about equivalences, which is more or
> less what I'm envisioning here.  That caused some of the existing
> regression tests to fail, so there's something that the idea isn't
> covering.  I didn't dig any further at the time, and in particular
> failed to check whether the problems were specifically about outer
> joins, which'd be unsurprising given the above.)
>

I think it would not work for outer joins if we only check
exprs_known_equal() for equivalences. If the equi-join conditions
involving pairs of matching partition keys are outer join quals
mentioning nonnullable side rels, they would not exist in any EC
according to the current EC infrastructure. So we still have to look
through restrictlist.

Thanks
Richard


Re: A problem about partitionwise join

2020-04-08 Thread Tom Lane
Richard Guo  writes:
> On Sun, Apr 5, 2020 at 4:38 AM Tom Lane  wrote:
>> There is already something in equivclass.c that would almost do what
>> we want here: exprs_known_equal() would tell us whether the partkeys
>> can be found in the same eclass, without having to generate data
>> structures along the way.  The current implementation is not watertight
>> because it doesn't check opclass semantics, but that consideration
>> can be bolted on readily enough.  So that leads me to something like
>> the attached.

> I have some concern about we only check non-nullable partexprs. Is it
> possible that two nullable partexprs come from the same EC? I tried to
> give an example but failed.

Currently the EC infrastructure doesn't really cope with outer join
equijoins.  They are not treated as producing true equivalences,
so I think that the case you're worried about can't occur (which is why
I didn't code for it).  I have hopes of being able to incorporate outer
joins into the EC logic in a less squishy way in the future, by making
the representation of Vars distinguish explicitly between
value-before-outer-join and value-after-outer-join, after which we could
make bulletproof assertions about what is equal to what, even with outer
joins in the mix.  If that works out it might produce a cleaner answer
in this area too.

TBH, now that I have had some exposure to the partitionwise join
matching logic I don't much like any of it.  I feel like it's doing
about the same job as ECs, but in an unprincipled and not very
efficient manner.  Right now is no time to redesign it, of course,
but maybe at some point we could do that.  (I did experiment with
removing all the rest of have_partkey_equi_join() and having it
*only* ask exprs_known_equal() about equivalences, which is more or
less what I'm envisioning here.  That caused some of the existing
regression tests to fail, so there's something that the idea isn't
covering.  I didn't dig any further at the time, and in particular
failed to check whether the problems were specifically about outer
joins, which'd be unsurprising given the above.)

Anyway, this work has missed the window for v13, so we've got plenty
of time to think about it.

regards, tom lane




Re: A problem about partitionwise join

2020-04-07 Thread Richard Guo
On Sun, Apr 5, 2020 at 4:38 AM Tom Lane  wrote:

> Richard Guo  writes:
> > Rebased the patch with latest master and also addressed the test case
> > failure reported by PostgreSQL Patch Tester.
>
> I looked this patch over, but I don't like it too much: it seems very
> brute-force (and badly under-commented).  Creating all those extra
> RestrictInfos isn't too cheap in itself, plus they'll jam up the
> equivalence-class machinery for future tests.
>

Thanks for the review.


>
> There is already something in equivclass.c that would almost do what
> we want here: exprs_known_equal() would tell us whether the partkeys
> can be found in the same eclass, without having to generate data
> structures along the way.  The current implementation is not watertight
> because it doesn't check opclass semantics, but that consideration
> can be bolted on readily enough.  So that leads me to something like
> the attached.
>

I looked through this patch and it's much more elegant than the previous
one. Thank you for working on it.

For partkeys which fail to be identified as equal by looking through
restrictlist, it's a good idea to check them in ECs with the help of
exprs_known_equal().

I have some concern about we only check non-nullable partexprs. Is it
possible that two nullable partexprs come from the same EC? I tried to
give an example but failed.


>
> One argument that could be made against this approach is that if there
> are a lot of partkey expressions, this requires O(N^2) calls to
> exprs_known_equal, something that's already not too cheap.  I think
> that that's not a big problem because the number of partkey expressions
> would only be equal to the join degree (ie it doesn't scale with the
> number of partitions of the baserels) ... but maybe I'm wrong about
> that?


You are right. According to how partexpr is formed for joinrel in
set_joinrel_partition_key_exprs(), each base relation within the join
contributes one partexpr, so the number of partexprs would be equal to
the join degree.


>   I also wonder if it's really necessary to check every pair
> of partkey expressions.  It seems at least plausible that in the
> cases we care about, all the partkeys on each side would be in the same
> eclasses anyway, so that comparing the first members of each list would
> be sufficient.  But I haven't beat on that point.
>

Not sure about it. But cannot come out with a counterexample.

Thanks
Richard


Re: A problem about partitionwise join

2020-04-04 Thread Tom Lane
Richard Guo  writes:
> Rebased the patch with latest master and also addressed the test case
> failure reported by PostgreSQL Patch Tester.

I looked this patch over, but I don't like it too much: it seems very
brute-force (and badly under-commented).  Creating all those extra
RestrictInfos isn't too cheap in itself, plus they'll jam up the
equivalence-class machinery for future tests.

There is already something in equivclass.c that would almost do what
we want here: exprs_known_equal() would tell us whether the partkeys
can be found in the same eclass, without having to generate data
structures along the way.  The current implementation is not watertight
because it doesn't check opclass semantics, but that consideration
can be bolted on readily enough.  So that leads me to something like
the attached.

One argument that could be made against this approach is that if there
are a lot of partkey expressions, this requires O(N^2) calls to
exprs_known_equal, something that's already not too cheap.  I think
that that's not a big problem because the number of partkey expressions
would only be equal to the join degree (ie it doesn't scale with the
number of partitions of the baserels) ... but maybe I'm wrong about
that?  I also wonder if it's really necessary to check every pair
of partkey expressions.  It seems at least plausible that in the
cases we care about, all the partkeys on each side would be in the same
eclasses anyway, so that comparing the first members of each list would
be sufficient.  But I haven't beat on that point.

regards, tom lane

diff --git a/src/backend/optimizer/path/equivclass.c b/src/backend/optimizer/path/equivclass.c
index 4ef1254..7c21692 100644
--- a/src/backend/optimizer/path/equivclass.c
+++ b/src/backend/optimizer/path/equivclass.c
@@ -2074,15 +2074,17 @@ reconsider_full_join_clause(PlannerInfo *root, RestrictInfo *rinfo)
  *	  Detect whether two expressions are known equal due to equivalence
  *	  relationships.
  *
- * Actually, this only shows that the expressions are equal according
- * to some opfamily's notion of equality --- but we only use it for
- * selectivity estimation, so a fuzzy idea of equality is OK.
+ * If opfamily is given, the expressions must be known equal per the semantics
+ * of that opfamily (note it has to be a btree opfamily, since those are the
+ * only opfamilies equivclass.c deals with).  If opfamily is InvalidOid, we'll
+ * return true if they're equal according to any opfamily, which is fuzzy but
+ * OK for estimation purposes.
  *
  * Note: does not bother to check for "equal(item1, item2)"; caller must
  * check that case if it's possible to pass identical items.
  */
 bool
-exprs_known_equal(PlannerInfo *root, Node *item1, Node *item2)
+exprs_known_equal(PlannerInfo *root, Node *item1, Node *item2, Oid opfamily)
 {
 	ListCell   *lc1;
 
@@ -2097,6 +2099,17 @@ exprs_known_equal(PlannerInfo *root, Node *item1, Node *item2)
 		if (ec->ec_has_volatile)
 			continue;
 
+		/*
+		 * It's okay to consider ec_broken ECs here.  Brokenness just means we
+		 * couldn't derive all the implied clauses we'd have liked to; it does
+		 * not invalidate our knowledge that the members are equal.
+		 */
+
+		/* Ignore if this EC doesn't use specified opfamily */
+		if (OidIsValid(opfamily) &&
+			!list_member_oid(ec->ec_opfamilies, opfamily))
+			continue;
+
 		foreach(lc2, ec->ec_members)
 		{
 			EquivalenceMember *em = (EquivalenceMember *) lfirst(lc2);
@@ -2125,8 +2138,7 @@ exprs_known_equal(PlannerInfo *root, Node *item1, Node *item2)
  * (In principle there might be more than one matching eclass if multiple
  * collations are involved, but since collation doesn't matter for equality,
  * we ignore that fine point here.)  This is much like exprs_known_equal,
- * except that we insist on the comparison operator matching the eclass, so
- * that the result is definite not approximate.
+ * except for the format of the input.
  */
 EquivalenceClass *
 match_eclasses_to_foreign_key_col(PlannerInfo *root,
@@ -2163,7 +2175,7 @@ match_eclasses_to_foreign_key_col(PlannerInfo *root,
 		/* Never match to a volatile EC */
 		if (ec->ec_has_volatile)
 			continue;
-		/* Note: it seems okay to match to "broken" eclasses here */
+		/* It's okay to consider "broken" ECs here, see exprs_known_equal */
 
 		foreach(lc2, ec->ec_members)
 		{
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index af1fb48..18474d8 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -56,10 +56,10 @@ static List *subbuild_joinrel_joinlist(RelOptInfo *joinrel,
 static void set_foreign_rel_properties(RelOptInfo *joinrel,
 	   RelOptInfo *outer_rel, RelOptInfo *inner_rel);
 static void add_join_rel(PlannerInfo *root, RelOptInfo *joinrel);
-static void build_joinrel_partition_info(RelOptInfo *joinrel,
+static void build_joinrel_partition_info(PlannerInfo *root, RelOptInfo *joinrel,
 

Re: A problem about partitionwise join

2020-01-18 Thread Richard Guo
Rebased the patch with latest master and also addressed the test case
failure reported by PostgreSQL Patch Tester.

Thanks
Richard


On Fri, Nov 29, 2019 at 11:35 AM Etsuro Fujita 
wrote:

> On Fri, Nov 29, 2019 at 12:08 PM Richard Guo  wrote:
> > On Fri, Nov 29, 2019 at 11:03 AM Michael Paquier 
> wrote:
> >> On Tue, Nov 26, 2019 at 08:35:33PM +0900, Etsuro Fujita wrote:
> >> > I've just started reviewing this patch.  One comment I have for now
> >> > is: this is categorized into Bug Fixes, but we have a workaround at
> >> > least to the regression test case in the patch (ie, just reorder join
> >> > clauses), so this seems to me more like an improvement than a bug fix.
> >>
> >> Hmm.  Agreed.  Changed the category and moved to next CF.
>
> > thanks Michael for the change.
>
> +1
>
> Best regards,
> Etsuro Fujita
>


v2-0001-Fix-up-partitionwise-join.patch
Description: Binary data


Re: A problem about partitionwise join

2019-11-28 Thread Etsuro Fujita
On Fri, Nov 29, 2019 at 12:08 PM Richard Guo  wrote:
> On Fri, Nov 29, 2019 at 11:03 AM Michael Paquier  wrote:
>> On Tue, Nov 26, 2019 at 08:35:33PM +0900, Etsuro Fujita wrote:
>> > I've just started reviewing this patch.  One comment I have for now
>> > is: this is categorized into Bug Fixes, but we have a workaround at
>> > least to the regression test case in the patch (ie, just reorder join
>> > clauses), so this seems to me more like an improvement than a bug fix.
>>
>> Hmm.  Agreed.  Changed the category and moved to next CF.

> thanks Michael for the change.

+1

Best regards,
Etsuro Fujita




Re: A problem about partitionwise join

2019-11-28 Thread Richard Guo
On Fri, Nov 29, 2019 at 11:03 AM Michael Paquier 
wrote:

> On Tue, Nov 26, 2019 at 08:35:33PM +0900, Etsuro Fujita wrote:
> > I've just started reviewing this patch.  One comment I have for now
> > is: this is categorized into Bug Fixes, but we have a workaround at
> > least to the regression test case in the patch (ie, just reorder join
> > clauses), so this seems to me more like an improvement than a bug fix.
>
> Hmm.  Agreed.  Changed the category and moved to next CF.
>

Thanks Etsuro for the comment and thanks Michael for the change.

Thanks
Richard


Re: A problem about partitionwise join

2019-11-28 Thread Michael Paquier
On Tue, Nov 26, 2019 at 08:35:33PM +0900, Etsuro Fujita wrote:
> I've just started reviewing this patch.  One comment I have for now
> is: this is categorized into Bug Fixes, but we have a workaround at
> least to the regression test case in the patch (ie, just reorder join
> clauses), so this seems to me more like an improvement than a bug fix.

Hmm.  Agreed.  Changed the category and moved to next CF.
--
Michael


signature.asc
Description: PGP signature


Re: A problem about partitionwise join

2019-11-26 Thread Etsuro Fujita
Hi Richard,

On Fri, Aug 30, 2019 at 3:08 AM Etsuro Fujita  wrote:
> On Thu, Aug 29, 2019 at 6:45 PM Richard Guo  wrote:
> > Attached is a patch as an attempt to address this issue. The idea is
> > quite straightforward. When building partition info for joinrel, we
> > generate any possible EC-derived joinclauses of form 'outer_em =
> > inner_em', which will be used together with the original restrictlist to
> > check if there exists an equi-join condition for each pair of partition
> > keys.
>
> Will review.

I've just started reviewing this patch.  One comment I have for now
is: this is categorized into Bug Fixes, but we have a workaround at
least to the regression test case in the patch (ie, just reorder join
clauses), so this seems to me more like an improvement than a bug fix.

Sorry for the delay.

Best regards,
Etsuro Fujita




Re: A problem about partitionwise join

2019-09-21 Thread Dilip Kumar
On Fri, Sep 20, 2019 at 2:33 PM Richard Guo  wrote:
>
> Hi Dilip,
>
> Thank you for reviewing this patch.
>
> On Fri, Sep 20, 2019 at 12:48 PM Dilip Kumar  wrote:
>>
>> On Thu, Aug 29, 2019 at 3:15 PM Richard Guo  wrote:
>> >
>> >
>> > Attached is a patch as an attempt to address this issue. The idea is
>> > quite straightforward. When building partition info for joinrel, we
>> > generate any possible EC-derived joinclauses of form 'outer_em =
>> > inner_em', which will be used together with the original restrictlist to
>> > check if there exists an equi-join condition for each pair of partition
>> > keys.
>> >
>> > Any comments are welcome!
>>  /*
>> + * generate_join_implied_equalities_for_all
>> + *   Create any EC-derived joinclauses of form 'outer_em = inner_em'.
>> + *
>> + * This is used when building partition info for joinrel.
>> + */
>> +List *
>> +generate_join_implied_equalities_for_all(PlannerInfo *root,
>> + Relids join_relids,
>> + Relids outer_relids,
>> + Relids inner_relids)
>>
>> I think we need to have more detailed comments about why we need this
>> separate function, we can also explain that
>> generate_join_implied_equalities function will avoid
>> the join clause if EC has the constant but for partition-wise join, we
>> need that clause too.
>
>
> Thank you for the suggestion.
>
>>
>>
>> + while ((i = bms_next_member(matching_ecs, i)) >= 0)
>> + {
>> + EquivalenceClass *ec = (EquivalenceClass *) list_nth(root->eq_classes, i);
>> + List*outer_members = NIL;
>> + List*inner_members = NIL;
>> + ListCell   *lc1;
>> +
>> + /* Do not consider this EC if it's ec_broken */
>> + if (ec->ec_broken)
>> + continue;
>> +
>> + /* Single-member ECs won't generate any deductions */
>> + if (list_length(ec->ec_members) <= 1)
>> + continue;
>> +
>>
>> I am wondering isn't it possible to just process the missing join
>> clause?  I mean 'generate_join_implied_equalities' has only skipped
>> the ECs which has const so
>> can't we create join clause only for those ECs and append it the
>> "Restrictlist" we already have?  I might be missing something?
>
>
> For ECs without const, 'generate_join_implied_equalities' also has
> skipped some join clauses since it only selects the joinclause with
> 'best_score' between outer members and inner members. And the missing
> join clauses are needed to generate partitionwise join. That's why the
> query below cannot be planned as partitionwise join, as we have missed
> joinclause 'foo.k = bar.k'.

oh right.  I missed that part.

> select * from p as foo join p as bar on foo.k = bar.val and foo.k = bar.k;
>
> And yes 'generate_join_implied_equalities_for_all' will create join
> clauses that have existed in restrictlist. I think it's OK since the
> same RestrictInfo deduced from EC will share the same pointer and
> list_concat_unique_ptr will make sure there are no duplicates in the
> restrictlist used by have_partkey_equi_join.
>
ok

-- 
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com




Re: A problem about partitionwise join

2019-09-20 Thread Richard Guo
Hi Dilip,

Thank you for reviewing this patch.

On Fri, Sep 20, 2019 at 12:48 PM Dilip Kumar  wrote:

> On Thu, Aug 29, 2019 at 3:15 PM Richard Guo  wrote:
> >
> >
> > Attached is a patch as an attempt to address this issue. The idea is
> > quite straightforward. When building partition info for joinrel, we
> > generate any possible EC-derived joinclauses of form 'outer_em =
> > inner_em', which will be used together with the original restrictlist to
> > check if there exists an equi-join condition for each pair of partition
> > keys.
> >
> > Any comments are welcome!
>  /*
> + * generate_join_implied_equalities_for_all
> + *   Create any EC-derived joinclauses of form 'outer_em = inner_em'.
> + *
> + * This is used when building partition info for joinrel.
> + */
> +List *
> +generate_join_implied_equalities_for_all(PlannerInfo *root,
> + Relids join_relids,
> + Relids outer_relids,
> + Relids inner_relids)
>
> I think we need to have more detailed comments about why we need this
> separate function, we can also explain that
> generate_join_implied_equalities function will avoid
> the join clause if EC has the constant but for partition-wise join, we
> need that clause too.
>

Thank you for the suggestion.


>
> + while ((i = bms_next_member(matching_ecs, i)) >= 0)
> + {
> + EquivalenceClass *ec = (EquivalenceClass *) list_nth(root->eq_classes,
> i);
> + List*outer_members = NIL;
> + List*inner_members = NIL;
> + ListCell   *lc1;
> +
> + /* Do not consider this EC if it's ec_broken */
> + if (ec->ec_broken)
> + continue;
> +
> + /* Single-member ECs won't generate any deductions */
> + if (list_length(ec->ec_members) <= 1)
> + continue;
> +
>
> I am wondering isn't it possible to just process the missing join
> clause?  I mean 'generate_join_implied_equalities' has only skipped
> the ECs which has const so
> can't we create join clause only for those ECs and append it the
> "Restrictlist" we already have?  I might be missing something?
>

For ECs without const, 'generate_join_implied_equalities' also has
skipped some join clauses since it only selects the joinclause with
'best_score' between outer members and inner members. And the missing
join clauses are needed to generate partitionwise join. That's why the
query below cannot be planned as partitionwise join, as we have missed
joinclause 'foo.k = bar.k'.

select * from p as foo join p as bar on foo.k = bar.val and foo.k = bar.k;

And yes 'generate_join_implied_equalities_for_all' will create join
clauses that have existed in restrictlist. I think it's OK since the
same RestrictInfo deduced from EC will share the same pointer and
list_concat_unique_ptr will make sure there are no duplicates in the
restrictlist used by have_partkey_equi_join.

Thanks
Richard


Re: A problem about partitionwise join

2019-09-19 Thread Dilip Kumar
On Thu, Aug 29, 2019 at 3:15 PM Richard Guo  wrote:
>
>
> Attached is a patch as an attempt to address this issue. The idea is
> quite straightforward. When building partition info for joinrel, we
> generate any possible EC-derived joinclauses of form 'outer_em =
> inner_em', which will be used together with the original restrictlist to
> check if there exists an equi-join condition for each pair of partition
> keys.
>
> Any comments are welcome!
 /*
+ * generate_join_implied_equalities_for_all
+ *   Create any EC-derived joinclauses of form 'outer_em = inner_em'.
+ *
+ * This is used when building partition info for joinrel.
+ */
+List *
+generate_join_implied_equalities_for_all(PlannerInfo *root,
+ Relids join_relids,
+ Relids outer_relids,
+ Relids inner_relids)

I think we need to have more detailed comments about why we need this
separate function, we can also explain that
generate_join_implied_equalities function will avoid
the join clause if EC has the constant but for partition-wise join, we
need that clause too.


+ while ((i = bms_next_member(matching_ecs, i)) >= 0)
+ {
+ EquivalenceClass *ec = (EquivalenceClass *) list_nth(root->eq_classes, i);
+ List*outer_members = NIL;
+ List*inner_members = NIL;
+ ListCell   *lc1;
+
+ /* Do not consider this EC if it's ec_broken */
+ if (ec->ec_broken)
+ continue;
+
+ /* Single-member ECs won't generate any deductions */
+ if (list_length(ec->ec_members) <= 1)
+ continue;
+

I am wondering isn't it possible to just process the missing join
clause?  I mean 'generate_join_implied_equalities' has only skipped
the ECs which has const so
can't we create join clause only for those ECs and append it the
"Restrictlist" we already have?  I might be missing something?

-- 
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com




Re: A problem about partitionwise join

2019-09-10 Thread Richard Guo
Hi Alvaro,

Thank you for reviewing this patch.

On Wed, Sep 11, 2019 at 4:48 AM Alvaro Herrera from 2ndQuadrant <
alvhe...@alvh.no-ip.org> wrote:

> So in this patch, the input restrictlist is modified to include the
> clauses generated by generate_join_implied_equalities_for_all.  That
> doesn't seem okay -- doesn't it affect downstream usage of the
> restrictlist in the caller of set_joinrel_size_estimates?
>

Actually the joinclauses generated by
generate_join_implied_equalities_for_all only affects the restrictlist
used in have_partkey_equi_join to check equi-join conditions for
partition keys.  The input restrictlist would not be altered.


>
> I wonder if it's possible to do this by using the ECs directly in
> have_partkey_equi_join instead of using them to create fake join
> clauses.
>

Hmm.. I thought about this option and at last figured that what we need
to do in have_partkey_equi_join with the ECs is actually the same as in
generate_join_implied_equalities_for_all. Maybe I didn't think it
correctly.

Thanks
Richard


Re: A problem about partitionwise join

2019-09-10 Thread Alvaro Herrera from 2ndQuadrant
So in this patch, the input restrictlist is modified to include the
clauses generated by generate_join_implied_equalities_for_all.  That
doesn't seem okay -- doesn't it affect downstream usage of the
restrictlist in the caller of set_joinrel_size_estimates?

I wonder if it's possible to do this by using the ECs directly in
have_partkey_equi_join instead of using them to create fake join
clauses.

-- 
Álvaro Herrerahttps://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services




Re: A problem about partitionwise join

2019-08-30 Thread Etsuro Fujita
On Fri, Aug 30, 2019 at 12:15 PM Richard Guo  wrote:
> On Fri, Aug 30, 2019 at 2:08 AM Etsuro Fujita  wrote:
>> On Thu, Aug 29, 2019 at 6:45 PM Richard Guo  wrote:
>> > Attached is a patch as an attempt to address this issue. The idea is
>> > quite straightforward. When building partition info for joinrel, we
>> > generate any possible EC-derived joinclauses of form 'outer_em =
>> > inner_em', which will be used together with the original restrictlist to
>> > check if there exists an equi-join condition for each pair of partition
>> > keys.

>> Could you add the patch to the
>> upcoming CF so that it doesn’t get lost?

> Added this patch: https://commitfest.postgresql.org/24/2266/

Thanks!

Best regards,
Etsuro Fujita




Re: A problem about partitionwise join

2019-08-29 Thread Richard Guo
On Fri, Aug 30, 2019 at 2:08 AM Etsuro Fujita 
wrote:

> On Thu, Aug 29, 2019 at 6:45 PM Richard Guo  wrote:
> > On Wed, Aug 28, 2019 at 6:49 PM Etsuro Fujita 
> wrote:
> >> On Tue, Aug 27, 2019 at 4:57 PM Richard Guo  wrote:
> >> > Check the query below as a more illustrative example:
> >> >
> >> > create table p (k int, val int) partition by range(k);
> >> > create table p_1 partition of p for values from (1) to (10);
> >> > create table p_2 partition of p for values from (10) to (100);
> >> >
> >> > If we use quals 'foo.k = bar.k and foo.k = bar.val', we can generate
> >> > partitionwise join:
> >> >
> >> > # explain (costs off)
> >> > select * from p as foo join p as bar on foo.k = bar.k and foo.k =
> bar.val;
> >> >QUERY PLAN
> >> > -
> >> >  Append
> >> >->  Hash Join
> >> >  Hash Cond: (foo.k = bar.k)
> >> >  ->  Seq Scan on p_1 foo
> >> >  ->  Hash
> >> >->  Seq Scan on p_1 bar
> >> >  Filter: (k = val)
> >> >->  Hash Join
> >> >  Hash Cond: (foo_1.k = bar_1.k)
> >> >  ->  Seq Scan on p_2 foo_1
> >> >  ->  Hash
> >> >->  Seq Scan on p_2 bar_1
> >> >  Filter: (k = val)
> >> > (13 rows)
> >> >
> >> > But if we exchange the order of the two quals to 'foo.k = bar.val and
> >> > foo.k = bar.k', then partitionwise join cannot be generated any more,
> >> > because we only have joinclause 'foo.k = bar.val' as it first reached
> >> > score of 3. We have missed the joinclause on the partition key
> although
> >> > it does exist.
> >> >
> >> > # explain (costs off)
> >> > select * from p as foo join p as bar on foo.k = bar.val and foo.k =
> bar.k;
> >> >QUERY PLAN
> >> > -
> >> >  Hash Join
> >> >Hash Cond: (foo.k = bar.val)
> >> >->  Append
> >> >  ->  Seq Scan on p_1 foo
> >> >  ->  Seq Scan on p_2 foo_1
> >> >->  Hash
> >> >  ->  Append
> >> >->  Seq Scan on p_1 bar
> >> >  Filter: (val = k)
> >> >->  Seq Scan on p_2 bar_1
> >> >  Filter: (val = k)
> >> > (11 rows)
> >>
> >> I think it would be nice if we can address this issue.
>
> > Attached is a patch as an attempt to address this issue. The idea is
> > quite straightforward. When building partition info for joinrel, we
> > generate any possible EC-derived joinclauses of form 'outer_em =
> > inner_em', which will be used together with the original restrictlist to
> > check if there exists an equi-join condition for each pair of partition
> > keys.
>
> Thank you for the patch!  Will review.  Could you add the patch to the
> upcoming CF so that it doesn’t get lost?
>

Added this patch: https://commitfest.postgresql.org/24/2266/

Thanks
Richard


Re: A problem about partitionwise join

2019-08-29 Thread Etsuro Fujita
On Thu, Aug 29, 2019 at 6:45 PM Richard Guo  wrote:
> On Wed, Aug 28, 2019 at 6:49 PM Etsuro Fujita  wrote:
>> On Tue, Aug 27, 2019 at 4:57 PM Richard Guo  wrote:
>> > Check the query below as a more illustrative example:
>> >
>> > create table p (k int, val int) partition by range(k);
>> > create table p_1 partition of p for values from (1) to (10);
>> > create table p_2 partition of p for values from (10) to (100);
>> >
>> > If we use quals 'foo.k = bar.k and foo.k = bar.val', we can generate
>> > partitionwise join:
>> >
>> > # explain (costs off)
>> > select * from p as foo join p as bar on foo.k = bar.k and foo.k = bar.val;
>> >QUERY PLAN
>> > -
>> >  Append
>> >->  Hash Join
>> >  Hash Cond: (foo.k = bar.k)
>> >  ->  Seq Scan on p_1 foo
>> >  ->  Hash
>> >->  Seq Scan on p_1 bar
>> >  Filter: (k = val)
>> >->  Hash Join
>> >  Hash Cond: (foo_1.k = bar_1.k)
>> >  ->  Seq Scan on p_2 foo_1
>> >  ->  Hash
>> >->  Seq Scan on p_2 bar_1
>> >  Filter: (k = val)
>> > (13 rows)
>> >
>> > But if we exchange the order of the two quals to 'foo.k = bar.val and
>> > foo.k = bar.k', then partitionwise join cannot be generated any more,
>> > because we only have joinclause 'foo.k = bar.val' as it first reached
>> > score of 3. We have missed the joinclause on the partition key although
>> > it does exist.
>> >
>> > # explain (costs off)
>> > select * from p as foo join p as bar on foo.k = bar.val and foo.k = bar.k;
>> >QUERY PLAN
>> > -
>> >  Hash Join
>> >Hash Cond: (foo.k = bar.val)
>> >->  Append
>> >  ->  Seq Scan on p_1 foo
>> >  ->  Seq Scan on p_2 foo_1
>> >->  Hash
>> >  ->  Append
>> >->  Seq Scan on p_1 bar
>> >  Filter: (val = k)
>> >->  Seq Scan on p_2 bar_1
>> >  Filter: (val = k)
>> > (11 rows)
>>
>> I think it would be nice if we can address this issue.

> Attached is a patch as an attempt to address this issue. The idea is
> quite straightforward. When building partition info for joinrel, we
> generate any possible EC-derived joinclauses of form 'outer_em =
> inner_em', which will be used together with the original restrictlist to
> check if there exists an equi-join condition for each pair of partition
> keys.

Thank you for the patch!  Will review.  Could you add the patch to the
upcoming CF so that it doesn’t get lost?

Best regards,
Etsuro Fujita




Re: A problem about partitionwise join

2019-08-29 Thread Richard Guo
On Wed, Aug 28, 2019 at 6:49 PM Etsuro Fujita 
wrote:

> Hi,
>
> On Tue, Aug 27, 2019 at 4:57 PM Richard Guo  wrote:
> > Check the query below as a more illustrative example:
> >
> > create table p (k int, val int) partition by range(k);
> > create table p_1 partition of p for values from (1) to (10);
> > create table p_2 partition of p for values from (10) to (100);
> >
> > If we use quals 'foo.k = bar.k and foo.k = bar.val', we can generate
> > partitionwise join:
> >
> > # explain (costs off)
> > select * from p as foo join p as bar on foo.k = bar.k and foo.k =
> bar.val;
> >QUERY PLAN
> > -
> >  Append
> >->  Hash Join
> >  Hash Cond: (foo.k = bar.k)
> >  ->  Seq Scan on p_1 foo
> >  ->  Hash
> >->  Seq Scan on p_1 bar
> >  Filter: (k = val)
> >->  Hash Join
> >  Hash Cond: (foo_1.k = bar_1.k)
> >  ->  Seq Scan on p_2 foo_1
> >  ->  Hash
> >->  Seq Scan on p_2 bar_1
> >  Filter: (k = val)
> > (13 rows)
> >
> > But if we exchange the order of the two quals to 'foo.k = bar.val and
> > foo.k = bar.k', then partitionwise join cannot be generated any more,
> > because we only have joinclause 'foo.k = bar.val' as it first reached
> > score of 3. We have missed the joinclause on the partition key although
> > it does exist.
> >
> > # explain (costs off)
> > select * from p as foo join p as bar on foo.k = bar.val and foo.k =
> bar.k;
> >QUERY PLAN
> > -
> >  Hash Join
> >Hash Cond: (foo.k = bar.val)
> >->  Append
> >  ->  Seq Scan on p_1 foo
> >  ->  Seq Scan on p_2 foo_1
> >->  Hash
> >  ->  Append
> >->  Seq Scan on p_1 bar
> >  Filter: (val = k)
> >->  Seq Scan on p_2 bar_1
> >  Filter: (val = k)
> > (11 rows)
>
> I think it would be nice if we can address this issue.
>

Thank you.

Attached is a patch as an attempt to address this issue. The idea is
quite straightforward. When building partition info for joinrel, we
generate any possible EC-derived joinclauses of form 'outer_em =
inner_em', which will be used together with the original restrictlist to
check if there exists an equi-join condition for each pair of partition
keys.

Any comments are welcome!

Thanks
Richard


v1-0001-Fix-up-partitionwise-join.patch
Description: Binary data


Re: A problem about partitionwise join

2019-08-28 Thread Etsuro Fujita
Hi,

On Tue, Aug 27, 2019 at 4:57 PM Richard Guo  wrote:
> Check the query below as a more illustrative example:
>
> create table p (k int, val int) partition by range(k);
> create table p_1 partition of p for values from (1) to (10);
> create table p_2 partition of p for values from (10) to (100);
>
> If we use quals 'foo.k = bar.k and foo.k = bar.val', we can generate
> partitionwise join:
>
> # explain (costs off)
> select * from p as foo join p as bar on foo.k = bar.k and foo.k = bar.val;
>QUERY PLAN
> -
>  Append
>->  Hash Join
>  Hash Cond: (foo.k = bar.k)
>  ->  Seq Scan on p_1 foo
>  ->  Hash
>->  Seq Scan on p_1 bar
>  Filter: (k = val)
>->  Hash Join
>  Hash Cond: (foo_1.k = bar_1.k)
>  ->  Seq Scan on p_2 foo_1
>  ->  Hash
>->  Seq Scan on p_2 bar_1
>  Filter: (k = val)
> (13 rows)
>
> But if we exchange the order of the two quals to 'foo.k = bar.val and
> foo.k = bar.k', then partitionwise join cannot be generated any more,
> because we only have joinclause 'foo.k = bar.val' as it first reached
> score of 3. We have missed the joinclause on the partition key although
> it does exist.
>
> # explain (costs off)
> select * from p as foo join p as bar on foo.k = bar.val and foo.k = bar.k;
>QUERY PLAN
> -
>  Hash Join
>Hash Cond: (foo.k = bar.val)
>->  Append
>  ->  Seq Scan on p_1 foo
>  ->  Seq Scan on p_2 foo_1
>->  Hash
>  ->  Append
>->  Seq Scan on p_1 bar
>  Filter: (val = k)
>->  Seq Scan on p_2 bar_1
>  Filter: (val = k)
> (11 rows)

I think it would be nice if we can address this issue.

Best regards,
Etsuro Fujita




Re: A problem about partitionwise join

2019-08-27 Thread Richard Guo
On Tue, Aug 27, 2019 at 8:51 AM Amit Langote 
wrote:

> Hi Richard,
>
> On Mon, Aug 26, 2019 at 6:33 PM Richard Guo  wrote:
> >
> > Hi All,
> >
> > To generate partitionwise join, we need to make sure there exists an
> > equi-join condition for each pair of partition keys, which is performed
> > by have_partkey_equi_join(). This makes sense and works well.
> >
> > But if, let's say, one certain pair of partition keys (foo.k = bar.k)
> > has formed an equivalence class containing consts, no join clause would
> > be generated for it, since we have already generated 'foo.k = const' and
> > 'bar.k = const' and pushed them into the proper restrictions earlier.
> >
> > This will make partitionwise join fail to be planned if there are
> > multiple partition keys and the pushed-down restrictions 'xxx = const'
> > fail to prune away any partitions.
> >
> > Consider the examples below:
> >
> > create table p (k1 int, k2 int, val int) partition by range(k1,k2);
> > create table p_1 partition of p for values from (1,1) to (10,100);
> > create table p_2 partition of p for values from (10,100) to (20,200);
> >
> > If we are joining on each pair of partition keys, we can generate
> > partitionwise join:
> >
> > # explain (costs off)
> > select * from p as foo join p as bar on foo.k1 = bar.k1 and foo.k2 =
> bar.k2;
> >   QUERY PLAN
> > --
> >  Append
> >->  Hash Join
> >  Hash Cond: ((foo.k1 = bar.k1) AND (foo.k2 = bar.k2))
> >  ->  Seq Scan on p_1 foo
> >  ->  Hash
> >->  Seq Scan on p_1 bar
> >->  Hash Join
> >  Hash Cond: ((foo_1.k1 = bar_1.k1) AND (foo_1.k2 = bar_1.k2))
> >  ->  Seq Scan on p_2 foo_1
> >  ->  Hash
> >->  Seq Scan on p_2 bar_1
> > (11 rows)
> >
> > But if we add another qual 'foo.k2 = const', we will be unable to
> > generate partitionwise join any more, because have_partkey_equi_join()
> > thinks not every partition key has an equi-join condition.
> >
> > # explain (costs off)
> > select * from p as foo join p as bar on foo.k1 = bar.k1 and foo.k2 =
> bar.k2 and foo.k2 = 16;
> >QUERY PLAN
> > -
> >  Hash Join
> >Hash Cond: (foo.k1 = bar.k1)
> >->  Append
> >  ->  Seq Scan on p_1 foo
> >Filter: (k2 = 16)
> >  ->  Seq Scan on p_2 foo_1
> >Filter: (k2 = 16)
> >->  Hash
> >  ->  Append
> >->  Seq Scan on p_1 bar
> >  Filter: (k2 = 16)
> >->  Seq Scan on p_2 bar_1
> >  Filter: (k2 = 16)
> > (13 rows)
> >
> > Is this a problem?
>
> Perhaps.  Maybe it has to do with the way have_partkey_equi_join() has
> been coded.  If it was coded such that it figured out on its own that
> the equivalence (foo.k2, bar.k2, ...) does exist, then that would
> allow partitionwise join to occur, which I think would be OK to do.
> But maybe I'm missing something.
>
>
This should be caused by how we deduce join clauses from equivalence
classes. ECs containing consts will not be considered so we cannot
generate (foo.k2 = bar.k2) for the query above.

In addition, when generating join clauses from equivalence classes, we
only select the joinclause with the 'best score', or the first
joinclause with a score of 3. This may make us miss some joinclause on
partition keys.

Check the query below as a more illustrative example:

create table p (k int, val int) partition by range(k);
create table p_1 partition of p for values from (1) to (10);
create table p_2 partition of p for values from (10) to (100);

If we use quals 'foo.k = bar.k and foo.k = bar.val', we can generate
partitionwise join:

# explain (costs off)
select * from p as foo join p as bar on foo.k = bar.k and foo.k = bar.val;
   QUERY PLAN
-
 Append
   ->  Hash Join
 Hash Cond: (foo.k = bar.k)
 ->  Seq Scan on p_1 foo
 ->  Hash
   ->  Seq Scan on p_1 bar
 Filter: (k = val)
   ->  Hash Join
 Hash Cond: (foo_1.k = bar_1.k)
 ->  Seq Scan on p_2 foo_1
 ->  Hash
   ->  Seq Scan on p_2 bar_1
 Filter: (k = val)
(13 rows)

But if we exchange the order of the two quals to 'foo.k = bar.val and
foo.k = bar.k', then partitionwise join cannot be generated any more,
because we only have joinclause 'foo.k = bar.val' as it first reached
score of 3. We have missed the joinclause on the partition key although
it does exist.

# explain (costs off)
select * from p as foo join p as bar on foo.k = bar.val and foo.k = bar.k;
   QUERY PLAN
-
 Hash Join
   Hash Cond: (foo.k = bar.val)
   ->  Append
 ->  Seq Scan on p_1 foo
 ->  Seq Scan on p_2 foo_1
   ->  Hash
 ->  Append
   ->  

Re: A problem about partitionwise join

2019-08-26 Thread Amit Langote
Hi Richard,

On Mon, Aug 26, 2019 at 6:33 PM Richard Guo  wrote:
>
> Hi All,
>
> To generate partitionwise join, we need to make sure there exists an
> equi-join condition for each pair of partition keys, which is performed
> by have_partkey_equi_join(). This makes sense and works well.
>
> But if, let's say, one certain pair of partition keys (foo.k = bar.k)
> has formed an equivalence class containing consts, no join clause would
> be generated for it, since we have already generated 'foo.k = const' and
> 'bar.k = const' and pushed them into the proper restrictions earlier.
>
> This will make partitionwise join fail to be planned if there are
> multiple partition keys and the pushed-down restrictions 'xxx = const'
> fail to prune away any partitions.
>
> Consider the examples below:
>
> create table p (k1 int, k2 int, val int) partition by range(k1,k2);
> create table p_1 partition of p for values from (1,1) to (10,100);
> create table p_2 partition of p for values from (10,100) to (20,200);
>
> If we are joining on each pair of partition keys, we can generate
> partitionwise join:
>
> # explain (costs off)
> select * from p as foo join p as bar on foo.k1 = bar.k1 and foo.k2 = bar.k2;
>   QUERY PLAN
> --
>  Append
>->  Hash Join
>  Hash Cond: ((foo.k1 = bar.k1) AND (foo.k2 = bar.k2))
>  ->  Seq Scan on p_1 foo
>  ->  Hash
>->  Seq Scan on p_1 bar
>->  Hash Join
>  Hash Cond: ((foo_1.k1 = bar_1.k1) AND (foo_1.k2 = bar_1.k2))
>  ->  Seq Scan on p_2 foo_1
>  ->  Hash
>->  Seq Scan on p_2 bar_1
> (11 rows)
>
> But if we add another qual 'foo.k2 = const', we will be unable to
> generate partitionwise join any more, because have_partkey_equi_join()
> thinks not every partition key has an equi-join condition.
>
> # explain (costs off)
> select * from p as foo join p as bar on foo.k1 = bar.k1 and foo.k2 = bar.k2 
> and foo.k2 = 16;
>QUERY PLAN
> -
>  Hash Join
>Hash Cond: (foo.k1 = bar.k1)
>->  Append
>  ->  Seq Scan on p_1 foo
>Filter: (k2 = 16)
>  ->  Seq Scan on p_2 foo_1
>Filter: (k2 = 16)
>->  Hash
>  ->  Append
>->  Seq Scan on p_1 bar
>  Filter: (k2 = 16)
>->  Seq Scan on p_2 bar_1
>  Filter: (k2 = 16)
> (13 rows)
>
> Is this a problem?

Perhaps.  Maybe it has to do with the way have_partkey_equi_join() has
been coded.  If it was coded such that it figured out on its own that
the equivalence (foo.k2, bar.k2, ...) does exist, then that would
allow partitionwise join to occur, which I think would be OK to do.
But maybe I'm missing something.

Thanks,
Amit