diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveLockObject.java b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveLockObject.java index 1cc30745988964a8e8bb36f8fc670f153efee6b2..fe30f61ae965b8b94be1b74b98e176f97c2d6122 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveLockObject.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveLockObject.java @@ -70,6 +70,9 @@ public HiveLockObjectData(String data) { lockTime = elem[1]; lockMode = elem[2]; queryStr = elem[3]; + if (elem.length >= 5) { + clientIp = elem[4]; + } } public String getQueryId() { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/zookeeper/ZooKeeperHiveLockManager.java b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/zookeeper/ZooKeeperHiveLockManager.java index 65b913637b167896d1b622224ba958da12ab728e..cbc8d137dbec79f0b679b3438489e5e330957ca6 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/zookeeper/ZooKeeperHiveLockManager.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/zookeeper/ZooKeeperHiveLockManager.java @@ -28,6 +28,7 @@ import org.apache.hadoop.hive.ql.metadata.*; import org.apache.hadoop.hive.ql.session.SessionState.LogHelper; import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.data.Stat; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.Watcher; import org.apache.zookeeper.ZooDefs.Ids; @@ -57,6 +58,8 @@ private int numRetriesForUnLock; private static String clientIp; + + private final Set conflictingLocks = new HashSet(); static { clientIp = "UNKNOWN"; @@ -325,14 +328,43 @@ private ZooKeeperHiveLock lock (HiveLockObject key, HiveLockMode mode, } } if (tryNum >= numRetriesForLock) { + console.printError("Unable to acquire lock for " + key.getDisplayName() + + " mode " + key.getData().getLockMode()); + LOG.error("Exceeds maximum retries with errors: ", e1); + printConflictingLocks(key,conflictingLocks); + conflictingLocks.clear(); throw new LockException(e1); } } } while (tryNum < numRetriesForLock); + if (ret == null) { + console.printError("Unable to acquire lock for " + key.getDisplayName() + + " mode " + key.getData().getLockMode()); + printConflictingLocks(key,conflictingLocks); + } + + conflictingLocks.clear(); return ret; } + private void printConflictingLocks(HiveLockObject key, Set conflictingLocks) { + if (!conflictingLocks.isEmpty()) { + HiveLockObjectData requestedLock = new HiveLockObjectData(key.getData().toString()); + LOG.debug("Requested lock " + key.getDisplayName() + + ":: mode:" + requestedLock.getLockMode() + + "; query:" + requestedLock.getQueryStr()); + for (String conflictingLock : conflictingLocks) { + HiveLockObjectData conflictingLockData = new HiveLockObjectData(conflictingLock); + LOG.debug("Conflicting lock to " + key.getDisplayName() + + ":: mode:" + conflictingLockData.getLockMode() + + ";query:" + conflictingLockData.getQueryStr() + + ";queryId:" + conflictingLockData.getQueryId() + + ";clientIp:" + conflictingLockData.getClientIp()); + } + } + } + private ZooKeeperHiveLock lockPrimitive(HiveLockObject key, HiveLockMode mode, boolean keepAlive, boolean parentCreated) throws KeeperException, InterruptedException { @@ -394,13 +426,23 @@ private ZooKeeperHiveLock lockPrimitive(HiveLockObject key, } if ((childSeq >= 0) && (childSeq < seqNo)) { - zooKeeper.delete(res, -1); - console.printError("conflicting lock present for " - + key.getDisplayName() + " mode " + mode); + try{ + zooKeeper.delete(res, -1); + } finally { + if (LOG.isDebugEnabled()) { + Stat stat = new Stat(); + try { + String data = new String(zooKeeper.getData(child, false, stat)); + conflictingLocks.add(data); + } catch (Exception e) { + //ignored, + } + } + } return null; } } - + return new ZooKeeperHiveLock(res, key, mode); }