You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hive.apache.org by "ASF GitHub Bot (Jira)" <ji...@apache.org> on 2022/05/11 12:21:00 UTC

[jira] [Work logged] (HIVE-26006) TopNKey and PTF with more than one column is failing with IOBE

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

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

                Author: ASF GitHub Bot
            Created on: 11/May/22 12:20
            Start Date: 11/May/22 12:20
    Worklog Time Spent: 10m 
      Work Description: zabetak commented on code in PR #3082:
URL: https://github.com/apache/hive/pull/3082#discussion_r870221178


##########
ql/src/java/org/apache/hadoop/hive/ql/optimizer/topnkey/TopNKeyPushdownProcessor.java:
##########
@@ -244,18 +223,41 @@ private void pushdownThroughLeftOuterJoin(TopNKeyOperator topNKey) throws Semant
             reduceSinkDesc.getColumnExprMap(),
             reduceSinkDesc.getOrder(),
             reduceSinkDesc.getNullOrder());
+
+    pushDownThrough(commonKeyPrefix, topNKey, join, reduceSinkOperator);
+  }
+
+  private <T extends AbstractOperatorDesc> void pushDownThrough(
+          CommonKeyPrefix commonKeyPrefix, TopNKeyOperator topNKey, Operator<T> operator)
+          throws SemanticException {
+
+    pushDownThrough(commonKeyPrefix, topNKey, operator, operator);
+  }
+
+  private <TDesc extends AbstractOperatorDesc, TParentDesc extends AbstractOperatorDesc> void pushDownThrough(
+          CommonKeyPrefix commonKeyPrefix, TopNKeyOperator topNKey,
+          Operator<TDesc> operator, Operator<TParentDesc> parentOperator)

Review Comment:
   Are there any advantages in using type parameters in this method (i.e., `TDesc`, `TParentDesc`) instead of wildcards? 
   
   It seems that the conf of the operator is not used anywhere in this method so we could possibly simplify the signature to:
   `private void pushDownThrough(CommonKeyPrefix commonKeyPrefix, TopNKeyOperator topNKey, Operator<?> operator, Operator<?> parentOperator)`



##########
ql/src/java/org/apache/hadoop/hive/ql/optimizer/topnkey/TopNKeyPushdownProcessor.java:
##########
@@ -244,18 +223,41 @@ private void pushdownThroughLeftOuterJoin(TopNKeyOperator topNKey) throws Semant
             reduceSinkDesc.getColumnExprMap(),
             reduceSinkDesc.getOrder(),
             reduceSinkDesc.getNullOrder());
+
+    pushDownThrough(commonKeyPrefix, topNKey, join, reduceSinkOperator);
+  }
+
+  private <T extends AbstractOperatorDesc> void pushDownThrough(
+          CommonKeyPrefix commonKeyPrefix, TopNKeyOperator topNKey, Operator<T> operator)
+          throws SemanticException {
+
+    pushDownThrough(commonKeyPrefix, topNKey, operator, operator);
+  }

Review Comment:
   Wildcard instead of `T extends AbstractOperatorDesc`? See previous comments.



##########
ql/src/java/org/apache/hadoop/hive/ql/optimizer/topnkey/TopNKeyPushdownProcessor.java:
##########
@@ -34,6 +34,7 @@
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.plan.AbstractOperatorDesc;

Review Comment:
   Possibly can be removed.



