hive-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From prasan...@apache.org
Subject hive git commit: HIVE-16103: LLAP: Scheduler timeout monitor never stops with slot nodes (Prasanth Jayachandran reviewed by Sergey Shelukhin)
Date Thu, 09 Mar 2017 10:23:06 GMT
Repository: hive
Updated Branches:
  refs/heads/master ed2f46aa7 -> 0a5ea20a1


HIVE-16103: LLAP: Scheduler timeout monitor never stops with slot nodes (Prasanth Jayachandran
reviewed by Sergey Shelukhin)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/0a5ea20a
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/0a5ea20a
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/0a5ea20a

Branch: refs/heads/master
Commit: 0a5ea20a18bc3e208ff54cb6a81992974b2979f7
Parents: ed2f46a
Author: Prasanth Jayachandran <prasanthj@apache.org>
Authored: Thu Mar 9 02:22:51 2017 -0800
Committer: Prasanth Jayachandran <prasanthj@apache.org>
Committed: Thu Mar 9 02:22:51 2017 -0800

----------------------------------------------------------------------
 .../hive/llap/registry/impl/LlapZookeeperRegistryImpl.java     | 3 ++-
 .../hadoop/hive/llap/tezplugins/LlapTaskSchedulerService.java  | 6 +++++-
 2 files changed, 7 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/0a5ea20a/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java
----------------------------------------------------------------------
diff --git a/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java
b/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java
index 0f8ff66..ad17144 100644
--- a/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java
+++ b/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java
@@ -711,7 +711,8 @@ public class LlapZookeeperRegistryImpl implements ServiceRegistry {
 
     @Override
     public int size() {
-      return instancesCache.getCurrentData().size();
+      // not using the path child cache here as there could be more than 1 path per host
(worker and slot znodes)
+      return nodeToInstanceCache.size();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0a5ea20a/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskSchedulerService.java
----------------------------------------------------------------------
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 8fb0966..6bedccb 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
@@ -47,6 +47,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.locks.Condition;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
@@ -181,6 +182,7 @@ public class LlapTaskSchedulerService extends TaskScheduler {
   private final ScheduledExecutorService scheduledLoggingExecutor;
   private final SchedulerTimeoutMonitor timeoutMonitor;
   private ScheduledFuture<?> timeoutFuture;
+  private final AtomicReference<ScheduledFuture<?>> timeoutFutureRef = new AtomicReference<>(null);
 
   private final AtomicInteger assignedTaskCounter = new AtomicInteger(0);
 
@@ -389,6 +391,7 @@ public class LlapTaskSchedulerService extends TaskScheduler {
       if ((timeoutFuture == null || (timeoutFuture != null && timeoutFuture.isDone()))
           && activeInstances.size() == 0) {
         timeoutFuture = timeoutExecutor.schedule(timeoutMonitor, timeout, TimeUnit.MILLISECONDS);
+        timeoutFutureRef.set(timeoutFuture);
         LOG.info("Scheduled timeout monitor task to run after {} ms", timeout);
       } else {
         LOG.info("Timeout monitor task not started. Timeout future state: {}, #instances:
{}",
@@ -403,6 +406,7 @@ public class LlapTaskSchedulerService extends TaskScheduler {
     timeoutLock.lock();
     try {
       if (timeoutFuture != null && activeInstances.size() != 0 && timeoutFuture.cancel(false))
{
+        timeoutFutureRef.set(null);
         LOG.info("Stopped timeout monitor task");
       } else {
         LOG.info("Timeout monitor task not stopped. Timeout future state: {}, #instances:
{}",
@@ -916,7 +920,7 @@ public class LlapTaskSchedulerService extends TaskScheduler {
 
   private void addNode(NodeInfo node, ServiceInstance serviceInstance) {
     // we have just added a new node. Signal timeout monitor to reset timer
-    if (activeInstances.size() == 1) {
+    if (activeInstances.size() != 0 && timeoutFutureRef.get() != null) {
       LOG.info("New node added. Signalling scheduler timeout monitor thread to stop timer.");
       stopTimeoutMonitor();
     }


Mime
View raw message