Index: ql/src/test/results/clientnegative/lockneg1.q.out =================================================================== --- ql/src/test/results/clientnegative/lockneg1.q.out (revision 1034004) +++ ql/src/test/results/clientnegative/lockneg1.q.out (working copy) @@ -31,5 +31,5 @@ POSTHOOK: Lineage: tstsrc.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] PREHOOK: query: LOCK TABLE tstsrc EXCLUSIVE PREHOOK: type: LOCKTABLE -conflicting lock present +conflicting lock present for default@tstsrc mode EXCLUSIVE FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask Index: ql/src/java/org/apache/hadoop/hive/ql/lockmgr/zookeeper/ZooKeeperHiveLockManager.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/lockmgr/zookeeper/ZooKeeperHiveLockManager.java (revision 1034004) +++ ql/src/java/org/apache/hadoop/hive/ql/lockmgr/zookeeper/ZooKeeperHiveLockManager.java (working copy) @@ -161,7 +161,8 @@ if ((childSeq >= 0) && (childSeq < seqNo)) { zooKeeper.delete(res, -1); - console.printError("conflicting lock present "); + console.printError("conflicting lock present for " + key.getName() + + " mode " + mode); return null; } } Index: ql/src/java/org/apache/hadoop/hive/ql/Driver.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/Driver.java (revision 1034004) +++ ql/src/java/org/apache/hadoop/hive/ql/Driver.java (working copy) @@ -423,6 +423,27 @@ } } + public static class LockObjectContainer { + LockObject lck; + + public LockObjectContainer() { + this.lck = null; + } + + public LockObjectContainer(LockObject lck) { + this.lck = lck; + } + + public LockObject getLck() { + return lck; + } + + public void setLck(LockObject lck) { + this.lck = lck; + } + + } + /** * @param t The table to be locked * @param p The partition to be locked @@ -533,11 +554,15 @@ }); // walk the list and acquire the locks - if any lock cant be acquired, release all locks, sleep and retry + LockObjectContainer notFound = new LockObjectContainer(); while (true) { - List hiveLocks = acquireLocks(lockObjects); + notFound.setLck(null); + List hiveLocks = acquireLocks(lockObjects, notFound); if (hiveLocks == null) { if (tryNum == numRetries) { + console.printError("Lock for " + notFound.getLck().getObj().getName() + + " cannot be acquired in " + notFound.getLck().getMode()); throw new SemanticException(ErrorMsg.LOCK_CANNOT_BE_ACQUIRED.getMsg()); } tryNum++; @@ -567,7 +592,7 @@ * Lock the objects specified in the list. The same object is not locked twice, and the list passed is sorted * such that EXCLUSIVE locks occur before SHARED locks. **/ - private List acquireLocks(List lockObjects) throws SemanticException { + private List acquireLocks(List lockObjects, LockObjectContainer notFound) throws SemanticException { // walk the list and acquire the locks - if any lock cant be acquired, release all locks, sleep and retry LockObject prevLockObj = null; List hiveLocks = new ArrayList(); @@ -588,6 +613,7 @@ } if (lock == null) { + notFound.setLck(lockObject); releaseLocks(hiveLocks); return null; }