dengzhhu653 commented on a change in pull request #2693:
URL: https://github.com/apache/hive/pull/2693#discussion_r733382781



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/exec/LimitOperator.java
##########
@@ -69,8 +69,8 @@ protected void initializeOp(Configuration hconf) throws 
HiveException {
     this.runtimeCache = ObjectCacheFactory.getCache(getConfiguration(), 
queryId, false, true);
 
     // this can happen in HS2 while doing local fetch optimization, where 
LimitOperator is used
-    if (runtimeCache == null) {
-      if (!HiveConf.isLoadHiveServer2Config()) {
+    if (runtimeCache == null || !HiveConf.getVar(hconf, 
HiveConf.ConfVars.HIVE_EXECUTION_ENGINE).equals("tez")) {

Review comment:
        when the engine is not tez, 
[org.apache.hadoop.hive.ql.exec.mr.ObjectCache](https://github.com/apache/hive/blob/1090c93b1a02d480bdee2af2cecf503f8a54efc6/ql/src/java/org/apache/hadoop/hive/ql/exec/ObjectCacheFactory.java#L80-L83)
 will be the runtime cache, we do not persist any thing in this cache. so every 
time we [fetch the row counter from the 
cache](https://github.com/apache/hive/blob/7b3ecf617a6d46f48a3b6f77e0339fd4ad95a420/ql/src/java/org/apache/hadoop/hive/ql/exec/LimitOperator.java#L150-L158),
 a newly created AtomicInteger will be returned,  the condition `offset <= 
currentCountForAllTasksInt` 
https://github.com/apache/hive/blob/7b3ecf617a6d46f48a3b6f77e0339fd4ad95a420/ql/src/java/org/apache/hadoop/hive/ql/exec/LimitOperator.java#L98
 will be evaluated to false with offset > 0, making the operator unable to 
forward any rows to downstream.




-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to