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 2008/08/17 00:03:35 UTC

svn commit: r686567 [2/3] - in /hadoop/hbase/branches/0.2: ./ lib/ src/examples/thrift/ src/java/org/apache/hadoop/hbase/thrift/ src/java/org/apache/hadoop/hbase/thrift/generated/

Modified: hadoop/hbase/branches/0.2/src/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.2/src/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java?rev=686567&r1=686566&r2=686567&view=diff
==============================================================================
--- hadoop/hbase/branches/0.2/src/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java (original)
+++ hadoop/hbase/branches/0.2/src/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java Sat Aug 16 15:03:35 2008
@@ -23,9 +23,11 @@
  */
 package org.apache.hadoop.hbase.thrift.generated;
 
+import java.util.List;
 import java.util.ArrayList;
-import java.util.AbstractMap;
+import java.util.Map;
 import java.util.HashMap;
+import java.util.Set;
 import java.util.HashSet;
 import com.facebook.thrift.*;
 
@@ -37,24 +39,43 @@
   public interface Iface {
 
     /**
+     * Brings a table on-line (enables it)
+     * @param tableName name of the table
+     */
+    public void enableTable(byte[] tableName) throws IOError, TException;
+
+    /**
+     * Disables a table (takes it off-line) If it is being served, the master
+     * will tell the servers to stop serving it.
+     * @param tableName name of the table
+     */
+    public void disableTable(byte[] tableName) throws IOError, TException;
+
+    /**
+     * @param tableName name of table to check
+     * @return true if table is on-line
+     */
+    public boolean isTableEnabled(byte[] tableName) throws IOError, TException;
+
+    /**
      * List all the userspace tables.
      * @return - returns a list of names
      */
-    public ArrayList<byte[]> getTableNames() throws IOError, TException;
+    public List<byte[]> getTableNames() throws IOError, TException;
 
     /**
      * List all the column families assoicated with a table.
      * @param tableName table name
      * @return list of column family descriptors
      */
-    public AbstractMap<byte[],ColumnDescriptor> getColumnDescriptors(byte[] tableName) throws IOError, TException;
+    public Map<byte[],ColumnDescriptor> getColumnDescriptors(byte[] tableName) throws IOError, TException;
 
     /**
      * List the regions associated with a table.
      * @param tableName table name
      * @return list of region descriptors
      */
-    public ArrayList<RegionDescriptor> getTableRegions(byte[] tableName) throws IOError, TException;
+    public List<TRegionInfo> getTableRegions(byte[] tableName) throws IOError, TException;
 
     /**
      * Create a table with the specified column families.  The name
@@ -68,7 +89,7 @@
      * @throws IllegalArgument if an input parameter is invalid
      * @throws AlreadyExists if the table name already exists
      */
-    public void createTable(byte[] tableName, ArrayList<ColumnDescriptor> columnFamilies) throws IOError, IllegalArgument, AlreadyExists, TException;
+    public void createTable(byte[] tableName, List<ColumnDescriptor> columnFamilies) throws IOError, IllegalArgument, AlreadyExists, TException;
 
     /**
      * Deletes a table
@@ -78,7 +99,7 @@
     public void deleteTable(byte[] tableName) throws IOError, NotFound, TException;
 
     /**
-     * Get a single value for the specified table, row, and column at the
+     * Get a single TCell for the specified table, row, and column at the
      * latest timestamp.
      * 
      * @param tableName name of table
@@ -86,7 +107,7 @@
      * @param column column name
      * @return value for specified row/column
      */
-    public byte[] get(byte[] tableName, byte[] row, byte[] column) throws IOError, NotFound, TException;
+    public TCell get(byte[] tableName, byte[] row, byte[] column) throws IOError, NotFound, TException;
 
     /**
      * Get the specified number of versions for the specified table,
@@ -96,9 +117,9 @@
      * @param row row key
      * @param column column name
      * @param numVersions number of versions to retrieve
-     * @return list of values for specified row/column
+     * @return list of cells for specified row/column
      */
-    public ArrayList<byte[]> getVer(byte[] tableName, byte[] row, byte[] column, int numVersions) throws IOError, NotFound, TException;
+    public List<TCell> getVer(byte[] tableName, byte[] row, byte[] column, int numVersions) throws IOError, NotFound, TException;
 
     /**
      * Get the specified number of versions for the specified table,
@@ -110,9 +131,9 @@
      * @param column column name
      * @param timestamp timestamp
      * @param numVersions number of versions to retrieve
-     * @return list of values for specified row/column
+     * @return list of cells for specified row/column
      */
-    public ArrayList<byte[]> getVerTs(byte[] tableName, byte[] row, byte[] column, long timestamp, int numVersions) throws IOError, NotFound, TException;
+    public List<TCell> getVerTs(byte[] tableName, byte[] row, byte[] column, long timestamp, int numVersions) throws IOError, NotFound, TException;
 
     /**
      * Get all the data for the specified table and row at the latest
@@ -120,9 +141,9 @@
      * 
      * @param tableName name of table
      * @param row row key
-     * @return Map of columns to values.  Map is empty if row does not exist.
+     * @return TRowResult containing the row and map of columns to TCells. Map is empty if row does not exist.
      */
-    public AbstractMap<byte[],byte[]> getRow(byte[] tableName, byte[] row) throws IOError, TException;
+    public TRowResult getRow(byte[] tableName, byte[] row) throws IOError, TException;
 
     /**
      * Get all the data for the specified table and row at the specified
@@ -131,21 +152,9 @@
      * @param tableName of table
      * @param row row key
      * @param timestamp timestamp
-     * @return Map of columns to values.  Map is empty if row does not exist.
-     */
-    public AbstractMap<byte[],byte[]> getRowTs(byte[] tableName, byte[] row, long timestamp) throws IOError, TException;
-
-    /**
-     * Put a single value at the specified table, row, and column.
-     * To put muliple values in a single transaction, or to specify
-     * a non-default timestamp, use {@link #mutateRow} and/or
-     * {@link #mutateRowTs}
-     * 
-     * @param tableName name of table
-     * @param row row key
-     * @param column column name
+     * @return TRowResult containing the row and map of columns to TCells. Map is empty if row does not exist.
      */
-    public void put(byte[] tableName, byte[] row, byte[] column, byte[] value) throws IOError, IllegalArgument, TException;
+    public TRowResult getRowTs(byte[] tableName, byte[] row, long timestamp) throws IOError, TException;
 
     /**
      * Apply a series of mutations (updates/deletes) to a row in a
@@ -157,7 +166,7 @@
      * @param row row key
      * @param mutations list of mutation commands
      */
-    public void mutateRow(byte[] tableName, byte[] row, ArrayList<Mutation> mutations) throws IOError, IllegalArgument, TException;
+    public void mutateRow(byte[] tableName, byte[] row, List<Mutation> mutations) throws IOError, IllegalArgument, TException;
 
     /**
      * Apply a series of mutations (updates/deletes) to a row in a
@@ -170,7 +179,7 @@
      * @param mutations list of mutation commands
      * @param timestamp timestamp
      */
-    public void mutateRowTs(byte[] tableName, byte[] row, ArrayList<Mutation> mutations, long timestamp) throws IOError, IllegalArgument, TException;
+    public void mutateRowTs(byte[] tableName, byte[] row, List<Mutation> mutations, long timestamp) throws IOError, IllegalArgument, TException;
 
     /**
      * Apply a series of batches (each a series of mutations on a single row)
@@ -181,7 +190,7 @@
      * @param tableName name of table
      * @param rowBatches list of row batches
      */
-    public void mutateRows(byte[] tableName, ArrayList<BatchMutation> rowBatches) throws IOError, IllegalArgument, TException;
+    public void mutateRows(byte[] tableName, List<BatchMutation> rowBatches) throws IOError, IllegalArgument, TException;
 
     /**
      * Apply a series of batches (each a series of mutations on a single row)
@@ -193,7 +202,7 @@
      * @param rowBatches list of row batches
      * @param timestamp timestamp
      */
-    public void mutateRowsTs(byte[] tableName, ArrayList<BatchMutation> rowBatches, long timestamp) throws IOError, IllegalArgument, TException;
+    public void mutateRowsTs(byte[] tableName, List<BatchMutation> rowBatches, long timestamp) throws IOError, IllegalArgument, TException;
 
     /**
      * Delete all cells that match the passed row and column.
@@ -246,7 +255,7 @@
      * 
      * @return scanner id to be used with other scanner procedures
      */
-    public int scannerOpen(byte[] tableName, byte[] startRow, ArrayList<byte[]> columns) throws IOError, TException;
+    public int scannerOpen(byte[] tableName, byte[] startRow, List<byte[]> columns) throws IOError, TException;
 
     /**
      * Get a scanner on the current table starting and stopping at the
@@ -264,7 +273,7 @@
      * 
      * @return scanner id to be used with other scanner procedures
      */
-    public int scannerOpenWithStop(byte[] tableName, byte[] startRow, byte[] stopRow, ArrayList<byte[]> columns) throws IOError, TException;
+    public int scannerOpenWithStop(byte[] tableName, byte[] startRow, byte[] stopRow, List<byte[]> columns) throws IOError, TException;
 
     /**
      * Get a scanner on the current table starting at the specified row and
@@ -281,7 +290,7 @@
      * 
      * @return scanner id to be used with other scanner procedures
      */
-    public int scannerOpenTs(byte[] tableName, byte[] startRow, ArrayList<byte[]> columns, long timestamp) throws IOError, TException;
+    public int scannerOpenTs(byte[] tableName, byte[] startRow, List<byte[]> columns, long timestamp) throws IOError, TException;
 
     /**
      * Get a scanner on the current table starting and stopping at the
@@ -301,7 +310,7 @@
      * 
      * @return scanner id to be used with other scanner procedures
      */
-    public int scannerOpenWithStopTs(byte[] tableName, byte[] startRow, byte[] stopRow, ArrayList<byte[]> columns, long timestamp) throws IOError, TException;
+    public int scannerOpenWithStopTs(byte[] tableName, byte[] startRow, byte[] stopRow, List<byte[]> columns, long timestamp) throws IOError, TException;
 
     /**
      * Returns the scanner's current row value and advances to the next
@@ -310,11 +319,11 @@
      * a NotFound exception is returned.
      * 
      * @param id id of a scanner returned by scannerOpen
-     * @return a ScanEntry object representing the current row's values
+     * @return a TRowResult containing the current row and a map of the columns to TCells.
      * @throws IllegalArgument if ScannerID is invalid
      * @throws NotFound when the scanner reaches the end
      */
-    public ScanEntry scannerGet(int id) throws IOError, IllegalArgument, NotFound, TException;
+    public TRowResult scannerGet(int id) throws IOError, IllegalArgument, NotFound, TException;
 
     /**
      * Closes the server-state associated with an open scanner.
@@ -343,7 +352,119 @@
 
     protected int seqid_;
 
-    public ArrayList<byte[]> getTableNames() throws IOError, TException
+    public TProtocol getInputProtocol()
+    {
+      return this.iprot_;
+    }
+
+    public TProtocol getOutputProtocol()
+    {
+      return this.oprot_;
+    }
+
+    public void enableTable(byte[] tableName) throws IOError, TException
+    {
+      send_enableTable(tableName);
+      recv_enableTable();
+    }
+
+    public void send_enableTable(byte[] tableName) throws TException
+    {
+      oprot_.writeMessageBegin(new TMessage("enableTable", TMessageType.CALL, seqid_));
+      enableTable_args args = new enableTable_args();
+      args.tableName = tableName;
+      args.write(oprot_);
+      oprot_.writeMessageEnd();
+      oprot_.getTransport().flush();
+    }
+
+    public void recv_enableTable() throws IOError, TException
+    {
+      TMessage msg = iprot_.readMessageBegin();
+      if (msg.type == TMessageType.EXCEPTION) {
+        TApplicationException x = TApplicationException.read(iprot_);
+        iprot_.readMessageEnd();
+        throw x;
+      }
+      enableTable_result result = new enableTable_result();
+      result.read(iprot_);
+      iprot_.readMessageEnd();
+      if (result.__isset.io) {
+        throw result.io;
+      }
+      return;
+    }
+
+    public void disableTable(byte[] tableName) throws IOError, TException
+    {
+      send_disableTable(tableName);
+      recv_disableTable();
+    }
+
+    public void send_disableTable(byte[] tableName) throws TException
+    {
+      oprot_.writeMessageBegin(new TMessage("disableTable", TMessageType.CALL, seqid_));
+      disableTable_args args = new disableTable_args();
+      args.tableName = tableName;
+      args.write(oprot_);
+      oprot_.writeMessageEnd();
+      oprot_.getTransport().flush();
+    }
+
+    public void recv_disableTable() throws IOError, TException
+    {
+      TMessage msg = iprot_.readMessageBegin();
+      if (msg.type == TMessageType.EXCEPTION) {
+        TApplicationException x = TApplicationException.read(iprot_);
+        iprot_.readMessageEnd();
+        throw x;
+      }
+      disableTable_result result = new disableTable_result();
+      result.read(iprot_);
+      iprot_.readMessageEnd();
+      if (result.__isset.io) {
+        throw result.io;
+      }
+      return;
+    }
+
+    public boolean isTableEnabled(byte[] tableName) throws IOError, TException
+    {
+      send_isTableEnabled(tableName);
+      return recv_isTableEnabled();
+    }
+
+    public void send_isTableEnabled(byte[] tableName) throws TException
+    {
+      oprot_.writeMessageBegin(new TMessage("isTableEnabled", TMessageType.CALL, seqid_));
+      isTableEnabled_args args = new isTableEnabled_args();
+      args.tableName = tableName;
+      args.write(oprot_);
+      oprot_.writeMessageEnd();
+      oprot_.getTransport().flush();
+    }
+
+    public boolean recv_isTableEnabled() throws IOError, TException
+    {
+      TMessage msg = iprot_.readMessageBegin();
+      if (msg.type == TMessageType.EXCEPTION) {
+        TApplicationException x = TApplicationException.read(iprot_);
+        iprot_.readMessageEnd();
+        throw x;
+      }
+      isTableEnabled_result result = new isTableEnabled_result();
+      result.read(iprot_);
+      iprot_.readMessageEnd();
+      if (result.__isset.success) {
+        return result.success;
+      }
+      if (result.__isset.io) {
+        throw result.io;
+      }
+      throw new TApplicationException(TApplicationException.MISSING_RESULT, "isTableEnabled failed: unknown result");
+    }
+
+    public List<byte[]> getTableNames() throws IOError, TException
     {
       send_getTableNames();
       return recv_getTableNames();
@@ -358,7 +479,7 @@
       oprot_.getTransport().flush();
     }
 
-    public ArrayList<byte[]> recv_getTableNames() throws IOError, TException
+    public List<byte[]> recv_getTableNames() throws IOError, TException
     {
       TMessage msg = iprot_.readMessageBegin();
       if (msg.type == TMessageType.EXCEPTION) {
@@ -378,7 +499,7 @@
       throw new TApplicationException(TApplicationException.MISSING_RESULT, "getTableNames failed: unknown result");
     }
 
-    public AbstractMap<byte[],ColumnDescriptor> getColumnDescriptors(byte[] tableName) throws IOError, TException
+    public Map<byte[],ColumnDescriptor> getColumnDescriptors(byte[] tableName) throws IOError, TException
     {
       send_getColumnDescriptors(tableName);
       return recv_getColumnDescriptors();
@@ -394,7 +515,7 @@
       oprot_.getTransport().flush();
     }
 
-    public AbstractMap<byte[],ColumnDescriptor> recv_getColumnDescriptors() throws IOError, TException
+    public Map<byte[],ColumnDescriptor> recv_getColumnDescriptors() throws IOError, TException
     {
       TMessage msg = iprot_.readMessageBegin();
       if (msg.type == TMessageType.EXCEPTION) {
@@ -414,7 +535,7 @@
       throw new TApplicationException(TApplicationException.MISSING_RESULT, "getColumnDescriptors failed: unknown result");
     }
 
-    public ArrayList<RegionDescriptor> getTableRegions(byte[] tableName) throws IOError, TException
+    public List<TRegionInfo> getTableRegions(byte[] tableName) throws IOError, TException
     {
       send_getTableRegions(tableName);
       return recv_getTableRegions();
@@ -430,7 +551,7 @@
       oprot_.getTransport().flush();
     }
 
-    public ArrayList<RegionDescriptor> recv_getTableRegions() throws IOError, TException
+    public List<TRegionInfo> recv_getTableRegions() throws IOError, TException
     {
       TMessage msg = iprot_.readMessageBegin();
       if (msg.type == TMessageType.EXCEPTION) {
@@ -450,13 +571,13 @@
       throw new TApplicationException(TApplicationException.MISSING_RESULT, "getTableRegions failed: unknown result");
     }
 
-    public void createTable(byte[] tableName, ArrayList<ColumnDescriptor> columnFamilies) throws IOError, IllegalArgument, AlreadyExists, TException
+    public void createTable(byte[] tableName, List<ColumnDescriptor> columnFamilies) throws IOError, IllegalArgument, AlreadyExists, TException
     {
       send_createTable(tableName, columnFamilies);
       recv_createTable();
     }
 
-    public void send_createTable(byte[] tableName, ArrayList<ColumnDescriptor> columnFamilies) throws TException
+    public void send_createTable(byte[] tableName, List<ColumnDescriptor> columnFamilies) throws TException
     {
       oprot_.writeMessageBegin(new TMessage("createTable", TMessageType.CALL, seqid_));
       createTable_args args = new createTable_args();
@@ -526,7 +647,7 @@
       return;
     }
 
-    public byte[] get(byte[] tableName, byte[] row, byte[] column) throws IOError, NotFound, TException
+    public TCell get(byte[] tableName, byte[] row, byte[] column) throws IOError, NotFound, TException
     {
       send_get(tableName, row, column);
       return recv_get();
@@ -544,7 +665,7 @@
       oprot_.getTransport().flush();
     }
 
-    public byte[] recv_get() throws IOError, NotFound, TException
+    public TCell recv_get() throws IOError, NotFound, TException
     {
       TMessage msg = iprot_.readMessageBegin();
       if (msg.type == TMessageType.EXCEPTION) {
@@ -567,7 +688,7 @@
       throw new TApplicationException(TApplicationException.MISSING_RESULT, "get failed: unknown result");
     }
 
-    public ArrayList<byte[]> getVer(byte[] tableName, byte[] row, byte[] column, int numVersions) throws IOError, NotFound, TException
+    public List<TCell> getVer(byte[] tableName, byte[] row, byte[] column, int numVersions) throws IOError, NotFound, TException
     {
       send_getVer(tableName, row, column, numVersions);
       return recv_getVer();
@@ -586,7 +707,7 @@
       oprot_.getTransport().flush();
     }
 
-    public ArrayList<byte[]> recv_getVer() throws IOError, NotFound, TException
+    public List<TCell> recv_getVer() throws IOError, NotFound, TException
     {
       TMessage msg = iprot_.readMessageBegin();
       if (msg.type == TMessageType.EXCEPTION) {
@@ -609,7 +730,7 @@
       throw new TApplicationException(TApplicationException.MISSING_RESULT, "getVer failed: unknown result");
     }
 
-    public ArrayList<byte[]> getVerTs(byte[] tableName, byte[] row, byte[] column, long timestamp, int numVersions) throws IOError, NotFound, TException
+    public List<TCell> getVerTs(byte[] tableName, byte[] row, byte[] column, long timestamp, int numVersions) throws IOError, NotFound, TException
     {
       send_getVerTs(tableName, row, column, timestamp, numVersions);
       return recv_getVerTs();
@@ -629,7 +750,7 @@
       oprot_.getTransport().flush();
     }
 
-    public ArrayList<byte[]> recv_getVerTs() throws IOError, NotFound, TException
+    public List<TCell> recv_getVerTs() throws IOError, NotFound, TException
     {
       TMessage msg = iprot_.readMessageBegin();
       if (msg.type == TMessageType.EXCEPTION) {
@@ -652,7 +773,7 @@
       throw new TApplicationException(TApplicationException.MISSING_RESULT, "getVerTs failed: unknown result");
     }
 
-    public AbstractMap<byte[],byte[]> getRow(byte[] tableName, byte[] row) throws IOError, TException
+    public TRowResult getRow(byte[] tableName, byte[] row) throws IOError, TException
     {
       send_getRow(tableName, row);
       return recv_getRow();
@@ -669,7 +790,7 @@
       oprot_.getTransport().flush();
     }
 
-    public AbstractMap<byte[],byte[]> recv_getRow() throws IOError, TException
+    public TRowResult recv_getRow() throws IOError, TException
     {
       TMessage msg = iprot_.readMessageBegin();
       if (msg.type == TMessageType.EXCEPTION) {
@@ -689,7 +810,7 @@
       throw new TApplicationException(TApplicationException.MISSING_RESULT, "getRow failed: unknown result");
     }
 
-    public AbstractMap<byte[],byte[]> getRowTs(byte[] tableName, byte[] row, long timestamp) throws IOError, TException
+    public TRowResult getRowTs(byte[] tableName, byte[] row, long timestamp) throws IOError, TException
     {
       send_getRowTs(tableName, row, timestamp);
       return recv_getRowTs();
@@ -707,7 +828,7 @@
       oprot_.getTransport().flush();
     }
 
-    public AbstractMap<byte[],byte[]> recv_getRowTs() throws IOError, TException
+    public TRowResult recv_getRowTs() throws IOError, TException
     {
       TMessage msg = iprot_.readMessageBegin();
       if (msg.type == TMessageType.EXCEPTION) {
@@ -727,52 +848,13 @@
       throw new TApplicationException(TApplicationException.MISSING_RESULT, "getRowTs failed: unknown result");
     }
 
-    public void put(byte[] tableName, byte[] row, byte[] column, byte[] value) throws IOError, IllegalArgument, TException
-    {
-      send_put(tableName, row, column, value);
-      recv_put();
-    }
-
-    public void send_put(byte[] tableName, byte[] row, byte[] column, byte[] value) throws TException
-    {
-      oprot_.writeMessageBegin(new TMessage("put", TMessageType.CALL, seqid_));
-      put_args args = new put_args();
-      args.tableName = tableName;
-      args.row = row;
-      args.column = column;
-      args.value = value;
-      args.write(oprot_);
-      oprot_.writeMessageEnd();
-      oprot_.getTransport().flush();
-    }
-
-    public void recv_put() throws IOError, IllegalArgument, TException
-    {
-      TMessage msg = iprot_.readMessageBegin();
-      if (msg.type == TMessageType.EXCEPTION) {
-        TApplicationException x = TApplicationException.read(iprot_);
-        iprot_.readMessageEnd();
-        throw x;
-      }
-      put_result result = new put_result();
-      result.read(iprot_);
-      iprot_.readMessageEnd();
-      if (result.__isset.io) {
-        throw result.io;
-      }
-      if (result.__isset.ia) {
-        throw result.ia;
-      }
-      return;
-    }
-
-    public void mutateRow(byte[] tableName, byte[] row, ArrayList<Mutation> mutations) throws IOError, IllegalArgument, TException
+    public void mutateRow(byte[] tableName, byte[] row, List<Mutation> mutations) throws IOError, IllegalArgument, TException
     {
       send_mutateRow(tableName, row, mutations);
       recv_mutateRow();
     }
 
-    public void send_mutateRow(byte[] tableName, byte[] row, ArrayList<Mutation> mutations) throws TException
+    public void send_mutateRow(byte[] tableName, byte[] row, List<Mutation> mutations) throws TException
     {
       oprot_.writeMessageBegin(new TMessage("mutateRow", TMessageType.CALL, seqid_));
       mutateRow_args args = new mutateRow_args();
@@ -804,13 +886,13 @@
       return;
     }
 
-    public void mutateRowTs(byte[] tableName, byte[] row, ArrayList<Mutation> mutations, long timestamp) throws IOError, IllegalArgument, TException
+    public void mutateRowTs(byte[] tableName, byte[] row, List<Mutation> mutations, long timestamp) throws IOError, IllegalArgument, TException
     {
       send_mutateRowTs(tableName, row, mutations, timestamp);
       recv_mutateRowTs();
     }
 
-    public void send_mutateRowTs(byte[] tableName, byte[] row, ArrayList<Mutation> mutations, long timestamp) throws TException
+    public void send_mutateRowTs(byte[] tableName, byte[] row, List<Mutation> mutations, long timestamp) throws TException
     {
       oprot_.writeMessageBegin(new TMessage("mutateRowTs", TMessageType.CALL, seqid_));
       mutateRowTs_args args = new mutateRowTs_args();
@@ -843,13 +925,13 @@
       return;
     }
 
-    public void mutateRows(byte[] tableName, ArrayList<BatchMutation> rowBatches) throws IOError, IllegalArgument, TException
+    public void mutateRows(byte[] tableName, List<BatchMutation> rowBatches) throws IOError, IllegalArgument, TException
     {
       send_mutateRows(tableName, rowBatches);
       recv_mutateRows();
     }
 
-    public void send_mutateRows(byte[] tableName, ArrayList<BatchMutation> rowBatches) throws TException
+    public void send_mutateRows(byte[] tableName, List<BatchMutation> rowBatches) throws TException
     {
       oprot_.writeMessageBegin(new TMessage("mutateRows", TMessageType.CALL, seqid_));
       mutateRows_args args = new mutateRows_args();
@@ -880,13 +962,13 @@
       return;
     }
 
-    public void mutateRowsTs(byte[] tableName, ArrayList<BatchMutation> rowBatches, long timestamp) throws IOError, IllegalArgument, TException
+    public void mutateRowsTs(byte[] tableName, List<BatchMutation> rowBatches, long timestamp) throws IOError, IllegalArgument, TException
     {
       send_mutateRowsTs(tableName, rowBatches, timestamp);
       recv_mutateRowsTs();
     }
 
-    public void send_mutateRowsTs(byte[] tableName, ArrayList<BatchMutation> rowBatches, long timestamp) throws TException
+    public void send_mutateRowsTs(byte[] tableName, List<BatchMutation> rowBatches, long timestamp) throws TException
     {
       oprot_.writeMessageBegin(new TMessage("mutateRowsTs", TMessageType.CALL, seqid_));
       mutateRowsTs_args args = new mutateRowsTs_args();
@@ -1058,13 +1140,13 @@
       return;
     }
 
-    public int scannerOpen(byte[] tableName, byte[] startRow, ArrayList<byte[]> columns) throws IOError, TException
+    public int scannerOpen(byte[] tableName, byte[] startRow, List<byte[]> columns) throws IOError, TException
     {
       send_scannerOpen(tableName, startRow, columns);
       return recv_scannerOpen();
     }
 
-    public void send_scannerOpen(byte[] tableName, byte[] startRow, ArrayList<byte[]> columns) throws TException
+    public void send_scannerOpen(byte[] tableName, byte[] startRow, List<byte[]> columns) throws TException
     {
       oprot_.writeMessageBegin(new TMessage("scannerOpen", TMessageType.CALL, seqid_));
       scannerOpen_args args = new scannerOpen_args();
@@ -1096,13 +1178,13 @@
       throw new TApplicationException(TApplicationException.MISSING_RESULT, "scannerOpen failed: unknown result");
     }
 
-    public int scannerOpenWithStop(byte[] tableName, byte[] startRow, byte[] stopRow, ArrayList<byte[]> columns) throws IOError, TException
+    public int scannerOpenWithStop(byte[] tableName, byte[] startRow, byte[] stopRow, List<byte[]> columns) throws IOError, TException
     {
       send_scannerOpenWithStop(tableName, startRow, stopRow, columns);
       return recv_scannerOpenWithStop();
     }
 
-    public void send_scannerOpenWithStop(byte[] tableName, byte[] startRow, byte[] stopRow, ArrayList<byte[]> columns) throws TException
+    public void send_scannerOpenWithStop(byte[] tableName, byte[] startRow, byte[] stopRow, List<byte[]> columns) throws TException
     {
       oprot_.writeMessageBegin(new TMessage("scannerOpenWithStop", TMessageType.CALL, seqid_));
       scannerOpenWithStop_args args = new scannerOpenWithStop_args();
@@ -1135,13 +1217,13 @@
       throw new TApplicationException(TApplicationException.MISSING_RESULT, "scannerOpenWithStop failed: unknown result");
     }
 
-    public int scannerOpenTs(byte[] tableName, byte[] startRow, ArrayList<byte[]> columns, long timestamp) throws IOError, TException
+    public int scannerOpenTs(byte[] tableName, byte[] startRow, List<byte[]> columns, long timestamp) throws IOError, TException
     {
       send_scannerOpenTs(tableName, startRow, columns, timestamp);
       return recv_scannerOpenTs();
     }
 
-    public void send_scannerOpenTs(byte[] tableName, byte[] startRow, ArrayList<byte[]> columns, long timestamp) throws TException
+    public void send_scannerOpenTs(byte[] tableName, byte[] startRow, List<byte[]> columns, long timestamp) throws TException
     {
       oprot_.writeMessageBegin(new TMessage("scannerOpenTs", TMessageType.CALL, seqid_));
       scannerOpenTs_args args = new scannerOpenTs_args();
@@ -1174,13 +1256,13 @@
       throw new TApplicationException(TApplicationException.MISSING_RESULT, "scannerOpenTs failed: unknown result");
     }
 
-    public int scannerOpenWithStopTs(byte[] tableName, byte[] startRow, byte[] stopRow, ArrayList<byte[]> columns, long timestamp) throws IOError, TException
+    public int scannerOpenWithStopTs(byte[] tableName, byte[] startRow, byte[] stopRow, List<byte[]> columns, long timestamp) throws IOError, TException
     {
       send_scannerOpenWithStopTs(tableName, startRow, stopRow, columns, timestamp);
       return recv_scannerOpenWithStopTs();
     }
 
-    public void send_scannerOpenWithStopTs(byte[] tableName, byte[] startRow, byte[] stopRow, ArrayList<byte[]> columns, long timestamp) throws TException
+    public void send_scannerOpenWithStopTs(byte[] tableName, byte[] startRow, byte[] stopRow, List<byte[]> columns, long timestamp) throws TException
     {
       oprot_.writeMessageBegin(new TMessage("scannerOpenWithStopTs", TMessageType.CALL, seqid_));
       scannerOpenWithStopTs_args args = new scannerOpenWithStopTs_args();
@@ -1214,7 +1296,7 @@
       throw new TApplicationException(TApplicationException.MISSING_RESULT, "scannerOpenWithStopTs failed: unknown result");
     }
 
-    public ScanEntry scannerGet(int id) throws IOError, IllegalArgument, NotFound, TException
+    public TRowResult scannerGet(int id) throws IOError, IllegalArgument, NotFound, TException
     {
       send_scannerGet(id);
       return recv_scannerGet();
@@ -1230,7 +1312,7 @@
       oprot_.getTransport().flush();
     }
 
-    public ScanEntry recv_scannerGet() throws IOError, IllegalArgument, NotFound, TException
+    public TRowResult recv_scannerGet() throws IOError, IllegalArgument, NotFound, TException
     {
       TMessage msg = iprot_.readMessageBegin();
       if (msg.type == TMessageType.EXCEPTION) {
@@ -1297,6 +1379,9 @@
     public Processor(Iface iface)
     {
       iface_ = iface;
+      processMap_.put("enableTable", new enableTable());
+      processMap_.put("disableTable", new disableTable());
+      processMap_.put("isTableEnabled", new isTableEnabled());
       processMap_.put("getTableNames", new getTableNames());
       processMap_.put("getColumnDescriptors", new getColumnDescriptors());
       processMap_.put("getTableRegions", new getTableRegions());
@@ -1307,7 +1392,6 @@
       processMap_.put("getVerTs", new getVerTs());
       processMap_.put("getRow", new getRow());
       processMap_.put("getRowTs", new getRowTs());
-      processMap_.put("put", new put());
       processMap_.put("mutateRow", new mutateRow());
       processMap_.put("mutateRowTs", new mutateRowTs());
       processMap_.put("mutateRows", new mutateRows());
@@ -1349,6 +1433,70 @@
       return true;
     }
 
+    private class enableTable implements ProcessFunction {
+      public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException
+      {
+        enableTable_args args = new enableTable_args();
+        args.read(iprot);
+        iprot.readMessageEnd();
+        enableTable_result result = new enableTable_result();
+        try {
+          iface_.enableTable(args.tableName);
+        } catch (IOError io) {
+          result.io = io;
+          result.__isset.io = true;
+        }
+        oprot.writeMessageBegin(new TMessage("enableTable", TMessageType.REPLY, seqid));
+        result.write(oprot);
+        oprot.writeMessageEnd();
+        oprot.getTransport().flush();
+      }
+
+    }
+
+    private class disableTable implements ProcessFunction {
+      public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException
+      {
+        disableTable_args args = new disableTable_args();
+        args.read(iprot);
+        iprot.readMessageEnd();
+        disableTable_result result = new disableTable_result();
+        try {
+          iface_.disableTable(args.tableName);
+        } catch (IOError io) {
+          result.io = io;
+          result.__isset.io = true;
+        }
+        oprot.writeMessageBegin(new TMessage("disableTable", TMessageType.REPLY, seqid));
+        result.write(oprot);
+        oprot.writeMessageEnd();
+        oprot.getTransport().flush();
+      }
+
+    }
+
+    private class isTableEnabled implements ProcessFunction {
+      public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException
+      {
+        isTableEnabled_args args = new isTableEnabled_args();
+        args.read(iprot);
+        iprot.readMessageEnd();
+        isTableEnabled_result result = new isTableEnabled_result();
+        try {
+          result.success = iface_.isTableEnabled(args.tableName);
+          result.__isset.success = true;
+        } catch (IOError io) {
+          result.io = io;
+          result.__isset.io = true;
+        }
+        oprot.writeMessageBegin(new TMessage("isTableEnabled", TMessageType.REPLY, seqid));
+        result.write(oprot);
+        oprot.writeMessageEnd();
+        oprot.getTransport().flush();
+      }
+
+    }
+
     private class getTableNames implements ProcessFunction {
       public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException
       {
@@ -1585,30 +1733,6 @@
 
     }
 
-    private class put implements ProcessFunction {
-      public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException
-      {
-        put_args args = new put_args();
-        args.read(iprot);
-        iprot.readMessageEnd();
-        put_result result = new put_result();
-        try {
-          iface_.put(args.tableName, args.row, args.column, args.value);
-        } catch (IOError io) {
-          result.io = io;
-          result.__isset.io = true;
-        } catch (IllegalArgument ia) {
-          result.ia = ia;
-          result.__isset.ia = true;
-        }
-        oprot.writeMessageBegin(new TMessage("put", TMessageType.REPLY, seqid));
-        result.write(oprot);
-        oprot.writeMessageEnd();
-        oprot.getTransport().flush();
-      }
-
-    }
-
     private class mutateRow implements ProcessFunction {
       public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException
       {
@@ -1931,8 +2055,51 @@
 
   }
 
-  public static class getTableNames_args implements TBase, java.io.Serializable   {
-    public getTableNames_args() {
+  public static class enableTable_args implements TBase, java.io.Serializable   {
+    public byte[] tableName;
+
+    public final Isset __isset = new Isset();
+    public static final class Isset implements java.io.Serializable {
+      public boolean tableName = false;
+    }
+
+    public enableTable_args() {
+    }
+
+    public enableTable_args(
+      byte[] tableName)
+    {
+      this();
+      this.tableName = tableName;
+      this.__isset.tableName = true;
+    }
+
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof enableTable_args)
+        return this.equals((enableTable_args)that);
+      return false;
+    }
+
+    public boolean equals(enableTable_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_tableName = true && (this.tableName != null);
+      boolean that_present_tableName = true && (that.tableName != null);
+      if (this_present_tableName || that_present_tableName) {
+        if (!(this_present_tableName && that_present_tableName))
+          return false;
+        if (!java.util.Arrays.equals(this.tableName, that.tableName))
+          return false;
+      }
+
+      return true;
+    }
+
+    public int hashCode() {
+      return 0;
     }
 
     public void read(TProtocol iprot) throws TException {
@@ -1946,6 +2113,14 @@
         }
         switch (field.id)
         {
+          case 1:
+            if (field.type == TType.STRING) {
+              this.tableName = iprot.readBinary();
+              this.__isset.tableName = true;
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
           default:
             TProtocolUtil.skip(iprot, field.type);
             break;
@@ -1956,44 +2131,78 @@
     }
 
     public void write(TProtocol oprot) throws TException {
-      TStruct struct = new TStruct("getTableNames_args");
+      TStruct struct = new TStruct("enableTable_args");
       oprot.writeStructBegin(struct);
+      TField field = new TField();
+      if (this.tableName != null) {
+        field.name = "tableName";
+        field.type = TType.STRING;
+        field.id = 1;
+        oprot.writeFieldBegin(field);
+        oprot.writeBinary(this.tableName);
+        oprot.writeFieldEnd();
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
 
     public String toString() {
-      StringBuilder sb = new StringBuilder("getTableNames_args(");
+      StringBuilder sb = new StringBuilder("enableTable_args(");
+      sb.append("tableName:");
+      sb.append(this.tableName);
       sb.append(")");
       return sb.toString();
     }
 
   }
 
-  public static class getTableNames_result implements TBase, java.io.Serializable   {
-    public ArrayList<byte[]> success;
+  public static class enableTable_result implements TBase, java.io.Serializable   {
     public IOError io;
 
     public final Isset __isset = new Isset();
-    public static final class Isset {
-      public boolean success = false;
+    public static final class Isset implements java.io.Serializable {
       public boolean io = false;
     }
 
-    public getTableNames_result() {
+    public enableTable_result() {
     }
 
-    public getTableNames_result(
-      ArrayList<byte[]> success,
+    public enableTable_result(
       IOError io)
     {
       this();
-      this.success = success;
-      this.__isset.success = true;
       this.io = io;
       this.__isset.io = true;
     }
 
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof enableTable_result)
+        return this.equals((enableTable_result)that);
+      return false;
+    }
+
+    public boolean equals(enableTable_result that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_io = true && (this.io != null);
+      boolean that_present_io = true && (that.io != null);
+      if (this_present_io || that_present_io) {
+        if (!(this_present_io && that_present_io))
+          return false;
+        if (!this.io.equals(that.io))
+          return false;
+      }
+
+      return true;
+    }
+
+    public int hashCode() {
+      return 0;
+    }
+
     public void read(TProtocol iprot) throws TException {
       TField field;
       iprot.readStructBegin();
@@ -2005,9 +2214,629 @@
         }
         switch (field.id)
         {
-          case 0:
-            if (field.type == TType.LIST) {
-              {
+          case 1:
+            if (field.type == TType.STRUCT) {
+              this.io = new IOError();
+              this.io.read(iprot);
+              this.__isset.io = true;
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          default:
+            TProtocolUtil.skip(iprot, field.type);
+            break;
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+    }
+
+    public void write(TProtocol oprot) throws TException {
+      TStruct struct = new TStruct("enableTable_result");
+      oprot.writeStructBegin(struct);
+      TField field = new TField();
+
+      if (this.__isset.io) {
+        if (this.io != null) {
+          field.name = "io";
+          field.type = TType.STRUCT;
+          field.id = 1;
+          oprot.writeFieldBegin(field);
+          this.io.write(oprot);
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    public String toString() {
+      StringBuilder sb = new StringBuilder("enableTable_result(");
+      sb.append("io:");
+      sb.append(this.io.toString());
+      sb.append(")");
+      return sb.toString();
+    }
+
+  }
+
+  public static class disableTable_args implements TBase, java.io.Serializable   {
+    public byte[] tableName;
+
+    public final Isset __isset = new Isset();
+    public static final class Isset implements java.io.Serializable {
+      public boolean tableName = false;
+    }
+
+    public disableTable_args() {
+    }
+
+    public disableTable_args(
+      byte[] tableName)
+    {
+      this();
+      this.tableName = tableName;
+      this.__isset.tableName = true;
+    }
+
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof disableTable_args)
+        return this.equals((disableTable_args)that);
+      return false;
+    }
+
+    public boolean equals(disableTable_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_tableName = true && (this.tableName != null);
+      boolean that_present_tableName = true && (that.tableName != null);
+      if (this_present_tableName || that_present_tableName) {
+        if (!(this_present_tableName && that_present_tableName))
+          return false;
+        if (!java.util.Arrays.equals(this.tableName, that.tableName))
+          return false;
+      }
+
+      return true;
+    }
+
+    public int hashCode() {
+      return 0;
+    }
+
+    public void read(TProtocol iprot) throws TException {
+      TField field;
+      iprot.readStructBegin();
+      while (true)
+      {
+        field = iprot.readFieldBegin();
+        if (field.type == TType.STOP) { 
+          break;
+        }
+        switch (field.id)
+        {
+          case 1:
+            if (field.type == TType.STRING) {
+              this.tableName = iprot.readBinary();
+              this.__isset.tableName = true;
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          default:
+            TProtocolUtil.skip(iprot, field.type);
+            break;
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+    }
+
+    public void write(TProtocol oprot) throws TException {
+      TStruct struct = new TStruct("disableTable_args");
+      oprot.writeStructBegin(struct);
+      TField field = new TField();
+      if (this.tableName != null) {
+        field.name = "tableName";
+        field.type = TType.STRING;
+        field.id = 1;
+        oprot.writeFieldBegin(field);
+        oprot.writeBinary(this.tableName);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    public String toString() {
+      StringBuilder sb = new StringBuilder("disableTable_args(");
+      sb.append("tableName:");
+      sb.append(this.tableName);
+      sb.append(")");
+      return sb.toString();
+    }
+
+  }
+
+  public static class disableTable_result implements TBase, java.io.Serializable   {
+    public IOError io;
+
+    public final Isset __isset = new Isset();
+    public static final class Isset implements java.io.Serializable {
+      public boolean io = false;
+    }
+
+    public disableTable_result() {
+    }
+
+    public disableTable_result(
+      IOError io)
+    {
+      this();
+      this.io = io;
+      this.__isset.io = true;
+    }
+
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof disableTable_result)
+        return this.equals((disableTable_result)that);
+      return false;
+    }
+
+    public boolean equals(disableTable_result that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_io = true && (this.io != null);
+      boolean that_present_io = true && (that.io != null);
+      if (this_present_io || that_present_io) {
+        if (!(this_present_io && that_present_io))
+          return false;
+        if (!this.io.equals(that.io))
+          return false;
+      }
+
+      return true;
+    }
+
+    public int hashCode() {
+      return 0;
+    }
+
+    public void read(TProtocol iprot) throws TException {
+      TField field;
+      iprot.readStructBegin();
+      while (true)
+      {
+        field = iprot.readFieldBegin();
+        if (field.type == TType.STOP) { 
+          break;
+        }
+        switch (field.id)
+        {
+          case 1:
+            if (field.type == TType.STRUCT) {
+              this.io = new IOError();
+              this.io.read(iprot);
+              this.__isset.io = true;
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          default:
+            TProtocolUtil.skip(iprot, field.type);
+            break;
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+    }
+
+    public void write(TProtocol oprot) throws TException {
+      TStruct struct = new TStruct("disableTable_result");
+      oprot.writeStructBegin(struct);
+      TField field = new TField();
+
+      if (this.__isset.io) {
+        if (this.io != null) {
+          field.name = "io";
+          field.type = TType.STRUCT;
+          field.id = 1;
+          oprot.writeFieldBegin(field);
+          this.io.write(oprot);
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    public String toString() {
+      StringBuilder sb = new StringBuilder("disableTable_result(");
+      sb.append("io:");
+      sb.append(this.io.toString());
+      sb.append(")");
+      return sb.toString();
+    }
+
+  }
+
+  public static class isTableEnabled_args implements TBase, java.io.Serializable   {
+    public byte[] tableName;
+
+    public final Isset __isset = new Isset();
+    public static final class Isset implements java.io.Serializable {
+      public boolean tableName = false;
+    }
+
+    public isTableEnabled_args() {
+    }
+
+    public isTableEnabled_args(
+      byte[] tableName)
+    {
+      this();
+      this.tableName = tableName;
+      this.__isset.tableName = true;
+    }
+
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof isTableEnabled_args)
+        return this.equals((isTableEnabled_args)that);
+      return false;
+    }
+
+    public boolean equals(isTableEnabled_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_tableName = true && (this.tableName != null);
+      boolean that_present_tableName = true && (that.tableName != null);
+      if (this_present_tableName || that_present_tableName) {
+        if (!(this_present_tableName && that_present_tableName))
+          return false;
+        if (!java.util.Arrays.equals(this.tableName, that.tableName))
+          return false;
+      }
+
+      return true;
+    }
+
+    public int hashCode() {
+      return 0;
+    }
+
+    public void read(TProtocol iprot) throws TException {
+      TField field;
+      iprot.readStructBegin();
+      while (true)
+      {
+        field = iprot.readFieldBegin();
+        if (field.type == TType.STOP) { 
+          break;
+        }
+        switch (field.id)
+        {
+          case 1:
+            if (field.type == TType.STRING) {
+              this.tableName = iprot.readBinary();
+              this.__isset.tableName = true;
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          default:
+            TProtocolUtil.skip(iprot, field.type);
+            break;
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+    }
+
+    public void write(TProtocol oprot) throws TException {
+      TStruct struct = new TStruct("isTableEnabled_args");
+      oprot.writeStructBegin(struct);
+      TField field = new TField();
+      if (this.tableName != null) {
+        field.name = "tableName";
+        field.type = TType.STRING;
+        field.id = 1;
+        oprot.writeFieldBegin(field);
+        oprot.writeBinary(this.tableName);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    public String toString() {
+      StringBuilder sb = new StringBuilder("isTableEnabled_args(");
+      sb.append("tableName:");
+      sb.append(this.tableName);
+      sb.append(")");
+      return sb.toString();
+    }
+
+  }
+
+  public static class isTableEnabled_result implements TBase, java.io.Serializable   {
+    public boolean success;
+    public IOError io;
+
+    public final Isset __isset = new Isset();
+    public static final class Isset implements java.io.Serializable {
+      public boolean success = false;
+      public boolean io = false;
+    }
+
+    public isTableEnabled_result() {
+    }
+
+    public isTableEnabled_result(
+      boolean success,
+      IOError io)
+    {
+      this();
+      this.success = success;
+      this.__isset.success = true;
+      this.io = io;
+      this.__isset.io = true;
+    }
+
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof isTableEnabled_result)
+        return this.equals((isTableEnabled_result)that);
+      return false;
+    }
+
+    public boolean equals(isTableEnabled_result that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_success = true;
+      boolean that_present_success = true;
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (this.success != that.success)
+          return false;
+      }
+
+      boolean this_present_io = true && (this.io != null);
+      boolean that_present_io = true && (that.io != null);
+      if (this_present_io || that_present_io) {
+        if (!(this_present_io && that_present_io))
+          return false;
+        if (!this.io.equals(that.io))
+          return false;
+      }
+
+      return true;
+    }
+
+    public int hashCode() {
+      return 0;
+    }
+
+    public void read(TProtocol iprot) throws TException {
+      TField field;
+      iprot.readStructBegin();
+      while (true)
+      {
+        field = iprot.readFieldBegin();
+        if (field.type == TType.STOP) { 
+          break;
+        }
+        switch (field.id)
+        {
+          case 0:
+            if (field.type == TType.BOOL) {
+              this.success = iprot.readBool();
+              this.__isset.success = true;
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          case 1:
+            if (field.type == TType.STRUCT) {
+              this.io = new IOError();
+              this.io.read(iprot);
+              this.__isset.io = true;
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          default:
+            TProtocolUtil.skip(iprot, field.type);
+            break;
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+    }
+
+    public void write(TProtocol oprot) throws TException {
+      TStruct struct = new TStruct("isTableEnabled_result");
+      oprot.writeStructBegin(struct);
+      TField field = new TField();
+
+      if (this.__isset.success) {
+        field.name = "success";
+        field.type = TType.BOOL;
+        field.id = 0;
+        oprot.writeFieldBegin(field);
+        oprot.writeBool(this.success);
+        oprot.writeFieldEnd();
+      } else if (this.__isset.io) {
+        if (this.io != null) {
+          field.name = "io";
+          field.type = TType.STRUCT;
+          field.id = 1;
+          oprot.writeFieldBegin(field);
+          this.io.write(oprot);
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    public String toString() {
+      StringBuilder sb = new StringBuilder("isTableEnabled_result(");
+      sb.append("success:");
+      sb.append(this.success);
+      sb.append(",io:");
+      sb.append(this.io.toString());
+      sb.append(")");
+      return sb.toString();
+    }
+
+  }
+
+  public static class getTableNames_args implements TBase, java.io.Serializable   {
+    public getTableNames_args() {
+    }
+
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof getTableNames_args)
+        return this.equals((getTableNames_args)that);
+      return false;
+    }
+
+    public boolean equals(getTableNames_args that) {
+      if (that == null)
+        return false;
+
+      return true;
+    }
+
+    public int hashCode() {
+      return 0;
+    }
+
+    public void read(TProtocol iprot) throws TException {
+      TField field;
+      iprot.readStructBegin();
+      while (true)
+      {
+        field = iprot.readFieldBegin();
+        if (field.type == TType.STOP) { 
+          break;
+        }
+        switch (field.id)
+        {
+          default:
+            TProtocolUtil.skip(iprot, field.type);
+            break;
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+    }
+
+    public void write(TProtocol oprot) throws TException {
+      TStruct struct = new TStruct("getTableNames_args");
+      oprot.writeStructBegin(struct);
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    public String toString() {
+      StringBuilder sb = new StringBuilder("getTableNames_args(");
+      sb.append(")");
+      return sb.toString();
+    }
+
+  }
+
+  public static class getTableNames_result implements TBase, java.io.Serializable   {
+    public List<byte[]> success;
+    public IOError io;
+
+    public final Isset __isset = new Isset();
+    public static final class Isset implements java.io.Serializable {
+      public boolean success = false;
+      public boolean io = false;
+    }
+
+    public getTableNames_result() {
+    }
+
+    public getTableNames_result(
+      List<byte[]> success,
+      IOError io)
+    {
+      this();
+      this.success = success;
+      this.__isset.success = true;
+      this.io = io;
+      this.__isset.io = true;
+    }
+
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof getTableNames_result)
+        return this.equals((getTableNames_result)that);
+      return false;
+    }
+
+    public boolean equals(getTableNames_result that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_success = true && (this.success != null);
+      boolean that_present_success = true && (that.success != null);
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (!this.success.equals(that.success))
+          return false;
+      }
+
+      boolean this_present_io = true && (this.io != null);
+      boolean that_present_io = true && (that.io != null);
+      if (this_present_io || that_present_io) {
+        if (!(this_present_io && that_present_io))
+          return false;
+        if (!this.io.equals(that.io))
+          return false;
+      }
+
+      return true;
+    }
+
+    public int hashCode() {
+      return 0;
+    }
+
+    public void read(TProtocol iprot) throws TException {
+      TField field;
+      iprot.readStructBegin();
+      while (true)
+      {
+        field = iprot.readFieldBegin();
+        if (field.type == TType.STOP) { 
+          break;
+        }
+        switch (field.id)
+        {
+          case 0:
+            if (field.type == TType.LIST) {
+              {
                 TList _list9 = iprot.readListBegin();
                 this.success = new ArrayList<byte[]>(_list9.size);
                 for (int _i10 = 0; _i10 < _list9.size; ++_i10)
@@ -2091,7 +2920,7 @@
     public byte[] tableName;
 
     public final Isset __isset = new Isset();
-    public static final class Isset {
+    public static final class Isset implements java.io.Serializable {
       public boolean tableName = false;
     }
 
@@ -2106,6 +2935,34 @@
       this.__isset.tableName = true;
     }
 
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof getColumnDescriptors_args)
+        return this.equals((getColumnDescriptors_args)that);
+      return false;
+    }
+
+    public boolean equals(getColumnDescriptors_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_tableName = true && (this.tableName != null);
+      boolean that_present_tableName = true && (that.tableName != null);
+      if (this_present_tableName || that_present_tableName) {
+        if (!(this_present_tableName && that_present_tableName))
+          return false;
+        if (!java.util.Arrays.equals(this.tableName, that.tableName))
+          return false;
+      }
+
+      return true;
+    }
+
+    public int hashCode() {
+      return 0;
+    }
+
     public void read(TProtocol iprot) throws TException {
       TField field;
       iprot.readStructBegin();
@@ -2161,11 +3018,11 @@
   }
 
   public static class getColumnDescriptors_result implements TBase, java.io.Serializable   {
-    public AbstractMap<byte[],ColumnDescriptor> success;
+    public Map<byte[],ColumnDescriptor> success;
     public IOError io;
 
     public final Isset __isset = new Isset();
-    public static final class Isset {
+    public static final class Isset implements java.io.Serializable {
       public boolean success = false;
       public boolean io = false;
     }
@@ -2174,7 +3031,7 @@
     }
 
     public getColumnDescriptors_result(
-      AbstractMap<byte[],ColumnDescriptor> success,
+      Map<byte[],ColumnDescriptor> success,
       IOError io)
     {
       this();
@@ -2184,6 +3041,43 @@
       this.__isset.io = true;
     }
 
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof getColumnDescriptors_result)
+        return this.equals((getColumnDescriptors_result)that);
+      return false;
+    }
+
+    public boolean equals(getColumnDescriptors_result that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_success = true && (this.success != null);
+      boolean that_present_success = true && (that.success != null);
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (!this.success.equals(that.success))
+          return false;
+      }
+
+      boolean this_present_io = true && (this.io != null);
+      boolean that_present_io = true && (that.io != null);
+      if (this_present_io || that_present_io) {
+        if (!(this_present_io && that_present_io))
+          return false;
+        if (!this.io.equals(that.io))
+          return false;
+      }
+
+      return true;
+    }
+
+    public int hashCode() {
+      return 0;
+    }
+
     public void read(TProtocol iprot) throws TException {
       TField field;
       iprot.readStructBegin();
@@ -2285,7 +3179,7 @@
     public byte[] tableName;
 
     public final Isset __isset = new Isset();
-    public static final class Isset {
+    public static final class Isset implements java.io.Serializable {
       public boolean tableName = false;
     }
 
@@ -2300,6 +3194,34 @@
       this.__isset.tableName = true;
     }
 
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof getTableRegions_args)
+        return this.equals((getTableRegions_args)that);
+      return false;
+    }
+
+    public boolean equals(getTableRegions_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_tableName = true && (this.tableName != null);
+      boolean that_present_tableName = true && (that.tableName != null);
+      if (this_present_tableName || that_present_tableName) {
+        if (!(this_present_tableName && that_present_tableName))
+          return false;
+        if (!java.util.Arrays.equals(this.tableName, that.tableName))
+          return false;
+      }
+
+      return true;
+    }
+
+    public int hashCode() {
+      return 0;
+    }
+
     public void read(TProtocol iprot) throws TException {
       TField field;
       iprot.readStructBegin();
@@ -2355,11 +3277,11 @@
   }
 
   public static class getTableRegions_result implements TBase, java.io.Serializable   {
-    public ArrayList<RegionDescriptor> success;
+    public List<TRegionInfo> success;
     public IOError io;
 
     public final Isset __isset = new Isset();
-    public static final class Isset {
+    public static final class Isset implements java.io.Serializable {
       public boolean success = false;
       public boolean io = false;
     }
@@ -2368,7 +3290,7 @@
     }
 
     public getTableRegions_result(
-      ArrayList<RegionDescriptor> success,
+      List<TRegionInfo> success,
       IOError io)
     {
       this();
@@ -2378,6 +3300,43 @@
       this.__isset.io = true;
     }
 
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof getTableRegions_result)
+        return this.equals((getTableRegions_result)that);
+      return false;
+    }
+
+    public boolean equals(getTableRegions_result that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_success = true && (this.success != null);
+      boolean that_present_success = true && (that.success != null);
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (!this.success.equals(that.success))
+          return false;
+      }
+
+      boolean this_present_io = true && (this.io != null);
+      boolean that_present_io = true && (that.io != null);
+      if (this_present_io || that_present_io) {
+        if (!(this_present_io && that_present_io))
+          return false;
+        if (!this.io.equals(that.io))
+          return false;
+      }
+
+      return true;
+    }
+
+    public int hashCode() {
+      return 0;
+    }
+
     public void read(TProtocol iprot) throws TException {
       TField field;
       iprot.readStructBegin();
@@ -2393,11 +3352,11 @@
             if (field.type == TType.LIST) {
               {
                 TList _list18 = iprot.readListBegin();
-                this.success = new ArrayList<RegionDescriptor>(_list18.size);
+                this.success = new ArrayList<TRegionInfo>(_list18.size);
                 for (int _i19 = 0; _i19 < _list18.size; ++_i19)
                 {
-                  RegionDescriptor _elem20 = new RegionDescriptor();
-                  _elem20 = new RegionDescriptor();
+                  TRegionInfo _elem20 = new TRegionInfo();
+                  _elem20 = new TRegionInfo();
                   _elem20.read(iprot);
                   this.success.add(_elem20);
                 }
@@ -2439,7 +3398,7 @@
           oprot.writeFieldBegin(field);
           {
             oprot.writeListBegin(new TList(TType.STRUCT, this.success.size()));
-            for (RegionDescriptor _iter21 : this.success)            {
+            for (TRegionInfo _iter21 : this.success)            {
               _iter21.write(oprot);
             }
             oprot.writeListEnd();
@@ -2474,10 +3433,10 @@
 
   public static class createTable_args implements TBase, java.io.Serializable   {
     public byte[] tableName;
-    public ArrayList<ColumnDescriptor> columnFamilies;
+    public List<ColumnDescriptor> columnFamilies;
 
     public final Isset __isset = new Isset();
-    public static final class Isset {
+    public static final class Isset implements java.io.Serializable {
       public boolean tableName = false;
       public boolean columnFamilies = false;
     }
@@ -2487,7 +3446,7 @@
 
     public createTable_args(
       byte[] tableName,
-      ArrayList<ColumnDescriptor> columnFamilies)
+      List<ColumnDescriptor> columnFamilies)
     {
       this();
       this.tableName = tableName;
@@ -2496,6 +3455,43 @@
       this.__isset.columnFamilies = true;
     }
 
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof createTable_args)
+        return this.equals((createTable_args)that);
+      return false;
+    }
+
+    public boolean equals(createTable_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_tableName = true && (this.tableName != null);
+      boolean that_present_tableName = true && (that.tableName != null);
+      if (this_present_tableName || that_present_tableName) {
+        if (!(this_present_tableName && that_present_tableName))
+          return false;
+        if (!java.util.Arrays.equals(this.tableName, that.tableName))
+          return false;
+      }
+
+      boolean this_present_columnFamilies = true && (this.columnFamilies != null);
+      boolean that_present_columnFamilies = true && (that.columnFamilies != null);
+      if (this_present_columnFamilies || that_present_columnFamilies) {
+        if (!(this_present_columnFamilies && that_present_columnFamilies))
+          return false;
+        if (!this.columnFamilies.equals(that.columnFamilies))
+          return false;
+      }
+
+      return true;
+    }
+
+    public int hashCode() {
+      return 0;
+    }
+
     public void read(TProtocol iprot) throws TException {
       TField field;
       iprot.readStructBegin();
@@ -2591,7 +3587,7 @@
     public AlreadyExists exist;
 
     public final Isset __isset = new Isset();
-    public static final class Isset {
+    public static final class Isset implements java.io.Serializable {
       public boolean io = false;
       public boolean ia = false;
       public boolean exist = false;
@@ -2614,6 +3610,52 @@
       this.__isset.exist = true;
     }
 
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof createTable_result)
+        return this.equals((createTable_result)that);
+      return false;
+    }
+
+    public boolean equals(createTable_result that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_io = true && (this.io != null);
+      boolean that_present_io = true && (that.io != null);
+      if (this_present_io || that_present_io) {
+        if (!(this_present_io && that_present_io))
+          return false;
+        if (!this.io.equals(that.io))
+          return false;
+      }
+
+      boolean this_present_ia = true && (this.ia != null);
+      boolean that_present_ia = true && (that.ia != null);
+      if (this_present_ia || that_present_ia) {
+        if (!(this_present_ia && that_present_ia))
+          return false;
+        if (!this.ia.equals(that.ia))
+          return false;
+      }
+
+      boolean this_present_exist = true && (this.exist != null);
+      boolean that_present_exist = true && (that.exist != null);
+      if (this_present_exist || that_present_exist) {
+        if (!(this_present_exist && that_present_exist))
+          return false;
+        if (!this.exist.equals(that.exist))
+          return false;
+      }
+
+      return true;
+    }
+
+    public int hashCode() {
+      return 0;
+    }
+
     public void read(TProtocol iprot) throws TException {
       TField field;
       iprot.readStructBegin();
@@ -2716,7 +3758,7 @@
     public byte[] tableName;
 
     public final Isset __isset = new Isset();
-    public static final class Isset {
+    public static final class Isset implements java.io.Serializable {
       public boolean tableName = false;
     }
 
@@ -2731,189 +3773,32 @@
       this.__isset.tableName = true;
     }
 
-    public void read(TProtocol iprot) throws TException {
-      TField field;
-      iprot.readStructBegin();
-      while (true)
-      {
-        field = iprot.readFieldBegin();
-        if (field.type == TType.STOP) { 
-          break;
-        }
-        switch (field.id)
-        {
-          case 1:
-            if (field.type == TType.STRING) {
-              this.tableName = iprot.readBinary();
-              this.__isset.tableName = true;
-            } else { 
-              TProtocolUtil.skip(iprot, field.type);
-            }
-            break;
-          default:
-            TProtocolUtil.skip(iprot, field.type);
-            break;
-        }
-        iprot.readFieldEnd();
-      }
-      iprot.readStructEnd();
-    }
-
-    public void write(TProtocol oprot) throws TException {
-      TStruct struct = new TStruct("deleteTable_args");
-      oprot.writeStructBegin(struct);
-      TField field = new TField();
-      if (this.tableName != null) {
-        field.name = "tableName";
-        field.type = TType.STRING;
-        field.id = 1;
-        oprot.writeFieldBegin(field);
-        oprot.writeBinary(this.tableName);
-        oprot.writeFieldEnd();
-      }
-      oprot.writeFieldStop();
-      oprot.writeStructEnd();
-    }
-
-    public String toString() {
-      StringBuilder sb = new StringBuilder("deleteTable_args(");
-      sb.append("tableName:");
-      sb.append(this.tableName);
-      sb.append(")");
-      return sb.toString();
-    }
-
-  }
-
-  public static class deleteTable_result implements TBase, java.io.Serializable   {
-    public IOError io;
-    public NotFound nf;
-
-    public final Isset __isset = new Isset();
-    public static final class Isset {
-      public boolean io = false;
-      public boolean nf = false;
-    }
-
-    public deleteTable_result() {
-    }
-
-    public deleteTable_result(
-      IOError io,
-      NotFound nf)
-    {
-      this();
-      this.io = io;
-      this.__isset.io = true;
-      this.nf = nf;
-      this.__isset.nf = true;
-    }
-
-    public void read(TProtocol iprot) throws TException {
-      TField field;
-      iprot.readStructBegin();
-      while (true)
-      {
-        field = iprot.readFieldBegin();
-        if (field.type == TType.STOP) { 
-          break;
-        }
-        switch (field.id)
-        {
-          case 1:
-            if (field.type == TType.STRUCT) {
-              this.io = new IOError();
-              this.io.read(iprot);
-              this.__isset.io = true;
-            } else { 
-              TProtocolUtil.skip(iprot, field.type);
-            }
-            break;
-          case 2:
-            if (field.type == TType.STRUCT) {
-              this.nf = new NotFound();
-              this.nf.read(iprot);
-              this.__isset.nf = true;
-            } else { 
-              TProtocolUtil.skip(iprot, field.type);
-            }
-            break;
-          default:
-            TProtocolUtil.skip(iprot, field.type);
-            break;
-        }
-        iprot.readFieldEnd();
-      }
-      iprot.readStructEnd();
-    }
-
-    public void write(TProtocol oprot) throws TException {
-      TStruct struct = new TStruct("deleteTable_result");
-      oprot.writeStructBegin(struct);
-      TField field = new TField();
-
-      if (this.__isset.io) {
-        if (this.io != null) {
-          field.name = "io";
-          field.type = TType.STRUCT;
-          field.id = 1;
-          oprot.writeFieldBegin(field);
-          this.io.write(oprot);
-          oprot.writeFieldEnd();
-        }
-      } else if (this.__isset.nf) {
-        if (this.nf != null) {
-          field.name = "nf";
-          field.type = TType.STRUCT;
-          field.id = 2;
-          oprot.writeFieldBegin(field);
-          this.nf.write(oprot);
-          oprot.writeFieldEnd();
-        }
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof deleteTable_args)
+        return this.equals((deleteTable_args)that);
+      return false;
+    }
+
+    public boolean equals(deleteTable_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_tableName = true && (this.tableName != null);
+      boolean that_present_tableName = true && (that.tableName != null);
+      if (this_present_tableName || that_present_tableName) {
+        if (!(this_present_tableName && that_present_tableName))
+          return false;
+        if (!java.util.Arrays.equals(this.tableName, that.tableName))
+          return false;
       }
-      oprot.writeFieldStop();
-      oprot.writeStructEnd();
-    }
-
-    public String toString() {
-      StringBuilder sb = new StringBuilder("deleteTable_result(");
-      sb.append("io:");
-      sb.append(this.io.toString());
-      sb.append(",nf:");
-      sb.append(this.nf.toString());
-      sb.append(")");
-      return sb.toString();
-    }
-
-  }
-
-  public static class get_args implements TBase, java.io.Serializable   {
-    public byte[] tableName;
-    public byte[] row;
-    public byte[] column;
-
-    public final Isset __isset = new Isset();
-    public static final class Isset {
-      public boolean tableName = false;
-      public boolean row = false;
-      public boolean column = false;
-    }
 
-    public get_args() {
+      return true;
     }
-
-    public get_args(
-      byte[] tableName,
-      byte[] row,
-      byte[] column)
-    {
-      this();
-      this.tableName = tableName;
-      this.__isset.tableName = true;
-      this.row = row;
-      this.__isset.row = true;
-      this.column = column;
-      this.__isset.column = true;
+
+    public int hashCode() {
+      return 0;
     }
 
     public void read(TProtocol iprot) throws TException {
@@ -2935,22 +3820,6 @@
               TProtocolUtil.skip(iprot, field.type);
             }
             break;
-          case 2:
-            if (field.type == TType.STRING) {
-              this.row = iprot.readBinary();
-              this.__isset.row = true;
-            } else { 
-              TProtocolUtil.skip(iprot, field.type);
-            }
-            break;
-          case 3:
-            if (field.type == TType.STRING) {
-              this.column = iprot.readBinary();
-              this.__isset.column = true;
-            } else { 
-              TProtocolUtil.skip(iprot, field.type);
-            }
-            break;
           default:
             TProtocolUtil.skip(iprot, field.type);
             break;
@@ -2961,7 +3830,7 @@
     }
 
     public void write(TProtocol oprot) throws TException {
-      TStruct struct = new TStruct("get_args");
+      TStruct struct = new TStruct("deleteTable_args");
       oprot.writeStructBegin(struct);
       TField field = new TField();
       if (this.tableName != null) {
@@ -2972,69 +3841,81 @@
         oprot.writeBinary(this.tableName);
         oprot.writeFieldEnd();
       }
-      if (this.row != null) {
-        field.name = "row";
-        field.type = TType.STRING;
-        field.id = 2;
-        oprot.writeFieldBegin(field);
-        oprot.writeBinary(this.row);
-        oprot.writeFieldEnd();
-      }
-      if (this.column != null) {
-        field.name = "column";
-        field.type = TType.STRING;
-        field.id = 3;
-        oprot.writeFieldBegin(field);
-        oprot.writeBinary(this.column);
-        oprot.writeFieldEnd();
-      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
 
     public String toString() {
-      StringBuilder sb = new StringBuilder("get_args(");
+      StringBuilder sb = new StringBuilder("deleteTable_args(");
       sb.append("tableName:");
       sb.append(this.tableName);
-      sb.append(",row:");
-      sb.append(this.row);
-      sb.append(",column:");
-      sb.append(this.column);
       sb.append(")");
       return sb.toString();
     }
 
   }
 
-  public static class get_result implements TBase, java.io.Serializable   {
-    public byte[] success;
+  public static class deleteTable_result implements TBase, java.io.Serializable   {
     public IOError io;
     public NotFound nf;
 
     public final Isset __isset = new Isset();
-    public static final class Isset {
-      public boolean success = false;
+    public static final class Isset implements java.io.Serializable {
       public boolean io = false;
       public boolean nf = false;
     }
 
-    public get_result() {
+    public deleteTable_result() {
     }
 
-    public get_result(
-      byte[] success,
+    public deleteTable_result(
       IOError io,
       NotFound nf)
     {
       this();
-      this.success = success;
-      this.__isset.success = true;
       this.io = io;
       this.__isset.io = true;
       this.nf = nf;
       this.__isset.nf = true;
     }
 
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof deleteTable_result)
+        return this.equals((deleteTable_result)that);
+      return false;
+    }
+
+    public boolean equals(deleteTable_result that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_io = true && (this.io != null);
+      boolean that_present_io = true && (that.io != null);
+      if (this_present_io || that_present_io) {
+        if (!(this_present_io && that_present_io))
+          return false;
+        if (!this.io.equals(that.io))
+          return false;
+      }
+
+      boolean this_present_nf = true && (this.nf != null);
+      boolean that_present_nf = true && (that.nf != null);
+      if (this_present_nf || that_present_nf) {
+        if (!(this_present_nf && that_present_nf))
+          return false;
+        if (!this.nf.equals(that.nf))
+          return false;
+      }
+
+      return true;
+    }
+
+    public int hashCode() {
+      return 0;
+    }
+
     public void read(TProtocol iprot) throws TException {
       TField field;
       iprot.readStructBegin();
@@ -3046,14 +3927,6 @@
         }
         switch (field.id)
         {
-          case 0:
-            if (field.type == TType.STRING) {
-              this.success = iprot.readBinary();
-              this.__isset.success = true;
-            } else { 
-              TProtocolUtil.skip(iprot, field.type);
-            }
-            break;
           case 1:
             if (field.type == TType.STRUCT) {
               this.io = new IOError();
@@ -3082,20 +3955,11 @@
     }
 
     public void write(TProtocol oprot) throws TException {
-      TStruct struct = new TStruct("get_result");
+      TStruct struct = new TStruct("deleteTable_result");
       oprot.writeStructBegin(struct);
       TField field = new TField();
 
-      if (this.__isset.success) {
-        if (this.success != null) {
-          field.name = "success";
-          field.type = TType.STRING;
-          field.id = 0;
-          oprot.writeFieldBegin(field);
-          oprot.writeBinary(this.success);
-          oprot.writeFieldEnd();
-        }
-      } else if (this.__isset.io) {
+      if (this.__isset.io) {
         if (this.io != null) {
           field.name = "io";
           field.type = TType.STRUCT;
@@ -3119,10 +3983,8 @@
     }
 
     public String toString() {
-      StringBuilder sb = new StringBuilder("get_result(");
-      sb.append("success:");
-      sb.append(this.success);
-      sb.append(",io:");
+      StringBuilder sb = new StringBuilder("deleteTable_result(");
+      sb.append("io:");
       sb.append(this.io.toString());
       sb.append(",nf:");
       sb.append(this.nf.toString());
@@ -3132,28 +3994,25 @@
 
   }
 
-  public static class getVer_args implements TBase, java.io.Serializable   {
+  public static class get_args implements TBase, java.io.Serializable   {
     public byte[] tableName;
     public byte[] row;
     public byte[] column;
-    public int numVersions;
 
     public final Isset __isset = new Isset();
-    public static final class Isset {
+    public static final class Isset implements java.io.Serializable {
       public boolean tableName = false;
       public boolean row = false;
       public boolean column = false;
-      public boolean numVersions = false;
     }
 
-    public getVer_args() {
+    public get_args() {
     }
 
-    public getVer_args(
+    public get_args(
       byte[] tableName,
       byte[] row,
-      byte[] column,
-      int numVersions)
+      byte[] column)
     {
       this();
       this.tableName = tableName;
@@ -3162,8 +4021,52 @@
       this.__isset.row = true;
       this.column = column;
       this.__isset.column = true;
-      this.numVersions = numVersions;
-      this.__isset.numVersions = true;
+    }
+
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof get_args)
+        return this.equals((get_args)that);
+      return false;
+    }
+
+    public boolean equals(get_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_tableName = true && (this.tableName != null);
+      boolean that_present_tableName = true && (that.tableName != null);
+      if (this_present_tableName || that_present_tableName) {
+        if (!(this_present_tableName && that_present_tableName))
+          return false;
+        if (!java.util.Arrays.equals(this.tableName, that.tableName))
+          return false;
+      }
+
+      boolean this_present_row = true && (this.row != null);
+      boolean that_present_row = true && (that.row != null);
+      if (this_present_row || that_present_row) {
+        if (!(this_present_row && that_present_row))
+          return false;
+        if (!java.util.Arrays.equals(this.row, that.row))
+          return false;
+      }
+
+      boolean this_present_column = true && (this.column != null);
+      boolean that_present_column = true && (that.column != null);
+      if (this_present_column || that_present_column) {
+        if (!(this_present_column && that_present_column))
+          return false;
+        if (!java.util.Arrays.equals(this.column, that.column))
+          return false;
+      }
+
+      return true;
+    }
+
+    public int hashCode() {
+      return 0;
     }
 
     public void read(TProtocol iprot) throws TException {
@@ -3201,14 +4104,6 @@
               TProtocolUtil.skip(iprot, field.type);
             }
             break;
-          case 4:
-            if (field.type == TType.I32) {
-              this.numVersions = iprot.readI32();
-              this.__isset.numVersions = true;
-            } else { 
-              TProtocolUtil.skip(iprot, field.type);
-            }
-            break;
           default:
             TProtocolUtil.skip(iprot, field.type);
             break;
@@ -3219,7 +4114,7 @@
     }
 
     public void write(TProtocol oprot) throws TException {
-      TStruct struct = new TStruct("getVer_args");
+      TStruct struct = new TStruct("get_args");
       oprot.writeStructBegin(struct);
       TField field = new TField();
       if (this.tableName != null) {
@@ -3246,49 +4141,41 @@
         oprot.writeBinary(this.column);
         oprot.writeFieldEnd();
       }
-      field.name = "numVersions";
-      field.type = TType.I32;
-      field.id = 4;
-      oprot.writeFieldBegin(field);
-      oprot.writeI32(this.numVersions);
-      oprot.writeFieldEnd();
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
 
     public String toString() {
-      StringBuilder sb = new StringBuilder("getVer_args(");
+      StringBuilder sb = new StringBuilder("get_args(");
       sb.append("tableName:");
       sb.append(this.tableName);
       sb.append(",row:");
       sb.append(this.row);
       sb.append(",column:");
       sb.append(this.column);
-      sb.append(",numVersions:");
-      sb.append(this.numVersions);
       sb.append(")");
       return sb.toString();
     }
 
   }
 
-  public static class getVer_result implements TBase, java.io.Serializable   {
-    public ArrayList<byte[]> success;
+  public static class get_result implements TBase, java.io.Serializable   {
+    public TCell success;
     public IOError io;
     public NotFound nf;
 
     public final Isset __isset = new Isset();
-    public static final class Isset {
+    public static final class Isset implements java.io.Serializable {
       public boolean success = false;
       public boolean io = false;
       public boolean nf = false;
     }
 
-    public getVer_result() {
+    public get_result() {
     }
 
-    public getVer_result(
-      ArrayList<byte[]> success,
+    public get_result(
+      TCell success,
       IOError io,
       NotFound nf)
     {
@@ -3301,6 +4188,52 @@
       this.__isset.nf = true;
     }
 
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof get_result)
+        return this.equals((get_result)that);
+      return false;
+    }
+
+    public boolean equals(get_result that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_success = true && (this.success != null);
+      boolean that_present_success = true && (that.success != null);
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (!this.success.equals(that.success))
+          return false;
+      }
+
+      boolean this_present_io = true && (this.io != null);
+      boolean that_present_io = true && (that.io != null);
+      if (this_present_io || that_present_io) {
+        if (!(this_present_io && that_present_io))
+          return false;
+        if (!this.io.equals(that.io))
+          return false;
+      }
+
+      boolean this_present_nf = true && (this.nf != null);
+      boolean that_present_nf = true && (that.nf != null);
+      if (this_present_nf || that_present_nf) {
+        if (!(this_present_nf && that_present_nf))
+          return false;
+        if (!this.nf.equals(that.nf))
+          return false;
+      }
+
+      return true;
+    }
+
+    public int hashCode() {
+      return 0;
+    }
+
     public void read(TProtocol iprot) throws TException {
       TField field;
       iprot.readStructBegin();
@@ -3313,18 +4246,9 @@
         switch (field.id)
         {
           case 0:
-            if (field.type == TType.LIST) {
-              {
-                TList _list26 = iprot.readListBegin();
-                this.success = new ArrayList<byte[]>(_list26.size);
-                for (int _i27 = 0; _i27 < _list26.size; ++_i27)
-                {
-                  byte[] _elem28 = null;
-                  _elem28 = iprot.readBinary();
-                  this.success.add(_elem28);
-                }
-                iprot.readListEnd();
-              }
+            if (field.type == TType.STRUCT) {
+              this.success = new TCell();
+              this.success.read(iprot);
               this.__isset.success = true;
             } else { 
               TProtocolUtil.skip(iprot, field.type);
@@ -3358,23 +4282,17 @@
     }
 
     public void write(TProtocol oprot) throws TException {
-      TStruct struct = new TStruct("getVer_result");
+      TStruct struct = new TStruct("get_result");
       oprot.writeStructBegin(struct);
       TField field = new TField();
 
       if (this.__isset.success) {
         if (this.success != null) {
           field.name = "success";
-          field.type = TType.LIST;
+          field.type = TType.STRUCT;
           field.id = 0;
           oprot.writeFieldBegin(field);
-          {
-            oprot.writeListBegin(new TList(TType.STRING, this.success.size()));
-            for (byte[] _iter29 : this.success)            {
-              oprot.writeBinary(_iter29);
-            }
-            oprot.writeListEnd();
-          }
+          this.success.write(oprot);
           oprot.writeFieldEnd();
         }
       } else if (this.__isset.io) {
@@ -3401,9 +4319,9 @@
     }
 
     public String toString() {
-      StringBuilder sb = new StringBuilder("getVer_result(");
+      StringBuilder sb = new StringBuilder("get_result(");
       sb.append("success:");
-      sb.append(this.success);
+      sb.append(this.success.toString());
       sb.append(",io:");
       sb.append(this.io.toString());
       sb.append(",nf:");
@@ -3414,30 +4332,27 @@
 
   }
 
-  public static class getVerTs_args implements TBase, java.io.Serializable   {
+  public static class getVer_args implements TBase, java.io.Serializable   {
     public byte[] tableName;
     public byte[] row;
     public byte[] column;
-    public long timestamp;
     public int numVersions;
 
     public final Isset __isset = new Isset();
-    public static final class Isset {
+    public static final class Isset implements java.io.Serializable {
       public boolean tableName = false;
       public boolean row = false;
       public boolean column = false;
-      public boolean timestamp = false;
       public boolean numVersions = false;
     }
 
-    public getVerTs_args() {
+    public getVer_args() {
     }
 
-    public getVerTs_args(
+    public getVer_args(
       byte[] tableName,

[... 3232 lines stripped ...]