diff --git a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java index e344ab5..27e2755 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java @@ -1189,6 +1189,8 @@ private int compileInternal(String command) { return ret; } + private final static String lockAcquiredMsg = "Acquired the compile lock"; + private final static String waitingForLockMsg = "Waiting to acquire compile lock"; /** * Acquires the compile lock. If the compile lock wait timeout is configured, * it will acquire the lock if it is not held by another thread within the given @@ -1205,11 +1207,25 @@ private ReentrantLock tryAcquireCompileLock(boolean isParallelEnabled, long maxCompileLockWaitTime = HiveConf.getTimeVar( this.conf, ConfVars.HIVE_SERVER2_COMPILE_LOCK_TIMEOUT, TimeUnit.SECONDS); + + if (LOG.isDebugEnabled()) { + LOG.debug(waitingForLockMsg + ": " + command); + } + + // First shot without waiting. + if (compileLock.tryLock()) { + LOG.debug(lockAcquiredMsg); + return compileLock; + } + + // If the first shot fails, then we add the waiting message to operational log. + OperationLog ol = OperationLog.getCurrentOperationLog(); + if (ol != null) { + ol.writeOperationLog(LoggingLevel.EXECUTION, waitingForLockMsg + "\n"); + } + if (maxCompileLockWaitTime > 0) { - try { - if (LOG.isDebugEnabled()) { - LOG.debug("Waiting to acquire compile lock: " + command); - } + try { if(!compileLock.tryLock(maxCompileLockWaitTime, TimeUnit.SECONDS)) { errorMessage = ErrorMsg.COMPILE_LOCK_TIMED_OUT.getErrorCodedMsg(); LOG.error(errorMessage + ": " + command); @@ -1226,7 +1242,7 @@ private ReentrantLock tryAcquireCompileLock(boolean isParallelEnabled, compileLock.lock(); } - LOG.debug("Acquired the compile lock"); + LOG.debug(lockAcquiredMsg); return compileLock; }