You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by jx...@apache.org on 2013/04/26 04:19:44 UTC

svn commit: r1476028 - in /hbase/branches/0.94/src: main/java/org/apache/hadoop/hbase/rest/client/RemoteAdmin.java test/java/org/apache/hadoop/hbase/rest/client/TestRemoteAdmin.java

Author: jxiang
Date: Fri Apr 26 02:19:44 2013
New Revision: 1476028

URL: http://svn.apache.org/r1476028
Log:
HBASE-8345 Add all available resources in RootResource and VersionResource to rest RemoteAdmin (Aleksandr Shulman)

Modified:
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteAdmin.java
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteAdmin.java

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteAdmin.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteAdmin.java?rev=1476028&r1=1476027&r2=1476028&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteAdmin.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteAdmin.java Fri Apr 26 02:19:44 2013
@@ -20,14 +20,32 @@
 
 package org.apache.hadoop.hbase.rest.client;
 
+import java.io.ByteArrayInputStream;
 import java.io.IOException;
 
+import javax.xml.bind.JAXBContext;
+import javax.xml.bind.JAXBException;
+import javax.xml.bind.Unmarshaller;
+
 import org.apache.hadoop.conf.Configuration;
 
+import org.apache.hadoop.hbase.ClusterStatus;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.rest.Constants;
+import org.apache.hadoop.hbase.rest.ProtobufMessageHandler;
+import org.apache.hadoop.hbase.rest.VersionResource;
+import org.apache.hadoop.hbase.rest.model.StorageClusterStatusModel;
+import org.apache.hadoop.hbase.rest.model.StorageClusterVersionModel;
+import org.apache.hadoop.hbase.rest.model.TableListModel;
 import org.apache.hadoop.hbase.rest.model.TableSchemaModel;
