You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2012/05/24 02:33:48 UTC

svn commit: r1342108 [3/3] - in /hbase/trunk/src: main/java/org/apache/hadoop/hbase/client/ main/java/org/apache/hadoop/hbase/ipc/ main/java/org/apache/hadoop/hbase/master/ main/java/org/apache/hadoop/hbase/protobuf/ main/java/org/apache/hadoop/hbase/p...

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java?rev=1342108&r1=1342107&r2=1342108&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java Thu May 24 00:33:48 2012
@@ -90,6 +90,7 @@ import com.google.common.base.Preconditi
 import com.google.common.collect.Lists;
 import com.google.common.collect.Multimap;
 import com.google.common.collect.TreeMultimap;
+import com.google.protobuf.ServiceException;
 
 /**
  * HBaseFsck (hbck) is a tool for checking and repairing region consistency and
@@ -376,7 +377,7 @@ public class HBaseFsck {
    * Contacts the master and prints out cluster-wide information
    * @return 0 on success, non-zero on failure
    */
-  public int onlineHbck() throws IOException, KeeperException, InterruptedException {
+  public int onlineHbck() throws IOException, KeeperException, InterruptedException, ServiceException {
     // print hbase server version
     errors.print("Version: " + status.getHBaseVersion());
     offlineHdfsIntegrityRepair();

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java?rev=1342108&r1=1342107&r2=1342108&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java Thu May 24 00:33:48 2012
@@ -57,6 +57,8 @@ import org.apache.hadoop.hbase.zookeeper
 import org.junit.*;
 import org.junit.experimental.categories.Category;
 
+import com.google.protobuf.ServiceException;
+
 
 /**
  * Class to test HBaseAdmin.
@@ -1574,6 +1576,7 @@ public class TestAdmin {
       assertTrue(false);
     } catch (MasterNotRunningException ignored) {
     } catch (ZooKeeperConnectionException ignored) {
+    } catch (ServiceException ignored) {
     }
     long end = System.currentTimeMillis();
 

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestHMasterRPCException.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestHMasterRPCException.java?rev=1342108&r1=1342107&r2=1342108&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestHMasterRPCException.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestHMasterRPCException.java Thu May 24 00:33:48 2012
@@ -29,6 +29,7 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.hbase.*;
 import org.apache.hadoop.hbase.ipc.HBaseRPC;
 import org.apache.hadoop.hbase.ipc.HMasterInterface;
+import org.apache.hadoop.hbase.protobuf.RequestConverter;
 import org.apache.hadoop.ipc.RemoteException;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -50,7 +51,7 @@ public class TestHMasterRPCException {
     try {
       HMasterInterface inf = (HMasterInterface) HBaseRPC.getProxy(
         HMasterInterface.class,  HMasterInterface.VERSION, isa, conf, 100);
-      inf.isMasterRunning();
+      inf.isMasterRunning(null,RequestConverter.buildIsMasterRunningRequest());
       fail();
     } catch (RemoteException ex) {
       assertTrue(ex.getMessage().startsWith(

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java?rev=1342108&r1=1342107&r2=1342108&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java Thu May 24 00:33:48 2012
@@ -52,6 +52,8 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import com.google.protobuf.ServiceException;
+
 /**
  * Like {@link TestSplitTransaction} in that we're testing {@link SplitTransaction}
  * only the below tests are against a running cluster where {@link TestSplitTransaction}
@@ -104,7 +106,7 @@ public class TestSplitTransactionOnClust
    * @throws DeserializationException 
    */
   @Test (timeout = 300000) public void testRSSplitEphemeralsDisappearButDaughtersAreOnlinedAfterShutdownHandling()
-  throws IOException, InterruptedException, NodeExistsException, KeeperException, DeserializationException {
+  throws IOException, InterruptedException, NodeExistsException, KeeperException, DeserializationException, ServiceException {
     final byte [] tableName =
       Bytes.toBytes("ephemeral");
 
@@ -174,7 +176,7 @@ public class TestSplitTransactionOnClust
   }
 
   @Test (timeout = 300000) public void testExistingZnodeBlocksSplitAndWeRollback()
-  throws IOException, InterruptedException, NodeExistsException, KeeperException {
+  throws IOException, InterruptedException, NodeExistsException, KeeperException, ServiceException {
     final byte [] tableName =
       Bytes.toBytes("testExistingZnodeBlocksSplitAndWeRollback");
 
@@ -234,7 +236,7 @@ public class TestSplitTransactionOnClust
    * @throws InterruptedException
    */
   @Test (timeout = 300000) public void testShutdownSimpleFixup()
-  throws IOException, InterruptedException {
+  throws IOException, InterruptedException, ServiceException {
     final byte [] tableName = Bytes.toBytes("testShutdownSimpleFixup");
 
     // Create table then get the single region for our new table.
@@ -290,7 +292,7 @@ public class TestSplitTransactionOnClust
    * @throws InterruptedException
    */
   @Test (timeout=300000) public void testShutdownFixupWhenDaughterHasSplit()
-  throws IOException, InterruptedException {
+  throws IOException, InterruptedException, ServiceException {
     final byte [] tableName =
       Bytes.toBytes("testShutdownFixupWhenDaughterHasSplit");
 
@@ -371,7 +373,7 @@ public class TestSplitTransactionOnClust
   @Test(timeout = 300000)
   public void testMasterRestartWhenSplittingIsPartial()
       throws IOException, InterruptedException, NodeExistsException,
-      KeeperException, DeserializationException {
+      KeeperException, DeserializationException, ServiceException {
     final byte[] tableName = Bytes.toBytes("testMasterRestartWhenSplittingIsPartial");
 
     // Create table then get the single region for our new table.
@@ -451,7 +453,7 @@ public class TestSplitTransactionOnClust
   @Test (timeout = 300000)
   public void testMasterRestartAtRegionSplitPendingCatalogJanitor()
       throws IOException, InterruptedException, NodeExistsException,
-      KeeperException {
+      KeeperException, ServiceException {
     final byte[] tableName = Bytes.toBytes("testMasterRestartAtRegionSplitPendingCatalogJanitor");
 
     // Create table then get the single region for our new table.