[ 
https://issues.apache.org/jira/browse/HIVE-22256?focusedWorklogId=429726&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-429726
 ]

ASF GitHub Bot logged work on HIVE-22256:
-----------------------------------------

                Author: ASF GitHub Bot
            Created on: 01/May/20 23:38
            Start Date: 01/May/20 23:38
    Worklog Time Spent: 10m 
      Work Description: vineetgarg02 commented on a change in pull request 
#1002:
URL: https://github.com/apache/hive/pull/1002#discussion_r418778365



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
##########
@@ -2463,6 +2473,15 @@ private RelNode applyPostJoinOrderingTransform(RelNode 
basePlan, RelMetadataProv
                 HiveProjectSortExchangeTransposeRule.INSTANCE, 
HiveProjectMergeRule.INSTANCE);
       }
 
+      // 10. We need to expand IN/BETWEEN expressions when loading a 
materialized view

Review comment:
       Why do we need to call these rules again in post ordering?

##########
File path: 
ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveInBetweenExpandRule.java
##########
@@ -0,0 +1,198 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.optimizer.calcite.rules;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelFactories;
+import org.apache.hadoop.hive.ql.optimizer.calcite.translator.RexNodeConverter;
+
+/**
+ * This class contains rules to rewrite IN/BETWEEN clauses into their
+ * corresponding AND/OR versions.
+ * It is the counterpart to {@link HivePointLookupOptimizerRule}.
+ */
+public class HiveInBetweenExpandRule {
+
+  public static final FilterRule FILTER_INSTANCE = new FilterRule();
+  public static final JoinRule JOIN_INSTANCE = new JoinRule();
+  public static final ProjectRule PROJECT_INSTANCE = new ProjectRule();
+
+  /** Rule adapter to apply the transformation to Filter conditions. */
+  private static class FilterRule extends RelOptRule {
+
+    FilterRule() {
+      super(operand(Filter.class, any()), HiveRelFactories.HIVE_BUILDER, null);
+    }
+
+    @Override
+    public void onMatch(RelOptRuleCall call) {
+      final Filter filter = call.rel(0);
+      RexInBetweenExpander expander = new RexInBetweenExpander(
+          filter.getCluster().getRexBuilder());
+      RexNode condition = expander.apply(filter.getCondition());
+
+      if (!expander.modified) {
+        return;
+      }
+
+      RelNode newFilter = filter.copy(filter.getTraitSet(),
+          filter.getInput(), condition);
+
+      call.transformTo(newFilter);
+    }
+  }
+
+  /** Rule adapter to apply the transformation to Join conditions. */
+  private static class JoinRule extends RelOptRule {
+
+    JoinRule() {
+      super(operand(Join.class, any()), HiveRelFactories.HIVE_BUILDER, null);
+    }
+
+    @Override
+    public void onMatch(RelOptRuleCall call) {
+      final Join join = call.rel(0);
+      RexInBetweenExpander expander = new RexInBetweenExpander(
+          join.getCluster().getRexBuilder());
+      RexNode condition = expander.apply(join.getCondition());
+
+      if (!expander.modified) {
+        return;
+      }
+
+      RelNode newJoin = join.copy(join.getTraitSet(),
+          condition,
+          join.getLeft(),
+          join.getRight(),
+          join.getJoinType(),
+          join.isSemiJoinDone());
+
+      call.transformTo(newJoin);
+    }
+  }
+
+  /** Rule adapter to apply the transformation to Project expressions. */
+  private static class ProjectRule extends RelOptRule {
+
+    ProjectRule() {
+      super(operand(Project.class, any()), HiveRelFactories.HIVE_BUILDER, 
null);
+    }
+
+    @Override
+    public void onMatch(RelOptRuleCall call) {
+      final Project project = call.rel(0);
+      RexInBetweenExpander expander = new RexInBetweenExpander(
+          project.getCluster().getRexBuilder());
+      List<RexNode> newProjects = new ArrayList<>();
+      for (RexNode expr : project.getProjects()) {
+        newProjects.add(expander.apply(expr));
+      }
+
+      if (!expander.modified) {
+        return;
+      }
+
+      Project newProject = project.copy(project.getTraitSet(),
+          project.getInput(), newProjects, project.getRowType());
+
+      call.transformTo(newProject);
+    }
+  }
+
+
+  /**
+   * Class that transforms IN/BETWEEN clauses in an expression.
+   * If any call is modified, the modified flag will be set to
+   * true after its execution.
+   */
+  private static final class RexInBetweenExpander extends RexShuttle {
+
+    private final RexBuilder rexBuilder;
+    private boolean modified;
+
+    private RexInBetweenExpander(RexBuilder rexBuilder) {
+      this.rexBuilder = rexBuilder;
+      this.modified = false;
+    }
+
+    @Override
+    public RexNode visitCall(final RexCall call) {
+      switch (call.getKind()) {
+      case AND: {
+        boolean[] update = {false};
+        List<RexNode> newOperands = visitList(call.operands, update);
+        if (update[0]) {
+          return RexUtil.composeConjunction(rexBuilder, newOperands);
+        }
+        return call;
+      }
+      case OR: {
+        boolean[] update = {false};
+        List<RexNode> newOperands = visitList(call.operands, update);
+        if (update[0]) {
+          return RexUtil.composeDisjunction(rexBuilder, newOperands);
+        }
+        return call;
+      }
+      case IN: {
+        List<RexNode> newOperands = RexNodeConverter.transformInToOrOperands(
+            call.getOperands(), rexBuilder);
+        if (newOperands == null) {
+          // We could not execute transformation, return expression
+          return call;
+        }
+        modified = true;
+        if (newOperands.size() > 1) {
+          return rexBuilder.makeCall(SqlStdOperatorTable.OR, newOperands);
+        }
+        return newOperands.get(0);
+      }
+      case BETWEEN: {
+        List<RexNode> newOperands = RexNodeConverter.rewriteBetweenChildren(
+            call.getOperands(), rexBuilder);
+        modified = true;
+        if (call.getOperands().get(0).isAlwaysTrue()) {

Review comment:
       I am not sure if I understand this. In what case BETWEEN is rewritten 
into OR?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 429726)
    Time Spent: 20m  (was: 10m)

> Rewriting fails when `IN` clause has items in different order in MV and query.
> ------------------------------------------------------------------------------
>
>                 Key: HIVE-22256
>                 URL: https://issues.apache.org/jira/browse/HIVE-22256
>             Project: Hive
>          Issue Type: Sub-task
>          Components: CBO, Materialized views
>    Affects Versions: 3.1.2
>            Reporter: Steve Carlin
>            Assignee: Jesus Camacho Rodriguez
>            Priority: Major
>              Labels: pull-request-available
>         Attachments: HIVE-22256.01.patch, HIVE-22256.patch, expr2.sql
>
>          Time Spent: 20m
>  Remaining Estimate: 0h
>
> Rewriting fails on following materialized view and query (script is also 
> attached):
> create materialized view view2 stored as orc as (select prod_id, cust_id, 
> store_id, sale_date, qty, amt, descr from sales where cust_id in (1,2,3,4,5));
> explain extended select prod_id, cust_id  from sales where cust_id in 
> (5,1,2,3,4);



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to