+import org.apache.hadoop.hbase.rest.model.VersionModel;
+import org.apache.hadoop.hbase.rest.protobuf.generated.StorageClusterStatusMessage.StorageClusterStatus;
+import org.apache.hadoop.hbase.rest.protobuf.generated.TableListMessage;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.jboss.netty.handler.codec.protobuf.ProtobufDecoder;
+import org.mortbay.jetty.MimeTypes;
+import org.xml.sax.InputSource;
 
 public class RemoteAdmin {
 
@@ -37,8 +55,14 @@ public class RemoteAdmin {
   final int maxRetries;
   final long sleepTime;
 
+  // This unmarshaller is necessary for getting the /version/cluster resource.
+  // This resource does not support protobufs. Therefore this is necessary to
+  // request/interpret it as XML.
+  private static volatile Unmarshaller versionClusterUnmarshaller;
+
   /**
    * Constructor
+   * 
    * @param client
    * @param conf
    */
@@ -46,6 +70,16 @@ public class RemoteAdmin {
     this(client, conf, null);
   }
 
+  static Unmarshaller getUnmarsheller() throws JAXBException {
+
+    if (versionClusterUnmarshaller == null) {
+
+      RemoteAdmin.versionClusterUnmarshaller = JAXBContext.newInstance(
+          StorageClusterVersionModel.class).createUnmarshaller();
+    }
+    return RemoteAdmin.versionClusterUnmarshaller;
+  }
+
   /**
    * Constructor
    * @param client
@@ -70,23 +104,155 @@ public class RemoteAdmin {
   }
 
   /**
+   * @return string representing the rest api's version
+   * @throws IOEXception
+   *           if the endpoint does not exist, there is a timeout, or some other
+   *           general failure mode
+   */
+  public VersionModel getRestVersion() throws IOException {
+
+    StringBuilder path = new StringBuilder();
+    path.append('/');
+    if (accessToken != null) {
+      path.append(accessToken);
+      path.append('/');
+    }
+
+    path.append("version/rest");
+
+    int code = 0;
+    for (int i = 0; i < maxRetries; i++) {
+      Response response = client.get(path.toString(),
+          Constants.MIMETYPE_PROTOBUF);
+      code = response.getCode();
+      switch (code) {
+      case 200:
+
+        VersionModel v = new VersionModel();
+        return (VersionModel) v.getObjectFromMessage(response.getBody());
+      case 404:
+        throw new IOException("REST version not found");
+      case 509:
+        try {
+          Thread.sleep(sleepTime);
+        } catch (InterruptedException e) {
+        }
+        break;
+      default:
+        throw new IOException("get request to " + path.toString()
+            + " returned " + code);
+      }
+    }
+    throw new IOException("get request to " + path.toString() + " timed out");
+  }
+
+  /**
+   * @return string representing the cluster's version
+   * @throws IOEXception if the endpoint does not exist, there is a timeout, or some other general failure mode
+   */
+  public StorageClusterStatusModel getClusterStatus() throws IOException {
+
+      StringBuilder path = new StringBuilder();
+      path.append('/');
+      if (accessToken !=null) {
+          path.append(accessToken);
+          path.append('/');
+      }
+
+    path.append("status/cluster");
+
+    int code = 0;
+    for (int i = 0; i < maxRetries; i++) {
+      Response response = client.get(path.toString(),
+          Constants.MIMETYPE_PROTOBUF);
+      code = response.getCode();
+      switch (code) {
+      case 200:
+        StorageClusterStatusModel s = new StorageClusterStatusModel();
+        return (StorageClusterStatusModel) s.getObjectFromMessage(response
+            .getBody());
+      case 404:
+        throw new IOException("Cluster version not found");
+      case 509:
+        try {
+          Thread.sleep(sleepTime);
+        } catch (InterruptedException e) {
+        }
+        break;
+      default:
+        throw new IOException("get request to " + path + " returned " + code);
+      }
+    }
+    throw new IOException("get request to " + path + " timed out");
+  }
+
+  /**
+   * @return string representing the cluster's version
+   * @throws IOEXception
+   *           if the endpoint does not exist, there is a timeout, or some other
+   *           general failure mode
+   */
+  public StorageClusterVersionModel getClusterVersion() throws IOException {
+
+    StringBuilder path = new StringBuilder();
+    path.append('/');
+    if (accessToken != null) {
+      path.append(accessToken);
+      path.append('/');
+    }
+
+    path.append("version/cluster");
+
+    int code = 0;
+    for (int i = 0; i < maxRetries; i++) {
+      Response response = client.get(path.toString(), Constants.MIMETYPE_XML);
+      code = response.getCode();
+      switch (code) {
+      case 200:
+        try {
+
+          return (StorageClusterVersionModel) getUnmarsheller().unmarshal(
+              new ByteArrayInputStream(response.getBody()));
+        } catch (JAXBException jaxbe) {
+
+          throw new IOException(
+              "Issue parsing StorageClusterVersionModel object in XML form: "
+                  + jaxbe.getLocalizedMessage());
+        }
+      case 404:
+        throw new IOException("Cluster version not found");
+      case 509:
+        try {
+          Thread.sleep(sleepTime);
+        } catch (InterruptedException e) {
+        }
+        break;
+      default:
+        throw new IOException(path.toString() + " request returned " + code);
+      }
+    }
+    throw new IOException("get request to " + path.toString()
+        + " request timed out");
+  }
+
+  /**
    * @param tableName name of table to check
    * @return true if all regions of the table are available
    * @throws IOException if a remote or network exception occurs
    */
   public boolean isTableAvailable(byte[] tableName) throws IOException {
-    StringBuilder sb = new StringBuilder();
-    sb.append('/');
+    StringBuilder path = new StringBuilder();
+    path.append('/');
     if (accessToken != null) {
-      sb.append(accessToken);
-      sb.append('/');
+      path.append(accessToken);
+      path.append('/');
     }
-    sb.append(Bytes.toStringBinary(tableName));
-    sb.append('/');
-    sb.append("exists");
+    path.append(Bytes.toStringBinary(tableName));
+    path.append('/');
+    path.append("exists");
     int code = 0;
     for (int i = 0; i < maxRetries; i++) {
-      Response response = client.get(sb.toString());
+      Response response = client.get(path.toString(), Constants.MIMETYPE_PROTOBUF);
       code = response.getCode();
       switch (code) {
       case 200:
@@ -99,10 +265,10 @@ public class RemoteAdmin {
         } catch (InterruptedException e) { }
         break;
       default:
-        throw new IOException("exists request returned " + code);
+        throw new IOException("get request to " + path.toString() + " returned " + code);
       }
     }
-    throw new IOException("exists request timed out");
+    throw new IOException("get request to " + path.toString() + " timed out");
   }
 
   /**
@@ -113,18 +279,18 @@ public class RemoteAdmin {
   public void createTable(HTableDescriptor desc)
       throws IOException {
     TableSchemaModel model = new TableSchemaModel(desc);
-    StringBuilder sb = new StringBuilder();
-    sb.append('/');
+    StringBuilder path = new StringBuilder();
+    path.append('/');
     if (accessToken != null) {
-      sb.append(accessToken);
-      sb.append('/');
+      path.append(accessToken);
+      path.append('/');
     }
-    sb.append(Bytes.toStringBinary(desc.getName()));
-    sb.append('/');
-    sb.append("schema");
+    path.append(Bytes.toStringBinary(desc.getName()));
+    path.append('/');
+    path.append("schema");
     int code = 0;
     for (int i = 0; i < maxRetries; i++) {
-      Response response = client.put(sb.toString(), Constants.MIMETYPE_PROTOBUF,
+      Response response = client.put(path.toString(), Constants.MIMETYPE_PROTOBUF,
         model.createProtobufOutput());
       code = response.getCode();
       switch (code) {
@@ -136,10 +302,10 @@ public class RemoteAdmin {
         } catch (InterruptedException e) { }
         break;
       default:
-        throw new IOException("create request returned " + code);
+        throw new IOException("create request to " + path.toString() + " returned " + code);
       }
     }
-    throw new IOException("create request timed out");
+    throw new IOException("create request to " + path.toString() + " timed out");
   }
 
   /**
@@ -157,18 +323,18 @@ public class RemoteAdmin {
    * @throws IOException if a remote or network exception occurs
    */
   public void deleteTable(final byte [] tableName) throws IOException {
-    StringBuilder sb = new StringBuilder();
-    sb.append('/');
+    StringBuilder path = new StringBuilder();
+    path.append('/');
     if (accessToken != null) {
-      sb.append(accessToken);
-      sb.append('/');
+      path.append(accessToken);
+      path.append('/');
     }
-    sb.append(Bytes.toStringBinary(tableName));
-    sb.append('/');
-    sb.append("schema");
+    path.append(Bytes.toStringBinary(tableName));
+    path.append('/');
+    path.append("schema");
     int code = 0;
-    for (int i = 0; i < maxRetries; i++) { 
-      Response response = client.delete(sb.toString());
+    for (int i = 0; i < maxRetries; i++) {
+      Response response = client.delete(path.toString());
       code = response.getCode();
       switch (code) {
       case 200:
@@ -179,10 +345,52 @@ public class RemoteAdmin {
         } catch (InterruptedException e) { }
         break;
       default:
-        throw new IOException("delete request returned " + code);
+        throw new IOException("delete request to " + path.toString() + " returned " + code);
       }
     }
-    throw new IOException("delete request timed out");
+    throw new IOException("delete request to " + path.toString() + " timed out");
   }
 
+  /**
+   * @return string representing the cluster's version
+   * @throws IOEXception
+   *           if the endpoint does not exist, there is a timeout, or some other
+   *           general failure mode
+   */
+  public TableListModel getTableList() throws IOException {
+
+    StringBuilder path = new StringBuilder();
+    path.append('/');
+    if (accessToken != null) {
+      path.append(accessToken);
+      path.append('/');
+    }
+
+    int code = 0;
+    for (int i = 0; i < maxRetries; i++) {
+      // Response response = client.get(path.toString(),
+      // Constants.MIMETYPE_XML);
+      Response response = client.get(path.toString(),
+          Constants.MIMETYPE_PROTOBUF);
+      code = response.getCode();
+      switch (code) {
+      case 200:
+        TableListModel t = new TableListModel();
+        return (TableListModel) t.getObjectFromMessage(response.getBody());
+      case 404:
+        throw new IOException("Table list not found");
+      case 509:
+        try {
+          Thread.sleep(sleepTime);
+        } catch (InterruptedException e) {
+        }
+        break;
+      default:
+        throw new IOException("get request to " + path.toString()
+            + " request returned " + code);
+      }
+    }
+    throw new IOException("get request to " + path.toString()
+        + " request timed out");
+  }
 }

Modified: hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteAdmin.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteAdmin.java?rev=1476028&r1=1476027&r2=1476028&view=diff
==============================================================================
--- hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteAdmin.java (original)
+++ hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteAdmin.java Fri Apr 26 02:19:44 2013
@@ -20,13 +20,22 @@
 
 package org.apache.hadoop.hbase.rest.client;
 
-import org.apache.hadoop.hbase.*;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.List;
+
+import org.apache.hadoop.hbase.ClusterStatus;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.MediumTests;
 import org.apache.hadoop.hbase.rest.HBaseRESTTestingUtility;
-import org.apache.hadoop.hbase.rest.client.Client;
+import org.apache.hadoop.hbase.rest.model.StorageClusterStatusModel;
+import org.apache.hadoop.hbase.rest.model.TableModel;
+import org.apache.hadoop.hbase.rest.model.VersionModel;
 import org.apache.hadoop.hbase.util.Bytes;
-
-import static org.junit.Assert.*;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -34,13 +43,15 @@ import org.junit.experimental.categories
 
 @Category(MediumTests.class)
 public class TestRemoteAdmin {
-  private static final String TABLE_1 = "TestRemoteAdmin_Table_1";
-  private static final byte[] COLUMN_1 = Bytes.toBytes("a");
-  static final HTableDescriptor DESC_1 =  new HTableDescriptor(TABLE_1);
   private static final HBaseTestingUtility TEST_UTIL =
     new HBaseTestingUtility();
-  private static final HBaseRESTTestingUtility REST_TEST_UTIL = 
+  private static final HBaseRESTTestingUtility REST_TEST_UTIL =
     new HBaseRESTTestingUtility();
+  private static final String TABLE_1 = "TestRemoteAdmin_Table_1";
+  private static final String TABLE_2 = TABLE_1 + System.currentTimeMillis();
+  private static final byte[] COLUMN_1 = Bytes.toBytes("a");
+  static final HTableDescriptor DESC_1 =  new HTableDescriptor(TABLE_1);
+  static final HTableDescriptor DESC_2 =  new HTableDescriptor(TABLE_2);
   private static RemoteAdmin remoteAdmin;
 
   @BeforeClass
@@ -70,8 +81,66 @@ public class TestRemoteAdmin {
     assertFalse(remoteAdmin.isTableAvailable(TABLE_1));
   }
 
+  @Test
+  public void testGetRestVersion() throws Exception {
+
+    VersionModel RETURNED_REST_VERSION = remoteAdmin.getRestVersion();
+    System.out.print("Returned version is: " + RETURNED_REST_VERSION);
+
+    // Assert that it contains info about rest version, OS, JVM
+    assertTrue("Returned REST version did not contain info about rest.",
+        RETURNED_REST_VERSION.toString().contains("rest"));
+    assertTrue("Returned REST version did not contain info about the JVM.",
+        RETURNED_REST_VERSION.toString().contains("JVM"));
+    assertTrue("Returned REST version did not contain info about OS.",
+        RETURNED_REST_VERSION.toString().contains("OS"));
+  }
+
+  @Test
+  public void testClusterVersion() throws Exception {
+    // testing the /version/cluster endpoint
+    final String HBASE_VERSION = TEST_UTIL.getHBaseCluster().getClusterStatus()
+        .getHBaseVersion();
+    assertEquals("Cluster status from REST API did not match. ", HBASE_VERSION,
+        remoteAdmin.getClusterVersion().getVersion());
+  }
+
+  @Test
+  public void testClusterStatus() throws Exception {
+
+    ClusterStatus status = TEST_UTIL.getHBaseClusterInterface()
+        .getClusterStatus();
+    StorageClusterStatusModel returnedStatus = remoteAdmin.getClusterStatus();
+    assertEquals(
+        "Region count from cluster status and returned status did not match up. ",
+        status.getRegionsCount(), returnedStatus.getRegions());
+    assertEquals(
+        "Dead server count from cluster status and returned status did not match up. ",
+        status.getDeadServers(), returnedStatus.getDeadNodes().size());
+    assertEquals(
+        "Number of requests from cluster status and returned status did not match up. ",
+        status.getRequestsCount(), returnedStatus.getRequests());
+  }
+
+  @Test
+  public void testListTables() throws Exception {
+
+    remoteAdmin.createTable(DESC_2);
+    List<TableModel> tableList = remoteAdmin.getTableList().getTables();
+    System.out.println("List of tables is: ");
+    boolean found = false;
+    for (TableModel tm : tableList) {
+
+      if (tm.getName().equals(TABLE_2)) {
+        found = true;
+        break;
+      }
+    }
+    assertTrue("Table " + TABLE_2 + " was not found by get request to '/'",
+        found);
+  }
+
   @org.junit.Rule
   public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
     new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
 }
-