[ 
https://issues.apache.org/jira/browse/PHOENIX-5389?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

chenglei updated PHOENIX-5389:
------------------------------
    Description: 
Given following tables:
{code:java}
   create table order_table
   (order_id varchar(15) not null primary key, 
    customer_id varchar(10), 
    item_id varchar(10), 
    price integer, 
    quantity integer, 
    date timestamp)

   create table item_table
   (item_id varchar(10) not null primary key, 
     name varchar, 
     price integer, 
     discount1 integer, 
     discount2 integer, 
     supplier_id varchar(10), 
     description varchar)

   create table supplier_table
    (supplier_id varchar(10) not null primary key, 
      name varchar, 
      phone varchar(12), 
      address varchar, 
      loc_id varchar(5))
{code}

for following sql :
{code:java}
select /*+ USE_SORT_MERGE_JOIN */  o.order_id,i.item_id, s.supplier_id, s.name 
from 
supplier_table s inner join item_table i on  s.supplier_id = i.supplier_id 
inner join order_table o on  i.item_id = o.item_id 
where  s.address = 'hai' or i.supplier_id != 'medi'
{code}

When using SortMergeJoin,  the above join is composited as: 
{{(supplier_table s inner join item_table i)  inner join order_table}}.
The where condition {{s.address = 'hai' or i.supplier_id != 'medi'}}  could be 
pushed to the first join  {{(supplier_table  inner join item_table)}},because 
it only references the columns of {{supplier_table}} and {{item_table}}, 
but unfortunately, this where condition  {{s.address = 'hai' or i.supplier_id 
!= 'medi'}} does not  be pushed down to SubJoinTable as expected,  and it 
becomes the {{postFilter}} of the whole join {{(supplier_table s inner join 
item_table i)  inner join order_table}}.

This problem is caused by the following code line 431 in 
QueryCompiler.compileJoinQuery:
{code:java}
430    case SORT_MERGE: {
431                JoinTable lhsJoin = 
joinTable.getSubJoinTableWithoutPostFilters();
432                JoinSpec lastJoinSpec = joinSpecs.get(joinSpecs.size() - 1);
433                JoinType type = lastJoinSpec.getType();
434                JoinTable rhsJoin = lastJoinSpec.getJoinTable();
{code}

At the beginning of SORT_MERGE, line 431 create a new JoinTable for  
{{(supplier_table  inner join item_table)}} by 
{{JoinTable.getSubJoinTableWithoutPostFilters()}}, 
{code:java}
441  public JoinTable getSubJoinTableWithoutPostFilters() {
442            return joinSpecs.size() > 1 ? new JoinTable(table, 
joinSpecs.subList(0, joinSpecs.size() - 1)) :
443                new JoinTable(table);
444        }
{code}
{{JoinTable.getSubJoinTableWithoutPostFilters()}}  only create the new 
JoinTable and does not to push down  the where condition  {{s.address = 'hai' 
or i.supplier_id != 'medi'}} to the new created JoinTable.


  was:
Given following tables:
{code:java}
   create table order_table
   (order_id varchar(15) not null primary key, 
    customer_id varchar(10), 
    item_id varchar(10), 
    price integer, 
    quantity integer, 
    date timestamp)

   create table item_table
   (item_id varchar(10) not null primary key, 
     name varchar, 
     price integer, 
     discount1 integer, 
     discount2 integer, 
     supplier_id varchar(10), 
     description varchar)

   create table supplier_table
    (supplier_id varchar(10) not null primary key, 
      name varchar, 
      phone varchar(12), 
      address varchar, 
      loc_id varchar(5))
{code}

for following sql :
{code:java}
select /*+ USE_SORT_MERGE_JOIN */  o.order_id,i.item_id, s.supplier_id, s.name 
from 
supplier_table s inner join item_table i on  s.supplier_id = i.supplier_id 
inner join order_table o on  i.item_id = o.item_id 
where  s.address = 'hai' or i.supplier_id != 'medi'
{code}

for SortMergeJoin,  the above join is composited as: 
{{(supplier_table s inner join item_table i)  inner join order_table}}.
The where condition {{s.address = 'hai' or i.supplier_id != 'medi'}}  could be 
pushed to the first join  {{(supplier_table  inner join item_table)}},because 
it only references the columns of {{supplier_table}} and {{item_table}}, 
but unfortunately, this where condition  {{s.address = 'hai' or i.supplier_id 
!= 'medi'}} does not pushed down to SubJoinTable as expected,  and it as the 
postFilter of the whole join {{(supplier_table s inner join item_table i)  
inner join order_table}}.





> Push down PostFilter to Sub-JoinTable for SortMergeJoin and NoStarJoin
> ----------------------------------------------------------------------
>
>                 Key: PHOENIX-5389
>                 URL: https://issues.apache.org/jira/browse/PHOENIX-5389
>             Project: Phoenix
>          Issue Type: Improvement
>    Affects Versions: 4.14.2
>            Reporter: chenglei
>            Assignee: chenglei
>            Priority: Major
>             Fix For: 4.15.0, 5.1.0
>
>
> Given following tables:
> {code:java}
>    create table order_table
>    (order_id varchar(15) not null primary key, 
>     customer_id varchar(10), 
>     item_id varchar(10), 
>     price integer, 
>     quantity integer, 
>     date timestamp)
>    create table item_table
>    (item_id varchar(10) not null primary key, 
>      name varchar, 
>      price integer, 
>      discount1 integer, 
>      discount2 integer, 
>      supplier_id varchar(10), 
>      description varchar)
>    create table supplier_table
>     (supplier_id varchar(10) not null primary key, 
>       name varchar, 
>       phone varchar(12), 
>       address varchar, 
>       loc_id varchar(5))
> {code}
> for following sql :
> {code:java}
> select /*+ USE_SORT_MERGE_JOIN */  o.order_id,i.item_id, s.supplier_id, 
> s.name from 
> supplier_table s inner join item_table i on  s.supplier_id = i.supplier_id 
> inner join order_table o on  i.item_id = o.item_id 
> where  s.address = 'hai' or i.supplier_id != 'medi'
> {code}
> When using SortMergeJoin,  the above join is composited as: 
> {{(supplier_table s inner join item_table i)  inner join order_table}}.
> The where condition {{s.address = 'hai' or i.supplier_id != 'medi'}}  could 
> be pushed to the first join  {{(supplier_table  inner join 
> item_table)}},because it only references the columns of {{supplier_table}} 
> and {{item_table}}, 
> but unfortunately, this where condition  {{s.address = 'hai' or i.supplier_id 
> != 'medi'}} does not  be pushed down to SubJoinTable as expected,  and it 
> becomes the {{postFilter}} of the whole join {{(supplier_table s inner join 
> item_table i)  inner join order_table}}.
> This problem is caused by the following code line 431 in 
> QueryCompiler.compileJoinQuery:
> {code:java}
> 430    case SORT_MERGE: {
> 431                JoinTable lhsJoin = 
> joinTable.getSubJoinTableWithoutPostFilters();
> 432                JoinSpec lastJoinSpec = joinSpecs.get(joinSpecs.size() - 
> 1);
> 433                JoinType type = lastJoinSpec.getType();
> 434                JoinTable rhsJoin = lastJoinSpec.getJoinTable();
> {code}
> At the beginning of SORT_MERGE, line 431 create a new JoinTable for  
> {{(supplier_table  inner join item_table)}} by 
> {{JoinTable.getSubJoinTableWithoutPostFilters()}}, 
> {code:java}
> 441  public JoinTable getSubJoinTableWithoutPostFilters() {
> 442            return joinSpecs.size() > 1 ? new JoinTable(table, 
> joinSpecs.subList(0, joinSpecs.size() - 1)) :
> 443                new JoinTable(table);
> 444        }
> {code}
> {{JoinTable.getSubJoinTableWithoutPostFilters()}}  only create the new 
> JoinTable and does not to push down  the where condition  {{s.address = 'hai' 
> or i.supplier_id != 'medi'}} to the new created JoinTable.



--
This message was sent by Atlassian JIRA
(v7.6.14#76016)

Reply via email to