This is an automated email from the ASF dual-hosted git repository.

hashutosh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/master by this push:
     new b632b2a  HIVE-23208 : Update guaranteed capacity in ZK only when WM is 
enabled (Ramesh Kumar via Ashutosh Chauhan)
b632b2a is described below

commit b632b2a2afe90ff37971143a983897cb55238a94
Author: RAMESH KUMAR THANGARAJAN <rameshkumarthangara...@gmail.com>
AuthorDate: Thu Apr 16 08:38:14 2020 -0700

    HIVE-23208 : Update guaranteed capacity in ZK only when WM is enabled 
(Ramesh Kumar via Ashutosh Chauhan)
    
    Signed-off-by: Ashutosh Chauhan <hashut...@apache.org>
---
 .../apache/hadoop/hive/llap/tezplugins/LlapTaskSchedulerService.java | 5 ++++-
 1 file changed, 4 insertions(+), 1 deletion(-)

diff --git 
a/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskSchedulerService.java
 
b/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskSchedulerService.java
index 48501e5..565afdc 100644
--- 
a/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskSchedulerService.java
+++ 
b/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskSchedulerService.java
@@ -16,6 +16,7 @@ package org.apache.hadoop.hive.llap.tezplugins;
 
 import com.google.common.io.ByteArrayDataOutput;
 
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.hive.llap.tezplugins.metrics.LlapMetricsCollector;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.metrics2.MetricsSource;
@@ -1088,7 +1089,9 @@ public class LlapTaskSchedulerService extends 
TaskScheduler {
     } finally {
       writeLock.unlock();
     }
-    updateGuaranteedInRegistry(tgVersionForZk, 0);
+    if 
(!StringUtils.isEmpty(conf.get(ConfVars.HIVE_SERVER2_TEZ_INTERACTIVE_QUEUE.varname,
 "").trim())) {
+      updateGuaranteedInRegistry(tgVersionForZk, 0);
+    }
     // TODO Cleanup pending tasks etc, so that the next dag is not affected.
   }
 

Reply via email to