You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by pv...@apache.org on 2018/10/15 12:24:45 UTC

hive git commit: HIVE-20722: Switch HS2 CompileLock to use fair locks (Denys Kuzmenko via Peter Vary)

Repository: hive
Updated Branches:
  refs/heads/master 369f0f241 -> 15c53c4c9


HIVE-20722: Switch HS2 CompileLock to use fair locks (Denys Kuzmenko via Peter Vary)


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

Branch: refs/heads/master
Commit: 15c53c4c9b199b03fd308f30502db54305befdf8
Parents: 369f0f2
Author: denys kuzmenko <dk...@cloudera.com>
Authored: Mon Oct 15 14:24:04 2018 +0200
Committer: Peter Vary <pv...@cloudera.com>
Committed: Mon Oct 15 14:24:04 2018 +0200

----------------------------------------------------------------------
 .../apache/hadoop/hive/ql/lock/CompileLock.java  | 16 ++++++++++++----
 .../hadoop/hive/ql/lock/CompileLockFactory.java  | 19 ++++++++-----------
 .../hadoop/hive/ql/session/SessionState.java     |  2 +-
 3 files changed, 21 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/15c53c4c/ql/src/java/org/apache/hadoop/hive/ql/lock/CompileLock.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lock/CompileLock.java b/ql/src/java/org/apache/hadoop/hive/ql/lock/CompileLock.java
index 6cb5fba..90fbfe4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lock/CompileLock.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lock/CompileLock.java
@@ -66,14 +66,14 @@ public final class CompileLock implements AutoCloseable {
     try {
       if (underlying.tryLock(0, unit)) {
         LOG.debug(LOCK_ACQUIRED_MSG);
-        return locked(true);
+        return aquired();
       }
     } catch (InterruptedException e) {
       Thread.currentThread().interrupt();
       if (LOG.isDebugEnabled()) {
         LOG.debug("Interrupted Exception ignored", e);
       }
-      return locked(false);
+      return failedToAquire();
     }
 
     // If the first shot fails, then we log the waiting messages.
@@ -85,23 +85,31 @@ public final class CompileLock implements AutoCloseable {
       try {
         if (!underlying.tryLock(timeout, unit)) {
           LOG.error(ErrorMsg.COMPILE_LOCK_TIMED_OUT.getErrorCodedMsg() + ": " + command);
-          return locked(false);
+          return failedToAquire();
         }
       } catch (InterruptedException e) {
         Thread.currentThread().interrupt();
         if (LOG.isDebugEnabled()) {
           LOG.debug("Interrupted Exception ignored", e);
         }
-        return locked(false);
+        return failedToAquire();
       }
     } else {
       underlying.lock();
     }
 
     LOG.debug(LOCK_ACQUIRED_MSG);
+    return aquired();
+  }
+
+  private boolean aquired() {
     return locked(true);
   }
 
+  private boolean failedToAquire() {
+    return locked(false);
+  }
+
   private boolean locked(boolean isLocked) {
     this.isLocked  = isLocked;
     return isLocked;

http://git-wip-us.apache.org/repos/asf/hive/blob/15c53c4c/ql/src/java/org/apache/hadoop/hive/ql/lock/CompileLockFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lock/CompileLockFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/lock/CompileLockFactory.java
index 848cbc5..3061400 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lock/CompileLockFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lock/CompileLockFactory.java
@@ -32,7 +32,7 @@ import org.apache.hadoop.hive.ql.session.SessionState;
  */
 public final class CompileLockFactory {
 
-  private static final ReentrantLock SERIALIZABLE_COMPILE_LOCK = new ReentrantLock();
+  private static final ReentrantLock SERIALIZABLE_COMPILE_LOCK = new ReentrantLock(true);
 
   private CompileLockFactory() {
   }
@@ -67,12 +67,12 @@ public final class CompileLockFactory {
     private final Semaphore globalCompileQuotas;
 
     SessionWithQuotaCompileLock(int compilePoolSize) {
-      globalCompileQuotas = new Semaphore(compilePoolSize);
+      globalCompileQuotas = new Semaphore(compilePoolSize, true);
     }
 
     @Override
     public void lock() {
-      getSessionLock().lock();
+      SessionState.get().getCompileLock().lock();
       globalCompileQuotas.acquireUninterruptibly();
     }
 
@@ -81,13 +81,14 @@ public final class CompileLockFactory {
       boolean result = false;
       long startTime = System.nanoTime();
 
+      ReentrantLock compileLock = SessionState.get().getCompileLock();
       try {
-        result = getSessionLock().tryLock(time, unit)
+        result = compileLock.tryLock(time, unit)
             && globalCompileQuotas.tryAcquire(
                 getRemainingTime(startTime, unit.toNanos(time)), TimeUnit.NANOSECONDS);
       } finally {
-        if (!result && getSessionLock().isHeldByCurrentThread()) {
-          getSessionLock().unlock();
+        if (!result && compileLock.isHeldByCurrentThread()) {
+          compileLock.unlock();
         }
       }
       return result;
@@ -95,14 +96,10 @@ public final class CompileLockFactory {
 
     @Override
     public void unlock() {
-      getSessionLock().unlock();
+      SessionState.get().getCompileLock().unlock();
       globalCompileQuotas.release();
     }
 
-    private ReentrantLock getSessionLock() {
-      return SessionState.get().getCompileLock();
-    }
-
     private long getRemainingTime(long startTime, long time) {
       long timeout = time - (System.nanoTime() - startTime);
       return (timeout < 0) ? 0 : timeout;

http://git-wip-us.apache.org/repos/asf/hive/blob/15c53c4c/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java b/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
index 71e130b..9291e11 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
@@ -135,7 +135,7 @@ public class SessionState {
   protected ClassLoader parentLoader;
 
   // Session-scope compile lock.
-  private final ReentrantLock compileLock = new ReentrantLock();
+  private final ReentrantLock compileLock = new ReentrantLock(true);
 
   /**
    * current configuration.