You are viewing a plain text version of this content. The canonical link for it is here.
Posted to gitbox@hive.apache.org by GitBox <gi...@apache.org> on 2021/02/08 16:12:16 UTC

[GitHub] [hive] soumyakanti3578 opened a new pull request #1959: [DRAFT] Upgrade calcite to 1.25

soumyakanti3578 opened a new pull request #1959:
URL: https://github.com/apache/hive/pull/1959


   This is a draft PR with changes to upgrade calcite to 1.25


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] zabetak commented on a change in pull request #1959: [DRAFT] Upgrade calcite to 1.25

Posted by GitBox <gi...@apache.org>.
zabetak commented on a change in pull request #1959:
URL: https://github.com/apache/hive/pull/1959#discussion_r592206948



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
##########
@@ -520,6 +520,8 @@ public static void closeCurrent() {
    */
   private Hive(HiveConf c, boolean doRegisterAllFns) throws HiveException {
     conf = c;
+    // turn off calcite rexnode normalization
+    System.setProperty("calcite.enable.rexnode.digest.normalize", "false");

Review comment:
       Maybe the best option in terms of code changes and to be sure that the property is going to be taken into account is to add a `saffron.properties` file in the classpath and put the property in it.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] jcamachor commented on a change in pull request #1959: [DRAFT] Upgrade calcite to 1.25

Posted by GitBox <gi...@apache.org>.
jcamachor commented on a change in pull request #1959:
URL: https://github.com/apache/hive/pull/1959#discussion_r592028619



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
##########
@@ -520,6 +520,8 @@ public static void closeCurrent() {
    */
   private Hive(HiveConf c, boolean doRegisterAllFns) throws HiveException {
     conf = c;
+    // turn off calcite rexnode normalization
+    System.setProperty("calcite.enable.rexnode.digest.normalize", "false");

Review comment:
       I think this should be set here? https://github.com/apache/hive/blob/master/pom.xml#L1512




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] soumyakanti3578 commented on a change in pull request #1959: [DRAFT] Upgrade calcite to 1.25

Posted by GitBox <gi...@apache.org>.
soumyakanti3578 commented on a change in pull request #1959:
URL: https://github.com/apache/hive/pull/1959#discussion_r578818462



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/cost/HiveVolcanoPlanner.java
##########
@@ -85,7 +85,6 @@ public void registerClass(RelNode node) {
       addRule(HiveDruidRules.FILTER_PROJECT_TRANSPOSE);
       addRule(HiveDruidRules.SORT_PROJECT_TRANSPOSE);
       addRule(HiveDruidRules.SORT);
-      addRule(HiveDruidRules.PROJECT_SORT_TRANSPOSE);

Review comment:
       Double check this




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] zabetak commented on a change in pull request #1959: [DRAFT] Upgrade calcite to 1.25

Posted by GitBox <gi...@apache.org>.
zabetak commented on a change in pull request #1959:
URL: https://github.com/apache/hive/pull/1959#discussion_r572850089



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelFactories.java
##########
@@ -143,40 +144,24 @@ public RelNode createFilter(RelNode child, RexNode condition, Set<CorrelationId>
      *          Right input
      * @param condition
      *          Join condition
-     * @param joinType
-     *          Join type
-     * @param variablesStopped
+     * @param variablesStoppedd
      *          Set of names of variables which are set by the LHS and used by
      *          the RHS and are not available to nodes above this JoinRel in the
      *          tree
+     *@param joinType
+     *             Join type
      * @param semiJoinDone
      *          Whether this join has been translated to a semi-join
      */
     @Override
-    public RelNode createJoin(RelNode left, RelNode right, RexNode condition, JoinRelType joinType,
-        Set<String> variablesStopped, boolean semiJoinDone) {
+    public RelNode createJoin(RelNode left, RelNode right, List<RelHint> hints, RexNode condition,
+                              Set<CorrelationId> variablesStoppedd, JoinRelType joinType, boolean semiJoinDone) {

Review comment:
       Fix indendation?

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveTableScan.java
##########
@@ -268,7 +268,7 @@ public boolean isInsideView() {
   // Also include partition list key to trigger cost evaluation even if an
   // expression was already generated.
   public String computeDigest() {

Review comment:
       Is the method still used somewhere? Maybe now it is necessary to adapt `explainTerms` method instead by adding more items. For instance:  `.itemIf("htColumns", this.neededColIndxsFrmReloptHT, pw.getDetailLevel() == SqlExplainLevel.DIGEST_ATTRIBUTES)`

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinConstraintsRule.java
##########
@@ -334,11 +335,13 @@ private void rewrite(final Mode mode, final RelNode inputToKeep, final RelNode i
       }
     } else { // Mode.TRANSFORM
       // Trigger transformation
+      List<RexNode> exps = new ArrayList<>();
+      project.accept(new ChildExpsRexShuttle(exps));
       call.transformTo(call.builder()
           .push(leftInput).push(rightInput)
           .join(JoinRelType.INNER, join.getCondition())
           .convert(call.rel(1).getRowType(), false) // Preserve nullability
-          .project(project.getChildExps())

Review comment:
       Why not `project.getProjects()` as in other places?

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdRowCount.java
##########
@@ -170,7 +170,7 @@ public Double getRowCount(Sort rel, RelMetadataQuery mq) {
   @Override
   public Double getRowCount(Filter rel, RelMetadataQuery mq) {
     if (rel instanceof StatEnhancedHiveFilter) {
-      return rel.getRows();
+      return 1.0D;

Review comment:
       This seems wrong. It seems that we should rather do `((StatEnhancedHiveFilter) rel).getRowCount()`.

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRelFieldTrimmer.java
##########
@@ -555,7 +556,8 @@ private Aggregate rewriteGBConstantKeys(Aggregate aggregate, ImmutableBitSet fie
     RexBuilder rexBuilder = aggregate.getCluster().getRexBuilder();
     final List<RexNode> newProjects = new ArrayList<>();
 
-    final List<RexNode> inputExprs = input.getChildExps();
+    final List<RexNode> inputExprs = new ArrayList<>();
+    input.accept(new ChildExpsRexShuttle(inputExprs));
     if (inputExprs == null || inputExprs.isEmpty()) {
       return aggregate;
     }

Review comment:
       This is comment is not related with the changes but it is a bit more general. The intention of this piece of code seems to be to find if all group by expressions are constant. However, I am not sure if `getChildExps()` is the right way to go. Most likely the code in `AggregateProjectPullUpConstantsRule` is more appropriate for this.

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/ChildExpsRexShuttle.java
##########
@@ -0,0 +1,113 @@
+package org.apache.hadoop.hive.ql.optimizer.calcite;
+
+import org.apache.calcite.rex.*;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+public class ChildExpsRexShuttle extends RexShuttle {
+    private final List<RexNode> exps;
+
+    public ChildExpsRexShuttle(List<RexNode> exps) {
+        this.exps = exps;
+    }
+
+    @Override
+    public RexNode visitOver(RexOver over) {
+        exps.add(over);
+        return super.visitOver(over);

Review comment:
       Do not call `super.visitXx` here and methods below to avoid recursively adding children of the input expression.

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/HiveMaterializedViewRule.java
##########
@@ -74,25 +74,21 @@
               HiveJoinProjectTransposeRule.LEFT_PROJECT,
               HiveJoinProjectTransposeRule.RIGHT_PROJECT,
               HiveProjectMergeRule.INSTANCE))
-      .addRuleInstance(ProjectRemoveRule.INSTANCE)
+      .addRuleInstance(ProjectRemoveRule.Config.DEFAULT.toRule())
       .addRuleInstance(HiveRootJoinProjectInsert.INSTANCE)
       .build();
 
   public static final MaterializedViewProjectFilterRule INSTANCE_PROJECT_FILTER =
-      new MaterializedViewProjectFilterRule(HiveRelFactories.HIVE_BUILDER,
-          true, PROGRAM, false);
+      MaterializedViewProjectFilterRule.Config.DEFAULT.toRule();

Review comment:
       I think we should pass the `HIVE_BUILDER` in the rule. Moreover we should double-check that the other parameters of the rule (`generateUnionRewritting`, `fastBailOut`, etc.) are set correctly. The comment applies also to the rules below.

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveDruidRules.java
##########
@@ -59,40 +54,35 @@
  */
 public class HiveDruidRules {
 
-  public static final DruidFilterRule FILTER = new DruidFilterRule(HiveRelFactories.HIVE_BUILDER);
+  public static final DruidFilterRule FILTER = (DruidFilterRule) DruidFilterRule.Config.EMPTY
+          .withRelBuilderFactory(HiveRelFactories.HIVE_BUILDER).toRule();
 
-  public static final DruidProjectRule PROJECT = new DruidProjectRule(HiveRelFactories.HIVE_BUILDER);
+  public static final DruidProjectRule PROJECT = (DruidProjectRule) DruidProjectRule.Config.EMPTY
+          .withRelBuilderFactory(HiveRelFactories.HIVE_BUILDER).toRule();
 
-  public static final DruidAggregateRule AGGREGATE = new DruidAggregateRule(HiveRelFactories.HIVE_BUILDER);
+  public static final DruidAggregateRule AGGREGATE = (DruidAggregateRule) DruidAggregateRule.Config.EMPTY
+          .withRelBuilderFactory(HiveRelFactories.HIVE_BUILDER).toRule();
 
   public static final DruidAggregateProjectRule AGGREGATE_PROJECT =
-      new DruidAggregateProjectRule(HiveRelFactories.HIVE_BUILDER);
+          (DruidAggregateProjectRule) DruidAggregateProjectRule.Config.EMPTY.
+                  withRelBuilderFactory(HiveRelFactories.HIVE_BUILDER).toRule();
 
-  public static final DruidSortRule SORT = new DruidSortRule(HiveRelFactories.HIVE_BUILDER);
+  public static final DruidSortRule SORT = (DruidSortRule) DruidSortRule.Config.EMPTY
+          .withRelBuilderFactory(HiveRelFactories.HIVE_BUILDER).toRule();
 
-  public static final DruidSortProjectTransposeRule SORT_PROJECT_TRANSPOSE =
-      new DruidSortProjectTransposeRule(HiveRelFactories.HIVE_BUILDER);
+  public static final SortProjectTransposeRule SORT_PROJECT_TRANSPOSE = DruidRules.SORT_PROJECT_TRANSPOSE;

Review comment:
       Shouldn't we pass the `HIVE_BUILDER` to this rule (and those that follow)?

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveCalciteUtil.java
##########
@@ -17,13 +17,8 @@
  */
 package org.apache.hadoop.hive.ql.optimizer.calcite;
 
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
+import java.util.*;

Review comment:
       I think that Hive coding conventions forbid `*` imports.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] soumyakanti3578 commented on a change in pull request #1959: [DRAFT] Upgrade calcite to 1.25

Posted by GitBox <gi...@apache.org>.
soumyakanti3578 commented on a change in pull request #1959:
URL: https://github.com/apache/hive/pull/1959#discussion_r578816667



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelFactories.java
##########
@@ -143,40 +144,24 @@ public RelNode createFilter(RelNode child, RexNode condition, Set<CorrelationId>
      *          Right input
      * @param condition
      *          Join condition
-     * @param joinType
-     *          Join type
-     * @param variablesStopped
+     * @param variablesStoppedd
      *          Set of names of variables which are set by the LHS and used by
      *          the RHS and are not available to nodes above this JoinRel in the
      *          tree
+     *@param joinType
+     *             Join type
      * @param semiJoinDone
      *          Whether this join has been translated to a semi-join
      */
     @Override
-    public RelNode createJoin(RelNode left, RelNode right, RexNode condition, JoinRelType joinType,
-        Set<String> variablesStopped, boolean semiJoinDone) {
+    public RelNode createJoin(RelNode left, RelNode right, List<RelHint> hints, RexNode condition,
+      Set<CorrelationId> variablesStoppedd, JoinRelType joinType, boolean semiJoinDone) {
       if (joinType == JoinRelType.SEMI) {
         final RelOptCluster cluster = left.getCluster();
         return HiveSemiJoin.getSemiJoin(cluster, left.getTraitSet(), left, right, condition);
       }
       return HiveJoin.getJoin(left.getCluster(), left, right, condition, joinType);
     }
-
-    @Override
-    public RelNode createJoin(RelNode left, RelNode right, RexNode condition,
-        Set<CorrelationId> variablesSet, JoinRelType joinType, boolean semiJoinDone) {
-      // According to calcite, it is going to be removed before Calcite-2.0
-      // TODO: to handle CorrelationId
-      if (joinType == JoinRelType.SEMI) {
-        final RelOptCluster cluster = left.getCluster();
-        return HiveSemiJoin.getSemiJoin(cluster, left.getTraitSet(), left, right, condition);
-      }
-      if (joinType == JoinRelType.ANTI) {

Review comment:
       move this block above




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] soumyakanti3578 commented on a change in pull request #1959: [DRAFT] Upgrade calcite to 1.25

Posted by GitBox <gi...@apache.org>.
soumyakanti3578 commented on a change in pull request #1959:
URL: https://github.com/apache/hive/pull/1959#discussion_r578820179



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveTableScan.java
##########
@@ -137,13 +138,15 @@ public HiveTableScan copyIncludingTable(RelDataType newRowtype) {
   }
 
   @Override public RelWriter explainTerms(RelWriter pw) {
-    if (this.useQBIdInDigest) {
-      // TODO: Only the qualified name should be left here
-      return super.explainTerms(pw)
-          .item("qbid:alias", concatQbIDAlias);
-    } else {
-      return super.explainTerms(pw).item("table:alias", tblAlias);
-    }
+    return super.explainTerms(pw)

Review comment:
       Check inheritances of explainTerms & computeDigest




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] zabetak commented on a change in pull request #1959: [DRAFT] Upgrade calcite to 1.25

Posted by GitBox <gi...@apache.org>.
zabetak commented on a change in pull request #1959:
URL: https://github.com/apache/hive/pull/1959#discussion_r584272811



##########
File path: ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query84.q.out
##########
@@ -10,22 +10,22 @@ STAGE PLANS:
         Map 1 <- Map 3 (BROADCAST_EDGE), Map 4 (BROADCAST_EDGE)
         Map 4 <- Map 5 (BROADCAST_EDGE), Map 6 (BROADCAST_EDGE)
         Map 6 <- Map 7 (BROADCAST_EDGE)
+        Map 8 <- Reducer 2 (BROADCAST_EDGE)

Review comment:
       We should investigate why there is an extra mapper/reducer.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] soumyakanti3578 commented on a change in pull request #1959: [DRAFT] Upgrade calcite to 1.25

Posted by GitBox <gi...@apache.org>.
soumyakanti3578 commented on a change in pull request #1959:
URL: https://github.com/apache/hive/pull/1959#discussion_r578892516



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveTableScan.java
##########
@@ -137,13 +138,15 @@ public HiveTableScan copyIncludingTable(RelDataType newRowtype) {
   }
 
   @Override public RelWriter explainTerms(RelWriter pw) {
-    if (this.useQBIdInDigest) {
-      // TODO: Only the qualified name should be left here
-      return super.explainTerms(pw)
-          .item("qbid:alias", concatQbIDAlias);
-    } else {
-      return super.explainTerms(pw).item("table:alias", tblAlias);
-    }
+    return super.explainTerms(pw)

Review comment:
       `computeDigest` can be safely removed from `HiveTableScan` as it has been removed from `AbstractRelNode`.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] soumyakanti3578 commented on a change in pull request #1959: [DRAFT] Upgrade calcite to 1.25

Posted by GitBox <gi...@apache.org>.
soumyakanti3578 commented on a change in pull request #1959:
URL: https://github.com/apache/hive/pull/1959#discussion_r578883447



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveProject.java
##########
@@ -200,9 +200,4 @@ public boolean isSynthetic() {
     }
     return shuttle.visit(this);
   }
-
-  public boolean containsOver() {

Review comment:
       Moved to `Project.java`




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] soumyakanti3578 commented on a change in pull request #1959: [DRAFT] Upgrade calcite to 1.25

Posted by GitBox <gi...@apache.org>.
soumyakanti3578 commented on a change in pull request #1959:
URL: https://github.com/apache/hive/pull/1959#discussion_r578878891



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelFactories.java
##########
@@ -143,40 +144,24 @@ public RelNode createFilter(RelNode child, RexNode condition, Set<CorrelationId>
      *          Right input
      * @param condition
      *          Join condition
-     * @param joinType
-     *          Join type
-     * @param variablesStopped
+     * @param variablesStoppedd
      *          Set of names of variables which are set by the LHS and used by
      *          the RHS and are not available to nodes above this JoinRel in the
      *          tree
+     *@param joinType
+     *             Join type
      * @param semiJoinDone
      *          Whether this join has been translated to a semi-join
      */
     @Override
-    public RelNode createJoin(RelNode left, RelNode right, RexNode condition, JoinRelType joinType,
-        Set<String> variablesStopped, boolean semiJoinDone) {
+    public RelNode createJoin(RelNode left, RelNode right, List<RelHint> hints, RexNode condition,
+      Set<CorrelationId> variablesStoppedd, JoinRelType joinType, boolean semiJoinDone) {
       if (joinType == JoinRelType.SEMI) {
         final RelOptCluster cluster = left.getCluster();
         return HiveSemiJoin.getSemiJoin(cluster, left.getTraitSet(), left, right, condition);
       }
       return HiveJoin.getJoin(left.getCluster(), left, right, condition, joinType);
     }
-
-    @Override
-    public RelNode createJoin(RelNode left, RelNode right, RexNode condition,
-        Set<CorrelationId> variablesSet, JoinRelType joinType, boolean semiJoinDone) {
-      // According to calcite, it is going to be removed before Calcite-2.0
-      // TODO: to handle CorrelationId
-      if (joinType == JoinRelType.SEMI) {
-        final RelOptCluster cluster = left.getCluster();
-        return HiveSemiJoin.getSemiJoin(cluster, left.getTraitSet(), left, right, condition);
-      }
-      if (joinType == JoinRelType.ANTI) {

Review comment:
       Added another `if` block for AntiJoin.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] soumyakanti3578 commented on a change in pull request #1959: [DRAFT] Upgrade calcite to 1.25

Posted by GitBox <gi...@apache.org>.
soumyakanti3578 commented on a change in pull request #1959:
URL: https://github.com/apache/hive/pull/1959#discussion_r578818872



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveProject.java
##########
@@ -200,9 +200,4 @@ public boolean isSynthetic() {
     }
     return shuttle.visit(this);
   }
-
-  public boolean containsOver() {

Review comment:
       Double check




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] soumyakanti3578 commented on a change in pull request #1959: [DRAFT] Upgrade calcite to 1.25

Posted by GitBox <gi...@apache.org>.
soumyakanti3578 commented on a change in pull request #1959:
URL: https://github.com/apache/hive/pull/1959#discussion_r572918961



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/ChildExpsRexShuttle.java
##########
@@ -0,0 +1,113 @@
+package org.apache.hadoop.hive.ql.optimizer.calcite;
+
+import org.apache.calcite.rex.*;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+public class ChildExpsRexShuttle extends RexShuttle {
+    private final List<RexNode> exps;
+
+    public ChildExpsRexShuttle(List<RexNode> exps) {
+        this.exps = exps;
+    }
+
+    @Override
+    public RexNode visitOver(RexOver over) {
+        exps.add(over);
+        return super.visitOver(over);

Review comment:
       In that case, should we just return the appropriate method arguments here or just return `null` as we are not calling these methods explicitly anywhere?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] soumyakanti3578 commented on a change in pull request #1959: [DRAFT] Upgrade calcite to 1.25

Posted by GitBox <gi...@apache.org>.
soumyakanti3578 commented on a change in pull request #1959:
URL: https://github.com/apache/hive/pull/1959#discussion_r572930028



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveDruidRules.java
##########
@@ -59,40 +54,35 @@
  */
 public class HiveDruidRules {
 
-  public static final DruidFilterRule FILTER = new DruidFilterRule(HiveRelFactories.HIVE_BUILDER);
+  public static final DruidFilterRule FILTER = (DruidFilterRule) DruidFilterRule.Config.EMPTY
+          .withRelBuilderFactory(HiveRelFactories.HIVE_BUILDER).toRule();
 
-  public static final DruidProjectRule PROJECT = new DruidProjectRule(HiveRelFactories.HIVE_BUILDER);
+  public static final DruidProjectRule PROJECT = (DruidProjectRule) DruidProjectRule.Config.EMPTY
+          .withRelBuilderFactory(HiveRelFactories.HIVE_BUILDER).toRule();
 
-  public static final DruidAggregateRule AGGREGATE = new DruidAggregateRule(HiveRelFactories.HIVE_BUILDER);
+  public static final DruidAggregateRule AGGREGATE = (DruidAggregateRule) DruidAggregateRule.Config.EMPTY
+          .withRelBuilderFactory(HiveRelFactories.HIVE_BUILDER).toRule();
 
   public static final DruidAggregateProjectRule AGGREGATE_PROJECT =
-      new DruidAggregateProjectRule(HiveRelFactories.HIVE_BUILDER);
+          (DruidAggregateProjectRule) DruidAggregateProjectRule.Config.EMPTY.
+                  withRelBuilderFactory(HiveRelFactories.HIVE_BUILDER).toRule();
 
-  public static final DruidSortRule SORT = new DruidSortRule(HiveRelFactories.HIVE_BUILDER);
+  public static final DruidSortRule SORT = (DruidSortRule) DruidSortRule.Config.EMPTY
+          .withRelBuilderFactory(HiveRelFactories.HIVE_BUILDER).toRule();
 
-  public static final DruidSortProjectTransposeRule SORT_PROJECT_TRANSPOSE =
-      new DruidSortProjectTransposeRule(HiveRelFactories.HIVE_BUILDER);
+  public static final SortProjectTransposeRule SORT_PROJECT_TRANSPOSE = DruidRules.SORT_PROJECT_TRANSPOSE;

Review comment:
       Oh yes, so I think I should update it like this?
   `public static final SortProjectTransposeRule SORT_PROJECT_TRANSPOSE = 
       (SortProjectTransposeRule) DruidRules.SORT_PROJECT_TRANSPOSE.config.withRelBuilderFactory(HiveRelFactories.HIVE_BUILDER).toRule();`




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] github-actions[bot] commented on pull request #1959: [DRAFT] Upgrade calcite to 1.25

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #1959:
URL: https://github.com/apache/hive/pull/1959#issuecomment-855489299


   This pull request has been automatically marked as stale because it has not had recent activity. It will be closed if no further activity occurs.
   Feel free to reach out on the dev@hive.apache.org list if the patch is in need of reviews.


-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] zabetak commented on a change in pull request #1959: [DRAFT] Upgrade calcite to 1.25

Posted by GitBox <gi...@apache.org>.
zabetak commented on a change in pull request #1959:
URL: https://github.com/apache/hive/pull/1959#discussion_r584272640



##########
File path: ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query86.q.out
##########
@@ -140,7 +140,7 @@ STAGE PLANS:
                     Statistics: Num rows: 3267 Data size: 986634 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col0 (type: char(50)), _col1 (type: char(50)), _col3 (type: bigint)
         Reducer 3 
-            Execution mode: vectorized, llap

Review comment:
       We should examine why vectorized execution is lost.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] kgyrtkirk commented on a change in pull request #1959: [DRAFT] Upgrade calcite to 1.25

Posted by GitBox <gi...@apache.org>.
kgyrtkirk commented on a change in pull request #1959:
URL: https://github.com/apache/hive/pull/1959#discussion_r592197745



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelBuilder.java
##########
@@ -158,7 +158,6 @@ public static SqlAggFunction getRollup(SqlAggFunction aggregation) {
     return null;
   }
 
-  @Override

Review comment:
       CALCITE-3774 have removed this method - and replaced it with `config.bloat()` 
   
   as the comment suggests this feature caused incorrectly merged `Project`-s which contained `Over` expressions
   * the new code have this bloat enabled by default with 100
   * since the new check seems is heuristic based - it may not happen by default
   
   I'm not sure if our issue is gone now or not - setting bloat to `-1` might be a way to disable this; and consider enabling it later 




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] soumyakanti3578 commented on a change in pull request #1959: [DRAFT] Upgrade calcite to 1.25

Posted by GitBox <gi...@apache.org>.
soumyakanti3578 commented on a change in pull request #1959:
URL: https://github.com/apache/hive/pull/1959#discussion_r572938384



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinConstraintsRule.java
##########
@@ -334,11 +335,13 @@ private void rewrite(final Mode mode, final RelNode inputToKeep, final RelNode i
       }
     } else { // Mode.TRANSFORM
       // Trigger transformation
+      List<RexNode> exps = new ArrayList<>();
+      project.accept(new ChildExpsRexShuttle(exps));
       call.transformTo(call.builder()
           .push(leftInput).push(rightInput)
           .join(JoinRelType.INNER, join.getCondition())
           .convert(call.rel(1).getRowType(), false) // Preserve nullability
-          .project(project.getChildExps())

Review comment:
       The project here is of type `RelNode` which doesn't have a `getProjects` method. In other places, generally the variable is of `Project` type.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] soumyakanti3578 commented on a change in pull request #1959: [DRAFT] Upgrade calcite to 1.25

Posted by GitBox <gi...@apache.org>.
soumyakanti3578 commented on a change in pull request #1959:
URL: https://github.com/apache/hive/pull/1959#discussion_r578883447



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveProject.java
##########
@@ -200,9 +200,4 @@ public boolean isSynthetic() {
     }
     return shuttle.visit(this);
   }
-
-  public boolean containsOver() {

Review comment:
       Moved to `org.apache.calcite.rel.core.Project.java`




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] soumyakanti3578 commented on a change in pull request #1959: [DRAFT] Upgrade calcite to 1.25

Posted by GitBox <gi...@apache.org>.
soumyakanti3578 commented on a change in pull request #1959:
URL: https://github.com/apache/hive/pull/1959#discussion_r578880512



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/cost/HiveVolcanoPlanner.java
##########
@@ -85,7 +85,6 @@ public void registerClass(RelNode node) {
       addRule(HiveDruidRules.FILTER_PROJECT_TRANSPOSE);
       addRule(HiveDruidRules.SORT_PROJECT_TRANSPOSE);
       addRule(HiveDruidRules.SORT);
-      addRule(HiveDruidRules.PROJECT_SORT_TRANSPOSE);

Review comment:
       This rule is useless, and has been removed in 1.24. [CALCITE-4023](https://issues.apache.org/jira/browse/CALCITE-4023)




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] soumyakanti3578 commented on a change in pull request #1959: [DRAFT] Upgrade calcite to 1.25

Posted by GitBox <gi...@apache.org>.
soumyakanti3578 commented on a change in pull request #1959:
URL: https://github.com/apache/hive/pull/1959#discussion_r585680145



##########
File path: ql/src/test/results/clientpositive/llap/auto_join32.q.out
##########
@@ -563,61 +583,39 @@ STAGE PLANS:
                           1 _col0 (type: string)
                         outputColumnNames: _col0, _col2
                         input vertices:
-                          1 Map 3
-                        Statistics: Num rows: 1 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
+                          0 Map 1
+                        Statistics: Num rows: 1 Data size: 368 Basic stats: COMPLETE Column stats: PARTIAL
                         Group By Operator
                           keys: _col0 (type: string), _col2 (type: string)
                           minReductionHashAggr: 0.99
                           mode: hash
                           outputColumnNames: _col0, _col1
-                          Statistics: Num rows: 1 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 1 Data size: 368 Basic stats: COMPLETE Column stats: PARTIAL
                           Reduce Output Operator
                             key expressions: _col0 (type: string), _col1 (type: string)
                             null sort order: zz
                             sort order: ++
                             Map-reduce partition columns: _col0 (type: string)
-                            Statistics: Num rows: 1 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: vectorized, llap
-            LLAP IO: unknown
-        Map 3 
-            Map Operator Tree:
-                TableScan
-                  alias: v
-                  filterExpr: ((p = 'bar') and name is not null) (type: boolean)
-                  Statistics: Num rows: 1 Data size: 352 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: ((p = 'bar') and name is not null) (type: boolean)
-                    Statistics: Num rows: 1 Data size: 352 Basic stats: COMPLETE Column stats: COMPLETE
-                    Select Operator
-                      expressions: name (type: string), registration (type: string)
-                      outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 1 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: string)
-                        null sort order: z
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: string)
-                        Statistics: Num rows: 1 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions: _col1 (type: string)
+                            Statistics: Num rows: 1 Data size: 368 Basic stats: COMPLETE Column stats: PARTIAL
             Execution mode: vectorized, llap
             LLAP IO: unknown
-        Reducer 2 
+        Reducer 3 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
                 keys: KEY._col0 (type: string), KEY._col1 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 1 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1 Data size: 368 Basic stats: COMPLETE Column stats: PARTIAL

Review comment:
       Complete -> Partial is problematic.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] zabetak commented on a change in pull request #1959: [DRAFT] Upgrade calcite to 1.25

Posted by GitBox <gi...@apache.org>.
zabetak commented on a change in pull request #1959:
URL: https://github.com/apache/hive/pull/1959#discussion_r584272249



##########
File path: ql/src/test/results/clientpositive/perf/tpcds30tb/tez/cbo_query15.q.out
##########
@@ -15,6 +15,6 @@ HiveSortLimit(sort0=[$0], dir0=[ASC], fetch=[100])
               HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($33))])
                 HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
             HiveProject(d_date_sk=[$0])
-              HiveFilter(condition=[AND(=($6, 2000), =($10, 2))])
+              HiveFilter(condition=[AND(=($10, 2), =($6, 2000))])

Review comment:
       Thanks for the info @jcamachor . @soumyakanti3578 also expressed some concerns around these changes. Truth is that it didn't cross my mind that condition order matters. I was suspecting that the changes were related to https://issues.apache.org/jira/browse/CALCITE-4073 or https://issues.apache.org/jira/browse/CALCITE-2450. We will check it out ;)




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] github-actions[bot] closed pull request #1959: [DRAFT] Upgrade calcite to 1.25

Posted by GitBox <gi...@apache.org>.
github-actions[bot] closed pull request #1959:
URL: https://github.com/apache/hive/pull/1959


   


-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] soumyakanti3578 commented on a change in pull request #1959: [DRAFT] Upgrade calcite to 1.25

Posted by GitBox <gi...@apache.org>.
soumyakanti3578 commented on a change in pull request #1959:
URL: https://github.com/apache/hive/pull/1959#discussion_r578817232



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelShuttleImpl.java
##########
@@ -23,18 +23,7 @@
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.TableFunctionScan;
 import org.apache.calcite.rel.core.TableScan;
-import org.apache.calcite.rel.logical.LogicalAggregate;
-import org.apache.calcite.rel.logical.LogicalCorrelate;
-import org.apache.calcite.rel.logical.LogicalExchange;
-import org.apache.calcite.rel.logical.LogicalFilter;
-import org.apache.calcite.rel.logical.LogicalIntersect;
-import org.apache.calcite.rel.logical.LogicalJoin;
-import org.apache.calcite.rel.logical.LogicalMatch;
-import org.apache.calcite.rel.logical.LogicalMinus;
-import org.apache.calcite.rel.logical.LogicalProject;
-import org.apache.calcite.rel.logical.LogicalSort;
-import org.apache.calcite.rel.logical.LogicalUnion;
-import org.apache.calcite.rel.logical.LogicalValues;
+import org.apache.calcite.rel.logical.*;

Review comment:
       revert




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] jcamachor commented on a change in pull request #1959: [DRAFT] Upgrade calcite to 1.25

Posted by GitBox <gi...@apache.org>.
jcamachor commented on a change in pull request #1959:
URL: https://github.com/apache/hive/pull/1959#discussion_r583929715



##########
File path: ql/src/test/results/clientpositive/perf/tpcds30tb/tez/cbo_query15.q.out
##########
@@ -15,6 +15,6 @@ HiveSortLimit(sort0=[$0], dir0=[ASC], fetch=[100])
               HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($33))])
                 HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
             HiveProject(d_date_sk=[$0])
-              HiveFilter(condition=[AND(=($6, 2000), =($10, 2))])
+              HiveFilter(condition=[AND(=($10, 2), =($6, 2000))])

Review comment:
       The order in the filter conditions is determined by `HiveFilterSortPredicates`. This change and other similar ones could happen due to change in selectivity estimates for each of the conjuncts. We should double check that these changes are expected. ^ @zabetak fyi




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] kgyrtkirk commented on a change in pull request #1959: [DRAFT] Upgrade calcite to 1.25

Posted by GitBox <gi...@apache.org>.
kgyrtkirk commented on a change in pull request #1959:
URL: https://github.com/apache/hive/pull/1959#discussion_r592190573



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
##########
@@ -520,6 +520,8 @@ public static void closeCurrent() {
    */
   private Hive(HiveConf c, boolean doRegisterAllFns) throws HiveException {
     conf = c;
+    // turn off calcite rexnode normalization
+    System.setProperty("calcite.enable.rexnode.digest.normalize", "false");

Review comment:
       I'm worried that it would only change this setting for the scope of the tests - and not for the release itself.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] soumyakanti3578 commented on a change in pull request #1959: [DRAFT] Upgrade calcite to 1.25

Posted by GitBox <gi...@apache.org>.
soumyakanti3578 commented on a change in pull request #1959:
URL: https://github.com/apache/hive/pull/1959#discussion_r578876839



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelBuilder.java
##########
@@ -158,7 +158,6 @@ public static SqlAggFunction getRollup(SqlAggFunction aggregation) {
     return null;
   }
 
-  @Override

Review comment:
       Didn't find any usage in `HiveRelBuilder.java` or `RelBuilder.java` in 1.21




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] zabetak commented on a change in pull request #1959: [DRAFT] Upgrade calcite to 1.25

Posted by GitBox <gi...@apache.org>.
zabetak commented on a change in pull request #1959:
URL: https://github.com/apache/hive/pull/1959#discussion_r578318173



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveDruidRules.java
##########
@@ -54,49 +58,102 @@
  */
 public class HiveDruidRules {

Review comment:
       `HiveDruidRules` can be simplified when we upgrade to Calcite 1.26 onwards (exploiting the work done in https://issues.apache.org/jira/browse/CALCITE-4200). Please add an entry in `org.apache.hadoop.hive.ql.optimizer.calcite.Bug` and link it somehow (e.g., reference it via javadoc) with this class to stand as a reminder.
   




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] soumyakanti3578 commented on a change in pull request #1959: [DRAFT] Upgrade calcite to 1.25

Posted by GitBox <gi...@apache.org>.
soumyakanti3578 commented on a change in pull request #1959:
URL: https://github.com/apache/hive/pull/1959#discussion_r578816775



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelBuilder.java
##########
@@ -158,7 +158,6 @@ public static SqlAggFunction getRollup(SqlAggFunction aggregation) {
     return null;
   }
 
-  @Override

Review comment:
       investigate this 




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org