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/30 21:54:03 UTC

svn commit: r1344433 [1/3] - in /hbase/trunk/hbase-server/src/main: java/org/apache/hadoop/hbase/ java/org/apache/hadoop/hbase/client/ java/org/apache/hadoop/hbase/ipc/ java/org/apache/hadoop/hbase/master/ java/org/apache/hadoop/hbase/protobuf/ java/or...

Author: stack
Date: Wed May 30 19:54:02 2012
New Revision: 1344433

URL: http://svn.apache.org/viewvc?rev=1344433&view=rev
Log:
HBASE-5936 Add Column-level PB-based calls to HMasterInterface

Modified:
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/HMasterInterface.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/protobuf/ProtobufUtil.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java
    hbase/trunk/hbase-server/src/main/protobuf/Master.proto

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java?rev=1344433&r1=1344432&r2=1344433&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java Wed May 30 19:54:02 2012
@@ -1130,7 +1130,7 @@ public class HColumnDescriptor implement
    * @param cfs
    * @return An {@link HColumnDescriptor} made from the passed in <code>cfs</code>
    */
-  static HColumnDescriptor convert(final ColumnFamilySchema cfs) {
+  public static HColumnDescriptor convert(final ColumnFamilySchema cfs) {
     // Use the empty constructor so we preserve the initial values set on construction for things
     // like maxVersion.  Otherwise, we pick up wrong values on deserialization which makes for
     // unrelated-looking test failures that are hard to trace back to here.
@@ -1145,7 +1145,7 @@ public class HColumnDescriptor implement
   /**
    * @return Convert this instance to a the pb column family type
    */
-  ColumnFamilySchema convert() {
+  public ColumnFamilySchema convert() {
     ColumnFamilySchema.Builder builder = ColumnFamilySchema.newBuilder();
     builder.setName(ByteString.copyFrom(getName()));
     for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> e: this.values.entrySet()) {

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java?rev=1344433&r1=1344432&r2=1344433&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java Wed May 30 19:54:02 2012
@@ -1251,7 +1251,7 @@ public class HTableDescriptor implements
   /**
    * @return Convert the current {@link HTableDescriptor} into a pb TableSchema instance.
    */
-  TableSchema convert() {
+  public TableSchema convert() {
     TableSchema.Builder builder = TableSchema.newBuilder();
     builder.setName(ByteString.copyFrom(getName()));
     for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> e: this.values.entrySet()) {
@@ -1270,7 +1270,7 @@ public class HTableDescriptor implements
    * @param ts A pb TableSchema instance.
    * @return An {@link HTableDescriptor} made from the passed in pb <code>ts</code>.
    */
-  static HTableDescriptor convert(final TableSchema ts) {
+  public static HTableDescriptor convert(final TableSchema ts) {
     List<ColumnFamilySchema> list = ts.getColumnFamiliesList();
     HColumnDescriptor [] hcds = new HColumnDescriptor[list.size()];
     int index = 0;

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java?rev=1344433&r1=1344432&r2=1344433&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java Wed May 30 19:54:02 2012
@@ -74,6 +74,19 @@ import org.apache.hadoop.hbase.protobuf.
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteColumnRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteTableRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DisableTableRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableTableRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyColumnRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyTableRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AssignRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetBalancerRunningRequest;
@@ -493,8 +506,9 @@ public class HBaseAdmin implements Abort
 
     execute(new MasterCallable<Void>() {
       @Override
-      public Void call() throws IOException {
-        master.createTable(desc, splitKeys);
+      public Void call() throws ServiceException {
+        CreateTableRequest request = RequestConverter.buildCreateTableRequest(desc, splitKeys);
+        master.createTable(null, request);
         return null;
       }
     });
@@ -525,8 +539,9 @@ public class HBaseAdmin implements Abort
 
     execute(new MasterCallable<Void>() {
       @Override
-      public Void call() throws IOException {
-        master.deleteTable(tableName);
+      public Void call() throws ServiceException {
+        DeleteTableRequest req = RequestConverter.buildDeleteTableRequest(tableName);
+        master.deleteTable(null,req);
         return null;
       }
     });
@@ -554,19 +569,21 @@ public class HBaseAdmin implements Abort
         // HMaster removes the table from its HTableDescriptors
         if (values == null || values.length == 0) {
           tableExists = false;
-          HTableDescriptor[] htds;
+          GetTableDescriptorsResponse htds;
           MasterKeepAliveConnection master = connection.getKeepAliveMaster();
           try {
-            htds = master.getHTableDescriptors();
+            GetTableDescriptorsRequest req =
+              RequestConverter.buildGetTableDescriptorsRequest(null);
+            htds = master.getTableDescriptors(null, req);
+          } catch (ServiceException se) {
+            throw ProtobufUtil.getRemoteException(se);
           } finally {
             master.close();
           }
-          if (htds != null && htds.length > 0) {
-            for (HTableDescriptor htd: htds) {
-              if (Bytes.equals(tableName, htd.getName())) {
-                tableExists = true;
-                break;
-              }
+          for (TableSchema ts : htds.getTableSchemaList()) {
+            if (Bytes.equals(tableName, ts.getName().toByteArray())) {
+              tableExists = true;
+              break;
             }
           }
           if (!tableExists) {
@@ -709,9 +726,10 @@ public class HBaseAdmin implements Abort
   throws IOException {
     execute(new MasterCallable<Void>() {
       @Override
-      public Void call() throws IOException {
+      public Void call() throws ServiceException {
         LOG.info("Started enable of " + Bytes.toString(tableName));
-        master.enableTable(tableName);
+        EnableTableRequest req = RequestConverter.buildEnableTableRequest(tableName);
+        master.enableTable(null,req);
         return null;
       }
     });
@@ -778,9 +796,10 @@ public class HBaseAdmin implements Abort
   public void disableTableAsync(final byte [] tableName) throws IOException {
     execute(new MasterCallable<Void>() {
       @Override
-      public Void call() throws IOException {
+      public Void call() throws ServiceException {
         LOG.info("Started disable of " + Bytes.toString(tableName));
-        master.disableTable(tableName);
+        DisableTableRequest req = RequestConverter.buildDisableTableRequest(tableName);
+        master.disableTable(null,req);
         return null;
       }
     });
@@ -948,8 +967,14 @@ public class HBaseAdmin implements Abort
     HTableDescriptor.isLegalTableName(tableName);
     return execute(new MasterCallable<Pair<Integer, Integer>>() {
       @Override
-      public Pair<Integer, Integer> call() throws IOException {
-        return master.getAlterStatus(tableName);
+      public Pair<Integer, Integer> call() throws ServiceException {
+        GetSchemaAlterStatusRequest req =
+          RequestConverter.buildGetSchemaAlterStatusRequest(tableName);
+        GetSchemaAlterStatusResponse ret = master.getSchemaAlterStatus(null,req);
+        Pair<Integer,Integer> pair =
+          new Pair<Integer,Integer>(
+            new Integer(ret.getYetToUpdateRegions()),new Integer(ret.getTotalRegions()));
+        return pair;
       }
     });
   }
@@ -979,8 +1004,9 @@ public class HBaseAdmin implements Abort
   throws IOException {
     execute(new MasterCallable<Void>() {
       @Override
-      public Void call() throws IOException {
-        master.addColumn(tableName, column);
+      public Void call() throws ServiceException {
+        AddColumnRequest req = RequestConverter.buildAddColumnRequest(tableName, column);
+        master.addColumn(null,req);
         return null;
       }
     });
@@ -1011,8 +1037,9 @@ public class HBaseAdmin implements Abort
   throws IOException {
     execute(new MasterCallable<Void>() {
       @Override
-      public Void call() throws IOException {
-        master.deleteColumn(tableName, columnName);
+      public Void call() throws ServiceException {
+        DeleteColumnRequest req = RequestConverter.buildDeleteColumnRequest(tableName, columnName);
+        master.deleteColumn(null,req);
         return null;
       }
     });
@@ -1045,8 +1072,9 @@ public class HBaseAdmin implements Abort
   throws IOException {
     execute(new MasterCallable<Void>() {
       @Override
-      public Void call() throws IOException {
-        master.modifyColumn(tableName, descriptor);
+      public Void call() throws ServiceException {
+        ModifyColumnRequest req = RequestConverter.buildModifyColumnRequest(tableName, descriptor);
+        master.modifyColumn(null,req);
         return null;
       }
     });
@@ -1565,8 +1593,9 @@ public class HBaseAdmin implements Abort
   throws IOException {
     execute(new MasterCallable<Void>() {
       @Override
-      public Void call() throws IOException {
-        master.modifyTable(tableName, htd);
+      public Void call() throws ServiceException {
+        ModifyTableRequest request = RequestConverter.buildModifyTableRequest(tableName, htd);
+        master.modifyTable(null, request);
         return null;
       }
     });

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java?rev=1344433&r1=1344432&r2=1344433&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java Wed May 30 19:54:02 2012
@@ -79,6 +79,9 @@ import org.apache.hadoop.hbase.ipc.HMast
 import org.apache.hadoop.hbase.ipc.VersionedProtocol;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningRequest;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Addressing;
@@ -2245,7 +2248,11 @@ public class HConnectionManager {
     public HTableDescriptor[] listTables() throws IOException {
       MasterKeepAliveConnection master = getKeepAliveMaster();
       try {
-        return master.getHTableDescriptors();
+        GetTableDescriptorsRequest req =
+          RequestConverter.buildGetTableDescriptorsRequest(null);
+        return ProtobufUtil.getHTableDescriptorArray(master.getTableDescriptors(null, req));
+      } catch (ServiceException se) {
+        throw ProtobufUtil.getRemoteException(se);
       } finally {
         master.close();
       }
@@ -2256,8 +2263,12 @@ public class HConnectionManager {
       if (tableNames == null || tableNames.isEmpty()) return new HTableDescriptor[0];
       MasterKeepAliveConnection master = getKeepAliveMaster();
       try {
-        return master.getHTableDescriptors(tableNames);
-      }finally {
+        GetTableDescriptorsRequest req =
+          RequestConverter.buildGetTableDescriptorsRequest(tableNames);
+        return ProtobufUtil.getHTableDescriptorArray(master.getTableDescriptors(null, req));
+      } catch (ServiceException se) {
+        throw ProtobufUtil.getRemoteException(se);
+      } finally {
         master.close();
       }
     }
@@ -2280,17 +2291,19 @@ public class HConnectionManager {
         return HTableDescriptor.META_TABLEDESC;
       }
       MasterKeepAliveConnection master = getKeepAliveMaster();
-      HTableDescriptor[] htds;
+      GetTableDescriptorsResponse htds;
       try {
-        htds = master.getHTableDescriptors();
-      }finally {
+        GetTableDescriptorsRequest req =
+          RequestConverter.buildGetTableDescriptorsRequest(null);
+        htds = master.getTableDescriptors(null, req);
+      } catch (ServiceException se) {
+        throw ProtobufUtil.getRemoteException(se);
+      } finally {
         master.close();
       }
-      if (htds != null && htds.length > 0) {
-        for (HTableDescriptor htd: htds) {
-          if (Bytes.equals(tableName, htd.getName())) {
-            return htd;
-          }
+      for (TableSchema ts : htds.getTableSchemaList()) {
+        if (Bytes.equals(tableName, ts.getName().toByteArray())) {
+          return HTableDescriptor.convert(ts);
         }
       }
       throw new TableNotFoundException(Bytes.toString(tableName));

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/HMasterInterface.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/HMasterInterface.java?rev=1344433&r1=1344432&r2=1344433&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/HMasterInterface.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/HMasterInterface.java Wed May 30 19:54:02 2012
@@ -27,8 +27,28 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.hbase.ClusterStatus;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteColumnRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteColumnResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AssignRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AssignRegionResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteTableRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteTableResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DisableTableRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DisableTableResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableTableRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableTableResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyColumnRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyColumnResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyTableRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyTableResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveRegionResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetBalancerRunningRequest;
@@ -94,85 +114,106 @@ public interface HMasterInterface extend
    * Creates a new table asynchronously.  If splitKeys are specified, then the
    * table will be created with an initial set of multiple regions.
    * If splitKeys is null, the table will be created with a single region.
-   * @param desc table descriptor
-   * @param splitKeys
-   * @throws IOException
+   * @param controller Unused (set to null).
+   * @param req CreateTableRequest that contains:<br>
+   * - tablesSchema: table descriptor<br>
+   * - splitKeys
+   * @throws ServiceException
    */
-  public void createTable(HTableDescriptor desc, byte [][] splitKeys)
-  throws IOException;
+  public CreateTableResponse createTable(RpcController controller, CreateTableRequest req)
+  throws ServiceException;
 
   /**
    * Deletes a table
-   * @param tableName table to delete
-   * @throws IOException e
+   * @param controller Unused (set to null).
+   * @param req DeleteTableRequest that contains:<br>
+   * - tableName: table to delete
+   * @throws ServiceException
    */
-  public void deleteTable(final byte [] tableName) throws IOException;
+  public DeleteTableResponse deleteTable(RpcController controller, DeleteTableRequest req)
+  throws ServiceException;
 
   /**
    * Used by the client to get the number of regions that have received the
    * updated schema
    *
-   * @param tableName
-   * @return Pair indicating the number of regions updated Pair.getFirst() is the
-   *         regions that are yet to be updated Pair.getSecond() is the total number
-   *         of regions of the table
-   * @throws IOException
+   * @param controller Unused (set to null).
+   * @param req GetSchemaAlterStatusRequest that contains:<br>
+   * - tableName
+   * @return GetSchemaAlterStatusResponse indicating the number of regions updated.
+   *         yetToUpdateRegions is the regions that are yet to be updated totalRegions
+   *         is the total number of regions of the table
+   * @throws ServiceException
    */
-  public Pair<Integer, Integer> getAlterStatus(byte[] tableName)
-  throws IOException;
+  public GetSchemaAlterStatusResponse getSchemaAlterStatus(
+    RpcController controller, GetSchemaAlterStatusRequest req) throws ServiceException;
 
   /**
    * Adds a column to the specified table
-   * @param tableName table to modify
-   * @param column column descriptor
-   * @throws IOException e
+   * @param controller Unused (set to null).
+   * @param req AddColumnRequest that contains:<br>
+   * - tableName: table to modify<br>
+   * - column: column descriptor
+   * @throws ServiceException
    */
-  public void addColumn(final byte [] tableName, HColumnDescriptor column)
-  throws IOException;
+  public AddColumnResponse addColumn(RpcController controller, AddColumnRequest req)
+  throws ServiceException;
 
   /**
    * Modifies an existing column on the specified table
-   * @param tableName table name
-   * @param descriptor new column descriptor
+   * @param controller Unused (set to null).
+   * @param req ModifyColumnRequest that contains:<br>
+   * - tableName: table name<br>
+   * - descriptor: new column descriptor
    * @throws IOException e
    */
-  public void modifyColumn(final byte [] tableName, HColumnDescriptor descriptor)
-  throws IOException;
+  public ModifyColumnResponse modifyColumn(RpcController controller, ModifyColumnRequest req)
+  throws ServiceException;
 
 
   /**
    * Deletes a column from the specified table. Table must be disabled.
-   * @param tableName table to alter
-   * @param columnName column family to remove
-   * @throws IOException e
+   * @param controller Unused (set to null).
+   * @param req DeleteColumnRequest that contains:<br>
+   * - tableName: table to alter<br>
+   * - columnName: column family to remove
+   * @throws ServiceException
    */
-  public void deleteColumn(final byte [] tableName, final byte [] columnName)
-  throws IOException;
+  public DeleteColumnResponse deleteColumn(RpcController controller, DeleteColumnRequest req)
+  throws ServiceException;
 
   /**
    * Puts the table on-line (only needed if table has been previously taken offline)
-   * @param tableName table to enable
-   * @throws IOException e
+   * @param controller Unused (set to null).
+   * @param req EnableTableRequest that contains:<br>
+   * - tableName: table to enable
+   * @throws ServiceException
    */
-  public void enableTable(final byte [] tableName) throws IOException;
+  public EnableTableResponse enableTable(RpcController controller, EnableTableRequest req)
+  throws ServiceException;
 
   /**
    * Take table offline
    *
-   * @param tableName table to take offline
-   * @throws IOException e
+   * @param controller Unused (set to null).
+   * @param req DisableTableRequest that contains:<br>
+   * - tableName: table to take offline
+   * @throws ServiceException
    */
-  public void disableTable(final byte [] tableName) throws IOException;
+  public DisableTableResponse disableTable(RpcController controller, DisableTableRequest req)
+  throws ServiceException;
 
   /**
    * Modify a table's metadata
    *
-   * @param tableName table to modify
-   * @param htd new descriptor for table
-   * @throws IOException e
+   * @param controller Unused (set to null).
+   * @param req ModifyTableRequest that contains:<br>
+   * - tableName: table to modify<br>
+   * - tableSchema: new descriptor for table
+   * @throws ServiceException
    */
-  public void modifyTable(byte[] tableName, HTableDescriptor htd)
-  throws IOException;
+  public ModifyTableResponse modifyTable(RpcController controller, ModifyTableRequest req)
+  throws ServiceException;
 
   /**
    * Shutdown an HBase cluster.
@@ -239,17 +280,15 @@ public interface HMasterInterface extend
   throws ServiceException;
 
   /**
-   * Get array of all HTDs.
-   * @return array of HTableDescriptor
-   */
-  public HTableDescriptor[] getHTableDescriptors();
-
-  /**
-   * Get array of HTDs for requested tables.
-   * @param tableNames
-   * @return array of HTableDescriptor
+   * Get list of TableDescriptors for requested tables.
+   * @param controller Unused (set to null).
+   * @param req GetTableDescriptorsRequest that contains:<br>
+   * - tableNames: requested tables, or if empty, all are requested
+   * @return GetTableDescriptorsResponse
+   * @throws ServiceException
    */
-  public HTableDescriptor[] getHTableDescriptors(List<String> tableNames);
+  public GetTableDescriptorsResponse getTableDescriptors(
+      RpcController controller, GetTableDescriptorsRequest req) throws ServiceException;
 
   /**
    * Assign a region to a server chosen at random.
@@ -267,7 +306,7 @@ public interface HMasterInterface extend
    * back to the same server.  Use {@link #moveRegion(RpcController,MoveRegionRequest}
    * if you want to control the region movement.
    * @param controller Unused (set to null).
-   * @param req The request which contains:<br>
+   * @param req The request that contains:<br>
    * - region: Region to unassign. Will clear any existing RegionPlan
    * if one found.<br>
    * - force: If true, force unassign (Will remove region from
@@ -281,7 +320,7 @@ public interface HMasterInterface extend
   /**
    * Move a region to a specified destination server.
    * @param controller Unused (set to null).
-   * @param req The request which contains:<br>
+   * @param req The request that contains:<br>
    * - region: The encoded region name; i.e. the hash that makes
    * up the region name suffix: e.g. if regionname is
    * <code>TestTable,0094429456,1289497600452.527db22f95c8a9e0116f0cc13c680396.</code>,
@@ -294,5 +333,5 @@ public interface HMasterInterface extend
    * region named <code>encodedRegionName</code>
    */
   public MoveRegionResponse moveRegion(RpcController controller, MoveRegionRequest req)
-		  throws ServiceException;
+  throws ServiceException;
 }

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=1344433&r1=1344432&r2=1344433&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 Wed May 30 19:54:02 2012
@@ -123,8 +123,28 @@ import org.apache.hadoop.hbase.protobuf.
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
 import com.google.protobuf.RpcController;
 
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AssignRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AssignRegionResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteColumnRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteColumnResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteTableRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteTableResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DisableTableRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DisableTableResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableTableRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableTableResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyColumnRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyColumnResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyTableRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyTableResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveRegionResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.UnassignRegionRequest;
@@ -1005,7 +1025,7 @@ Server {
       resp.addMapEntries(entry.build());
 
       return resp.build();
-    } catch(IOException ioe) {
+    } catch (IOException ioe) {
       throw new ServiceException(ioe);
     }
   }
@@ -1050,7 +1070,7 @@ Server {
         // Up our metrics.
         this.metrics.incrementRequests(sl.getTotalNumberOfRequests());
       }
-    } catch(IOException ioe) {
+    } catch (IOException ioe) {
       throw new ServiceException(ioe);
     }
 
@@ -1294,6 +1314,7 @@ Server {
     return mrr;
   }
 
+  @Override
   public void createTable(HTableDescriptor hTableDescriptor,
     byte [][] splitKeys)
   throws IOException {
@@ -1310,10 +1331,23 @@ Server {
     this.executorService.submit(new CreateTableHandler(this,
       this.fileSystemManager, this.serverManager, hTableDescriptor, conf,
       newRegions, catalogTracker, assignmentManager));
-
     if (cpHost != null) {
       cpHost.postCreateTable(hTableDescriptor, newRegions);
     }
+
+  }
+
+  @Override
+  public CreateTableResponse createTable(RpcController controller, CreateTableRequest req)
+  throws ServiceException {
+    HTableDescriptor hTableDescriptor = HTableDescriptor.convert(req.getTableSchema());
+    byte [][] splitKeys = ProtobufUtil.getSplitKeysArray(req);
+    try {
+      createTable(hTableDescriptor,splitKeys);
+    } catch (IOException ioe) {
+      throw new ServiceException(ioe);
+    }
+    return CreateTableResponse.newBuilder().build();
   }
 
   private HRegionInfo[] getHRegionInfos(HTableDescriptor hTableDescriptor,
@@ -1343,15 +1377,23 @@ Server {
   }
 
   @Override
-  public void deleteTable(final byte [] tableName) throws IOException {
-    checkInitialized();
-    if (cpHost != null) {
-      cpHost.preDeleteTable(tableName);
-    }
-    this.executorService.submit(new DeleteTableHandler(tableName, this, this));
-    if (cpHost != null) {
-      cpHost.postDeleteTable(tableName);
+  public DeleteTableResponse deleteTable(RpcController controller, DeleteTableRequest request)
+  throws ServiceException {
+    byte [] tableName = request.getTableName().toByteArray();
+    try {
+      checkInitialized();
+      if (cpHost != null) {
+        cpHost.preDeleteTable(tableName);
+      }
+      this.executorService.submit(new DeleteTableHandler(tableName, this, this));
+
+      if (cpHost != null) {
+        cpHost.postDeleteTable(tableName);
+      }
+    } catch (IOException ioe) {
+      throw new ServiceException(ioe);
     }
+    return DeleteTableResponse.newBuilder().build();
   }
 
   /**
@@ -1362,81 +1404,132 @@ Server {
    *         of regions of the table
    * @throws IOException 
    */
-  public Pair<Integer, Integer> getAlterStatus(byte[] tableName)
-  throws IOException {
+  @Override
+  public GetSchemaAlterStatusResponse getSchemaAlterStatus(
+      RpcController controller, GetSchemaAlterStatusRequest req) throws ServiceException {
     // TODO: currently, we query using the table name on the client side. this
     // may overlap with other table operations or the table operation may
     // have completed before querying this API. We need to refactor to a
     // transaction system in the future to avoid these ambiguities.
-    return this.assignmentManager.getReopenStatus(tableName);
+    byte [] tableName = req.getTableName().toByteArray();
+
+    try {
+      Pair<Integer,Integer> pair = this.assignmentManager.getReopenStatus(tableName);
+      GetSchemaAlterStatusResponse.Builder ret = GetSchemaAlterStatusResponse.newBuilder();
+      ret.setYetToUpdateRegions(pair.getFirst());
+      ret.setTotalRegions(pair.getSecond());
+      return ret.build();
+    } catch (IOException ioe) {
+      throw new ServiceException(ioe);
+    }
   }
 
-  public void addColumn(byte [] tableName, HColumnDescriptor column)
-  throws IOException {
-    checkInitialized();
-    if (cpHost != null) {
-      if (cpHost.preAddColumn(tableName, column)) {
-        return;
+  public AddColumnResponse addColumn(RpcController controller, AddColumnRequest req)
+  throws ServiceException {
+    byte [] tableName = req.getTableName().toByteArray();
+    HColumnDescriptor column = HColumnDescriptor.convert(req.getColumnFamilies());
+
+    try {
+      checkInitialized();
+      if (cpHost != null) {
+        if (cpHost.preAddColumn(tableName, column)) {
+          return AddColumnResponse.newBuilder().build();
+        }
       }
+      new TableAddFamilyHandler(tableName, column, this, this).process();
+      if (cpHost != null) {
+        cpHost.postAddColumn(tableName, column);
+      }
+    } catch (IOException ioe) {
+      throw new ServiceException(ioe);
     }
-    new TableAddFamilyHandler(tableName, column, this, this).process();
-    if (cpHost != null) {
-      cpHost.postAddColumn(tableName, column);
-    }
+    return AddColumnResponse.newBuilder().build();
   }
 
-  public void modifyColumn(byte [] tableName, HColumnDescriptor descriptor)
-  throws IOException {
-    checkInitialized();
-    if (cpHost != null) {
-      if (cpHost.preModifyColumn(tableName, descriptor)) {
-        return;
+  public ModifyColumnResponse modifyColumn(RpcController controller, ModifyColumnRequest req)
+  throws ServiceException {
+    byte [] tableName = req.getTableName().toByteArray();
+    HColumnDescriptor descriptor = HColumnDescriptor.convert(req.getColumnFamilies());
+
+    try {
+      checkInitialized();
+      if (cpHost != null) {
+        if (cpHost.preModifyColumn(tableName, descriptor)) {
+          return ModifyColumnResponse.newBuilder().build();
+        }
       }
+      new TableModifyFamilyHandler(tableName, descriptor, this, this).process();
+      if (cpHost != null) {
+        cpHost.postModifyColumn(tableName, descriptor);
+      }
+    } catch (IOException ioe) {
+      throw new ServiceException(ioe);
     }
-    new TableModifyFamilyHandler(tableName, descriptor, this, this).process();
-    if (cpHost != null) {
-      cpHost.postModifyColumn(tableName, descriptor);
-    }
+    return ModifyColumnResponse.newBuilder().build();
   }
 
-  public void deleteColumn(final byte [] tableName, final byte [] c)
-  throws IOException {
-    checkInitialized();
-    if (cpHost != null) {
-      if (cpHost.preDeleteColumn(tableName, c)) {
-        return;
+  @Override
+  public DeleteColumnResponse deleteColumn(RpcController controller, DeleteColumnRequest req)
+  throws ServiceException {
+    final byte [] tableName = req.getTableName().toByteArray();
+    final byte [] columnName = req.getColumnName().toByteArray();
+    try {
+      checkInitialized();
+      if (cpHost != null) {
+        if (cpHost.preDeleteColumn(tableName, columnName)) {
+          return DeleteColumnResponse.newBuilder().build();
+        }
       }
+      new TableDeleteFamilyHandler(tableName, columnName, this, this).process();
+      if (cpHost != null) {
+        cpHost.postDeleteColumn(tableName, columnName);
+      }
+    } catch (IOException ioe) {
+      throw new ServiceException(ioe);
     }
-    new TableDeleteFamilyHandler(tableName, c, this, this).process();
-    if (cpHost != null) {
-      cpHost.postDeleteColumn(tableName, c);
-    }
+    return DeleteColumnResponse.newBuilder().build();
   }
 
-  public void enableTable(final byte [] tableName) throws IOException {
-    checkInitialized();
-    if (cpHost != null) {
-      cpHost.preEnableTable(tableName);
-    }
-    this.executorService.submit(new EnableTableHandler(this, tableName,
-      catalogTracker, assignmentManager, false));
+  @Override
+  public EnableTableResponse enableTable(RpcController controller, EnableTableRequest request)
+  throws ServiceException {
+    byte [] tableName = request.getTableName().toByteArray();
+    try {
+      checkInitialized();
+      if (cpHost != null) {
+        cpHost.preEnableTable(tableName);
+      }
+      this.executorService.submit(new EnableTableHandler(this, tableName,
+        catalogTracker, assignmentManager, false));
 
-    if (cpHost != null) {
-      cpHost.postEnableTable(tableName);
+      if (cpHost != null) {
+        cpHost.postEnableTable(tableName);
+     }
+    } catch (IOException ioe) {
+      throw new ServiceException(ioe);
     }
+    return EnableTableResponse.newBuilder().build();
   }
 
-  public void disableTable(final byte [] tableName) throws IOException {
-    checkInitialized();
-    if (cpHost != null) {
-      cpHost.preDisableTable(tableName);
-    }
-    this.executorService.submit(new DisableTableHandler(this, tableName,
-      catalogTracker, assignmentManager, false));
+  @Override
+  public DisableTableResponse disableTable(RpcController controller, DisableTableRequest request)
+  throws ServiceException {
+    byte [] tableName = request.getTableName().toByteArray();
+    try {
+      checkInitialized();
+      if (cpHost != null) {
+        cpHost.preDisableTable(tableName);
+      }
+      this.executorService.submit(new DisableTableHandler(this, tableName,
+        catalogTracker, assignmentManager, false));
 
-    if (cpHost != null) {
-      cpHost.postDisableTable(tableName);
+      if (cpHost != null) {
+        cpHost.postDisableTable(tableName);
+      }
+    } catch (IOException ioe) {
+      throw new ServiceException(ioe);
     }
+    return DisableTableResponse.newBuilder().build();
   }
 
   /**
@@ -1475,19 +1568,26 @@ Server {
   }
 
   @Override
-  public void modifyTable(final byte[] tableName, HTableDescriptor htd)
-      throws IOException {
-    checkInitialized();
-    if (cpHost != null) {
-      cpHost.preModifyTable(tableName, htd);
-    }
-    TableEventHandler tblHandle = new ModifyTableHandler(tableName, htd, this, this);
-    this.executorService.submit(tblHandle);
-    tblHandle.waitForPersist();
+  public ModifyTableResponse modifyTable(RpcController controller, ModifyTableRequest req)
+  throws ServiceException {
+    final byte [] tableName = req.getTableName().toByteArray();
+    HTableDescriptor htd = HTableDescriptor.convert(req.getTableSchema());
+    try {
+      checkInitialized();
+      if (cpHost != null) {
+        cpHost.preModifyTable(tableName, htd);
+      }
+      TableEventHandler tblHandle = new ModifyTableHandler(tableName, htd, this, this);
+      this.executorService.submit(tblHandle);
+      tblHandle.waitForPersist();
 
-    if (cpHost != null) {
-      cpHost.postModifyTable(tableName, htd);
+      if (cpHost != null) {
+        cpHost.postModifyTable(tableName, htd);
+      }
+    } catch (IOException ioe) {
+        throw new ServiceException(ioe);
     }
+    return ModifyTableResponse.newBuilder().build();
   }
 
   @Override
@@ -1919,39 +2019,43 @@ Server {
   }
 
   /**
-   * Get HTD array for given tables 
-   * @param tableNames
-   * @return HTableDescriptor[]
-   */
-  public HTableDescriptor[] getHTableDescriptors(List<String> tableNames) {
-    List<HTableDescriptor> list =
-      new ArrayList<HTableDescriptor>(tableNames.size());
-    for (String s: tableNames) {
-      HTableDescriptor htd = null;
+   * Get list of TableDescriptors for requested tables.
+   * @param controller Unused (set to null).
+   * @param req GetTableDescriptorsRequest that contains:
+   * - tableNames: requested tables, or if empty, all are requested
+   * @return GetTableDescriptorsResponse
+   * @throws ServiceException
+   */
+  public GetTableDescriptorsResponse getTableDescriptors(
+	      RpcController controller, GetTableDescriptorsRequest req) throws ServiceException {
+    GetTableDescriptorsResponse.Builder builder = GetTableDescriptorsResponse.newBuilder();
+    if (req.getTableNamesCount() == 0) {
+      // request for all TableDescriptors
+      Map<String, HTableDescriptor> descriptors = null;
       try {
-        htd = this.tableDescriptors.get(s);
+        descriptors = this.tableDescriptors.getAll();
       } catch (IOException e) {
-        LOG.warn("Failed getting descriptor for " + s, e);
+          LOG.warn("Failed getting all descriptors", e);
+      }
+      if (descriptors != null) {
+        for (HTableDescriptor htd : descriptors.values()) {
+          builder.addTableSchema(htd.convert());
+        }
       }
-      if (htd == null) continue;
-      list.add(htd);
     }
-    return list.toArray(new HTableDescriptor [] {});
-  }
-
-  /**
-   * Get all table descriptors
-   * @return All descriptors or null if none.
-   */
-  public HTableDescriptor [] getHTableDescriptors() {
-    Map<String, HTableDescriptor> descriptors = null;
-    try {
-      descriptors = this.tableDescriptors.getAll();
-    } catch (IOException e) {
-      LOG.warn("Failed getting all descriptors", e);
+    else {
+      for (String s: req.getTableNamesList()) {
+        HTableDescriptor htd = null;
+        try {
+          htd = this.tableDescriptors.get(s);
+        } catch (IOException e) {
+          LOG.warn("Failed getting descriptor for " + s, e);
+        }
+        if (htd == null) continue;
+        builder.addTableSchema(htd.convert());
+      }
     }
-    return descriptors == null?
-      null: descriptors.values().toArray(new HTableDescriptor [] {});
+    return builder.build();
   }
 
   /**

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java?rev=1344433&r1=1344432&r2=1344433&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java Wed May 30 19:54:02 2012
@@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.DoNotRetr
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.Action;
@@ -104,6 +105,8 @@ import org.apache.hadoop.hbase.protobuf.
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionLoad;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsResponse;
 import org.apache.hadoop.hbase.regionserver.wal.HLog;
 import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
@@ -290,6 +293,36 @@ public final class ProtobufUtil {
   }
 
   /**
+   * Get HTableDescriptor[] from GetTableDescriptorsResponse protobuf
+   *
+   * @param proto the GetTableDescriptorsResponse
+   * @return HTableDescriptor[]
+   */
+  public static HTableDescriptor[] getHTableDescriptorArray(GetTableDescriptorsResponse proto) {
+    if (proto == null) return null;
+
+    HTableDescriptor[] ret = new HTableDescriptor[proto.getTableSchemaCount()];
+    for (int i = 0; i < proto.getTableSchemaCount(); ++i) {
+      ret[i] = HTableDescriptor.convert(proto.getTableSchema(i));
+    }
+    return ret;
+  }
+
+  /**
+   * get the split keys in form "byte [][]" from a CreateTableRequest proto
+   *
+   * @param proto the CreateTableRequest
+   * @return the split keys
+   */
+  public static byte [][] getSplitKeysArray(final CreateTableRequest proto) {
+    byte [][] splitKeys = new byte[proto.getSplitKeysCount()][];
+    for (int i = 0; i < proto.getSplitKeysCount(); ++i) {
+      splitKeys[i] = proto.getSplitKeys(i).toByteArray();
+    }
+    return splitKeys;
+  }
+
+  /**
    * Convert a protocol buffer Get to a client Get
    *
    * @param get the protocol buffer Get to convert

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java?rev=1344433&r1=1344432&r2=1344433&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java Wed May 30 19:54:02 2012
@@ -26,7 +26,9 @@ import java.util.UUID;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.DeserializationException;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.Action;
@@ -78,7 +80,17 @@ import org.apache.hadoop.hbase.protobuf.
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.UnlockRowRequest;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AssignRegionRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteColumnRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteTableRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DisableTableRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableTableRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyColumnRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyTableRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.UnassignRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalanceRequest;
@@ -857,6 +869,51 @@ public final class RequestConverter {
   }
 
   /**
+   * Create a protocol buffer AddColumnRequest
+   *
+   * @param tableName
+   * @param column
+   * @return an AddColumnRequest
+   */
+  public static AddColumnRequest buildAddColumnRequest(
+      final byte [] tableName, final HColumnDescriptor column) {
+    AddColumnRequest.Builder builder = AddColumnRequest.newBuilder();
+    builder.setTableName(ByteString.copyFrom(tableName));
+    builder.setColumnFamilies(column.convert());
+    return builder.build();
+  }
+
+  /**
+   * Create a protocol buffer DeleteColumnRequest
+   *
+   * @param tableName
+   * @param columnName
+   * @return a DeleteColumnRequest
+   */
+  public static DeleteColumnRequest buildDeleteColumnRequest(
+      final byte [] tableName, final byte [] columnName) {
+    DeleteColumnRequest.Builder builder = DeleteColumnRequest.newBuilder();
+    builder.setTableName(ByteString.copyFrom(tableName));
+    builder.setColumnName(ByteString.copyFrom(columnName));
+    return builder.build();
+  }
+
+  /**
+   * Create a protocol buffer ModifyColumnRequest
+   *
+   * @param tableName
+   * @param column
+   * @return an ModifyColumnRequest
+   */
+  public static ModifyColumnRequest buildModifyColumnRequest(
+      final byte [] tableName, final HColumnDescriptor column) {
+    ModifyColumnRequest.Builder builder = ModifyColumnRequest.newBuilder();
+    builder.setTableName(ByteString.copyFrom(tableName));
+    builder.setColumnFamilies(column.convert());
+    return builder.build();
+  }
+
+  /**
    * Create a protocol buffer MoveRegionRequest
    *
    * @param encodedRegionName
@@ -880,7 +937,7 @@ public final class RequestConverter {
    * Create a protocol buffer AssignRegionRequest
    *
    * @param regionName
-   * @return An AssignRegionRequest
+   * @return an AssignRegionRequest
    */
   public static AssignRegionRequest buildAssignRegionRequest(final byte [] regionName) {
     AssignRegionRequest.Builder builder = AssignRegionRequest.newBuilder();
@@ -893,7 +950,7 @@ public final class RequestConverter {
    *
    * @param regionName
    * @param force
-   * @return An UnassignRegionRequest
+   * @return an UnassignRegionRequest
    */
   public static UnassignRegionRequest buildUnassignRegionRequest(
       final byte [] regionName, final boolean force) {
@@ -904,6 +961,106 @@ public final class RequestConverter {
   }
 
   /**
+   * Creates a protocol buffer DeleteTableRequest
+   *
+   * @param tableName
+   * @return a DeleteTableRequest
+   */
+  public static DeleteTableRequest buildDeleteTableRequest(final byte [] tableName) {
+    DeleteTableRequest.Builder builder = DeleteTableRequest.newBuilder();
+    builder.setTableName(ByteString.copyFrom(tableName));
+    return builder.build();
+  }
+
+  /**
+   * Creates a protocol buffer EnableTableRequest
+   *
+   * @param tableName
+   * @return an EnableTableRequest
+   */
+  public static EnableTableRequest buildEnableTableRequest(final byte [] tableName) {
+    EnableTableRequest.Builder builder = EnableTableRequest.newBuilder();
+    builder.setTableName(ByteString.copyFrom(tableName));
+    return builder.build();
+  }
+
+  /**
+   * Creates a protocol buffer DisableTableRequest
+   *
+   * @param tableName
+   * @return a DisableTableRequest
+   */
+  public static DisableTableRequest buildDisableTableRequest(final byte [] tableName) {
+    DisableTableRequest.Builder builder = DisableTableRequest.newBuilder();
+    builder.setTableName(ByteString.copyFrom(tableName));
+    return builder.build();
+  }
+
+  /**
+   * Creates a protocol buffer CreateTableRequest
+   *
+   * @param hTableDesc
+   * @param splitKeys
+   * @return a CreateTableRequest
+   */
+  public static CreateTableRequest buildCreateTableRequest(
+      final HTableDescriptor hTableDesc, final byte [][] splitKeys) {
+    CreateTableRequest.Builder builder = CreateTableRequest.newBuilder();
+    builder.setTableSchema(hTableDesc.convert());
+    if (splitKeys != null) {
+      for (byte [] splitKey : splitKeys) {
+        builder.addSplitKeys(ByteString.copyFrom(splitKey));
+      }
+    }
+    return builder.build();
+  }
+
+
+  /**
+   * Creates a protocol buffer ModifyTableRequest
+   *
+   * @param table
+   * @param hTableDesc
+   * @return a ModifyTableRequest
+   */
+  public static ModifyTableRequest buildModifyTableRequest(
+      final byte [] table, final HTableDescriptor hTableDesc) {
+    ModifyTableRequest.Builder builder = ModifyTableRequest.newBuilder();
+    builder.setTableName(ByteString.copyFrom(table));
+    builder.setTableSchema(hTableDesc.convert());
+    return builder.build();
+  }
+
+  /**
+   * Creates a protocol buffer GetSchemaAlterStatusRequest
+   *
+   * @param tableName
+   * @return a GetSchemaAlterStatusRequest
+   */
+  public static GetSchemaAlterStatusRequest buildGetSchemaAlterStatusRequest(final byte [] table) {
+    GetSchemaAlterStatusRequest.Builder builder = GetSchemaAlterStatusRequest.newBuilder();
+    builder.setTableName(ByteString.copyFrom(table));
+    return builder.build();
+  }
+
+  /**
+   * Creates a protocol buffer GetTableDescriptorsRequest
+   *
+   * @param tableNames
+   * @return a GetTableDescriptorsRequest
+   */
+  public static GetTableDescriptorsRequest buildGetTableDescriptorsRequest(
+      final List<String> tableNames) {
+    GetTableDescriptorsRequest.Builder builder = GetTableDescriptorsRequest.newBuilder();
+    if (tableNames != null) {
+      for (String str : tableNames) {
+        builder.addTableNames(str);
+      }
+    }
+    return builder.build();
+  }
+
+  /**
    * Creates a protocol buffer IsMasterRunningRequest
    *
    * @return a IsMasterRunningRequest