diff --git ql/src/java/org/apache/hadoop/hive/ql/lockmgr/zookeeper/ZooKeeperHiveLockManager.java ql/src/java/org/apache/hadoop/hive/ql/lockmgr/zookeeper/ZooKeeperHiveLockManager.java index c2a4806..9b46ae7 100644 --- ql/src/java/org/apache/hadoop/hive/ql/lockmgr/zookeeper/ZooKeeperHiveLockManager.java +++ ql/src/java/org/apache/hadoop/hive/ql/lockmgr/zookeeper/ZooKeeperHiveLockManager.java @@ -285,8 +285,10 @@ private ZooKeeperHiveLock lock (HiveLockObject key, HiveLockMode mode, int tryNum = 0; ZooKeeperHiveLock ret = null; Set conflictingLocks = new HashSet(); + Exception lastException = null; do { + lastException = null; tryNum++; try { if (tryNum > 1) { @@ -298,26 +300,22 @@ private ZooKeeperHiveLock lock (HiveLockObject key, HiveLockMode mode, break; } } catch (Exception e1) { + lastException = e1; if (e1 instanceof KeeperException) { KeeperException e = (KeeperException) e1; switch (e.code()) { case CONNECTIONLOSS: case OPERATIONTIMEOUT: + case NONODE: + case NODEEXISTS: LOG.debug("Possibly transient ZooKeeper exception: ", e); - continue; + break; default: LOG.error("Serious Zookeeper exception: ", e); break; } - } - if (tryNum >= numRetriesForLock) { - console.printError("Unable to acquire " + key.getData().getLockMode() - + ", " + mode + " lock " + key.getDisplayName() + " after " - + tryNum + " attempts."); - LOG.error("Exceeds maximum retries with errors: ", e1); - printConflictingLocks(key,mode,conflictingLocks); - conflictingLocks.clear(); - throw new LockException(e1); + } else { + LOG.error("Other unexpected exception: ", e1); } } } while (tryNum < numRetriesForLock); @@ -327,8 +325,11 @@ private ZooKeeperHiveLock lock (HiveLockObject key, HiveLockMode mode, + ", " + mode + " lock " + key.getDisplayName() + " after " + tryNum + " attempts."); printConflictingLocks(key,mode,conflictingLocks); + if (lastException != null) { + LOG.error("Exceeds maximum retries with errors: ", lastException); + throw new LockException(lastException); + } } - conflictingLocks.clear(); return ret; } @@ -350,6 +351,19 @@ private void printConflictingLocks(HiveLockObject key, HiveLockMode mode, } } + /** + * Creates a primitive lock object on ZooKeeper. + * @param key The lock data + * @param mode The lock mode (HiveLockMode - EXCLUSIVE/SHARED/SEMI_SHARED) + * @param keepAlive If true creating PERSISTENT ZooKeeper locks, otherwise EPHEMERAL ZooKeeper + * locks + * @param parentCreated If we expect, that the parent is already created then true, otherwise + * we will try to create the parents as well + * @param conflictingLocks The set where we should collect the conflicting locks when + * the logging level is set to DEBUG + * @return The created ZooKeeperHiveLock object, null if there was a conflicting lock + * @throws Exception If there was an unexpected Exception + */ private ZooKeeperHiveLock lockPrimitive(HiveLockObject key, HiveLockMode mode, boolean keepAlive, boolean parentCreated, Set conflictingLocks) @@ -390,7 +404,7 @@ private ZooKeeperHiveLock lockPrimitive(HiveLockObject key, int seqNo = getSequenceNumber(res, getLockName(lastName, mode)); if (seqNo == -1) { curatorFramework.delete().forPath(res); - return null; + throw new LockException("The created node does not contain a sequence number: " + res); } List children = curatorFramework.getChildren().forPath(lastName); @@ -584,7 +598,6 @@ public static void releaseAllLocks(HiveConf conf) throws Exception { /** * @param conf Hive configuration - * @param zkpClient The ZooKeeper client * @param key The object to be compared against - if key is null, then get all locks **/ private static List getLocks(HiveConf conf,