You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by nk...@apache.org on 2014/02/07 21:27:43 UTC

svn commit: r1565787 - in /hbase/trunk: hbase-client/src/main/java/org/apache/hadoop/hbase/client/ hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ hbase-server/src/main/java/...

Author: nkeywal
Date: Fri Feb  7 20:27:42 2014
New Revision: 1565787

URL: http://svn.apache.org/r1565787
Log:
HBASE-10472 Manage the interruption in ZKUtil#getData

Modified:
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperRegistry.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterAddressTracker.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaRegionTracker.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKClusterId.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKLeaderManager.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKTable.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKTableReadOnly.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BalancerChore.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/ClusterStatusChore.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinatorRpcs.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureMemberRpcs.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ZKDataMigrator.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionServerTracker.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/lock/ZKInterProcessLockBase.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKTable.java

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperRegistry.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperRegistry.java?rev=1565787&r1=1565786&r2=1565787&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperRegistry.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperRegistry.java Fri Feb  7 20:27:42 2014
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hbase.client;
 
 import java.io.IOException;
+import java.io.InterruptedIOException;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -105,6 +106,8 @@ class ZooKeeperRegistry implements Regis
       return ZKTableReadOnly.isDisabledTable(zkw, tableName);
     } catch (KeeperException e) {
       throw new IOException("Enable/Disable failed", e);
+    } catch (InterruptedException e) {
+      throw new InterruptedIOException();
     } finally {
        zkw.close();
     }

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java?rev=1565787&r1=1565786&r2=1565787&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java Fri Feb  7 20:27:42 2014
@@ -159,6 +159,8 @@ public class ReplicationPeersZKImpl exte
       throw new ReplicationException(e);
     } catch (DeserializationException e) {
       throw new ReplicationException(e);
+    } catch (InterruptedException e) {
+      throw new ReplicationException(e);
     }
   }
 
@@ -212,6 +214,8 @@ public class ReplicationPeersZKImpl exte
       }
     } catch (KeeperException e) {
       this.abortable.abort("Cannot get the list of peers ", e);
+    } catch (InterruptedException e) {
+      this.abortable.abort("Cannot get the list of peers ", e);
     }
     return peers;
   }
@@ -268,6 +272,11 @@ public class ReplicationPeersZKImpl exte
     } catch (KeeperException e) {
       throw new ReplicationException("Error getting configuration for peer with id="
           + peerId, e);
+    } catch (InterruptedException e) {
+      LOG.warn("Could not get configuration for peer because the thread " +
+          "was interrupted. peerId=" + peerId);
+      Thread.currentThread().interrupt();
+      return null;
     }
     if (data == null) {
       LOG.error("Could not get configuration for peer because it doesn't exist. peerId=" + peerId);

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java?rev=1565787&r1=1565786&r2=1565787&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java Fri Feb  7 20:27:42 2014
@@ -141,6 +141,9 @@ public class ReplicationQueuesZKImpl ext
     } catch (KeeperException e) {
       throw new ReplicationException("Internal Error: could not get position in log for queueId="
           + queueId + ", filename=" + filename, e);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      return 0;
     }
     try {
       return ZKUtil.parseHLogPositionFrom(bytes);
@@ -338,6 +341,10 @@ public class ReplicationQueuesZKImpl ext
       // Multi call failed; it looks like some other regionserver took away the logs.
       LOG.warn("Got exception in copyQueuesFromRSUsingMulti: ", e);
       queues.clear();
+    } catch (InterruptedException e) {
+      LOG.warn("Got exception in copyQueuesFromRSUsingMulti: ", e);
+      queues.clear();
+      Thread.currentThread().interrupt();
     }
     return queues;
   }
@@ -403,6 +410,9 @@ public class ReplicationQueuesZKImpl ext
       }
     } catch (KeeperException e) {
       this.abortable.abort("Copy queues from rs", e);
+    } catch (InterruptedException e) {
+      LOG.warn(e);
+      Thread.currentThread().interrupt();
     }
     return queues;
   }

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterAddressTracker.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterAddressTracker.java?rev=1565787&r1=1565786&r2=1565787&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterAddressTracker.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterAddressTracker.java Fri Feb  7 20:27:42 2014
@@ -29,6 +29,7 @@ import org.apache.zookeeper.KeeperExcept
 import org.apache.zookeeper.data.Stat;
 
 import java.io.IOException;
