Uploaded image for project: 'Calcite'
  1. Calcite
  2. CALCITE-3890

Derive IS NOT NULL filter for the inputs of inner join

Details

    • Improvement
    • Status: Closed
    • Major
    • Resolution: Fixed
    • None
    • 1.31.0
    • core

    Description

      We can infer IS NOT NULL predicate from join which implies some columns may not be null. For instance, 

       

      select * from a join b on a.id = b.id;
      

      we can infer a.id is not null/b.id is not null and push down them into the child node of the join. Then it becomes

      select * from (select* from a where id is null) t1 join (select * from b where id is not null) on t1.id = t2.id;
      

       

       

      Attachments

        Issue Links

          Activity

            julianhyde Julian Hyde added a comment -

            I agree. And we can also push down filters: ‘id is not null’ to both inputs, in this case.

            Hopefully we can use existing logic, e.g. class Strong, for this deduction.

            julianhyde Julian Hyde added a comment - I agree. And we can also push down filters: ‘id is not null’ to both inputs, in this case. Hopefully we can use existing logic, e.g. class Strong , for this deduction.

            Chunwei Lei: in Hive we have a rule which somewhat does this
            I was sure that a Calcite rule adds all the is not null conditions (and it is)...but apparently it was not contributed back; and it's still only available inside Hive.
            it might worth taking a look at it: https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinAddNotNullRule.java

            kgyrtkirk Zoltan Haindrich added a comment - Chunwei Lei : in Hive we have a rule which somewhat does this I was sure that a Calcite rule adds all the is not null conditions (and it is)...but apparently it was not contributed back; and it's still only available inside Hive. it might worth taking a look at it: https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinAddNotNullRule.java
            vgarg Vineet Garg added a comment -

            As kgyrtkirk pointed out we have doing this in Hive for a while and it is very useful rule. I will be more than glad to work on contributing this code to calcite.

            vgarg Vineet Garg added a comment - As kgyrtkirk pointed out we have doing this in Hive for a while and it is very useful rule. I will be more than glad to work on contributing this code to calcite.
            Chunwei Lei Chunwei Lei added a comment - - edited

            vgarg, welcome to contribute. I assigned the issue to you.

            Chunwei Lei Chunwei Lei added a comment - - edited vgarg , welcome to contribute. I assigned the issue to you.
            Chunwei Lei Chunwei Lei added a comment -

            vgarg, kgyrtkirk Instead of using a rule to add ISNOTNULL predicate which is the way in Hive, I am wondering if we can do it when creating the JOIN operator. Maybe we can add a configuration to indicate whether to add ISNOTNULL predicate when creating the JOIN operator using RelBuilder.

            Chunwei Lei Chunwei Lei added a comment - vgarg , kgyrtkirk Instead of using a rule to add ISNOTNULL predicate which is the way in Hive, I am wondering if we can do it when creating the JOIN operator. Maybe we can add a configuration to indicate whether to add ISNOTNULL predicate when creating the JOIN operator using RelBuilder.
            julianhyde Julian Hyde added a comment -

            I'm worried about adding a configuration parameter. We don't want to end up with hundreds of them. If it makes sense, let's just do it.

            julianhyde Julian Hyde added a comment - I'm worried about adding a configuration parameter. We don't want to end up with hundreds of them. If it makes sense, let's just do it.
            Chunwei Lei Chunwei Lei added a comment - - edited

            vgarg any progress on this work? I would like to take over if you have no time.

            Chunwei Lei Chunwei Lei added a comment - - edited vgarg any progress on this work? I would like to take over if you have no time.
            Chunwei Lei Chunwei Lei added a comment -

            I opened a PR for this feature: https://github.com/apache/calcite/pull/2800. It would be great if someone can review it.

            Chunwei Lei Chunwei Lei added a comment - I opened a PR for this feature: https://github.com/apache/calcite/pull/2800.  It would be great if someone can review it.
            julianhyde Julian Hyde added a comment -

            The description does not say that the filter is a relational expression, nor does it say that the filter is before the join rather than after.

            I understand why FULL join is not covered, but could you apply the rule to the non-NULL-generating sides of LEFT and RIGHT join? E.g.

            Emp e LEFT JOIN Dept d USING (deptno)

            becomes

            (SELECT * FROM Emp WHERE deptno IS NOT NULL) LEFT JOIN Dept d USING (deptno)

            Were you able to use Strong as I suggested? It should be easy to cover cases such as

            Emp e JOIN Dept d ON e.deptno > d.deptno

            or

            Emp e JOIN Dept d ON e.deptno + d.deptno < 10
            julianhyde Julian Hyde added a comment - The description does not say that the filter is a relational expression, nor does it say that the filter is before the join rather than after. I understand why FULL join is not covered, but could you apply the rule to the non-NULL-generating sides of LEFT and RIGHT join? E.g. Emp e LEFT JOIN Dept d USING (deptno) becomes (SELECT * FROM Emp WHERE deptno IS NOT NULL) LEFT JOIN Dept d USING (deptno) Were you able to use Strong as I suggested? It should be easy to cover cases such as Emp e JOIN Dept d ON e.deptno > d.deptno or Emp e JOIN Dept d ON e.deptno + d.deptno < 10
            Chunwei Lei Chunwei Lei added a comment -

            Emp e LEFT JOIN Dept d USING (deptno)

            becomes

            (SELECT * FROM Emp WHERE deptno IS NOT NULL) LEFT JOIN Dept d USING (deptno)

            AFAIK, this transformation is wrong. For left/right join, the non-NULL-generating sides may have null values and they can not be filtered in advance.

            Chunwei Lei Chunwei Lei added a comment - Emp e LEFT JOIN Dept d USING (deptno) becomes (SELECT * FROM Emp WHERE deptno IS NOT NULL) LEFT JOIN Dept d USING (deptno) AFAIK, this transformation is wrong. For left/right join, the non-NULL-generating sides may have null values and they can not be filtered in advance.
            Chunwei Lei Chunwei Lei added a comment -

            The description does not say that the filter is a relational expression, nor does it say that the filter is before the join rather than after.

            The description has been updated.

            Chunwei Lei Chunwei Lei added a comment - The description does not say that the filter is a relational expression, nor does it say that the filter is before the join rather than after. The description has been updated.
            wojustme Xurenhe added a comment -

            I left some doubts in the pr.
            I am curious about how to analyze the join condition.

            such as: 

            join on t1.id IS NOT DISTINCT FROM t2.id 
            wojustme Xurenhe added a comment - I left some doubts in the pr. I am curious about how to analyze the join condition. such as:  join on t1.id IS NOT DISTINCT FROM t2.id
            Chunwei Lei Chunwei Lei added a comment -

            I am trying to use Strong to analyze the join condition, as Julian said.

            Chunwei Lei Chunwei Lei added a comment - I am trying to use Strong to analyze the join condition, as Julian said.
            julianhyde Julian Hyde added a comment -

            In the particular case of IS NOT DISTINCT FROM it is not safe to add IS NOT NULL filters. For example, Emp e JOIN Dept d ON e.deptno IS NOT DISTINCT FROM d.deptno will return a row where both e.deptno and d.deptno are both null. Hopefully Strong knows this.

            julianhyde Julian Hyde added a comment - In the particular case of IS NOT DISTINCT FROM it is not safe to add IS NOT NULL filters. For example, Emp e JOIN Dept d ON e.deptno IS NOT DISTINCT FROM d.deptno will return a row where both e.deptno and d.deptno are both null. Hopefully Strong knows this.
            Chunwei Lei Chunwei Lei added a comment -

            Exactly. I think Strong can handle it well.

            Chunwei Lei Chunwei Lei added a comment - Exactly. I think Strong can handle it well.
            Chunwei Lei Chunwei Lei added a comment - Fixed in https://github.com/apache/calcite/commit/acf82f7784823c30fb7a64e905c3acacd0ed4f2b .

            Hey Chunwei Lei , I didn't notice that this Jira was resolved, thats great! I see that there was a discussion before to take inspiration from HiveJoinAddNotNullRule . Can you clarify the similarities/differences (if any) between the new rule that you added and the one used in Hive. I am asking cause ideally I would like to avoid maintaining the same code in multiple places.

            zabetak Stamatis Zampetakis added a comment - Hey Chunwei Lei , I didn't notice that this Jira was resolved, thats great! I see that there was a discussion before to take inspiration from HiveJoinAddNotNullRule . Can you clarify the similarities/differences (if any) between the new rule that you added and the one used in Hive. I am asking cause ideally I would like to avoid maintaining the same code in multiple places.
            Chunwei Lei Chunwei Lei added a comment -

            Thank you for your attention, zabetak. IMHO, the most important point about how to implement this rule is how to avoid applying the rule infinitely, which is the big difference between the new rule I added and the one used in Hive. To achieve this goal, Hive uses some extra data structures to save the generated predicates, while the new rule I added just uses MetadataQuery and RexSimplify to see whether the ISNOTNULL predicate is redundant or not.

            Chunwei Lei Chunwei Lei added a comment - Thank you for your attention, zabetak . IMHO, the most important point about how to implement this rule is how to avoid applying the rule infinitely, which is the big difference between the new rule I added and the one used in Hive. To achieve this goal, Hive uses some extra data structures to save the generated predicates, while the new rule I added just uses MetadataQuery and RexSimplify to see whether the ISNOTNULL predicate is redundant or not.
            sereda Andrei Sereda added a comment -

            Resolved in release 1.31.0

            sereda Andrei Sereda added a comment - Resolved in release 1.31.0

            People

              Chunwei Lei Chunwei Lei
              Chunwei Lei Chunwei Lei
              Votes:
              0 Vote for this issue
              Watchers:
              10 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved:

                Time Tracking

                  Estimated:
                  Original Estimate - Not Specified
                  Not Specified
                  Remaining:
                  Remaining Estimate - 0h
                  0h
                  Logged:
                  Time Spent - 4h 50m
                  4h 50m