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 2022/05/23 08:00:24 UTC

[GitHub] [hive] kgyrtkirk commented on a diff in pull request #3266: HIVE-25589: SQL: Implement HAVING/QUALIFY predicates for ROW_NUMBER()=1

kgyrtkirk commented on code in PR #3266:
URL: https://github.com/apache/hive/pull/3266#discussion_r879132187


##########
ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java:
##########
@@ -4779,34 +4779,29 @@ && isRegex(
               throw new SemanticException(SemanticAnalyzer.generateErrorMessage(obAST, error));
             }
           }
-          List<RexNode> originalInputRefs = Lists.transform(srcRel.getRowType().getFieldList(),
-              new Function<RelDataTypeField, RexNode>() {
-                @Override
-                public RexNode apply(RelDataTypeField input) {
-                  return new RexInputRef(input.getIndex(), input.getType());
-                }
-              });
-          originalRR = outputRR.duplicate();
-          for (int i = 0; i < inputRR.getColumnInfos().size(); i++) {
-            ColumnInfo colInfo = new ColumnInfo(inputRR.getColumnInfos().get(i));
-            String internalName = SemanticAnalyzer.getColumnInternalName(outputRR.getColumnInfos()
-                .size() + i);
-            colInfo.setInternalName(internalName);
-            // if there is any confict, then we do not generate it in the new select
-            // otherwise, we add it into the calciteColLst and generate the new select
-            if (!outputRR.putWithCheck(colInfo.getTabAlias(), colInfo.getAlias(), internalName,
-                colInfo)) {
-              LOG.trace("Column already present in RR. skipping.");
-            } else {
-              columnList.add(originalInputRefs.get(i));
-            }
-          }
+          originalRR = appendInputColumns(srcRel, columnList, outputRR, inputRR);
           outputRel = genSelectRelNode(columnList, outputRR, srcRel);
           // outputRel is the generated augmented select with extra unselected
           // columns, and originalRR is the original generated select
           return new Pair<RelNode, RowResolver>(outputRel, originalRR);
         } else {
-          outputRel = genSelectRelNode(columnList, outputRR, srcRel);
+          if (qbp.getQualifyExprForClause(dest) != null) {

Review Comment:
   right now I think that this feature will only work on the CBO path ; and its not supported on the default path
   do we already throw an exception in case cbo is off or when we fallen back to non-cbo path?
   can we have a testcase for that?



##########
ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java:
##########
@@ -4779,34 +4779,29 @@ && isRegex(
               throw new SemanticException(SemanticAnalyzer.generateErrorMessage(obAST, error));
             }
           }
-          List<RexNode> originalInputRefs = Lists.transform(srcRel.getRowType().getFieldList(),
-              new Function<RelDataTypeField, RexNode>() {
-                @Override
-                public RexNode apply(RelDataTypeField input) {
-                  return new RexInputRef(input.getIndex(), input.getType());
-                }
-              });
-          originalRR = outputRR.duplicate();
-          for (int i = 0; i < inputRR.getColumnInfos().size(); i++) {
-            ColumnInfo colInfo = new ColumnInfo(inputRR.getColumnInfos().get(i));
-            String internalName = SemanticAnalyzer.getColumnInternalName(outputRR.getColumnInfos()
-                .size() + i);
-            colInfo.setInternalName(internalName);
-            // if there is any confict, then we do not generate it in the new select
-            // otherwise, we add it into the calciteColLst and generate the new select
-            if (!outputRR.putWithCheck(colInfo.getTabAlias(), colInfo.getAlias(), internalName,
-                colInfo)) {
-              LOG.trace("Column already present in RR. skipping.");
-            } else {
-              columnList.add(originalInputRefs.get(i));
-            }
-          }
+          originalRR = appendInputColumns(srcRel, columnList, outputRR, inputRR);
           outputRel = genSelectRelNode(columnList, outputRR, srcRel);
           // outputRel is the generated augmented select with extra unselected
           // columns, and originalRR is the original generated select
           return new Pair<RelNode, RowResolver>(outputRel, originalRR);
         } else {
-          outputRel = genSelectRelNode(columnList, outputRR, srcRel);
+          if (qbp.getQualifyExprForClause(dest) != null) {
+            int originalColumnListSize = columnList.size();
+            originalRR = appendInputColumns(srcRel, columnList, outputRR, inputRR);
+            RelNode combinedProject = genSelectRelNode(columnList, outputRR, srcRel);
+            RelNode qualifyRel = genQualifyLogicalPlan(qb, combinedProject);
+            if (qualifyRel != null) {

Review Comment:
   in case this condition is not met; `outputRel/outputRR` won't be set - is that expected?
   is that case was covered with a testcase?
   Right now I think instead of `return null` an exception should be thrown from `genQualifyLogicalPlan`



-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

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