+import java.io.InterruptedIOException;
 
 /**
  * Manages the location of the current active Master for the RegionServer.
@@ -103,7 +104,12 @@ public class MasterAddressTracker extend
    */
   public static ServerName getMasterAddress(final ZooKeeperWatcher zkw)
   throws KeeperException, IOException {
-    byte [] data = ZKUtil.getData(zkw, zkw.getMasterAddressZNode());
+    byte [] data;
+    try {
+      data = ZKUtil.getData(zkw, zkw.getMasterAddressZNode());
+    } catch (InterruptedException e) {
+      throw new InterruptedIOException();
+    }
     if (data == null){
       throw new IOException("Can't get master address from ZooKeeper; znode data == null");
     }

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaRegionTracker.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaRegionTracker.java?rev=1565787&r1=1565786&r2=1565787&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaRegionTracker.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaRegionTracker.java Fri Feb  7 20:27:42 2014
@@ -81,6 +81,9 @@ public class MetaRegionTracker extends Z
       return ServerName.parseFrom(ZKUtil.getData(zkw, zkw.metaServerZNode));
     } catch (DeserializationException e) {
       throw ZKUtil.convert(e);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      return null;
     }
   }
 

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKClusterId.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKClusterId.java?rev=1565787&r1=1565786&r2=1565787&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKClusterId.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKClusterId.java Fri Feb  7 20:27:42 2014
@@ -63,7 +63,13 @@ public class ZKClusterId {
   public static String readClusterIdZNode(ZooKeeperWatcher watcher)
   throws KeeperException {
     if (ZKUtil.checkExists(watcher, watcher.clusterIdZNode) != -1) {
-      byte [] data = ZKUtil.getData(watcher, watcher.clusterIdZNode);
+      byte [] data;
+      try {
+        data = ZKUtil.getData(watcher, watcher.clusterIdZNode);
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        return null;
+      }
       if (data != null) {
         try {
           return ClusterId.parseFrom(data).toString();

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKLeaderManager.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKLeaderManager.java?rev=1565787&r1=1565786&r2=1565787&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKLeaderManager.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKLeaderManager.java Fri Feb  7 20:27:42 2014
@@ -170,6 +170,10 @@ public class ZKLeaderManager extends Zoo
       watcher.abort("Unhandled zookeeper exception removing leader node", ke);
       candidate.stop("Unhandled zookeeper exception removing leader node: "
           + ke.getMessage());
+    } catch (InterruptedException e) {
+      watcher.abort("Unhandled zookeeper exception removing leader node", e);
+      candidate.stop("Unhandled zookeeper exception removing leader node: "
+          + e.getMessage());
     }
   }
 

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKTable.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKTable.java?rev=1565787&r1=1565786&r2=1565787&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKTable.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKTable.java Fri Feb  7 20:27:42 2014
@@ -26,6 +26,7 @@ import org.apache.hadoop.hbase.protobuf.
 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
 import org.apache.zookeeper.KeeperException;
 
+import java.io.InterruptedIOException;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
@@ -63,7 +64,7 @@ public class ZKTable {
   // TODO: Make it so always a table znode. Put table schema here as well as table state.
   // Have watcher on table znode so all are notified of state or schema change.
 
-  public ZKTable(final ZooKeeperWatcher zkw) throws KeeperException {
+  public ZKTable(final ZooKeeperWatcher zkw) throws KeeperException, InterruptedException {
     super();
     this.watcher = zkw;
     populateTableStates();
@@ -74,7 +75,7 @@ public class ZKTable {
    * @throws KeeperException
    */
   private void populateTableStates()
-  throws KeeperException {
+      throws KeeperException, InterruptedException {
     synchronized (this.cache) {
       List<String> children = ZKUtil.listChildrenNoWatch(this.watcher, this.watcher.tableZNode);
       if (children == null) return;
@@ -316,7 +317,7 @@ public class ZKTable {
    * @throws KeeperException
    */
   public static Set<TableName> getDisabledTables(ZooKeeperWatcher zkw)
-      throws KeeperException {
+      throws KeeperException, InterruptedIOException {
     return getAllTables(zkw, ZooKeeperProtos.Table.State.DISABLED);
   }
 
@@ -326,7 +327,7 @@ public class ZKTable {
    * @throws KeeperException
    */
   public static Set<TableName> getDisablingTables(ZooKeeperWatcher zkw)
-      throws KeeperException {
+      throws KeeperException, InterruptedIOException {
     return getAllTables(zkw, ZooKeeperProtos.Table.State.DISABLING);
   }
 
@@ -336,7 +337,7 @@ public class ZKTable {
    * @throws KeeperException
    */
   public static Set<TableName> getEnablingTables(ZooKeeperWatcher zkw)
-      throws KeeperException {
+      throws KeeperException, InterruptedIOException {
     return getAllTables(zkw, ZooKeeperProtos.Table.State.ENABLING);
   }
 
@@ -346,7 +347,7 @@ public class ZKTable {
    * @throws KeeperException
    */
   public static Set<TableName> getDisabledOrDisablingTables(ZooKeeperWatcher zkw)
-      throws KeeperException {
+      throws KeeperException, InterruptedIOException {
     return getAllTables(zkw, ZooKeeperProtos.Table.State.DISABLED,
       ZooKeeperProtos.Table.State.DISABLING);
   }
@@ -380,14 +381,19 @@ public class ZKTable {
    * @throws KeeperException
    */
   static Set<TableName> getAllTables(final ZooKeeperWatcher zkw,
-      final ZooKeeperProtos.Table.State... states) throws KeeperException {
+      final ZooKeeperProtos.Table.State... states) throws KeeperException, InterruptedIOException {
     Set<TableName> allTables = new HashSet<TableName>();
     List<String> children =
       ZKUtil.listChildrenNoWatch(zkw, zkw.tableZNode);
     if(children == null) return allTables;
     for (String child: children) {
       TableName tableName = TableName.valueOf(child);
-      ZooKeeperProtos.Table.State state = ZKTableReadOnly.getTableState(zkw, tableName);
+      ZooKeeperProtos.Table.State state = null;
+      try {
+        state = ZKTableReadOnly.getTableState(zkw, tableName);
+      } catch (InterruptedException e) {
+        throw new InterruptedIOException();
+      }
       for (ZooKeeperProtos.Table.State expectedState: states) {
         if (state == expectedState) {
           allTables.add(tableName);

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKTableReadOnly.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKTableReadOnly.java?rev=1565787&r1=1565786&r2=1565787&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKTableReadOnly.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKTableReadOnly.java Fri Feb  7 20:27:42 2014
@@ -55,7 +55,7 @@ public class ZKTableReadOnly {
    */
   public static boolean isDisabledTable(final ZooKeeperWatcher zkw,
       final TableName tableName)
-  throws KeeperException {
+      throws KeeperException, InterruptedException {
     ZooKeeperProtos.Table.State state = getTableState(zkw, tableName);
     return isTableState(ZooKeeperProtos.Table.State.DISABLED, state);
   }
@@ -71,7 +71,7 @@ public class ZKTableReadOnly {
    */
   public static boolean isEnabledTable(final ZooKeeperWatcher zkw,
       final TableName tableName)
-  throws KeeperException {
+      throws KeeperException, InterruptedException {
     return getTableState(zkw, tableName) == ZooKeeperProtos.Table.State.ENABLED;
   }
 
@@ -87,7 +87,7 @@ public class ZKTableReadOnly {
    */
   public static boolean isDisablingOrDisabledTable(final ZooKeeperWatcher zkw,
       final TableName tableName)
-  throws KeeperException {
+      throws KeeperException, InterruptedException {
     ZooKeeperProtos.Table.State state = getTableState(zkw, tableName);
     return isTableState(ZooKeeperProtos.Table.State.DISABLING, state) ||
       isTableState(ZooKeeperProtos.Table.State.DISABLED, state);
@@ -99,7 +99,7 @@ public class ZKTableReadOnly {
    * @throws KeeperException
    */
   public static Set<TableName> getDisabledTables(ZooKeeperWatcher zkw)
-  throws KeeperException {
+      throws KeeperException, InterruptedException {
     Set<TableName> disabledTables = new HashSet<TableName>();
     List<String> children =
       ZKUtil.listChildrenNoWatch(zkw, zkw.tableZNode);
@@ -118,7 +118,7 @@ public class ZKTableReadOnly {
    * @throws KeeperException
    */
   public static Set<TableName> getDisabledOrDisablingTables(ZooKeeperWatcher zkw)
-  throws KeeperException {
+      throws KeeperException, InterruptedException {
     Set<TableName> disabledTables = new HashSet<TableName>();
     List<String> children =
       ZKUtil.listChildrenNoWatch(zkw, zkw.tableZNode);
@@ -146,7 +146,7 @@ public class ZKTableReadOnly {
    */
   static ZooKeeperProtos.Table.State getTableState(final ZooKeeperWatcher zkw,
       final TableName tableName)
-  throws KeeperException {
+      throws KeeperException, InterruptedException {
     String znode = ZKUtil.joinZNode(zkw.tableZNode, tableName.getNameAsString());
     byte [] data = ZKUtil.getData(zkw, znode);
     if (data == null || data.length <= 0) return null;

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java?rev=1565787&r1=1565786&r2=1565787&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java Fri Feb  7 20:27:42 2014
@@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.protobuf.
 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionStoreSequenceIds;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.ExceptionUtil;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp.CreateAndFailSilent;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp.DeleteNodeFailSilent;
@@ -678,23 +679,19 @@ public class ZKUtil {
    *  error.
    */
   public static byte [] getData(ZooKeeperWatcher zkw, String znode)
-  throws KeeperException {
+      throws KeeperException, InterruptedException {
     try {
       byte [] data = zkw.getRecoverableZooKeeper().getData(znode, null, null);
       logRetrievedMsg(zkw, znode, data, false);
       return data;
     } catch (KeeperException.NoNodeException e) {
       LOG.debug(zkw.prefix("Unable to get data of znode " + znode + " " +
-        "because node does not exist (not an error)"));
+          "because node does not exist (not an error)"));
       return null;
     } catch (KeeperException e) {
       LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e);
       zkw.keeperException(e);
       return null;
-    } catch (InterruptedException e) {
-      LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e);
-      zkw.interruptedException(e);
-      return null;
     }
   }
 
@@ -1661,13 +1658,22 @@ public class ZKUtil {
     do {
       String znodeToProcess = stack.remove(stack.size() - 1);
       sb.append("\n").append(znodeToProcess).append(": ");
-      byte[] data = ZKUtil.getData(zkw, znodeToProcess);
+      byte[] data;
+      try {
+        data = ZKUtil.getData(zkw, znodeToProcess);
+      } catch (InterruptedException e) {
+        zkw.interruptedException(e);
+        return;
+      }
       if (data != null && data.length > 0) { // log position
         long position = 0;
         try {
           position = ZKUtil.parseHLogPositionFrom(ZKUtil.getData(zkw, znodeToProcess));
           sb.append(position);
-        } catch (Exception e) {
+        } catch (DeserializationException ignored) {
+        } catch (InterruptedException e) {
+          zkw.interruptedException(e);
+          return;
         }
       }
       for (String zNodeChild : ZKUtil.listChildrenNoWatch(zkw, znodeToProcess)) {
@@ -1682,7 +1688,13 @@ public class ZKUtil {
     sb.append("\n").append(peersZnode).append(": ");
     for (String peerIdZnode : ZKUtil.listChildrenNoWatch(zkw, peersZnode)) {
       String znodeToProcess = ZKUtil.joinZNode(peersZnode, peerIdZnode);
-      byte[] data = ZKUtil.getData(zkw, znodeToProcess);
+      byte[] data;
+      try {
+        data = ZKUtil.getData(zkw, znodeToProcess);
+      } catch (InterruptedException e) {
+        zkw.interruptedException(e);
+        return;
+      }
       // parse the data of the above peer znode.
       try {
       String clusterKey = ZooKeeperProtos.ReplicationPeer.newBuilder().
@@ -1705,9 +1717,15 @@ public class ZKUtil {
       if (!child.equals(peerState)) continue;
       String peerStateZnode = ZKUtil.joinZNode(znodeToProcess, child);
       sb.append("\n").append(peerStateZnode).append(": ");
-      byte[] peerStateData = ZKUtil.getData(zkw, peerStateZnode);
-      sb.append(ZooKeeperProtos.ReplicationState.newBuilder()
-          .mergeFrom(peerStateData, pblen, peerStateData.length - pblen).getState().name());
+      byte[] peerStateData;
+      try {
+        peerStateData = ZKUtil.getData(zkw, peerStateZnode);
+        sb.append(ZooKeeperProtos.ReplicationState.newBuilder()
+            .mergeFrom(peerStateData, pblen, peerStateData.length - pblen).getState().name());
+      } catch (InterruptedException e) {
+        zkw.interruptedException(e);
+        return;
+      }
     }
   }
 

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java?rev=1565787&r1=1565786&r2=1565787&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java Fri Feb  7 20:27:42 2014
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hbase.master;
 
 import java.io.IOException;
+import java.io.InterruptedIOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -282,7 +283,11 @@ public class AssignmentManager extends Z
       this.timeoutMonitor = null;
       this.timerUpdater = null;
     }
-    this.zkTable = new ZKTable(this.watcher);
+    try {
+      this.zkTable = new ZKTable(this.watcher);
+    } catch (InterruptedException e) {
+      throw new InterruptedIOException();
+    }
     // This is the max attempts, not retries, so it should be at least 1.
     this.maximumAttempts = Math.max(1,
       this.server.getConfiguration().getInt("hbase.assignment.maximum.attempts", 10));

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java?rev=1565787&r1=1565786&r2=1565787&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java Fri Feb  7 20:27:42 2014
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hbase.master;
 
 import java.io.IOException;
+import java.io.InterruptedIOException;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.net.InetAddress;
@@ -1426,7 +1427,7 @@ MasterServices, Server {
     return balancerCutoffTime;
   }
 
-  public boolean balance() throws HBaseIOException {
+  public boolean balance() throws IOException {
     // if master not initialized, don't run balancer.
     if (!this.initialized) {
       LOG.debug("Master has not been initialized, don't run balancer.");
@@ -1513,7 +1514,7 @@ MasterServices, Server {
   public BalanceResponse balance(RpcController c, BalanceRequest request) throws ServiceException {
     try {
       return BalanceResponse.newBuilder().setBalancerRan(balance()).build();
-    } catch (HBaseIOException ex) {
+    } catch (IOException ex) {
       throw new ServiceException(ex);
     }
   }
@@ -2078,14 +2079,18 @@ MasterServices, Server {
       GetClusterStatusRequest req)
   throws ServiceException {
     GetClusterStatusResponse.Builder response = GetClusterStatusResponse.newBuilder();
-    response.setClusterStatus(getClusterStatus().convert());
+    try {
+      response.setClusterStatus(getClusterStatus().convert());
+    } catch (InterruptedIOException e) {
+      throw new ServiceException(e);
+    }
     return response.build();
   }
 
   /**
    * @return cluster status
    */
-  public ClusterStatus getClusterStatus() {
+  public ClusterStatus getClusterStatus() throws InterruptedIOException {
     // Build Set of backup masters from ZK nodes
     List<String> backupMasterStrings;
     try {
@@ -2099,9 +2104,13 @@ MasterServices, Server {
                                           backupMasterStrings.size());
     for (String s: backupMasterStrings) {
       try {
-        byte [] bytes =
-            ZKUtil.getData(this.zooKeeper, ZKUtil.joinZNode(
-                this.zooKeeper.backupMasterAddressesZNode, s));
+        byte [] bytes;
+        try {
+          bytes = ZKUtil.getData(this.zooKeeper, ZKUtil.joinZNode(
+              this.zooKeeper.backupMasterAddressesZNode, s));
+        } catch (InterruptedException e) {
+          throw new InterruptedIOException();
+        }
         if (bytes != null) {
           ServerName sn;
           try {

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java?rev=1565787&r1=1565786&r2=1565787&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java Fri Feb  7 20:27:42 2014
@@ -390,7 +390,7 @@ public class MasterFileSystem {
    * @throws KeeperException
    */
   void removeStaleRecoveringRegionsFromZK(final Set<ServerName> failedServers)
-      throws KeeperException {
+      throws KeeperException, InterruptedIOException {
     this.splitLogManager.removeStaleRecoveringRegionsFromZK(failedServers);
   }
 

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java?rev=1565787&r1=1565786&r2=1565787&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java Fri Feb  7 20:27:42 2014
@@ -25,6 +25,7 @@ import static org.apache.hadoop.hbase.ma
 import static org.apache.hadoop.hbase.master.SplitLogManager.TerminationStatus.SUCCESS;
 
 import java.io.IOException;
+import java.io.InterruptedIOException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashSet;
@@ -569,7 +570,7 @@ public class SplitLogManager extends Zoo
    * @throws KeeperException
    */
   void removeStaleRecoveringRegionsFromZK(final Set<ServerName> failedServers)
-      throws KeeperException {
+      throws KeeperException, InterruptedIOException {
 
     if (!this.distributedLogReplay) {
       // remove any regions in recovery from ZK which could happen when we turn the feature on
@@ -591,7 +592,12 @@ public class SplitLogManager extends Zoo
       List<String> tasks = ZKUtil.listChildrenNoWatch(watcher, watcher.splitLogZNode);
       if (tasks != null) {
         for (String t : tasks) {
-          byte[] data = ZKUtil.getData(this.watcher, ZKUtil.joinZNode(watcher.splitLogZNode, t));
+          byte[] data;
+          try {
+            data = ZKUtil.getData(this.watcher, ZKUtil.joinZNode(watcher.splitLogZNode, t));
+          } catch (InterruptedException e) {
+            throw new InterruptedIOException();
+          }
           if (data != null) {
             SplitLogTask slt = null;
             try {
@@ -1115,7 +1121,7 @@ public class SplitLogManager extends Zoo
    * @param userRegions user regiones assigned on the region server
    */
   void markRegionsRecoveringInZK(final ServerName serverName, Set<HRegionInfo> userRegions)
-      throws KeeperException {
+      throws KeeperException, InterruptedIOException {
     if (userRegions == null || !this.distributedLogReplay) {
       return;
     }
@@ -1172,9 +1178,11 @@ public class SplitLogManager extends Zoo
             // wait a little bit for retry
             try {
               Thread.sleep(20);
-            } catch (Exception ignoreE) {
-              // ignore
+            } catch (InterruptedException e1) {
+              throw new InterruptedIOException();
             }
+          } catch (InterruptedException e) {
+            throw new InterruptedIOException();
           }
         } while ((--retries) > 0 && (!this.stopper.isStopped()));
       }
@@ -1240,7 +1248,12 @@ public class SplitLogManager extends Zoo
     String nodePath = ZKUtil.joinZNode(zkw.recoveringRegionsZNode, encodedRegionName);
     nodePath = ZKUtil.joinZNode(nodePath, serverName);
     try {
-      byte[] data = ZKUtil.getData(zkw, nodePath);
+      byte[] data;
+      try {
+        data = ZKUtil.getData(zkw, nodePath);
+      } catch (InterruptedException e) {
+        throw new InterruptedIOException();
+      }
       if (data != null) {
         result = ZKUtil.parseRegionStoreSequenceIds(data);
       }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BalancerChore.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BalancerChore.java?rev=1565787&r1=1565786&r2=1565787&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BalancerChore.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BalancerChore.java Fri Feb  7 20:27:42 2014
@@ -25,6 +25,8 @@ import org.apache.hadoop.hbase.Chore;
 import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.master.HMaster;
 
+import java.io.IOException;
+
 /**
  * Chore that will call HMaster.balance{@link org.apache.hadoop.hbase.master.HMaster#balance()} when
  * needed.
@@ -46,7 +48,7 @@ public class BalancerChore extends Chore
   protected void chore() {
     try {
       master.balance();
-    } catch (HBaseIOException e) {
+    } catch (IOException e) {
       LOG.error("Failed to balance.", e);
     }
   }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/ClusterStatusChore.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/ClusterStatusChore.java?rev=1565787&r1=1565786&r2=1565787&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/ClusterStatusChore.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/ClusterStatusChore.java Fri Feb  7 20:27:42 2014
@@ -18,17 +18,21 @@
 
 package org.apache.hadoop.hbase.master.balancer;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.Chore;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.LoadBalancer;
 
+import java.io.InterruptedIOException;
+
 /**
  * Chore that will feed the balancer the cluster status.
  */
 @InterfaceAudience.Private
 public class ClusterStatusChore extends Chore {
-
+  private static final Log LOG = LogFactory.getLog(ClusterStatusChore.class);
   private final HMaster master;
   private final LoadBalancer balancer;
 
@@ -42,6 +46,10 @@ public class ClusterStatusChore extends 
 
   @Override
   protected void chore() {
-     balancer.setClusterStatus(master.getClusterStatus());
+    try {
+      balancer.setClusterStatus(master.getClusterStatus());
+    } catch (InterruptedIOException e) {
+      LOG.warn("Ignoring interruption", e);
+    }
   }
 }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinatorRpcs.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinatorRpcs.java?rev=1565787&r1=1565786&r2=1565787&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinatorRpcs.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinatorRpcs.java Fri Feb  7 20:27:42 2014
@@ -250,6 +250,10 @@ public class ZKProcedureCoordinatorRpcs 
     } catch (KeeperException e) {
       coordinator.rpcConnectionFailure("Failed to get data for abort node:" + abortNode
           + zkProc.getAbortZnode(), new IOException(e));
+    } catch (InterruptedException e) {
+      coordinator.rpcConnectionFailure("Failed to get data for abort node:" + abortNode
+          + zkProc.getAbortZnode(), new IOException(e));
+      Thread.currentThread().interrupt();
     }
     coordinator.abortProcedure(procName, ee);
   }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureMemberRpcs.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureMemberRpcs.java?rev=1565787&r1=1565786&r2=1565787&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureMemberRpcs.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureMemberRpcs.java Fri Feb  7 20:27:42 2014
@@ -221,6 +221,10 @@ public class ZKProcedureMemberRpcs imple
     } catch (KeeperException e) {
       member.controllerConnectionFailure("Failed to get data for new procedure:" + opName,
         new IOException(e));
+    } catch (InterruptedException e) {
+      member.controllerConnectionFailure("Failed to get data for new procedure:" + opName,
+          new IOException(e));
+      Thread.currentThread().interrupt();
     }
   }
 
@@ -330,6 +334,9 @@ public class ZKProcedureMemberRpcs imple
     } catch (KeeperException e) {
       member.controllerConnectionFailure("Failed to get data for abort znode:" + abortZNode
           + zkController.getAbortZnode(), new IOException(e));
+    } catch (InterruptedException e) {
+      LOG.warn("abort already in progress", e);
+      Thread.currentThread().interrupt();
     }
   }
 

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java?rev=1565787&r1=1565786&r2=1565787&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java Fri Feb  7 20:27:42 2014
@@ -4513,7 +4513,12 @@ public class HRegionServer implements Cl
     String nodePath = ZKUtil.joinZNode(this.zooKeeper.recoveringRegionsZNode,
       region.getEncodedName());
     // recovering-region level
-    byte[] data = ZKUtil.getData(zkw, nodePath);
+    byte[] data;
+    try {
+      data = ZKUtil.getData(zkw, nodePath);
+    } catch (InterruptedException e) {
+      throw new InterruptedIOException();
+    }
     if (data != null) {
       lastRecordedFlushedSequenceId = SplitLogManager.parseLastFlushedSequenceIdFrom(data);
     }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java?rev=1565787&r1=1565786&r2=1565787&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java Fri Feb  7 20:27:42 2014
@@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.util;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.io.InterruptedIOException;
 import java.io.PrintWriter;
 import java.io.StringWriter;
 import java.net.URI;
@@ -1384,6 +1385,8 @@ public class HBaseFsck extends Configure
           }
         } catch (KeeperException ke) {
           throw new IOException(ke);
+        } catch (InterruptedException e) {
+          throw new InterruptedIOException();
         } finally {
           zkw.close();
         }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ZKDataMigrator.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ZKDataMigrator.java?rev=1565787&r1=1565786&r2=1565787&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ZKDataMigrator.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ZKDataMigrator.java Fri Feb  7 20:27:42 2014
@@ -131,7 +131,8 @@ public class ZKDataMigrator extends Conf
     return 0;
   }
 
-  private void checkAndMigrateTableStatesToPB(ZooKeeperWatcher zkw) throws KeeperException {
+  private void checkAndMigrateTableStatesToPB(ZooKeeperWatcher zkw) throws KeeperException,
+      InterruptedException {
     List<String> tables = ZKUtil.listChildrenNoWatch(zkw, zkw.tableZNode);
     if (tables == null) {
       LOG.info("No table present to migrate table state to PB. returning..");
@@ -154,7 +155,8 @@ public class ZKDataMigrator extends Conf
     }
   }
 
-  private void checkAndMigrateReplicationNodesToPB(ZooKeeperWatcher zkw) throws KeeperException {
+  private void checkAndMigrateReplicationNodesToPB(ZooKeeperWatcher zkw) throws KeeperException,
+      InterruptedException {
     String replicationZnodeName = getConf().get("zookeeper.znode.replication", "replication");
     String replicationPath = ZKUtil.joinZNode(zkw.baseZNode, replicationZnodeName);
     List<String> replicationZnodes = ZKUtil.listChildrenNoWatch(zkw, replicationPath);
@@ -185,7 +187,7 @@ public class ZKDataMigrator extends Conf
   }
 
   private void checkAndMigrateQueuesToPB(ZooKeeperWatcher zkw, String znode, String rs)
-      throws KeeperException, NoNodeException {
+      throws KeeperException, NoNodeException, InterruptedException {
     String rsPath = ZKUtil.joinZNode(znode, rs);
     List<String> peers = ZKUtil.listChildrenNoWatch(zkw, rsPath);
     if (peers == null || peers.isEmpty()) return;
@@ -207,7 +209,7 @@ public class ZKDataMigrator extends Conf
   }
 
   private void checkAndMigratePeerZnodesToPB(ZooKeeperWatcher zkw, String znode,
-      List<String> peers) throws KeeperException, NoNodeException {
+      List<String> peers) throws KeeperException, NoNodeException, InterruptedException {
     for (String peer : peers) {
       String peerZnode = ZKUtil.joinZNode(znode, peer);
       byte[] data = ZKUtil.getData(zkw, peerZnode);

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionServerTracker.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionServerTracker.java?rev=1565787&r1=1565786&r2=1565787&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionServerTracker.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionServerTracker.java Fri Feb  7 20:27:42 2014
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hbase.zookeeper;
 
 import java.io.IOException;
+import java.io.InterruptedIOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.NavigableMap;
@@ -98,6 +99,8 @@ public class RegionServerTracker extends
             LOG.warn("Get Rs info port from ephemeral node", e);
           } catch (IOException e) {
             LOG.warn("Illegal data from ephemeral node", e);
+          } catch (InterruptedException e) {
+            throw new InterruptedIOException();
           }
           this.regionServers.put(sn, rsInfoBuilder.build());
         }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/lock/ZKInterProcessLockBase.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/lock/ZKInterProcessLockBase.java?rev=1565787&r1=1565786&r2=1565787&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/lock/ZKInterProcessLockBase.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/lock/ZKInterProcessLockBase.java Fri Feb  7 20:27:42 2014
@@ -362,6 +362,10 @@ public abstract class ZKInterProcessLock
     } catch (KeeperException ex) {
       LOG.warn("Error processing lock metadata in " + lockZNode);
       return false;
+    } catch (InterruptedException e) {
+      LOG.warn("InterruptedException processing lock metadata in " + lockZNode);
+      Thread.currentThread().interrupt();
+      return false;
     }
     return true;
   }

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java?rev=1565787&r1=1565786&r2=1565787&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java Fri Feb  7 20:27:42 2014
@@ -476,7 +476,7 @@ public class TestRegionPlacement {
    * Verify the number of region movement is expected
    */
   private void verifyRegionMovementNum(int expected)
-  throws InterruptedException, HBaseIOException {
+      throws InterruptedException, IOException {
     MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
     HMaster m = cluster.getMaster();
     int lastRegionOpenedCount = m.assignmentManager.getNumRegionsOpened();

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java?rev=1565787&r1=1565786&r2=1565787&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java Fri Feb  7 20:27:42 2014
@@ -27,6 +27,7 @@ import static org.junit.Assert.assertTru
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
+import java.io.InterruptedIOException;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.CountDownLatch;
@@ -1255,7 +1256,7 @@ public class TestSplitTransactionOnClust
     }
   }
 
-  private void waitUntilRegionServerDead() throws InterruptedException {
+  private void waitUntilRegionServerDead() throws InterruptedException, InterruptedIOException {
     // Wait until the master processes the RS shutdown
     for (int i=0; cluster.getMaster().getClusterStatus().
         getServers().size() == NB_SERVERS && i<100; i++) {

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKTable.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKTable.java?rev=1565787&r1=1565786&r2=1565787&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKTable.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKTable.java Fri Feb  7 20:27:42 2014
@@ -50,7 +50,7 @@ public class TestZKTable {
 
   @Test
   public void testTableStates()
-  throws ZooKeeperConnectionException, IOException, KeeperException {
+      throws ZooKeeperConnectionException, IOException, KeeperException, InterruptedException {
     final TableName name =
         TableName.valueOf("testDisabled");
     Abortable abortable = new Abortable() {