##########
ql/src/java/org/apache/hadoop/hive/ql/plan/TopNKeyDesc.java:
##########
@@ -252,7 +252,8 @@ public TopNKeyDescExplainVectorization getTopNKeyVectorization() {
   public TopNKeyDesc combine(CommonKeyPrefix commonKeyPrefix) {
     return new TopNKeyDesc(topN, commonKeyPrefix.getMappedOrder(),
             commonKeyPrefix.getMappedNullOrder(), commonKeyPrefix.getMappedColumns(),
-            commonKeyPrefix.getMappedColumns().subList(0, partitionKeyColumns.size()),
+            commonKeyPrefix.getMappedColumns()
+                    .subList(0, Math.min(partitionKeyColumns.size(), commonKeyPrefix.getMappedColumns().size())),

Review Comment:
   This new change guarantees that the following condition `keys >= partitionKeys` is always true no matter the input (since the partition keys are a sublist of keys). Maybe, I am missing something but I cannot see how we can have `keys < partitionKeys` at this stage.
   
   If what I am saying is correct then the inequality in the new bail out condition is misleading.
   
   More generally does it make sense to create a new TNK operator when `partitionKeyColumns.size() >= commonKeyPrefix.size()`? If not then maybe we should add appropriate assertion instead of using `Math.min`.



##########
ql/src/java/org/apache/hadoop/hive/ql/optimizer/topnkey/TopNKeyPushdownProcessor.java:
##########
@@ -244,13 +223,35 @@ private void pushdownThroughLeftOuterJoin(TopNKeyOperator topNKey) throws Semant
             reduceSinkDesc.getColumnExprMap(),
             reduceSinkDesc.getOrder(),
             reduceSinkDesc.getNullOrder());
+
+    pushDownThrough(commonKeyPrefix, topNKey, join, reduceSinkOperator);
+  }
+
+  private <T extends AbstractOperatorDesc> void pushDownThrough(
+          CommonKeyPrefix commonKeyPrefix, TopNKeyOperator topNKey, Operator<T> operator)
+          throws SemanticException {
+
+    pushDownThrough(commonKeyPrefix, topNKey, operator, operator);
+  }
+
+  private <TDesc extends AbstractOperatorDesc, TParentDesc extends AbstractOperatorDesc> void pushDownThrough(
+          CommonKeyPrefix commonKeyPrefix, TopNKeyOperator topNKey,
+          Operator<TDesc> join, Operator<TParentDesc> reduceSinkOperator)
+          throws SemanticException {
+
+    final TopNKeyDesc topNKeyDesc = topNKey.getConf();
     if (commonKeyPrefix.isEmpty() || commonKeyPrefix.size() == topNKeyDesc.getPartitionKeyColumns().size()) {
       return;
     }
 
+    final TopNKeyDesc newTopNKeyDesc = topNKeyDesc.combine(commonKeyPrefix);
+    if (newTopNKeyDesc.getKeyColumns().size() > 0 &&
+            newTopNKeyDesc.getKeyColumns().size() <= newTopNKeyDesc.getPartitionKeyColumns().size()) {

Review Comment:
   I was thinking that we could exploit `commonKeyPrefix` and `topNKeyDesc` to bail out before calling `combine` and creating the new TNK. Possibly the logic could be merged with the existing bail out condition just above.
   
   Moreover, if I got it right it only makes sense to introduce a new TNK operator when `keys > partitionKeys`. 





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

    Worklog Id:     (was: 769034)
    Time Spent: 1h 10m  (was: 1h)

> TopNKey and PTF with more than one column is failing with IOBE
> --------------------------------------------------------------
>
>                 Key: HIVE-26006
>                 URL: https://issues.apache.org/jira/browse/HIVE-26006
>             Project: Hive
>          Issue Type: Bug
>            Reporter: Naresh P R
>            Assignee: Krisztian Kasa
>            Priority: Major
>              Labels: pull-request-available
>          Time Spent: 1h 10m
>  Remaining Estimate: 0h
>
> {code:java}
> java.lang.IndexOutOfBoundsException: toIndex = 2
> at java.util.ArrayList.subListRangeCheck(ArrayList.java:1014)
> at java.util.ArrayList.subList(ArrayList.java:1006)
> at org.apache.hadoop.hive.ql.plan.TopNKeyDesc.combine(TopNKeyDesc.java:201)
> at org.apache.hadoop.hive.ql.optimizer.topnkey.TopNKeyPushdownProcessor.pushdownThroughGroupBy(TopNKeyPushdownProcessor.java:162)
> at org.apache.hadoop.hive.ql.optimizer.topnkey.TopNKeyPushdownProcessor.pushdown(TopNKeyPushdownProcessor.java:76)
> at org.apache.hadoop.hive.ql.optimizer.topnkey.TopNKeyPushdownProcessor.process(TopNKeyPushdownProcessor.java:57)
> at org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher.dispatch(DefaultRuleDispatcher.java:90)
> at org.apache.hadoop.hive.ql.lib.DefaultGraphWalker.dispatchAndReturn(DefaultGraphWalker.java:105)
> at org.apache.hadoop.hive.ql.lib.DefaultGraphWalker.dispatch(DefaultGraphWalker.java:89)
> at org.apache.hadoop.hive.ql.lib.DefaultGraphWalker.walk(DefaultGraphWalker.java:158)
> at org.apache.hadoop.hive.ql.lib.DefaultGraphWalker.startWalking(DefaultGraphWalker.java:120)
> at org.apache.hadoop.hive.ql.parse.TezCompiler.runTopNKeyOptimization(TezCompiler.java:1305)
> at org.apache.hadoop.hive.ql.parse.TezCompiler.optimizeOperatorPlan(TezCompiler.java:173)
> at org.apache.hadoop.hive.ql.parse.TaskCompiler.compile(TaskCompiler.java:159)
> at org.apache.hadoop.hive.ql.parse.SemanticAnalyzer.analyzeInternal(SemanticAnalyzer.java:12646)
> at org.apache.hadoop.hive.ql.parse.CalcitePlanner.analyzeInternal(CalcitePlanner.java:358)
> at org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer.analyze(BaseSemanticAnalyzer.java:283)
> at org.apache.hadoop.hive.ql.Compiler.analyze(Compiler.java:219)
> at org.apache.hadoop.hive.ql.Compiler.compile(Compiler.java:103)
> at org.apache.hadoop.hive.ql.Driver.compile(Driver.java:215){code}



--
This message was sent by Atlassian Jira
(v8.20.7#820007)