diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java index 42c1546992f..7b153ec3dac 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java @@ -901,6 +901,7 @@ class AsyncProcess { LOG.info("#" + id + ", waiting for " + currentInProgress + " actions to finish"); } synchronized (actionsInProgress) { + if (actionsInProgress.get() == 0) break; actionsInProgress.wait(100); } } @@ -979,6 +980,7 @@ class AsyncProcess { oldInProgress = currentInProgress; try { synchronized (this.tasksInProgress) { + if (tasksInProgress.get() != oldInProgress) break; this.tasksInProgress.wait(100); } } catch (InterruptedException e) { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java index 91ea6903092..4802f1d7db2 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java @@ -1488,6 +1488,7 @@ public class RpcClient { } try { synchronized (call) { + if (call.done) break; call.wait(Math.min(call.remainingTime(), 1000) + 1); } } catch (InterruptedException e) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java index 411cb706991..a1f9a507216 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java @@ -334,7 +334,8 @@ public class BucketCache implements BlockCache, HeapSize { if (!successfulAddition && wait) { synchronized (cacheWaitSignals[queueNum]) { try { - cacheWaitSignals[queueNum].wait(DEFAULT_CACHE_WAIT_TIME); + successfulAddition = bq.offer(re); + if (!successfulAddition) cacheWaitSignals[queueNum].wait(DEFAULT_CACHE_WAIT_TIME); } catch (InterruptedException ie) { Thread.currentThread().interrupt(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index c92c020d829..e7b9db897f8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -543,8 +543,8 @@ public class HMaster extends HRegionServer implements MasterServices, Server { this.initializationBeforeMetaAssignment = true; // Wait for regionserver to finish initialization. - while (!isStopped() && !isOnline()) { - synchronized (online) { + synchronized (online) { + while (!isStopped() && !isOnline()) { online.wait(100); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java index 3591188816a..b20231bce51 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java @@ -511,7 +511,8 @@ public class ServerManager { long previousLogTime = 0; ServerName sn = master.getServerName(); ZooKeeperWatcher zkw = master.getZooKeeper(); - while (!onlineServers.isEmpty()) { + int onlineServersCt; + while ((onlineServersCt = onlineServers.size()) > 0){ if (System.currentTimeMillis() > (previousLogTime + 1000)) { Set remainingServers = onlineServers.keySet(); @@ -548,7 +549,7 @@ public class ServerManager { } synchronized (onlineServers) { try { - onlineServers.wait(100); + if (onlineServersCt == onlineServers.size()) onlineServers.wait(100); } catch (InterruptedException ignored) { // continue } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java index c8d3b6aaa16..62441754ce9 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java @@ -75,7 +75,7 @@ class LogRoller extends HasThread implements WALActionsListener { if (!periodic) { synchronized (rollLog) { try { - rollLog.wait(this.threadWakeFrequency); + if (!rollLog.get()) rollLog.wait(this.threadWakeFrequency); } catch (InterruptedException e) { // Fall through } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java index d0005bccd14..2749b18c6e0 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java @@ -264,7 +264,7 @@ public class OpenRegionHandler extends EventHandler { try { // Wait for 10 seconds, so that server shutdown // won't take too long if this thread happens to run. - signaller.wait(10000); + if (!signaller.get()) signaller.wait(10000); } catch (InterruptedException e) { // Go to the loop check. }